From f29bc1546dcacd89fc2d6421722cc1365421a659 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 28 Nov 2023 11:25:22 +0530 Subject: [PATCH 001/224] 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 002/224] 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 003/224] 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 004/224] 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 005/224] 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 006/224] 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 007/224] 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 008/224] 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 009/224] 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 010/224] 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 011/224] 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 012/224] 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 013/224] 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 014/224] 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 015/224] 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 016/224] 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 017/224] 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 018/224] 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 019/224] 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 020/224] 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 17b06e22d6a59a6dad8a952e4c2007ddd9081825 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 13 Dec 2023 10:59:39 +0530 Subject: [PATCH 021/224] Add logger support to configuration --- lib/kafkajs/_common.js | 85 ++++++++++++++++++++++++++++++++++++++++ lib/kafkajs/_consumer.js | 38 ++++++++++++------ lib/kafkajs/_producer.js | 39 +++++++++++++----- 3 files changed, 141 insertions(+), 21 deletions(-) diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 7bdb59f4..97925c30 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -9,6 +9,89 @@ const logLevel = Object.freeze({ DEBUG: 4, }); +const severityToLogLevel = Object.freeze({ + 0: logLevel.NOTHING, + 1: logLevel.ERROR, + 2: logLevel.ERROR, + 3: logLevel.ERROR, + 4: logLevel.WARN, + 5: logLevel.WARN, + 6: logLevel.INFO, + 7: logLevel.DEBUG, +}); + +/** + * Default logger implementation. + * @type import("../../types/kafkajs").Logger + */ +class DefaultLogger { + constructor() { + this.logLevel = logLevel.INFO; + } + + setLogLevel(logLevel) { + this.logLevel = logLevel; + } + + info(message, extra) { + if (this.logLevel >= logLevel.INFO) + console.info({ message, ...extra }); + } + + error(message, extra) { + if (this.logLevel >= logLevel.ERROR) + console.error({ message, ...extra }); + } + + warn(message, extra) { + if (this.logLevel >= logLevel.WARN) + console.warn({ message, ...extra }); + } + + debug(message, extra) { + if (this.logLevel >= logLevel.DEBUG) + console.log({ message, ...extra }); + } + + namespace() { + return this; + } +} + +/** + * Trampoline for user defined logger, if any. + * @param {{severity: number, fac: string, message: string}} msg + * + */ +function loggerTrampoline(msg, logger) { + if (!logger) { + return; + } + + const level = severityToLogLevel[msg.severity]; + switch (level) { + case logLevel.NOTHING: + break; + case logLevel.ERROR: + logger.error(msg.message, { fac: msg.fac, timestamp: Date.now() }); + break; + case logLevel.WARN: + logger.warn(msg.message, { fac: msg.fac, timestamp: Date.now() }); + break; + case logLevel.INFO: + logger.info(msg.message, { fac: msg.fac, timestamp: Date.now() }); + break; + case logLevel.DEBUG: + logger.debug(msg.message, { fac: msg.fac, timestamp: Date.now() }); + break; + default: + throw new error.KafkaJSError("Invalid logLevel", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + +} + /** * Converts the common configuration from KafkaJS to a format that can be used by node-rdkafka. * @param {object} config @@ -249,4 +332,6 @@ module.exports = { convertToRdKafkaHeaders, notImplemented, logLevel, + loggerTrampoline, + DefaultLogger, }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 769ef3c3..a40d443b 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -5,7 +5,10 @@ const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, createKafkaJsErrorFromLibRdKafkaError, - notImplemented + notImplemented, + loggerTrampoline, + DefaultLogger, + logLevel, } = require('./_common'); const { Buffer } = require('buffer'); @@ -79,6 +82,12 @@ class Consumer { */ #storedSubscriptions = []; + /** + * A logger for the consumer. + * @type {import("../../types/kafkajs").Logger} + */ + #logger = new DefaultLogger(); + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -109,16 +118,16 @@ class Consumer { case LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS: call = (this.#kJSConfig.rebalanceListener.onPartitionsAssigned ? this.#kJSConfig.rebalanceListener.onPartitionsAssigned(assignment) : - Promise.resolve()).catch(console.error); + Promise.resolve()).catch(this.#logger.error); break; case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: call = (this.#kJSConfig.rebalanceListener.onPartitionsRevoked ? this.#kJSConfig.rebalanceListener.onPartitionsRevoked(assignment) : - Promise.resolve()).catch(console.error); + Promise.resolve()).catch(this.#logger.error); break; default: call = Promise.reject(`Unexpected rebalanceListener error code ${err.code}`).catch((e) => { - console.error(e); + this.#logger.error(e); }); break; } @@ -166,6 +175,12 @@ class Consumer { /* This sets the common configuration options for the client. */ const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + /* Set the logger */ + if (Object.hasOwn(this.#kJSConfig, 'logger')) { + this.#logger = this.#kJSConfig.logger; + } + this.#logger.setLogLevel(this.#kJSConfig.logLevel || logLevel.INFO); + /* Consumer specific configuration */ if (Object.hasOwn(this.#kJSConfig, 'groupId')) { @@ -264,11 +279,7 @@ class Consumer { if (this.#state === ConsumerState.CONNECTING) { this.#connectPromiseFunc['reject'](err); } else { - /* 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); + this.#logger.error(err); } } @@ -377,7 +388,7 @@ class Consumer { 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); + this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; @@ -636,7 +647,7 @@ class Consumer { /* 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); + this.#internalClient.seek(topicPartitionOffset, 0, err => err ? this.#logger.error(err) : null); offsetsToCommit.push({ topic: topicPartition.topic, partition: topicPartition.partition, @@ -800,8 +811,11 @@ class Consumer { notImplemented(); } + /** + * @returns {import("../../types/kafkajs").Logger} the logger associated to this consumer. + */ logger() { - notImplemented(); + return this.#logger; } get events() { diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 220dbb47..48d9eb86 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -1,5 +1,11 @@ const RdKafka = require('../rdkafka'); -const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders } = require('./_common'); +const { kafkaJSToRdKafkaConfig, + topicPartitionOffsetToRdKafka, + createKafkaJsErrorFromLibRdKafkaError, + convertToRdKafkaHeaders, + DefaultLogger, + loggerTrampoline, + logLevel, } = require('./_common'); const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); @@ -51,6 +57,12 @@ class Producer { */ #ongoingTransaction = false; + /** + * A logger for the producer. + * @type {import("../../types/kafkajs").Logger} + */ + #logger = new DefaultLogger(); + /** * @constructor * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig @@ -69,6 +81,12 @@ class Producer { /* This sets the common configuration options for the client. */ const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + /* Set the logger */ + if (Object.hasOwn(this.#kJSConfig, 'logger')) { + this.#logger = this.#kJSConfig.logger; + } + this.#logger.setLogLevel(this.#kJSConfig.logLevel || logLevel.INFO); + /* Create producer specific configuration. */ globalConfig.dr_cb = 'true'; @@ -169,7 +187,7 @@ class Producer { throw new error.KafkaJSError(`Ready callback called in invalid state ${this.#state}`, { code: error.ErrorCodes.ERR__STATE }); } - const {globalConfig} = await this.#config(); + 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)); @@ -201,11 +219,7 @@ class Producer { if (this.#state === ProducerState.CONNECTING) { this.#connectPromiseFunc["reject"](err); } else { - /* 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); + this.#logger.error(err); } } @@ -224,7 +238,7 @@ class Producer { 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); + this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; @@ -452,7 +466,7 @@ class Producer { const opaque = { resolve, reject }; try { this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp, opaque, msg.headers); - } catch(err) { + } catch (err) { reject(err); } })); @@ -528,6 +542,13 @@ class Producer { const records = await Promise.all(sentPromises); return records.flat(); } + + /** + * @returns {import("../../types/kafkajs").Logger} the logger associated to this producer. + */ + logger() { + return this.#logger; + } } module.exports = { Producer } From 99ca4ec402b8a798240973d5eb7bcf1430f0c613 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 13 Dec 2023 17:57:29 +0530 Subject: [PATCH 022/224] Bump version --- package.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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", From 9428b6708e144b2b1c32141484a96a9f89701fb3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 14 Dec 2023 08:34:55 +0530 Subject: [PATCH 023/224] Add quality-of-life improvements to error messages and improve config defaults --- MIGRATION.md | 95 ++++------ lib/kafkajs/_common.js | 198 ++++++++++++++++---- lib/kafkajs/_consumer.js | 106 +++++++++-- lib/kafkajs/_kafka.js | 9 +- lib/kafkajs/_producer.js | 91 ++++++--- test/promisified/consumer/subscribe.spec.js | 4 +- 6 files changed, 368 insertions(+), 135 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index be3cfbb8..955778c8 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -37,19 +37,18 @@ * `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`. + enforced. The default value of the sum is 11000, same as for KafkaJS. + 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. + - `maxRetryTime`: maximum time to backoff a retry, in milliseconds. Corresponds to librdkafka's `retry.backoff.max.ms`. + - `initialRetryTime`: minimum time to backoff a retry, in milliseconds. Corresponds to librdkafka's `retry.backoff.ms`. + - `retries`: maximum number of retries, *only* applicable to Produce messages. + - **`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 automatically 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 @@ -119,9 +118,9 @@ 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. + * **`createPartitioner`**: this is not supported yet. The default behaviour is identical to the DefaultPartitioner, and compatible with Java client's default + partitioner. + This corresponds to the librdkafka property `partitioner` and the value `murmur2_random`. * **`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`). @@ -131,7 +130,7 @@ 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. + * **`maxInFlightRequests`**: Maximum number of in-flight requests *per broker connection*. If not set, it is practically unbounded (same as KafkaJS). * `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 @@ -140,7 +139,8 @@ #### Semantic and Per-Method Changes * Changes to `send`: - * `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 top-level configuration. + Additionally, there are several more compression types available by default besides GZIP. Before: ```javascript const kafka = new Kafka({/* ... */}); @@ -160,13 +160,9 @@ ```javascript const kafka = new Kafka({/* ... */}); const producer = kafka.producer({ - rdKafka: { - topicConfig: { - "acks": "1", - "compression.codec": "gzip", - "message.timeout.ms": "30000", - }, - } + acks: 1, + compression: CompressionTypes.GZIP|CompressionTypes.SNAPPY|CompressionTypes.LZ4|CompressionTypes.ZSTD|CompressionTypes.NONE, + timeout: 30000, }); await producer.connect(); @@ -191,27 +187,28 @@ 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. + * `partitionAssigners`: Support for range and roundRobin assignors is provided. Custom assignors are not supported. + The default value of this remains `[PartitionAssigners.roundRobin]`. + Support for cooperative-sticky assignor will be added soon. + 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. * **`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. + * `allowAutoTopicCreation`: determines if a topic should be created if it doesn't exist while producing. * **`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, + There is a 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. + * `maxBytes`: Maximum number of bytes the broker responds with. + * `maxWaitTimeInMs`: Maximum time in milliseconds the broker waits for the `minBytes` to be fulfilled. * **`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. + * **`maxInFlightRequests`**: Maximum number of in-flight requests *per broker connection*. If not set, it is practically unbounded (same as KafkaJS). * `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 @@ -225,10 +222,9 @@ * 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. + * While passing a list of topics to `subscribe`, the `fromBeginning` is not set on a per-subscribe basis. Rather, it must be configured in the top-level configuration. Before: ```javascript - const kafka = new Kafka({ /* ... */ }); const consumer = kafka.consumer({ groupId: 'test-group', }); @@ -237,23 +233,17 @@ ``` After: ```javascript - const kafka = new Kafka({ /* ... */ }); const consumer = kafka.consumer({ groupId: 'test-group', - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - }, - } + fromBeginning: true, }); await consumer.connect(); await consumer.subscribe({ topics: ["topic"] }); ``` - * 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. + * For auto-committing using a consumer, the properties `autoCommit` and `autoCommitInterval` on `run` are not set on a per-subscribe basis. + Rather, they must be configured in the top-level configuration. + `autoCommitThreshold` is not supported. Before: ```javascript @@ -273,12 +263,8 @@ const kafka = new Kafka({ /* ... */ }); const consumer = kafka.consumer({ /* ... */, - rdKafka: { - globalConfig: { - "enable.auto.commit": "true", - "auto.commit.interval.ms": "5000", - } - }, + autoCommit: true, + autoCommitThreshold: 5000, }); await consumer.connect(); await consumer.subscribe({ topics: ["topic"] }); @@ -287,18 +273,17 @@ }); ``` + * The `partitionsConsumedConcurrently` property is not supported at the moment. + * The `eachBatch` method is not supported. * 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 committed. + * `commitOffsets` does not yet support sending metadata for topic partitions being committed. * `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. 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. + * `stop` is not yet supported, and the user must disconnect the consumer. ### Admin Client diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 97925c30..dc0d65d7 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -89,9 +89,93 @@ function loggerTrampoline(msg, logger) { code: error.ErrorCodes.ERR__INVALID_ARG, }); } +} +function createReplacementErrorMessage(cOrP, fnCall, property, propertyVal, replacementVal, isLK = false) { + if (isLK) { + replacementVal = `rdKafka: { ${replacementVal}, ... }` + } + return `'${property}' is not supported as a property to '${fnCall}', but must be passed to the ${cOrP} during creation.\n` + + `Before: \n` + + `\tconst ${cOrP} = kafka.${cOrP}({ ... });\n` + + `\tawait ${cOrP}.connect();\n` + + `\t${cOrP}.${fnCall}({ ${propertyVal}, ... });\n` + + `After: \n` + + `\tconst ${cOrP} = kafka.${cOrP}({ ${replacementVal}, ... });\n` + + `\tawait ${cOrP}.connect();\n` + + `\t${cOrP}.${fnCall}({ ... });\n` + + (isLK ? `For more details on what can be used inside the rdKafka block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n` : ''); } +const CompatibilityErrorMessages = Object.freeze({ + /* Common */ + brokerString: () => + "The 'brokers' property must be an array of strings.\n" + + "For example: ['kafka:9092', 'kafka2:9093']\n", + saslOauthbearerUnsupported: () => + "SASL mechanism OAUTHBEARER is not supported yet.", + saslUnsupportedMechanism: (mechanism) => + `SASL mechanism ${mechanism} is not supported.`, + saslUsernamePasswordString: (mechanism) => + `The 'sasl.username' and 'sasl.password' properties must be strings and must be present for the mechanism ${mechanism}.`, + sslObject: () => + "The 'ssl' property must be a boolean. Any additional configuration must be provided through the 'rdKafka' property.\n" + + "Before: \n" + + "\tconst kafka = new Kafka({ ssl: { rejectUnauthorized: false, ca: [...], key: ..., cert: ... }, }); \n" + + "After: \n" + + "\tconst kafka = new Kafka({ ssl: true, rdKafka: { enable.ssl.certificate.verification: false, ssl.ca.location: ..., ssl.certificate.pem: ... } });\n" + + `For more details on what can be used inside the rdKafka block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n`, + retryFactorMultiplier: () => + "The 'retry.factor' and 'retry.multiplier' are not supported. They are always set to the default of 0.2 and 2 respectively.", + retryRestartOnFailure: () => + "The restartOnFailure property is ignored. The client always retries on failure.", + socketFactory: () => + "The socketFactory property is not supported.", + logLevelName: (setLevel) => + "The log level must be one of: " + logLevel.keys().join(", ") + ", was " + setLevel, + reauthenticationThreshold: () => + "Reauthentication threshold cannot be set, and reauthentication is automated when 80% of connections.max.reauth.ms is reached.", + unsupportedKey: (key) => + `The '${key}' property is not supported.`, + + /* Producer */ + createPartitioner: () => + "The 'createPartitioner' property is not supported yet. The default partitioner is set to murmur2_random, compatible with the DefaultPartitioner and the Java partitioner.\n" + + "A number of alternative partioning strategies are available through the 'rdKafka' property, for example: \n" + + "\tconst kafka = new Kafka({ rdKafka: { 'partitioner': 'random|consistent_random|consistent|murmur2|murmur2_random|fnv1a|fnv1a_random' } });\n" + + `For more details on what can be used inside the rdKafka block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n`, + sendOptionsMandatoryMissing: () => + "The argument passed to send must be an object, and must contain the 'topic' and 'messages' properties: {topic: string, messages: Message[]}\n", + sendOptionsAcks: (fn) => + createReplacementErrorMessage('producer', fn, 'acks', 'acks: ', 'acks: ', false), + sendOptionsCompression: (fn) => + createReplacementErrorMessage('producer', fn, 'compression', 'compression: ', 'compression: CompressionTypes.GZIP|SNAPPY|LZ4|ZSTD', false), + sendOptionsTimeout: (fn) => + createReplacementErrorMessage('producer', fn, 'timeout', 'timeout: ', 'timeout: ', false), + sendBatchMandatoryMissing: () => + "The argument passed to sendbatch must be an object, and must contain the 'topicMessages' property: { topicMessages: {topic: string, messages: Message[]}[] } \n", + + /* Consumer */ + partitionAssignors: () => + 'partitionAssignors must be a list of strings from within `PartitionAssignors`, and the cooperativeSticky assignor is not yet supported.\n', + subscribeOptionsFromBeginning: () => + createReplacementErrorMessage('consumer', 'subscribe', 'fromBeginning', 'fromBeginning: ', 'fromBeginning: ', false), + subscribeOptionsMandatoryMissing: () => + "The argument passed to subscribe must be an object, and must contain the 'topics' or the 'topic' property: {topics: string[]} or {topic: string}\n", + subscribeOptionsRegexFlag: () => + "If subscribing to topic by RegExp, no flags are allowed. /abcd/ is okay, but /abcd/i is not.\n", + runOptionsAutoCommit: () => + createReplacementErrorMessage('consumer', 'run', 'autoCommit', 'autoCommit: ', 'autoCommit: ', false), + runOptionsAutoCommitInterval: () => + createReplacementErrorMessage('consumer', 'run', 'autoCommitInterval', 'autoCommitInterval: ', 'autoCommitInterval: ', false), + runOptionsAutoCommitThreshold: () => + "The property 'autoCommitThreshold' is not supported by run.\n", + runOptionsRunConcurrently: () => + "The property 'partitionsConsumedConcurrently' is not currently supported by run\n", + runOptionsEachBatch: () => + "The property 'eachBatch' is not currently supported by run\n", +}); + /** * Converts the common configuration from KafkaJS to a format that can be used by node-rdkafka. * @param {object} config @@ -105,7 +189,7 @@ async function kafkaJSToRdKafkaConfig(config) { const topicConfig = {}; if (!Array.isArray(config["brokers"])) { - throw new error.KafkaJSError("brokers must be an list of strings", { + throw new error.KafkaJSError(CompatibilityErrorMessages.brokerString(), { code: error.ErrorCodes.ERR__INVALID_ARG, }); } @@ -122,15 +206,20 @@ async function kafkaJSToRdKafkaConfig(config) { const mechanism = sasl.mechanism.toUpperCase(); if (mechanism === 'OAUTHBEARER') { - throw new error.KafkaJSError("OAUTHBEARER is not supported", { + throw new error.KafkaJSError(CompatibilityErrorMessages.saslOauthbearerUnsupported(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED, }); } /* The mechanism must be PLAIN or SCRAM. */ + if (mechanism !== 'PLAIN' && !mechanism.startsWith('SCRAM')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.saslUnsupportedMechanism(mechanism), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } if (typeof sasl.username !== "string" || typeof sasl.password !== "string") { - throw new error.KafkaJSError("username and password must be present and be strings", { + throw new error.KafkaJSError(CompatibilityErrorMessages.saslUsernamePasswordString(mechanism), { code: error.ErrorCodes.ERR__INVALID_ARG, }); } @@ -141,60 +230,67 @@ async function kafkaJSToRdKafkaConfig(config) { withSASL = true; } - if (Object.hasOwn(config, "ssl") && withSASL) { + if (Object.hasOwn(config, "ssl") && config.ssl && withSASL) { globalConfig["security.protocol"] = "sasl_ssl"; } else if (withSASL) { globalConfig["security.protocol"] = "sasl_plaintext"; + } else if (Object.hasOwn(config, "ssl") && config.ssl) { + globalConfig["security.protocol"] = "ssl"; + } + + /* TODO: add best-effort support for ssl besides just true/false */ + if (Object.hasOwn(config, "ssl") && typeof config.ssl !== "boolean") { + throw new error.KafkaJSError(CompatibilityErrorMessages.sslObject(), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); } if (Object.hasOwn(config, "requestTimeout")) { globalConfig["socket.timeout.ms"] = config.requestTimeout; + } else { + /* KafkaJS default */ + globalConfig["socket.timeout.ms"] = 30000; } - if (Object.hasOwn(config, "enforceRequestTimeout")) { + if (Object.hasOwn(config, "enforceRequestTimeout") && !config.enforceRequestTimeout) { globalConfig["socket.timeout.ms"] = 300000; } - const connectionTimeout = config.connectionTimeout ?? 0; - const authenticationTimeout = config.authenticationTimeout ?? 0; + const connectionTimeout = config.connectionTimeout ?? 1000; + const authenticationTimeout = config.authenticationTimeout ?? 10000; 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; - } + 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; + const retry = config.retry ?? {}; + const { maxRetryTime, initialRetryTime, factor, multiplier, retries, restartOnFailure } = retry; - if (maxRetryTime) { - globalConfig["retry.backoff.max.ms"] = maxRetryTime; - } - - if (initialRetryTime) { - globalConfig["retry.backoff.ms"] = initialRetryTime; - } - - if (retries) { - globalConfig["retries"] = retries; - } + globalConfig["retry.backoff.max.ms"] = maxRetryTime ?? 30000; + globalConfig["retry.backoff.ms"] = initialRetryTime ?? 300; + globalConfig["retries"] = retries ?? 5; - if (factor || multiplier) { - throw new error.KafkaJSError("retry.factor and retry.multiplier are not supported", { - code: error.ErrorCodes.ERR__INVALID_ARG, - }); - } + if ((typeof factor === 'number') || (typeof multiplier === 'number')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.retryFactorMultiplier(), { + 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", { + if (restartOnFailure) { + throw new error.KafkaJSError(CompatibilityErrorMessages.retryRestartOnFailure(), { code: error.ErrorCodes.ERR__INVALID_ARG, }); } if (Object.hasOwn(config, "socketFactory")) { - throw new error.KafkaJSError("socketFactory is not supported", { + throw new error.KafkaJSError(CompatibilityErrorMessages.socketFactory(), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + + if (Object.hasOwn(config, "reauthenticationThreshold")) { + throw new error.KafkaJSError(CompatibilityErrorMessages.reauthenticationThreshold(), { code: error.ErrorCodes.ERR__INVALID_ARG, }); } @@ -205,7 +301,6 @@ async function kafkaJSToRdKafkaConfig(config) { 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 */ @@ -223,7 +318,7 @@ async function kafkaJSToRdKafkaConfig(config) { globalConfig["debug"] = "all" /* this will set librdkafka log_level to 7 */; break; default: - throw new error.KafkaJSError("Invalid logLevel", { + throw new error.KafkaJSError(CompatibilityErrorMessages.logLevelName(setLevel), { code: error.ErrorCodes.ERR__INVALID_ARG, }); } @@ -238,9 +333,40 @@ async function kafkaJSToRdKafkaConfig(config) { } } + + if (!Object.hasOwn(globalConfig, 'log_level')) + globalConfig['log_level'] = Object.hasOwn(globalConfig, 'debug') ? 7 /* LOG_DEBUG */ : 6 /* LOG_INFO */; + return { globalConfig, topicConfig }; } +function checkAllowedKeys(allowedKeysSpecific, config) { + const allowedKeysCommon = [ + "brokers", + "clientId", + "sasl", + "ssl", + "requestTimeout", + "enforceRequestTimeout", + "connectionTimeout", + "authenticationTimeout", + "retry", + "socketFactory", + "reauthenticationThreshold", + "logLevel", + "rdKafka", + 'logger', + ]; + + for (const key of Object.keys(config)) { + if (!allowedKeysCommon.includes(key) && !allowedKeysSpecific.includes(key)) { + throw new error.KafkaJSError(CompatibilityErrorMessages.unsupportedKey(key), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + } +} + /** * Converts a topicPartitionOffset from KafkaJS to a format that can be used by node-rdkafka. * @param {import("../../types/kafkajs").TopicPartitionOffset} tpo @@ -334,4 +460,8 @@ module.exports = { logLevel, loggerTrampoline, DefaultLogger, + createReplacementErrorMessage, + CompatibilityErrorMessages, + severityToLogLevel, + checkAllowedKeys }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index a40d443b..b9cdcb9c 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -8,7 +8,9 @@ const { notImplemented, loggerTrampoline, DefaultLogger, - logLevel, + CompatibilityErrorMessages, + severityToLogLevel, + checkAllowedKeys } = require('./_common'); const { Buffer } = require('buffer'); @@ -175,11 +177,30 @@ class Consumer { /* This sets the common configuration options for the client. */ const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); - /* Set the logger */ - if (Object.hasOwn(this.#kJSConfig, 'logger')) { - this.#logger = this.#kJSConfig.logger; - } - this.#logger.setLogLevel(this.#kJSConfig.logLevel || logLevel.INFO); + const allowedKeysSpecific = [ + 'groupId', + 'partitionAssigners', + 'partitionAssignors', + 'sessionTimeout', + 'rebalanceTimeout', + 'heartbeatInterval', + 'metadataMaxAge', + 'allowAutoTopicCreation', + 'maxBytesPerPartition', + 'maxWaitTimeInMs', + 'minBytes', + 'maxBytes', + 'readUncommitted', + 'maxInFlightRequests', + 'rackId', + 'fromBeginning', + 'autoCommit', + 'autoCommitInterval', + 'autoCommitThreshold', + 'rebalanceListener', + ]; + + checkAllowedKeys(allowedKeysSpecific, this.#kJSConfig); /* Consumer specific configuration */ @@ -194,14 +215,20 @@ class Consumer { 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 }); + throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + if (assignor === PartitionAssigners.cooperativeSticky) + throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); }); globalConfig['partition.assignment.strategy'] = this.#kJSConfig.partitionAssignors.join(','); + } else { + globalConfig['partition.assignment.strategy'] = PartitionAssigners.roundRobin; } if (Object.hasOwn(this.#kJSConfig, 'sessionTimeout')) { globalConfig['session.timeout.ms'] = this.#kJSConfig.sessionTimeout; + } else { + globalConfig['session.timeout.ms'] = 30000; } if (Object.hasOwn(this.#kJSConfig, 'rebalanceTimeout')) { @@ -219,14 +246,20 @@ class Consumer { if (Object.hasOwn(this.#kJSConfig, 'allowAutoTopicCreation')) { globalConfig['allow.auto.create.topics'] = this.#kJSConfig.allowAutoTopicCreation; + } else { + globalConfig['allow.auto.create.topics'] = true; } if (Object.hasOwn(this.#kJSConfig, 'maxBytesPerPartition')) { globalConfig['max.partition.fetch.bytes'] = this.#kJSConfig.maxBytesPerPartition; + } else { + globalConfig['max.partition.fetch.bytes'] = 1048576; } if (Object.hasOwn(this.#kJSConfig, 'maxWaitTimeInMs')) { globalConfig['fetch.wait.max.ms'] = this.#kJSConfig.maxWaitTimeInMs; + } else { + globalConfig['fetch.wait.max.ms'] = 5000; } if (Object.hasOwn(this.#kJSConfig, 'minBytes')) { @@ -235,6 +268,8 @@ class Consumer { if (Object.hasOwn(this.#kJSConfig, 'maxBytes')) { globalConfig['fetch.message.max.bytes'] = this.#kJSConfig.maxBytes; + } else { + globalConfig['fetch.message.max.bytes'] = 10485760; } if (Object.hasOwn(this.#kJSConfig, 'readUncommitted')) { @@ -249,6 +284,22 @@ class Consumer { globalConfig['client.rack'] = this.#kJSConfig.rackId; } + if (Object.hasOwn(this.#kJSConfig, 'fromBeginning')) { + topicConfig['auto.offset.reset'] = this.#kJSConfig.fromBeginning ? 'earliest' : 'latest'; + } + + if (Object.hasOwn(this.#kJSConfig, 'autoCommit')) { + globalConfig['enable.auto.commit'] = this.#kJSConfig.autoCommit; + } + + if (Object.hasOwn(this.#kJSConfig, 'autoCommitInterval')) { + globalConfig['auto.commit.interval.ms'] = this.#kJSConfig.autoCommitInterval; + } + + if (Object.hasOwn(this.#kJSConfig, 'autoCommitThreshold')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + globalConfig['offset_commit_cb'] = true; if (!Object.hasOwn(this.#kJSConfig, 'rebalanceListener')) { @@ -257,6 +308,12 @@ class Consumer { } globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); + /* Set the logger */ + if (Object.hasOwn(this.#kJSConfig, 'logger')) { + this.#logger = this.#kJSConfig.logger; + } + this.#logger.setLogLevel(severityToLogLevel[globalConfig.log_level]); + return { globalConfig, topicConfig }; } @@ -410,12 +467,12 @@ class Consumer { if (typeof subscription.fromBeginning === 'boolean') { throw new error.KafkaJSError( - 'fromBeginning is not supported by subscribe(), but must be passed as an rdKafka property to the consumer.', - { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + CompatibilityErrorMessages.subscribeOptionsFromBeginning(), + { code: error.ErrorCodes.ERR__INVALID_ARG }); } 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 }); + throw new error.KafkaJSError(CompatibilityErrorMessages.subscribeOptionsMandatoryMissing(), { code: error.ErrorCodes.ERR__INVALID_ARG }); } let topics = []; @@ -424,7 +481,7 @@ class Consumer { } 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 }); + throw new error.KafkaJSError(CompatibilityErrorMessages.subscribeOptionsMandatoryMissing(), { code: error.ErrorCodes.ERR__INVALID_ARG }); } topics = topics.map(topic => { @@ -432,6 +489,9 @@ class Consumer { 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 ^. + if (topic.flags) { + throw new error.KafkaJSError(CompatibilityErrorMessages.subscribeOptionsRegexFlag(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } const regexSource = topic.source; if (regexSource.charAt(0) !== '^') return '^' + regexSource; @@ -442,7 +502,7 @@ class Consumer { } }); - this.#storedSubscriptions = subscription.replace ? topics : this.#storedSubscriptions.concat(topics); + this.#storedSubscriptions = subscription.replace ? topics : this.#storedSubscriptions.concat(topics); this.#internalClient.subscribe(this.#storedSubscriptions); } @@ -459,10 +519,24 @@ class Consumer { 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 }); + if (Object.hasOwn(config, 'autoCommit')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommit(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (Object.hasOwn(config, 'autoCommitInterval')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitInterval(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (Object.hasOwn(config, 'autoCommitThreshold')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + + if (Object.hasOwn(config, 'eachBatch')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsEachBatch(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + + if (Object.hasOwn(config, 'partitionsConsumedConcurrently')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsPartitionsConsumedConcurrently(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } /* We deliberately don't await this. */ diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index cc0f2ab2..08a7e0a6 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,4 +1,4 @@ -const { Producer } = require('./_producer'); +const { Producer, CompressionTypes } = require('./_producer'); const { Consumer, PartitionAssigners } = require('./_consumer'); const { Admin } = require('./_admin'); const error = require('./_error'); @@ -27,7 +27,8 @@ class Kafka { mergedConfig.rdKafka = mergedConfig.rdKafka ?? {}; if (typeof config.rdKafka === 'object') { - Object.assign(mergedConfig.rdKafka, config.rdKafka); + Object.assign(mergedConfig.rdKafka.globalConfig ?? {}, config.rdKafka.globalConfig ?? {}); + Object.assign(mergedConfig.rdKafka.topicConfig ?? {}, config.rdKafka.topicConfig ?? {}); delete config.rdKafka; } @@ -47,7 +48,7 @@ class Kafka { /** * Creates a new consumer. - * @param {import("../../types/kafkajs").Consumer} config + * @param {import("../../types/kafkajs").ConsumerConfig} config * @returns {Consumer} */ consumer(config) { @@ -59,4 +60,4 @@ class Kafka { } } -module.exports = { Kafka, ...error, logLevel, PartitionAssigners, PartitionAssignors: PartitionAssigners }; +module.exports = { Kafka, ...error, logLevel, PartitionAssigners, PartitionAssignors: PartitionAssigners, CompressionTypes }; diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 48d9eb86..07d1e64c 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -5,7 +5,9 @@ const { kafkaJSToRdKafkaConfig, convertToRdKafkaHeaders, DefaultLogger, loggerTrampoline, - logLevel, } = require('./_common'); + severityToLogLevel, + checkAllowedKeys, + CompatibilityErrorMessages } = require('./_common'); const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); @@ -20,6 +22,14 @@ const ProducerState = Object.freeze({ DISCONNECTED: 6, }); +const CompressionTypes = Object.freeze({ + None: 'none', + GZIP: 'gzip', + SNAPPY: 'snappy', + LZ4: 'lz4', + ZSTD: 'zstd', +}) + class Producer { /** * kJSConfig is the kafkaJS config object. @@ -81,15 +91,30 @@ class Producer { /* This sets the common configuration options for the client. */ const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); - /* Set the logger */ - if (Object.hasOwn(this.#kJSConfig, 'logger')) { - this.#logger = this.#kJSConfig.logger; - } - this.#logger.setLogLevel(this.#kJSConfig.logLevel || logLevel.INFO); + const allowedKeysSpecific = [ + 'createPartitioner', + 'metadataMaxAge', + 'allowAutoTopicCreation', + 'transactionTimeout', + 'idempotent', + 'maxInFlightRequests', + 'transactionalId', + 'compression', + 'acks', + 'timeout', + 'logger', + ]; + + checkAllowedKeys(allowedKeysSpecific, this.#kJSConfig); /* Create producer specific configuration. */ globalConfig.dr_cb = 'true'; + if (Object.hasOwn(this.#kJSConfig, 'createPartitioner')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.createPartitioner(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + globalConfig['partitioner'] = 'murmur2_random'; + if (Object.hasOwn(this.#kJSConfig, 'metadataMaxAge')) { globalConfig['topic.metadata.refresh.interval.ms'] = this.#kJSConfig.metadataMaxAge; } @@ -114,10 +139,24 @@ class Producer { 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 }); + if (Object.hasOwn(this.#kJSConfig, 'compression')) { + globalConfig['compression.codec'] = this.#kJSConfig.compression; } + if (Object.hasOwn(this.#kJSConfig, 'acks')) { + globalConfig['acks'] = this.#kJSConfig.acks; + } + + if (Object.hasOwn(this.#kJSConfig, 'timeout')) { + globalConfig['message.timeout.ms'] = this.#kJSConfig.timeout; + } + + /* Set the logger */ + if (Object.hasOwn(this.#kJSConfig, 'logger')) { + this.#logger = this.#kJSConfig.logger; + } + this.#logger.setLogLevel(severityToLogLevel[globalConfig.log_level]); + return { globalConfig, topicConfig }; } @@ -428,18 +467,17 @@ class Producer { } if (sendOptions === null || !(sendOptions instanceof Object)) { - throw new error.KafkaJSError("sendOptions must be set correctly", { code: error.ErrorCodes.ERR__INVALID_ARG }); + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsMandatoryMissing(), { code: error.ErrorCodes.ERR__INVALID_ARG }); } - // Ignore all properties except topic and messages. - // TODO: log a warning instead of ignoring. - if (!Object.hasOwn(sendOptions, "topic") || !Object.hasOwn(sendOptions, "messages") || !Array.isArray(sendOptions["messages"])) { - // TODO: add further validations. - 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')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsAcks('send'), { 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 }); + if (Object.hasOwn(sendOptions, 'timeout')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsTimeout('send'), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + if (Object.hasOwn(sendOptions, 'compression')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsCompression('send'), { code: error.ErrorCodes.ERR__INVALID_ARG }); } const msgPromises = []; @@ -512,20 +550,25 @@ class Producer { */ async sendBatch(sendOptions) { if (this.#state !== ProducerState.CONNECTED) { - throw new error.KafkaJSError("Cannot send without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); + throw new error.KafkaJSError("Cannot sendBatch 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 }); + throw new error.KafkaJSError(CompatibilityErrorMessages.sendBatchMandatoryMissing(), { 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 (Object.hasOwn(sendOptions, 'acks')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsAcks('sendBatch'), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + if (Object.hasOwn(sendOptions, 'timeout')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsTimeout('sendBatch'), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + if (Object.hasOwn(sendOptions, 'compression')) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsCompression('sendBatch'), { 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 }); + throw new error.KafkaJSError(CompatibilityErrorMessages.sendBatchMandatoryMissing(), { code: error.ErrorCodes.ERR__INVALID_ARG }); } if (!sendOptions.topicMessages || sendOptions.topicMessages.length === 0) { @@ -551,4 +594,4 @@ class Producer { } } -module.exports = { Producer } +module.exports = { Producer, CompressionTypes }; diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index 30493c82..67dc2d80 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -52,7 +52,7 @@ describe('Consumer', () => { const messagesConsumed = []; await consumer.connect(); await consumer.subscribe({ - topics: [topics[0], topics[1], new RegExp(`^pattern-${testScope}-regex-.*`, 'i')], + topics: [topics[0], topics[1], new RegExp(`^pattern-${testScope}-regex-.*`)], }); consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); @@ -130,7 +130,7 @@ describe('Consumer', () => { const messagesConsumed = []; await consumer.connect(); await consumer.subscribe({ - topic: new RegExp(`pattern-${testScope}-(se|br)-.*`, 'i'), + topic: new RegExp(`pattern-${testScope}-(se|br)-.*`), }); consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); From 96add1bf372f4eb058ffae9cf26564b7ec5c426b Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 13 Dec 2023 17:57:29 +0530 Subject: [PATCH 024/224] 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 025/224] 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 026/224] 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 027/224] 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 028/224] 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 029/224] 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": [ From 00a4d03035198e5bda8334a21c794397f8bcb0be Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 14 Dec 2023 11:12:31 +0100 Subject: [PATCH 030/224] Use single condition in gyp and avoid parallel targets execution --- binding.gyp | 40 ++++++++++++++++++++------------------- deps/librdkafka.gyp | 46 +++++++++++++++++++++++++++++++-------------- 2 files changed, 53 insertions(+), 33 deletions(-) diff --git a/binding.gyp b/binding.gyp index 9ec16668..258ff021 100644 --- a/binding.gyp +++ b/binding.gyp @@ -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 3dc6ff48..7f24cc12 100644 --- a/deps/librdkafka.gyp +++ b/deps/librdkafka.gyp @@ -1,4 +1,7 @@ { + "variables": { + "CKJS_LINKING%": " Date: Thu, 14 Dec 2023 20:11:35 +0530 Subject: [PATCH 031/224] Improve MIGRATION.md, add licensing, start changing README.md --- LICENCE.node-rdkafka-txt | 0 LICENSE.kafkajs | 30 +++ LICENSE.node-rdkafka | 25 +++ MIGRATION.md | 451 ++++++++++++++++++++++----------------- README.md | 89 ++++---- lib/kafkajs/_producer.js | 2 +- 6 files changed, 350 insertions(+), 247 deletions(-) create mode 100644 LICENCE.node-rdkafka-txt create mode 100644 LICENSE.kafkajs create mode 100644 LICENSE.node-rdkafka diff --git a/LICENCE.node-rdkafka-txt b/LICENCE.node-rdkafka-txt new file mode 100644 index 00000000..e69de29b diff --git a/LICENSE.kafkajs b/LICENSE.kafkajs new file mode 100644 index 00000000..3c57956e --- /dev/null +++ b/LICENSE.kafkajs @@ -0,0 +1,30 @@ +The promisified API (lib/kafkajs) is inspired by kafkajs (github.com/tulios/kafkajs). +The promisified tests (test/promisified) are also adapted from there. +The license notice is reproduced below. + +---- + +The MIT License + +Copyright (c) 2018 Túlio Ornelas (ornelas.tulio@gmail.com) + +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 the Software without restriction, including +without limitation the rights to use, copy, modify, +merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom +the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice +shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR +ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, +TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE +SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/LICENSE.node-rdkafka b/LICENSE.node-rdkafka new file mode 100644 index 00000000..e7a18a6f --- /dev/null +++ b/LICENSE.node-rdkafka @@ -0,0 +1,25 @@ +This project is based on node-rdkafka (github.com/Blizzard/node-rdkafka). +The license notice is reproduced below. + +-------- + +The MIT License (MIT) +Copyright (c) 2016 Blizzard Entertainment + +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 +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS +IN THE SOFTWARE. \ No newline at end of file diff --git a/MIGRATION.md b/MIGRATION.md index 955778c8..fe3253c0 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -2,144 +2,144 @@ ## KafkaJS -### Common - -#### 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. - - * 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: +1. Change the import statement, from ```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 */ } - }, - /* ... */ - }); + const { Kafka } = require('kafkajs'); + ``` + to + ```javascript + const { Kafka } = require('confluent-kafka-js').KafkaJS; ``` - * **`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 11000, same as for KafkaJS. - 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`. - - `initialRetryTime`: minimum time to backoff a retry, in milliseconds. Corresponds to librdkafka's `retry.backoff.ms`. - - `retries`: maximum number of retries, *only* applicable to Produce messages. - - **`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 automatically 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`. - 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. +2. Try running your program. In case a migration is needed, an informative error will be thrown. + +3. The most common expected changes to the code are: + - For the **producer**: `acks`, `compression` and `timeout` are not set on a per-send() basis. + Rather, they must be configured in the top-level configuration while creating the producer. + - For the **consumer**: + - `fromBeginning` is not set on a per-subscribe() basis. + Rather, it must be configured in the top-level configuration while creating the consumer. + - `autoCommit` and `autoCommitInterval` are not set on a per-run() basis. + Rather, they must be configured in the top-level configuration while creating the consumer. + - `autoCommitThreshold` and `eachBatch` are not supported. + - For errors: Check the `error.code` rather than the error `name` or `type`. + +4. A more exhaustive list of semantic and configuration differences is [presented below](#common). + +5. An example migration: + +```diff +-const { Kafka } = require('kafkajs'); ++const { Kafka } = require('confluent-kafka-js').KafkaJS; + +const kafka = new Kafka({ + clientId: 'my-app', + brokers: ['kafka1:9092', 'kafka2:9092'] +}) + +const producerRun = async () => { +- const producer = kafka.producer(); ++ const producer = kafka.producer({ acks: 1 }); + await producer.connect(); + await producer.send({ + topic: 'test-topic', +- acks: 1, + messages: [ + { value: 'Hello confluent-kafka-js user!' }, + ], + }); +}; + + +const consumerRun = async () => { + // Consuming +- const consumer = kafka.consumer({ groupId: 'test-group' }); ++ const consumer = kafka.consumer({ groupId: 'test-group', fromBeginning: true }); + await consumer.connect(); +- await consumer.subscribe({ topic: 'test-topic', fromBeginning: true }); ++ await consumer.subscribe({ topic: 'test-topic' }); + + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + partition, + offset: message.offset, + value: message.value.toString(), + }) + }, + }); +}; + +producerRun().then(consumerRun).catch(console.error); +``` - For compatibility, as many error types as possible have been retained, but it is - better to switch to checking the `error.code`. +### Common - **Action**: Convert any checks based on `instanceof` and `error.name` or to error - checks based on `error.code` or `error.type`. - - **Example:** - ```javascript - 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 */ - } - ``` +#### Configuration changes + ```javascript + const kafka = new Kafka({/* common configuration changes */}); + ``` + Each allowed config property is discussed in the table below. + If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. + + | Property | Default Value | Comment | + |-------------------------------|--------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | + | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified using the rdKafka block.** | + | **sasl** | - | An optional object of the form `{ mechanism: 'plain' or 'scram-sha-256' or 'scram-sha-512', username: string, password: string }`. **Additional authentication types are not yet supported.** | + | clientId | "rdkafka" | An optional string used to identify the client. | + | **connectionTimeout** | 1000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | + | **authenticationTimeout** | 10000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | + | **reauthenticationThreshold** | **80% of connections.max.reauth.ms** | **No longer checked, the default is always used.** | + | requestTimeout | 30000 | number of milliseconds for a network request to timeout. | + | **enforceRequestTimeout** | true | When set to false, `requestTimeout` is set to 5 minutes. **This cannot be completely disabled.** | + | retry | object | Properties individually discussed below. | + | retry.maxRetryTime | 30000 | maximum time to backoff a retry, in milliseconds. | + | retry.initialRetryTime | 300 | minimum time to backoff a retry, in milliseconds | + | **retry.retries** | 5 | Total cap on the number of retries. **Applicable only to Produce requests.** | + | **retry.factor** | 0.2 | Randomization factor (jitter) for backoff. **Cannot be changed**. | + | **retry.multiplier** | 2 | Multiplier for exponential factor of backoff. **Cannot be changed.** | + | **retry.restartOnFailure** | true | Consumer only. **Cannot be changed**. Consumer will always make an attempt to restart. | + | logLevel | `logLevel.INFO` | Decides the severity level of the logger created by the underlying library. A logger created with the `INFO` level will not be able to log `DEBUG` messages later. | + | **socketFactory** | null | **No longer supported.** | + | **rdKafka** | {} | This is a **new** property. It can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | - 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 -#### Configuration changes +#### 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. The default behaviour is identical to the DefaultPartitioner, and compatible with Java client's default - partitioner. - This corresponds to the librdkafka property `partitioner` and the value `murmur2_random`. - * **`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, it is practically unbounded (same as KafkaJS). - * `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). + Each allowed config property is discussed in the table below. + If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. + + | Property | Default Value | Comment | + |-------------------------|------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | **createPartitioner** | DefaultPartioner (murmur2_random) - Java client compatible | Custom partitioner support is not yet provided. The default partitioner's behaviour is retained, and a number of partitioners are provided via the `rdKafka.partitioner` property. | + | **retry** | object | Identical to `retry` in the common configuration. This takes precedence over the common config retry. | + | metadataMaxAge | 5 minutes | Time in milliseconds after which to refresh metadata for known topics | + | allowAutoTopicCreation | true | Determines if a topic should be created if it doesn't exist while producing. | + | transactionTimeout | 60000 | 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. Only applicable when `transactionalId` is set to true. | + | idempotent | false | If set to true, ensures that messages are delivered exactly once and in order. If true, certain constraints must be respected for other properties, `maxInFlightRequests <= 5`, `retry.retries >= 0` | + | **maxInFlightRequests** | null | Maximum number of in-flight requests **per broker connection**. If not set, it is practically unbounded (same as KafkaJS). | + | transactionalId | null | If set, turns this into a transactional producer with this identifier. This also automatically sets `idempotent` to true. | + | **acks** | -1 | The number of required acks before a Produce succeeds. **This is set on a per-producer level, not on a per `send` level**. -1 denotes it will wait for all brokers in the in-sync replica set. | + | **compression** | CompressionTypes.NONE | Compression codec for Produce messages. **This is set on a per-producer level, not on a per `send` level**. It must be a key of the object CompressionType, namely GZIP, SNAPPY, LZ4, ZSTD or NONE. | + | **timeout** | 30000 | The ack timeout of the producer request in milliseconds. This value is only enforced by the broker. **This is set on a per-producer level, not on a per `send` level**. | + | **rdKafka** | {} | This is a **new** property. It can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION. md). If set on the common config, the values are merged. | + + #### Semantic and Per-Method Changes -* Changes to `send`: - * `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the top-level configuration. +* `send`: and `sendBatch`:` + - While sending multiple messages, even if one of the messages fails, the method throws an error. + - While `sendBatch` is available, it acts as a wrapper around send, and the actual batching is handled by librdkafka. + - `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the top-level configuration. See [configuration changes](#producer-configuration-changes). Additionally, there are several more compression types available by default besides GZIP. Before: ```javascript @@ -172,78 +172,74 @@ }); ``` - * 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()`. +* A transactional producer (with a `transactionId`) set, **cannot** send messages without initiating a transaction using `producer.transaction()`. ### Consumer -#### Configuration changes +#### 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`: Support for range and roundRobin assignors is provided. Custom assignors are not supported. - The default value of this remains `[PartitionAssigners.roundRobin]`. - Support for cooperative-sticky assignor will be added soon. - 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. - * **`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. - * **`maxBytesPerPartition`**: determines how many bytes can be fetched in one request from a single partition. The default value remains 1048576. - There is a 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. - * `maxWaitTimeInMs`: Maximum time in milliseconds the broker waits for the `minBytes` to be fulfilled. - * **`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, it is practically unbounded (same as KafkaJS). - * `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). + Each allowed config property is discussed in the table below. + If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. + + | Property | Default Value | Comment | + |--------------------------|-----------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | groupId | null | A mandatory string denoting consumer group name that this consumer is a part of. | + | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range and roundRobin assignors is provided. Custom assignors are not supported. Support for cooperative-sticky assignor is yet to be added | + | **partitionAssignors** | `[PartitionAssignors.roundRobin]` | Alias for `partitionAssigners` | + | **rebalanceTimeout** | **300000** | The maximum allowed time for each member to join the group once a rebalance has begun. Note, that setting this value *also* changes the max poll interval. Message processing in `eachMessage` must not take more than this time. | + | heartbeatInterval | 3000 | The expected time in milliseconds between heartbeats to the consumer coordinator. | + | metadataMaxAge | 5 minutes | Time in milliseconds after which to refresh metadata for known topics | + | allowAutoTopicCreation | true | Determines if a topic should be created if it doesn't exist while consuming. | + | **maxBytesPerPartition** | 1048576 (1MB) | determines how many bytes can be fetched in one request from a single partition. There is a change in semantics, this size grows dynamically if a single message larger than this is encountered, and the client does not get stuck. | + | minBytes | 1 | Minimum number of bytes the broker responds with (or wait until `maxWaitTimeInMs`) | + | maxBytes | 10485760 (10MB) | Maximum number of bytes the broker responds with. | + | **retry** | object | Identical to `retry` in the common configuration. This takes precedence over the common config retry. | + | readUncommitted | false | If true, consumer will read transactional messages which have not been committed. | + | **maxInFlightRequests** | null | Maximum number of in-flight requests **per broker connection**. If not set, it is practically unbounded (same as KafkaJS). | + | rackId | null | Can be set to an arbitrary string which will be used for fetch-from-follower if set up on the cluster. | + | **fromBeginning** | false | If there is initial offset in offset store or the desired offset is out of range, and this is true, we consume the earliest possible offset. **This is set on a per-consumer level, not on a per `subscribe` level**. | + | **autoCommit** | true | Whether to periodically auto-commit offsets to the broker while consuming. **This is set on a per-consumer level, not on a per `run` level**. | + | **autoCommitInterval** | 5000 | Offsets are committed periodically at this interval, if autoCommit is true. **This is set on a per-consumer level, not on a per `run` level. The default value is changed to 5 seconds.**. | + | **rdKafka** | {} | This is a **new** property. It can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). If set on the common config, the values are merged. | + #### Semantic and Per-Method Changes - * 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` is not set on a per-subscribe basis. Rather, it must be configured in the top-level configuration. - Before: - ```javascript - const consumer = kafka.consumer({ - groupId: 'test-group', - }); - await consumer.connect(); - await consumer.subscribe({ topics: ["topic"], fromBeginning: true}); - ``` - After: - ```javascript - const consumer = kafka.consumer({ - groupId: 'test-group', - fromBeginning: true, - }); - await consumer.connect(); - await consumer.subscribe({ topics: ["topic"] }); - ``` +* `subscribe`: + - Regex flags are ignored while passing a topic subscription (like 'i' or 'g'). + - Subscribe must be called only 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, for example, + `consumer.subscribe({ topics: ['topic1'], replace: true});`. + The default value is false to retain existing behaviour. + - While passing a list of topics to `subscribe`, the `fromBeginning` is not set on a per-subscribe basis. Rather, it must be configured in the top-level configuration. - * For auto-committing using a consumer, the properties `autoCommit` and `autoCommitInterval` on `run` are not set on a per-subscribe basis. - Rather, they must be configured in the top-level configuration. - `autoCommitThreshold` is not supported. + Before: + ```javascript + const consumer = kafka.consumer({ + groupId: 'test-group', + }); + await consumer.connect(); + await consumer.subscribe({ topics: ["topic"], fromBeginning: true}); + ``` + After: + ```javascript + const consumer = kafka.consumer({ + groupId: 'test-group', + fromBeginning: true, + }); + await consumer.connect(); + await consumer.subscribe({ topics: ["topic"] }); + ``` + +* `run` : + - For auto-committing using a consumer, the properties `autoCommit` and `autoCommitInterval` on `run` are not set on a per-subscribe basis. + Rather, they must be configured in the top-level configuration. + `autoCommitThreshold` is not supported. Before: ```javascript @@ -254,17 +250,16 @@ consumer.run({ eachMessage: someFunc, autoCommit: true, - autoCommitThreshold: 5000, + autoCommitInterval: 5000, }); ``` - After: ```javascript const kafka = new Kafka({ /* ... */ }); const consumer = kafka.consumer({ /* ... */, autoCommit: true, - autoCommitThreshold: 5000, + autoCommitInterval: 5000, }); await consumer.connect(); await consumer.subscribe({ topics: ["topic"] }); @@ -272,18 +267,18 @@ eachMessage: someFunc, }); ``` - - * The `partitionsConsumedConcurrently` property is not supported at the moment. - * The `eachBatch` method is not supported. - * For the `eachMessage` method while running the consumer: - * The `heartbeat()` no longer needs to be called. Heartbeats are automatically managed by librdkafka. - * `commitOffsets` does not yet support sending metadata for topic partitions being committed. - * `paused()` is supported without any changes. - * Changes to `seek`: - * The restriction to call seek only after `run` is removed. It can be called any time. - * `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` + - The `heartbeat()` no longer needs to be called by the user in the `eachMessage callback`. Heartbeats are automatically managed by librdkafka. + - The `partitionsConsumedConcurrently` property is not supported at the moment. + - The `eachBatch` method is not supported. +* `commitOffsets`: + - Does not yet support sending metadata for topic partitions being committed. +* `seek`: + - The restriction to call seek only after `run` is removed. It can be called any time. +* `pause` and `resume`: + - These methods 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. - * `stop` is not yet supported, and the user must disconnect the consumer. +* `stop` is not yet supported, and the user must disconnect the consumer. ### Admin Client @@ -293,4 +288,72 @@ does not support `replicaAssignment`. * The `deleteTopics` method is fully supported. + +#### Error Handling + + **Action**: Convert any checks based on `instanceof` and `error.name` or to error + checks based on `error.code` or `error.type`. + + **Example**: + ```javascript + 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 */ + } + ``` + + **Comprehensive Changes**: + + Some possible subtypes of `KafkaJSError` have been removed, + and additional information has been added into `KafkaJSError`. + 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 compatibility, as many error types as possible have been retained, but it is + better to switch to checking the `error.code`. + + + Exhaustive list of error types and error fields removed: + | Error | Change | + |-------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | `KafkaJSNonRetriableError` | Removed. 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. | + | `KafkaJSOffsetOutOfRange` | `topic` and `partition` are removed from this object. | + | `KafkaJSMemberIdRequired` | Removed. Automatically handled by librdkafka. | + | `KafkaJSNumberOfRetriesExceeded` | Removed. Retries are handled by librdkafka. | + | `KafkaJSNumberOfRetriesExceeded` | `broker, correlationId, createdAt, sentAt` and `pendingDuration` are removed from this object. | + | `KafkaJSMetadataNotLoaded` | Removed. Metadata is automatically reloaded by librdkafka. | + | `KafkaJSTopicMetadataNotLoaded` | Removed. Topic metadata is automatically reloaded by librdkafka. | + | `KafkaJSStaleTopicMetadataAssignment` | removed as it's automatically refreshed by librdkafka. | + | `KafkaJSDeleteGroupsError` | Removed, as the Admin Client doesn't have this yet. May be added back again, or changed. | + | `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`. | + | `KafkaJSBrokerNotFound` | Removed. This error isn't exposed directly to the user in librdkafka. | + | `KafkaJSLockTimeout` | Removed. This error is not applicable while using librdkafka. | + | `KafkaJSUnsupportedMagicByteInMessageSet` | Removed. It is subsumed into `KafkaJSError` where `error.code === Kafka.ErrorCode.ERR_UNSUPPORTED_VERSION`. | + | `KafkaJSDeleteTopicRecordsError` | Removed, as the Admin Client doesn't have this yet. May be added back again, or changed. | + | `KafkaJSInvariantViolation` | Removed, as it's not applicable to librdkafka. Errors in internal state are subsumed into `KafkaJSError` where `error.code === Kafka.ErrorCode.ERR__STATE`. | + | `KafkaJSInvalidVarIntError` | Removed. This error isn't exposed directly to the user in librdkafka. | + | `KafkaJSInvalidLongError` | Removed. This error isn't exposed directly to the user in librdkafka. | + | `KafkaJSCreateTopicError` | Removed, as the Admin Client doesn't have this yet. May be added back again, or changed.. | + | `KafkaJSAlterPartitionReassignmentsError` | removed, as the RPC is not used in librdkafka. | + | `KafkaJSFetcherRebalanceError` | Removed. This error isn't exposed directly to the user in librdkafka. | + | `KafkaJSConnectionError` | `broker` is removed from this object. | + | `KafkaJSConnectionClosedError` | Removed. Subsumed into `KafkaJSConnectionError` as librdkafka treats them equivalently. | + ## node-rdkafka + +Change the import statement, from + ```javascript + const Kafka = require('node-rdkafka'); + ``` + to + ```javascript + const Kafka = require('confluent-kafka-js'); + ``` +The rest of the functionality should work as usual. \ No newline at end of file diff --git a/README.md b/README.md index cdba3ce0..50edbb5e 100644 --- a/README.md +++ b/README.md @@ -1,67 +1,64 @@ -confluent-kafka-js - Node.js wrapper for Kafka C/C++ library -============================================== +Confluent's Javascript Client for Apache KafkaTM +===================================================== -Copyright (c) 2016-2023 Blizzard Entertainment. +**confluent-kafka-js** is Confluent's Javascript client for [Apache Kafka](http://kafka.apache.org/) and the +[Confluent Platform](https://www.confluent.io/product/compare/). -[https://github.com/confluentinc/confluent-kafka-js](https://github.com/confluentinc/confluent-kafka-js) -# Looking for Collaborators! +Features: -I am looking for *your* help to make this project even better! If you're interested, check [this out](https://github.com/confluentinc/confluent-kafka-js/issues/628) +- **High performance** - confluent-kafka-js is a lightweight wrapper around +[librdkafka](https://github.com/confluentinc/librdkafka), a finely tuned C +client. -# Overview +- **Reliability** - There are a lot of details to get right when writing an Apache Kafka +client. We get them right in one place (librdkafka) and leverage this work +across all of our clients (also [confluent-kafka-python](https://github.com/confluentinc/confluent-kafka-python), +[confluent-kafka-go](https://github.com/confluentinc/confluent-kafka-go) and +and [confluent-kafka-dotnet](https://github.com/confluentinc/confluent-kafka-dotnet)). -The `confluent-kafka-js` library is a high-performance NodeJS client for [Apache Kafka](http://kafka.apache.org/) that wraps the native [librdkafka](https://github.com/edenhill/librdkafka) library. All the complexity of balancing writes across partitions and managing (possibly ever-changing) brokers should be encapsulated in the library. +- **Future proof** - Confluent, founded by the +creators of Kafka, is building a [streaming platform](https://www.confluent.io/product/compare/) +with Apache Kafka at its core. It's high priority for us that client features keep +pace with core Apache Kafka and components of the [Confluent Platform](https://www.confluent.io/product/compare/). -__This library currently uses `librdkafka` version `2.3.0`.__ +## This library is currently not ready for production use. It's an early-access preview in active development, pre-1.0.0, and there might be breaking changes. -## Reference Docs +This library is based heavily on [node-rdkafka](https://github.com/Blizzard/node-rdkafka). -To view the reference docs for the current version, go [here](https://confluentinc.github.io/confluent-kafka-js/current/) +This library contains a promisified API, very similar to the one in [kafkajs](https://github.com/tulios/kafkajs). Some of the tests are also based on the ones in kafkajs. -## Contributing - -For guidelines on contributing please see [CONTRIBUTING.md](https://github.com/confluentinc/confluent-kafka-js/blob/master/CONTRIBUTING.md) +__This library currently uses `librdkafka` based off of the master branch.__ -## Code of Conduct - -Play nice; Play fair. ## Requirements -* Apache Kafka >=0.9 -* Node.js >=4 -* Linux/Mac -* Windows?! See below -* OpenSSL +The following configurations are supported for this early access preview: -### Mac OS High Sierra / Mojave +* Any supported version of Node.js (The two LTS versions, 18 and 20, and the latest version, 21). +* Linux (x64 and arm64) - only glibc, not musl/alpine. +* macOS - arm64/m1 only -OpenSSL has been upgraded in High Sierra and homebrew does not overwrite default system libraries. That means when building confluent-kafka-js, because you are using openssl, you need to tell the linker where to find it: +Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed +from GitHub: -```sh -export CPPFLAGS=-I/usr/local/opt/openssl/include -export LDFLAGS=-L/usr/local/opt/openssl/lib +```bash +$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.4-devel" ``` -Then you can run `npm install` on your application to get it to build correctly. - -__NOTE:__ From the `librdkafka` docs +# Getting Started -> WARNING: Due to a bug in Apache Kafka 0.9.0.x, the ApiVersionRequest (as sent by the client when connecting to the broker) will be silently ignored by the broker causing the request to time out after 10 seconds. This causes client-broker connections to stall for 10 seconds during connection-setup before librdkafka falls back on the `broker.version.fallback` protocol features. The workaround is to explicitly configure `api.version.request` to `false` on clients communicating with <=0.9.0.x brokers. +1. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md) (TBA). +2. If you're migrating from `kafkajs`, you can use the [migration guide](MIGRATION.md#kafkajs). +2. If you're migrating from `node-rdkafka`, you can use the [migration guide](MIGRATION.md#node-rdkafka). -### Alpine -Using Alpine Linux? Check out the [docs](https://github.com/confluentinc/confluent-kafka-js/blob/master/examples/docker-alpine.md). - -### Windows - -Windows build **is not** compiled from `librdkafka` source but it is rather linked against the appropriate version of [NuGet librdkafka.redist](https://www.nuget.org/packages/librdkafka.redist/) static binary that gets downloaded from `https://globalcdn.nuget.org/packages/librdkafka.redist.2.3.0.nupkg` during installation. This download link can be changed using the environment variable `NODE_RDKAFKA_NUGET_BASE_URL` that defaults to `https://globalcdn.nuget.org/packages/` when it's no set. +## Contributing -Requirements: - * [node-gyp for Windows](https://github.com/nodejs/node-gyp#on-windows) +Bug reports and early feedback is appreciated in the form of Github Issues. +For guidelines on contributing please see [CONTRIBUTING.md](https://github.com/confluentinc/confluent-kafka-js/blob/master/CONTRIBUTING.md) -**Note:** I _still_ do not recommend using `confluent-kafka-js` in production on Windows. This feature was in high demand and is provided to help develop, but we do not test against Windows, and windows support may lag behind Linux/Mac support because those platforms are the ones used to develop this library. Contributors are welcome if any Windows issues are found :) +(README below this point TBA - it's just the older stuff). ## Tests @@ -79,18 +76,6 @@ You can run both types of tests by using `Makefile`. Doing so calls `mocha` in y # Usage -You can install the `confluent-kafka-js` module like any other module: - -``` -npm install confluent-kafka-js -``` - -To use the module, you must `require` it. - -```js -const Kafka = require('confluent-kafka-js'); -``` - ## Configuration You can pass many configuration options to `librdkafka`. A full list can be found in `librdkafka`'s [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 07d1e64c..b6bba73a 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -148,7 +148,7 @@ class Producer { } if (Object.hasOwn(this.#kJSConfig, 'timeout')) { - globalConfig['message.timeout.ms'] = this.#kJSConfig.timeout; + globalConfig['request.timeout.ms'] = this.#kJSConfig.timeout; } /* Set the logger */ From 44db29c348818b9e6455392d6f05d70a83d869f6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 14 Dec 2023 20:30:30 +0530 Subject: [PATCH 032/224] Update license and add missing error --- LICENSE.txt | 3 +-- lib/kafkajs/_consumer.js | 4 ++++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/LICENSE.txt b/LICENSE.txt index ea9b1c3b..101cc5a7 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -1,6 +1,5 @@ The MIT License (MIT) -Copyright (c) 2016-2023 Blizzard Entertainment - 2023 Confluent, Inc. +Copyright (c) 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/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index b9cdcb9c..b20288d1 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -213,6 +213,10 @@ class Consumer { } if (Object.hasOwn(this.#kJSConfig, 'partitionAssignors')) { + if (!Array.isArray(this.#kJSConfig.partitionAssignors)) { + throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + this.#kJSConfig.partitionAssignors.forEach(assignor => { if (typeof assignor !== 'string') throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__INVALID_ARG }); From 1fe91434483bf12b5ffc2eacdf56fc3e92d42395 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Dec 2023 10:11:04 +0530 Subject: [PATCH 033/224] Add changes for 0.1.4-devel release --- .semaphore/semaphore.yml | 2 +- package.json | 16 +++------------- 2 files changed, 4 insertions(+), 14 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 8eca7fe0..31209b1d 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -28,7 +28,7 @@ global_job_config: - git submodule update --init --recursive - cd deps/librdkafka - git fetch origin - - git checkout dev_libdir_fix_for_zstd_zlib + - git checkout master - cd ../../ - export MKL_DEBUG=1 diff --git a/package.json b/package.json index 9b755823..f977ba86 100644 --- a/package.json +++ b/package.json @@ -1,8 +1,8 @@ { "name": "confluent-kafka-js", - "version": "v0.1.3-devel", + "version": "v0.1.4-devel", "description": "Node.js bindings for librdkafka", - "librdkafka": "2.3.0", + "librdkafka": "master", "main": "lib/index.js", "types": "types/index.d.ts", "scripts": { @@ -28,16 +28,6 @@ "type": "git", "url": "git@github.com:confluentinc/confluent-kafka-js.git" }, - "contributors": [ - { - "name": "Stephen Parente", - "email": "webmakersteve@gmail.com" - }, - { - "name": "Matt Gollob", - "email": "mattness@users.noreply.github.com" - } - ], "license": "MIT", "devDependencies": { "@types/node": "^20.4.5", @@ -57,6 +47,6 @@ "nan": "^2.17.0" }, "engines": { - "node": ">=14.0.0" + "node": ">=18.0.0" } } From d4f671f993cd6f5bffe6f749b73afc4fbd5c7d9a Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Dec 2023 12:54:16 +0530 Subject: [PATCH 034/224] Fix merge issue in conflict, update security dependency --- examples/kafkajs/consumer.js | 20 ++++++----------- lib/kafkajs/_kafka.js | 4 ++-- package-lock.json | 42 ++++++++++++++++++++---------------- package.json | 4 ++-- 4 files changed, 35 insertions(+), 35 deletions(-) diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index cd8df5c7..0774a7e0 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,4 +1,4 @@ -const { Kafka } = require('../..').KafkaJS; +const { Kafka, logLevel } = require('../..').KafkaJS; //const { Kafka } = require('kafkajs') async function consumerStart() { @@ -6,18 +6,11 @@ async function consumerStart() { var stopped = false; const kafka = new Kafka({ - brokers: [''], - ssl: true, - connectionTimeout: 5000, - sasl: { - mechanism: 'plain', - username: '', - password: '', - }, + brokers: ['localhost:9092'], }); consumer = kafka.consumer({ - groupId: 'test-group', + groupId: 'test-group22' + Math.random(), rebalanceListener: { onPartitionsAssigned: async (assignment) => { console.log(`Assigned partitions ${JSON.stringify(assignment)}`); @@ -33,10 +26,11 @@ async function consumerStart() { }, rdKafka: { globalConfig: { - 'enable.auto.commit': false + 'debug': 'conf', + 'enable.auto.commit': false, + 'auto.offset.reset': 'error' }, topicConfig: { - 'auto.offset.reset': 'earliest' }, } }); @@ -46,7 +40,7 @@ async function consumerStart() { await consumer.subscribe({ topics: [ - "topic2" + "test-topic" ] }) diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 08a7e0a6..134dbf63 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -27,8 +27,8 @@ class Kafka { mergedConfig.rdKafka = mergedConfig.rdKafka ?? {}; if (typeof config.rdKafka === 'object') { - Object.assign(mergedConfig.rdKafka.globalConfig ?? {}, config.rdKafka.globalConfig ?? {}); - Object.assign(mergedConfig.rdKafka.topicConfig ?? {}, config.rdKafka.topicConfig ?? {}); + mergedConfig.rdKafka.globalConfig = Object.assign(mergedConfig.rdKafka.globalConfig ?? {}, config.rdKafka.globalConfig ?? {}); + mergedConfig.rdKafka.topicConfig = Object.assign(mergedConfig.rdKafka.topicConfig ?? {}, config.rdKafka.topicConfig ?? {}); delete config.rdKafka; } diff --git a/package-lock.json b/package-lock.json index 337bebea..812c4e84 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "confluent-kafka-js", - "version": "v0.1.3-devel", + "version": "v0.1.4-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "confluent-kafka-js", - "version": "v0.1.3-devel", + "version": "v0.1.4-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { @@ -19,7 +19,7 @@ "bluebird": "^3.5.3", "eslint": "8.54.0", "jest": "^29.7.0", - "jsdoc": "^3.4.0", + "jsdoc": "^4.0.2", "jshint": "^2.10.1", "mocha": "^10.2.0", "node-gyp": "^9.3.1", @@ -27,7 +27,7 @@ "typescript": "^5.1.6" }, "engines": { - "node": ">=14.0.0" + "node": ">=18.0.0" } }, "node_modules/@aashutoshrathi/word-wrap": { @@ -1230,6 +1230,18 @@ "@jridgewell/sourcemap-codec": "^1.4.14" } }, + "node_modules/@jsdoc/salty": { + "version": "0.2.7", + "resolved": "https://registry.npmjs.org/@jsdoc/salty/-/salty-0.2.7.tgz", + "integrity": "sha512-mh8LbS9d4Jq84KLw8pzho7XC2q2/IJGiJss3xwRoLD1A+EE16SjN4PfaG4jRCzKegTFLlN0Zd8SdUPE6XdoPFg==", + "dev": true, + "dependencies": { + "lodash": "^4.17.21" + }, + "engines": { + "node": ">=v12.0.0" + } + }, "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", @@ -2425,9 +2437,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.612", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.612.tgz", - "integrity": "sha512-dM8BMtXtlH237ecSMnYdYuCkib2QHq0kpWfUnavjdYsyr/6OsAwg5ZGUfnQ9KD1Ga4QgB2sqXlB2NT8zy2GnVg==", + "version": "1.4.613", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.613.tgz", + "integrity": "sha512-r4x5+FowKG6q+/Wj0W9nidx7QO31BJwmR2uEo+Qh3YLGQ8SbBAFuDFpTxzly/I2gsbrFwBuIjrMp423L3O5U3w==", "dev": true }, "node_modules/emittery": { @@ -3986,12 +3998,13 @@ } }, "node_modules/jsdoc": { - "version": "3.6.11", - "resolved": "https://registry.npmjs.org/jsdoc/-/jsdoc-3.6.11.tgz", - "integrity": "sha512-8UCU0TYeIYD9KeLzEcAu2q8N/mx9O3phAGl32nmHlE0LpaJL71mMkP4d+QE5zWfNt50qheHtOZ0qoxVrsX5TUg==", + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/jsdoc/-/jsdoc-4.0.2.tgz", + "integrity": "sha512-e8cIg2z62InH7azBBi3EsSEqrKx+nUtAS5bBcYTSpZFA+vhNPyhv8PTFZ0WsjOPDj04/dOLlm08EDcQJDqaGQg==", "dev": true, "dependencies": { - "@babel/parser": "^7.9.4", + "@babel/parser": "^7.20.15", + "@jsdoc/salty": "^0.2.1", "@types/markdown-it": "^12.2.3", "bluebird": "^3.7.2", "catharsis": "^0.9.0", @@ -4004,7 +4017,6 @@ "mkdirp": "^1.0.4", "requizzle": "^0.2.3", "strip-json-comments": "^3.1.0", - "taffydb": "2.6.2", "underscore": "~1.13.2" }, "bin": { @@ -5703,12 +5715,6 @@ "url": "https://github.com/sponsors/ljharb" } }, - "node_modules/taffydb": { - "version": "2.6.2", - "resolved": "https://registry.npmjs.org/taffydb/-/taffydb-2.6.2.tgz", - "integrity": "sha512-y3JaeRSplks6NYQuCOj3ZFMO3j60rTwbuKCvZxsAraGYH2epusatvZ0baZYA01WsGqJBq/Dl6vOrMUJqyMj8kA==", - "dev": true - }, "node_modules/tar": { "version": "6.2.0", "resolved": "https://registry.npmjs.org/tar/-/tar-6.2.0.tgz", diff --git a/package.json b/package.json index f977ba86..7cc676c1 100644 --- a/package.json +++ b/package.json @@ -32,9 +32,9 @@ "devDependencies": { "@types/node": "^20.4.5", "bluebird": "^3.5.3", - "jest": "^29.7.0", "eslint": "8.54.0", - "jsdoc": "^3.4.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", "jshint": "^2.10.1", "mocha": "^10.2.0", "node-gyp": "^9.3.1", From a5cbdfd610e87603f45662b0e2f02ac2c5dbb519 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Dec 2023 14:45:11 +0530 Subject: [PATCH 035/224] Remove insecure dependency and add promisified API to docs --- Makefile | 6 +++--- jsdoc.conf | 16 ++++++++++++++++ package-lock.json | 7 ------- package.json | 1 - 4 files changed, 19 insertions(+), 11 deletions(-) create mode 100644 jsdoc.conf diff --git a/Makefile b/Makefile index 666d4832..3e1865a8 100644 --- a/Makefile +++ b/Makefile @@ -80,10 +80,10 @@ endef docs: node_modules/.dirstamp @rm -rf docs - @./node_modules/jsdoc/jsdoc.js --destination docs \ + @./node_modules/jsdoc/jsdoc.js --debug --destination docs \ --recurse -R ./README.md \ - -t "./node_modules/toolkit-jsdoc/" \ - --tutorials examples ./lib + -c ./jsdoc.conf \ + --tutorials examples/ ./lib gh-pages: node_modules/.dirstamp @./make_docs.sh diff --git a/jsdoc.conf b/jsdoc.conf new file mode 100644 index 00000000..37fac1d5 --- /dev/null +++ b/jsdoc.conf @@ -0,0 +1,16 @@ +{ + "plugins": [], + "recurseDepth": 10, + "source": { + "includePattern": ".+\\.js(doc|x)?$" + }, + "sourceType": "module", + "tags": { + "allowUnknownTags": true, + "dictionaries": ["jsdoc","closure"] + }, + "templates": { + "cleverLinks": false, + "monospaceLinks": false + } +} diff --git a/package-lock.json b/package-lock.json index 812c4e84..30fe2b50 100644 --- a/package-lock.json +++ b/package-lock.json @@ -23,7 +23,6 @@ "jshint": "^2.10.1", "mocha": "^10.2.0", "node-gyp": "^9.3.1", - "toolkit-jsdoc": "^1.0.0", "typescript": "^5.1.6" }, "engines": { @@ -5791,12 +5790,6 @@ "node": ">=8.0" } }, - "node_modules/toolkit-jsdoc": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/toolkit-jsdoc/-/toolkit-jsdoc-1.0.0.tgz", - "integrity": "sha512-57bpRaZgZ8M2FUblW3OJVWDfbING/rBvCda/mxXEth6fCp3M1m6/tX+pvXSJyqq24tVzdTYaGM+ZduPlwcDFHw==", - "dev": true - }, "node_modules/tr46": { "version": "0.0.3", "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", diff --git a/package.json b/package.json index 7cc676c1..f09ea442 100644 --- a/package.json +++ b/package.json @@ -38,7 +38,6 @@ "jshint": "^2.10.1", "mocha": "^10.2.0", "node-gyp": "^9.3.1", - "toolkit-jsdoc": "^1.0.0", "typescript": "^5.1.6" }, "dependencies": { From e5f216dc6de6a95d88efc81cce12a4a0cad10143 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 20 Dec 2023 13:38:58 +0530 Subject: [PATCH 036/224] Add functional changes for kafkaJs block --- lib/kafkajs/_common.js | 84 +++++++------- lib/kafkajs/_consumer.js | 236 +++++++++++++++++++++++---------------- lib/kafkajs/_kafka.js | 20 ++-- lib/kafkajs/_producer.js | 128 +++++++++++++-------- types/kafkajs.d.ts | 19 +++- 5 files changed, 282 insertions(+), 205 deletions(-) diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index dc0d65d7..a364496c 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -97,11 +97,11 @@ function createReplacementErrorMessage(cOrP, fnCall, property, propertyVal, repl } return `'${property}' is not supported as a property to '${fnCall}', but must be passed to the ${cOrP} during creation.\n` + `Before: \n` + - `\tconst ${cOrP} = kafka.${cOrP}({ ... });\n` + + `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJs: { ... }, });\n` + `\tawait ${cOrP}.connect();\n` + `\t${cOrP}.${fnCall}({ ${propertyVal}, ... });\n` + `After: \n` + - `\tconst ${cOrP} = kafka.${cOrP}({ ${replacementVal}, ... });\n` + + `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJs: { ${replacementVal}, ... }, });\n` + `\tawait ${cOrP}.connect();\n` + `\t${cOrP}.${fnCall}({ ... });\n` + (isLK ? `For more details on what can be used inside the rdKafka block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n` : ''); @@ -179,24 +179,28 @@ const CompatibilityErrorMessages = Object.freeze({ /** * 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}} + * @returns {import('../../types/config').ProducerGlobalConfig | import('../../types/config').ConsumerGlobalConfig} the converted configuration * @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 = {}; - const topicConfig = {}; - - if (!Array.isArray(config["brokers"])) { - throw new error.KafkaJSError(CompatibilityErrorMessages.brokerString(), { - code: error.ErrorCodes.ERR__INVALID_ARG, - }); +function kafkaJSToRdKafkaConfig(config) { + /* Since the kafkaJs block is specified, we operate in + * kafkaJs compatibility mode. That means we change the defaults + * match the kafkaJs defaults. */ + const rdkafkaConfig = {}; + + if (Object.hasOwn(config, "brokers")) { + if (!Array.isArray(config["brokers"])) { + throw new error.KafkaJSError(CompatibilityErrorMessages.brokerString(), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + rdkafkaConfig["bootstrap.servers"] = config["brokers"].join(","); } - globalConfig["bootstrap.servers"] = config["brokers"].join(","); if (Object.hasOwn(config, "clientId")) { - globalConfig["client.id"] = config.clientId; + rdkafkaConfig["client.id"] = config.clientId; } let withSASL = false; @@ -224,18 +228,18 @@ async function kafkaJSToRdKafkaConfig(config) { }); } - globalConfig["sasl.mechanism"] = mechanism; - globalConfig["sasl.username"] = sasl.username; - globalConfig["sasl.password"] = sasl.password; + rdkafkaConfig["sasl.mechanism"] = mechanism; + rdkafkaConfig["sasl.username"] = sasl.username; + rdkafkaConfig["sasl.password"] = sasl.password; withSASL = true; } if (Object.hasOwn(config, "ssl") && config.ssl && withSASL) { - globalConfig["security.protocol"] = "sasl_ssl"; + rdkafkaConfig["security.protocol"] = "sasl_ssl"; } else if (withSASL) { - globalConfig["security.protocol"] = "sasl_plaintext"; + rdkafkaConfig["security.protocol"] = "sasl_plaintext"; } else if (Object.hasOwn(config, "ssl") && config.ssl) { - globalConfig["security.protocol"] = "ssl"; + rdkafkaConfig["security.protocol"] = "ssl"; } /* TODO: add best-effort support for ssl besides just true/false */ @@ -246,14 +250,14 @@ async function kafkaJSToRdKafkaConfig(config) { } if (Object.hasOwn(config, "requestTimeout")) { - globalConfig["socket.timeout.ms"] = config.requestTimeout; + rdkafkaConfig["socket.timeout.ms"] = config.requestTimeout; } else { /* KafkaJS default */ - globalConfig["socket.timeout.ms"] = 30000; + rdkafkaConfig["socket.timeout.ms"] = 30000; } if (Object.hasOwn(config, "enforceRequestTimeout") && !config.enforceRequestTimeout) { - globalConfig["socket.timeout.ms"] = 300000; + rdkafkaConfig["socket.timeout.ms"] = 300000; } const connectionTimeout = config.connectionTimeout ?? 1000; @@ -262,14 +266,14 @@ async function kafkaJSToRdKafkaConfig(config) { /* The minimum value for socket.connection.setup.timeout.ms is 1000. */ totalConnectionTimeout = Math.max(totalConnectionTimeout, 1000); - globalConfig["socket.connection.setup.timeout.ms"] = totalConnectionTimeout; + rdkafkaConfig["socket.connection.setup.timeout.ms"] = totalConnectionTimeout; const retry = config.retry ?? {}; const { maxRetryTime, initialRetryTime, factor, multiplier, retries, restartOnFailure } = retry; - globalConfig["retry.backoff.max.ms"] = maxRetryTime ?? 30000; - globalConfig["retry.backoff.ms"] = initialRetryTime ?? 300; - globalConfig["retries"] = retries ?? 5; + rdkafkaConfig["retry.backoff.max.ms"] = maxRetryTime ?? 30000; + rdkafkaConfig["retry.backoff.ms"] = initialRetryTime ?? 300; + rdkafkaConfig["retries"] = retries ?? 5; if ((typeof factor === 'number') || (typeof multiplier === 'number')) { throw new error.KafkaJSError(CompatibilityErrorMessages.retryFactorMultiplier(), { @@ -295,6 +299,7 @@ async function kafkaJSToRdKafkaConfig(config) { }); } + rdkafkaConfig["log_level"] = 6 /* LOG_INFO - default in KafkaJS compatibility mode. */; if (Object.hasOwn(config, "logLevel")) { let setLevel = config.logLevel; @@ -303,19 +308,20 @@ async function kafkaJSToRdKafkaConfig(config) { } switch (setLevel) { case logLevel.NOTHING: - globalConfig["log_level"] = 0; /* LOG_EMERG - we don't have a true log nothing yet */ + rdkafkaConfig["log_level"] = 0; /* LOG_EMERG - we don't have a true log nothing yet */ break; case logLevel.ERROR: - globalConfig["log_level"] = 3 /* LOG_ERR */; + rdkafkaConfig["log_level"] = 3 /* LOG_ERR */; break; case logLevel.WARN: - globalConfig["log_level"] = 4 /* LOG_WARNING */; + rdkafkaConfig["log_level"] = 4 /* LOG_WARNING */; break; case logLevel.INFO: - globalConfig["log_level"] = 6 /* LOG_INFO */; + rdkafkaConfig["log_level"] = 6 /* LOG_INFO */; break; case logLevel.DEBUG: - globalConfig["debug"] = "all" /* this will set librdkafka log_level to 7 */; + rdkafkaConfig["debug"] = "all" /* Turn on debug logs for everything, otherwise this log level is not useful*/; + rdkafkaConfig["log_level"] = 7 /* LOG_DEBUG */; break; default: throw new error.KafkaJSError(CompatibilityErrorMessages.logLevelName(setLevel), { @@ -324,20 +330,7 @@ async function kafkaJSToRdKafkaConfig(config) { } } - if (config.rdKafka) { - if (config.rdKafka.constructor === Function) { - await config.rdKafka(globalConfig, topicConfig); - } else { - Object.assign(globalConfig, config.rdKafka.globalConfig); - Object.assign(topicConfig, config.rdKafka.topicConfig); - } - } - - - if (!Object.hasOwn(globalConfig, 'log_level')) - globalConfig['log_level'] = Object.hasOwn(globalConfig, 'debug') ? 7 /* LOG_DEBUG */ : 6 /* LOG_INFO */; - - return { globalConfig, topicConfig }; + return rdkafkaConfig; } function checkAllowedKeys(allowedKeysSpecific, config) { @@ -421,7 +414,6 @@ function createKafkaJsErrorFromLibRdKafkaError(librdKafkaError) { err = new error.KafkaJSError(librdKafkaError, properties); } - console.log("Converted err = " + JSON.stringify(err, null, 2) + " librdkafka erro = " + JSON.stringify(librdKafkaError, null, 2)); return err; } diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index b20288d1..17aa9fb2 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -10,7 +10,8 @@ const { DefaultLogger, CompatibilityErrorMessages, severityToLogLevel, - checkAllowedKeys + checkAllowedKeys, + logLevel } = require('./_common'); const { Buffer } = require('buffer'); @@ -30,16 +31,16 @@ const PartitionAssigners = Object.freeze({ class Consumer { /** - * kJSConfig is the kafkaJS config object. - * @type {import("../../types/kafkajs").ConsumerConfig|null} + * The config supplied by the user. + * @type {import("../../types/kafkajs").ConsumerConstructorConfig|null} */ - #kJSConfig = null; + #userConfig = 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} + * The config realized after processing any compatibility options. + * @type {import("../../types/config").ConsumerGlobalConfig|null} */ - #rdKafkaConfig = null; + #internalConfig = null; /** * internalClient is the node-rdkafka client used by the API. @@ -95,13 +96,13 @@ class Consumer { * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig */ constructor(kJSConfig) { - this.#kJSConfig = kJSConfig; + this.#userConfig = kJSConfig; } - async #config() { - if (!this.#rdKafkaConfig) - this.#rdKafkaConfig = await this.#finalizedConfig(); - return this.#rdKafkaConfig; + #config() { + if (!this.#internalConfig) + this.#internalConfig = this.#finalizedConfig(); + return this.#internalConfig; } /** @@ -112,26 +113,38 @@ class Consumer { #rebalanceCallback(err, assignment) { // Create the librdkafka error err = LibrdKafkaError.create(err); + const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; 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(this.#logger.error); - break; - case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: - call = (this.#kJSConfig.rebalanceListener.onPartitionsRevoked ? - this.#kJSConfig.rebalanceListener.onPartitionsRevoked(assignment) : - Promise.resolve()).catch(this.#logger.error); - break; - default: - call = Promise.reject(`Unexpected rebalanceListener error code ${err.code}`).catch((e) => { - this.#logger.error(e); - }); - break; + if (typeof userSpecifiedRebalanceCb === 'function') { + call = new Promise((resolve, reject) => setImmediate(() => { + try { + userSpecifiedRebalanceCb(err, assignment); + resolve(); + } catch (e) { + reject(e); + } + })); + } else { + 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.#userConfig.rebalanceListener.onPartitionsAssigned ? + this.#userConfig.rebalanceListener.onPartitionsAssigned(assignment) : + Promise.resolve()).catch(this.#logger.error); + break; + case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: + call = (this.#userConfig.rebalanceListener.onPartitionsRevoked ? + this.#userConfig.rebalanceListener.onPartitionsRevoked(assignment) : + Promise.resolve()).catch(this.#logger.error); + break; + default: + call = Promise.reject(`Unexpected rebalanceListener error code ${err.code}`).catch((e) => { + this.#logger.error(e); + }); + break; + } } call @@ -156,7 +169,7 @@ class Consumer { offset: String(topicPartition.offset), })); - if (offsetsToCommit.length !== 0 && this.#rdKafkaConfig.globalConfig['enable.auto.commit']) { + if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); } } @@ -173,11 +186,12 @@ class Consumer { }); } - async #finalizedConfig() { - /* This sets the common configuration options for the client. */ - const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + #kafkaJSToConsumerConfig(kjsConfig) { + if (!kjsConfig || Object.keys(kjsConfig).length === 0) { + return {}; + } - const allowedKeysSpecific = [ + const allowedKeysSpecificToConsumer = [ 'groupId', 'partitionAssigners', 'partitionAssignors', @@ -200,125 +214,150 @@ class Consumer { 'rebalanceListener', ]; - checkAllowedKeys(allowedKeysSpecific, this.#kJSConfig); + checkAllowedKeys(allowedKeysSpecificToConsumer, kjsConfig); + const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); /* Consumer specific configuration */ - - if (Object.hasOwn(this.#kJSConfig, 'groupId')) { - globalConfig['group.id'] = this.#kJSConfig.groupId; + if (Object.hasOwn(kjsConfig, 'groupId')) { + rdKafkaConfig['group.id'] = kjsConfig.groupId; } - if (Object.hasOwn(this.#kJSConfig, 'partitionAssigners')) { - this.#kJSConfig.partitionAssignors = this.#kJSConfig.partitionAssigners; + if (Object.hasOwn(kjsConfig, 'partitionAssigners')) { + kjsConfig.partitionAssignors = kjsConfig.partitionAssigners; } - if (Object.hasOwn(this.#kJSConfig, 'partitionAssignors')) { - if (!Array.isArray(this.#kJSConfig.partitionAssignors)) { + if (Object.hasOwn(kjsConfig, 'partitionAssignors')) { + if (!Array.isArray(kjsConfig.partitionAssignors)) { throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__INVALID_ARG }); } - this.#kJSConfig.partitionAssignors.forEach(assignor => { + kjsConfig.partitionAssignors.forEach(assignor => { if (typeof assignor !== 'string') throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__INVALID_ARG }); if (assignor === PartitionAssigners.cooperativeSticky) throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); }); - globalConfig['partition.assignment.strategy'] = this.#kJSConfig.partitionAssignors.join(','); + rdKafkaConfig['partition.assignment.strategy'] = kjsConfig.partitionAssignors.join(','); } else { - globalConfig['partition.assignment.strategy'] = PartitionAssigners.roundRobin; + rdKafkaConfig['partition.assignment.strategy'] = PartitionAssigners.roundRobin; } - if (Object.hasOwn(this.#kJSConfig, 'sessionTimeout')) { - globalConfig['session.timeout.ms'] = this.#kJSConfig.sessionTimeout; + if (Object.hasOwn(kjsConfig, 'sessionTimeout')) { + rdKafkaConfig['session.timeout.ms'] = kjsConfig.sessionTimeout; } else { - globalConfig['session.timeout.ms'] = 30000; + rdKafkaConfig['session.timeout.ms'] = 30000; } - if (Object.hasOwn(this.#kJSConfig, 'rebalanceTimeout')) { + if (Object.hasOwn(kjsConfig, 'rebalanceTimeout')) { /* In librdkafka, we use the max poll interval as the rebalance timeout as well. */ - globalConfig['max.poll.interval.ms'] = this.#kJSConfig.rebalanceTimeout; + rdKafkaConfig['max.poll.interval.ms'] = kjsConfig.rebalanceTimeout; } - if (Object.hasOwn(this.#kJSConfig, 'heartbeatInterval')) { - globalConfig['heartbeat.interval.ms'] = this.#kJSConfig.heartbeatInterval; + if (Object.hasOwn(kjsConfig, 'heartbeatInterval')) { + rdKafkaConfig['heartbeat.interval.ms'] = kjsConfig.heartbeatInterval; } - if (Object.hasOwn(this.#kJSConfig, 'metadataMaxAge')) { - globalConfig['topic.metadata.refresh.interval.ms'] = this.#kJSConfig.metadataMaxAge; + if (Object.hasOwn(kjsConfig, 'metadataMaxAge')) { + rdKafkaConfig['topic.metadata.refresh.interval.ms'] = kjsConfig.metadataMaxAge; } - if (Object.hasOwn(this.#kJSConfig, 'allowAutoTopicCreation')) { - globalConfig['allow.auto.create.topics'] = this.#kJSConfig.allowAutoTopicCreation; + if (Object.hasOwn(kjsConfig, 'allowAutoTopicCreation')) { + rdKafkaConfig['allow.auto.create.topics'] = kjsConfig.allowAutoTopicCreation; } else { - globalConfig['allow.auto.create.topics'] = true; + rdKafkaConfig['allow.auto.create.topics'] = true; } - if (Object.hasOwn(this.#kJSConfig, 'maxBytesPerPartition')) { - globalConfig['max.partition.fetch.bytes'] = this.#kJSConfig.maxBytesPerPartition; + if (Object.hasOwn(kjsConfig, 'maxBytesPerPartition')) { + rdKafkaConfig['max.partition.fetch.bytes'] = kjsConfig.maxBytesPerPartition; } else { - globalConfig['max.partition.fetch.bytes'] = 1048576; + rdKafkaConfig['max.partition.fetch.bytes'] = 1048576; } - if (Object.hasOwn(this.#kJSConfig, 'maxWaitTimeInMs')) { - globalConfig['fetch.wait.max.ms'] = this.#kJSConfig.maxWaitTimeInMs; + if (Object.hasOwn(kjsConfig, 'maxWaitTimeInMs')) { + rdKafkaConfig['fetch.wait.max.ms'] = kjsConfig.maxWaitTimeInMs; } else { - globalConfig['fetch.wait.max.ms'] = 5000; + rdKafkaConfig['fetch.wait.max.ms'] = 5000; } - if (Object.hasOwn(this.#kJSConfig, 'minBytes')) { - globalConfig['fetch.min.bytes'] = this.#kJSConfig.minBytes; + if (Object.hasOwn(kjsConfig, 'minBytes')) { + rdKafkaConfig['fetch.min.bytes'] = kjsConfig.minBytes; } - if (Object.hasOwn(this.#kJSConfig, 'maxBytes')) { - globalConfig['fetch.message.max.bytes'] = this.#kJSConfig.maxBytes; + if (Object.hasOwn(kjsConfig, 'maxBytes')) { + rdKafkaConfig['fetch.message.max.bytes'] = kjsConfig.maxBytes; } else { - globalConfig['fetch.message.max.bytes'] = 10485760; + rdKafkaConfig['fetch.message.max.bytes'] = 10485760; } - if (Object.hasOwn(this.#kJSConfig, 'readUncommitted')) { - globalConfig['isolation.level'] = this.#kJSConfig.readUncommitted ? 'read_uncommitted' : 'read_committed'; + if (Object.hasOwn(kjsConfig, 'readUncommitted')) { + rdKafkaConfig['isolation.level'] = kjsConfig.readUncommitted ? 'read_uncommitted' : 'read_committed'; } - if (Object.hasOwn(this.#kJSConfig, 'maxInFlightRequests')) { - globalConfig['max.in.flight'] = this.#kJSConfig.maxInFlightRequests; + if (Object.hasOwn(kjsConfig, 'maxInFlightRequests')) { + rdKafkaConfig['max.in.flight'] = kjsConfig.maxInFlightRequests; } - if (Object.hasOwn(this.#kJSConfig, 'rackId')) { - globalConfig['client.rack'] = this.#kJSConfig.rackId; + if (Object.hasOwn(kjsConfig, 'rackId')) { + rdKafkaConfig['client.rack'] = kjsConfig.rackId; } - if (Object.hasOwn(this.#kJSConfig, 'fromBeginning')) { - topicConfig['auto.offset.reset'] = this.#kJSConfig.fromBeginning ? 'earliest' : 'latest'; + if (Object.hasOwn(kjsConfig, 'fromBeginning')) { + rdKafkaConfig['auto.offset.reset'] = kjsConfig.fromBeginning ? 'earliest' : 'latest'; } - if (Object.hasOwn(this.#kJSConfig, 'autoCommit')) { - globalConfig['enable.auto.commit'] = this.#kJSConfig.autoCommit; + if (Object.hasOwn(kjsConfig, 'autoCommit')) { + rdKafkaConfig['enable.auto.commit'] = kjsConfig.autoCommit; } - if (Object.hasOwn(this.#kJSConfig, 'autoCommitInterval')) { - globalConfig['auto.commit.interval.ms'] = this.#kJSConfig.autoCommitInterval; + if (Object.hasOwn(kjsConfig, 'autoCommitInterval')) { + rdKafkaConfig['auto.commit.interval.ms'] = kjsConfig.autoCommitInterval; } - if (Object.hasOwn(this.#kJSConfig, 'autoCommitThreshold')) { + if (Object.hasOwn(kjsConfig, 'autoCommitThreshold')) { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - globalConfig['offset_commit_cb'] = true; + /* Set the logger */ + if (Object.hasOwn(kjsConfig, 'logger')) { + this.#logger = kjsConfig.logger; + } - 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 = {}; + /* Set the log level - INFO for compatibility with kafkaJS, or DEBUG if that is turned + * on using the logLevel property. rdKafkaConfig.log_level is guaranteed to be set if we're + * here, and containing the correct value. */ + this.#logger.setLogLevel(severityToLogLevel[rdKafkaConfig.log_level]); + + return rdKafkaConfig; + } + + #finalizedConfig() { + /* Creates an rdkafka config based off the kafkaJS block. Switches to compatibility mode if the block exists. */ + let compatibleConfig = this.#kafkaJSToConsumerConfig(this.#userConfig.kafkaJs); + + /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest + * log level, as librdkafka will control the granularity. */ + if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { + this.#logger.setLogLevel(logLevel.DEBUG); } - globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); - /* Set the logger */ - if (Object.hasOwn(this.#kJSConfig, 'logger')) { - this.#logger = this.#kJSConfig.logger; + let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); + + /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ + delete rdKafkaConfig.kafkaJs; + delete rdKafkaConfig.rebalanceListener; + + /* Certain properties that the user has set are overridden. We use trampolines to accommodate the user's callbacks. + * TODO: add trampoline method for offset commit callback. */ + rdKafkaConfig['offset_commit_cb'] = true; + + if (!Object.hasOwn(this.#userConfig, '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.#userConfig.rebalanceListener = {}; } - this.#logger.setLogLevel(severityToLogLevel[globalConfig.log_level]); + rdKafkaConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); - return { globalConfig, topicConfig }; + return rdKafkaConfig; } #readyCb() { @@ -443,10 +482,10 @@ class Consumer { throw new error.KafkaJSError('Connect has already been called elsewhere.', { code: error.ErrorCodes.ERR__STATE }); } - const { globalConfig, topicConfig } = await this.#config(); + const rdKafkaConfig = this.#config(); this.#state = ConsumerState.CONNECTING; - this.#internalClient = new RdKafka.KafkaConsumer(globalConfig, topicConfig); + this.#internalClient = new RdKafka.KafkaConsumer(rdKafkaConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); this.#internalClient.on('event.error', this.#errorCb.bind(this)); this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); @@ -550,7 +589,12 @@ class Consumer { /* 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(); + const m = await this.#consumeSingle().catch(e => { + /* Since this error cannot be exposed to the user in the current situation, just log and retry. + * This is due to restartOnFailure being set to always true. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${JSON.stringify(e)}`); + }); /* 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. @@ -739,7 +783,7 @@ class Consumer { } } - if (offsetsToCommit.length !== 0 && this.#rdKafkaConfig.globalConfig['enable.auto.commit']) { + if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); } diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 134dbf63..e9a1a006 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -5,11 +5,12 @@ const error = require('./_error'); const { logLevel } = require('./_common'); class Kafka { + /* @type{import("../../types/kafkajs").CommonConstructorConfig} */ #commonClientConfig = {}; /** * - * @param {import("../../types/kafkajs").KafkaConfig} config + * @param {import("../../types/kafkajs").CommonConstructorConfig} config */ constructor(config) { this.#commonClientConfig = config ?? {}; @@ -17,19 +18,18 @@ class Kafka { /** * Merge the producer/consumer specific configuration with the common configuration. - * @param {import("../../types/kafkajs").ProducerConfig|import("../../types/kafkajs").ConsumerConfig} config - * @returns {(import("../../types/kafkajs").ProducerConfig & import("../../types/kafkajs").KafkaConfig) | (import("../../types/kafkajs").ConsumerConfig & import("../../types/kafkajs").KafkaConfig)} + * @param {import("../../types/kafkajs").ProducerConstructorConfig|import("../../types/kafkajs").ConsumerConstructorConfig} config + * @returns {(import("../../types/kafkajs").ProducerConstructorConfig & import("../../types/kafkajs").CommonConstructorConfig) | (import("../../types/kafkajs").ConsumerConstructorConfig & import("../../types/kafkajs").CommonConstructorConfig)} */ #mergeConfiguration(config) { config = config ?? {}; const mergedConfig = Object.assign({}, this.#commonClientConfig); - mergedConfig.rdKafka = mergedConfig.rdKafka ?? {}; + mergedConfig.kafkaJs = mergedConfig.kafkaJs ?? {}; - if (typeof config.rdKafka === 'object') { - mergedConfig.rdKafka.globalConfig = Object.assign(mergedConfig.rdKafka.globalConfig ?? {}, config.rdKafka.globalConfig ?? {}); - mergedConfig.rdKafka.topicConfig = Object.assign(mergedConfig.rdKafka.topicConfig ?? {}, config.rdKafka.topicConfig ?? {}); - delete config.rdKafka; + if (typeof config.kafkaJs === 'object') { + mergedConfig.kafkaJs = Object.assign(mergedConfig.kafkaJs, config.kafkaJs); + delete config.kafkaJs; } Object.assign(mergedConfig, config); @@ -39,7 +39,7 @@ class Kafka { /** * Creates a new producer. - * @param {import("../../types/kafkajs").ProducerConfig} config + * @param {import("../../types/kafkajs").ProducerConstructorConfig} config * @returns {Producer} */ producer(config) { @@ -48,7 +48,7 @@ class Kafka { /** * Creates a new consumer. - * @param {import("../../types/kafkajs").ConsumerConfig} config + * @param {import("../../types/kafkajs").ConsumerConstructorConfig} config * @returns {Consumer} */ consumer(config) { diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index b6bba73a..5e042fd2 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -7,7 +7,8 @@ const { kafkaJSToRdKafkaConfig, loggerTrampoline, severityToLogLevel, checkAllowedKeys, - CompatibilityErrorMessages } = require('./_common'); + CompatibilityErrorMessages, + logLevel } = require('./_common'); const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); @@ -32,16 +33,16 @@ const CompressionTypes = Object.freeze({ class Producer { /** - * kJSConfig is the kafkaJS config object. - * @type {import("../../types/kafkajs").ProducerConfig|null} + * The config supplied by the user. + * @type {import("../../types/kafkajs").ProducerConstructorConfig|null} */ - #kJSConfig = null; + #userConfig = 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} + * The config realized after processing any compatibility options. + * @type {import("../../types/config").ProducerGlobalConfig|null} */ - #rdKafkaConfig = null; + #internalConfig = null; /** * internalClient is the node-rdkafka client used by the API. @@ -78,20 +79,21 @@ class Producer { * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig */ constructor(kJSConfig) { - this.#kJSConfig = kJSConfig; + this.#userConfig = kJSConfig; } - async #config() { - if (!this.#rdKafkaConfig) - this.#rdKafkaConfig = await this.#finalizedConfig(); - return this.#rdKafkaConfig; + #config() { + if (!this.#internalConfig) + this.#internalConfig = this.#finalizedConfig(); + return this.#internalConfig; } - async #finalizedConfig() { - /* This sets the common configuration options for the client. */ - const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + #kafkaJSToProducerConfig(kjsConfig) { + if (!kjsConfig || Object.keys(kjsConfig).length === 0) { + return {}; + } - const allowedKeysSpecific = [ + const allowedKeysSpecificToProducer = [ 'createPartitioner', 'metadataMaxAge', 'allowAutoTopicCreation', @@ -105,59 +107,86 @@ class Producer { 'logger', ]; - checkAllowedKeys(allowedKeysSpecific, this.#kJSConfig); - - /* Create producer specific configuration. */ - globalConfig.dr_cb = 'true'; + checkAllowedKeys(allowedKeysSpecificToProducer, kjsConfig); + const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); - if (Object.hasOwn(this.#kJSConfig, 'createPartitioner')) { + /* Producer specific configuration. */ + if (Object.hasOwn(kjsConfig, 'createPartitioner')) { throw new error.KafkaJSError(CompatibilityErrorMessages.createPartitioner(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - globalConfig['partitioner'] = 'murmur2_random'; + rdKafkaConfig['partitioner'] = 'murmur2_random'; - if (Object.hasOwn(this.#kJSConfig, 'metadataMaxAge')) { - globalConfig['topic.metadata.refresh.interval.ms'] = this.#kJSConfig.metadataMaxAge; + if (Object.hasOwn(kjsConfig, 'metadataMaxAge')) { + rdKafkaConfig['topic.metadata.refresh.interval.ms'] = kjsConfig.metadataMaxAge; } - if (Object.hasOwn(this.#kJSConfig, 'allowAutoTopicCreation')) { - globalConfig['allow.auto.create.topics'] = this.#kJSConfig.allowAutoTopicCreation; + if (Object.hasOwn(kjsConfig, 'allowAutoTopicCreation')) { + rdKafkaConfig['allow.auto.create.topics'] = kjsConfig.allowAutoTopicCreation; } - if (Object.hasOwn(this.#kJSConfig, 'transactionTimeout')) { - globalConfig['transaction.timeout.ms'] = this.#kJSConfig.transactionTimeout; + if (Object.hasOwn(kjsConfig, 'transactionTimeout')) { + rdKafkaConfig['transaction.timeout.ms'] = kjsConfig.transactionTimeout; } - if (Object.hasOwn(this.#kJSConfig, 'idempotent')) { - globalConfig['enable.idempotence'] = this.#kJSConfig.idempotent; + if (Object.hasOwn(kjsConfig, 'idempotent')) { + rdKafkaConfig['enable.idempotence'] = kjsConfig.idempotent; } - if (Object.hasOwn(this.#kJSConfig, 'maxInFlightRequests')) { - globalConfig['max.in.flight'] = this.#kJSConfig.maxInFlightRequests; + if (Object.hasOwn(kjsConfig, 'maxInFlightRequests')) { + rdKafkaConfig['max.in.flight'] = kjsConfig.maxInFlightRequests; } - if (Object.hasOwn(this.#kJSConfig, 'transactionalId')) { - globalConfig['transactional.id'] = this.#kJSConfig.transactionalId; + if (Object.hasOwn(kjsConfig, 'transactionalId')) { + rdKafkaConfig['transactional.id'] = kjsConfig.transactionalId; } - if (Object.hasOwn(this.#kJSConfig, 'compression')) { - globalConfig['compression.codec'] = this.#kJSConfig.compression; + if (Object.hasOwn(kjsConfig, 'compression')) { + rdKafkaConfig['compression.codec'] = kjsConfig.compression; } - if (Object.hasOwn(this.#kJSConfig, 'acks')) { - globalConfig['acks'] = this.#kJSConfig.acks; + if (Object.hasOwn(kjsConfig, 'acks')) { + rdKafkaConfig['acks'] = kjsConfig.acks; } - if (Object.hasOwn(this.#kJSConfig, 'timeout')) { - globalConfig['request.timeout.ms'] = this.#kJSConfig.timeout; + if (Object.hasOwn(kjsConfig, 'timeout')) { + rdKafkaConfig['request.timeout.ms'] = kjsConfig.timeout; } /* Set the logger */ - if (Object.hasOwn(this.#kJSConfig, 'logger')) { - this.#logger = this.#kJSConfig.logger; + if (Object.hasOwn(kjsConfig, 'logger')) { + this.#logger = kjsConfig.logger; + } + + /* Set the log level - INFO for compatibility with kafkaJS, or DEBUG if that is turned + * on using the logLevel property. rdKafkaConfig.log_level is guaranteed to be set if we're + * here, and containing the correct value. */ + this.#logger.setLogLevel(severityToLogLevel[rdKafkaConfig.log_level]); + + return rdKafkaConfig; + } + + #finalizedConfig() { + /* Creates an rdkafka config based off the kafkaJS block. Switches to compatibility mode if the block exists. */ + let compatibleConfig = this.#kafkaJSToProducerConfig(this.#userConfig.kafkaJs); + + /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest + * log level, as librdkafka will control the granularity. */ + if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { + this.#logger.setLogLevel(logLevel.DEBUG); } - this.#logger.setLogLevel(severityToLogLevel[globalConfig.log_level]); - return { globalConfig, topicConfig }; + let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); + + /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ + delete rdKafkaConfig.kafkaJs; + + /* Certain properties that the user has set are overridden. There is + * no longer a delivery report, rather, results are made available on + * awaiting. */ + /* TODO: Add a warning if dr_cb is set? Or else, create a trampoline for it. */ + rdKafkaConfig.dr_cb = true; + + return rdKafkaConfig; } /** @@ -226,8 +255,8 @@ class Producer { throw new error.KafkaJSError(`Ready callback called in invalid state ${this.#state}`, { code: error.ErrorCodes.ERR__STATE }); } - const { globalConfig } = await this.#config(); - if (Object.hasOwn(globalConfig, 'transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { + const rdKafkaConfig = this.#config(); + if (Object.hasOwn(rdKafkaConfig, 'transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { this.#state = ProducerState.INITIALIZING_TRANSACTIONS; this.#internalClient.initTransactions(5000 /* default: 5s */, this.#readyTransactions.bind(this)); return; @@ -273,8 +302,9 @@ class Producer { this.#state = ProducerState.CONNECTING; - const { globalConfig, topicConfig } = await this.#config(); - this.#internalClient = new RdKafka.Producer(globalConfig, topicConfig); + const rdKafkaConfig = this.#config(); + + this.#internalClient = new RdKafka.Producer(rdKafkaConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); this.#internalClient.on('event.error', this.#errorCb.bind(this)); this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); @@ -434,7 +464,7 @@ class Producer { let consumerCreated = false; if (!consumer) { - const config = Object.assign({ groupId: consumerGroupId }, this.#kJSConfig); + const config = Object.assign({ 'group.id': consumerGroupId }, this.rdKafkaConfig); consumer = new Consumer(config); consumerCreated = true; await consumer.connect(); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 30697952..86565a6a 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -29,7 +29,10 @@ export interface KafkaConfig { reauthenticationThreshold?: number requestTimeout?: number enforceRequestTimeout?: boolean - rdKafka?: { topicConfig?: TopicConfig, globalConfig?: GlobalConfig }; +} + +export interface CommonConstructorConfig extends GlobalConfig { + kafkaJs?: KafkaConfig; } export interface ProducerConfig { @@ -42,6 +45,10 @@ export interface ProducerConfig { rdKafka?: { topicConfig?: ProducerTopicConfig, globalConfig?: ProducerGlobalConfig } } +export interface ProducerConstructorConfig extends ProducerGlobalConfig { + kafkaJs?: ProducerConfig; +} + export interface IHeaders { [key: string]: Buffer | string | (Buffer | string)[] | undefined } @@ -101,9 +108,9 @@ export type RecordMetadata = { } export class Kafka { - constructor(config: KafkaConfig) - producer(config?: ProducerConfig): Producer - consumer(config: ConsumerConfig): Consumer + constructor(config: CommonConstructorConfig) + producer(config?: ProducerConstructorConfig): Producer + consumer(config: ConsumerConstructorConfig): Consumer } type Sender = { @@ -142,6 +149,10 @@ export interface ConsumerConfig { rdKafka?: { topicConfig?: ConsumerTopicConfig, globalConfig?: ConsumerGlobalConfig } } +export interface ConsumerConstructorConfig extends ConsumerGlobalConfig { + kafkaJs?: ConsumerConfig; +} + export type ConsumerEvents = { HEARTBEAT: 'consumer.heartbeat' COMMIT_OFFSETS: 'consumer.commit_offsets' From 25eec759b068db642a93ae87c9a821fba95d407f Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 26 Dec 2023 12:09:28 +0530 Subject: [PATCH 037/224] Documentation changes for kakfaJs: block --- MIGRATION.md | 93 +++++++++++++---------- lib/kafkajs/_common.js | 159 +++++++++++++++++++++++++++++++-------- lib/kafkajs/_consumer.js | 29 ++----- lib/kafkajs/_kafka.js | 22 +++++- lib/kafkajs/_producer.js | 24 ++---- 5 files changed, 215 insertions(+), 112 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index fe3253c0..248542fe 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -3,13 +3,17 @@ ## KafkaJS -1. Change the import statement, from +1. Change the import statement, and add a `kafkaJs` block around your configs. ```javascript const { Kafka } = require('kafkajs'); + const kafka = new Kafka({ brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ }); + const producer = kafka.producer({ /* ... */, }); ``` to ```javascript const { Kafka } = require('confluent-kafka-js').KafkaJS; + const kafka = new Kafka({ kafkaJs: { brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ } }); + const producer = kafka.producer({ kafkaJs: { /* ... */, } }); ``` 2. Try running your program. In case a migration is needed, an informative error will be thrown. @@ -34,13 +38,15 @@ +const { Kafka } = require('confluent-kafka-js').KafkaJS; const kafka = new Kafka({ ++ kafkaJs: { clientId: 'my-app', brokers: ['kafka1:9092', 'kafka2:9092'] ++ } }) const producerRun = async () => { - const producer = kafka.producer(); -+ const producer = kafka.producer({ acks: 1 }); ++ const producer = kafka.producer({ kafkaJs: { acks: 1 } }); await producer.connect(); await producer.send({ topic: 'test-topic', @@ -55,7 +61,7 @@ const producerRun = async () => { const consumerRun = async () => { // Consuming - const consumer = kafka.consumer({ groupId: 'test-group' }); -+ const consumer = kafka.consumer({ groupId: 'test-group', fromBeginning: true }); ++ const consumer = kafka.consumer({ kafkaJs: { groupId: 'test-group', fromBeginning: true } }); await consumer.connect(); - await consumer.subscribe({ topic: 'test-topic', fromBeginning: true }); + await consumer.subscribe({ topic: 'test-topic' }); @@ -78,32 +84,32 @@ producerRun().then(consumerRun).catch(console.error); #### Configuration changes ```javascript - const kafka = new Kafka({/* common configuration changes */}); + const kafka = new Kafka({ kafkaJs: { /* common configuration changes */ } }); ``` Each allowed config property is discussed in the table below. If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. - | Property | Default Value | Comment | - |-------------------------------|--------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| - | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | - | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified using the rdKafka block.** | - | **sasl** | - | An optional object of the form `{ mechanism: 'plain' or 'scram-sha-256' or 'scram-sha-512', username: string, password: string }`. **Additional authentication types are not yet supported.** | - | clientId | "rdkafka" | An optional string used to identify the client. | - | **connectionTimeout** | 1000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | - | **authenticationTimeout** | 10000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | - | **reauthenticationThreshold** | **80% of connections.max.reauth.ms** | **No longer checked, the default is always used.** | - | requestTimeout | 30000 | number of milliseconds for a network request to timeout. | - | **enforceRequestTimeout** | true | When set to false, `requestTimeout` is set to 5 minutes. **This cannot be completely disabled.** | - | retry | object | Properties individually discussed below. | - | retry.maxRetryTime | 30000 | maximum time to backoff a retry, in milliseconds. | - | retry.initialRetryTime | 300 | minimum time to backoff a retry, in milliseconds | - | **retry.retries** | 5 | Total cap on the number of retries. **Applicable only to Produce requests.** | - | **retry.factor** | 0.2 | Randomization factor (jitter) for backoff. **Cannot be changed**. | - | **retry.multiplier** | 2 | Multiplier for exponential factor of backoff. **Cannot be changed.** | - | **retry.restartOnFailure** | true | Consumer only. **Cannot be changed**. Consumer will always make an attempt to restart. | - | logLevel | `logLevel.INFO` | Decides the severity level of the logger created by the underlying library. A logger created with the `INFO` level will not be able to log `DEBUG` messages later. | - | **socketFactory** | null | **No longer supported.** | - | **rdKafka** | {} | This is a **new** property. It can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | + | Property | Default Value | Comment | + |-------------------------------|--------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | + | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified outside the kafkaJs block.** | + | **sasl** | - | An optional object of the form `{ mechanism: 'plain' or 'scram-sha-256' or 'scram-sha-512', username: string, password: string }`. **Additional authentication types are not yet supported.** | + | clientId | "rdkafka" | An optional string used to identify the client. | + | **connectionTimeout** | 1000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | + | **authenticationTimeout** | 10000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | + | **reauthenticationThreshold** | **80% of connections.max.reauth.ms** | **No longer checked, the default is always used.** | + | requestTimeout | 30000 | number of milliseconds for a network request to timeout. | + | **enforceRequestTimeout** | true | When set to false, `requestTimeout` is set to 5 minutes. **This cannot be completely disabled.** | + | retry | object | Properties individually discussed below. | + | retry.maxRetryTime | 30000 | maximum time to backoff a retry, in milliseconds. | + | retry.initialRetryTime | 300 | minimum time to backoff a retry, in milliseconds | + | **retry.retries** | 5 | Total cap on the number of retries. **Applicable only to Produce requests.** | + | **retry.factor** | 0.2 | Randomization factor (jitter) for backoff. **Cannot be changed**. | + | **retry.multiplier** | 2 | Multiplier for exponential factor of backoff. **Cannot be changed.** | + | **retry.restartOnFailure** | true | Consumer only. **Cannot be changed**. Consumer will always make an attempt to restart. | + | logLevel | `logLevel.INFO` | Decides the severity level of the logger created by the underlying library. A logger created with the `INFO` level will not be able to log `DEBUG` messages later. | + | **socketFactory** | null | **No longer supported.** | + | outer config | {} | The configuration outside the kafkaJs block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | ### Producer @@ -111,7 +117,7 @@ producerRun().then(consumerRun).catch(console.error); #### Producer Configuration Changes ```javascript - const producer = kafka.producer({ /* producer-specific configuration changes. */}); + const producer = kafka.producer({ kafkaJs: { /* producer-specific configuration changes. */ } }); ``` Each allowed config property is discussed in the table below. @@ -119,7 +125,7 @@ producerRun().then(consumerRun).catch(console.error); | Property | Default Value | Comment | |-------------------------|------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| - | **createPartitioner** | DefaultPartioner (murmur2_random) - Java client compatible | Custom partitioner support is not yet provided. The default partitioner's behaviour is retained, and a number of partitioners are provided via the `rdKafka.partitioner` property. | + | **createPartitioner** | DefaultPartioner (murmur2_random) - Java client compatible | Custom partitioner support is not yet provided. The default partitioner's behaviour is retained, and a number of partitioners are provided via the `partitioner` property, which is specified outside the `kafkaJs` block. | | **retry** | object | Identical to `retry` in the common configuration. This takes precedence over the common config retry. | | metadataMaxAge | 5 minutes | Time in milliseconds after which to refresh metadata for known topics | | allowAutoTopicCreation | true | Determines if a topic should be created if it doesn't exist while producing. | @@ -130,7 +136,7 @@ producerRun().then(consumerRun).catch(console.error); | **acks** | -1 | The number of required acks before a Produce succeeds. **This is set on a per-producer level, not on a per `send` level**. -1 denotes it will wait for all brokers in the in-sync replica set. | | **compression** | CompressionTypes.NONE | Compression codec for Produce messages. **This is set on a per-producer level, not on a per `send` level**. It must be a key of the object CompressionType, namely GZIP, SNAPPY, LZ4, ZSTD or NONE. | | **timeout** | 30000 | The ack timeout of the producer request in milliseconds. This value is only enforced by the broker. **This is set on a per-producer level, not on a per `send` level**. | - | **rdKafka** | {} | This is a **new** property. It can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION. md). If set on the common config, the values are merged. | + | outer config | {} | The configuration outside the kafkaJs block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | @@ -158,11 +164,13 @@ producerRun().then(consumerRun).catch(console.error); After: ```javascript - const kafka = new Kafka({/* ... */}); + const kafka = new Kafka({ kafkaJs: { /* ... */ }}); const producer = kafka.producer({ - acks: 1, - compression: CompressionTypes.GZIP|CompressionTypes.SNAPPY|CompressionTypes.LZ4|CompressionTypes.ZSTD|CompressionTypes.NONE, - timeout: 30000, + kafkaJs: { + acks: 1, + compression: CompressionTypes.GZIP|CompressionTypes.SNAPPY|CompressionTypes.LZ4|CompressionTypes.ZSTD|CompressionTypes.NONE, + timeout: 30000, + } }); await producer.connect(); @@ -179,7 +187,7 @@ producerRun().then(consumerRun).catch(console.error); #### Consumer Configuration Changes ```javascript - const consumer = kafka.consumer({ /* producer-specific configuration changes. */}); + const consumer = kafka.consumer({ kafkaJs: { /* producer-specific configuration changes. */ } }); ``` Each allowed config property is discussed in the table below. If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. @@ -203,7 +211,7 @@ producerRun().then(consumerRun).catch(console.error); | **fromBeginning** | false | If there is initial offset in offset store or the desired offset is out of range, and this is true, we consume the earliest possible offset. **This is set on a per-consumer level, not on a per `subscribe` level**. | | **autoCommit** | true | Whether to periodically auto-commit offsets to the broker while consuming. **This is set on a per-consumer level, not on a per `run` level**. | | **autoCommitInterval** | 5000 | Offsets are committed periodically at this interval, if autoCommit is true. **This is set on a per-consumer level, not on a per `run` level. The default value is changed to 5 seconds.**. | - | **rdKafka** | {} | This is a **new** property. It can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). If set on the common config, the values are merged. | + | outer config | {} | The configuration outside the kafkaJs block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | #### Semantic and Per-Method Changes @@ -229,8 +237,10 @@ producerRun().then(consumerRun).catch(console.error); After: ```javascript const consumer = kafka.consumer({ - groupId: 'test-group', - fromBeginning: true, + kafkaJs: { + groupId: 'test-group', + fromBeginning: true, + } }); await consumer.connect(); await consumer.subscribe({ topics: ["topic"] }); @@ -240,6 +250,7 @@ producerRun().then(consumerRun).catch(console.error); - For auto-committing using a consumer, the properties `autoCommit` and `autoCommitInterval` on `run` are not set on a per-subscribe basis. Rather, they must be configured in the top-level configuration. `autoCommitThreshold` is not supported. + If `autoCommit` is set to true, messages are *not* committed per-message, but rather periodically at the interval specified by `autoCommitInterval` (default 5 seconds). Before: ```javascript @@ -255,11 +266,13 @@ producerRun().then(consumerRun).catch(console.error); ``` After: ```javascript - const kafka = new Kafka({ /* ... */ }); + const kafka = new Kafka({ kafkaJs: { /* ... */ } }); const consumer = kafka.consumer({ - /* ... */, - autoCommit: true, - autoCommitInterval: 5000, + kafkaJs: { + /* ... */, + autoCommit: true, + autoCommitInterval: 5000, + }, }); await consumer.connect(); await consumer.subscribe({ topics: ["topic"] }); diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index a364496c..798d9ff8 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,6 +1,61 @@ const error = require("./_error"); const process = require("process"); +/* A list of kafkaJs compatible properties that we process. + * All of these are not necessarily supported, and an error will be + * thrown if they aren't. */ +const kafkaJsProperties = { + common: [ + "brokers", + "clientId", + "sasl", + "ssl", + "requestTimeout", + "enforceRequestTimeout", + "connectionTimeout", + "authenticationTimeout", + "retry", + "socketFactory", + "reauthenticationThreshold", + "logLevel", + 'logger', + ], + producer: [ + 'createPartitioner', + 'metadataMaxAge', + 'allowAutoTopicCreation', + 'transactionTimeout', + 'idempotent', + 'maxInFlightRequests', + 'transactionalId', + 'compression', + 'acks', + 'timeout', + ], + consumer: [ + 'groupId', + 'partitionAssigners', + 'partitionAssignors', + 'sessionTimeout', + 'rebalanceTimeout', + 'heartbeatInterval', + 'metadataMaxAge', + 'allowAutoTopicCreation', + 'maxBytesPerPartition', + 'maxWaitTimeInMs', + 'minBytes', + 'maxBytes', + 'readUncommitted', + 'maxInFlightRequests', + 'rackId', + 'fromBeginning', + 'autoCommit', + 'autoCommitInterval', + 'autoCommitThreshold', + 'rebalanceListener', + ] +} + const logLevel = Object.freeze({ NOTHING: 0, ERROR: 1, @@ -92,19 +147,19 @@ function loggerTrampoline(msg, logger) { } function createReplacementErrorMessage(cOrP, fnCall, property, propertyVal, replacementVal, isLK = false) { - if (isLK) { - replacementVal = `rdKafka: { ${replacementVal}, ... }` + if (!isLK) { + replacementVal = `kafkaJs: { ${replacementVal}, ... }` } return `'${property}' is not supported as a property to '${fnCall}', but must be passed to the ${cOrP} during creation.\n` + `Before: \n` + - `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJs: { ... }, });\n` + + `\tconst ${cOrP} = kafka.${cOrP}({ ... });\n` + `\tawait ${cOrP}.connect();\n` + `\t${cOrP}.${fnCall}({ ${propertyVal}, ... });\n` + `After: \n` + - `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJs: { ${replacementVal}, ... }, });\n` + + `\tconst ${cOrP} = kafka.${cOrP}({ ${replacementVal}, ... });\n` + `\tawait ${cOrP}.connect();\n` + `\t${cOrP}.${fnCall}({ ... });\n` + - (isLK ? `For more details on what can be used inside the rdKafka block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n` : ''); + (isLK ? `For more details on what can be used outside the kafkaJs block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n` : ''); } const CompatibilityErrorMessages = Object.freeze({ @@ -119,14 +174,14 @@ const CompatibilityErrorMessages = Object.freeze({ saslUsernamePasswordString: (mechanism) => `The 'sasl.username' and 'sasl.password' properties must be strings and must be present for the mechanism ${mechanism}.`, sslObject: () => - "The 'ssl' property must be a boolean. Any additional configuration must be provided through the 'rdKafka' property.\n" + + "The 'ssl' property must be a boolean. Any additional configuration must be provided outside the kafkaJs block.\n" + "Before: \n" + - "\tconst kafka = new Kafka({ ssl: { rejectUnauthorized: false, ca: [...], key: ..., cert: ... }, }); \n" + + "\tconst kafka = new Kafka({ kafkaJs: { ssl: { rejectUnauthorized: false, ca: [ ... ], key: ..., cert: ... }, } }); \n" + "After: \n" + - "\tconst kafka = new Kafka({ ssl: true, rdKafka: { enable.ssl.certificate.verification: false, ssl.ca.location: ..., ssl.certificate.pem: ... } });\n" + - `For more details on what can be used inside the rdKafka block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n`, + '\tconst kafka = new Kafka({ kafkaJs: { ssl: true, }, "enable.ssl.certificate.verification": false, "ssl.ca.location": ..., "ssl.certificate.pem": ... });\n' + + `For more details on what can be used outside the kafkaJs block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n`, retryFactorMultiplier: () => - "The 'retry.factor' and 'retry.multiplier' are not supported. They are always set to the default of 0.2 and 2 respectively.", + + "The 'retry.factor' and 'retry.multiplier' are not supported. They are always set to the default of 0.2 and 2 respectively.", retryRestartOnFailure: () => "The restartOnFailure property is ignored. The client always retries on failure.", socketFactory: () => @@ -137,6 +192,22 @@ const CompatibilityErrorMessages = Object.freeze({ "Reauthentication threshold cannot be set, and reauthentication is automated when 80% of connections.max.reauth.ms is reached.", unsupportedKey: (key) => `The '${key}' property is not supported.`, + kafkaJsCommonKey: (key) => + `The '${key}' property seems to be a KafkaJS property in the main config block.` + + `It must be moved to the kafkaJS block.` + + `\nBefore: \n` + + `\tconst kafka = new Kafka({ ${key}: , ... });\n` + + `After: \n` + + `\tconst kafka = new Kafka({ kafkaJs: { ${key}: , ... }, ... });\n`, + kafkaJsClientKey: (key, cOrP) => + `The '${key}' property seems to be a KafkaJS property in the main config block. ` + + `It must be moved to the kafkaJS block.` + + `\nBefore: \n` + + `\tconst kafka = new Kafka({ ... });\n` + + `\tconst ${cOrP} = kafka.${cOrP}({ ${key}: , ... });\n` + + `After: \n` + + `\tconst kafka = new Kafka({ ... });\n` + + `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJs: { ${key}: , ... }, ... });\n`, /* Producer */ createPartitioner: () => @@ -333,33 +404,58 @@ function kafkaJSToRdKafkaConfig(config) { return rdkafkaConfig; } -function checkAllowedKeys(allowedKeysSpecific, config) { - const allowedKeysCommon = [ - "brokers", - "clientId", - "sasl", - "ssl", - "requestTimeout", - "enforceRequestTimeout", - "connectionTimeout", - "authenticationTimeout", - "retry", - "socketFactory", - "reauthenticationThreshold", - "logLevel", - "rdKafka", - 'logger', - ]; +/** + * Checks if the config object contains any keys not allowed by KafkaJS. + * @param {'producer'|'consumer'|'admin'} clientType + * @param {any} config + * @returns {string|null} the first unsupported key, or null if all keys are supported. + */ +function checkAllowedKeys(clientType, config) { + const allowedKeysCommon = kafkaJsProperties.common; + + if (!Object.hasOwn(kafkaJsProperties, clientType)) { + throw new error.KafkaJSError(`Unknown client type ${clientType}`, { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + const allowedKeysSpecific = kafkaJsProperties[clientType]; for (const key of Object.keys(config)) { if (!allowedKeysCommon.includes(key) && !allowedKeysSpecific.includes(key)) { - throw new error.KafkaJSError(CompatibilityErrorMessages.unsupportedKey(key), { - code: error.ErrorCodes.ERR__INVALID_ARG, - }); + return key; } } + + return null; } +/** + * Checks if the config object contains any keys specific to KafkaJS. + * @param {'producer'|'consumer'|'admin'|'common'} propertyType + * @param {any} config + * @returns {string|null} the first KafkaJS specific key, or null if none is present. + */ +function checkIfKafkaJsKeysPresent(propertyType, config) { + if (!Object.hasOwn(kafkaJsProperties, propertyType)) { + throw new error.KafkaJSError(`Unknown config type for ${propertyType}`, { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + const kjsKeys = kafkaJsProperties[propertyType]; + + for (const key of Object.keys(config)) { + /* We exclude 'acks' since it's common to both librdkafka and kafkaJs. + * We don't intend to keep up with new properties, so we don't need to really worry about making it extensible. */ + if (kjsKeys.includes(key) && key !== 'acks') { + return key; + } + } + + return null; +} + + + /** * Converts a topicPartitionOffset from KafkaJS to a format that can be used by node-rdkafka. * @param {import("../../types/kafkajs").TopicPartitionOffset} tpo @@ -455,5 +551,6 @@ module.exports = { createReplacementErrorMessage, CompatibilityErrorMessages, severityToLogLevel, - checkAllowedKeys + checkAllowedKeys, + checkIfKafkaJsKeysPresent }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 17aa9fb2..f128a18e 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -191,30 +191,11 @@ class Consumer { return {}; } - const allowedKeysSpecificToConsumer = [ - 'groupId', - 'partitionAssigners', - 'partitionAssignors', - 'sessionTimeout', - 'rebalanceTimeout', - 'heartbeatInterval', - 'metadataMaxAge', - 'allowAutoTopicCreation', - 'maxBytesPerPartition', - 'maxWaitTimeInMs', - 'minBytes', - 'maxBytes', - 'readUncommitted', - 'maxInFlightRequests', - 'rackId', - 'fromBeginning', - 'autoCommit', - 'autoCommitInterval', - 'autoCommitThreshold', - 'rebalanceListener', - ]; - - checkAllowedKeys(allowedKeysSpecificToConsumer, kjsConfig); + const disallowedKey = checkAllowedKeys('consumer', kjsConfig); + if (disallowedKey !== null) { + throw new error.KafkaJSError(CompatibilityErrorMessages.unsupportedKey(disallowedKey), + { code: error.ErrorCodes.ERR__INVALID_ARG }); + } const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); /* Consumer specific configuration */ diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index e9a1a006..09fe0efc 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -2,7 +2,7 @@ const { Producer, CompressionTypes } = require('./_producer'); const { Consumer, PartitionAssigners } = require('./_consumer'); const { Admin } = require('./_admin'); const error = require('./_error'); -const { logLevel } = require('./_common'); +const { logLevel, checkIfKafkaJsKeysPresent, CompatibilityErrorMessages } = require('./_common'); class Kafka { /* @type{import("../../types/kafkajs").CommonConstructorConfig} */ @@ -14,6 +14,11 @@ class Kafka { */ constructor(config) { this.#commonClientConfig = config ?? {}; + + const disallowedKey = checkIfKafkaJsKeysPresent('common', this.#commonClientConfig); + if (disallowedKey !== null) { + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsCommonKey(disallowedKey)); + } } /** @@ -43,6 +48,11 @@ class Kafka { * @returns {Producer} */ producer(config) { + const disallowedKey = checkIfKafkaJsKeysPresent('producer', config ?? {}); + if (disallowedKey !== null) { + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsClientKey(disallowedKey, 'producer')); + } + return new Producer(this.#mergeConfiguration(config)); } @@ -52,10 +62,20 @@ class Kafka { * @returns {Consumer} */ consumer(config) { + const disallowedKey = checkIfKafkaJsKeysPresent('consumer', config ?? {}); + if (disallowedKey !== null) { + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsClientKey(disallowedKey, 'consumer')); + } + return new Consumer(this.#mergeConfiguration(config)); } admin(config) { + const disallowedKey = checkIfKafkaJsKeysPresent('admin', config ?? {}); + if (disallowedKey !== null) { + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsClientKey(disallowedKey, 'admin')); + } + return new Admin(this.#mergeConfiguration(config)); } } diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 5e042fd2..3b43b90b 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -8,7 +8,8 @@ const { kafkaJSToRdKafkaConfig, severityToLogLevel, checkAllowedKeys, CompatibilityErrorMessages, - logLevel } = require('./_common'); + logLevel, +} = require('./_common'); const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); @@ -83,6 +84,7 @@ class Producer { } #config() { + this.#internalClient if (!this.#internalConfig) this.#internalConfig = this.#finalizedConfig(); return this.#internalConfig; @@ -93,21 +95,11 @@ class Producer { return {}; } - const allowedKeysSpecificToProducer = [ - 'createPartitioner', - 'metadataMaxAge', - 'allowAutoTopicCreation', - 'transactionTimeout', - 'idempotent', - 'maxInFlightRequests', - 'transactionalId', - 'compression', - 'acks', - 'timeout', - 'logger', - ]; - - checkAllowedKeys(allowedKeysSpecificToProducer, kjsConfig); + const disallowedKey = checkAllowedKeys('producer', kjsConfig); + if (disallowedKey) { + throw new error.KafkaJSError(CompatibilityErrorMessages.unsupportedKey(disallowedKey), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); /* Producer specific configuration. */ From 1b6d4bfbb0b7f128d8857bf3b8e7d41c320846f5 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 26 Dec 2023 12:53:26 +0530 Subject: [PATCH 038/224] Update examples for kafkaJs: block --- examples/kafkajs/consumer.js | 164 ++++++++++++++++++----------------- examples/kafkajs/eos.js | 32 +++---- examples/kafkajs/producer.js | 22 ++--- 3 files changed, 113 insertions(+), 105 deletions(-) diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 0774a7e0..9d24635c 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,16 +1,27 @@ -const { Kafka, logLevel } = require('../..').KafkaJS; +const { Kafka } = require('../..').KafkaJS //const { Kafka } = require('kafkajs') async function consumerStart() { - let consumer; - var stopped = false; + let consumer; + var stopped = false; - const kafka = new Kafka({ - brokers: ['localhost:9092'], - }); + const kafka = new Kafka({ + kafkaJs: { + brokers: ['localhost:9092'], + ssl: true, + connectionTimeout: 5000, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, + } + }); - consumer = kafka.consumer({ - groupId: 'test-group22' + Math.random(), + consumer = kafka.consumer({ + kafkaJs: { + groupId: 'test-group', + autoCommit: false, rebalanceListener: { onPartitionsAssigned: async (assignment) => { console.log(`Assigned partitions ${JSON.stringify(assignment)}`); @@ -24,95 +35,90 @@ async function consumerStart() { } } }, - rdKafka: { - globalConfig: { - 'debug': 'conf', - 'enable.auto.commit': false, - 'auto.offset.reset': 'error' - }, - topicConfig: { - }, - } - }); + }, + + /* Properties from librdkafka can also be used */ + 'auto.commit.interval.ms': 6000, + }); + + await consumer.connect(); + console.log("Connected successfully"); - await consumer.connect(); - console.log("Connected successfully"); + await consumer.subscribe({ + topics: [ + "topic2" + ] + }) - await consumer.subscribe({ - topics: [ - "test-topic" - ] - }) + // Batch consumer, commit and seek example + var batch = 0; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }) - // Batch consumer, commit and seek example - var batch = 0; - consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - console.log({ + if (++batch % 100 == 0) { + await consumer.seek({ topic, partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }) - - if (++batch % 100 == 0) { - await consumer.seek({ - topic, - partition, - offset: -2 - }); - await consumer.commitOffsets(); - batch = 0; - } - }, - }); + offset: -2 + }); + await consumer.commitOffsets(); + batch = 0; + } + }, + }); - // Pause/Resume example - const pauseResumeLoop = async () => { - let paused = false; - let ticks = 0; - while (!stopped) { - await new Promise((resolve) => setTimeout(resolve, 100)); - if (stopped) - break; + // Pause/Resume example + const pauseResumeLoop = async () => { + let paused = false; + let ticks = 0; + while (!stopped) { + await new Promise((resolve) => setTimeout(resolve, 100)); + if (stopped) + break; - ticks++; - if (ticks == 200) { - ticks = 0; - const assignment = consumer.assignment(); - if (paused) { - console.log(`Resuming partitions ${JSON.stringify(assignment)}`) - consumer.resume(assignment); - } else { - console.log(`Pausing partitions ${JSON.stringify(assignment)}`); - consumer.pause(assignment); - } - paused = !paused; + ticks++; + if (ticks == 200) { + ticks = 0; + const assignment = consumer.assignment(); + if (paused) { + console.log(`Resuming partitions ${JSON.stringify(assignment)}`) + consumer.resume(assignment); + } else { + console.log(`Pausing partitions ${JSON.stringify(assignment)}`); + consumer.pause(assignment); } + paused = !paused; } } + } - if (consumer.assignment) { - // KafkaJS doesn't have assignment() - pauseResumeLoop() - } + if (consumer.assignment()) { + // KafkaJS doesn't have assignment() + pauseResumeLoop() + } - // Disconnect example - const disconnect = () => { - process.off('SIGINT', disconnect); - process.off('SIGTERM', disconnect); - stopped = true; - consumer.commitOffsets() + // Disconnect example + const disconnect = () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + stopped = true; + consumer.commitOffsets() .finally(() => consumer.disconnect() ) .finally(() => console.log("Disconnected successfully") ); - } - process.on('SIGINT', disconnect); - process.on('SIGTERM', disconnect); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); } consumerStart() diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index d0aabcf3..0f85503c 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -3,26 +3,28 @@ const { Kafka } = require('../..').KafkaJS async function eosStart() { const kafka = new Kafka({ - brokers: [''], - ssl: true, - sasl: { - mechanism: 'plain', - username: '', - password: '', + kafkaJs: { + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', + } } }); const consumer = kafka.consumer({ - groupId: 'groupId', - rdKafka: { - globalConfig: { - "enable.auto.commit": false, - } - }, + kafkaJs: { + groupId: 'groupId', + autoCommit: false, + } }); const producer = kafka.producer({ - transactionalId: 'txid' + kafkaJs: { + transactionalId: 'txid' + } }); await consumer.connect(); @@ -66,9 +68,7 @@ async function eosStart() { { topic, partitions: [ - /* 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 }, + { partition, offset: message.offset }, ], } ], diff --git a/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index 57c1e6ac..b8d2e5ee 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -3,12 +3,14 @@ const { Kafka } = require('../..').KafkaJS async function producerStart() { const kafka = new Kafka({ - brokers: [''], - ssl: true, - sasl: { - mechanism: 'plain', - username: '', - password: '', + kafkaJs: { + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, } }); @@ -19,16 +21,16 @@ async function producerStart() { console.log("Connected successfully"); const res = [] - for(let i = 0; i < 50; i++) { + for (let i = 0; i < 50; i++) { res.push(producer.send({ topic: 'topic2', messages: [ - {value: 'v222', partition: 0}, - {value: 'v11', partition: 0, key: 'x'}, + { value: 'v222', partition: 0 }, + { value: 'v11', partition: 0, key: 'x' }, ] })); } - await Promise.allSettled(res); + await Promise.all(res); await producer.disconnect(); From 27245a7e37c70c20c2cac65d985e11e951eede2b Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 26 Dec 2023 12:56:16 +0530 Subject: [PATCH 039/224] Update Admin Client for config changes --- examples/kafkajs/admin.js | 4 ++- lib/kafkajs/_admin.js | 73 ++++++++++++++++++++++++++++----------- lib/kafkajs/_common.js | 3 +- lib/kafkajs/_producer.js | 1 - types/kafkajs.d.ts | 4 +++ 5 files changed, 62 insertions(+), 23 deletions(-) diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index 83c20acf..ebad8674 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -3,7 +3,9 @@ const { Kafka } = require('../..').KafkaJS async function adminStart() { const kafka = new Kafka({ - brokers: [''], + kafkaJs: { + brokers: ['localhost:9092'], + } }); const admin = kafka.admin(); diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 5e86ba75..8197110c 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -1,5 +1,8 @@ const RdKafka = require('../rdkafka'); -const { kafkaJSToRdKafkaConfig, createKafkaJsErrorFromLibRdKafkaError } = require('./_common'); +const { kafkaJSToRdKafkaConfig, + createKafkaJsErrorFromLibRdKafkaError, + DefaultLogger, + checkAllowedKeys } = require('./_common'); const error = require('./_error'); /** @@ -17,16 +20,16 @@ const AdminState = Object.freeze({ class Admin { /** - * kJSConfig is the merged kafkaJS config object. - * @type {import("../../types/kafkajs").AdminConfig & import("../../types/kafkajs").KafkaConfig} + * The config supplied by the user. + * @type {import("../../types/kafkajs").AdminConstructorConfig|null} */ - #kJSConfig = null; + #userConfig = null; /** - * rdKafkaConfig contains the config objects that will be passed to node-rdkafka. - * @type {{globalConfig: import("../../types/config").GlobalConfig}|null} + * The config realized after processing any compatibility options. + * @type {import("../../types/config").GlobalConfig|null} */ - #rdKafkaConfig = null; + #internalConfig = null; /** * internalClient is the node-rdkafka client used by the API. @@ -39,25 +42,55 @@ class Admin { */ #state = AdminState.INIT; + /** + * A logger for the admin client. + * @type {import("../../types/kafkajs").Logger} + */ + #logger = new DefaultLogger(); + /** * @constructor - * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig + * @param {import("../../types/kafkajs").AdminConstructorConfig} config */ - constructor(kJSConfig) { - this.#kJSConfig = kJSConfig; + constructor(config) { + this.#userConfig = config; } - async #config() { - if (!this.#rdKafkaConfig) - this.#rdKafkaConfig = await this.#finalizedConfig(); - return this.#rdKafkaConfig; + #config() { + if (!this.#internalConfig) + this.#internalConfig = this.#finalizedConfig(); + return this.#internalConfig; } - async #finalizedConfig() { - /* This sets the common configuration options for the client. */ - const { globalConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + #kafkaJSToAdminConfig(kjsConfig) { + if (!kjsConfig || Object.keys(kjsConfig).length === 0) { + return {}; + } + + const disallowedKey = checkAllowedKeys('admin', kjsConfig); + if (disallowedKey) { + throw new error.KafkaJSError(CompatibilityErrorMessages.unsupportedKey(disallowedKey), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); + return rdKafkaConfig; + } + + #finalizedConfig() { + let compatibleConfig = this.#kafkaJSToAdminConfig(this.#userConfig.kafkaJs); + + /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest + * log level, as librdkafka will control the granularity. */ + if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { + this.#logger.setLogLevel(logLevel.DEBUG); + } + + let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); + + /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ + delete rdKafkaConfig.kafkaJs; - return { globalConfig }; + return rdKafkaConfig; } /** @@ -71,12 +104,12 @@ class Admin { this.#state = AdminState.CONNECTING; - const { globalConfig } = await this.#config(); + const config = this.#config(); return new Promise((resolve, reject) => { try { /* AdminClient creation is a synchronous operation for node-rdkafka */ - this.#internalClient = RdKafka.AdminClient.create(globalConfig); + this.#internalClient = RdKafka.AdminClient.create(config); this.#state = AdminState.CONNECTED; resolve(); } catch (err) { diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 798d9ff8..3585f1d9 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -53,7 +53,8 @@ const kafkaJsProperties = { 'autoCommitInterval', 'autoCommitThreshold', 'rebalanceListener', - ] + ], + admin: [], } const logLevel = Object.freeze({ diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 3b43b90b..e3220201 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -84,7 +84,6 @@ class Producer { } #config() { - this.#internalClient if (!this.#internalConfig) this.#internalConfig = this.#finalizedConfig(); return this.#internalConfig; diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 86565a6a..d6438721 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -176,6 +176,10 @@ export interface AdminConfig { retry?: RetryOptions } +export interface AdminConstructorConfig extends GlobalConfig { + kafkaJs?: AdminConfig; +} + export interface ITopicConfig { topic: string numPartitions?: number From 94bec0acabf4be43d298ca4ee13f14c1cb20d258 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 26 Dec 2023 17:53:58 +0530 Subject: [PATCH 040/224] Add alpine builds for releases for Linux arm64 and amd64 --- .semaphore/build-docker.sh | 10 ++++ .semaphore/semaphore.yml | 100 ++++++++++++++++++++++++++++++++++++- deps/librdkafka | 2 +- 3 files changed, 110 insertions(+), 2 deletions(-) create mode 100755 .semaphore/build-docker.sh diff --git a/.semaphore/build-docker.sh b/.semaphore/build-docker.sh new file mode 100755 index 00000000..3f148428 --- /dev/null +++ b/.semaphore/build-docker.sh @@ -0,0 +1,10 @@ +#!/bin/sh +# This script is used to build the project within a docker image. +# The docker image is assumed to be an alpine docker image, for glibc based builds, we use +# the semaphhore agent directly. + +apk add -U ca-certificates openssl ncurses coreutils python3 make gcc g++ libgcc linux-headers grep util-linux binutils findutils perl patch musl-dev bash +# /v is the volume mount point for the project root +cd /v +npm install +npx node-pre-gyp package diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 31209b1d..1a70537f 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -35,6 +35,28 @@ global_job_config: blocks: + - name: "Linux amd64 (musl): Build and test" + dependencies: [ ] + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-2 + jobs: + - name: "Build from source and test for musl" + commands: + - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh + + - name: "Linux arm64 (musl): Build and test" + dependencies: [ ] + task: + agent: + machine: + type: s1-prod-ubuntu20-04-arm64-1 + jobs: + - name: "Build from source and test for musl" + commands: + - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh + - name: "Linux arm64: Build and test" dependencies: [ ] task: @@ -76,7 +98,6 @@ blocks: commands: - npx eslint lib/kafkajs - - name: "Linux amd64: Release" dependencies: [ ] run: @@ -164,6 +185,83 @@ blocks: - 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 musl: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-1 + env_vars: + - name: ARCHITECTURE + value: "x64" + - name: PLATFORM + value: "linux" + - name: LIBC + value: "musl" + jobs: + - name: "Release: LTS:18" + commands: + - export NODE_ABI=108 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh + - 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: + - export NODE_ABI=115 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:20-alpine /v/.semaphore/build-docker.sh + - 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: + - export NODE_ABI=120 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh + - 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 musl: 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: "musl" + jobs: + - name: "Release: LTS:18" + commands: + - export NODE_ABI=108 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh + - 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: + - export NODE_ABI=115 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:20-alpine /v/.semaphore/build-docker.sh + - 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: + - export NODE_ABI=120 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh + - 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: diff --git a/deps/librdkafka b/deps/librdkafka index 840ab459..e03d3bb9 160000 --- a/deps/librdkafka +++ b/deps/librdkafka @@ -1 +1 @@ -Subproject commit 840ab45990d29914fbd5aa3649ba1c102340f1f8 +Subproject commit e03d3bb91ed92a38f38d9806b8d8deffe78a1de5 From b1736ccb3657641f7c723098822ef76675cb9e25 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 27 Dec 2023 10:59:12 +0530 Subject: [PATCH 041/224] Fix producer tests after kafkaJs: block addition --- lib/kafkajs/_admin.js | 2 ++ lib/kafkajs/_producer.js | 7 ++++ .../producer/idempotentProduceMessage.spec.js | 6 +--- test/promisified/testhelpers.js | 36 ++++++++++--------- 4 files changed, 29 insertions(+), 22 deletions(-) diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 8197110c..f59e4ca8 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -2,6 +2,8 @@ const RdKafka = require('../rdkafka'); const { kafkaJSToRdKafkaConfig, createKafkaJsErrorFromLibRdKafkaError, DefaultLogger, + CompatibilityErrorMessages, + logLevel, checkAllowedKeys } = require('./_common'); const error = require('./_error'); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index e3220201..0588f656 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -117,6 +117,13 @@ class Producer { if (Object.hasOwn(kjsConfig, 'transactionTimeout')) { rdKafkaConfig['transaction.timeout.ms'] = kjsConfig.transactionTimeout; + } else { + rdKafkaConfig['transaction.timeout.ms'] = 60000; + } + + // `socket.timeout.ms` must be set <= `transaction.timeout.ms` + 100 + if (rdKafkaConfig['socket.timeout.ms'] > rdKafkaConfig['transaction.timeout.ms'] + 100) { + rdKafkaConfig['socket.timeout.ms'] = rdKafkaConfig['transaction.timeout.ms'] + 100; } if (Object.hasOwn(kjsConfig, 'idempotent')) { diff --git a/test/promisified/producer/idempotentProduceMessage.spec.js b/test/promisified/producer/idempotentProduceMessage.spec.js index 3ef6b355..00d0de18 100644 --- a/test/promisified/producer/idempotentProduceMessage.spec.js +++ b/test/promisified/producer/idempotentProduceMessage.spec.js @@ -29,11 +29,7 @@ describe('Producer > Idempotent producer', () => { consumer = createConsumer({ groupId: `consumer-group-id-${secureRandom()}`, maxWaitTimeInMs: 0, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - }, - } + fromBeginning: true, }) await createTopic({ topic: topicName, partitions: 1 }); await Promise.all([producer.connect(), consumer.connect()]); diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index 47bf18fc..f195a554 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -7,31 +7,33 @@ const clusterInformation = { }; function createConsumer(config) { - const kafka = new Kafka(Object.assign(config, clusterInformation)); + const kafka = new Kafka({ kafkaJs: Object.assign(config, clusterInformation) }); return kafka.consumer(); } function createProducer(config) { - const kafka = new Kafka(Object.assign(config, clusterInformation)); + const kafka = new Kafka({ kafkaJs: Object.assign(config, clusterInformation) }); return kafka.producer(); } function createAdmin(config) { - const kafka = new Kafka(Object.assign(config, clusterInformation)); + const kafka = new Kafka({ kafkaJs: Object.assign(config, clusterInformation) }); return kafka.admin(); } function secureRandom(length = 10) { - return `${crypto.randomBytes(length).toString('hex')}-${process.pid}-${crypto.randomUUID()}`; + return `${crypto.randomBytes(length).toString('hex')}-${process.pid}-${crypto.randomUUID()}`; } async function createTopic(args) { const { topic, partitions } = args; const admin = createAdmin({}); await admin.connect(); - await admin.createTopics({ topics: [ - { topic, numPartitions: partitions ?? 1 } - ] }); + await admin.createTopics({ + topics: [ + { topic, numPartitions: partitions ?? 1 } + ] + }); await admin.disconnect(); } @@ -40,7 +42,7 @@ async function waitForConsumerToJoinGroup(consumer) { return new Promise(resolve => setTimeout(resolve, 2500)); } -async function waitFor(check, resolveValue, { delay = 50 } = {}) { +async function waitFor(check, resolveValue, { delay = 50 } = {}) { return new Promise(resolve => { const interval = setInterval(() => { if (check()) { @@ -64,15 +66,15 @@ const generateMessages = 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}`, - } - }) - } + .fill() + .map((v, i) => { + const value = secureRandom() + return { + key: `key${prefixOrEmpty}-${i}-${value}`, + value: `value${prefixOrEmpty}-${i}-${value}`, + } + }) +} module.exports = { createConsumer, From a825fc0bb989d6e1d688b23766dbf4a962121411 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 27 Dec 2023 11:41:17 +0530 Subject: [PATCH 042/224] Fix consumer tests after kafkaJs: block addition --- .../consumer/consumeMessages.spec.js | 95 +++----------- test/promisified/consumer/pause.spec.js | 121 +++++++++--------- test/promisified/consumer/seek.spec.js | 44 +++---- test/promisified/consumer/subscribe.spec.js | 12 +- 4 files changed, 94 insertions(+), 178 deletions(-) diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index de3ebb88..ebd59dd6 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -1,6 +1,6 @@ jest.setTimeout(30000) -const { ErrorCodes } = require('../../../lib').KafkaJS; +const { ErrorCodes, CompressionTypes } = require('../../../lib').KafkaJS; const { secureRandom, @@ -28,11 +28,7 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, }); }); @@ -135,11 +131,7 @@ describe('Consumer', () => { it('consume GZIP messages', async () => { /* Discard and recreate producer with the compression set */ producer = createProducer({ - rdKafka: { - globalConfig: { - 'compression.codec': 'gzip', - } - } + compression: CompressionTypes.GZIP, }); await consumer.connect(); @@ -274,11 +266,7 @@ describe('Consumer', () => { groupId, minBytes: 1024, maxWaitTimeInMs: 500, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, }); const messages = Array(10) @@ -317,11 +305,7 @@ describe('Consumer', () => { // make sure we fetch a batch of messages minBytes: 1024, maxWaitTimeInMs: 500, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, }) const messages = Array(10) @@ -469,11 +453,7 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, }); await consumer.connect(); @@ -510,14 +490,7 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - globalConfig: { - // debug: 'cgrp,topic', - }, - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true }); await consumer.connect(); @@ -573,11 +546,7 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, }); await consumer.connect(); @@ -634,11 +603,7 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 100, readUncommitted: true, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, }) await consumer.connect(); @@ -699,14 +664,8 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - globalConfig: { - 'enable.auto.commit': false, - }, - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, + autoCommit: false, }); await consumer.connect(); @@ -763,14 +722,8 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - globalConfig: { - 'enable.auto.commit': false, - }, - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, + autoCommit: false, }); await consumer.connect(); @@ -819,14 +772,8 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - }, - globalConfig: { - 'enable.auto.commit': false, - }, - } + fromBeginning: true, + autoCommit: false, }); await consumer.connect(); @@ -861,14 +808,8 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - rdKafka: { - globalConfig: { - 'enable.auto.commit': false, - }, - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true, + autoCommit: false, }); await consumer.connect(); @@ -897,7 +838,7 @@ describe('Consumer', () => { uncommittedOffsetsPerMessage = [] consumer.run({ - eachMessage + eachMessage }) await waitForMessages(messagesConsumed, { number: 1 }); diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index b563ce0b..ee94ce81 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -29,14 +29,7 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 1, maxBytesPerPartition: 180, - rdKafka: { - globalConfig: { - // 'debug': 'fetch', - }, - topicConfig: { - 'auto.offset.reset': 'earliest', - }, - } + fromBeginning: true, }); }) @@ -196,63 +189,63 @@ describe('Consumer', () => { /* 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) - ) + 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 } + }) - 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 }) + 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 () => { diff --git a/test/promisified/consumer/seek.spec.js b/test/promisified/consumer/seek.spec.js index 50001c6e..f3ba6a47 100644 --- a/test/promisified/consumer/seek.spec.js +++ b/test/promisified/consumer/seek.spec.js @@ -1,3 +1,5 @@ +jest.setTimeout(30000); + const { createConsumer, createProducer, @@ -17,11 +19,7 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - }, - } + fromBeginning: true, }); }); @@ -81,21 +79,15 @@ describe('Consumer', () => { 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', - }, - } + fromBeginning: true, + autoCommit: false, }); }); @@ -129,14 +121,8 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, - rdKafka: { - globalConfig: { - 'enable.auto.commit': false, - }, - topicConfig: { - 'auto.offset.reset': 'earliest', - }, - } + fromBeginning: true, + autoCommit: false, }); await consumer.connect(); await consumer.subscribe({ topic: topicName }); @@ -163,7 +149,7 @@ describe('Consumer', () => { message: expect.objectContaining({ offset: '2' }), }), ]); - }, 10000); + }); }); }); @@ -232,7 +218,7 @@ describe('Consumer', () => { ]) ); - }, 10000); + }); it('works for both partitions', async () => { await consumer.connect(); @@ -301,18 +287,18 @@ describe('Consumer', () => { ]) ); - }, 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 message1 = { key: `key-0`, value: `value-${value1}`, partition: 0 } const value2 = secureRandom() - const message2 = { key: `key-0`, value: `value-${value2}` } + const message2 = { key: `key-0`, value: `value-${value2}`, partition: 0 } const value3 = secureRandom() - const message3 = { key: `key-0`, value: `value-${value3}` } + const message3 = { key: `key-0`, value: `value-${value3}`, partition: 0 } await producer.send({ topic: topicName, messages: [message1, message2, message3] }) await consumer.subscribe({ topic: topicName, }) @@ -353,7 +339,7 @@ describe('Consumer', () => { }), ]) ); - }, 10000); + }); }); }); }) diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index 67dc2d80..c6005910 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -18,11 +18,7 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 1, maxBytesPerPartition: 180, - rdKafka: { - topicConfig: { - 'auto.offset.reset': 'earliest', - } - } + fromBeginning: true }); producer = createProducer({}); @@ -59,17 +55,17 @@ describe('Consumer', () => { await waitForConsumerToJoinGroup(consumer); await producer.connect(); - let records = await producer.sendBatch({ + 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' }] }, + { topic: topics[2], messages: [{ key: 'tea', value: 'tea' }] }, ], }); await waitForMessages(messagesConsumed, { number: 3 }); expect(messagesConsumed.map(m => m.message.value.toString())).toEqual( - expect.arrayContaining(['drink', 'your', 'ovaltine']) + expect.arrayContaining(['drink', 'your', 'tea']) ); }); }) From 4a91af2788bfe44bf789163e88f54c34f742ac04 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 27 Dec 2023 12:12:23 +0530 Subject: [PATCH 043/224] Add eslint changes --- Makefile | 6 +++++- test/promisified/testhelpers.js | 10 ++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/Makefile b/Makefile index 3e1865a8..fb836e7b 100644 --- a/Makefile +++ b/Makefile @@ -26,6 +26,7 @@ CONFIG_OUTPUTS = \ CPPLINT_FILES = $(wildcard src/*.cc src/*.h) CPPLINT_FILTER = -legal/copyright JSLINT_FILES = lib/*.js test/*.js e2e/*.js lib/kafkajs/*.js +ESLINT_FILES = lib/kafkajs/*.js test/promisified/*.js PACKAGE = $(shell node -pe 'require("./package.json").name.split("/")[1]') VERSION = $(shell node -pe 'require("./package.json").version') @@ -39,7 +40,7 @@ endif all: lint lib test e2e -lint: cpplint jslint +lint: cpplint jslint eslint cpplint: @$(PYTHON) $(CPPLINT) --filter=$(CPPLINT_FILTER) $(CPPLINT_FILES) @@ -47,6 +48,9 @@ cpplint: jslint: node_modules/.dirstamp @./node_modules/.bin/jshint --verbose $(JSLINT_FILES) +eslint: node_modules/.dirstamp + @./node_modules/.bin/eslint $(ESLINT_FILES) + lib: node_modules/.dirstamp $(CONFIG_OUTPUTS) @PYTHONHTTPSVERIFY=0 $(NODE-GYP) build $(GYPBUILDARGS) diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index f195a554..9de7895a 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -1,5 +1,6 @@ -const crypto = require('crypto') -const { Kafka, ErrorCodes } = require('../../lib').KafkaJS; +const crypto = require('crypto'); +const process = require('process'); +const { Kafka } = require('../../lib').KafkaJS; // TODO: pick this up from a file const clusterInformation = { @@ -37,8 +38,10 @@ async function createTopic(args) { await admin.disconnect(); } -async function waitForConsumerToJoinGroup(consumer) { +async function waitForConsumerToJoinGroup(/* consumer is passed as the first argument, and ignored */) { // We don't yet have a deterministic way to test this, so we just wait for a bit. + // TODO: we can probably wait for consumer.assignment() to be not empty, but that only + // works if the assignment exists. return new Promise(resolve => setTimeout(resolve, 2500)); } @@ -82,7 +85,6 @@ module.exports = { createAdmin, secureRandom, waitForMessages, - waitForMessages, createTopic, waitForConsumerToJoinGroup, waitFor, From 32681e314b7d1ea7d5d379370a68741df4c07f95 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 27 Dec 2023 12:29:41 +0530 Subject: [PATCH 044/224] Separate promisified tests as they're all e2e --- Makefile | 2 ++ test/promisified/testhelpers.js | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index fb836e7b..3d12a428 100644 --- a/Makefile +++ b/Makefile @@ -63,6 +63,8 @@ $(CONFIG_OUTPUTS): node_modules/.dirstamp binding.gyp test: node_modules/.dirstamp @./node_modules/.bin/mocha --ui exports $(TEST_REPORTER) $(TESTS) $(TEST_OUTPUT) + +promisified_test: node_modules/.dirstamp @./node_modules/.bin/jest --ci --runInBand $(PROMISIFIED_TESTS) check: node_modules/.dirstamp diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index 9de7895a..a0b61f06 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -4,7 +4,7 @@ const { Kafka } = require('../../lib').KafkaJS; // TODO: pick this up from a file const clusterInformation = { - brokers: ['localhost:9092'], + brokers: process.env.KAFKA_HOST ? process.env.KAFKA_HOST.split(',') : ['localhost:9092'], }; function createConsumer(config) { From 962defb91b33e3f1ad0a3e7c9f33537c9dcf5cb3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 28 Dec 2023 10:53:26 +0530 Subject: [PATCH 045/224] Fix consumeMessages test and add TEST_DEBUG env var --- lib/kafkajs/_consumer.js | 5 +++ .../consumer/consumeMessages.spec.js | 38 +++++++++---------- test/promisified/testhelpers.js | 19 ++++++++-- 3 files changed, 39 insertions(+), 23 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index f128a18e..d4df7e50 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -322,6 +322,11 @@ class Consumer { this.#logger.setLogLevel(logLevel.DEBUG); } + /* Even if we are in compability mode, setting a 'debug' in the main config must override the logger's level. */ + if (Object.hasOwn(this.#userConfig, 'debug')) { + this.#logger.setLogLevel(logLevel.DEBUG); + } + let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index ebd59dd6..72fac333 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -642,7 +642,6 @@ describe('Consumer', () => { 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, @@ -691,7 +690,7 @@ describe('Consumer', () => { eachMessage, }) - // Consume pre-produced messages. + // 2. Consume pre-produced messages. const number = messages.length; await waitForMessages(messagesConsumed, { @@ -729,8 +728,8 @@ describe('Consumer', () => { await consumer.connect(); await consumer.subscribe({ topic: topicName }); - messagesConsumed = [] - uncommittedOffsetsPerMessage = [] + messagesConsumed = []; + uncommittedOffsetsPerMessage = []; consumer.run({ eachMessage }) @@ -802,19 +801,6 @@ describe('Consumer', () => { // 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, - fromBeginning: true, - autoCommit: false, - }); - - 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); @@ -833,13 +819,25 @@ describe('Consumer', () => { }); await txnToAbort.abort() - // Restart consumer + /* Restart consumer - we cannot stop it, so we recreate it. */ messagesConsumed = [] uncommittedOffsetsPerMessage = [] + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: false, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ - eachMessage - }) + eachMessage, + }); await waitForMessages(messagesConsumed, { number: 1 }); expect(messagesConsumed[0].value.toString()).toMatch(/value-0/) diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index a0b61f06..dc11e2b5 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -1,5 +1,6 @@ const crypto = require('crypto'); const process = require('process'); +const { logLevel } = require('../../lib/kafkajs'); const { Kafka } = require('../../lib').KafkaJS; // TODO: pick this up from a file @@ -7,18 +8,30 @@ const clusterInformation = { brokers: process.env.KAFKA_HOST ? process.env.KAFKA_HOST.split(',') : ['localhost:9092'], }; +const debug = process.env.TEST_DEBUG; + +function makeConfig(config) { + const kafkaJs = Object.assign(config, clusterInformation); + const common = {}; + if (debug) { + common['debug'] = debug; + } + + return Object.assign(common, { kafkaJs }); +} + function createConsumer(config) { - const kafka = new Kafka({ kafkaJs: Object.assign(config, clusterInformation) }); + const kafka = new Kafka(makeConfig(config)); return kafka.consumer(); } function createProducer(config) { - const kafka = new Kafka({ kafkaJs: Object.assign(config, clusterInformation) }); + const kafka = new Kafka(makeConfig(config)); return kafka.producer(); } function createAdmin(config) { - const kafka = new Kafka({ kafkaJs: Object.assign(config, clusterInformation) }); + const kafka = new Kafka(makeConfig(config)); return kafka.admin(); } From b7fd389a552e7a3ccdb8c1536917517a7fbc5c69 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Jan 2024 10:42:38 +0530 Subject: [PATCH 046/224] Add async lock for consume/disconnect --- LICENSE.kafkajs | 1 + lib/kafkajs/_common.js | 81 ++++++++++++++++++- lib/kafkajs/_consumer.js | 58 ++++++------- lib/kafkajs/_error.js | 8 ++ .../consumer/consumeMessages.spec.js | 38 +++++++++ test/promisified/testhelpers.js | 1 - 6 files changed, 156 insertions(+), 31 deletions(-) diff --git a/LICENSE.kafkajs b/LICENSE.kafkajs index 3c57956e..d303b06e 100644 --- a/LICENSE.kafkajs +++ b/LICENSE.kafkajs @@ -1,5 +1,6 @@ The promisified API (lib/kafkajs) is inspired by kafkajs (github.com/tulios/kafkajs). The promisified tests (test/promisified) are also adapted from there. +An async lock implementation and many error types are also adapted from there. The license notice is reproduced below. ---- diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 3585f1d9..526b04e4 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -540,6 +540,83 @@ function notImplemented(msg = 'Not implemented') { throw new error.KafkaJSError(msg, { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } +/* Code from the async lock is from github.com/tulios/kafkajs. + * For more details, see LICENSE.kafkajs */ +const LockStates = Object.freeze({ + LOCKED: 'locked', + TIMEOUT: 'timeout', + WAITING: 'waiting', + TIMEOUT_ERROR_MESSAGE: 'timeoutErrorMessage', +}); + +class Lock { + constructor({ timeout, description = null } = {}) { + if (typeof timeout !== 'number') { + throw new TypeError(`'timeout' is not a number, received '${typeof timeout}'`); + } + + this[LockStates.LOCKED] = false; + this[LockStates.TIMEOUT] = timeout; + this[LockStates.WAITING] = new Set(); + this[LockStates.TIMEOUT_ERROR_MESSAGE] = () => { + const timeoutMessage = `Timeout while acquiring lock (${this[LockStates.WAITING].size} waiting locks)`; + return description ? `${timeoutMessage}: "${description}"` : timeoutMessage; + } + } + + async acquire() { + return new Promise((resolve, reject) => { + if (!this[LockStates.LOCKED]) { + this[LockStates.LOCKED] = true; + return resolve(); + } + + let timeoutId = null; + const tryToAcquire = async () => { + if (!this[LockStates.LOCKED]) { + this[LockStates.LOCKED] = true; + clearTimeout(timeoutId); + this[LockStates.WAITING].delete(tryToAcquire); + return resolve(); + } + } + + this[LockStates.WAITING].add(tryToAcquire); + timeoutId = setTimeout(() => { + // The message should contain the number of waiters _including_ this one + const e = new error.KafkaJSLockTimeout(this[LockStates.TIMEOUT_ERROR_MESSAGE]()); + this[LockStates.WAITING].delete(tryToAcquire); + reject(e); + }, this[LockStates.TIMEOUT]); + }) + } + + async release() { + this[LockStates.LOCKED] = false; + const waitingLock = this[LockStates.WAITING].values().next().value; + + if (waitingLock) { + return waitingLock(); + } + } +} + +/** + * Acquires a lock, or logs an error if it fails. + * @param {Lock} lock + * @param {import("../../types/kafkajs").Logger} logger + * @returns {boolean} true if the lock was acquired, false otherwise. + */ +async function acquireOrLog(lock, logger) { + try { + await lock.acquire(); + return true; + } catch (e) { + logger.error(`Failed to acquire lock: ${e.message}`); + } + return false; +} + module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, @@ -553,5 +630,7 @@ module.exports = { CompatibilityErrorMessages, severityToLogLevel, checkAllowedKeys, - checkIfKafkaJsKeysPresent + checkIfKafkaJsKeysPresent, + Lock, + acquireOrLog, }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index d4df7e50..67187ba3 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -11,7 +11,9 @@ const { CompatibilityErrorMessages, severityToLogLevel, checkAllowedKeys, - logLevel + logLevel, + Lock, + acquireOrLog } = require('./_common'); const { Buffer } = require('buffer'); @@ -91,6 +93,15 @@ class Consumer { */ #logger = new DefaultLogger(); + /** + * A lock for consuming and disconnecting. + * This lock should be held whenever we want to change the state from CONNECTED to any state other than CONNECTED. + * In practical terms, this lock is held whenever we're consuming a message, or disconnecting. + * We set the timeout to 5 seconds, after which we log an error, but keep trying to acquire the lock. + * @type {Lock} + */ + #lock = new Lock({ timeout: 5000 }); + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -575,6 +586,12 @@ class Consumer { /* Internal polling loop. It accepts the same config object that `run` accepts. */ async #runInternal(config) { while (this.#state === ConsumerState.CONNECTED) { + + /* We need to acquire a lock here, because we need to ensure that we don't + * disconnect while in the middle of processing a message. */ + if (!(await acquireOrLog(this.#lock, this.#logger))) + continue; + const m = await this.#consumeSingle().catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ @@ -582,14 +599,8 @@ class Consumer { this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${JSON.stringify(e)}`); }); - /* 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) { + this.#lock.release(); continue; } @@ -608,27 +619,15 @@ class Consumer { 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. */ + this.#lock.release(); 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 @@ -645,7 +644,7 @@ class Consumer { /* 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 && this.#state === ConsumerState.CONNECTED) + if (this.#checkPendingSeeks) await this.#seekInternal(); /* TODO: another check we need to do here is to see how kafkaJS is handling @@ -654,13 +653,8 @@ class Consumer { * 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)); + /* Release the lock so that any pending disconnect can go through. */ + await this.#lock.release(); } } @@ -952,7 +946,13 @@ class Consumer { return; } + while (!(await acquireOrLog(this.#lock, this.#logger))); /* Just retry... */ + this.#state = ConsumerState.DISCONNECTING; + + /* Since there are state-checks before everything, we are safe to proceed without the lock. */ + await this.#lock.release(); + await new Promise((resolve, reject) => { const cb = (err) => { if (err) { diff --git a/lib/kafkajs/_error.js b/lib/kafkajs/_error.js index fc1885af..49a0f9b3 100644 --- a/lib/kafkajs/_error.js +++ b/lib/kafkajs/_error.js @@ -130,6 +130,13 @@ class KafkaJSTimeout extends KafkaJSError { } } +class KafkaJSLockTimeout extends KafkaJSTimeout { + constructor() { + super(...arguments) + this.name = 'KafkaJSLockTimeout' + } +} + /** * @typedef {Object} KafkaJSAggregateError represents an error raised when multiple errors occur at once. */ @@ -179,6 +186,7 @@ module.exports = { KafkaJSGroupCoordinatorNotFound, KafkaJSNotImplemented, KafkaJSTimeout, + KafkaJSLockTimeout, KafkaJSAggregateError, KafkaJSNoBrokerAvailableError, isRebalancing, diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 72fac333..4b4b117b 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -2,6 +2,7 @@ jest.setTimeout(30000) const { ErrorCodes, CompressionTypes } = require('../../../lib').KafkaJS; +const { doesNotMatch } = require('assert'); const { secureRandom, createTopic, @@ -443,6 +444,43 @@ describe('Consumer', () => { expect(offsetsConsumed.length).toEqual(messages.length) }); + it('does not disconnect in the middle of message processing', async () => { + await producer.connect(); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + let calls = 0; + let failedSeek = false; + consumer.run({ + eachMessage: async ({ message }) => { + /* Take a long time to process the message. */ + await sleep(7000); + try { + consumer.seek({ topic: topicName, partition: 0, offset: message.offset }); + } catch (e) { + failedSeek = true; + } + calls++; + } + }); + + await producer.send({ + topic: topicName, + messages: [{ key: '1', value: '1' }], + }); + + /* Waiting for assignment and then a bit more means that the first eachMessage starts running. */ + await waitFor(() => consumer.assignment().length > 0, () => { }, { delay: 50 }); + await sleep(200); + await consumer.disconnect(); + + /* Even without explicitly waiting for it, a pending call to eachMessage must complete before disconnect does. */ + expect(calls).toEqual(1); + expect(failedSeek).toEqual(false); + + await producer.disconnect(); + }); + describe('transactions', () => { it('accepts messages from an idempotent producer', async () => { producer = createProducer({ diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index dc11e2b5..d8a49015 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -1,6 +1,5 @@ const crypto = require('crypto'); const process = require('process'); -const { logLevel } = require('../../lib/kafkajs'); const { Kafka } = require('../../lib').KafkaJS; // TODO: pick this up from a file From 2dd25d82168637daab9b7330ddc6b92bc5b219b4 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Jan 2024 12:19:08 +0530 Subject: [PATCH 047/224] Add quickstart --- QUICKSTART.md | 62 +++++++++++++++++++++++++++++++++++++++++++++++++++ README.md | 6 ++--- 2 files changed, 65 insertions(+), 3 deletions(-) create mode 100644 QUICKSTART.md diff --git a/QUICKSTART.md b/QUICKSTART.md new file mode 100644 index 00000000..5eed5916 --- /dev/null +++ b/QUICKSTART.md @@ -0,0 +1,62 @@ +# Basic Producer Example + +```javascript +const { Kafka } = require('../..').KafkaJS + +async function producerStart() { + const producer = new Kafka().producer({ + 'bootstrap.servers': '', + }); + + await producer.connect(); + + const deliveryReports = await producer.send({ + topic: 'topic2', + messages: [ + { value: 'v222', partition: 0 }, + { value: 'v11', partition: 0, key: 'x' }, + ] + }); + + await producer.disconnect(); +} + +producerStart(); +``` + +# Basic Consumer Example + +```javascript +const { Kafka } = require('../..').KafkaJS + +async function consumerStart() { + const consumer = new Kafka().consumer({ + 'bootstrap.servers': '', + 'group.id': 'test', + 'auto.offset.reset': 'earliest', + }); + + await consumer.connect(); + + await consumer.subscribe({ topics: [ "topic" ] }); + + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); + }, + }); + + // When done consuming + // await consumer.disconnect(); +} + +consumerStart(); +``` + +See the examples in the [examples](examples) directory for more in-depth examples. \ No newline at end of file diff --git a/README.md b/README.md index 50edbb5e..2cbc7f3b 100644 --- a/README.md +++ b/README.md @@ -36,8 +36,8 @@ __This library currently uses `librdkafka` based off of the master branch.__ The following configurations are supported for this early access preview: * Any supported version of Node.js (The two LTS versions, 18 and 20, and the latest version, 21). -* Linux (x64 and arm64) - only glibc, not musl/alpine. -* macOS - arm64/m1 only +* Linux (x64 and arm64) - both glibc and musl/alpine. +* macOS - arm64/m1. Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: @@ -48,7 +48,7 @@ $ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1 # Getting Started -1. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md) (TBA). +1. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md). 2. If you're migrating from `kafkajs`, you can use the [migration guide](MIGRATION.md#kafkajs). 2. If you're migrating from `node-rdkafka`, you can use the [migration guide](MIGRATION.md#node-rdkafka). From 4c7be5c9f74f49fa688bb037f92808db6b41ab72 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Jan 2024 17:55:43 +0530 Subject: [PATCH 048/224] Add examples for fresh users of library --- README.md | 36 +++---- examples/consumer.js | 97 +++++++++++++++++++ examples/eos.js | 78 +++++++++++++++ examples/{ => node-rdkafka}/consumer-flow.md | 0 examples/{ => node-rdkafka}/consumer.md | 0 examples/{ => node-rdkafka}/docker-alpine.md | 0 .../{ => node-rdkafka}/high-level-producer.md | 0 examples/{ => node-rdkafka}/metadata.md | 0 .../{ => node-rdkafka}/producer-cluster.md | 0 examples/{ => node-rdkafka}/producer.md | 0 examples/producer.js | 37 +++++++ 11 files changed, 231 insertions(+), 17 deletions(-) create mode 100644 examples/consumer.js create mode 100644 examples/eos.js rename examples/{ => node-rdkafka}/consumer-flow.md (100%) rename examples/{ => node-rdkafka}/consumer.md (100%) rename examples/{ => node-rdkafka}/docker-alpine.md (100%) rename examples/{ => node-rdkafka}/high-level-producer.md (100%) rename examples/{ => node-rdkafka}/metadata.md (100%) rename examples/{ => node-rdkafka}/producer-cluster.md (100%) rename examples/{ => node-rdkafka}/producer.md (100%) create mode 100644 examples/producer.js diff --git a/README.md b/README.md index 2cbc7f3b..4a40e44c 100644 --- a/README.md +++ b/README.md @@ -46,6 +46,8 @@ from GitHub: $ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.4-devel" ``` +Yarn and pnpm support is experimental. + # Getting Started 1. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md). @@ -56,23 +58,7 @@ $ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1 ## Contributing Bug reports and early feedback is appreciated in the form of Github Issues. -For guidelines on contributing please see [CONTRIBUTING.md](https://github.com/confluentinc/confluent-kafka-js/blob/master/CONTRIBUTING.md) - -(README below this point TBA - it's just the older stuff). - -## Tests - -This project includes two types of unit tests in this project: -* end-to-end integration tests -* unit tests - -You can run both types of tests by using `Makefile`. Doing so calls `mocha` in your locally installed `node_modules` directory. - -* Before you run the tests, be sure to init and update the submodules: - 1. `git submodule init` - 2. `git submodule update` -* To run the unit tests, you can run `make lint` or `make test`. -* To run the integration tests, you must have a running Kafka installation available. By default, the test tries to connect to `localhost:9092`; however, you can supply the `KAFKA_HOST` environment variable to override this default behavior. Run `make e2e`. +For guidelines on contributing please see [CONTRIBUTING.md](CONTRIBUTING.md) # Usage @@ -94,6 +80,8 @@ The library currently supports the following callbacks: * `rebalance_cb` (see [Rebalancing](#rebalancing)) * `offset_commit_cb` (see [Commits](#commits)) +(README below this point TBA - it's just the older stuff). + ### Librdkafka Methods This library includes two utility functions for detecting the status of your installation. Please try to include these when making issue reports where applicable. @@ -614,3 +602,17 @@ The following table lists important methods for this API. |`client.createPartitions(topicName, desiredPartitions, timeout, cb)` | Create partitions until the topic has the desired number of partitions. | Check the tests for an example of how to use this API! + +## Tests + +This project includes two types of unit tests in this project: +* end-to-end integration tests +* unit tests + +You can run both types of tests by using `Makefile`. Doing so calls `mocha` in your locally installed `node_modules` directory. + +* Before you run the tests, be sure to init and update the submodules: + 1. `git submodule init` + 2. `git submodule update` +* To run the unit tests, you can run `make lint` or `make test`. +* To run the integration tests, you must have a running Kafka installation available. By default, the test tries to connect to `localhost:9092`; however, you can supply the `KAFKA_HOST` environment variable to override this default behavior. Run `make e2e`. diff --git a/examples/consumer.js b/examples/consumer.js new file mode 100644 index 00000000..7fcc8228 --- /dev/null +++ b/examples/consumer.js @@ -0,0 +1,97 @@ +const { Kafka, ErrorCodes } = require('../').KafkaJS + +async function consumerStart() { + let stopped = false; + + // Initialization + const consumer = new Kafka().consumer({ + 'bootstrap.servers': 'localhost:9092', + 'group.id': 'test-group2', + 'auto.offset.reset': 'earliest', + 'rebalance_cb': (err, assignment) => { + switch (err.code) { + case ErrorCodes.ERR__ASSIGN_PARTITIONS: + console.log(`Assigned partitions ${JSON.stringify(assignment)}`); + pauseResumeLoop(); + break; + case ErrorCodes.ERR__REVOKE_PARTITIONS: + console.log(`Revoked partitions ${JSON.stringify(assignment)}`); + break; + default: + console.error(err); + } + }, + }); + + await consumer.connect(); + console.log("Connected successfully"); + + await consumer.subscribe({ topics: [ "topic2" ] }); + + // Consume example with seek. + let batch = 0; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }) + + // Seek example: seek to beginning every 100 messages + if (++batch % 100 == 0) { + await consumer.seek({ + topic, + partition, + offset: -2 /* Offset beginning is -2 */ + }); + batch = 0; + } + }, + }); + + // Pause/Resume example + const pauseResumeLoop = async () => { + let paused = false; + let ticks = 0; + while (!stopped) { + await new Promise((resolve) => setTimeout(resolve, 100)); + if (stopped) + break; + + ticks++; + if (ticks == 200) { + ticks = 0; + const assignment = consumer.assignment(); + if (paused) { + console.log(`Resuming partitions ${JSON.stringify(assignment)}`) + consumer.resume(assignment); + } else { + console.log(`Pausing partitions ${JSON.stringify(assignment)}`); + consumer.pause(assignment); + } + paused = !paused; + } + } + } + + // Disconnect example + const disconnect = () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + stopped = true; + consumer.commitOffsets() + .finally(() => + consumer.disconnect() + ) + .finally(() => + console.log("Disconnected successfully") + ); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); +} + +consumerStart(); diff --git a/examples/eos.js b/examples/eos.js new file mode 100644 index 00000000..e60b30ee --- /dev/null +++ b/examples/eos.js @@ -0,0 +1,78 @@ +const { Kafka } = require('../').KafkaJS; +//const { Kafka } = require('kafkajs') + +async function eosStart() { + const consumer = new Kafka().consumer({ + 'bootstrap.servers': 'localhost:9092', + 'group.id': 'test-group4', + 'enable.auto.commit': false, + 'auto.offset.reset': 'earliest', + }); + + const producer = new Kafka().producer({ + 'bootstrap.servers': 'localhost:9092', + 'transactional.id': 'txid', + }); + + await consumer.connect(); + await producer.connect(); + + await consumer.subscribe({ + topics: ["consumeTopic"] + }); + + // The run method acts like a consume-transform-produce loop. + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + const msgAckString = JSON.stringify({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString() + }); + + console.log(msgAckString); + + try { + const transaction = await producer.transaction(); + + await transaction.send({ + topic: 'produceTopic', + messages: [ + { value: 'consumed a message: ' + msgAckString }, + ] + }); + + await transaction.sendOffsets({ + consumer, + topics: [ + { + topic, + partitions: [ + { partition, offset: message.offset }, + ], + } + ], + }); + + await transaction.commit(); + + } catch (e) { + console.log({ e, s: "ERROR" }); + await transaction.abort(); + } + }, + }); + + const disconnect = async () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + await consumer.disconnect(); + await producer.disconnect(); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); +} + +eosStart(); diff --git a/examples/consumer-flow.md b/examples/node-rdkafka/consumer-flow.md similarity index 100% rename from examples/consumer-flow.md rename to examples/node-rdkafka/consumer-flow.md diff --git a/examples/consumer.md b/examples/node-rdkafka/consumer.md similarity index 100% rename from examples/consumer.md rename to examples/node-rdkafka/consumer.md diff --git a/examples/docker-alpine.md b/examples/node-rdkafka/docker-alpine.md similarity index 100% rename from examples/docker-alpine.md rename to examples/node-rdkafka/docker-alpine.md diff --git a/examples/high-level-producer.md b/examples/node-rdkafka/high-level-producer.md similarity index 100% rename from examples/high-level-producer.md rename to examples/node-rdkafka/high-level-producer.md diff --git a/examples/metadata.md b/examples/node-rdkafka/metadata.md similarity index 100% rename from examples/metadata.md rename to examples/node-rdkafka/metadata.md diff --git a/examples/producer-cluster.md b/examples/node-rdkafka/producer-cluster.md similarity index 100% rename from examples/producer-cluster.md rename to examples/node-rdkafka/producer-cluster.md diff --git a/examples/producer.md b/examples/node-rdkafka/producer.md similarity index 100% rename from examples/producer.md rename to examples/node-rdkafka/producer.md diff --git a/examples/producer.js b/examples/producer.js new file mode 100644 index 00000000..3f551680 --- /dev/null +++ b/examples/producer.js @@ -0,0 +1,37 @@ +const { Kafka } = require('../').KafkaJS +//const { Kafka } = require('kafkajs') + +async function producerStart() { + const producer = new Kafka().producer({ + 'bootstrap.servers': 'localhost:9092', + 'acks': 'all', + }); + + await producer.connect(); + console.log("Connected successfully"); + + const res = [] + for (let i = 0; i < 50; i++) { + res.push(producer.send({ + topic: 'test-topic', + messages: [ + { value: 'v222', partition: 1 }, + { value: 'v11', partition: 0, key: 'x' }, + ] + })); + } + + const produceRecords = await Promise.all(res); + + // Produce records is an array of delivery reports for each call to `send`. + // In case `messages` contains more than one message to the same topic-partition, only the last + // delivery report is included in the array. + console.log("Produced messages, first delivery report:\n" + JSON.stringify(produceRecords[0], null, 2)); + console.log("Produced messages, last delivery report:\n" + JSON.stringify(produceRecords[produceRecords.length - 1], null, 2)); + + await producer.disconnect(); + + console.log("Disconnected successfully"); +} + +producerStart(); From aa91dbe2b09969d85377190e286956c6c4163fd5 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Jan 2024 10:17:15 +0530 Subject: [PATCH 049/224] Copy config rather than modifying it so that same Kafka object can be used again and again --- lib/kafkajs/_kafka.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 09fe0efc..65a9edd0 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -27,10 +27,10 @@ class Kafka { * @returns {(import("../../types/kafkajs").ProducerConstructorConfig & import("../../types/kafkajs").CommonConstructorConfig) | (import("../../types/kafkajs").ConsumerConstructorConfig & import("../../types/kafkajs").CommonConstructorConfig)} */ #mergeConfiguration(config) { - config = config ?? {}; + config = Object.assign({}, config) ?? {}; const mergedConfig = Object.assign({}, this.#commonClientConfig); - mergedConfig.kafkaJs = mergedConfig.kafkaJs ?? {}; + mergedConfig.kafkaJs = Object.assign({}, mergedConfig.kafkaJs) ?? {}; if (typeof config.kafkaJs === 'object') { mergedConfig.kafkaJs = Object.assign(mergedConfig.kafkaJs, config.kafkaJs); From c4057d3223c449a7d42aa11f844c0e008173b78a Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Jan 2024 12:20:33 +0530 Subject: [PATCH 050/224] Add simpler consume() function besides run() and update example --- examples/consumer.js | 125 ++++++++++++++++++--------------------- lib/kafkajs/_consumer.js | 44 +++++++++++++- 2 files changed, 101 insertions(+), 68 deletions(-) diff --git a/examples/consumer.js b/examples/consumer.js index 7fcc8228..d7dd7fa3 100644 --- a/examples/consumer.js +++ b/examples/consumer.js @@ -1,83 +1,33 @@ const { Kafka, ErrorCodes } = require('../').KafkaJS + async function consumerStart() { + let consumer; let stopped = false; - // Initialization - const consumer = new Kafka().consumer({ - 'bootstrap.servers': 'localhost:9092', - 'group.id': 'test-group2', - 'auto.offset.reset': 'earliest', - 'rebalance_cb': (err, assignment) => { - switch (err.code) { - case ErrorCodes.ERR__ASSIGN_PARTITIONS: - console.log(`Assigned partitions ${JSON.stringify(assignment)}`); - pauseResumeLoop(); - break; - case ErrorCodes.ERR__REVOKE_PARTITIONS: - console.log(`Revoked partitions ${JSON.stringify(assignment)}`); - break; - default: - console.error(err); - } - }, - }); - - await consumer.connect(); - console.log("Connected successfully"); - - await consumer.subscribe({ topics: [ "topic2" ] }); - - // Consume example with seek. - let batch = 0; - consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - console.log({ - topic, - partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }) - - // Seek example: seek to beginning every 100 messages - if (++batch % 100 == 0) { - await consumer.seek({ - topic, - partition, - offset: -2 /* Offset beginning is -2 */ - }); - batch = 0; - } - }, - }); - - // Pause/Resume example + // Pause/Resume example, pause and resume alternately every 2 seconds. + let pauseResumeLoopStarted = false; const pauseResumeLoop = async () => { let paused = false; - let ticks = 0; + pauseResumeLoopStarted = true; while (!stopped) { - await new Promise((resolve) => setTimeout(resolve, 100)); + await new Promise((resolve) => setTimeout(resolve, 2000)); if (stopped) break; - ticks++; - if (ticks == 200) { - ticks = 0; - const assignment = consumer.assignment(); - if (paused) { - console.log(`Resuming partitions ${JSON.stringify(assignment)}`) - consumer.resume(assignment); - } else { - console.log(`Pausing partitions ${JSON.stringify(assignment)}`); - consumer.pause(assignment); - } - paused = !paused; + const assignment = consumer.assignment(); + if (paused) { + console.log(`Resuming partitions ${JSON.stringify(assignment)}`) + consumer.resume(assignment); + } else { + console.log(`Pausing partitions ${JSON.stringify(assignment)}`); + consumer.pause(assignment); } + paused = !paused; } - } + }; - // Disconnect example + // Set up signals for a graceful shutdown. const disconnect = () => { process.off('SIGINT', disconnect); process.off('SIGTERM', disconnect); @@ -92,6 +42,49 @@ async function consumerStart() { } process.on('SIGINT', disconnect); process.on('SIGTERM', disconnect); + + + + // Initialization + consumer = new Kafka().consumer({ + 'bootstrap.servers': 'localhost:9092', + 'group.id': 'test-group3', + 'auto.offset.reset': 'earliest', + 'enable.partition.eof': 'true', + 'rebalance_cb': (err, assignment) => { + switch (err.code) { + case ErrorCodes.ERR__ASSIGN_PARTITIONS: + console.log(`Assigned partitions ${JSON.stringify(assignment)}`); + if (!pauseResumeLoopStarted) // Start the pause/resume loop for the example. + pauseResumeLoop(); + break; + case ErrorCodes.ERR__REVOKE_PARTITIONS: + console.log(`Revoked partitions ${JSON.stringify(assignment)}`); + break; + default: + console.error(err); + } + }, + }); + + await consumer.connect(); + console.log("Connected successfully"); + await consumer.subscribe({ topics: ["test-topic"] }); + + // Start the consumer. + while (!stopped) { + const message = await consumer.consume(1000); + if (!message) { + continue; + } + console.log({ + topic: message.topic, + partition: message.partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); + } } consumerStart(); diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 67187ba3..ee44a0e9 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -102,6 +102,12 @@ class Consumer { */ #lock = new Lock({ timeout: 5000 }); + /** + * Whether the consumer is running. + * @type {boolean} + */ + #running = false; + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -128,14 +134,14 @@ class Consumer { let call; if (typeof userSpecifiedRebalanceCb === 'function') { - call = new Promise((resolve, reject) => setImmediate(() => { + call = new Promise((resolve, reject) => { try { userSpecifiedRebalanceCb(err, assignment); resolve(); } catch (e) { reject(e); } - })); + }); } else { switch (err.code) { // TODO: is this the right way to handle this error? @@ -579,6 +585,11 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsPartitionsConsumedConcurrently(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } + if (this.#running) { + throw new error.KafkaJSError('Consumer is already running.', { code: error.ErrorCodes.ERR__STATE }); + } + this.#running = true; + /* We deliberately don't await this. */ this.#runInternal(config); } @@ -658,6 +669,35 @@ class Consumer { } } + /** + * Consumes a single message from the consumer within the given timeout. + * @note This method cannot be used with run(). Either that, or this must be used. + * + * @param {any} args + * @param {number} args.timeout - the timeout in milliseconds, defaults to 1000. + * @returns {import("../..").Message|null} a message, or null if the timeout was reached. + */ + async consume({timeout} = {timeout: 1000}) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('consume can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + if (this.#running) { + throw new error.KafkaJSError('consume() and run() cannot be used together.', { code: error.ErrorCodes.ERR__CONFLICT }); + } + + this.#internalClient.setDefaultConsumeTimeout(timeout); + let m = null; + + try { + m = await this.#consumeSingle(); + } finally { + this.#internalClient.setDefaultConsumeTimeout(undefined); + } + + return m; + } + async #commitOffsetsUntilNoStateErr(offsetsToCommit) { let err = { code: error.ErrorCodes.ERR_NO_ERROR }; do { From 6dad27b8ca53bd51ff5f0fcd2af0f1162a0d4c64 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Jan 2024 12:24:05 +0530 Subject: [PATCH 051/224] Create empty changelog file --- CHANGELOG.md | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 CHANGELOG.md diff --git a/CHANGELOG.md b/CHANGELOG.md new file mode 100644 index 00000000..e69de29b From 6861eaf627ad03293939a537929dbe36333aa92a Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Jan 2024 12:47:08 +0530 Subject: [PATCH 052/224] Update CONTRIBUTING.md and lint checks --- .jshintignore | 1 + .jshintrc | 3 ++- CONTRIBUTING.md | 9 +------ package-lock.json | 68 +++++++++++++++++++++++------------------------ package.json | 2 +- 5 files changed, 39 insertions(+), 44 deletions(-) diff --git a/.jshintignore b/.jshintignore index b43bf86b..d08618fd 100644 --- a/.jshintignore +++ b/.jshintignore @@ -1 +1,2 @@ README.md +lib/kafkajs/* \ No newline at end of file diff --git a/.jshintrc b/.jshintrc index 09968b8b..a97e0dc7 100644 --- a/.jshintrc +++ b/.jshintrc @@ -19,5 +19,6 @@ "undef": true, "strict": false, "white": false, - "eqnull": true + "eqnull": true, + "esversion": 11 } diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 0d965dc5..7e0f01c5 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -103,15 +103,8 @@ In short: ### JavaScript Styleguide -* Place `module.exports` at or near the top of the file. - * Defined functions are hoisted, so it is appropriate to define the - function after you export it. - * When exporting an object, define it first, then export it, and then add - methods or properties. -* Do not use ES2015 specific features (for example, do not use `let`, `const`, - or `class`). * All callbacks should follow the standard Node.js callback signature. -* Your JavaScript should properly pass the linter (`make jslint`). +* Your JavaScript should properly pass the linter (`make jslint` and `make eslint`). ### C++ Styleguide diff --git a/package-lock.json b/package-lock.json index 30fe2b50..d910aa31 100644 --- a/package-lock.json +++ b/package-lock.json @@ -20,7 +20,7 @@ "eslint": "8.54.0", "jest": "^29.7.0", "jsdoc": "^4.0.2", - "jshint": "^2.10.1", + "jshint": "^2.13.6", "mocha": "^10.2.0", "node-gyp": "^9.3.1", "typescript": "^5.1.6" @@ -145,9 +145,9 @@ } }, "node_modules/@babel/core": { - "version": "7.23.6", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.6.tgz", - "integrity": "sha512-FxpRyGjrMJXh7X3wGLGhNDCRiwpWEF74sKjTLDJSG5Kyvow3QZaG0Adbqzi9ZrVjTWpsX+2cxWXD71NMg93kdw==", + "version": "7.23.7", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.7.tgz", + "integrity": "sha512-+UpDgowcmqe36d4NwqvKsyPMlOLNGMsfMmQ5WGCu+siCe3t3dfe9njrzGfdN4qq+bcNUt0+Vw6haRxBOycs4dw==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", @@ -155,10 +155,10 @@ "@babel/generator": "^7.23.6", "@babel/helper-compilation-targets": "^7.23.6", "@babel/helper-module-transforms": "^7.23.3", - "@babel/helpers": "^7.23.6", + "@babel/helpers": "^7.23.7", "@babel/parser": "^7.23.6", "@babel/template": "^7.22.15", - "@babel/traverse": "^7.23.6", + "@babel/traverse": "^7.23.7", "@babel/types": "^7.23.6", "convert-source-map": "^2.0.0", "debug": "^4.1.0", @@ -349,13 +349,13 @@ } }, "node_modules/@babel/helpers": { - "version": "7.23.6", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.6.tgz", - "integrity": "sha512-wCfsbN4nBidDRhpDhvcKlzHWCTlgJYUUdSJfzXb2NuBssDSIjc3xcb+znA7l+zYsFljAcGM0aFkN40cR3lXiGA==", + "version": "7.23.8", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.8.tgz", + "integrity": "sha512-KDqYz4PiOWvDFrdHLPhKtCThtIcKVy6avWD2oG4GEvyQ+XDZwHD4YQd+H2vNMnq2rkdxsDkU82T+Vk8U/WXHRQ==", "dev": true, "dependencies": { "@babel/template": "^7.22.15", - "@babel/traverse": "^7.23.6", + "@babel/traverse": "^7.23.7", "@babel/types": "^7.23.6" }, "engines": { @@ -651,9 +651,9 @@ } }, "node_modules/@babel/traverse": { - "version": "7.23.6", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.6.tgz", - "integrity": "sha512-czastdK1e8YByZqezMPFiZ8ahwVMh/ESl9vPgvgdB9AmFMGP5jfpFax74AQgl5zj4XHzqeYAg2l8PuUeRS1MgQ==", + "version": "7.23.7", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.7.tgz", + "integrity": "sha512-tY3mM8rH9jM0YHFGyfC0/xf+SB5eKUu7HPj7/k3fpi9dAlsMc5YbQvDi0Sh2QTPXqMhyaAtzAr807TIyfQrmyg==", "dev": true, "dependencies": { "@babel/code-frame": "^7.23.5", @@ -1369,9 +1369,9 @@ } }, "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==", + "version": "7.6.8", + "resolved": "https://registry.npmjs.org/@types/babel__generator/-/babel__generator-7.6.8.tgz", + "integrity": "sha512-ASsj+tpEDsEiFr1arWrlN6V3mdfjRMZt6LtK/Vp/kreFLnr5QH5+DhvD5nINYZXzwJvXeGq+05iUXcAzVrqWtw==", "dev": true, "dependencies": { "@babel/types": "^7.0.0" @@ -1388,9 +1388,9 @@ } }, "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==", + "version": "7.20.5", + "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.5.tgz", + "integrity": "sha512-WXCyOcRtH37HAUkpXhUduaxdm82b4GSlyTqajXviN4EfiuPgNYR109xMCKvpl6zPIpua0DGlMEDCq+g8EdoheQ==", "dev": true, "dependencies": { "@babel/types": "^7.20.7" @@ -1452,9 +1452,9 @@ "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==", + "version": "20.10.7", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.10.7.tgz", + "integrity": "sha512-fRbIKb8C/Y2lXxB5eVMj4IU7xpdox0Lh8bUPEdtLysaylsml1hOOx1+STloRs/B9nf7C6kPRmmg/V7aQW7usNg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -1493,9 +1493,9 @@ "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==", + "version": "8.11.3", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.11.3.tgz", + "integrity": "sha512-Y9rRfJG5jcKOE0CLisYbojUjIrIEE7AGMzA/Sm4BslANhbS+cDMpgBdcPT91oJ7OuJ9hYJBx59RjbhxVnrF8Xg==", "dev": true, "bin": { "acorn": "bin/acorn" @@ -1996,9 +1996,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001570", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001570.tgz", - "integrity": "sha512-+3e0ASu4sw1SWaoCtvPeyXp+5PsjigkSt8OXZbF9StH5pQWbxEjLAZE3n8Aup5udop1uRiKA7a4utUk/uoSpUw==", + "version": "1.0.30001576", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001576.tgz", + "integrity": "sha512-ff5BdakGe2P3SQsMsiqmt1Lc8221NR1VzHj5jXN5vBny9A6fpze94HiVV/n7XRosOlsShJcvMv5mdnpjOGCEgg==", "dev": true, "funding": [ { @@ -2436,9 +2436,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.613", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.613.tgz", - "integrity": "sha512-r4x5+FowKG6q+/Wj0W9nidx7QO31BJwmR2uEo+Qh3YLGQ8SbBAFuDFpTxzly/I2gsbrFwBuIjrMp423L3O5U3w==", + "version": "1.4.625", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.625.tgz", + "integrity": "sha512-DENMhh3MFgaPDoXWrVIqSPInQoLImywfCwrSmVl3cf9QHzoZSiutHwGaB/Ql3VkqcQV30rzgdM+BjKqBAJxo5Q==", "dev": true }, "node_modules/emittery": { @@ -2746,9 +2746,9 @@ "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==", + "version": "1.16.0", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.16.0.tgz", + "integrity": "sha512-ifCoaXsDrsdkWTtiNJX5uzHDsrck5TzfKKDcuFFTIrrc/BS076qgEIfoIy1VeZqViznfKiysPYTh/QeHtnIsYA==", "dev": true, "dependencies": { "reusify": "^1.0.4" diff --git a/package.json b/package.json index f09ea442..026aca64 100644 --- a/package.json +++ b/package.json @@ -35,7 +35,7 @@ "eslint": "8.54.0", "jest": "^29.7.0", "jsdoc": "^4.0.2", - "jshint": "^2.10.1", + "jshint": "^2.13.6", "mocha": "^10.2.0", "node-gyp": "^9.3.1", "typescript": "^5.1.6" From 6f589424bac7427ad939a280518679dd9ed14b04 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Jan 2024 12:49:34 +0530 Subject: [PATCH 053/224] Bump version --- package.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/package.json b/package.json index 026aca64..0e38fc10 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "confluent-kafka-js", - "version": "v0.1.4-devel", + "version": "v0.1.5-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "main": "lib/index.js", From c57969213931ead962978841f3f32cf854dd45e5 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Jan 2024 14:34:31 +0530 Subject: [PATCH 054/224] Update agent to newer macOS version --- .semaphore/semaphore.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 1a70537f..15a39fb9 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -74,7 +74,7 @@ blocks: task: agent: machine: - type: s1-prod-macos-arm64 + type: s1-prod-macos-13-5-arm64 jobs: - name: 'Build from source and test' commands: @@ -269,7 +269,7 @@ blocks: task: agent: machine: - type: s1-prod-macos-arm64 + type: s1-prod-macos-13-5-arm64 env_vars: - name: ARCHITECTURE value: "arm64" From 8c0a74813efff8c1238aeca9d337c9bbca7dc3e3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 10 Jan 2024 10:30:28 +0530 Subject: [PATCH 055/224] Bump version in README.md and package-lock --- README.md | 2 +- package-lock.json | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 4a40e44c..3b1a12a7 100644 --- a/README.md +++ b/README.md @@ -43,7 +43,7 @@ Installation on any of these platforms is meant to be seamless, without any C/C+ from GitHub: ```bash -$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.4-devel" +$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.5-devel" ``` Yarn and pnpm support is experimental. diff --git a/package-lock.json b/package-lock.json index d910aa31..c22fd758 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "confluent-kafka-js", - "version": "v0.1.4-devel", + "version": "v0.1.5-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "confluent-kafka-js", - "version": "v0.1.4-devel", + "version": "v0.1.5-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { From 384cd9ab26fcc6f5fb5d696191b3d0105fc60c46 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 11 Jan 2024 15:51:53 +0530 Subject: [PATCH 056/224] Update CHANGELOG.md --- CHANGELOG.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e69de29b..94715f14 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -0,0 +1,10 @@ +# confluent-kafka-js v0.1.5-devel + +v0.1.5-devel is a pre-production, early-access release. + +## Features + +1. Pre-built binaries for Linux (both amd64 and arm64, both musl and glibc), for macOS (m1), for node versions 18, 20 and 21. +2. Promisified API for Consumer, Producer and Admin Client. +3. Allow passing topic configuration properties via the global configuration block. +4. Remove dependencies with security issues. From a6267afed86bea038e329349134199980b03655e Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 16 Jan 2024 12:08:33 +0530 Subject: [PATCH 057/224] Create SECURITY.md --- SECURITY.md | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 SECURITY.md diff --git a/SECURITY.md b/SECURITY.md new file mode 100644 index 00000000..64bc0e3d --- /dev/null +++ b/SECURITY.md @@ -0,0 +1,11 @@ +# Security Policy + +## Supported Versions + +Post version 1.0.0, the latest version will be supported. + +## Reporting a Vulnerability + +To report a vulnerability, please notify security@confluent.io + +If an issue is confirmed, a github issue will be created to help track progress with its resolution. From 224860f34ab7d3b4a316992d76622735d20d74c6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 17 Jan 2024 14:36:16 +0530 Subject: [PATCH 058/224] Fix warning about retries being set in consumer --- lib/kafkajs/_common.js | 3 +-- lib/kafkajs/_producer.js | 4 ++++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 526b04e4..32958d63 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -341,11 +341,10 @@ function kafkaJSToRdKafkaConfig(config) { rdkafkaConfig["socket.connection.setup.timeout.ms"] = totalConnectionTimeout; const retry = config.retry ?? {}; - const { maxRetryTime, initialRetryTime, factor, multiplier, retries, restartOnFailure } = retry; + const { maxRetryTime, initialRetryTime, factor, multiplier, restartOnFailure } = retry; rdkafkaConfig["retry.backoff.max.ms"] = maxRetryTime ?? 30000; rdkafkaConfig["retry.backoff.ms"] = initialRetryTime ?? 300; - rdkafkaConfig["retries"] = retries ?? 5; if ((typeof factor === 'number') || (typeof multiplier === 'number')) { throw new error.KafkaJSError(CompatibilityErrorMessages.retryFactorMultiplier(), { diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 0588f656..782e9571 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -150,6 +150,10 @@ class Producer { rdKafkaConfig['request.timeout.ms'] = kjsConfig.timeout; } + const retry = kjsConfig.retry ?? {}; + const { retries } = retry; + rdKafkaConfig["retries"] = retries ?? 5; + /* Set the logger */ if (Object.hasOwn(kjsConfig, 'logger')) { this.#logger = kjsConfig.logger; From c9e062cd1d109f0d3565249705f78cc7793ed166 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 17 Jan 2024 14:37:15 +0530 Subject: [PATCH 059/224] Update CONTRIBUTING.md --- CONTRIBUTING.md | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 7e0f01c5..3514b495 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -206,7 +206,6 @@ Steps to update: ```bash node ci/librdkafka-defs-generator.js ``` - Note: This is ran automatically during CI flows but it's good to run it during the version upgrade pull request. 1. Run `npm install --lockfile-version 2` to build with the new version and fix any build errors that occur. @@ -214,8 +213,16 @@ Steps to update: 1. Update the version numbers referenced in the [`README.md`](https://github.com/confluentinc/confluent-kafka-js/blob/master/README.md) file to the new version. -## Publishing new npm version +## Releasing -1. Increment the `version` in `package.json` and merge that change in. +1. Increment the `version` in `package.json` and change README.md to point to the new version. -1. Create a new github release. Set the tag & release title to the same string as `version` in `package.json`. +1. Run `npm install` to update the `package-lock.json` file. + +1. Create a PR and merge the above changes, and tag the merged commit with the new version, e.g. `git tag vx.y.z && git push origin vx.y.z`. + This should be the same string as `version` in `package.json`. + +1. The CI will run on the tag, which will create the release artifacts in Semaphore CI. + +1. Create a new GitHub release with the tag, and upload the release artifacts from Semaphore CI. + The release title should be the same string as `version` in `package.json`. \ No newline at end of file From 265c2cef8afc14deeaacf857d4223eb5286194f4 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 17 Jan 2024 14:39:35 +0530 Subject: [PATCH 060/224] Update QUICKSTART.md --- QUICKSTART.md | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/QUICKSTART.md b/QUICKSTART.md index 5eed5916..1a027246 100644 --- a/QUICKSTART.md +++ b/QUICKSTART.md @@ -40,20 +40,26 @@ async function consumerStart() { await consumer.subscribe({ topics: [ "topic" ] }); - consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - console.log({ - topic, - partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }); - }, - }); + let stopped = false; + while (!stopped) { + const message = await consumer.consume(1000); + if (!message) { + continue; + } + console.log({ + topic: message.topic, + partition: message.partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); + + // Update stopped whenever we're done consuming. + // stopped = true; + } - // When done consuming - // await consumer.disconnect(); + // Disconnect and clean up. + await consumer.disconnect(); } consumerStart(); From 2f8f7fed152ddb66f3a0c3e36831dab188887408 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 17 Jan 2024 14:52:57 +0530 Subject: [PATCH 061/224] Change kafkaJs -> kafkaJS for uniformity --- MIGRATION.md | 38 ++++++++++++++--------------- examples/kafkajs/admin.js | 2 +- examples/kafkajs/consumer.js | 4 ++-- examples/kafkajs/eos.js | 6 ++--- examples/kafkajs/producer.js | 2 +- lib/kafkajs/_admin.js | 4 ++-- lib/kafkajs/_common.js | 42 ++++++++++++++++----------------- lib/kafkajs/_consumer.js | 4 ++-- lib/kafkajs/_kafka.js | 16 ++++++------- lib/kafkajs/_producer.js | 4 ++-- test/promisified/testhelpers.js | 4 ++-- types/kafkajs.d.ts | 8 +++---- 12 files changed, 67 insertions(+), 67 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 248542fe..9435a4f6 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -3,7 +3,7 @@ ## KafkaJS -1. Change the import statement, and add a `kafkaJs` block around your configs. +1. Change the import statement, and add a `kafkaJS` block around your configs. ```javascript const { Kafka } = require('kafkajs'); const kafka = new Kafka({ brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ }); @@ -12,8 +12,8 @@ to ```javascript const { Kafka } = require('confluent-kafka-js').KafkaJS; - const kafka = new Kafka({ kafkaJs: { brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ } }); - const producer = kafka.producer({ kafkaJs: { /* ... */, } }); + const kafka = new Kafka({ kafkaJS: { brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ } }); + const producer = kafka.producer({ kafkaJS: { /* ... */, } }); ``` 2. Try running your program. In case a migration is needed, an informative error will be thrown. @@ -38,7 +38,7 @@ +const { Kafka } = require('confluent-kafka-js').KafkaJS; const kafka = new Kafka({ -+ kafkaJs: { ++ kafkaJS: { clientId: 'my-app', brokers: ['kafka1:9092', 'kafka2:9092'] + } @@ -46,7 +46,7 @@ const kafka = new Kafka({ const producerRun = async () => { - const producer = kafka.producer(); -+ const producer = kafka.producer({ kafkaJs: { acks: 1 } }); ++ const producer = kafka.producer({ kafkaJS: { acks: 1 } }); await producer.connect(); await producer.send({ topic: 'test-topic', @@ -61,7 +61,7 @@ const producerRun = async () => { const consumerRun = async () => { // Consuming - const consumer = kafka.consumer({ groupId: 'test-group' }); -+ const consumer = kafka.consumer({ kafkaJs: { groupId: 'test-group', fromBeginning: true } }); ++ const consumer = kafka.consumer({ kafkaJS: { groupId: 'test-group', fromBeginning: true } }); await consumer.connect(); - await consumer.subscribe({ topic: 'test-topic', fromBeginning: true }); + await consumer.subscribe({ topic: 'test-topic' }); @@ -84,7 +84,7 @@ producerRun().then(consumerRun).catch(console.error); #### Configuration changes ```javascript - const kafka = new Kafka({ kafkaJs: { /* common configuration changes */ } }); + const kafka = new Kafka({ kafkaJS: { /* common configuration changes */ } }); ``` Each allowed config property is discussed in the table below. If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. @@ -92,7 +92,7 @@ producerRun().then(consumerRun).catch(console.error); | Property | Default Value | Comment | |-------------------------------|--------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | - | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified outside the kafkaJs block.** | + | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified outside the kafkaJS block.** | | **sasl** | - | An optional object of the form `{ mechanism: 'plain' or 'scram-sha-256' or 'scram-sha-512', username: string, password: string }`. **Additional authentication types are not yet supported.** | | clientId | "rdkafka" | An optional string used to identify the client. | | **connectionTimeout** | 1000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | @@ -109,7 +109,7 @@ producerRun().then(consumerRun).catch(console.error); | **retry.restartOnFailure** | true | Consumer only. **Cannot be changed**. Consumer will always make an attempt to restart. | | logLevel | `logLevel.INFO` | Decides the severity level of the logger created by the underlying library. A logger created with the `INFO` level will not be able to log `DEBUG` messages later. | | **socketFactory** | null | **No longer supported.** | - | outer config | {} | The configuration outside the kafkaJs block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | + | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | ### Producer @@ -117,7 +117,7 @@ producerRun().then(consumerRun).catch(console.error); #### Producer Configuration Changes ```javascript - const producer = kafka.producer({ kafkaJs: { /* producer-specific configuration changes. */ } }); + const producer = kafka.producer({ kafkaJS: { /* producer-specific configuration changes. */ } }); ``` Each allowed config property is discussed in the table below. @@ -125,7 +125,7 @@ producerRun().then(consumerRun).catch(console.error); | Property | Default Value | Comment | |-------------------------|------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| - | **createPartitioner** | DefaultPartioner (murmur2_random) - Java client compatible | Custom partitioner support is not yet provided. The default partitioner's behaviour is retained, and a number of partitioners are provided via the `partitioner` property, which is specified outside the `kafkaJs` block. | + | **createPartitioner** | DefaultPartioner (murmur2_random) - Java client compatible | Custom partitioner support is not yet provided. The default partitioner's behaviour is retained, and a number of partitioners are provided via the `partitioner` property, which is specified outside the `kafkaJS` block. | | **retry** | object | Identical to `retry` in the common configuration. This takes precedence over the common config retry. | | metadataMaxAge | 5 minutes | Time in milliseconds after which to refresh metadata for known topics | | allowAutoTopicCreation | true | Determines if a topic should be created if it doesn't exist while producing. | @@ -136,7 +136,7 @@ producerRun().then(consumerRun).catch(console.error); | **acks** | -1 | The number of required acks before a Produce succeeds. **This is set on a per-producer level, not on a per `send` level**. -1 denotes it will wait for all brokers in the in-sync replica set. | | **compression** | CompressionTypes.NONE | Compression codec for Produce messages. **This is set on a per-producer level, not on a per `send` level**. It must be a key of the object CompressionType, namely GZIP, SNAPPY, LZ4, ZSTD or NONE. | | **timeout** | 30000 | The ack timeout of the producer request in milliseconds. This value is only enforced by the broker. **This is set on a per-producer level, not on a per `send` level**. | - | outer config | {} | The configuration outside the kafkaJs block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | + | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | @@ -164,9 +164,9 @@ producerRun().then(consumerRun).catch(console.error); After: ```javascript - const kafka = new Kafka({ kafkaJs: { /* ... */ }}); + const kafka = new Kafka({ kafkaJS: { /* ... */ }}); const producer = kafka.producer({ - kafkaJs: { + kafkaJS: { acks: 1, compression: CompressionTypes.GZIP|CompressionTypes.SNAPPY|CompressionTypes.LZ4|CompressionTypes.ZSTD|CompressionTypes.NONE, timeout: 30000, @@ -187,7 +187,7 @@ producerRun().then(consumerRun).catch(console.error); #### Consumer Configuration Changes ```javascript - const consumer = kafka.consumer({ kafkaJs: { /* producer-specific configuration changes. */ } }); + const consumer = kafka.consumer({ kafkaJS: { /* producer-specific configuration changes. */ } }); ``` Each allowed config property is discussed in the table below. If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. @@ -211,7 +211,7 @@ producerRun().then(consumerRun).catch(console.error); | **fromBeginning** | false | If there is initial offset in offset store or the desired offset is out of range, and this is true, we consume the earliest possible offset. **This is set on a per-consumer level, not on a per `subscribe` level**. | | **autoCommit** | true | Whether to periodically auto-commit offsets to the broker while consuming. **This is set on a per-consumer level, not on a per `run` level**. | | **autoCommitInterval** | 5000 | Offsets are committed periodically at this interval, if autoCommit is true. **This is set on a per-consumer level, not on a per `run` level. The default value is changed to 5 seconds.**. | - | outer config | {} | The configuration outside the kafkaJs block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | + | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | #### Semantic and Per-Method Changes @@ -237,7 +237,7 @@ producerRun().then(consumerRun).catch(console.error); After: ```javascript const consumer = kafka.consumer({ - kafkaJs: { + kafkaJS: { groupId: 'test-group', fromBeginning: true, } @@ -266,9 +266,9 @@ producerRun().then(consumerRun).catch(console.error); ``` After: ```javascript - const kafka = new Kafka({ kafkaJs: { /* ... */ } }); + const kafka = new Kafka({ kafkaJS: { /* ... */ } }); const consumer = kafka.consumer({ - kafkaJs: { + kafkaJS: { /* ... */, autoCommit: true, autoCommitInterval: 5000, diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index ebad8674..e862bc65 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -3,7 +3,7 @@ const { Kafka } = require('../..').KafkaJS async function adminStart() { const kafka = new Kafka({ - kafkaJs: { + kafkaJS: { brokers: ['localhost:9092'], } }); diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 9d24635c..22a586fd 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -6,7 +6,7 @@ async function consumerStart() { var stopped = false; const kafka = new Kafka({ - kafkaJs: { + kafkaJS: { brokers: ['localhost:9092'], ssl: true, connectionTimeout: 5000, @@ -19,7 +19,7 @@ async function consumerStart() { }); consumer = kafka.consumer({ - kafkaJs: { + kafkaJS: { groupId: 'test-group', autoCommit: false, rebalanceListener: { diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index 0f85503c..78f096dc 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -3,7 +3,7 @@ const { Kafka } = require('../..').KafkaJS async function eosStart() { const kafka = new Kafka({ - kafkaJs: { + kafkaJS: { brokers: [''], ssl: true, sasl: { @@ -15,14 +15,14 @@ async function eosStart() { }); const consumer = kafka.consumer({ - kafkaJs: { + kafkaJS: { groupId: 'groupId', autoCommit: false, } }); const producer = kafka.producer({ - kafkaJs: { + kafkaJS: { transactionalId: 'txid' } }); diff --git a/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index b8d2e5ee..4d5e6fc4 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -3,7 +3,7 @@ const { Kafka } = require('../..').KafkaJS async function producerStart() { const kafka = new Kafka({ - kafkaJs: { + kafkaJS: { brokers: [''], ssl: true, sasl: { diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index f59e4ca8..4b976e3c 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -79,7 +79,7 @@ class Admin { } #finalizedConfig() { - let compatibleConfig = this.#kafkaJSToAdminConfig(this.#userConfig.kafkaJs); + let compatibleConfig = this.#kafkaJSToAdminConfig(this.#userConfig.kafkaJS); /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest * log level, as librdkafka will control the granularity. */ @@ -90,7 +90,7 @@ class Admin { let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ - delete rdKafkaConfig.kafkaJs; + delete rdKafkaConfig.kafkaJS; return rdKafkaConfig; } diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 32958d63..94ca0ceb 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,10 +1,10 @@ const error = require("./_error"); const process = require("process"); -/* A list of kafkaJs compatible properties that we process. +/* A list of kafkaJS compatible properties that we process. * All of these are not necessarily supported, and an error will be * thrown if they aren't. */ -const kafkaJsProperties = { +const kafkaJSProperties = { common: [ "brokers", "clientId", @@ -149,7 +149,7 @@ function loggerTrampoline(msg, logger) { function createReplacementErrorMessage(cOrP, fnCall, property, propertyVal, replacementVal, isLK = false) { if (!isLK) { - replacementVal = `kafkaJs: { ${replacementVal}, ... }` + replacementVal = `kafkaJS: { ${replacementVal}, ... }` } return `'${property}' is not supported as a property to '${fnCall}', but must be passed to the ${cOrP} during creation.\n` + `Before: \n` + @@ -160,7 +160,7 @@ function createReplacementErrorMessage(cOrP, fnCall, property, propertyVal, repl `\tconst ${cOrP} = kafka.${cOrP}({ ${replacementVal}, ... });\n` + `\tawait ${cOrP}.connect();\n` + `\t${cOrP}.${fnCall}({ ... });\n` + - (isLK ? `For more details on what can be used outside the kafkaJs block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n` : ''); + (isLK ? `For more details on what can be used outside the kafkaJS block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n` : ''); } const CompatibilityErrorMessages = Object.freeze({ @@ -175,12 +175,12 @@ const CompatibilityErrorMessages = Object.freeze({ saslUsernamePasswordString: (mechanism) => `The 'sasl.username' and 'sasl.password' properties must be strings and must be present for the mechanism ${mechanism}.`, sslObject: () => - "The 'ssl' property must be a boolean. Any additional configuration must be provided outside the kafkaJs block.\n" + + "The 'ssl' property must be a boolean. Any additional configuration must be provided outside the kafkaJS block.\n" + "Before: \n" + - "\tconst kafka = new Kafka({ kafkaJs: { ssl: { rejectUnauthorized: false, ca: [ ... ], key: ..., cert: ... }, } }); \n" + + "\tconst kafka = new Kafka({ kafkaJS: { ssl: { rejectUnauthorized: false, ca: [ ... ], key: ..., cert: ... }, } }); \n" + "After: \n" + - '\tconst kafka = new Kafka({ kafkaJs: { ssl: true, }, "enable.ssl.certificate.verification": false, "ssl.ca.location": ..., "ssl.certificate.pem": ... });\n' + - `For more details on what can be used outside the kafkaJs block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n`, + '\tconst kafka = new Kafka({ kafkaJS: { ssl: true, }, "enable.ssl.certificate.verification": false, "ssl.ca.location": ..., "ssl.certificate.pem": ... });\n' + + `For more details on what can be used outside the kafkaJS block, see https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md\n`, retryFactorMultiplier: () => + "The 'retry.factor' and 'retry.multiplier' are not supported. They are always set to the default of 0.2 and 2 respectively.", retryRestartOnFailure: () => @@ -193,14 +193,14 @@ const CompatibilityErrorMessages = Object.freeze({ "Reauthentication threshold cannot be set, and reauthentication is automated when 80% of connections.max.reauth.ms is reached.", unsupportedKey: (key) => `The '${key}' property is not supported.`, - kafkaJsCommonKey: (key) => + kafkaJSCommonKey: (key) => `The '${key}' property seems to be a KafkaJS property in the main config block.` + `It must be moved to the kafkaJS block.` + `\nBefore: \n` + `\tconst kafka = new Kafka({ ${key}: , ... });\n` + `After: \n` + - `\tconst kafka = new Kafka({ kafkaJs: { ${key}: , ... }, ... });\n`, - kafkaJsClientKey: (key, cOrP) => + `\tconst kafka = new Kafka({ kafkaJS: { ${key}: , ... }, ... });\n`, + kafkaJSClientKey: (key, cOrP) => `The '${key}' property seems to be a KafkaJS property in the main config block. ` + `It must be moved to the kafkaJS block.` + `\nBefore: \n` + @@ -208,7 +208,7 @@ const CompatibilityErrorMessages = Object.freeze({ `\tconst ${cOrP} = kafka.${cOrP}({ ${key}: , ... });\n` + `After: \n` + `\tconst kafka = new Kafka({ ... });\n` + - `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJs: { ${key}: , ... }, ... });\n`, + `\tconst ${cOrP} = kafka.${cOrP}({ kafkaJS: { ${key}: , ... }, ... });\n`, /* Producer */ createPartitioner: () => @@ -257,9 +257,9 @@ const CompatibilityErrorMessages = Object.freeze({ * The error code will be ERR__NOT_IMPLEMENTED in case of features that are not yet implemented. */ function kafkaJSToRdKafkaConfig(config) { - /* Since the kafkaJs block is specified, we operate in - * kafkaJs compatibility mode. That means we change the defaults - * match the kafkaJs defaults. */ + /* Since the kafkaJS block is specified, we operate in + * kafkaJS compatibility mode. That means we change the defaults + * match the kafkaJS defaults. */ const rdkafkaConfig = {}; if (Object.hasOwn(config, "brokers")) { @@ -411,14 +411,14 @@ function kafkaJSToRdKafkaConfig(config) { * @returns {string|null} the first unsupported key, or null if all keys are supported. */ function checkAllowedKeys(clientType, config) { - const allowedKeysCommon = kafkaJsProperties.common; + const allowedKeysCommon = kafkaJSProperties.common; - if (!Object.hasOwn(kafkaJsProperties, clientType)) { + if (!Object.hasOwn(kafkaJSProperties, clientType)) { throw new error.KafkaJSError(`Unknown client type ${clientType}`, { code: error.ErrorCodes.ERR__INVALID_ARG, }); } - const allowedKeysSpecific = kafkaJsProperties[clientType]; + const allowedKeysSpecific = kafkaJSProperties[clientType]; for (const key of Object.keys(config)) { if (!allowedKeysCommon.includes(key) && !allowedKeysSpecific.includes(key)) { @@ -436,15 +436,15 @@ function checkAllowedKeys(clientType, config) { * @returns {string|null} the first KafkaJS specific key, or null if none is present. */ function checkIfKafkaJsKeysPresent(propertyType, config) { - if (!Object.hasOwn(kafkaJsProperties, propertyType)) { + if (!Object.hasOwn(kafkaJSProperties, propertyType)) { throw new error.KafkaJSError(`Unknown config type for ${propertyType}`, { code: error.ErrorCodes.ERR__INVALID_ARG, }); } - const kjsKeys = kafkaJsProperties[propertyType]; + const kjsKeys = kafkaJSProperties[propertyType]; for (const key of Object.keys(config)) { - /* We exclude 'acks' since it's common to both librdkafka and kafkaJs. + /* We exclude 'acks' since it's common to both librdkafka and kafkaJS. * We don't intend to keep up with new properties, so we don't need to really worry about making it extensible. */ if (kjsKeys.includes(key) && key !== 'acks') { return key; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index ee44a0e9..a732e107 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -331,7 +331,7 @@ class Consumer { #finalizedConfig() { /* Creates an rdkafka config based off the kafkaJS block. Switches to compatibility mode if the block exists. */ - let compatibleConfig = this.#kafkaJSToConsumerConfig(this.#userConfig.kafkaJs); + let compatibleConfig = this.#kafkaJSToConsumerConfig(this.#userConfig.kafkaJS); /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest * log level, as librdkafka will control the granularity. */ @@ -347,7 +347,7 @@ class Consumer { let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ - delete rdKafkaConfig.kafkaJs; + delete rdKafkaConfig.kafkaJS; delete rdKafkaConfig.rebalanceListener; /* Certain properties that the user has set are overridden. We use trampolines to accommodate the user's callbacks. diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 65a9edd0..6196cb2a 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -17,7 +17,7 @@ class Kafka { const disallowedKey = checkIfKafkaJsKeysPresent('common', this.#commonClientConfig); if (disallowedKey !== null) { - throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsCommonKey(disallowedKey)); + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJSCommonKey(disallowedKey)); } } @@ -30,11 +30,11 @@ class Kafka { config = Object.assign({}, config) ?? {}; const mergedConfig = Object.assign({}, this.#commonClientConfig); - mergedConfig.kafkaJs = Object.assign({}, mergedConfig.kafkaJs) ?? {}; + mergedConfig.kafkaJS = Object.assign({}, mergedConfig.kafkaJS) ?? {}; - if (typeof config.kafkaJs === 'object') { - mergedConfig.kafkaJs = Object.assign(mergedConfig.kafkaJs, config.kafkaJs); - delete config.kafkaJs; + if (typeof config.kafkaJS === 'object') { + mergedConfig.kafkaJS = Object.assign(mergedConfig.kafkaJS, config.kafkaJS); + delete config.kafkaJS; } Object.assign(mergedConfig, config); @@ -50,7 +50,7 @@ class Kafka { producer(config) { const disallowedKey = checkIfKafkaJsKeysPresent('producer', config ?? {}); if (disallowedKey !== null) { - throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsClientKey(disallowedKey, 'producer')); + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJSClientKey(disallowedKey, 'producer')); } return new Producer(this.#mergeConfiguration(config)); @@ -64,7 +64,7 @@ class Kafka { consumer(config) { const disallowedKey = checkIfKafkaJsKeysPresent('consumer', config ?? {}); if (disallowedKey !== null) { - throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsClientKey(disallowedKey, 'consumer')); + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJSClientKey(disallowedKey, 'consumer')); } return new Consumer(this.#mergeConfiguration(config)); @@ -73,7 +73,7 @@ class Kafka { admin(config) { const disallowedKey = checkIfKafkaJsKeysPresent('admin', config ?? {}); if (disallowedKey !== null) { - throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJsClientKey(disallowedKey, 'admin')); + throw new error.KafkaJSError(CompatibilityErrorMessages.kafkaJSClientKey(disallowedKey, 'admin')); } return new Admin(this.#mergeConfiguration(config)); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 782e9571..42b9bee9 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -169,7 +169,7 @@ class Producer { #finalizedConfig() { /* Creates an rdkafka config based off the kafkaJS block. Switches to compatibility mode if the block exists. */ - let compatibleConfig = this.#kafkaJSToProducerConfig(this.#userConfig.kafkaJs); + let compatibleConfig = this.#kafkaJSToProducerConfig(this.#userConfig.kafkaJS); /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest * log level, as librdkafka will control the granularity. */ @@ -180,7 +180,7 @@ class Producer { let rdKafkaConfig = Object.assign(compatibleConfig, this.#userConfig); /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ - delete rdKafkaConfig.kafkaJs; + delete rdKafkaConfig.kafkaJS; /* Certain properties that the user has set are overridden. There is * no longer a delivery report, rather, results are made available on diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index d8a49015..dd50dad0 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -10,13 +10,13 @@ const clusterInformation = { const debug = process.env.TEST_DEBUG; function makeConfig(config) { - const kafkaJs = Object.assign(config, clusterInformation); + const kafkaJS = Object.assign(config, clusterInformation); const common = {}; if (debug) { common['debug'] = debug; } - return Object.assign(common, { kafkaJs }); + return Object.assign(common, { kafkaJS }); } function createConsumer(config) { diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index d6438721..767d92f5 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -32,7 +32,7 @@ export interface KafkaConfig { } export interface CommonConstructorConfig extends GlobalConfig { - kafkaJs?: KafkaConfig; + kafkaJS?: KafkaConfig; } export interface ProducerConfig { @@ -46,7 +46,7 @@ export interface ProducerConfig { } export interface ProducerConstructorConfig extends ProducerGlobalConfig { - kafkaJs?: ProducerConfig; + kafkaJS?: ProducerConfig; } export interface IHeaders { @@ -150,7 +150,7 @@ export interface ConsumerConfig { } export interface ConsumerConstructorConfig extends ConsumerGlobalConfig { - kafkaJs?: ConsumerConfig; + kafkaJS?: ConsumerConfig; } export type ConsumerEvents = { @@ -177,7 +177,7 @@ export interface AdminConfig { } export interface AdminConstructorConfig extends GlobalConfig { - kafkaJs?: AdminConfig; + kafkaJS?: AdminConfig; } export interface ITopicConfig { From 834dee32b0c58700a701007694e578c0ac8b1921 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 18 Jan 2024 10:42:23 +0530 Subject: [PATCH 062/224] Add schema registry example --- MIGRATION.md | 6 +- examples/kafkajs/admin.js | 6 +- examples/kafkajs/consumer.js | 6 +- examples/kafkajs/eos.js | 6 +- examples/kafkajs/producer.js | 6 +- examples/kafkajs/sr.js | 108 +++++++++++++++++++++++++++++++++++ lib/kafkajs/_consumer.js | 1 + 7 files changed, 130 insertions(+), 9 deletions(-) create mode 100644 examples/kafkajs/sr.js diff --git a/MIGRATION.md b/MIGRATION.md index 9435a4f6..bbd02624 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -301,8 +301,12 @@ producerRun().then(consumerRun).catch(console.error); does not support `replicaAssignment`. * The `deleteTopics` method is fully supported. +### Using the Schema Registry -#### Error Handling +In case you are using the Schema Registry client at `kafkajs/confluent-schema-registry`, you will not need to make any changes to the usage. +An example is made available [here](../examples/kafkajs/sr.js). + +### Error Handling **Action**: Convert any checks based on `instanceof` and `error.name` or to error checks based on `error.code` or `error.type`. diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index e862bc65..8450738f 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -1,5 +1,7 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use 'confluent-kafka-js'. +const { Kafka } = require('../..').KafkaJS; async function adminStart() { const kafka = new Kafka({ diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 22a586fd..4249c9dd 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,5 +1,7 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use 'confluent-kafka-js'. +const { Kafka } = require('../..').KafkaJS; async function consumerStart() { let consumer; diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index 78f096dc..9b63b8c5 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -1,5 +1,7 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use 'confluent-kafka-js'. +const { Kafka } = require('../..').KafkaJS; async function eosStart() { const kafka = new Kafka({ diff --git a/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index 4d5e6fc4..365468f6 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -1,5 +1,7 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use 'confluent-kafka-js'. +const { Kafka } = require('../..').KafkaJS; async function producerStart() { const kafka = new Kafka({ diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js new file mode 100644 index 00000000..9c172230 --- /dev/null +++ b/examples/kafkajs/sr.js @@ -0,0 +1,108 @@ +// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use 'confluent-kafka-js'. +const { Kafka } = require('../..').KafkaJS; + +// Note: The kafkajs/confluent-schema-registry will need to be installed separately to run this example, +// as it isn't a dependency of confluent-kafka-js. +const { SchemaRegistry, SchemaType } = require('@kafkajs/confluent-schema-registry'); + +const registry = new SchemaRegistry({ host: '' }) +const kafka = new Kafka({ + brokers: [''], + clientId: 'example-consumer', +}) +let consumer = kafka.consumer({ groupId: 'test-group' , fromBeginning: true, } ); +let producer = kafka.producer(); + +const schemaA = { + type: 'record', + namespace: 'test', + name: 'A', + fields: [ + { name: 'id', type: 'int' }, + { name: 'b', type: 'test.B' }, + ], +}; + +const schemaB = { + type: 'record', + namespace: 'test', + name: 'B', + fields: [{ name: 'id', type: 'int' }], +}; + +const topicName = 'test-topic'; + +const run = async () => { + // Register schemaB. + await registry.register( + { + type: SchemaType.AVRO, + schema: JSON.stringify(schemaB), + }, + { subject: 'Avro:B' }, + ); + const response = await registry.api.Subject.latestVersion({ subject: 'Avro:B' }); + const { version } = JSON.parse(response.responseData); + + // Register schemaA, which references schemaB. + const { id } = await registry.register( + { + type: SchemaType.AVRO, + schema: JSON.stringify(schemaA), + references: [ + { + name: 'test.B', + subject: 'Avro:B', + version, + }, + ], + }, + { subject: 'Avro:A' }, + ) + + // Produce a message with schemaA. + await producer.connect() + const outgoingMessage = { + key: 'key', + value: await registry.encode(id, { id: 1, b: { id: 2 } }) + } + await producer.send({ + topic: topicName, + messages: [outgoingMessage] + }); + console.log("Producer sent its message.") + await producer.disconnect(); + producer = null; + + await consumer.connect() + await consumer.subscribe({ topic: topicName }) + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await registry.decode(message.value) + }; + console.log("Consumer recieved message.\nBefore decoding: " + JSON.stringify(message) + "\nAfter decoding: " + JSON.stringify(decodedMessage)); + messageRcvd = true; + }, + }); + + // Wait around until we get a message, and then disconnect. + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + consumer = null; +} + +run().catch (async e => { + console.error(e); + consumer && await consumer.disconnect(); + producer && await producer.disconnect(); + process.exit(1); +}) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index a732e107..78f389c4 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -967,6 +967,7 @@ class Consumer { /** * Disconnects and cleans up the consumer. + * @note This cannot be called from within `eachMessage` callback of `Consumer.run`. * @returns {Promise} a promise that resolves when the consumer has disconnected. */ async disconnect() { From 6ba9e02f2a3ce92d020b16c67d417ba6586704ea Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 18 Jan 2024 11:09:36 +0530 Subject: [PATCH 063/224] Update CONTRIBUTING.md, and move in-depth reference into INTRODUCTION.md --- CONTRIBUTING.md | 54 ++--- INTRODUCTION.md | 546 ++++++++++++++++++++++++++++++++++++++++++++++ README.md | 564 +----------------------------------------------- 3 files changed, 580 insertions(+), 584 deletions(-) create mode 100644 INTRODUCTION.md diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 3514b495..fe5a38fd 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -7,33 +7,11 @@ which is hosted by [Confluent Inc.](https://github.com/confluentinc) on GitHub. This document lists rules, guidelines, and help getting started, so if you feel something is missing feel free to send a pull request. -#### Table Of Contents - -[What should I know before I get started?](#what-should-i-know-before-i-get-started) - * [Contributor Agreement](#contributor-agreement) - -[How Can I Contribute?](#how-can-i-contribute) - * [Reporting Bugs](#reporting-bugs) - * [Suggesting Enhancements](#suggesting-enhancements) - * [Pull Requests](#pull-requests) - -[Styleguides](#styleguides) - * [Git Commit Messages](#git-commit-messages) - * [JavaScript Styleguide](#javascript-styleguide) - * [C++ Styleguide](#c++-styleguide) - * [Specs Styleguide](#specs-styleguide) - * [Documentation Styleguide](#documentation-styleguide) - -[Debugging](#debugging) - * [Debugging C++](#debugging-c) - -[Updating librdkafka version](#updating-librdkafka-version) - ## What should I know before I get started? ### Contributor Agreement -Not currently required. +Required (please follow instructions after making any Pull Requests). ## How can I contribute? @@ -49,6 +27,10 @@ replicas, partitions, and brokers you are connecting to, because some issues might be related to Kafka. A list of `librdkafka` configuration key-value pairs also helps. +Adding the property `debug` in your `librdkafka` configuration will help us. A list of +possible values is available [here](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md), +but you can set it to `all` if verbose logs are okay. + ### Suggesting Enhancements Please use __Github Issues__ to suggest enhancements. We are happy to consider @@ -59,6 +41,9 @@ library's core. ### Pull Requests +NOTE: Pull requests while the library in EA are discouraged, as we are still +working on the API and may make breaking changes. + * Include new test cases (either end-to-end or unit tests) with your change. * Follow our style guides. * Make sure all tests are still passing and the `linter` does not report any issues. @@ -114,7 +99,8 @@ In short: ### Specs Styleguide -* Write all JavaScript tests by using the `mocha` testing framework. +* Write JavaScript tests by using the `mocha` testing framework for the + non-promisified API and `jest` for the promisified API. * All `mocha` tests should use exports syntax. * All `mocha` test files should be suffixed with `.spec.js` instead of `.js`. * Unit tests should mirror the JavaScript files they test (for example, @@ -169,6 +155,26 @@ Using Visual Studio code to develop on `confluent-kafka-js`. If you use it you c } ``` +## Tests + +This project includes three types of tests in this project: +* end-to-end integration tests (`mocha`) +* unit tests (`mocha`) +* integration tests for promisified API (`jest`) + +You can run all types of tests by using `Makefile`. Doing so calls `mocha` or `jest` in your locally installed `node_modules` directory. + +* Before you run the tests, be sure to init and update the submodules: + 1. `git submodule init` + 2. `git submodule update` +* To run the unit tests, you can run `make lint` or `make test`. +* To run the promisified integration tests, you can use `make promisified_test`. + You must have a running Kafka installation available. By default, the test tries to connect to `localhost:9092`; + however, you can supply the `KAFKA_HOST` environment variable to override this default behavior. +* To run the integration tests, you can use `make e2e`. + You must have a running Kafka installation available. By default, the test tries to connect to `localhost:9092`; + however, you can supply the `KAFKA_HOST` environment variable to override this default behavior. Run `make e2e`. + ## Debugging ### Debugging C++ diff --git a/INTRODUCTION.md b/INTRODUCTION.md new file mode 100644 index 00000000..a1c09318 --- /dev/null +++ b/INTRODUCTION.md @@ -0,0 +1,546 @@ +# Introduction to Confluent-Kafka-JS + +## Configuration + +You can pass many configuration options to `librdkafka`. A full list can be found in `librdkafka`'s [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) + +Configuration keys that have the suffix `_cb` are designated as callbacks. Some +of these keys are informational and you can choose to opt-in (for example, `dr_cb`). Others are callbacks designed to +return a value, such as `partitioner_cb`. + +Not all of these options are supported. +The library will throw an error if the value you send in is invalid. + +The library currently supports the following callbacks: +* `partitioner_cb` +* `dr_cb` or `dr_msg_cb` +* `event_cb` +* `rebalance_cb` (see [Rebalancing](#rebalancing)) +* `offset_commit_cb` (see [Commits](#commits)) + +### Librdkafka Methods + +This library includes two utility functions for detecting the status of your installation. Please try to include these when making issue reports where applicable. + +You can get the features supported by your compile of `librdkafka` by reading the variable "features" on the root of the `confluent-kafka-js` object. + +```js +const Kafka = require('confluent-kafka-js'); +console.log(Kafka.features); + +// #=> [ 'gzip', 'snappy', 'ssl', 'sasl', 'regex', 'lz4' ] +``` + +You can also get the version of `librdkafka` + +```js +const Kafka = require('confluent-kafka-js'); +console.log(Kafka.librdkafkaVersion); + +// #=> 2.3.0 +``` + +## Usage: promisified API + +Still being written. In the meantime, the [QUICKSTART](./QUICKSTART.md) has a good starting point. + +## Usage: non-promisified API + +### Sending Messages + +A `Producer` sends messages to Kafka. The `Producer` constructor takes a configuration object, as shown in the following example: + +```js +const producer = new Kafka.Producer({ + 'metadata.broker.list': 'kafka-host1:9092,kafka-host2:9092' +}); +``` + +A `Producer` requires only `metadata.broker.list` (the Kafka brokers) to be created. The values in this list are separated by commas. For other configuration options, see the [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) file described previously. + +The following example illustrates a list with several `librdkafka` options set. + +```js +const producer = new Kafka.Producer({ + 'client.id': 'kafka', + 'metadata.broker.list': 'localhost:9092', + 'compression.codec': 'gzip', + 'retry.backoff.ms': 200, + 'message.send.max.retries': 10, + 'socket.keepalive.enable': true, + 'queue.buffering.max.messages': 100000, + 'queue.buffering.max.ms': 1000, + 'batch.num.messages': 1000000, + 'dr_cb': true +}); +``` + +#### Stream API + +You can easily use the `Producer` as a writable stream immediately after creation (as shown in the following example): + +```js +// Our producer with its Kafka brokers +// This call returns a new writable stream to our topic 'topic-name' +const stream = Kafka.Producer.createWriteStream({ + 'metadata.broker.list': 'kafka-host1:9092,kafka-host2:9092' +}, {}, { + topic: 'topic-name' +}); + +// Writes a message to the stream +const queuedSuccess = stream.write(Buffer.from('Awesome message')); + +if (queuedSuccess) { + console.log('We queued our message!'); +} else { + // Note that this only tells us if the stream's queue is full, + // it does NOT tell us if the message got to Kafka! See below... + console.log('Too many messages in our queue already'); +} + +// NOTE: MAKE SURE TO LISTEN TO THIS IF YOU WANT THE STREAM TO BE DURABLE +// Otherwise, any error will bubble up as an uncaught exception. +stream.on('error', (err) => { + // Here's where we'll know if something went wrong sending to Kafka + console.error('Error in our kafka stream'); + console.error(err); +}) +``` + +If you do not want your code to crash when an error happens, ensure you have an `error` listener on the stream. Most errors are not necessarily fatal, but the ones that are will immediately destroy the stream. If you use `autoClose`, the stream will close itself at the first sign of a problem. + +#### Standard API + +The Standard API is more performant, particularly when handling high volumes of messages. +However, it requires more manual setup to use. The following example illustrates its use: + +```js +const producer = new Kafka.Producer({ + 'metadata.broker.list': 'localhost:9092', + 'dr_cb': true +}); + +// Connect to the broker manually +producer.connect(); + +// Wait for the ready event before proceeding +producer.on('ready', () => { + try { + producer.produce( + // Topic to send the message to + 'topic', + // optionally we can manually specify a partition for the message + // this defaults to -1 - which will use librdkafka's default partitioner (consistent random for keyed messages, random for unkeyed messages) + null, + // Message to send. Must be a buffer + Buffer.from('Awesome message'), + // for keyed messages, we also specify the key - note that this field is optional + 'Stormwind', + // you can send a timestamp here. If your broker version supports it, + // it will get added. Otherwise, we default to 0 + Date.now(), + // you can send an opaque token here, which gets passed along + // to your delivery reports + ); + } catch (err) { + console.error('A problem occurred when sending our message'); + console.error(err); + } +}); + +// Any errors we encounter, including connection errors +producer.on('event.error', (err) => { + console.error('Error from producer'); + console.error(err); +}) + +// We must either call .poll() manually after sending messages +// or set the producer to poll on an interval (.setPollInterval). +// Without this, we do not get delivery events and the queue +// will eventually fill up. +producer.setPollInterval(100); +``` + +To see the configuration options available to you, see the [Configuration](#configuration) section. + +##### Methods + +| Method | Description | +|---------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `producer.connect()` | Connects to the broker.

The `connect()` method emits the `ready` event when it connects successfully. If it does not, the error will be passed through the callback. | +| `producer.disconnect()` | Disconnects from the broker.

The `disconnect()` method emits the `disconnected` event when it has disconnected. If it does not, the error will be passed through the callback. | +| `producer.poll()` | Polls the producer for delivery reports or other events to be transmitted via the emitter.

In order to get the events in `librdkafka`'s queue to emit, you must call this regularly. | +| `producer.setPollInterval(interval)` | Polls the producer on this interval, handling disconnections and reconnection. Set it to 0 to turn it off. | +| `producer.produce(topic, partition, msg, key, timestamp, opaque)` | Sends a message.

The `produce()` method throws when produce would return an error. Ordinarily, this is just if the queue is full. | +| `producer.flush(timeout, callback)` | Flush the librdkafka internal queue, sending all messages. Default timeout is 500ms | +| `producer.initTransactions(timeout, callback)` | Initializes the transactional producer. | +| `producer.beginTransaction(callback)` | Starts a new transaction. | +| `producer.sendOffsetsToTransaction(offsets, consumer, timeout, callback)` | Sends consumed topic-partition-offsets to the broker, which will get committed along with the transaction. | +| `producer.abortTransaction(timeout, callback)` | Aborts the ongoing transaction. | +| `producer.commitTransaction(timeout, callback)` | Commits the ongoing transaction. | + +##### Events + +Some configuration properties that end in `_cb` indicate that an event should be generated for that option. You can either: + +* provide a value of `true` and react to the event +* provide a callback function directly + +The following example illustrates an event: + +```js +const producer = new Kafka.Producer({ + 'client.id': 'my-client', // Specifies an identifier to use to help trace activity in Kafka + 'metadata.broker.list': 'localhost:9092', // Connect to a Kafka instance on localhost + 'dr_cb': true // Specifies that we want a delivery-report event to be generated +}); + +// Poll for events every 100 ms +producer.setPollInterval(100); + +producer.on('delivery-report', (err, report) => { + // Report of delivery statistics here: + // + console.log(report); +}); +``` + +The following table describes types of events. + +| Event | Description | +|-------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `disconnected` | The `disconnected` event is emitted when the broker has disconnected.

This event is emitted only when `.disconnect` is called. The wrapper will always try to reconnect otherwise. | +| `ready` | The `ready` event is emitted when the `Producer` is ready to send messages. | +| `event` | The `event` event is emitted when `librdkafka` reports an event (if you opted in via the `event_cb` option). | +| `event.log` | The `event.log` event is emitted when logging events come in (if you opted into logging via the `event_cb` option).

You will need to set a value for `debug` if you want to send information. | +| `event.stats` | The `event.stats` event is emitted when `librdkafka` reports stats (if you opted in by setting the `statistics.interval.ms` to a non-zero value). | +| `event.error` | The `event.error` event is emitted when `librdkafka` reports an error | +| `event.throttle` | The `event.throttle` event emitted when `librdkafka` reports throttling. | +| `delivery-report` | The `delivery-report` event is emitted when a delivery report has been found via polling.

To use this event, you must set `request.required.acks` to `1` or `-1` in topic configuration and `dr_cb` (or `dr_msg_cb` if you want the report to contain the message payload) to `true` in the `Producer` constructor options. | + +#### Higher Level Producer + +The higher level producer is a variant of the producer which can propagate callbacks to you upon message delivery. + +```js +const producer = new Kafka.HighLevelProducer({ + 'metadata.broker.list': 'localhost:9092', +}); +``` + +This will enrich the produce call so it will have a callback to tell you when the message has been delivered. You lose the ability to specify opaque tokens. + +```js +producer.produce(topicName, null, Buffer.from('alliance4ever'), null, Date.now(), (err, offset) => { + // The offset if our acknowledgement level allows us to receive delivery offsets + console.log(offset); +}); +``` + +Additionally you can add serializers to modify the value of a produce for a key or value before it is sent over to Kafka. + +```js +producer.setValueSerializer((value) => { + return Buffer.from(JSON.stringify(value)); +}); +``` + +Otherwise the behavior of the class should be exactly the same. + +### Kafka.KafkaConsumer + +To read messages from Kafka, you use a `KafkaConsumer`. You instantiate a `KafkaConsumer` object as follows: + +```js +const consumer = new Kafka.KafkaConsumer({ + 'group.id': 'kafka', + 'metadata.broker.list': 'localhost:9092', +}, {}); +``` + +The first parameter is the global config, while the second parameter is the topic config that gets applied to all subscribed topics. To view a list of all supported configuration properties, see the [Configuration.md](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) file described previously. Look for the `C` and `*` keys. + +The `group.id` and `metadata.broker.list` properties are required for a consumer. + +#### Rebalancing + +Rebalancing is managed internally by `librdkafka` by default. If you would like to override this functionality, you may provide your own logic as a rebalance callback. + +```js +const consumer = new Kafka.KafkaConsumer({ + 'group.id': 'kafka', + 'metadata.broker.list': 'localhost:9092', + 'rebalance_cb': (err, assignment) => { + + if (err.code === Kafka.CODES.ERRORS.ERR__ASSIGN_PARTITIONS) { + // Note: this can throw when you are disconnected. Take care and wrap it in + // a try catch if that matters to you + this.assign(assignment); + } else if (err.code == Kafka.CODES.ERRORS.ERR__REVOKE_PARTITIONS){ + // Same as above + this.unassign(); + } else { + // We had a real error + console.error(err); + } + + } +}) +``` + +`this` is bound to the `KafkaConsumer` you have created. By specifying a `rebalance_cb` you can also listen to the `rebalance` event as an emitted event. This event is not emitted when using the internal `librdkafka` rebalancer. + +#### Commits + +When you commit in `confluent-kafka-js`, the standard way is to queue the commit request up with the next `librdkafka` request to the broker. When doing this, there isn't a way to know the result of the commit. Luckily there is another callback you can listen to to get this information + +```js +const consumer = new Kafka.KafkaConsumer({ + 'group.id': 'kafka', + 'metadata.broker.list': 'localhost:9092', + 'offset_commit_cb': (err, topicPartitions) => { + + if (err) { + // There was an error committing + console.error(err); + } else { + // Commit went through. Let's log the topic partitions + console.log(topicPartitions); + } + + } +}) +``` + +`this` is bound to the `KafkaConsumer` you have created. By specifying an `offset_commit_cb` you can also listen to the `offset.commit` event as an emitted event. It receives an error and the list of topic partitions as argument. This is not emitted unless opted in. + +#### Message Structure + +Messages that are returned by the `KafkaConsumer` have the following structure. + +```js +{ + value: Buffer.from('hi'), // message contents as a Buffer + size: 2, // size of the message, in bytes + topic: 'librdtesting-01', // topic the message comes from + offset: 1337, // offset the message was read from + partition: 1, // partition the message was on + key: 'someKey', // key of the message if present + timestamp: 1510325354780 // timestamp of message creation +} +``` + +#### Stream API + +The stream API is the easiest way to consume messages. The following example illustrates the use of the stream API: + +```js +// Read from the librdtesting-01 topic... note that this creates a new stream on each call! +const stream = KafkaConsumer.createReadStream(globalConfig, topicConfig, { + topics: ['librdtesting-01'] +}); + +stream.on('data', (message) => { + console.log('Got message'); + console.log(message.value.toString()); +}); +``` + +You can also get the `consumer` from the streamConsumer, for using consumer methods. The following example illustrates that: + +```js +stream.consumer.commit(); // Commits all locally stored offsets +``` + +#### Standard API + +You can also use the Standard API and manage callbacks and events yourself. You can choose different modes for consuming messages: + +* *Flowing mode*. This mode flows all of the messages it can read by maintaining an infinite loop in the event loop. It only stops when it detects the consumer has issued the `unsubscribe` or `disconnect` method. +* *Non-flowing mode*. This mode reads a single message from Kafka at a time manually. + +The following example illustrates flowing mode: +```js +// Flowing mode +consumer.connect(); + +consumer + .on('ready', () => { + consumer.subscribe(['librdtesting-01']); + + // Consume from the librdtesting-01 topic. This is what determines + // the mode we are running in. By not specifying a callback (or specifying + // only a callback) we get messages as soon as they are available. + consumer.consume(); + }) + .on('data', (data) => { + // Output the actual message contents + console.log(data.value.toString()); + }); +``` +The following example illustrates non-flowing mode: +```js +// Non-flowing mode +consumer.connect(); + +consumer + .on('ready', () => { + // Subscribe to the librdtesting-01 topic + // This makes subsequent consumes read from that topic. + consumer.subscribe(['librdtesting-01']); + + // Read one message every 1000 milliseconds + setInterval(() => { + consumer.consume(1); + }, 1000); + }) + .on('data', (data) => { + console.log('Message found! Contents below.'); + console.log(data.value.toString()); + }); +``` + +The following table lists important methods for this API. + +| Method | Description | +|-----------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `consumer.connect()` | Connects to the broker.

The `connect()` emits the event `ready` when it has successfully connected. If it does not, the error will be passed through the callback. | +| `consumer.disconnect()` | Disconnects from the broker.

The `disconnect()` method emits `disconnected` when it has disconnected. If it does not, the error will be passed through the callback. | +| `consumer.subscribe(topics)` | Subscribes to an array of topics. | +| `consumer.unsubscribe()` | Unsubscribes from the currently subscribed topics.

You cannot subscribe to different topics without calling the `unsubscribe()` method first. | +| `consumer.consume(cb)` | Gets messages from the existing subscription as quickly as possible. If `cb` is specified, invokes `cb(err, message)`.

This method keeps a background thread running to do the work. Note that the number of threads in nodejs process is limited by `UV_THREADPOOL_SIZE` (default value is 4) and using up all of them blocks other parts of the application that need threads. If you need multiple consumers then consider increasing `UV_THREADPOOL_SIZE` or using `consumer.consume(number, cb)` instead. | +| `consumer.consume(number, cb)` | Gets `number` of messages from the existing subscription. If `cb` is specified, invokes `cb(err, message)`. | +| `consumer.commit()` | Commits all locally stored offsets | +| `consumer.commit(topicPartition)` | Commits offsets specified by the topic partition | +| `consumer.commitMessage(message)` | Commits the offsets specified by the message | + +The following table lists events for this API. + +| Event | Description | +|------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `data` | When using the Standard API consumed messages are emitted in this event. | +| `partition.eof` | When using Standard API and the configuration option `enable.partition.eof` is set, `partition.eof` events are emitted in this event. The event contains `topic`, `partition` and `offset` properties. | +| `warning` | The event is emitted in case of `UNKNOWN_TOPIC_OR_PART` or `TOPIC_AUTHORIZATION_FAILED` errors when consuming in *Flowing mode*. Since the consumer will continue working if the error is still happening, the warning event should reappear after the next metadata refresh. To control the metadata refresh rate set `topic.metadata.refresh.interval.ms` property. Once you resolve the error, you can manually call `getMetadata` to speed up consumer recovery. | +| `disconnected` | The `disconnected` event is emitted when the broker disconnects.

This event is only emitted when `.disconnect` is called. The wrapper will always try to reconnect otherwise. | +| `ready` | The `ready` event is emitted when the `Consumer` is ready to read messages. | +| `event` | The `event` event is emitted when `librdkafka` reports an event (if you opted in via the `event_cb` option). | +| `event.log` | The `event.log` event is emitted when logging events occur (if you opted in for logging via the `event_cb` option).

You will need to set a value for `debug` if you want information to send. | +| `event.stats` | The `event.stats` event is emitted when `librdkafka` reports stats (if you opted in by setting the `statistics.interval.ms` to a non-zero value). | +| `event.error` | The `event.error` event is emitted when `librdkafka` reports an error | +| `event.throttle` | The `event.throttle` event is emitted when `librdkafka` reports throttling. | + +### Reading current offsets from the broker for a topic + +Some times you find yourself in the situation where you need to know the latest (and earliest) offset for one of your topics. Connected producers and consumers both allow you to query for these through `queryWaterMarkOffsets` like follows: + +```js +const timeout = 5000, partition = 0; +consumer.queryWatermarkOffsets('my-topic', partition, timeout, (err, offsets) => { + const high = offsets.highOffset; + const low = offsets.lowOffset; +}); + +producer.queryWatermarkOffsets('my-topic', partition, timeout, (err, offsets) => { + const high = offsets.highOffset; + const low = offsets.lowOffset; +}); + +An error will be returned if the client was not connected or the request timed out within the specified interval. + +``` + +### Metadata + +Both `Kafka.Producer` and `Kafka.KafkaConsumer` include a `getMetadata` method to retrieve metadata from Kafka. + +Getting metadata on any connection returns the following data structure: + +```js +{ + orig_broker_id: 1, + orig_broker_name: "broker_name", + brokers: [ + { + id: 1, + host: 'localhost', + port: 40 + } + ], + topics: [ + { + name: 'awesome-topic', + partitions: [ + { + id: 1, + leader: 20, + replicas: [1, 2], + isrs: [1, 2] + } + ] + } + ] +} +``` + +The following example illustrates how to use the `getMetadata` method. + +When fetching metadata for a specific topic, if a topic reference does not exist, one is created using the default config. +Please see the documentation on `Client.getMetadata` if you want to set configuration parameters, e.g. `acks`, on a topic to produce messages to. + +```js +const opts = { + topic: 'librdtesting-01', + timeout: 10000 +}; + +producer.getMetadata(opts, (err, metadata) => { + if (err) { + console.error('Error getting metadata'); + console.error(err); + } else { + console.log('Got metadata'); + console.log(metadata); + } +}); +``` + +### Admin Client + +`confluent-kafka-js` now supports the admin client for creating, deleting, and scaling out topics. The `librdkafka` APIs also support altering configuration of topics and broker, but that is not currently implemented. + +To create an Admin client, you can do as follows: + +```js +const Kafka = require('confluent-kafka-js'); + +const client = Kafka.AdminClient.create({ + 'client.id': 'kafka-admin', + 'metadata.broker.list': 'broker01' +}); +``` + +This will instantiate the `AdminClient`, which will allow the calling of the admin methods. + +```js +client.createTopic({ + topic: topicName, + num_partitions: 1, + replication_factor: 1 +}, (err) => { + // Done! +}); +``` + +All of the admin api methods can have an optional timeout as their penultimate parameter. + +The following table lists important methods for this API. + +| Method | Description | +|----------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------| +| `client.disconnect()` | Destroy the admin client, making it invalid for further use. | +| `client.createTopic(topic, timeout, cb)` | Create a topic on the broker with the given configuration. See JS doc for more on structure of the topic object | +| `client.deleteTopic(topicName, timeout, cb)` | Delete a topic of the given name | +| `client.createPartitions(topicName, desiredPartitions, timeout, cb)` | Create partitions until the topic has the desired number of partitions. | + +Check the tests for an example of how to use this API! diff --git a/README.md b/README.md index 3b1a12a7..ef51aeae 100644 --- a/README.md +++ b/README.md @@ -50,569 +50,13 @@ Yarn and pnpm support is experimental. # Getting Started -1. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md). -2. If you're migrating from `kafkajs`, you can use the [migration guide](MIGRATION.md#kafkajs). +1. If you're migrating from `kafkajs`, you can use the [migration guide](MIGRATION.md#kafkajs). 2. If you're migrating from `node-rdkafka`, you can use the [migration guide](MIGRATION.md#node-rdkafka). +3. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md). +An in-depth reference may be found at [INTRODUCTION.md](INTRODUCTION.md). ## Contributing -Bug reports and early feedback is appreciated in the form of Github Issues. +Bug reports and early-access feedback is appreciated in the form of Github Issues. For guidelines on contributing please see [CONTRIBUTING.md](CONTRIBUTING.md) - -# Usage - -## Configuration - -You can pass many configuration options to `librdkafka`. A full list can be found in `librdkafka`'s [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) - -Configuration keys that have the suffix `_cb` are designated as callbacks. Some -of these keys are informational and you can choose to opt-in (for example, `dr_cb`). Others are callbacks designed to -return a value, such as `partitioner_cb`. - -Not all of these options are supported. -The library will throw an error if the value you send in is invalid. - -The library currently supports the following callbacks: -* `partitioner_cb` -* `dr_cb` or `dr_msg_cb` -* `event_cb` -* `rebalance_cb` (see [Rebalancing](#rebalancing)) -* `offset_commit_cb` (see [Commits](#commits)) - -(README below this point TBA - it's just the older stuff). - -### Librdkafka Methods - -This library includes two utility functions for detecting the status of your installation. Please try to include these when making issue reports where applicable. - -You can get the features supported by your compile of `librdkafka` by reading the variable "features" on the root of the `confluent-kafka-js` object. - -```js -const Kafka = require('confluent-kafka-js'); -console.log(Kafka.features); - -// #=> [ 'gzip', 'snappy', 'ssl', 'sasl', 'regex', 'lz4' ] -``` - -You can also get the version of `librdkafka` - -```js -const Kafka = require('confluent-kafka-js'); -console.log(Kafka.librdkafkaVersion); - -// #=> 2.3.0 -``` - -## Sending Messages - -A `Producer` sends messages to Kafka. The `Producer` constructor takes a configuration object, as shown in the following example: - -```js -const producer = new Kafka.Producer({ - 'metadata.broker.list': 'kafka-host1:9092,kafka-host2:9092' -}); -``` - -A `Producer` requires only `metadata.broker.list` (the Kafka brokers) to be created. The values in this list are separated by commas. For other configuration options, see the [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) file described previously. - -The following example illustrates a list with several `librdkafka` options set. - -```js -const producer = new Kafka.Producer({ - 'client.id': 'kafka', - 'metadata.broker.list': 'localhost:9092', - 'compression.codec': 'gzip', - 'retry.backoff.ms': 200, - 'message.send.max.retries': 10, - 'socket.keepalive.enable': true, - 'queue.buffering.max.messages': 100000, - 'queue.buffering.max.ms': 1000, - 'batch.num.messages': 1000000, - 'dr_cb': true -}); -``` - -#### Stream API - -You can easily use the `Producer` as a writable stream immediately after creation (as shown in the following example): - -```js -// Our producer with its Kafka brokers -// This call returns a new writable stream to our topic 'topic-name' -const stream = Kafka.Producer.createWriteStream({ - 'metadata.broker.list': 'kafka-host1:9092,kafka-host2:9092' -}, {}, { - topic: 'topic-name' -}); - -// Writes a message to the stream -const queuedSuccess = stream.write(Buffer.from('Awesome message')); - -if (queuedSuccess) { - console.log('We queued our message!'); -} else { - // Note that this only tells us if the stream's queue is full, - // it does NOT tell us if the message got to Kafka! See below... - console.log('Too many messages in our queue already'); -} - -// NOTE: MAKE SURE TO LISTEN TO THIS IF YOU WANT THE STREAM TO BE DURABLE -// Otherwise, any error will bubble up as an uncaught exception. -stream.on('error', (err) => { - // Here's where we'll know if something went wrong sending to Kafka - console.error('Error in our kafka stream'); - console.error(err); -}) -``` - -If you do not want your code to crash when an error happens, ensure you have an `error` listener on the stream. Most errors are not necessarily fatal, but the ones that are will immediately destroy the stream. If you use `autoClose`, the stream will close itself at the first sign of a problem. - -#### Standard API - -The Standard API is more performant, particularly when handling high volumes of messages. -However, it requires more manual setup to use. The following example illustrates its use: - -```js -const producer = new Kafka.Producer({ - 'metadata.broker.list': 'localhost:9092', - 'dr_cb': true -}); - -// Connect to the broker manually -producer.connect(); - -// Wait for the ready event before proceeding -producer.on('ready', () => { - try { - producer.produce( - // Topic to send the message to - 'topic', - // optionally we can manually specify a partition for the message - // this defaults to -1 - which will use librdkafka's default partitioner (consistent random for keyed messages, random for unkeyed messages) - null, - // Message to send. Must be a buffer - Buffer.from('Awesome message'), - // for keyed messages, we also specify the key - note that this field is optional - 'Stormwind', - // you can send a timestamp here. If your broker version supports it, - // it will get added. Otherwise, we default to 0 - Date.now(), - // you can send an opaque token here, which gets passed along - // to your delivery reports - ); - } catch (err) { - console.error('A problem occurred when sending our message'); - console.error(err); - } -}); - -// Any errors we encounter, including connection errors -producer.on('event.error', (err) => { - console.error('Error from producer'); - console.error(err); -}) - -// We must either call .poll() manually after sending messages -// or set the producer to poll on an interval (.setPollInterval). -// Without this, we do not get delivery events and the queue -// will eventually fill up. -producer.setPollInterval(100); -``` - -To see the configuration options available to you, see the [Configuration](#configuration) section. - -##### Methods - -|Method|Description| -|-------|----------| -|`producer.connect()`| Connects to the broker.

The `connect()` method emits the `ready` event when it connects successfully. If it does not, the error will be passed through the callback. | -|`producer.disconnect()`| Disconnects from the broker.

The `disconnect()` method emits the `disconnected` event when it has disconnected. If it does not, the error will be passed through the callback. | -|`producer.poll()` | Polls the producer for delivery reports or other events to be transmitted via the emitter.

In order to get the events in `librdkafka`'s queue to emit, you must call this regularly. | -|`producer.setPollInterval(interval)` | Polls the producer on this interval, handling disconnections and reconnection. Set it to 0 to turn it off. | -|`producer.produce(topic, partition, msg, key, timestamp, opaque)`| Sends a message.

The `produce()` method throws when produce would return an error. Ordinarily, this is just if the queue is full. | -|`producer.flush(timeout, callback)`| Flush the librdkafka internal queue, sending all messages. Default timeout is 500ms | -|`producer.initTransactions(timeout, callback)`| Initializes the transactional producer. | -|`producer.beginTransaction(callback)`| Starts a new transaction. | -|`producer.sendOffsetsToTransaction(offsets, consumer, timeout, callback)`| Sends consumed topic-partition-offsets to the broker, which will get committed along with the transaction. | -|`producer.abortTransaction(timeout, callback)`| Aborts the ongoing transaction. | -|`producer.commitTransaction(timeout, callback)`| Commits the ongoing transaction. | - -##### Events - -Some configuration properties that end in `_cb` indicate that an event should be generated for that option. You can either: - -* provide a value of `true` and react to the event -* provide a callback function directly - -The following example illustrates an event: - -```js -const producer = new Kafka.Producer({ - 'client.id': 'my-client', // Specifies an identifier to use to help trace activity in Kafka - 'metadata.broker.list': 'localhost:9092', // Connect to a Kafka instance on localhost - 'dr_cb': true // Specifies that we want a delivery-report event to be generated -}); - -// Poll for events every 100 ms -producer.setPollInterval(100); - -producer.on('delivery-report', (err, report) => { - // Report of delivery statistics here: - // - console.log(report); -}); -``` - -The following table describes types of events. - -|Event|Description| -|-------|----------| -| `disconnected` | The `disconnected` event is emitted when the broker has disconnected.

This event is emitted only when `.disconnect` is called. The wrapper will always try to reconnect otherwise. | -| `ready` | The `ready` event is emitted when the `Producer` is ready to send messages. | -| `event` | The `event` event is emitted when `librdkafka` reports an event (if you opted in via the `event_cb` option). | -| `event.log` | The `event.log` event is emitted when logging events come in (if you opted into logging via the `event_cb` option).

You will need to set a value for `debug` if you want to send information. | -| `event.stats` | The `event.stats` event is emitted when `librdkafka` reports stats (if you opted in by setting the `statistics.interval.ms` to a non-zero value). | -| `event.error` | The `event.error` event is emitted when `librdkafka` reports an error | -| `event.throttle` | The `event.throttle` event emitted when `librdkafka` reports throttling. | -| `delivery-report` | The `delivery-report` event is emitted when a delivery report has been found via polling.

To use this event, you must set `request.required.acks` to `1` or `-1` in topic configuration and `dr_cb` (or `dr_msg_cb` if you want the report to contain the message payload) to `true` in the `Producer` constructor options. | - -### Higher Level Producer - -The higher level producer is a variant of the producer which can propagate callbacks to you upon message delivery. - -```js -const producer = new Kafka.HighLevelProducer({ - 'metadata.broker.list': 'localhost:9092', -}); -``` - -This will enrich the produce call so it will have a callback to tell you when the message has been delivered. You lose the ability to specify opaque tokens. - -```js -producer.produce(topicName, null, Buffer.from('alliance4ever'), null, Date.now(), (err, offset) => { - // The offset if our acknowledgement level allows us to receive delivery offsets - console.log(offset); -}); -``` - -Additionally you can add serializers to modify the value of a produce for a key or value before it is sent over to Kafka. - -```js -producer.setValueSerializer((value) => { - return Buffer.from(JSON.stringify(value)); -}); -``` - -Otherwise the behavior of the class should be exactly the same. - -## Kafka.KafkaConsumer - -To read messages from Kafka, you use a `KafkaConsumer`. You instantiate a `KafkaConsumer` object as follows: - -```js -const consumer = new Kafka.KafkaConsumer({ - 'group.id': 'kafka', - 'metadata.broker.list': 'localhost:9092', -}, {}); -``` - -The first parameter is the global config, while the second parameter is the topic config that gets applied to all subscribed topics. To view a list of all supported configuration properties, see the [Configuration.md](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) file described previously. Look for the `C` and `*` keys. - -The `group.id` and `metadata.broker.list` properties are required for a consumer. - -### Rebalancing - -Rebalancing is managed internally by `librdkafka` by default. If you would like to override this functionality, you may provide your own logic as a rebalance callback. - -```js -const consumer = new Kafka.KafkaConsumer({ - 'group.id': 'kafka', - 'metadata.broker.list': 'localhost:9092', - 'rebalance_cb': (err, assignment) => { - - if (err.code === Kafka.CODES.ERRORS.ERR__ASSIGN_PARTITIONS) { - // Note: this can throw when you are disconnected. Take care and wrap it in - // a try catch if that matters to you - this.assign(assignment); - } else if (err.code == Kafka.CODES.ERRORS.ERR__REVOKE_PARTITIONS){ - // Same as above - this.unassign(); - } else { - // We had a real error - console.error(err); - } - - } -}) -``` - -`this` is bound to the `KafkaConsumer` you have created. By specifying a `rebalance_cb` you can also listen to the `rebalance` event as an emitted event. This event is not emitted when using the internal `librdkafka` rebalancer. - -### Commits - -When you commit in `confluent-kafka-js`, the standard way is to queue the commit request up with the next `librdkafka` request to the broker. When doing this, there isn't a way to know the result of the commit. Luckily there is another callback you can listen to to get this information - -```js -const consumer = new Kafka.KafkaConsumer({ - 'group.id': 'kafka', - 'metadata.broker.list': 'localhost:9092', - 'offset_commit_cb': (err, topicPartitions) => { - - if (err) { - // There was an error committing - console.error(err); - } else { - // Commit went through. Let's log the topic partitions - console.log(topicPartitions); - } - - } -}) -``` - -`this` is bound to the `KafkaConsumer` you have created. By specifying an `offset_commit_cb` you can also listen to the `offset.commit` event as an emitted event. It receives an error and the list of topic partitions as argument. This is not emitted unless opted in. - -### Message Structure - -Messages that are returned by the `KafkaConsumer` have the following structure. - -```js -{ - value: Buffer.from('hi'), // message contents as a Buffer - size: 2, // size of the message, in bytes - topic: 'librdtesting-01', // topic the message comes from - offset: 1337, // offset the message was read from - partition: 1, // partition the message was on - key: 'someKey', // key of the message if present - timestamp: 1510325354780 // timestamp of message creation -} -``` - -### Stream API - -The stream API is the easiest way to consume messages. The following example illustrates the use of the stream API: - -```js -// Read from the librdtesting-01 topic... note that this creates a new stream on each call! -const stream = KafkaConsumer.createReadStream(globalConfig, topicConfig, { - topics: ['librdtesting-01'] -}); - -stream.on('data', (message) => { - console.log('Got message'); - console.log(message.value.toString()); -}); -``` - -You can also get the `consumer` from the streamConsumer, for using consumer methods. The following example illustrates that: - -```js -stream.consumer.commit(); // Commits all locally stored offsets -``` - -### Standard API - -You can also use the Standard API and manage callbacks and events yourself. You can choose different modes for consuming messages: - -* *Flowing mode*. This mode flows all of the messages it can read by maintaining an infinite loop in the event loop. It only stops when it detects the consumer has issued the `unsubscribe` or `disconnect` method. -* *Non-flowing mode*. This mode reads a single message from Kafka at a time manually. - -The following example illustrates flowing mode: -```js -// Flowing mode -consumer.connect(); - -consumer - .on('ready', () => { - consumer.subscribe(['librdtesting-01']); - - // Consume from the librdtesting-01 topic. This is what determines - // the mode we are running in. By not specifying a callback (or specifying - // only a callback) we get messages as soon as they are available. - consumer.consume(); - }) - .on('data', (data) => { - // Output the actual message contents - console.log(data.value.toString()); - }); -``` -The following example illustrates non-flowing mode: -```js -// Non-flowing mode -consumer.connect(); - -consumer - .on('ready', () => { - // Subscribe to the librdtesting-01 topic - // This makes subsequent consumes read from that topic. - consumer.subscribe(['librdtesting-01']); - - // Read one message every 1000 milliseconds - setInterval(() => { - consumer.consume(1); - }, 1000); - }) - .on('data', (data) => { - console.log('Message found! Contents below.'); - console.log(data.value.toString()); - }); -``` - -The following table lists important methods for this API. - -|Method|Description| -|-------|----------| -|`consumer.connect()` | Connects to the broker.

The `connect()` emits the event `ready` when it has successfully connected. If it does not, the error will be passed through the callback. | -|`consumer.disconnect()` | Disconnects from the broker.

The `disconnect()` method emits `disconnected` when it has disconnected. If it does not, the error will be passed through the callback. | -|`consumer.subscribe(topics)` | Subscribes to an array of topics. | -|`consumer.unsubscribe()` | Unsubscribes from the currently subscribed topics.

You cannot subscribe to different topics without calling the `unsubscribe()` method first. | -|`consumer.consume(cb)` | Gets messages from the existing subscription as quickly as possible. If `cb` is specified, invokes `cb(err, message)`.

This method keeps a background thread running to do the work. Note that the number of threads in nodejs process is limited by `UV_THREADPOOL_SIZE` (default value is 4) and using up all of them blocks other parts of the application that need threads. If you need multiple consumers then consider increasing `UV_THREADPOOL_SIZE` or using `consumer.consume(number, cb)` instead. | -|`consumer.consume(number, cb)` | Gets `number` of messages from the existing subscription. If `cb` is specified, invokes `cb(err, message)`. | -|`consumer.commit()` | Commits all locally stored offsets | -|`consumer.commit(topicPartition)` | Commits offsets specified by the topic partition | -|`consumer.commitMessage(message)` | Commits the offsets specified by the message | - -The following table lists events for this API. - -|Event|Description| -|-------|----------| -|`data` | When using the Standard API consumed messages are emitted in this event. | -|`partition.eof` | When using Standard API and the configuration option `enable.partition.eof` is set, `partition.eof` events are emitted in this event. The event contains `topic`, `partition` and `offset` properties. | -|`warning` | The event is emitted in case of `UNKNOWN_TOPIC_OR_PART` or `TOPIC_AUTHORIZATION_FAILED` errors when consuming in *Flowing mode*. Since the consumer will continue working if the error is still happening, the warning event should reappear after the next metadata refresh. To control the metadata refresh rate set `topic.metadata.refresh.interval.ms` property. Once you resolve the error, you can manually call `getMetadata` to speed up consumer recovery. | -|`disconnected` | The `disconnected` event is emitted when the broker disconnects.

This event is only emitted when `.disconnect` is called. The wrapper will always try to reconnect otherwise. | -|`ready` | The `ready` event is emitted when the `Consumer` is ready to read messages. | -|`event` | The `event` event is emitted when `librdkafka` reports an event (if you opted in via the `event_cb` option).| -|`event.log` | The `event.log` event is emitted when logging events occur (if you opted in for logging via the `event_cb` option).

You will need to set a value for `debug` if you want information to send. | -|`event.stats` | The `event.stats` event is emitted when `librdkafka` reports stats (if you opted in by setting the `statistics.interval.ms` to a non-zero value). | -|`event.error` | The `event.error` event is emitted when `librdkafka` reports an error | -|`event.throttle` | The `event.throttle` event is emitted when `librdkafka` reports throttling.| - -## Reading current offsets from the broker for a topic - -Some times you find yourself in the situation where you need to know the latest (and earliest) offset for one of your topics. Connected producers and consumers both allow you to query for these through `queryWaterMarkOffsets` like follows: - -```js -const timeout = 5000, partition = 0; -consumer.queryWatermarkOffsets('my-topic', partition, timeout, (err, offsets) => { - const high = offsets.highOffset; - const low = offsets.lowOffset; -}); - -producer.queryWatermarkOffsets('my-topic', partition, timeout, (err, offsets) => { - const high = offsets.highOffset; - const low = offsets.lowOffset; -}); - -An error will be returned if the client was not connected or the request timed out within the specified interval. - -``` - -## Metadata - -Both `Kafka.Producer` and `Kafka.KafkaConsumer` include a `getMetadata` method to retrieve metadata from Kafka. - -Getting metadata on any connection returns the following data structure: - -```js -{ - orig_broker_id: 1, - orig_broker_name: "broker_name", - brokers: [ - { - id: 1, - host: 'localhost', - port: 40 - } - ], - topics: [ - { - name: 'awesome-topic', - partitions: [ - { - id: 1, - leader: 20, - replicas: [1, 2], - isrs: [1, 2] - } - ] - } - ] -} -``` - -The following example illustrates how to use the `getMetadata` method. - -When fetching metadata for a specific topic, if a topic reference does not exist, one is created using the default config. -Please see the documentation on `Client.getMetadata` if you want to set configuration parameters, e.g. `acks`, on a topic to produce messages to. - -```js -const opts = { - topic: 'librdtesting-01', - timeout: 10000 -}; - -producer.getMetadata(opts, (err, metadata) => { - if (err) { - console.error('Error getting metadata'); - console.error(err); - } else { - console.log('Got metadata'); - console.log(metadata); - } -}); -``` - -## Admin Client - -`confluent-kafka-js` now supports the admin client for creating, deleting, and scaling out topics. The `librdkafka` APIs also support altering configuration of topics and broker, but that is not currently implemented. - -To create an Admin client, you can do as follows: - -```js -const Kafka = require('confluent-kafka-js'); - -const client = Kafka.AdminClient.create({ - 'client.id': 'kafka-admin', - 'metadata.broker.list': 'broker01' -}); -``` - -This will instantiate the `AdminClient`, which will allow the calling of the admin methods. - -```js -client.createTopic({ - topic: topicName, - num_partitions: 1, - replication_factor: 1 -}, (err) => { - // Done! -}); -``` - -All of the admin api methods can have an optional timeout as their penultimate parameter. - -The following table lists important methods for this API. - -|Method|Description| -|-------|----------| -|`client.disconnect()` | Destroy the admin client, making it invalid for further use. | -|`client.createTopic(topic, timeout, cb)` | Create a topic on the broker with the given configuration. See JS doc for more on structure of the topic object | -|`client.deleteTopic(topicName, timeout, cb)` | Delete a topic of the given name | -|`client.createPartitions(topicName, desiredPartitions, timeout, cb)` | Create partitions until the topic has the desired number of partitions. | - -Check the tests for an example of how to use this API! - -## Tests - -This project includes two types of unit tests in this project: -* end-to-end integration tests -* unit tests - -You can run both types of tests by using `Makefile`. Doing so calls `mocha` in your locally installed `node_modules` directory. - -* Before you run the tests, be sure to init and update the submodules: - 1. `git submodule init` - 2. `git submodule update` -* To run the unit tests, you can run `make lint` or `make test`. -* To run the integration tests, you must have a running Kafka installation available. By default, the test tries to connect to `localhost:9092`; however, you can supply the `KAFKA_HOST` environment variable to override this default behavior. Run `make e2e`. From 58c34491dfc72757ea8e09dc8a158704d1e25dcc Mon Sep 17 00:00:00 2001 From: Nusair Haq <136121717+nhaq-confluent@users.noreply.github.com> Date: Sun, 21 Jan 2024 09:37:00 -0500 Subject: [PATCH 064/224] Update sr.js (#22) * Update sr.js Noticed there wasn't a KafkaJS block so I added one. Please correct formatting if needed * Update sr.js --- examples/kafkajs/sr.js | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 9c172230..0ccf5a58 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -9,10 +9,22 @@ const { SchemaRegistry, SchemaType } = require('@kafkajs/confluent-schema-regist const registry = new SchemaRegistry({ host: '' }) const kafka = new Kafka({ - brokers: [''], - clientId: 'example-consumer', -}) -let consumer = kafka.consumer({ groupId: 'test-group' , fromBeginning: true, } ); + kafkaJS: { + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, + } +}); +let consumer = kafka.consumer({ + kafkaJS: { + groupId: "test-group", + fromBeginning: false, + }, +}); let producer = kafka.producer(); const schemaA = { From 9fb2203027be498bcaf2cf3c3918c7650cacd838 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 22 Jan 2024 09:10:59 +0530 Subject: [PATCH 065/224] Change fromBeginning in sr.js example --- examples/kafkajs/sr.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 0ccf5a58..49516c04 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -22,7 +22,7 @@ const kafka = new Kafka({ let consumer = kafka.consumer({ kafkaJS: { groupId: "test-group", - fromBeginning: false, + fromBeginning: true, }, }); let producer = kafka.producer(); From c56f0681e6c255ea9e34378a8c03cecc9fd53123 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 22 Jan 2024 09:28:59 +0530 Subject: [PATCH 066/224] Make packaging flow easier --- .semaphore/semaphore.yml | 50 ++++++++++++++++++++++++++++------------ 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index e7fc2f0e..e3f80d8b 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -120,7 +120,7 @@ blocks: - 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}" + - artifact push workflow "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 @@ -129,7 +129,7 @@ blocks: - 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}" + - artifact push workflow "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 @@ -138,7 +138,7 @@ blocks: - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Linux arm64: Release" dependencies: [ ] @@ -164,7 +164,7 @@ blocks: - 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}" + - artifact push workflow "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 @@ -173,7 +173,7 @@ blocks: - 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}" + - artifact push workflow "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 @@ -182,7 +182,7 @@ blocks: - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Linux amd64 musl: Release" dependencies: [ ] @@ -206,21 +206,21 @@ blocks: - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: LTS:20" commands: - export NODE_ABI=115 - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:20-alpine /v/.semaphore/build-docker.sh - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: latest: 21" commands: - export NODE_ABI=120 - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Linux arm64 musl: Release" dependencies: [ ] @@ -244,21 +244,21 @@ blocks: - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: LTS:20" commands: - export NODE_ABI=115 - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:20-alpine /v/.semaphore/build-docker.sh - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: latest: 21" commands: - export NODE_ABI=120 - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "macOS arm64/m1: Release" dependencies: [ ] @@ -284,7 +284,7 @@ blocks: - 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}" + - artifact push workflow "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 @@ -293,7 +293,7 @@ blocks: - 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}" + - artifact push workflow "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 @@ -302,4 +302,24 @@ blocks: - 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}" + - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + + - name: 'Packaging: tar all release artifacts' + dependencies: + - 'Linux amd64: Release' + - 'Linux arm64: Release' + - 'Linux amd64 musl: Release' + - 'Linux arm64 musl: Release' + - 'macOS arm64/m1: Release' + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-2 + jobs: + - name: "Tarball" + commands: + - artifact pull workflow artifacts + - tar -czvf artifacts.${SEMAPHORE_GIT_TAG_NAME}.tar.gz artifacts + - artifact push project "artifacts.${SEMAPHORE_GIT_TAG_NAME}.tar.gz" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/artifacts.${SEMAPHORE_GIT_TAG_NAME}.tar.gz" \ No newline at end of file From d7421ac8f02ed8576dd5300b49b36c8549378f17 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 22 Jan 2024 09:29:34 +0530 Subject: [PATCH 067/224] Bump version --- README.md | 2 +- package.json | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index ef51aeae..cbe7b43e 100644 --- a/README.md +++ b/README.md @@ -43,7 +43,7 @@ Installation on any of these platforms is meant to be seamless, without any C/C+ from GitHub: ```bash -$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.5-devel" +$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.6-devel" ``` Yarn and pnpm support is experimental. diff --git a/package.json b/package.json index 0e38fc10..7024a703 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "confluent-kafka-js", - "version": "v0.1.5-devel", + "version": "v0.1.6-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "main": "lib/index.js", From 3bd39696630acff9d9a086ad7a199094547745f0 Mon Sep 17 00:00:00 2001 From: Nusair Haq <136121717+nhaq-confluent@users.noreply.github.com> Date: Mon, 22 Jan 2024 09:57:14 -0500 Subject: [PATCH 068/224] Update README.md (#23) * Update README.md Updating the Readme to contain more direct messaging about our Early Access. * Update README.md Added suggestions --- README.md | 82 +++++++++++++++++++++++++++++++++++++++---------------- 1 file changed, 59 insertions(+), 23 deletions(-) diff --git a/README.md b/README.md index cbe7b43e..1e3961a8 100644 --- a/README.md +++ b/README.md @@ -1,34 +1,26 @@ Confluent's Javascript Client for Apache KafkaTM ===================================================== -**confluent-kafka-js** is Confluent's Javascript client for [Apache Kafka](http://kafka.apache.org/) and the -[Confluent Platform](https://www.confluent.io/product/compare/). +**confluent-kafka-js** is Confluent's JavaScript client for [Apache Kafka](http://kafka.apache.org/) and the +[Confluent Platform](https://www.confluent.io/product/compare/). This is an **early access** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client that is based on [node-rdkafka](https://github.com/Blizzard/node-rdkafka) yet also API compatible with [KafkaJS](https://github.com/tulios/kafkajs) to provide flexibility to users and streamline migrations from other clients. +This library leverages the work and concepts from two popular Apache Kafka JavaScript clients: [node-rdkafka](https://github.com/Blizzard/node-rdkafka) and [KafkaJS](https://github.com/tulios/kafkajs). The core is heavily based on the node-rdkafka library, which uses our own [librdkafka](https://github.com/confluentinc/librdkafka/tree/v2.3.0) library for core client functionality. However, we leverage a promisified API and a more idiomatic interface, similar to the one in KafkaJS, making it easy for developers to migrate and adopt this client depending on the patterns and interface they prefer. +__This library currently uses `librdkafka` based off of the master branch.__ -Features: - -- **High performance** - confluent-kafka-js is a lightweight wrapper around -[librdkafka](https://github.com/confluentinc/librdkafka), a finely tuned C -client. - -- **Reliability** - There are a lot of details to get right when writing an Apache Kafka -client. We get them right in one place (librdkafka) and leverage this work -across all of our clients (also [confluent-kafka-python](https://github.com/confluentinc/confluent-kafka-python), -[confluent-kafka-go](https://github.com/confluentinc/confluent-kafka-go) and -and [confluent-kafka-dotnet](https://github.com/confluentinc/confluent-kafka-dotnet)). +## This library is currently in early access and not meant for production use -- **Future proof** - Confluent, founded by the -creators of Kafka, is building a [streaming platform](https://www.confluent.io/product/compare/) -with Apache Kafka at its core. It's high priority for us that client features keep -pace with core Apache Kafka and components of the [Confluent Platform](https://www.confluent.io/product/compare/). +**This library is in active development, pre-1.0.0, and it is likely to have many breaking changes.** -## This library is currently not ready for production use. It's an early-access preview in active development, pre-1.0.0, and there might be breaking changes. +For this early-access release, we aim to get feedback from JavaScript developers within the Apache Kafka community to help meet your needs. Some areas of feedback we are looking for include: +- Usability of the API compared to other clients +- Migration experience from the node-rdkafka and KafkaJs +- Overall quality and reliability -This library is based heavily on [node-rdkafka](https://github.com/Blizzard/node-rdkafka). +We invite you to raise issues to highlight any feedback you may have. -This library contains a promisified API, very similar to the one in [kafkajs](https://github.com/tulios/kafkajs). Some of the tests are also based on the ones in kafkajs. +Within the early-access, only **basic produce and consume functionality** as well as the ability to **create and delete topics** are supported. All other admin client functionality is coming in future releases. See [INTRODUCTION.md](INTRODUCTION.md) for more details on what is supported. -__This library currently uses `librdkafka` based off of the master branch.__ +To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-js/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). ## Requirements @@ -39,8 +31,7 @@ The following configurations are supported for this early access preview: * Linux (x64 and arm64) - both glibc and musl/alpine. * macOS - arm64/m1. -Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed -from GitHub: +Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: ```bash $ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.6-devel" @@ -50,6 +41,51 @@ Yarn and pnpm support is experimental. # Getting Started +Below is a simple produce example for users migrating from KafkaJS. + +```javascript +// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +const { Kafka } = require("confluent-kafka-js").KafkaJS; + +async function producerStart() { + const kafka = new Kafka({ + kafkaJS: { + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, + } + }); + + const producer = kafka.producer(); + + await producer.connect(); + + console.log("Connected successfully"); + + const res = [] + for (let i = 0; i < 50; i++) { + res.push(producer.send({ + topic: 'test-topic', + messages: [ + { value: 'v222', partition: 0 }, + { value: 'v11', partition: 0, key: 'x' }, + ] + })); + } + await Promise.all(res); + + await producer.disconnect(); + + console.log("Disconnected successfully"); +} + +producerStart(); +``` + 1. If you're migrating from `kafkajs`, you can use the [migration guide](MIGRATION.md#kafkajs). 2. If you're migrating from `node-rdkafka`, you can use the [migration guide](MIGRATION.md#node-rdkafka). 3. If you're starting afresh, you can use the [quickstart guide](QUICKSTART.md). From b1036e0dd9be55086a5c503db243a994800ff8fb Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 22 Jan 2024 09:29:34 +0530 Subject: [PATCH 069/224] Bump version --- .semaphore/semaphore.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index e3f80d8b..ca2d6435 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -28,7 +28,7 @@ global_job_config: - git submodule update --init --recursive - cd deps/librdkafka - git fetch origin - - git checkout master + - git checkout dev_bump_openssl_version_3_0_12 - cd ../../ - export MKL_DEBUG=1 @@ -320,6 +320,6 @@ blocks: jobs: - name: "Tarball" commands: - - artifact pull workflow artifacts - - tar -czvf artifacts.${SEMAPHORE_GIT_TAG_NAME}.tar.gz artifacts - - artifact push project "artifacts.${SEMAPHORE_GIT_TAG_NAME}.tar.gz" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/artifacts.${SEMAPHORE_GIT_TAG_NAME}.tar.gz" \ No newline at end of file + - artifact pull workflow releases + - tar -czvf releases.${SEMAPHORE_GIT_TAG_NAME}.tar.gz releases + - artifact push project "releases.${SEMAPHORE_GIT_TAG_NAME}.tar.gz" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/releases.${SEMAPHORE_GIT_TAG_NAME}.tar.gz" \ No newline at end of file From 92a6fdc21b72a44dbbd843d67da8f7809e7f40af Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 24 Jan 2024 14:40:18 +0530 Subject: [PATCH 070/224] Poll producer in send() loops --- CONTRIBUTING.md | 10 ++++++++++ lib/kafkajs/_producer.js | 26 +++++++++++++++++++++++--- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index fe5a38fd..472325cd 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -192,6 +192,16 @@ gdb node You can add breakpoints and so on after that. +### Debugging and Profiling JavaScript + +Run the code with the `--inspect` flag, and then open `chrome://inspect` in Chrome and connect to the debugger. + +Example: + +``` +node --inspect path/to/file.js +``` + ## Updating librdkafka version The librdkafka should be periodically updated to the latest release in https://github.com/confluentinc/librdkafka/releases diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 42b9bee9..3e4e939d 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -25,12 +25,14 @@ const ProducerState = Object.freeze({ }); const CompressionTypes = Object.freeze({ - None: 'none', + NONE: 'none', GZIP: 'gzip', SNAPPY: 'snappy', LZ4: 'lz4', ZSTD: 'zstd', -}) +}); + +const producerPollIntervalMs = 500; class Producer { /** @@ -75,6 +77,14 @@ class Producer { */ #logger = new DefaultLogger(); + /** + * Stores the time of the last poll. + * In case we are producing in a tight loop, the interval timer will not + * fire, and we won't poll. By maintaining the last poll time, we can + * poll at the end of send() and sendBatch(). + */ + #lastPollTime = process.hrtime(); + /** * @constructor * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig @@ -273,7 +283,8 @@ class Producer { return; } this.#internalClient.poll(); - }, 500); + this.#lastPollTime = process.hrtime(); + }, producerPollIntervalMs); this.#internalClient.on('delivery-report', this.#deliveryCallback.bind(this)); @@ -541,6 +552,15 @@ class Producer { } })); + /* Poll if we haven't polled in a while. This can be the case if we're producing + * in a tight loop without awaiting the produce. */ + const elapsed = process.hrtime(this.#lastPollTime); + const elapsedInNanos = elapsed[0] * 1e9 + elapsed[1]; + if (elapsedInNanos > producerPollIntervalMs * 1000) { + this.#lastPollTime = process.hrtime(); + this.#internalClient.poll(); + } + } /* The delivery report will be handled by the delivery-report event handler, and we can simply wait for it here. */ From a62ca53567c6b505c1753aa41c225f6b7a546037 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 30 Jan 2024 11:19:02 +0530 Subject: [PATCH 071/224] Change name internally --- .github/ISSUE_TEMPLATE/bug_report.md | 4 +- .github/workflows/npm-publish.yml | 2 +- .semaphore/project.yml | 4 +- .semaphore/semaphore.yml | 30 ++--- CHANGELOG.md | 2 +- CONTRIBUTING.md | 12 +- INTRODUCTION.md | 12 +- MIGRATION.md | 8 +- README.md | 22 ++-- bench/consumer-raw-rdkafka.js | 4 +- bench/consumer-subscribe.js | 4 +- bench/kafka-consumer-stream.js | 4 +- bench/producer-raw-rdkafka.js | 4 +- bench/producer-rdkafka.js | 4 +- binding.gyp | 2 +- ci/librdkafka-defs-generator.js | 2 +- ci/update-version.js | 2 +- e2e/admin.spec.js | 2 +- e2e/both.spec.js | 2 +- e2e/consumer.spec.js | 2 +- e2e/groups.spec.js | 2 +- e2e/listener.js | 2 +- e2e/producer-transaction.spec.js | 2 +- e2e/producer.spec.js | 8 +- examples/kafkajs/admin.js | 4 +- examples/kafkajs/consumer.js | 4 +- examples/kafkajs/eos.js | 4 +- examples/kafkajs/producer.js | 4 +- examples/kafkajs/sr.js | 6 +- examples/node-rdkafka/consumer-flow.md | 4 +- examples/node-rdkafka/consumer.md | 2 +- examples/node-rdkafka/docker-alpine.md | 6 +- examples/node-rdkafka/metadata.md | 2 +- examples/node-rdkafka/producer-cluster.md | 2 +- examples/node-rdkafka/producer.md | 2 +- lib/admin.js | 4 +- lib/client.js | 2 +- lib/error.js | 2 +- lib/kafka-consumer-stream.js | 2 +- lib/kafka-consumer.js | 4 +- lib/producer-stream.js | 4 +- lib/producer.js | 2 +- lib/producer/high-level-producer.js | 2 +- lib/rdkafka.js | 2 +- lib/tools/ref-counter.js | 2 +- lib/topic-partition.js | 2 +- lib/topic.js | 2 +- lib/util.js | 2 +- librdkafka.js | 4 +- make_docs.sh | 4 +- package-lock.json | 132 +++++++++++----------- package.json | 6 +- service.yml | 2 +- src/admin.cc | 2 +- src/admin.h | 2 +- src/binding.cc | 2 +- src/binding.h | 2 +- src/callbacks.cc | 2 +- src/callbacks.h | 4 +- src/common.cc | 4 +- src/common.h | 2 +- src/config.cc | 2 +- src/config.h | 2 +- src/connection.cc | 2 +- src/connection.h | 2 +- src/errors.cc | 2 +- src/errors.h | 2 +- src/kafka-consumer.cc | 2 +- src/kafka-consumer.h | 2 +- src/producer.cc | 2 +- src/producer.h | 2 +- src/topic.cc | 2 +- src/topic.h | 2 +- src/workers.cc | 2 +- src/workers.h | 2 +- test/binding.spec.js | 4 +- test/consumer.spec.js | 4 +- test/error.spec.js | 2 +- test/index.spec.js | 2 +- test/kafka-consumer-stream.spec.js | 2 +- test/kafka-consumer.spec.js | 2 +- test/mock.js | 2 +- test/producer-stream.spec.js | 2 +- test/producer.spec.js | 2 +- test/producer/high-level-producer.spec.js | 2 +- test/topic-partition.spec.js | 2 +- test/util.spec.js | 2 +- 87 files changed, 219 insertions(+), 219 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md index cb301559..4724334d 100644 --- a/.github/ISSUE_TEMPLATE/bug_report.md +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -9,11 +9,11 @@ about: Create a report to help us improve - Node Version [e.g. 8.2.1]: - NPM Version [e.g. 5.4.2]: - C++ Toolchain [e.g. Visual Studio, llvm, g++]: - - confluent-kafka-js version [e.g. 2.3.3]: + - confluent-kafka-javascript version [e.g. 2.3.3]: **Steps to Reproduce** -**confluent-kafka-js Configuration Settings** +**confluent-kafka-javascript Configuration Settings** **Additional context** diff --git a/.github/workflows/npm-publish.yml b/.github/workflows/npm-publish.yml index c547ffba..0a21a501 100644 --- a/.github/workflows/npm-publish.yml +++ b/.github/workflows/npm-publish.yml @@ -1,7 +1,7 @@ # This workflow will run tests using node and then publish a package to GitHub Packages when a release is created # For more information see: https://help.github.com/actions/language-and-framework-guides/publishing-nodejs-packages -name: Publish confluent-kafka-js +name: Publish confluent-kafka-javascript on: release: diff --git a/.semaphore/project.yml b/.semaphore/project.yml index e20c3839..7bef71a4 100644 --- a/.semaphore/project.yml +++ b/.semaphore/project.yml @@ -6,12 +6,12 @@ apiVersion: v1alpha kind: Project metadata: - name: confluent-kafka-js + name: confluent-kafka-javascript description: "" spec: visibility: private repository: - url: git@github.com:confluentinc/confluent-kafka-js.git + url: git@github.com:confluentinc/confluent-kafka-javascript.git run_on: - branches - tags diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index ca2d6435..137aeea1 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -116,7 +116,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="confluent-kafka-javascript-${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} @@ -125,7 +125,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="confluent-kafka-javascript-${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} @@ -134,7 +134,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="confluent-kafka-javascript-${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} @@ -160,7 +160,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="confluent-kafka-javascript-${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} @@ -169,7 +169,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="confluent-kafka-javascript-${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} @@ -178,7 +178,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="confluent-kafka-javascript-${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} @@ -203,21 +203,21 @@ blocks: - name: "Release: LTS:18" commands: - export NODE_ABI=108 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: LTS:20" commands: - export NODE_ABI=115 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:20-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: latest: 21" commands: - export NODE_ABI=120 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" @@ -241,21 +241,21 @@ blocks: - name: "Release: LTS:18" commands: - export NODE_ABI=108 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:18-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: LTS:20" commands: - export NODE_ABI=115 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:20-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: latest: 21" commands: - export NODE_ABI=120 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" @@ -280,7 +280,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="confluent-kafka-javascript-${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} @@ -289,7 +289,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="confluent-kafka-javascript-${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} @@ -298,7 +298,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="confluent-kafka-javascript-${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/CHANGELOG.md b/CHANGELOG.md index 94715f14..ce4d127d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -# confluent-kafka-js v0.1.5-devel +# confluent-kafka-javascript v0.1.5-devel v0.1.5-devel is a pre-production, early-access release. diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 472325cd..ff872c4c 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,8 +1,8 @@ -# Contributing to `confluent-kafka-js` +# Contributing to `confluent-kafka-javascript` :+1::tada: First off, thanks for taking the time to contribute! :tada::+1: -The following is a set of guidelines for contributing to `confluent-kafka-js` +The following is a set of guidelines for contributing to `confluent-kafka-javascript` which is hosted by [Confluent Inc.](https://github.com/confluentinc) on GitHub. This document lists rules, guidelines, and help getting started, so if you feel something is missing feel free to send a pull request. @@ -123,7 +123,7 @@ In short: ## Editor -Using Visual Studio code to develop on `confluent-kafka-js`. If you use it you can configure the C++ plugin to resolve the paths needed to inform your intellisense. This is the config file I am using on a mac to resolve the required paths: +Using Visual Studio code to develop on `confluent-kafka-javascript`. If you use it you can configure the C++ plugin to resolve the paths needed to inform your intellisense. This is the config file I am using on a mac to resolve the required paths: `c_cpp_properties.json` ``` @@ -207,7 +207,7 @@ node --inspect path/to/file.js 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. +1. Update the `librdkafka` property in [`package.json`](https://github.com/confluentinc/confluent-kafka-javascript/blob/master/package.json) to the desired version. 1. Update the librdkafka git submodule to that versions release commit (example below) @@ -218,7 +218,7 @@ Steps to update: If you get an error during that checkout command, double check that the submodule was initialized / cloned! You may need to run `git submodule update --init --recursive` -1. Update [`config.d.ts`](https://github.com/confluentinc/confluent-kafka-js/blob/master/config.d.ts) and [`errors.d.ts`](https://github.com/confluentinc/confluent-kafka-js/blob/master/errors.d.ts) TypeScript definitions by running: +1. Update [`config.d.ts`](https://github.com/confluentinc/confluent-kafka-javascript/blob/master/config.d.ts) and [`errors.d.ts`](https://github.com/confluentinc/confluent-kafka-javascript/blob/master/errors.d.ts) TypeScript definitions by running: ```bash node ci/librdkafka-defs-generator.js ``` @@ -227,7 +227,7 @@ Steps to update: 1. Run unit tests: `npm run test` -1. Update the version numbers referenced in the [`README.md`](https://github.com/confluentinc/confluent-kafka-js/blob/master/README.md) file to the new version. +1. Update the version numbers referenced in the [`README.md`](https://github.com/confluentinc/confluent-kafka-javascript/blob/master/README.md) file to the new version. ## Releasing diff --git a/INTRODUCTION.md b/INTRODUCTION.md index a1c09318..896fcd8b 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -22,10 +22,10 @@ The library currently supports the following callbacks: This library includes two utility functions for detecting the status of your installation. Please try to include these when making issue reports where applicable. -You can get the features supported by your compile of `librdkafka` by reading the variable "features" on the root of the `confluent-kafka-js` object. +You can get the features supported by your compile of `librdkafka` by reading the variable "features" on the root of the `confluent-kafka-javascript` object. ```js -const Kafka = require('confluent-kafka-js'); +const Kafka = require('confluent-kafka-javascript'); console.log(Kafka.features); // #=> [ 'gzip', 'snappy', 'ssl', 'sasl', 'regex', 'lz4' ] @@ -34,7 +34,7 @@ console.log(Kafka.features); You can also get the version of `librdkafka` ```js -const Kafka = require('confluent-kafka-js'); +const Kafka = require('confluent-kafka-javascript'); console.log(Kafka.librdkafkaVersion); // #=> 2.3.0 @@ -293,7 +293,7 @@ const consumer = new Kafka.KafkaConsumer({ #### Commits -When you commit in `confluent-kafka-js`, the standard way is to queue the commit request up with the next `librdkafka` request to the broker. When doing this, there isn't a way to know the result of the commit. Luckily there is another callback you can listen to to get this information +When you commit in `confluent-kafka-javascript`, the standard way is to queue the commit request up with the next `librdkafka` request to the broker. When doing this, there isn't a way to know the result of the commit. Luckily there is another callback you can listen to to get this information ```js const consumer = new Kafka.KafkaConsumer({ @@ -507,12 +507,12 @@ producer.getMetadata(opts, (err, metadata) => { ### Admin Client -`confluent-kafka-js` now supports the admin client for creating, deleting, and scaling out topics. The `librdkafka` APIs also support altering configuration of topics and broker, but that is not currently implemented. +`confluent-kafka-javascript` now supports the admin client for creating, deleting, and scaling out topics. The `librdkafka` APIs also support altering configuration of topics and broker, but that is not currently implemented. To create an Admin client, you can do as follows: ```js -const Kafka = require('confluent-kafka-js'); +const Kafka = require('confluent-kafka-javascript'); const client = Kafka.AdminClient.create({ 'client.id': 'kafka-admin', diff --git a/MIGRATION.md b/MIGRATION.md index bbd02624..3eae2772 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -11,7 +11,7 @@ ``` to ```javascript - const { Kafka } = require('confluent-kafka-js').KafkaJS; + const { Kafka } = require('confluent-kafka-javascript').KafkaJS; const kafka = new Kafka({ kafkaJS: { brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ } }); const producer = kafka.producer({ kafkaJS: { /* ... */, } }); ``` @@ -35,7 +35,7 @@ ```diff -const { Kafka } = require('kafkajs'); -+const { Kafka } = require('confluent-kafka-js').KafkaJS; ++const { Kafka } = require('confluent-kafka-javascript').KafkaJS; const kafka = new Kafka({ + kafkaJS: { @@ -52,7 +52,7 @@ const producerRun = async () => { topic: 'test-topic', - acks: 1, messages: [ - { value: 'Hello confluent-kafka-js user!' }, + { value: 'Hello confluent-kafka-javascript user!' }, ], }); }; @@ -371,6 +371,6 @@ Change the import statement, from ``` to ```javascript - const Kafka = require('confluent-kafka-js'); + const Kafka = require('confluent-kafka-javascript'); ``` The rest of the functionality should work as usual. \ No newline at end of file diff --git a/README.md b/README.md index 1e3961a8..3f53466d 100644 --- a/README.md +++ b/README.md @@ -1,13 +1,13 @@ Confluent's Javascript Client for Apache KafkaTM ===================================================== -**confluent-kafka-js** is Confluent's JavaScript client for [Apache Kafka](http://kafka.apache.org/) and the -[Confluent Platform](https://www.confluent.io/product/compare/). This is an **early access** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client that is based on [node-rdkafka](https://github.com/Blizzard/node-rdkafka) yet also API compatible with [KafkaJS](https://github.com/tulios/kafkajs) to provide flexibility to users and streamline migrations from other clients. +**confluent-kafka-javascript** is Confluent's JavaScript client for [Apache Kafka](http://kafka.apache.org/) and the +[Confluent Platform](https://www.confluent.io/product/compare/). This is an **early access** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client that is based on [node-rdkafka](https://github.com/Blizzard/node-rdkafka) yet also API compatible with [KafkaJS](https://github.com/tulios/kafkajs) to provide flexibility to users and streamline migrations from other clients. This library leverages the work and concepts from two popular Apache Kafka JavaScript clients: [node-rdkafka](https://github.com/Blizzard/node-rdkafka) and [KafkaJS](https://github.com/tulios/kafkajs). The core is heavily based on the node-rdkafka library, which uses our own [librdkafka](https://github.com/confluentinc/librdkafka/tree/v2.3.0) library for core client functionality. However, we leverage a promisified API and a more idiomatic interface, similar to the one in KafkaJS, making it easy for developers to migrate and adopt this client depending on the patterns and interface they prefer. __This library currently uses `librdkafka` based off of the master branch.__ -## This library is currently in early access and not meant for production use +## This library is currently in early access and not meant for production use **This library is in active development, pre-1.0.0, and it is likely to have many breaking changes.** @@ -16,11 +16,11 @@ For this early-access release, we aim to get feedback from JavaScript developers - Migration experience from the node-rdkafka and KafkaJs - Overall quality and reliability -We invite you to raise issues to highlight any feedback you may have. +We invite you to raise issues to highlight any feedback you may have. -Within the early-access, only **basic produce and consume functionality** as well as the ability to **create and delete topics** are supported. All other admin client functionality is coming in future releases. See [INTRODUCTION.md](INTRODUCTION.md) for more details on what is supported. +Within the early-access, only **basic produce and consume functionality** as well as the ability to **create and delete topics** are supported. All other admin client functionality is coming in future releases. See [INTRODUCTION.md](INTRODUCTION.md) for more details on what is supported. -To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-js/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). +To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). ## Requirements @@ -34,18 +34,18 @@ The following configurations are supported for this early access preview: Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: ```bash -$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-js.git#v0.1.6-devel" +$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-javascript.git#v0.1.6-devel" ``` Yarn and pnpm support is experimental. # Getting Started -Below is a simple produce example for users migrating from KafkaJS. +Below is a simple produce example for users migrating from KafkaJS. -```javascript -// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. -const { Kafka } = require("confluent-kafka-js").KafkaJS; +```javascript +// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. +const { Kafka } = require("confluent-kafka-javascript").KafkaJS; async function producerStart() { const kafka = new Kafka({ diff --git a/bench/consumer-raw-rdkafka.js b/bench/consumer-raw-rdkafka.js index 5b27c1d2..a42461e8 100644 --- a/bench/consumer-raw-rdkafka.js +++ b/bench/consumer-raw-rdkafka.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -16,7 +16,7 @@ var topic = process.argv[3] || 'test'; var consumer = new Kafka.KafkaConsumer({ 'metadata.broker.list': host, - 'group.id': 'confluent-kafka-js-bench-s', + 'group.id': 'confluent-kafka-javascript-bench-s', 'fetch.wait.max.ms': 100, 'fetch.message.max.bytes': 1024 * 1024, 'enable.auto.commit': false diff --git a/bench/consumer-subscribe.js b/bench/consumer-subscribe.js index 6dbd7fc4..f46ee21d 100644 --- a/bench/consumer-subscribe.js +++ b/bench/consumer-subscribe.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -16,7 +16,7 @@ var topic = process.argv[3] || 'test'; var consumer = new Kafka.KafkaConsumer({ 'metadata.broker.list': host, - 'group.id': 'confluent-kafka-js-bench', + 'group.id': 'confluent-kafka-javascript-bench', 'fetch.wait.max.ms': 100, 'fetch.message.max.bytes': 1024 * 1024, 'enable.auto.commit': false diff --git a/bench/kafka-consumer-stream.js b/bench/kafka-consumer-stream.js index ff8888f9..6bf53779 100644 --- a/bench/kafka-consumer-stream.js +++ b/bench/kafka-consumer-stream.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -18,7 +18,7 @@ var topic = process.argv[3] || 'test'; var stream = Kafka.createReadStream({ 'metadata.broker.list': host, - 'group.id': 'confluent-kafka-js-benchs', + 'group.id': 'confluent-kafka-javascript-benchs', 'fetch.wait.max.ms': 100, 'fetch.message.max.bytes': 1024 * 1024, 'enable.auto.commit': false diff --git a/bench/producer-raw-rdkafka.js b/bench/producer-raw-rdkafka.js index e5d7cf56..c0d55e5c 100644 --- a/bench/producer-raw-rdkafka.js +++ b/bench/producer-raw-rdkafka.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -24,7 +24,7 @@ var MAX = process.argv[5] || 10000000; var producer = new Kafka.Producer({ 'metadata.broker.list': host, - 'group.id': 'confluent-kafka-js-bench', + 'group.id': 'confluent-kafka-javascript-bench', 'compression.codec': compression, 'retry.backoff.ms': 200, 'message.send.max.retries': 10, diff --git a/bench/producer-rdkafka.js b/bench/producer-rdkafka.js index d2fa37b2..97eb6bf6 100644 --- a/bench/producer-rdkafka.js +++ b/bench/producer-rdkafka.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -20,7 +20,7 @@ var MAX = process.argv[5] || 1000000; var stream = Kafka.Producer.createWriteStream({ 'metadata.broker.list': host, - 'group.id': 'confluent-kafka-js-bench', + 'group.id': 'confluent-kafka-javascript-bench', 'compression.codec': compression, 'retry.backoff.ms': 200, 'message.send.max.retries': 10, diff --git a/binding.gyp b/binding.gyp index e9671d01..a70df362 100644 --- a/binding.gyp +++ b/binding.gyp @@ -7,7 +7,7 @@ }, "targets": [ { - "target_name": "confluent-kafka-js", + "target_name": "confluent-kafka-javascript", 'sources': [ 'src/binding.cc', 'src/callbacks.cc', diff --git a/ci/librdkafka-defs-generator.js b/ci/librdkafka-defs-generator.js index be4113a1..f81b423e 100644 --- a/ci/librdkafka-defs-generator.js +++ b/ci/librdkafka-defs-generator.js @@ -52,7 +52,7 @@ function extractConfigItems(configStr) { } function processItem(configItem) { - // These items are overwritten by confluent-kafka-js + // These items are overwritten by confluent-kafka-javascript switch (configItem.property) { case 'dr_msg_cb': return { ...configItem, type: 'boolean' }; diff --git a/ci/update-version.js b/ci/update-version.js index 4265ea9b..f9df0617 100644 --- a/ci/update-version.js +++ b/ci/update-version.js @@ -89,7 +89,7 @@ function getPackageVersion(tag, branch) { console.log(`Package version is "${baseVersion}"`); // never publish with an suffix - // fixes https://github.com/confluentinc/confluent-kafka-js/issues/981 + // fixes https://github.com/confluentinc/confluent-kafka-javascript/issues/981 // baseVersion += '-'; // if (tag.commit === 0 && branch === 'master') { diff --git a/e2e/admin.spec.js b/e2e/admin.spec.js index 75cbceb8..9cb960cb 100644 --- a/e2e/admin.spec.js +++ b/e2e/admin.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/e2e/both.spec.js b/e2e/both.spec.js index 47947812..5b376dec 100644 --- a/e2e/both.spec.js +++ b/e2e/both.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/e2e/consumer.spec.js b/e2e/consumer.spec.js index 257055d4..15044cea 100644 --- a/e2e/consumer.spec.js +++ b/e2e/consumer.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * Copyright (c) 2016-2023 Blizzard Entertainment * * This software may be modified and distributed under the terms diff --git a/e2e/groups.spec.js b/e2e/groups.spec.js index bc22f13e..e5f989aa 100644 --- a/e2e/groups.spec.js +++ b/e2e/groups.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/e2e/listener.js b/e2e/listener.js index 81c9637f..bb3c1691 100644 --- a/e2e/listener.js +++ b/e2e/listener.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * Copyright (c) 2016-2023 Blizzard Entertainment * * This software may be modified and distributed under the terms diff --git a/e2e/producer-transaction.spec.js b/e2e/producer-transaction.spec.js index 452c5e45..652e8edb 100644 --- a/e2e/producer-transaction.spec.js +++ b/e2e/producer-transaction.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/e2e/producer.spec.js b/e2e/producer.spec.js index 0ed1ff24..7b5a7667 100644 --- a/e2e/producer.spec.js +++ b/e2e/producer.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -211,7 +211,7 @@ describe('Producer', function() { producer.produce('test', null, Buffer.from('hai'), 'key'); }); - it('should produce a message with an empty payload and empty key (https://github.com/confluentinc/confluent-kafka-js/issues/117)', function(done) { + it('should produce a message with an empty payload and empty key (https://github.com/confluentinc/confluent-kafka-javascript/issues/117)', function(done) { var tt = setInterval(function() { producer.poll(); }, 200); @@ -232,7 +232,7 @@ describe('Producer', function() { producer.produce('test', null, Buffer.from(''), ''); }); - it('should produce a message with a null payload and null key (https://github.com/confluentinc/confluent-kafka-js/issues/117)', function(done) { + it('should produce a message with a null payload and null key (https://github.com/confluentinc/confluent-kafka-javascript/issues/117)', function(done) { producer.setPollInterval(10); producer.once('delivery-report', function(err, report) { @@ -250,7 +250,7 @@ describe('Producer', function() { producer.produce('test', null, null, null); }); - it('should produce an int64 key (https://github.com/confluentinc/confluent-kafka-js/issues/208)', function(done) { + it('should produce an int64 key (https://github.com/confluentinc/confluent-kafka-javascript/issues/208)', function(done) { var v1 = 0x0000000000000084; var arr = new Uint8Array(8); diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index 8450738f..5092c491 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -1,6 +1,6 @@ -// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-js'. +// will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; async function adminStart() { diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 4249c9dd..5f01a172 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,6 +1,6 @@ -// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-js'. +// will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; async function consumerStart() { diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index 9b63b8c5..e2c1ff9c 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -1,6 +1,6 @@ -// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-js'. +// will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; async function eosStart() { diff --git a/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index 365468f6..1c0fabbf 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -1,6 +1,6 @@ -// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-js'. +// will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; async function producerStart() { diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 49516c04..7ecc32bd 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -1,10 +1,10 @@ -// require('kafkajs') is replaced with require('confluent-kafka-js').KafkaJS. +// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-js'. +// will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; // Note: The kafkajs/confluent-schema-registry will need to be installed separately to run this example, -// as it isn't a dependency of confluent-kafka-js. +// as it isn't a dependency of confluent-kafka-javascript. const { SchemaRegistry, SchemaType } = require('@kafkajs/confluent-schema-registry'); const registry = new SchemaRegistry({ host: '' }) diff --git a/examples/node-rdkafka/consumer-flow.md b/examples/node-rdkafka/consumer-flow.md index 62e5e31f..ee2470b8 100644 --- a/examples/node-rdkafka/consumer-flow.md +++ b/examples/node-rdkafka/consumer-flow.md @@ -3,7 +3,7 @@ the Flowing implementation ```js /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -16,7 +16,7 @@ var Kafka = require('../'); var consumer = new Kafka.KafkaConsumer({ //'debug': 'all', 'metadata.broker.list': 'localhost:9092', - 'group.id': 'confluent-kafka-js-consumer-flow-example', + 'group.id': 'confluent-kafka-javascript-consumer-flow-example', 'enable.auto.commit': false }); diff --git a/examples/node-rdkafka/consumer.md b/examples/node-rdkafka/consumer.md index 4e09cc2f..0c0b6752 100644 --- a/examples/node-rdkafka/consumer.md +++ b/examples/node-rdkafka/consumer.md @@ -3,7 +3,7 @@ the Stream implementation ```js /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/examples/node-rdkafka/docker-alpine.md b/examples/node-rdkafka/docker-alpine.md index 9339e191..5f407c5f 100644 --- a/examples/node-rdkafka/docker-alpine.md +++ b/examples/node-rdkafka/docker-alpine.md @@ -1,4 +1,4 @@ -When using docker to install `confluent-kafka-js`, you need to make sure you install appropriate library dependencies. Alpine linux is a lighter weight version of linux and does not come with the same base libraries as other distributions (like glibc). +When using docker to install `confluent-kafka-javascript`, you need to make sure you install appropriate library dependencies. Alpine linux is a lighter weight version of linux and does not come with the same base libraries as other distributions (like glibc). You can see some of the differences here: https://linuxacademy.com/blog/cloud/alpine-linux-and-docker/ @@ -24,7 +24,7 @@ RUN mkdir -p /usr/local/app # Move to the app directory WORKDIR /usr/local/app -# Install confluent-kafka-js -RUN npm install confluent-kafka-js +# Install confluent-kafka-javascript +RUN npm install confluent-kafka-javascript # Copy package.json first to check if an npm install is needed ``` diff --git a/examples/node-rdkafka/metadata.md b/examples/node-rdkafka/metadata.md index 0fd4c08b..92e8d538 100644 --- a/examples/node-rdkafka/metadata.md +++ b/examples/node-rdkafka/metadata.md @@ -1,6 +1,6 @@ ```js /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/examples/node-rdkafka/producer-cluster.md b/examples/node-rdkafka/producer-cluster.md index be3dee8e..76fa91ff 100644 --- a/examples/node-rdkafka/producer-cluster.md +++ b/examples/node-rdkafka/producer-cluster.md @@ -1,6 +1,6 @@ ```js /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/examples/node-rdkafka/producer.md b/examples/node-rdkafka/producer.md index 0518c675..de493e1a 100644 --- a/examples/node-rdkafka/producer.md +++ b/examples/node-rdkafka/producer.md @@ -1,6 +1,6 @@ ```js /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/admin.js b/lib/admin.js index 76c76de7..120a5353 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -47,7 +47,7 @@ function createAdminClient(conf) { * * * Once you instantiate this object, it will have a handle to the kafka broker. - * Unlike the other confluent-kafka-js classes, this class does not ensure that + * Unlike the other confluent-kafka-javascript classes, this class does not ensure that * it is connected to the upstream broker. Instead, making an action will * validate that. * diff --git a/lib/client.js b/lib/client.js index 89464845..d0d1c4b7 100644 --- a/lib/client.js +++ b/lib/client.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/error.js b/lib/error.js index 5cd6c4f0..952c80ee 100644 --- a/lib/error.js +++ b/lib/error.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/kafka-consumer-stream.js b/lib/kafka-consumer-stream.js index fcc07197..6deb85e7 100644 --- a/lib/kafka-consumer-stream.js +++ b/lib/kafka-consumer-stream.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index dd981ade..1df769af 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -399,7 +399,7 @@ KafkaConsumer.prototype.consume = function(number, cb) { this._consumeNum(timeoutMs, number, cb); } else { - // See https://github.com/confluentinc/confluent-kafka-js/issues/220 + // See https://github.com/confluentinc/confluent-kafka-javascript/issues/220 // Docs specify just a callback can be provided but really we needed // a fallback to the number argument // @deprecated diff --git a/lib/producer-stream.js b/lib/producer-stream.js index bb2d28ff..b583eb7a 100644 --- a/lib/producer-stream.js +++ b/lib/producer-stream.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -289,7 +289,7 @@ ProducerStream.prototype.close = function(cb) { self.producer.disconnect(function() { // Previously this set the producer to null. I'm not sure there is any benefit // to that other than I guess helping flag it for GC? - // https://github.com/confluentinc/confluent-kafka-js/issues/344 + // https://github.com/confluentinc/confluent-kafka-javascript/issues/344 close(); }); } else if (self.producer._isConnecting){ diff --git a/lib/producer.js b/lib/producer.js index 65b4c936..ddada902 100644 --- a/lib/producer.js +++ b/lib/producer.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/producer/high-level-producer.js b/lib/producer/high-level-producer.js index 506c2acd..6d1ef959 100644 --- a/lib/producer/high-level-producer.js +++ b/lib/producer/high-level-producer.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/rdkafka.js b/lib/rdkafka.js index c5e2e088..7a57e840 100644 --- a/lib/rdkafka.js +++ b/lib/rdkafka.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/tools/ref-counter.js b/lib/tools/ref-counter.js index a4800466..6347070a 100644 --- a/lib/tools/ref-counter.js +++ b/lib/tools/ref-counter.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/topic-partition.js b/lib/topic-partition.js index dc22375e..fd709068 100644 --- a/lib/topic-partition.js +++ b/lib/topic-partition.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/topic.js b/lib/topic.js index b591ceb7..c9089c15 100644 --- a/lib/topic.js +++ b/lib/topic.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/lib/util.js b/lib/util.js index 1b0187c0..c8bee360 100644 --- a/lib/util.js +++ b/lib/util.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/librdkafka.js b/librdkafka.js index 35e4e149..87ccb864 100644 --- a/librdkafka.js +++ b/librdkafka.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -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')('confluent-kafka-javascript'); module.exports = kafka; diff --git a/make_docs.sh b/make_docs.sh index 4ab333e8..bdb4d6d2 100755 --- a/make_docs.sh +++ b/make_docs.sh @@ -6,8 +6,8 @@ if [[ `git status --porcelain` ]]; then exit 1 fi -# REPO=git@github.com:confluentinc/confluent-kafka-js.git -REPO=https://github.com/confluentinc/confluent-kafka-js.git +# REPO=git@github.com:confluentinc/confluent-kafka-javascript.git +REPO=https://github.com/confluentinc/confluent-kafka-javascript.git git remote add deploy $REPO diff --git a/package-lock.json b/package-lock.json index c22fd758..9673ebd6 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { - "name": "confluent-kafka-js", - "version": "v0.1.5-devel", + "name": "confluent-kafka-javascript", + "version": "v0.1.6-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { - "name": "confluent-kafka-js", - "version": "v0.1.5-devel", + "name": "confluent-kafka-javascript", + "version": "v0.1.6-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { @@ -145,9 +145,9 @@ } }, "node_modules/@babel/core": { - "version": "7.23.7", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.7.tgz", - "integrity": "sha512-+UpDgowcmqe36d4NwqvKsyPMlOLNGMsfMmQ5WGCu+siCe3t3dfe9njrzGfdN4qq+bcNUt0+Vw6haRxBOycs4dw==", + "version": "7.23.9", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.9.tgz", + "integrity": "sha512-5q0175NOjddqpvvzU+kDiSOAk4PfdO6FvwCWoQ6RO7rTzEe8vlo+4HVfcnAREhD4npMs0e9uZypjTwzZPCf/cw==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", @@ -155,11 +155,11 @@ "@babel/generator": "^7.23.6", "@babel/helper-compilation-targets": "^7.23.6", "@babel/helper-module-transforms": "^7.23.3", - "@babel/helpers": "^7.23.7", - "@babel/parser": "^7.23.6", - "@babel/template": "^7.22.15", - "@babel/traverse": "^7.23.7", - "@babel/types": "^7.23.6", + "@babel/helpers": "^7.23.9", + "@babel/parser": "^7.23.9", + "@babel/template": "^7.23.9", + "@babel/traverse": "^7.23.9", + "@babel/types": "^7.23.9", "convert-source-map": "^2.0.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -349,14 +349,14 @@ } }, "node_modules/@babel/helpers": { - "version": "7.23.8", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.8.tgz", - "integrity": "sha512-KDqYz4PiOWvDFrdHLPhKtCThtIcKVy6avWD2oG4GEvyQ+XDZwHD4YQd+H2vNMnq2rkdxsDkU82T+Vk8U/WXHRQ==", + "version": "7.23.9", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.9.tgz", + "integrity": "sha512-87ICKgU5t5SzOT7sBMfCOZQ2rHjRU+Pcb9BoILMYz600W6DkVRLFBPwQ18gwUVvggqXivaUakpnxWQGbpywbBQ==", "dev": true, "dependencies": { - "@babel/template": "^7.22.15", - "@babel/traverse": "^7.23.7", - "@babel/types": "^7.23.6" + "@babel/template": "^7.23.9", + "@babel/traverse": "^7.23.9", + "@babel/types": "^7.23.9" }, "engines": { "node": ">=6.9.0" @@ -448,9 +448,9 @@ } }, "node_modules/@babel/parser": { - "version": "7.23.6", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.6.tgz", - "integrity": "sha512-Z2uID7YJ7oNvAI20O9X0bblw7Qqs8Q2hFy0R9tAfnfLkp5MW0UH9eUvnDSnFwKZ0AvgS1ucqR4KzvVHgnke1VQ==", + "version": "7.23.9", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.9.tgz", + "integrity": "sha512-9tcKgqKbs3xGJ+NtKF2ndOBBLVwPjl1SHxPQkd36r3Dlirw3xWUeGaTbqr7uGZcTaxkVNwc+03SVP7aCdWrTlA==", "dev": true, "bin": { "parser": "bin/babel-parser.js" @@ -637,23 +637,23 @@ } }, "node_modules/@babel/template": { - "version": "7.22.15", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.22.15.tgz", - "integrity": "sha512-QPErUVm4uyJa60rkI73qneDacvdvzxshT3kksGqlGWYdOTIUOwJ7RDUL8sGqslY1uXWSL6xMFKEXDS3ox2uF0w==", + "version": "7.23.9", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.23.9.tgz", + "integrity": "sha512-+xrD2BWLpvHKNmX2QbpdpsBaWnRxahMwJjO+KZk2JOElj5nSmKezyS1B4u+QbHMTX69t4ukm6hh9lsYQ7GHCKA==", "dev": true, "dependencies": { - "@babel/code-frame": "^7.22.13", - "@babel/parser": "^7.22.15", - "@babel/types": "^7.22.15" + "@babel/code-frame": "^7.23.5", + "@babel/parser": "^7.23.9", + "@babel/types": "^7.23.9" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/traverse": { - "version": "7.23.7", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.7.tgz", - "integrity": "sha512-tY3mM8rH9jM0YHFGyfC0/xf+SB5eKUu7HPj7/k3fpi9dAlsMc5YbQvDi0Sh2QTPXqMhyaAtzAr807TIyfQrmyg==", + "version": "7.23.9", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.9.tgz", + "integrity": "sha512-I/4UJ9vs90OkBtY6iiiTORVMyIhJ4kAVmsKo9KFc8UOxMeUfi2hvtIBsET5u9GizXE6/GFSuKCTNfgCswuEjRg==", "dev": true, "dependencies": { "@babel/code-frame": "^7.23.5", @@ -662,8 +662,8 @@ "@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.6", - "@babel/types": "^7.23.6", + "@babel/parser": "^7.23.9", + "@babel/types": "^7.23.9", "debug": "^4.3.1", "globals": "^11.1.0" }, @@ -681,9 +681,9 @@ } }, "node_modules/@babel/types": { - "version": "7.23.6", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.23.6.tgz", - "integrity": "sha512-+uarb83brBzPKN38NX1MkB6vb6+mwvR6amUulqAE7ccQw1pEl+bCia9TbdG1lsnFP7lZySvUn37CHyXQdfTwzg==", + "version": "7.23.9", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.23.9.tgz", + "integrity": "sha512-dQjSq/7HaSjRM43FFGnv5keM2HsxpmyV1PfaSVm0nzzjwwTmjOe6J4bC8e3+pTEIgHaHj+1ZlLThRJ2auc/w1Q==", "dev": true, "dependencies": { "@babel/helper-string-parser": "^7.23.4", @@ -763,13 +763,13 @@ "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==", + "version": "0.11.14", + "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.14.tgz", + "integrity": "sha512-3T8LkOmg45BV5FICb15QQMsyUSWrQ8AygVfC7ZG32zOalnqrilm018ZVCw0eapXux8FtA33q8PSRSstjee3jSg==", "dev": true, "dependencies": { - "@humanwhocodes/object-schema": "^2.0.1", - "debug": "^4.1.1", + "@humanwhocodes/object-schema": "^2.0.2", + "debug": "^4.3.1", "minimatch": "^3.0.5" }, "engines": { @@ -790,9 +790,9 @@ } }, "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==", + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.2.tgz", + "integrity": "sha512-6EwiSjwWYP7pTckG6I5eyFANjPhmPjUX9JRLUSfNPC7FX7zK9gyZAfUEaECL6ALTpGX5AjnBq3C9XmVWPitNpw==", "dev": true }, "node_modules/@istanbuljs/load-nyc-config": { @@ -1220,9 +1220,9 @@ "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==", + "version": "0.3.22", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.22.tgz", + "integrity": "sha512-Wf963MzWtA2sjrNt+g18IAln9lKnlRp+K2eH4jjIoF1wYeq3aMREpG09xhlhdzS0EjwU7qmUJYangWa+151vZw==", "dev": true, "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -1329,9 +1329,9 @@ "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==", + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.1.tgz", + "integrity": "sha512-K3mCHKQ9sVh8o1C9cxkwxaOmXoAMlDxC1mYyHrjqOWEcBjYr76t96zL2zlj5dUGZ3HSw240X1qgH3Mjf1yJWpQ==", "dev": true, "dependencies": { "type-detect": "4.0.8" @@ -1452,9 +1452,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.10.7", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.10.7.tgz", - "integrity": "sha512-fRbIKb8C/Y2lXxB5eVMj4IU7xpdox0Lh8bUPEdtLysaylsml1hOOx1+STloRs/B9nf7C6kPRmmg/V7aQW7usNg==", + "version": "20.11.10", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.10.tgz", + "integrity": "sha512-rZEfe/hJSGYmdfX9tvcPMYeYPW2sNl50nsw4jZmRcaG0HIAb0WYEpsB05GOb53vjqpyE9GUhlDQ4jLSoB5q9kg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -1853,9 +1853,9 @@ "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==", + "version": "4.22.3", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.22.3.tgz", + "integrity": "sha512-UAp55yfwNv0klWNapjs/ktHoguxuQNGnOzxYmfnXIS+8AsRDZkSDxg7R1AX3GKzn078SBI5dzwzj/Yx0Or0e3A==", "dev": true, "funding": [ { @@ -1872,8 +1872,8 @@ } ], "dependencies": { - "caniuse-lite": "^1.0.30001565", - "electron-to-chromium": "^1.4.601", + "caniuse-lite": "^1.0.30001580", + "electron-to-chromium": "^1.4.648", "node-releases": "^2.0.14", "update-browserslist-db": "^1.0.13" }, @@ -1996,9 +1996,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001576", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001576.tgz", - "integrity": "sha512-ff5BdakGe2P3SQsMsiqmt1Lc8221NR1VzHj5jXN5vBny9A6fpze94HiVV/n7XRosOlsShJcvMv5mdnpjOGCEgg==", + "version": "1.0.30001581", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001581.tgz", + "integrity": "sha512-whlTkwhqV2tUmP3oYhtNfaWGYHDdS3JYFQBKXxcUR9qqPWsRhFHhoISO2Xnl/g0xyKzht9mI1LZpiNWfMzHixQ==", "dev": true, "funding": [ { @@ -2436,9 +2436,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.625", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.625.tgz", - "integrity": "sha512-DENMhh3MFgaPDoXWrVIqSPInQoLImywfCwrSmVl3cf9QHzoZSiutHwGaB/Ql3VkqcQV30rzgdM+BjKqBAJxo5Q==", + "version": "1.4.650", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.650.tgz", + "integrity": "sha512-sYSQhJCJa4aGA1wYol5cMQgekDBlbVfTRavlGZVr3WZpDdOPcp6a6xUnFfrt8TqZhsBYYbDxJZCjGfHuGupCRQ==", "dev": true }, "node_modules/emittery": { @@ -2746,9 +2746,9 @@ "dev": true }, "node_modules/fastq": { - "version": "1.16.0", - "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.16.0.tgz", - "integrity": "sha512-ifCoaXsDrsdkWTtiNJX5uzHDsrck5TzfKKDcuFFTIrrc/BS076qgEIfoIy1VeZqViznfKiysPYTh/QeHtnIsYA==", + "version": "1.17.0", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.17.0.tgz", + "integrity": "sha512-zGygtijUMT7jnk3h26kUms3BkSDp4IfIKjmnqI2tvx6nuBfiF1UqOxbnLfzdv+apBy+53oaImsKtMw/xYbW+1w==", "dev": true, "dependencies": { "reusify": "^1.0.4" diff --git a/package.json b/package.json index 7024a703..f5fdcde2 100644 --- a/package.json +++ b/package.json @@ -1,5 +1,5 @@ { - "name": "confluent-kafka-js", + "name": "confluent-kafka-javascript", "version": "v0.1.6-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", @@ -14,7 +14,7 @@ "test:types": "tsc -p ." }, "binary": { - "module_name": "confluent-kafka-js", + "module_name": "confluent-kafka-javascript", "module_path": "./build/{configuration}/", "package_name": "{module_name}-v{version}-{node_abi}-{platform}-{libc}-{arch}.tar.gz", "host": "https://confluent-kafka-js.s3.us-west-1.amazonaws.com/releases/download/", @@ -26,7 +26,7 @@ ], "repository": { "type": "git", - "url": "git@github.com:confluentinc/confluent-kafka-js.git" + "url": "git@github.com:confluentinc/confluent-kafka-javascript.git" }, "license": "MIT", "devDependencies": { diff --git a/service.yml b/service.yml index 760002d2..ec678454 100644 --- a/service.yml +++ b/service.yml @@ -1,4 +1,4 @@ -name: confluent-kafka-js +name: confluent-kafka-javascript lang: unknown lang_version: unknown git: diff --git a/src/admin.cc b/src/admin.cc index a0e6a011..56e8ada5 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/admin.h b/src/admin.h index 6e597a2c..a44ce1b2 100644 --- a/src/admin.h +++ b/src/admin.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/binding.cc b/src/binding.cc index 85baf129..04222635 100644 --- a/src/binding.cc +++ b/src/binding.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/binding.h b/src/binding.h index bebb5bf8..45e08088 100644 --- a/src/binding.h +++ b/src/binding.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/callbacks.cc b/src/callbacks.cc index b0d8cd62..bbac1333 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/callbacks.h b/src/callbacks.h index 0944b61e..24d25a62 100644 --- a/src/callbacks.h +++ b/src/callbacks.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * Copyright (c) 2016-2023 Blizzard Entertainment * * This software may be modified and distributed under the terms @@ -118,7 +118,7 @@ class DeliveryReport { void* opaque; // Key. It is a pointer to avoid corrupted values - // https://github.com/confluentinc/confluent-kafka-js/issues/208 + // https://github.com/confluentinc/confluent-kafka-javascript/issues/208 void* key; size_t key_len; diff --git a/src/common.cc b/src/common.cc index 3d0a425d..836b8b0f 100644 --- a/src/common.cc +++ b/src/common.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -455,7 +455,7 @@ v8::Local ToV8Object(RdKafka::Message *message, if (key_payload) { // We want this to also be a buffer to avoid corruption - // https://github.com/confluentinc/confluent-kafka-js/issues/208 + // https://github.com/confluentinc/confluent-kafka-javascript/issues/208 Nan::Set(pack, Nan::New("key").ToLocalChecked(), Nan::Encode(key_payload, message->key_len(), Nan::Encoding::BUFFER)); } else { diff --git a/src/common.h b/src/common.h index 785a6515..956223a6 100644 --- a/src/common.h +++ b/src/common.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/config.cc b/src/config.cc index c1584b48..b8d883c7 100644 --- a/src/config.cc +++ b/src/config.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/config.h b/src/config.h index 5358ce49..cea57302 100644 --- a/src/config.h +++ b/src/config.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/connection.cc b/src/connection.cc index cd203494..654e2db0 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/connection.h b/src/connection.h index 9e335d63..92381156 100644 --- a/src/connection.h +++ b/src/connection.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/errors.cc b/src/errors.cc index 7ad40635..18830268 100644 --- a/src/errors.cc +++ b/src/errors.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/errors.h b/src/errors.h index 799719eb..fc593022 100644 --- a/src/errors.h +++ b/src/errors.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 5592d7fd..08deac73 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index d5991944..316ee901 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/producer.cc b/src/producer.cc index f4e9f56d..c1c5d842 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/producer.h b/src/producer.h index 4f0ca2c3..39dcbefe 100644 --- a/src/producer.h +++ b/src/producer.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/topic.cc b/src/topic.cc index 1f0de565..78653c41 100644 --- a/src/topic.cc +++ b/src/topic.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/topic.h b/src/topic.h index 1404d532..fea91314 100644 --- a/src/topic.h +++ b/src/topic.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/workers.cc b/src/workers.cc index 749732d0..acc5e249 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/src/workers.h b/src/workers.h index b290d253..ce79b75a 100644 --- a/src/workers.h +++ b/src/workers.h @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/binding.spec.js b/test/binding.spec.js index 466f3fe6..cde357f9 100644 --- a/test/binding.spec.js +++ b/test/binding.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -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')('confluent-kafka-javascript'); var t = require('assert'); var consumerConfig = { diff --git a/test/consumer.spec.js b/test/consumer.spec.js index 45b1b17e..1f1d61a7 100644 --- a/test/consumer.spec.js +++ b/test/consumer.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * @@ -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')('confluent-kafka-javascript'); var t = require('assert'); var client; diff --git a/test/error.spec.js b/test/error.spec.js index 1e54ca4f..894a17f0 100644 --- a/test/error.spec.js +++ b/test/error.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/index.spec.js b/test/index.spec.js index 1e54ca4f..894a17f0 100644 --- a/test/index.spec.js +++ b/test/index.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/kafka-consumer-stream.spec.js b/test/kafka-consumer-stream.spec.js index b12d95c8..198dfb82 100644 --- a/test/kafka-consumer-stream.spec.js +++ b/test/kafka-consumer-stream.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/kafka-consumer.spec.js b/test/kafka-consumer.spec.js index 9d05e01c..a6afe64c 100644 --- a/test/kafka-consumer.spec.js +++ b/test/kafka-consumer.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/mock.js b/test/mock.js index 8e72c822..a9f50748 100644 --- a/test/mock.js +++ b/test/mock.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/producer-stream.spec.js b/test/producer-stream.spec.js index 4bb3bfa7..bb70de96 100644 --- a/test/producer-stream.spec.js +++ b/test/producer-stream.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/producer.spec.js b/test/producer.spec.js index fc24ea4b..03a8f4be 100644 --- a/test/producer.spec.js +++ b/test/producer.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/producer/high-level-producer.spec.js b/test/producer/high-level-producer.spec.js index 41f665b8..4f8577a3 100644 --- a/test/producer/high-level-producer.spec.js +++ b/test/producer/high-level-producer.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/topic-partition.spec.js b/test/topic-partition.spec.js index 4d9eab38..b3030aad 100644 --- a/test/topic-partition.spec.js +++ b/test/topic-partition.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * diff --git a/test/util.spec.js b/test/util.spec.js index dfa11440..7185a800 100644 --- a/test/util.spec.js +++ b/test/util.spec.js @@ -1,5 +1,5 @@ /* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment * From dfe5a98ea9c923c6aeca044649d27d6db7a4489c Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 30 Jan 2024 11:38:32 +0530 Subject: [PATCH 072/224] Version bump for release --- .semaphore/semaphore.yml | 1 - INTRODUCTION.md | 2 +- README.md | 2 +- package.json | 2 +- 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 137aeea1..3a35008b 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -28,7 +28,6 @@ global_job_config: - git submodule update --init --recursive - cd deps/librdkafka - git fetch origin - - git checkout dev_bump_openssl_version_3_0_12 - cd ../../ - export MKL_DEBUG=1 diff --git a/INTRODUCTION.md b/INTRODUCTION.md index 896fcd8b..a95a6152 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -1,4 +1,4 @@ -# Introduction to Confluent-Kafka-JS +# Introduction to Confluent-Kafka-JavaScript ## Configuration diff --git a/README.md b/README.md index 3f53466d..1ba0401e 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ The following configurations are supported for this early access preview: Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: ```bash -$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-javascript.git#v0.1.6-devel" +$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-javascript.git#v0.1.7-devel" ``` Yarn and pnpm support is experimental. diff --git a/package.json b/package.json index f5fdcde2..2214513a 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "confluent-kafka-javascript", - "version": "v0.1.6-devel", + "version": "v0.1.7-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "main": "lib/index.js", From 1b035c25905c12661f3d937fc2871457ccc5faa8 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 1 Feb 2024 17:30:10 +0530 Subject: [PATCH 073/224] Add consumer message cache for KafkaJS compability mode --- examples/consumer.js | 3 +- examples/eos.js | 3 +- examples/producer.js | 3 +- lib/kafkajs/_consumer.js | 299 ++++++++++++++++++++++-- lib/kafkajs/_producer.js | 9 +- package-lock.json | 28 +-- test/promisified/consumer/pause.spec.js | 2 +- test/promisified/testhelpers.js | 10 +- 8 files changed, 315 insertions(+), 42 deletions(-) diff --git a/examples/consumer.js b/examples/consumer.js index d7dd7fa3..ee92fd6f 100644 --- a/examples/consumer.js +++ b/examples/consumer.js @@ -1,5 +1,4 @@ -const { Kafka, ErrorCodes } = require('../').KafkaJS - +const { Kafka, ErrorCodes } = require('confluent-kafka-javascript').KafkaJS; async function consumerStart() { let consumer; diff --git a/examples/eos.js b/examples/eos.js index e60b30ee..5e7debb0 100644 --- a/examples/eos.js +++ b/examples/eos.js @@ -1,5 +1,4 @@ -const { Kafka } = require('../').KafkaJS; -//const { Kafka } = require('kafkajs') +const { Kafka } = require('confluent-kafka-javascript').KafkaJS; async function eosStart() { const consumer = new Kafka().consumer({ diff --git a/examples/producer.js b/examples/producer.js index 3f551680..608b2519 100644 --- a/examples/producer.js +++ b/examples/producer.js @@ -1,5 +1,4 @@ -const { Kafka } = require('../').KafkaJS -//const { Kafka } = require('kafkajs') +const { Kafka } = require('confluent-kafka-javascript').KafkaJS; async function producerStart() { const producer = new Kafka().producer({ diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 78f389c4..c5111edb 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -16,6 +16,7 @@ const { acquireOrLog } = require('./_common'); const { Buffer } = require('buffer'); +const { hrtime } = require('process'); const ConsumerState = Object.freeze({ INIT: 0, @@ -31,6 +32,102 @@ const PartitionAssigners = Object.freeze({ cooperativeSticky: 'cooperative-sticky', }); + +/** + * MessageCache represents a cache of messages that have been consumed, + * but not yet passed to the user. + * It has a dynamic capacity, increased or decreased based on requirement. + */ +class MessageCache { + /* The cache is a list of messages. */ + cache = []; + /* The maximum size of the cache. Set to 1 initially. */ + maxSize = 1; + /* Index of next element to be fetched in the cache. */ + currentIndex = this.maxSize; + /* Whether the cache is stale. */ + stale = false; + /* Number of times the cache has been requested to be increased in size. */ + increaseCount = 0; + /* Last cached time */ + cachedTime = hrtime(); + /* Expiry duration for this cache */ + expiryDurationMs = 500; + + constructor(expiryDurationMs) { + this.expiryDurationMs = expiryDurationMs; + } + + /** + * Clears the cache. + */ + clear() { + this.cache = []; + this.maxSize = 1; + this.currentIndex = this.maxSize; + this.stale = false; + this.increaseCount = 0; + this.cachedTime = hrtime(); + } + + /** + * Request a size increase. + * It increases the size by 2x, but only if the size is less than 1024, + * only if the size has been requested to be increased twice in a row. + * @returns + */ + increaseMaxSize() { + if (this.maxSize === 1024) + return; + + this.increaseCount++; + if (this.increaseCount <= 1) + return; + + this.maxSize = Math.min(this.maxSize << 1, 1024); + this.increaseCount = 0; + } + + /** + * Request a size decrease. + * It decreases the size to 80% of the last received size, with a minimum of 1. + * @param {number} recvdSize - the number of messages received in the last poll. + */ + decreaseMaxSize(recvdSize) { + this.maxSize = Math.max(Math.floor((recvdSize * 8) / 10), 1); + this.increaseCount = 0; + } + + /** + * Sets cache and resets all the indices and timer. + * @param {*} messages + */ + setCache(messages) { + this.cache = messages; + this.currentIndex = 1; + this.cachedTime = hrtime(); + } + + /** + * @returns The next element in the cache or null if none exists. + * @warning Does not check for staleness. + */ + next() { + return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; + } + + /* Whether the cache is stale. */ + isStale() { + if (this.stale) + return true; + + const cacheTime = hrtime(this.cachedTime); + const cacheTimeMs = Math.floor(cacheTime[0] * 1000 + cacheTime[1] / 1000000); + return cacheTimeMs > this.expiryDurationMs; + } + +} + class Consumer { /** * The config supplied by the user. @@ -93,6 +190,14 @@ class Consumer { */ #logger = new DefaultLogger(); + /** + * A map of topic+partition to the offset that was last consumed. + * The keys are of the type "|". + * This is only populated when we're in the kafkaJS compatibility mode. + * @type {Map} + */ + #lastConsumedOffsets = new Map(); + /** * A lock for consuming and disconnecting. * This lock should be held whenever we want to change the state from CONNECTED to any state other than CONNECTED. @@ -108,6 +213,18 @@ class Consumer { */ #running = false; + /** + * Whether the consumer is in KafkaJS compatibility mode. + * @type {boolean} + */ + #kafkaJSCompatibilityMode = false; + + /** + * The message cache for KafkaJS compatibility mode. + * @type {MessageCache|null} + */ + #messageCache = null; + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -122,6 +239,49 @@ class Consumer { return this.#internalConfig; } + /** + * Clear the message cache. + * For simplicity, this always clears the entire message cache rather than being selective. + * + * @param {boolean} seek - whether to seek to the stored offsets after clearing the cache. + * this should be set to true if partitions are retained after this operation. + */ + async #clearCacheAndResetPositions(seek = true) { + /* Seek to stored offset for each topic partition so that if + * we've gotten further along then they have, we can come back. */ + if (seek) { + const assignment = this.assignment(); + const seekPromises = []; + for (const topicPartitionOffset of assignment) { + const key = `${topicPartitionOffset.topic}|${topicPartitionOffset.partition}`; + if (!this.#lastConsumedOffsets.has(key)) + continue; + + /* Fire off a seek */ + const seekPromise = new Promise((resolve, reject) => this.#internalClient.seek({ + topic: topicPartitionOffset.topic, + partition: topicPartitionOffset.partition, + offset: +this.#lastConsumedOffsets.get(key) + }, 10000, err => { + if (err) { + reject(err); + } else { + resolve(); + } + })); + seekPromises.push(seekPromise); + } + + /* TODO: we should cry more about this and render the consumer unusable. */ + await Promise.all(seekPromises).catch(err => this.#logger.error("Seek error. This is effectively a fatal error:" + err)); + } + + /* Clear the cache. */ + this.#messageCache.clear(); + /* Clear the offsets - no need to keep them around. */ + this.#lastConsumedOffsets.clear(); + } + /** * Used as a trampoline to the user's rebalance listener, if any. * @param {Error} err - error in rebalance @@ -166,9 +326,20 @@ class Consumer { call .finally(async () => { - // Emit the event + /* Emit the event */ this.#internalClient.emit('rebalance', err, assignment); + /** + * We never need to clear the cache in case of a rebalance. + * This is because rebalances are triggered ONLY when we call the consume() + * method of the internalClient. + * In case consume() is being called, we've already either consumed all the messages + * in the cache, or timed out (this.#messageCache.cachedTime is going to exceed max.poll.interval) + * and marked the cache stale. This means that the cache is always expired when a rebalance + * is triggered. + * This is applicable both for incremental and non-incremental rebalances. + */ + try { if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { @@ -250,6 +421,8 @@ class Consumer { if (Object.hasOwn(kjsConfig, 'rebalanceTimeout')) { /* In librdkafka, we use the max poll interval as the rebalance timeout as well. */ rdKafkaConfig['max.poll.interval.ms'] = kjsConfig.rebalanceTimeout; + } else { + rdKafkaConfig['max.poll.interval.ms'] = 300000; /* librdkafka default */ } if (Object.hasOwn(kjsConfig, 'heartbeatInterval')) { @@ -306,6 +479,8 @@ class Consumer { if (Object.hasOwn(kjsConfig, 'autoCommit')) { rdKafkaConfig['enable.auto.commit'] = kjsConfig.autoCommit; + } else { + rdKafkaConfig['enable.auto.commit'] = true; } if (Object.hasOwn(kjsConfig, 'autoCommitInterval')) { @@ -316,6 +491,11 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } + /* Offset storage management is manual in kafkaJS compability mode if auto-commit is turned on (ie by default). */ + if (rdKafkaConfig['enable.auto.commit']) { + rdKafkaConfig['enable.auto.offset.store'] = false; + } + /* Set the logger */ if (Object.hasOwn(kjsConfig, 'logger')) { this.#logger = kjsConfig.logger; @@ -337,6 +517,8 @@ class Consumer { * log level, as librdkafka will control the granularity. */ if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { this.#logger.setLogLevel(logLevel.DEBUG); + } else { + this.#kafkaJSCompatibilityMode = true; } /* Even if we are in compability mode, setting a 'debug' in the main config must override the logger's level. */ @@ -433,20 +615,51 @@ class Consumer { /** * Consumes a single message from the internal consumer. * @returns {Promise} a promise that resolves to a single message. + * @note this method caches messages as well, but returns only a single message. */ - async #consumeSingle() { + async #consumeSingleCached() { + const msg = this.#messageCache.next(); + if (msg) { + return msg; + } + return new Promise((resolve, reject) => { - this.#internalClient.consume(1, function (err, messages) { + this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { if (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } + this.#messageCache.setCache(messages); const message = messages[0]; + if (messages.length === this.#messageCache.maxSize) { + this.#messageCache.increaseMaxSize(); + } else { + this.#messageCache.decreaseMaxSize(messages.length); + } resolve(message); }); }); } + /** + * Consumes n messages from the internal consumer. + * @returns {Promise} a promise that resolves to a list of messages. + * The size of this list is guaranteed to be less + * than or equal to n. + * @note this method cannot be used in conjunction with #consumeSingleCached. + */ + async #consumeN(n) { + return new Promise((resolve, reject) => { + this.#internalClient.consume(n, (err, messages) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + return; + } + resolve(messages); + }); + }); + } + /** * Flattens a list of topics with partitions into a list of topic, partition. * @param {({topic: string, partitions: number[]}|{topic: string, partition: number})[]} topics @@ -487,6 +700,10 @@ class Consumer { const rdKafkaConfig = this.#config(); + if (this.#kafkaJSCompatibilityMode) { + this.#messageCache = new MessageCache(Math.floor(rdKafkaConfig['max.poll.interval.ms'] * 0.8)); + } + this.#state = ConsumerState.CONNECTING; this.#internalClient = new RdKafka.KafkaConsumer(rdKafkaConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); @@ -559,6 +776,8 @@ class Consumer { /** * Starts consumer polling. This method returns immediately. * @param {import("../../types/kafkajs").ConsumerRunConfig} config + * @note This should be used only when in KafkaJS compatibility mode (ie. with kafkaJS blocks in the config). + * If not in KafkaJS compatibility mode, this is deprecated, use consume() instead as the preferred method. */ async run(config) { if (this.#state !== ConsumerState.CONNECTED) { @@ -585,6 +804,10 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsPartitionsConsumedConcurrently(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } + if (!this.#kafkaJSCompatibilityMode) { + throw new error.KafkaJSError('run() can only be used in KafkaJS compatibility mode.', { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + if (this.#running) { throw new error.KafkaJSError('Consumer is already running.', { code: error.ErrorCodes.ERR__STATE }); } @@ -603,7 +826,14 @@ class Consumer { if (!(await acquireOrLog(this.#lock, this.#logger))) continue; - const m = await this.#consumeSingle().catch(e => { + /* Invalidate the message cache if needed. */ + if (this.#messageCache.isStale()) { + await this.#clearCacheAndResetPositions(true); + this.#lock.release(); + continue; + } + + const m = await this.#consumeSingleCached().catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) @@ -635,21 +865,36 @@ class Consumer { } } + let eachMessageProcessed = false; try { await config.eachMessage( this.#createPayload(m) - ) + ); + eachMessageProcessed = true; } 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). */ - if (this.#state === ConsumerState.CONNECTED) - this.seek({ - topic: m.topic, - partition: m.partition, - offset: m.offset, - }); + this.seek({ + topic: m.topic, + partition: m.partition, + offset: m.offset, + }); + } + + /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ + if (eachMessageProcessed) { + try { + if (this.#internalConfig['enable.auto.commit']) { + this.#internalClient.offsetsStore([{ topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1 }]); + } + this.#lastConsumedOffsets.set(`${m.topic}|${m.partition}`, Number(m.offset) + 1); + } catch (e) { + /* Not much we can do, except log the error. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); + } } /* Force a immediate seek here. It's possible that there are no more messages to be passed to the user, @@ -676,8 +921,11 @@ class Consumer { * @param {any} args * @param {number} args.timeout - the timeout in milliseconds, defaults to 1000. * @returns {import("../..").Message|null} a message, or null if the timeout was reached. + * + * @note This API is currently in an experimental stage and subject to change. + * This should not be used in KafkaJS compatibility mode (ie with kafkaJS blocks in the config). */ - async consume({timeout} = {timeout: 1000}) { + async consume({ timeout } = { timeout: 1000 }) { if (this.#state !== ConsumerState.CONNECTED) { throw new error.KafkaJSError('consume can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } @@ -686,16 +934,21 @@ class Consumer { throw new error.KafkaJSError('consume() and run() cannot be used together.', { code: error.ErrorCodes.ERR__CONFLICT }); } + if (this.#kafkaJSCompatibilityMode) { + throw new error.KafkaJSError('consume() cannot be used in KafkaJS compatibility mode.', { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + this.#internalClient.setDefaultConsumeTimeout(timeout); let m = null; try { - m = await this.#consumeSingle(); + const ms = await this.#consumeN(1); + m = ms[0]; } finally { this.#internalClient.setDefaultConsumeTimeout(undefined); } - return m; + return m ?? null; } async #commitOffsetsUntilNoStateErr(offsetsToCommit) { @@ -787,8 +1040,15 @@ class Consumer { offset }; + /* We need a complete reset of the cache if we're seeking to a different offset even for one partition. + * At a later point, this may be improved at the cost of added complexity of maintaining message generation, + * or else purging the cache of just those partitions which are seeked. */ + if (this.#kafkaJSCompatibilityMode) + await this.#clearCacheAndResetPositions(true); + /* 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. */ + /* TODO: is it work awaiting seeks to finish? */ this.#internalClient.seek(topicPartitionOffset, 0, err => err ? this.#logger.error(err) : null); offsetsToCommit.push({ topic: topicPartition.topic, @@ -803,7 +1063,8 @@ class Consumer { } } - if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { + /* Offsets are committed on seek only when in compatibility mode. */ + if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit'] && this.#kafkaJSCompatibilityMode) { await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); } @@ -817,6 +1078,7 @@ class Consumer { * 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|null} a promise that resolves when the seek has been performed (only when not in compatibility mode), or null (when in compatibility mode) */ seek(topicPartitionOffset) { if (this.#state !== ConsumerState.CONNECTED) { @@ -836,6 +1098,11 @@ class Consumer { this.#checkPendingSeeks = true; this.#pendingSeeks.set(`${rdKafkaTopicPartitionOffset.topic}|${rdKafkaTopicPartitionOffset.partition}`, rdKafkaTopicPartitionOffset.offset); + + /* Immediately realize the seek if we're not in compatibility mode. */ + if (!this.#kafkaJSCompatibilityMode) { + return this.#seekInternal(); + } } async describeGroup() { @@ -892,6 +1159,8 @@ class Consumer { return; } this.#internalClient.pause(topics); + if (this.#kafkaJSCompatibilityMode) + this.#messageCache.stale = true; topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 3e4e939d..4377dc8a 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -13,6 +13,7 @@ const { kafkaJSToRdKafkaConfig, const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); +const { hrtime } = require('process'); const ProducerState = Object.freeze({ INIT: 0, @@ -83,7 +84,7 @@ class Producer { * fire, and we won't poll. By maintaining the last poll time, we can * poll at the end of send() and sendBatch(). */ - #lastPollTime = process.hrtime(); + #lastPollTime = hrtime(); /** * @constructor @@ -283,7 +284,7 @@ class Producer { return; } this.#internalClient.poll(); - this.#lastPollTime = process.hrtime(); + this.#lastPollTime = hrtime(); }, producerPollIntervalMs); this.#internalClient.on('delivery-report', this.#deliveryCallback.bind(this)); @@ -554,10 +555,10 @@ class Producer { /* Poll if we haven't polled in a while. This can be the case if we're producing * in a tight loop without awaiting the produce. */ - const elapsed = process.hrtime(this.#lastPollTime); + const elapsed = hrtime(this.#lastPollTime); const elapsedInNanos = elapsed[0] * 1e9 + elapsed[1]; if (elapsedInNanos > producerPollIntervalMs * 1000) { - this.#lastPollTime = process.hrtime(); + this.#lastPollTime = hrtime(); this.#internalClient.poll(); } diff --git a/package-lock.json b/package-lock.json index 9673ebd6..6b0cc4b1 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "confluent-kafka-javascript", - "version": "v0.1.6-devel", + "version": "v0.1.7-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "confluent-kafka-javascript", - "version": "v0.1.6-devel", + "version": "v0.1.7-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { @@ -1452,9 +1452,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.11.10", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.10.tgz", - "integrity": "sha512-rZEfe/hJSGYmdfX9tvcPMYeYPW2sNl50nsw4jZmRcaG0HIAb0WYEpsB05GOb53vjqpyE9GUhlDQ4jLSoB5q9kg==", + "version": "20.11.15", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.15.tgz", + "integrity": "sha512-gscmuADZfvNULx1eyirVbr3kVOVZtpQtzKMCZpeSZcN6MfbkRXAR4s9/gsQ4CzxLHw6EStDtKLNtSDL3vbq05A==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -1996,9 +1996,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001581", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001581.tgz", - "integrity": "sha512-whlTkwhqV2tUmP3oYhtNfaWGYHDdS3JYFQBKXxcUR9qqPWsRhFHhoISO2Xnl/g0xyKzht9mI1LZpiNWfMzHixQ==", + "version": "1.0.30001582", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001582.tgz", + "integrity": "sha512-vsJG3V5vgfduaQGVxL53uSX/HUzxyr2eA8xCo36OLal7sRcSZbibJtLeh0qja4sFOr/QQGt4opB4tOy+eOgAxg==", "dev": true, "funding": [ { @@ -2436,9 +2436,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.650", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.650.tgz", - "integrity": "sha512-sYSQhJCJa4aGA1wYol5cMQgekDBlbVfTRavlGZVr3WZpDdOPcp6a6xUnFfrt8TqZhsBYYbDxJZCjGfHuGupCRQ==", + "version": "1.4.653", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.653.tgz", + "integrity": "sha512-wA2A2LQCqnEwQAvwADQq3KpMpNwgAUBnRmrFgRzHnPhbQUFArTR32Ab46f4p0MovDLcg4uqd4nCsN2hTltslpA==", "dev": true }, "node_modules/emittery": { @@ -3109,9 +3109,9 @@ } }, "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==", + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.3.1.tgz", + "integrity": "sha512-5Fytz/IraMjqpwfd34ke28PTVMjZjJG2MPn5t7OE4eUCUNf8BAa7b5WUS9/Qvr6mwOQS7Mk6vdsMno5he+T8Xw==", "dev": true, "engines": { "node": ">= 4" diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index ee94ce81..6e64c195 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -586,7 +586,7 @@ describe('Consumer', () => { ]); expect(consumer.paused()).toEqual([]); - }, 10000); + }); it('resumes fetching from earlier paused partitions', async () => { await consumer.connect(); diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index dd50dad0..746f147b 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -4,13 +4,18 @@ const { Kafka } = require('../../lib').KafkaJS; // TODO: pick this up from a file const clusterInformation = { - brokers: process.env.KAFKA_HOST ? process.env.KAFKA_HOST.split(',') : ['localhost:9092'], + kafkaJS: { + brokers: process.env.KAFKA_HOST ? process.env.KAFKA_HOST.split(',') : ['localhost:9092'], + }, + librdkafka: { + 'bootstrap.servers': process.env.KAFKA_HOST ? process.env.KAFKA_HOST : 'localhost:9092', + }, }; const debug = process.env.TEST_DEBUG; function makeConfig(config) { - const kafkaJS = Object.assign(config, clusterInformation); + const kafkaJS = Object.assign(config, clusterInformation.kafkaJS); const common = {}; if (debug) { common['debug'] = debug; @@ -102,4 +107,5 @@ module.exports = { waitFor, sleep, generateMessages, + clusterInformation, } From ea487f962d71191e219ae5da91f3918820050b31 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 1 Feb 2024 17:30:25 +0530 Subject: [PATCH 074/224] Add tests for cache and non-compatibility mode --- .../consumer/consumerCacheTests.spec.js | 272 ++++++++++++++++++ .../consumer/consumerNonCompability.spec.js | 220 ++++++++++++++ 2 files changed, 492 insertions(+) create mode 100644 test/promisified/consumer/consumerCacheTests.spec.js create mode 100644 test/promisified/consumer/consumerNonCompability.spec.js diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js new file mode 100644 index 00000000..2e4e415f --- /dev/null +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -0,0 +1,272 @@ +jest.setTimeout(30000) + +const { + secureRandom, + createTopic, + waitFor, + createProducer, + createConsumer, + waitForMessages, + sleep, +} = require('../testhelpers'); + +describe('Consumer message cache', () => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}` + + await createTopic({ topic: topicName, partitions: 3 }) + + producer = createProducer({}); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }); + + it('is cleared on pause', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + if (event.partition === 0 && (+event.message.offset) === 1023) { + consumer.pause([{ topic: topicName, partitions: [0] }]); + } + } + }); + + /* Evenly distribute 1024*9 messages across 3 partitions */ + let i = 0; + const messages = Array(1024 * 9) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: ((i++) % 3) } + }) + + await producer.send({ topic: topicName, messages }) + + // Wait for the messages. + // We consume 1024 messages from partition 0, and 1024*3 from partition 1 and 2. + await waitForMessages(messagesConsumed, { number: 1024 * 7 }); + + // We should not consume even one more message than that. + await sleep(1000); + expect(messagesConsumed.length).toEqual(1024 * 7); + + // check if all offsets are present + // partition 0 + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.message.offset)).toEqual(Array(1024).fill().map((_, i) => `${i}`)); + // partition 1 + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + // partition 2 + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + }); + + it('is cleared on seek', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + let hasBeenSeeked = false; + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + if (event.partition === 0 && (+event.message.offset) === 1023 && !hasBeenSeeked) { + consumer.seek({ topic: topicName, partition: 0, offset: 0 }); + hasBeenSeeked = true; + } + } + }); + + /* Evenly distribute 1024*9 messages across 3 partitions */ + let i = 0; + const messages = Array(1024 * 9) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: ((i++) % 3) } + }) + + await producer.send({ topic: topicName, messages }) + + // Wait for the messages. + // We consume 1024*4 messages from partition 0, and 1024*3 from partition 1 and 2. + await waitForMessages(messagesConsumed, { number: 1024 * 10 }); + + // We should not consume even one more message than that. + await sleep(1000); + expect(messagesConsumed.length).toEqual(1024 * 10); + + // check if all offsets are present + // partition 0 + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.message.offset)) + .toEqual(Array(1024 * 4).fill().map((_, i) => i < 1024 ? `${i}` : `${i - 1024}`)); + // partition 1 + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + // partition 2 + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + }); + + it('is cleared before rebalance', async () => { + const consumer2 = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + const messagesConsumedConsumer1 = []; + const messagesConsumedConsumer2 = []; + let consumer2ConsumeRunning = false; + + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + messagesConsumedConsumer1.push(event); + + /* Until the second consumer joins, consume messages slowly so as to not consume them all + * before the rebalance triggers. */ + if (messagesConsumed.length > 1024 && !consumer2ConsumeRunning) { + await sleep(10); + } + } + }); + + /* Evenly distribute 1024*9 messages across 3 partitions */ + let i = 0; + const messages = Array(1024 * 10) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.send({ topic: topicName, messages }) + + // Wait for the messages - some of them, before starting the + // second consumer. + await waitForMessages(messagesConsumed, { number: 1024 }); + + await consumer2.connect(); + await consumer2.subscribe({ topic: topicName }); + consumer2.run({ + eachMessage: async event => { + messagesConsumed.push(event); + messagesConsumedConsumer2.push(event); + } + }); + + await waitFor(() => consumer2.assignment().length > 0, () => null); + consumer2ConsumeRunning = true; + + /* Now that both consumers have joined, wait for all msgs to be consumed */ + await waitForMessages(messagesConsumed, { number: 1024 * 10 }); + + /* No extra messages should be consumed. */ + await sleep(1000); + expect(messagesConsumed.length).toEqual(1024 * 10); + + /* Check if all messages were consumed. */ + expect(messagesConsumed.map(event => (+event.message.offset)).sort((a, b) => a - b)) + .toEqual(Array(1024 * 10).fill().map((_, i) => Math.floor(i / 3))); + + /* Consumer2 should have consumed at least one message. */ + expect(messagesConsumedConsumer2.length).toBeGreaterThan(0); + + await consumer2.disconnect(); + }); + + it('does not hold up polling', async () => { + /* This consumer has a low max.poll.interval.ms */ + const impatientConsumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + rebalanceTimeout: 10000, + sessionTimeout: 10000, + clientId: "impatientConsumer", + }); + + await producer.connect(); + await impatientConsumer.connect(); + await impatientConsumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + let impatientConsumerMessages = []; + let consumer1Messages = []; + let consumer1TryingToJoin = false; + + impatientConsumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + impatientConsumerMessages.push(event); + /* When the second consumer is joining, deliberately slow down message consumption. + * We should still have a rebalance very soon, since we must expire the cache and + * trigger a rebalance before max.poll.interval.ms. + */ + if (consumer1TryingToJoin) { + await sleep(1000); + } + } + }); + + /* Distribute 1024*10 messages across 3 partitions */ + let i = 0; + const messages = Array(1024 * 10) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.send({ topic: topicName, messages }) + + /* Wait for the messages - some of them, before starting the + * second consumer. */ + await waitForMessages(messagesConsumed, { number: 1024 }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + consumer1Messages.push(event); + } + }); + consumer1TryingToJoin = true; + await waitFor(() => consumer.assignment().length > 0, () => null); + consumer1TryingToJoin = false; + + /* Now that both consumers have joined, wait for all msgs to be consumed */ + await waitForMessages(messagesConsumed, { number: 1024 * 10 }); + + // No extra messages should be consumed. + await sleep(1000); + expect(messagesConsumed.length).toEqual(1024 * 10); + + /* Each consumer should have consumed at least one message. */ + expect(consumer1Messages.length).toBeGreaterThan(0); + expect(impatientConsumerMessages.length).toBeGreaterThan(0); + + await impatientConsumer.disconnect(); + }); +}); diff --git a/test/promisified/consumer/consumerNonCompability.spec.js b/test/promisified/consumer/consumerNonCompability.spec.js new file mode 100644 index 00000000..a84ff777 --- /dev/null +++ b/test/promisified/consumer/consumerNonCompability.spec.js @@ -0,0 +1,220 @@ +jest.setTimeout(30000) + +const { Kafka } = require('../../../lib/kafkajs'); +const { + secureRandom, + createTopic, + waitFor, + createProducer, + createConsumer, + waitForMessages, + clusterInformation, + sleep, +} = require('../testhelpers'); + +describe('Consumer (non-compability)', () => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}` + + await createTopic({ topic: topicName, partitions: 3 }) + + producer = new Kafka().producer({ + 'bootstrap.servers': clusterInformation.librdkafka['bootstrap.servers'], + }); + + consumer = new Kafka().consumer({ + 'bootstrap.servers': clusterInformation.librdkafka['bootstrap.servers'], + 'group.id': groupId, + 'auto.offset.reset': 'earliest', + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }); + + it('is able to use consume()', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + /* Produce 10 messages */ + let i = 0; + const messages = Array(9) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: ((i++) % 3) } + }) + await producer.send({ topic: topicName, messages }) + + const messagesConsumed = []; + while (messagesConsumed.length < 9) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + + // check if all offsets are present + // partition 0 + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 1 + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 2 + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + }); + + it('cannot use consume() in compability mode', async () => { + consumer = createConsumer({ + groupId, + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + await expect(consumer.consume()).rejects.toThrow(/cannot be used in KafkaJS compatibility mode/); + }); + + it('is able to use consume() and pause()', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + /* Produce 10 messages */ + let i = 0; + const messages = Array(9) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: ((i++) % 3) } + }) + await producer.send({ topic: topicName, messages }); + + const messagesConsumed = []; + while (messagesConsumed.length < 5) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + + await consumer.pause([{ topic: topicName }]); + + let msg = await consumer.consume(); + expect(msg).toBeNull(); + + await consumer.resume([{ topic: topicName }]); + + while (messagesConsumed.length < 9) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + + // check if all offsets are present + // partition 0 + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 1 + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 2 + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + }); + + it('is able to use consume() and seek()', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + /* Produce 10 messages */ + let i = 0; + const messages = Array(9) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: ((i++) % 3) } + }) + await producer.send({ topic: topicName, messages }); + + let messagesConsumed = []; + while (messagesConsumed.length < 9) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + + await consumer.seek({ topic: topicName, partition: 0, offset: 0 }); + + while (messagesConsumed.length < 12) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + + // check if all offsets are present + // partition 0 - repeated completely because of seek + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(6).fill().map((_, i) => i % 3)); + // partition 1 + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 2 + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + }); + + it('is able to commit offsets (using auto commit)', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + /* Produce 10 messages */ + let i = 0; + const messages = Array(9) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: ((i++) % 3) } + }) + await producer.send({ topic: topicName, messages }) + + const messagesConsumed = []; + while (messagesConsumed.length < 6) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + await consumer.disconnect(); + + /* Create new consumer and append to that array. We should get the last 3 messages */ + consumer = new Kafka().consumer({ + 'bootstrap.servers': clusterInformation.librdkafka['bootstrap.servers'], + 'group.id': groupId, + 'auto.offset.reset': 'earliest', + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + + while (messagesConsumed.length < 9) { + const msg = await consumer.consume(); + if (!msg) + continue; + messagesConsumed.push(msg); + } + /* No more messages when the total adds up. */ + await expect(consumer.consume()).resolves.toBeNull(); + + // check if all offsets are present + // partition 0 + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 1 + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + // partition 2 + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); + + }); +}); From 6e49c4640b244361099e1b7eb4190319fd28ea43 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 2 Feb 2024 12:23:45 +0530 Subject: [PATCH 075/224] Add some checks for kafkaJS compability mode --- lib/kafkajs/_consumer.js | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index c5111edb..c765d240 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -343,7 +343,7 @@ class Consumer { try { if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { - if (this.#checkPendingSeeks) + if (this.#checkPendingSeeks && this.#kafkaJSCompatibilityMode) assignment = this.#assignAsPerSeekedOffsets(assignment); this.#internalClient.assign(assignment); @@ -1099,9 +1099,10 @@ class Consumer { this.#checkPendingSeeks = true; this.#pendingSeeks.set(`${rdKafkaTopicPartitionOffset.topic}|${rdKafkaTopicPartitionOffset.partition}`, rdKafkaTopicPartitionOffset.offset); - /* Immediately realize the seek if we're not in compatibility mode. */ + /* Immediately realize the seek if we're not in compatibility mode. And clear pending seeks. + * We don't need them for rebalance. */ if (!this.#kafkaJSCompatibilityMode) { - return this.#seekInternal(); + return this.#seekInternal().then(() => this.#pendingSeeks.clear()); } } From fa65e508f2c373255380ca8348dc7ee2922e662e Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 2 Feb 2024 12:24:47 +0530 Subject: [PATCH 076/224] Add parametrization to tests for auto commit --- .../consumer/consumeMessages.spec.js | 19 ++++++++++++++++--- .../consumer/consumerCacheTests.spec.js | 18 ++++++++++++++++-- test/promisified/consumer/subscribe.spec.js | 4 ++-- test/promisified/testhelpers.js | 15 +++++++-------- 4 files changed, 41 insertions(+), 15 deletions(-) diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 4b4b117b..440644eb 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -13,9 +13,9 @@ const { waitForConsumerToJoinGroup, sleep, generateMessages, -} = require('../testhelpers') +} = require('../testhelpers'); -describe('Consumer', () => { +describe.each([[true], [false]])('Consumer', (isAutoCommit) => { let topicName, groupId, producer, consumer; beforeEach(async () => { @@ -30,6 +30,7 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 100, fromBeginning: true, + autoCommit: isAutoCommit, }); }); @@ -307,6 +308,7 @@ describe('Consumer', () => { minBytes: 1024, maxWaitTimeInMs: 500, fromBeginning: true, + autoCommit: isAutoCommit, }) const messages = Array(10) @@ -492,6 +494,7 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 100, fromBeginning: true, + autoCommit: isAutoCommit }); await consumer.connect(); @@ -528,7 +531,8 @@ describe('Consumer', () => { consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, - fromBeginning: true + fromBeginning: true, + autoCommit: isAutoCommit, }); await consumer.connect(); @@ -585,6 +589,7 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 100, fromBeginning: true, + autoCommit: isAutoCommit, }); await consumer.connect(); @@ -642,6 +647,7 @@ describe('Consumer', () => { maxWaitTimeInMs: 100, readUncommitted: true, fromBeginning: true, + autoCommit: isAutoCommit, }) await consumer.connect(); @@ -678,6 +684,9 @@ describe('Consumer', () => { it( 'respects offsets sent by a committed transaction ("consume-transform-produce" flow)', async () => { + if (isAutoCommit) { /* only autoCommit: false makes sense for this test. */ + return; + } // Seed the topic with some messages. We don't need a tx producer for this. await producer.connect(); const partition = 0; @@ -785,6 +794,10 @@ describe('Consumer', () => { it( 'does not respect offsets sent by an aborted transaction ("consume-transform-produce" flow)', async () => { + if (isAutoCommit) { /* only autoCommit: false makes sense for this test. */ + return; + } + // Seed the topic with some messages. We don't need a tx producer for this. await producer.connect(); diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index 2e4e415f..56033c60 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -1,5 +1,6 @@ jest.setTimeout(30000) +const { is } = require('bluebird'); const { secureRandom, createTopic, @@ -10,7 +11,7 @@ const { sleep, } = require('../testhelpers'); -describe('Consumer message cache', () => { +describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { let topicName, groupId, producer, consumer; beforeEach(async () => { @@ -21,10 +22,12 @@ describe('Consumer message cache', () => { producer = createProducer({}); + const common = {}; consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, fromBeginning: true, + autoCommit: isAutoCommit, }); }); @@ -127,6 +130,7 @@ describe('Consumer message cache', () => { groupId, maxWaitTimeInMs: 100, fromBeginning: true, + autoCommit: isAutoCommit, }); await consumer.connect(); @@ -142,6 +146,10 @@ describe('Consumer message cache', () => { eachMessage: async event => { messagesConsumed.push(event); messagesConsumedConsumer1.push(event); + if (!isAutoCommit) + await consumer.commitOffsets([ + { topic: event.topic, partition: event.partition, offset: Number(event.message.offset) + 1 }, + ]); /* Until the second consumer joins, consume messages slowly so as to not consume them all * before the rebalance triggers. */ @@ -204,6 +212,7 @@ describe('Consumer message cache', () => { rebalanceTimeout: 10000, sessionTimeout: 10000, clientId: "impatientConsumer", + autoCommit: isAutoCommit, }); await producer.connect(); @@ -219,6 +228,11 @@ describe('Consumer message cache', () => { eachMessage: async event => { messagesConsumed.push(event); impatientConsumerMessages.push(event); + if (!isAutoCommit) + await impatientConsumer.commitOffsets([ + { topic: event.topic, partition: event.partition, offset: Number(event.message.offset) + 1 }, + ]); + /* When the second consumer is joining, deliberately slow down message consumption. * We should still have a rebalance very soon, since we must expire the cache and * trigger a rebalance before max.poll.interval.ms. @@ -268,5 +282,5 @@ describe('Consumer message cache', () => { expect(impatientConsumerMessages.length).toBeGreaterThan(0); await impatientConsumer.disconnect(); - }); + }, 60000); }); diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index c6005910..5d407b2c 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -3,6 +3,7 @@ jest.setTimeout(30000); const { Kafka, ErrorCodes } = require('../../../lib').KafkaJS; const { secureRandom, createTopic, + waitFor, waitForMessages, waitForConsumerToJoinGroup, createProducer, @@ -13,7 +14,6 @@ describe('Consumer', () => { beforeEach(async () => { groupId = `consumer-group-id-${secureRandom()}`; - consumer = createConsumer({ groupId, maxWaitTimeInMs: 1, @@ -52,7 +52,7 @@ describe('Consumer', () => { }); consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); - await waitForConsumerToJoinGroup(consumer); + await waitFor(() => consumer.assignment().length > 0, () => null); await producer.connect(); await producer.sendBatch({ diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index 746f147b..bcb1f99f 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -14,9 +14,8 @@ const clusterInformation = { const debug = process.env.TEST_DEBUG; -function makeConfig(config) { +function makeConfig(config, common) { const kafkaJS = Object.assign(config, clusterInformation.kafkaJS); - const common = {}; if (debug) { common['debug'] = debug; } @@ -24,18 +23,18 @@ function makeConfig(config) { return Object.assign(common, { kafkaJS }); } -function createConsumer(config) { - const kafka = new Kafka(makeConfig(config)); +function createConsumer(config, common = {}) { + const kafka = new Kafka(makeConfig(config, common)); return kafka.consumer(); } -function createProducer(config) { - const kafka = new Kafka(makeConfig(config)); +function createProducer(config, common = {}) { + const kafka = new Kafka(makeConfig(config, common)); return kafka.producer(); } -function createAdmin(config) { - const kafka = new Kafka(makeConfig(config)); +function createAdmin(config, common = {}) { + const kafka = new Kafka(makeConfig(config, common)); return kafka.admin(); } From 80d552d64db678b3a2042d7aecc05c24fcc08afe Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 2 Feb 2024 13:08:16 +0530 Subject: [PATCH 077/224] Add client software name and version --- CONTRIBUTING.md | 2 +- lib/client.js | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index ff872c4c..78bcd73f 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -231,7 +231,7 @@ Steps to update: ## Releasing -1. Increment the `version` in `package.json` and change README.md to point to the new version. +1. Increment the `version` in `package.json`. Change the version in `client.js` and `README.md`. 1. Run `npm install` to update the `package-lock.json` file. diff --git a/lib/client.js b/lib/client.js index d0d1c4b7..c67ee26b 100644 --- a/lib/client.js +++ b/lib/client.js @@ -14,6 +14,8 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); +const bindingVersion = 'v0.1.7-devel'; + var LibrdKafkaError = require('./error'); util.inherits(Client, Emitter); @@ -50,6 +52,9 @@ function Client(globalConf, SubClientType, topicConf) { // key is a real conf value delete globalConf.event_cb; + globalConf['client.software.name'] = 'confluent-kafka-javascript'; + globalConf['client.software.version'] = `${bindingVersion}-librdkafka-${Kafka.librdkafkaVersion}`; + this._client = new SubClientType(globalConf, topicConf); var extractFunctions = function(obj) { From 905ff9ec579b4a49508b7c51ba080d7b825bb857 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 2 Feb 2024 14:29:15 +0530 Subject: [PATCH 078/224] Don't modify globalConf while creating client. --- lib/client.js | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/lib/client.js b/lib/client.js index c67ee26b..e0724fcd 100644 --- a/lib/client.js +++ b/lib/client.js @@ -52,11 +52,18 @@ function Client(globalConf, SubClientType, topicConf) { // key is a real conf value delete globalConf.event_cb; + // These properties are not meant to be user-set. + // Clients derived from this might want to change them, but for + // now we override them. globalConf['client.software.name'] = 'confluent-kafka-javascript'; globalConf['client.software.version'] = `${bindingVersion}-librdkafka-${Kafka.librdkafkaVersion}`; this._client = new SubClientType(globalConf, topicConf); + // We should not modify the globalConf object. We have cloned it already. + delete globalConf['client.software.name']; + delete globalConf['client.software.version']; + var extractFunctions = function(obj) { obj = obj || {}; var obj2 = {}; From 6ad531d54ab32f0faa5a3e18ef7f75fcd579e51a Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 6 Feb 2024 15:18:44 +0530 Subject: [PATCH 079/224] Remove kafkaJsCompatibilityMode variable and assume true --- lib/kafkajs/_consumer.js | 82 +++---- .../consumer/consumerNonCompability.spec.js | 220 ------------------ 2 files changed, 35 insertions(+), 267 deletions(-) delete mode 100644 test/promisified/consumer/consumerNonCompability.spec.js diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index c765d240..2f8ac861 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -213,18 +213,17 @@ class Consumer { */ #running = false; - /** - * Whether the consumer is in KafkaJS compatibility mode. - * @type {boolean} - */ - #kafkaJSCompatibilityMode = false; - /** * The message cache for KafkaJS compatibility mode. * @type {MessageCache|null} */ #messageCache = null; + /** + * Whether the user has enabled manual offset management (stores). + */ + #userManagedStores = false; + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -343,12 +342,13 @@ class Consumer { try { if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { - if (this.#checkPendingSeeks && this.#kafkaJSCompatibilityMode) + const checkPendingSeeks = this.#pendingSeeks.size !== 0; + if (checkPendingSeeks) assignment = this.#assignAsPerSeekedOffsets(assignment); this.#internalClient.assign(assignment); - if (this.#checkPendingSeeks) { + if (checkPendingSeeks) { const offsetsToCommit = assignment .filter((topicPartition) => topicPartition.offset !== undefined) .map((topicPartition) => ({ @@ -491,11 +491,6 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - /* Offset storage management is manual in kafkaJS compability mode if auto-commit is turned on (ie by default). */ - if (rdKafkaConfig['enable.auto.commit']) { - rdKafkaConfig['enable.auto.offset.store'] = false; - } - /* Set the logger */ if (Object.hasOwn(kjsConfig, 'logger')) { this.#logger = kjsConfig.logger; @@ -517,8 +512,6 @@ class Consumer { * log level, as librdkafka will control the granularity. */ if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { this.#logger.setLogLevel(logLevel.DEBUG); - } else { - this.#kafkaJSCompatibilityMode = true; } /* Even if we are in compability mode, setting a 'debug' in the main config must override the logger's level. */ @@ -542,6 +535,21 @@ class Consumer { } rdKafkaConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); + /* Offset management is different from case to case. + * Case 1: User has changed value of enable.auto.offset.store. In this case, we respect that. + * Case 2: automatic committing is on. In this case, we turn off auto.offset.store and store offsets manually. + * this is necessary for cache invalidation and management, as we want to put things into the store + * after eachMessage is called, and not on consume itself. + * Case 3: automatic committing is off. In this case, we turn off auto.offset.store too. Since the user might + * call an empty commit() and expect things to work properly (ie. the right offsets be stored). + * All this works out a singular, simple condition. + */ + if (!Object.hasOwn(this.#userConfig, 'enable.auto.offset.store')) { + rdKafkaConfig['enable.auto.offset.store'] = false; + } else { + this.#userManagedStores = !rdKafkaConfig['enable.auto.offset.store']; + } + return rdKafkaConfig; } @@ -699,10 +707,7 @@ class Consumer { } const rdKafkaConfig = this.#config(); - - if (this.#kafkaJSCompatibilityMode) { - this.#messageCache = new MessageCache(Math.floor(rdKafkaConfig['max.poll.interval.ms'] * 0.8)); - } + this.#messageCache = new MessageCache(Math.floor(rdKafkaConfig['max.poll.interval.ms'] * 0.8)); this.#state = ConsumerState.CONNECTING; this.#internalClient = new RdKafka.KafkaConsumer(rdKafkaConfig); @@ -804,10 +809,6 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsPartitionsConsumedConcurrently(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - if (!this.#kafkaJSCompatibilityMode) { - throw new error.KafkaJSError('run() can only be used in KafkaJS compatibility mode.', { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); - } - if (this.#running) { throw new error.KafkaJSError('Consumer is already running.', { code: error.ErrorCodes.ERR__STATE }); } @@ -829,7 +830,7 @@ class Consumer { /* Invalidate the message cache if needed. */ if (this.#messageCache.isStale()) { await this.#clearCacheAndResetPositions(true); - this.#lock.release(); + await this.#lock.release(); continue; } @@ -841,7 +842,7 @@ class Consumer { }); if (!m) { - this.#lock.release(); + await this.#lock.release(); continue; } @@ -876,7 +877,7 @@ class Consumer { * 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({ + await this.seek({ topic: m.topic, partition: m.partition, offset: m.offset, @@ -886,7 +887,7 @@ class Consumer { /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ if (eachMessageProcessed) { try { - if (this.#internalConfig['enable.auto.commit']) { + if (!this.#userManagedStores) { this.#internalClient.offsetsStore([{ topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1 }]); } this.#lastConsumedOffsets.set(`${m.topic}|${m.partition}`, Number(m.offset) + 1); @@ -916,14 +917,12 @@ class Consumer { /** * Consumes a single message from the consumer within the given timeout. + * THIS METHOD IS NOT IMPLEMENTED. * @note This method cannot be used with run(). Either that, or this must be used. * * @param {any} args * @param {number} args.timeout - the timeout in milliseconds, defaults to 1000. * @returns {import("../..").Message|null} a message, or null if the timeout was reached. - * - * @note This API is currently in an experimental stage and subject to change. - * This should not be used in KafkaJS compatibility mode (ie with kafkaJS blocks in the config). */ async consume({ timeout } = { timeout: 1000 }) { if (this.#state !== ConsumerState.CONNECTED) { @@ -934,10 +933,6 @@ class Consumer { throw new error.KafkaJSError('consume() and run() cannot be used together.', { code: error.ErrorCodes.ERR__CONFLICT }); } - if (this.#kafkaJSCompatibilityMode) { - throw new error.KafkaJSError('consume() cannot be used in KafkaJS compatibility mode.', { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); - } - this.#internalClient.setDefaultConsumeTimeout(timeout); let m = null; @@ -948,7 +943,8 @@ class Consumer { this.#internalClient.setDefaultConsumeTimeout(undefined); } - return m ?? null; + throw new error.KafkaJSError('consume() is not implemented.' + m, { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + // return m ?? null; } async #commitOffsetsUntilNoStateErr(offsetsToCommit) { @@ -1043,8 +1039,7 @@ class Consumer { /* We need a complete reset of the cache if we're seeking to a different offset even for one partition. * At a later point, this may be improved at the cost of added complexity of maintaining message generation, * or else purging the cache of just those partitions which are seeked. */ - if (this.#kafkaJSCompatibilityMode) - await this.#clearCacheAndResetPositions(true); + await this.#clearCacheAndResetPositions(true); /* 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. */ @@ -1064,7 +1059,7 @@ class Consumer { } /* Offsets are committed on seek only when in compatibility mode. */ - if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit'] && this.#kafkaJSCompatibilityMode) { + if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); } @@ -1078,7 +1073,7 @@ class Consumer { * 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|null} a promise that resolves when the seek has been performed (only when not in compatibility mode), or null (when in compatibility mode) + * @returns {Promise|null} a promise that resolves when the seek has been performed. */ seek(topicPartitionOffset) { if (this.#state !== ConsumerState.CONNECTED) { @@ -1098,12 +1093,6 @@ class Consumer { this.#checkPendingSeeks = true; this.#pendingSeeks.set(`${rdKafkaTopicPartitionOffset.topic}|${rdKafkaTopicPartitionOffset.partition}`, rdKafkaTopicPartitionOffset.offset); - - /* Immediately realize the seek if we're not in compatibility mode. And clear pending seeks. - * We don't need them for rebalance. */ - if (!this.#kafkaJSCompatibilityMode) { - return this.#seekInternal().then(() => this.#pendingSeeks.clear()); - } } async describeGroup() { @@ -1160,8 +1149,7 @@ class Consumer { return; } this.#internalClient.pause(topics); - if (this.#kafkaJSCompatibilityMode) - this.#messageCache.stale = true; + this.#messageCache.stale = true; topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); diff --git a/test/promisified/consumer/consumerNonCompability.spec.js b/test/promisified/consumer/consumerNonCompability.spec.js deleted file mode 100644 index a84ff777..00000000 --- a/test/promisified/consumer/consumerNonCompability.spec.js +++ /dev/null @@ -1,220 +0,0 @@ -jest.setTimeout(30000) - -const { Kafka } = require('../../../lib/kafkajs'); -const { - secureRandom, - createTopic, - waitFor, - createProducer, - createConsumer, - waitForMessages, - clusterInformation, - sleep, -} = require('../testhelpers'); - -describe('Consumer (non-compability)', () => { - let topicName, groupId, producer, consumer; - - beforeEach(async () => { - topicName = `test-topic-${secureRandom()}` - groupId = `consumer-group-id-${secureRandom()}` - - await createTopic({ topic: topicName, partitions: 3 }) - - producer = new Kafka().producer({ - 'bootstrap.servers': clusterInformation.librdkafka['bootstrap.servers'], - }); - - consumer = new Kafka().consumer({ - 'bootstrap.servers': clusterInformation.librdkafka['bootstrap.servers'], - 'group.id': groupId, - 'auto.offset.reset': 'earliest', - }); - }); - - afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) - }); - - it('is able to use consume()', async () => { - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }) - - /* Produce 10 messages */ - let i = 0; - const messages = Array(9) - .fill() - .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: ((i++) % 3) } - }) - await producer.send({ topic: topicName, messages }) - - const messagesConsumed = []; - while (messagesConsumed.length < 9) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - - // check if all offsets are present - // partition 0 - expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 1 - expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 2 - expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - }); - - it('cannot use consume() in compability mode', async () => { - consumer = createConsumer({ - groupId, - }); - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - - await expect(consumer.consume()).rejects.toThrow(/cannot be used in KafkaJS compatibility mode/); - }); - - it('is able to use consume() and pause()', async () => { - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }) - - /* Produce 10 messages */ - let i = 0; - const messages = Array(9) - .fill() - .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: ((i++) % 3) } - }) - await producer.send({ topic: topicName, messages }); - - const messagesConsumed = []; - while (messagesConsumed.length < 5) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - - await consumer.pause([{ topic: topicName }]); - - let msg = await consumer.consume(); - expect(msg).toBeNull(); - - await consumer.resume([{ topic: topicName }]); - - while (messagesConsumed.length < 9) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - - // check if all offsets are present - // partition 0 - expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 1 - expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 2 - expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - }); - - it('is able to use consume() and seek()', async () => { - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }) - - /* Produce 10 messages */ - let i = 0; - const messages = Array(9) - .fill() - .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: ((i++) % 3) } - }) - await producer.send({ topic: topicName, messages }); - - let messagesConsumed = []; - while (messagesConsumed.length < 9) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - - await consumer.seek({ topic: topicName, partition: 0, offset: 0 }); - - while (messagesConsumed.length < 12) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - - // check if all offsets are present - // partition 0 - repeated completely because of seek - expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(6).fill().map((_, i) => i % 3)); - // partition 1 - expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 2 - expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - }); - - it('is able to commit offsets (using auto commit)', async () => { - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }) - - /* Produce 10 messages */ - let i = 0; - const messages = Array(9) - .fill() - .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: ((i++) % 3) } - }) - await producer.send({ topic: topicName, messages }) - - const messagesConsumed = []; - while (messagesConsumed.length < 6) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - await consumer.disconnect(); - - /* Create new consumer and append to that array. We should get the last 3 messages */ - consumer = new Kafka().consumer({ - 'bootstrap.servers': clusterInformation.librdkafka['bootstrap.servers'], - 'group.id': groupId, - 'auto.offset.reset': 'earliest', - }); - await consumer.connect(); - await consumer.subscribe({ topic: topicName }) - - while (messagesConsumed.length < 9) { - const msg = await consumer.consume(); - if (!msg) - continue; - messagesConsumed.push(msg); - } - /* No more messages when the total adds up. */ - await expect(consumer.consume()).resolves.toBeNull(); - - // check if all offsets are present - // partition 0 - expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 1 - expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - // partition 2 - expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.offset)).toEqual(Array(3).fill().map((_, i) => i)); - - }); -}); From 40119c667e744ee8c91136394ebd502d169f14ca Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 12 Feb 2024 15:31:09 +0530 Subject: [PATCH 080/224] Add setSaslCredentials for producer --- index.d.ts | 2 ++ lib/client.js | 15 ++++++++++ lib/kafkajs/_producer.js | 34 +++++++++++++++++++++ src/admin.cc | 1 + src/connection.cc | 64 ++++++++++++++++++++++++++++++++++++++++ src/connection.h | 3 ++ src/kafka-consumer.cc | 1 + src/producer.cc | 13 +------- 8 files changed, 121 insertions(+), 12 deletions(-) diff --git a/index.d.ts b/index.d.ts index e361e11c..c4ee556d 100644 --- a/index.d.ts +++ b/index.d.ts @@ -198,6 +198,8 @@ export abstract class Client extends EventEmitter { queryWatermarkOffsets(topic: string, partition: number, timeout: number, cb?: (err: LibrdKafkaError, offsets: WatermarkOffsets) => any): any; queryWatermarkOffsets(topic: string, partition: number, cb?: (err: LibrdKafkaError, offsets: WatermarkOffsets) => any): any; + setSaslCredentials(username: string, password: string): void; + on(event: E, listener: EventListener): this; once(event: E, listener: EventListener): this; } diff --git a/lib/client.js b/lib/client.js index e0724fcd..1c29594a 100644 --- a/lib/client.js +++ b/lib/client.js @@ -466,6 +466,21 @@ Client.prototype.offsetsForTimes = function(toppars, timeout, cb) { }); }; +/** + * Change SASL credentials to be sent on the next authentication attempt. + * + * @param {string} username + * @param {string} password + * @note Only applicable if SASL authentication is being used. + */ +Client.prototype.setSaslCredentials = function(username, password) { + if (!this.isConnected()) { + return; + } + + this._client.setSaslCredentials(username, password); +}; + /** * Wrap a potential RdKafka error. * diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 4377dc8a..3cc5e55b 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -645,6 +645,40 @@ class Producer { logger() { return this.#logger; } + + /** + * Change SASL credentials to be sent on the next authentication attempt. + * + * @param {string} args.username + * @param {string} args.password + * @note Only applicable if SASL authentication is being used. + */ + setSaslCredentials(args = {}) { + if (!Object.hasOwn(args, 'username')) { + throw new error.KafkaJSError("username must be set for setSaslCredentials", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (!Object.hasOwn(args, 'password')) { + throw new error.KafkaJSError("password must be set for setSaslCredentials", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + /** + * In case we've not started connecting yet, just modify the configuration for + * the first connection attempt. + */ + if (this.#state < ProducerState.CONNECTING) { + this.#userConfig['sasl.username'] = args.username; + this.#userConfig['sasl.password'] = args.password; + if (Object.hasOwn(this.#userConfig, 'kafkaJS') && Object.hasOwn(this.#userConfig.kafkaJS, 'sasl')) { + this.#userConfig.kafkaJS.sasl.username = args.username; + this.#userConfig.kafkaJS.sasl.password = args.password; + } + return; + } + + + this.#internalClient.setSaslCredentials(args.username, args.password); + } } module.exports = { Producer, CompressionTypes }; diff --git a/src/admin.cc b/src/admin.cc index 56e8ada5..18f53b27 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -88,6 +88,7 @@ void AdminClient::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "connect", NodeConnect); Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect); + Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); constructor.Reset( (tpl->GetFunction(Nan::GetCurrentContext())).ToLocalChecked()); diff --git a/src/connection.cc b/src/connection.cc index 654e2db0..ae19fe50 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -67,6 +67,18 @@ Connection::~Connection() { } } +Baton Connection::rdkafkaErrorToBaton(RdKafka::Error* error) { + if ( NULL == error) { + return Baton(RdKafka::ERR_NO_ERROR); + } + else { + Baton result(error->code(), error->str(), error->is_fatal(), + error->is_retriable(), error->txn_requires_abort()); + delete error; + return result; + } +} + RdKafka::TopicPartition* Connection::GetPartition(std::string &topic) { return RdKafka::TopicPartition::create(topic, RdKafka::Topic::PARTITION_UA); } @@ -215,6 +227,25 @@ Baton Connection::GetMetadata( } } +Baton Connection::SetSaslCredentials( + std::string username, std::string password) { + RdKafka::Error *error; + + if (IsConnected()) { + scoped_shared_read_lock lock(m_connection_lock); + if (IsConnected()) { + // Always send true - we + error = m_client->sasl_set_credentials(username, password); + } else { + return Baton(RdKafka::ERR__STATE); + } + } else { + return Baton(RdKafka::ERR__STATE); + } + + return rdkafkaErrorToBaton(error); +} + void Connection::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add) { if (string_key.compare("event_cb") == 0) { if (add) { @@ -337,6 +368,39 @@ NAN_METHOD(Connection::NodeQueryWatermarkOffsets) { info.GetReturnValue().Set(Nan::Null()); } +NAN_METHOD(Connection::NodeSetSaslCredentials) { + if (!info[0]->IsString()) { + Nan::ThrowError("1st parameter must be a username string"); + return; + } + + if (!info[1]->IsString()) { + Nan::ThrowError("2nd parameter must be a password string"); + return; + } + + // Get string pointer for the username + Nan::Utf8String usernameUTF8(Nan::To(info[0]).ToLocalChecked()); + // The first parameter is the username + std::string username(*usernameUTF8); + + // Get string pointer for the password + Nan::Utf8String passwordUTF8(Nan::To(info[1]).ToLocalChecked()); + // The first parameter is the password + std::string password(*passwordUTF8); + + Connection* obj = ObjectWrap::Unwrap(info.This()); + Baton b = obj->SetSaslCredentials(username, password); + + if (b.err() != RdKafka::ERR_NO_ERROR) { + v8::Local errorObject = b.ToObject(); + return Nan::ThrowError(errorObject); + } + + info.GetReturnValue().Set(Nan::Null()); +} + + // Node methods NAN_METHOD(Connection::NodeConfigureCallbacks) { Nan::HandleScope scope; diff --git a/src/connection.h b/src/connection.h index 92381156..e5fc5b37 100644 --- a/src/connection.h +++ b/src/connection.h @@ -55,6 +55,7 @@ class Connection : public Nan::ObjectWrap { Baton GetMetadata(bool, std::string, int); Baton QueryWatermarkOffsets(std::string, int32_t, int64_t*, int64_t*, int); Baton OffsetsForTimes(std::vector &, int); + Baton SetSaslCredentials(std::string, std::string); RdKafka::Handle* GetClient(); @@ -74,6 +75,7 @@ class Connection : public Nan::ObjectWrap { static Nan::Persistent constructor; static void New(const Nan::FunctionCallbackInfo& info); + Baton rdkafkaErrorToBaton(RdKafka::Error* error); bool m_has_been_disconnected; bool m_is_closing; @@ -90,6 +92,7 @@ class Connection : public Nan::ObjectWrap { static NAN_METHOD(NodeGetMetadata); static NAN_METHOD(NodeQueryWatermarkOffsets); static NAN_METHOD(NodeOffsetsForTimes); + static NAN_METHOD(NodeSetSaslCredentials); }; } // namespace NodeKafka diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 08deac73..4d392452 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -504,6 +504,7 @@ void KafkaConsumer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "queryWatermarkOffsets", NodeQueryWatermarkOffsets); // NOLINT Nan::SetPrototypeMethod(tpl, "offsetsForTimes", NodeOffsetsForTimes); Nan::SetPrototypeMethod(tpl, "getWatermarkOffsets", NodeGetWatermarkOffsets); + Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); /* * @brief Methods exposed to do with message retrieval diff --git a/src/producer.cc b/src/producer.cc index c1c5d842..f8c6df93 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -71,6 +71,7 @@ void Producer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "getMetadata", NodeGetMetadata); Nan::SetPrototypeMethod(tpl, "queryWatermarkOffsets", NodeQueryWatermarkOffsets); // NOLINT Nan::SetPrototypeMethod(tpl, "poll", NodePoll); + Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); /* * @brief Methods exposed to do with message production @@ -351,18 +352,6 @@ void Producer::ConfigureCallback(const std::string &string_key, const v8::Local< } } -Baton rdkafkaErrorToBaton(RdKafka::Error* error) { - if ( NULL == error) { - return Baton(RdKafka::ERR_NO_ERROR); - } - else { - Baton result(error->code(), error->str(), error->is_fatal(), - error->is_retriable(), error->txn_requires_abort()); - delete error; - return result; - } -} - Baton Producer::InitTransactions(int32_t timeout_ms) { if (!IsConnected()) { return Baton(RdKafka::ERR__STATE); From 64cba1895108c531a9f2a86ec5dd9258c6b25856 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 13 Feb 2024 13:33:07 +0530 Subject: [PATCH 081/224] Add Producer flush --- lib/kafkajs/_producer.js | 55 ++++++++++++++++ src/workers.h | 7 +- test/promisified/producer/flush.spec.js | 88 +++++++++++++++++++++++++ 3 files changed, 149 insertions(+), 1 deletion(-) create mode 100644 test/promisified/producer/flush.spec.js diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 3cc5e55b..12f42d87 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -679,6 +679,61 @@ class Producer { this.#internalClient.setSaslCredentials(args.username, args.password); } + + /** + * Flushes any pending messages. + * + * Messages are batched internally by librdkafka for performance reasons. + * Continously sent messages are batched upto a timeout, or upto a maximum + * size. Calling flush sends any pending messages immediately without + * waiting for this size or timeout. + * + * @param {number} args.timeout Time to try flushing for in milliseconds. + * @returns {Promise} Resolves on successful flush. + * @throws {KafkaJSTimeout} if the flush times out. + * + * @note This is only useful when using asynchronous sends. + * For example, the following code does not get any benefit from flushing, + * since `await`ing the send waits for the delivery report, and the message + * has already been sent by the time we start flushing: + * for (let i = 0; i < 100; i++) await send(...); + * await flush(...) // Not useful. + * + * However, using the following code may put these 5 messages into a batch + * and then the subsequent `flush` will send the batch altogether (as long as + * batch size, etc. are conducive to batching): + * for (let i = 0; i < 5; i++) send(...); + * await flush({timeout: 5000}); + */ + async flush(args = { timeout: 500 }) { + if (this.#state !== ProducerState.CONNECTED) { + throw new error.KafkaJSError("Cannot flush without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); + } + + if (!Object.hasOwn(args, 'timeout')) { + throw new error.KafkaJSError("timeout must be set for flushing", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + return new Promise((resolve, reject) => { + this.#internalClient.flush(args.timeout, (err) => { + if (err) { + const kjsErr = createKafkaJsErrorFromLibRdKafkaError(err); + if (err.code === error.ErrorCodes.ERR__TIMED_OUT) { + /* See reason below for yield. Same here - but for partially processed delivery reports. */ + setTimeout(() => reject(kjsErr), 0); + } else { + reject(kjsErr); + } + return; + } + /* Yielding here allows any 'then's and 'awaits' on associated sends to be scheduled + * before flush completes, which means that the user doesn't have to yield themselves. + * It's not necessary that all the 'then's and 'awaits' will be able to run, but + * it's better than nothing. */ + setTimeout(resolve, 0); + }); + }); + } } module.exports = { Producer, CompressionTypes }; diff --git a/src/workers.h b/src/workers.h index ce79b75a..a6de4de9 100644 --- a/src/workers.h +++ b/src/workers.h @@ -36,8 +36,13 @@ class ErrorAwareWorker : public Nan::AsyncWorker { void HandleErrorCallback() { Nan::HandleScope scope; + // Construct error and add code to it. + v8::Local error = Nan::Error(ErrorMessage()); + Nan::Set(error.As(), Nan::New("code").ToLocalChecked(), + Nan::New(GetErrorCode())); + const unsigned int argc = 1; - v8::Local argv[argc] = { Nan::Error(ErrorMessage()) }; + v8::Local argv[argc] = { error }; callback->Call(argc, argv); } diff --git a/test/promisified/producer/flush.spec.js b/test/promisified/producer/flush.spec.js new file mode 100644 index 00000000..debee0fe --- /dev/null +++ b/test/promisified/producer/flush.spec.js @@ -0,0 +1,88 @@ +jest.setTimeout(10000); + +const { + secureRandom, + createProducer, + createTopic, +} = require('../testhelpers'); +const { Kafka } = require('../../../lib').KafkaJS; +const process = require('process'); + +describe('Producer > Flush', () => { + let producer, topicName, message; + + beforeEach(async () => { + producer = createProducer({ + }, { + 'linger.ms': 5000, /* large linger ms to test flush */ + 'queue.buffering.max.kbytes': 2147483647, /* effectively unbounded */ + }); + + topicName = `test-topic-${secureRandom()}`; + message = { key: `key-${secureRandom()}`, value: `value-${secureRandom()}` }; + + await createTopic({ topic: topicName }); + }) + + afterEach(async () => { + producer && (await producer.disconnect()); + }) + + + it('does not wait for linger.ms', + async () => { + await producer.connect(); + let messageSent = false; + const startTime = process.hrtime(); + let diffTime; + + producer.send({ topic: topicName, messages: [message] }).then(() => { + messageSent = true; + diffTime = process.hrtime(startTime); + }); + + await producer.flush({ timeout: 5000 }); + expect(messageSent).toBe(true); + + const diffTimeSeconds = diffTime[0] + diffTime[1] / 1e9; + expect(diffTimeSeconds).toBeLessThan(5); + } + ); + + it('does not matter when awaiting sends', + async () => { + await producer.connect(); + let messageSent = false; + const startTime = process.hrtime(); + let diffTime; + + await producer.send({ topic: topicName, messages: [message] }).then(() => { + messageSent = true; + diffTime = process.hrtime(startTime); + }); + + await producer.flush({ timeout: 1000 }); + expect(messageSent).toBe(true); + + const diffTimeSeconds = diffTime[0] + diffTime[1] / 1e9; + expect(diffTimeSeconds).toBeGreaterThan(5); + } + ); + + it('times out if messages are pending', + async () => { + await producer.connect(); + let messageSent = false; + + /* Larger number of messages */ + producer.send({ topic: topicName, messages: Array(100).fill(message) }).then(() => { + messageSent = true; + }); + + /* Small timeout */ + await expect(producer.flush({ timeout: 1 })).rejects.toThrow(Kafka.KafkaJSTimeout); + expect(messageSent).toBe(false); + } + ); + +}) From 3e36c0b52274740b8c12622c4cd7d34ff470b53e Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 14 Feb 2024 11:24:58 +0530 Subject: [PATCH 082/224] Add cooperative rebalancing --- MIGRATION.md | 2 +- examples/consumer-flow.md | 76 +++++++++++++ examples/consumer.md | 45 ++++++++ examples/docker-alpine.md | 30 +++++ examples/high-level-producer.md | 32 ++++++ examples/kafkajs/consumer.js | 167 +++++++++++++--------------- examples/kafkajs/eos.js | 35 +++--- examples/kafkajs/producer.js | 28 ++--- examples/metadata.md | 27 +++++ examples/producer-cluster.md | 103 +++++++++++++++++ examples/producer.md | 73 ++++++++++++ lib/kafka-consumer.js | 44 +++++++- lib/kafkajs/_common.js | 2 +- lib/kafkajs/_consumer.js | 41 +++++-- src/kafka-consumer.cc | 191 ++++++++++++++++++++++++++++++++ src/kafka-consumer.h | 8 ++ 16 files changed, 768 insertions(+), 136 deletions(-) create mode 100644 examples/consumer-flow.md create mode 100644 examples/consumer.md create mode 100644 examples/docker-alpine.md create mode 100644 examples/high-level-producer.md create mode 100644 examples/metadata.md create mode 100644 examples/producer-cluster.md create mode 100644 examples/producer.md diff --git a/MIGRATION.md b/MIGRATION.md index 3eae2772..804e5c37 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -195,7 +195,7 @@ producerRun().then(consumerRun).catch(console.error); | Property | Default Value | Comment | |--------------------------|-----------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | groupId | null | A mandatory string denoting consumer group name that this consumer is a part of. | - | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range and roundRobin assignors is provided. Custom assignors are not supported. Support for cooperative-sticky assignor is yet to be added | + | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range, roundRobin, and cooperativeSticky assignors is provided. Custom assignors are not supported. | | **partitionAssignors** | `[PartitionAssignors.roundRobin]` | Alias for `partitionAssigners` | | **rebalanceTimeout** | **300000** | The maximum allowed time for each member to join the group once a rebalance has begun. Note, that setting this value *also* changes the max poll interval. Message processing in `eachMessage` must not take more than this time. | | heartbeatInterval | 3000 | The expected time in milliseconds between heartbeats to the consumer coordinator. | diff --git a/examples/consumer-flow.md b/examples/consumer-flow.md new file mode 100644 index 00000000..62e5e31f --- /dev/null +++ b/examples/consumer-flow.md @@ -0,0 +1,76 @@ +Connecting to a Kafka Consumer is easy. Let's try to connect to one using +the Flowing implementation + +```js +/* + * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2016-2023 Blizzard Entertainment + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +var Kafka = require('../'); + +var consumer = new Kafka.KafkaConsumer({ + //'debug': 'all', + 'metadata.broker.list': 'localhost:9092', + 'group.id': 'confluent-kafka-js-consumer-flow-example', + 'enable.auto.commit': false +}); + +var topicName = 'test'; + +//logging debug messages, if debug is enabled +consumer.on('event.log', function(log) { + console.log(log); +}); + +//logging all errors +consumer.on('event.error', function(err) { + console.error('Error from consumer'); + console.error(err); +}); + +//counter to commit offsets every numMessages are received +var counter = 0; +var numMessages = 5; + +consumer.on('ready', function(arg) { + console.log('consumer ready.' + JSON.stringify(arg)); + + consumer.subscribe([topicName]); + //start consuming messages + consumer.consume(); +}); + + +consumer.on('data', function(m) { + counter++; + + //committing offsets every numMessages + if (counter % numMessages === 0) { + console.log('calling commit'); + consumer.commit(m); + } + + // Output the actual message contents + console.log(JSON.stringify(m)); + console.log(m.value.toString()); + +}); + +consumer.on('disconnected', function(arg) { + console.log('consumer disconnected. ' + JSON.stringify(arg)); +}); + +//starting the consumer +consumer.connect(); + +//stopping this example after 30s +setTimeout(function() { + consumer.disconnect(); +}, 30000); + +``` diff --git a/examples/consumer.md b/examples/consumer.md new file mode 100644 index 00000000..4e09cc2f --- /dev/null +++ b/examples/consumer.md @@ -0,0 +1,45 @@ +Connecting to a Kafka Consumer is easy. Let's try to connect to one using +the Stream implementation + +```js +/* + * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2016-2023 Blizzard Entertainment + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +var Transform = require('stream').Transform; + +var Kafka = require('../'); + +var stream = Kafka.KafkaConsumer.createReadStream({ + 'metadata.broker.list': 'localhost:9092', + 'group.id': 'librd-test', + 'socket.keepalive.enable': true, + 'enable.auto.commit': false +}, {}, { + topics: 'test', + waitInterval: 0, + objectMode: false +}); + +stream.on('error', function(err) { + if (err) console.log(err); + process.exit(1); +}); + +stream + .pipe(process.stdout); + +stream.on('error', function(err) { + console.log(err); + process.exit(1); +}); + +stream.consumer.on('event.error', function(err) { + console.log(err); +}) +``` diff --git a/examples/docker-alpine.md b/examples/docker-alpine.md new file mode 100644 index 00000000..9339e191 --- /dev/null +++ b/examples/docker-alpine.md @@ -0,0 +1,30 @@ +When using docker to install `confluent-kafka-js`, you need to make sure you install appropriate library dependencies. Alpine linux is a lighter weight version of linux and does not come with the same base libraries as other distributions (like glibc). + +You can see some of the differences here: https://linuxacademy.com/blog/cloud/alpine-linux-and-docker/ + +```dockerfile +FROM node:14-alpine + +RUN apk --no-cache add \ + bash \ + g++ \ + ca-certificates \ + lz4-dev \ + musl-dev \ + cyrus-sasl-dev \ + openssl-dev \ + make \ + python3 + +RUN apk add --no-cache --virtual .build-deps gcc zlib-dev libc-dev bsd-compat-headers py-setuptools bash + +# Create app directory +RUN mkdir -p /usr/local/app + +# Move to the app directory +WORKDIR /usr/local/app + +# Install confluent-kafka-js +RUN npm install confluent-kafka-js +# Copy package.json first to check if an npm install is needed +``` diff --git a/examples/high-level-producer.md b/examples/high-level-producer.md new file mode 100644 index 00000000..2071b822 --- /dev/null +++ b/examples/high-level-producer.md @@ -0,0 +1,32 @@ +```js +var Kafka = require('../'); + +var producer = new Kafka.HighLevelProducer({ + 'metadata.broker.list': 'localhost:9092', +}); + +// Throw away the keys +producer.setKeySerializer(function(v) { + return new Promise((resolve, reject) => { + setTimeout(() => { + resolve(null); + }, 20); + }); +}); + +// Take the message field +producer.setValueSerializer(function(v) { + return Buffer.from(v.message); +}); + +producer.connect(null, function() { + producer.produce('test', null, { + message: 'alliance4ever', + }, null, Date.now(), function(err, offset) { + // The offset if our acknowledgement level allows us to receive delivery offsets + setImmediate(function() { + producer.disconnect(); + }); + }); +}); +``` diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 5f01a172..22d5ccc7 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,29 +1,23 @@ -// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('../..').KafkaJS +//const { Kafka } = require('kafkajs') async function consumerStart() { - let consumer; - var stopped = false; + let consumer; + var stopped = false; - const kafka = new Kafka({ - kafkaJS: { - brokers: ['localhost:9092'], - ssl: true, - connectionTimeout: 5000, - sasl: { - mechanism: 'plain', - username: '', - password: '', - }, - } - }); + const kafka = new Kafka({ + brokers: [''], + ssl: true, + connectionTimeout: 5000, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, + }); - consumer = kafka.consumer({ - kafkaJS: { + consumer = kafka.consumer({ groupId: 'test-group', - autoCommit: false, rebalanceListener: { onPartitionsAssigned: async (assignment) => { console.log(`Assigned partitions ${JSON.stringify(assignment)}`); @@ -37,90 +31,89 @@ async function consumerStart() { } } }, - }, - - /* Properties from librdkafka can also be used */ - 'auto.commit.interval.ms': 6000, - }); - - await consumer.connect(); - console.log("Connected successfully"); + rdKafka: { + 'enable.auto.commit': false + } + }); - await consumer.subscribe({ - topics: [ - "topic2" - ] - }) + await consumer.connect(); + console.log("Connected successfully"); - // Batch consumer, commit and seek example - var batch = 0; - consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - console.log({ - topic, - partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }) + await consumer.subscribe({ + topics: [ + "topic2" + ] + }) - if (++batch % 100 == 0) { - await consumer.seek({ + // Batch consumer, commit and seek example + var batch = 0; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ topic, partition, - offset: -2 - }); - await consumer.commitOffsets(); - batch = 0; - } - }, - }); + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }) - // Pause/Resume example - const pauseResumeLoop = async () => { - let paused = false; - let ticks = 0; - while (!stopped) { - await new Promise((resolve) => setTimeout(resolve, 100)); - if (stopped) - break; + if (++batch % 100 == 0) { + await consumer.seek({ + topic, + partition, + offset: -2 + }); + await consumer.commitOffsets(); + batch = 0; + } + }, + }); + + // Pause/Resume example + const pauseResumeLoop = async () => { + let paused = false; + let ticks = 0; + while (!stopped) { + await new Promise((resolve) => setTimeout(resolve, 100)); + if (stopped) + break; - ticks++; - if (ticks == 200) { - ticks = 0; - const assignment = consumer.assignment(); - if (paused) { - console.log(`Resuming partitions ${JSON.stringify(assignment)}`) - consumer.resume(assignment); - } else { - console.log(`Pausing partitions ${JSON.stringify(assignment)}`); - consumer.pause(assignment); + ticks++; + if (ticks == 200) { + ticks = 0; + const assignment = consumer.assignment(); + if (paused) { + console.log(`Resuming partitions ${JSON.stringify(assignment)}`) + consumer.resume(assignment); + } else { + console.log(`Pausing partitions ${JSON.stringify(assignment)}`); + consumer.pause(assignment); + } + paused = !paused; } - paused = !paused; } } - } - if (consumer.assignment()) { - // KafkaJS doesn't have assignment() - pauseResumeLoop() - } + if (consumer.assignment) { + // KafkaJS doesn't have assignment() + pauseResumeLoop() + } - // Disconnect example - const disconnect = () => { - process.off('SIGINT', disconnect); - process.off('SIGTERM', disconnect); - stopped = true; - consumer.commitOffsets() + // Disconnect example + const disconnect = () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + stopped = true; + consumer.commitOffsets() .finally(() => consumer.disconnect() ) .finally(() => console.log("Disconnected successfully") ); - } - process.on('SIGINT', disconnect); - process.on('SIGTERM', disconnect); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); } consumerStart() diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index e2c1ff9c..6c9f85ca 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -1,32 +1,26 @@ -// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('../..').KafkaJS +//const { Kafka } = require('kafkajs') async function eosStart() { const kafka = new Kafka({ - kafkaJS: { - brokers: [''], - ssl: true, - sasl: { - mechanism: 'plain', - username: '', - password: '', - } + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', } }); const consumer = kafka.consumer({ - kafkaJS: { - groupId: 'groupId', - autoCommit: false, - } + groupId: 'groupId', + rdKafka: { + "enable.auto.commit": false, + }, }); const producer = kafka.producer({ - kafkaJS: { - transactionalId: 'txid' - } + transactionalId: 'txid' }); await consumer.connect(); @@ -40,8 +34,7 @@ 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/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index 1c0fabbf..57c1e6ac 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -1,18 +1,14 @@ -// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('../..').KafkaJS +//const { Kafka } = require('kafkajs') async function producerStart() { const kafka = new Kafka({ - kafkaJS: { - brokers: [''], - ssl: true, - sasl: { - mechanism: 'plain', - username: '', - password: '', - }, + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', } }); @@ -23,16 +19,16 @@ async function producerStart() { console.log("Connected successfully"); const res = [] - for (let i = 0; i < 50; i++) { + for(let i = 0; i < 50; i++) { res.push(producer.send({ topic: 'topic2', messages: [ - { value: 'v222', partition: 0 }, - { value: 'v11', partition: 0, key: 'x' }, + {value: 'v222', partition: 0}, + {value: 'v11', partition: 0, key: 'x'}, ] })); } - await Promise.all(res); + await Promise.allSettled(res); await producer.disconnect(); diff --git a/examples/metadata.md b/examples/metadata.md new file mode 100644 index 00000000..0fd4c08b --- /dev/null +++ b/examples/metadata.md @@ -0,0 +1,27 @@ +```js +/* + * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2016-2023 Blizzard Entertainment + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +var Kafka = require('../'); + +var producer = new Kafka.Producer({ + 'metadata.broker.list': 'localhost:9092', + 'client.id': 'hey', + 'compression.codec': 'snappy' +}); + +producer.connect() + .on('ready', function(i, metadata) { + console.log(i); + console.log(metadata); + }) + .on('event.error', function(err) { + console.log(err); + }); +``` diff --git a/examples/producer-cluster.md b/examples/producer-cluster.md new file mode 100644 index 00000000..be3dee8e --- /dev/null +++ b/examples/producer-cluster.md @@ -0,0 +1,103 @@ +```js +/* + * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2016-2023 Blizzard Entertainment + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +var cluster = require('cluster'); +var numCPUs = 6; +var Kafka = require('../'); + +if (cluster.isMaster) { + // Fork workers. + for (var i = 0; i < numCPUs; i++) { + cluster.fork(); + } + + var exited_workers = 0; + + cluster.on('exit', function(worker, code, signal) { + exited_workers++; + if (exited_workers === numCPUs - 1) { + process.exit(); + } + }); +} else { + // Configure client + var producer = new Kafka.Producer({ + 'client.id': 'kafka', + 'metadata.broker.list': 'localhost:9092', + 'compression.codec': 'none', + 'retry.backoff.ms': 200, + 'message.send.max.retries': 10, + 'socket.keepalive.enable': true, + 'queue.buffering.max.messages': 100000, + 'queue.buffering.max.ms': 1000, + 'batch.num.messages': 1000000, + 'dr_cb': true + }); + + producer.setPollInterval(100); + + var total = 0; + var totalSent = 0; + var max = 20000; + var errors = 0; + var started = Date.now(); + + var sendMessage = function() { + var ret = producer.sendMessage({ + topic: 'librdtesting-01', + message: Buffer.from('message ' + total) + }, function() { + }); + total++; + if (total >= max) { + } else { + setImmediate(sendMessage); + } + }; + + var verified_received = 0; + var exitNextTick = false; + var errorsArr = []; + + var t = setInterval(function() { + producer.poll(); + + if (exitNextTick) { + clearInterval(t); + return setTimeout(function() { + console.log('[%d] Received: %d, Errors: %d, Total: %d', process.pid, verified_received, errors, total); + // console.log('[%d] Finished sending %d in %d seconds', process.pid, total, parseInt((Date.now() - started) / 1000)); + if (errors > 0) { + console.error(errorsArr[0]); + return process.exitCode = 1; + } + process.exitCode = 0; + setTimeout(process.exit, 1000); + }, 2000); + } + + if (verified_received + errors === max) { + exitNextTick = true; + } + + }, 1000); + producer.connect() + .on('event.error', function(e) { + errors++; + errorsArr.push(e); + }) + .on('delivery-report', function() { + verified_received++; + }) + .on('ready', sendMessage); + + +} +``` diff --git a/examples/producer.md b/examples/producer.md new file mode 100644 index 00000000..0518c675 --- /dev/null +++ b/examples/producer.md @@ -0,0 +1,73 @@ +```js +/* + * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2016-2023 Blizzard Entertainment + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +var Kafka = require('../'); + +var producer = new Kafka.Producer({ + //'debug' : 'all', + 'metadata.broker.list': 'localhost:9092', + 'dr_cb': true //delivery report callback +}); + +var topicName = 'test'; + +//logging debug messages, if debug is enabled +producer.on('event.log', function(log) { + console.log(log); +}); + +//logging all errors +producer.on('event.error', function(err) { + console.error('Error from producer'); + console.error(err); +}); + +//counter to stop this sample after maxMessages are sent +var counter = 0; +var maxMessages = 10; + +producer.on('delivery-report', function(err, report) { + console.log('delivery-report: ' + JSON.stringify(report)); + counter++; +}); + +//Wait for the ready event before producing +producer.on('ready', function(arg) { + console.log('producer ready.' + JSON.stringify(arg)); + + for (var i = 0; i < maxMessages; i++) { + var value = Buffer.from('value-' +i); + var key = "key-"+i; + // if partition is set to -1, librdkafka will use the default partitioner + var partition = -1; + var headers = [ + { header: "header value" } + ] + producer.produce(topicName, partition, value, key, Date.now(), "", headers); + } + + //need to keep polling for a while to ensure the delivery reports are received + var pollLoop = setInterval(function() { + producer.poll(); + if (counter === maxMessages) { + clearInterval(pollLoop); + producer.disconnect(); + } + }, 1000); + +}); + +producer.on('disconnected', function(arg) { + console.log('producer disconnected. ' + JSON.stringify(arg)); +}); + +//starting the producer +producer.connect(); +``` diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 1df769af..09e01b0c 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -252,11 +252,13 @@ KafkaConsumer.prototype.seek = function(toppar, timeout, cb) { }; /** - * Assign the consumer specific partitions and topics + * Assign the consumer specific partitions and topics. Used for + * eager (non-cooperative) rebalancing. * * @param {array} assignments - Assignments array. Should contain * objects with topic and partition set. * @return {Client} - Returns itself + * @sa KafkaConsumer::incrementalAssign */ KafkaConsumer.prototype.assign = function(assignments) { @@ -265,9 +267,11 @@ KafkaConsumer.prototype.assign = function(assignments) { }; /** - * Unassign the consumer from its assigned partitions and topics. + * Unassign the consumer from its assigned partitions and topics.Used for + * eager (non-cooperative) rebalancing. * * @return {Client} - Returns itself + * @sa KafkaConsumer::incrementalUnassign */ KafkaConsumer.prototype.unassign = function() { @@ -275,6 +279,33 @@ KafkaConsumer.prototype.unassign = function() { return this; }; +/** + * Assign the consumer specific partitions and topics. Used for + * cooperative rebalancing. + * + * @param {array} assignments - Assignments array. Should contain + * objects with topic and partition set. Assignments are additive. + * @return {Client} - Returns itself + * @sa KafkaConsumer::assign + */ +KafkaConsumer.prototype.incrementalAssign = function(assignments) { + this._client.incrementalAssign(TopicPartition.map(assignments)); + return this; +}; + +/** + * Unassign the consumer specific partitions and topics. Used for + * cooperative rebalancing. + * + * @param {array} assignments - Assignments array. Should contain + * objects with topic and partition set. Assignments are subtractive. + * @return {Client} - Returns itself + * @sa KafkaConsumer::unassign + */ +KafkaConsumer.prototype.incrementalUnassign = function(assignments) { + this._client.incrementalUnassign(TopicPartition.map(assignments)); + return this; +}; /** * Get the assignments for the consumer @@ -286,6 +317,15 @@ KafkaConsumer.prototype.assignments = function() { return this._errorWrap(this._client.assignments(), true); }; +/** + * Get the type of rebalance protocol used in the consumer group. + * + * @returns "NONE" (if not in a group yet), "COOPERATIVE" or "EAGER". + */ +KafkaConsumer.prototype.rebalanceProtocol = function() { + return this._client.rebalanceProtocol(); +} + /** * Subscribe to an array of topics (synchronously). * diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 94ca0ceb..b0b0b584 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -229,7 +229,7 @@ const CompatibilityErrorMessages = Object.freeze({ /* Consumer */ partitionAssignors: () => - 'partitionAssignors must be a list of strings from within `PartitionAssignors`, and the cooperativeSticky assignor is not yet supported.\n', + 'partitionAssignors must be a list of strings from within `PartitionAssignors`.\n', subscribeOptionsFromBeginning: () => createReplacementErrorMessage('consumer', 'subscribe', 'fromBeginning', 'fromBeginning: ', 'fromBeginning: ', false), subscribeOptionsMandatoryMissing: () => diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 2f8ac861..3807f7b5 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -292,10 +292,19 @@ class Consumer { const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; let call; + + /* Since we don't expose assign() or incremental_assign() methods, we allow the user + * to modify the assignment by returning it. If a truthy value is returned, we use that + * and do not apply any pending seeks to it either. */ + let assignmentModified = false; if (typeof userSpecifiedRebalanceCb === 'function') { call = new Promise((resolve, reject) => { try { - userSpecifiedRebalanceCb(err, assignment); + const alternateAssignment = userSpecifiedRebalanceCb(err, assignment); + if (alternateAssignment) { + assignment = alternateAssignment; + assignmentModified = true; + } resolve(); } catch (e) { reject(e); @@ -308,12 +317,12 @@ class Consumer { case LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS: call = (this.#userConfig.rebalanceListener.onPartitionsAssigned ? this.#userConfig.rebalanceListener.onPartitionsAssigned(assignment) : - Promise.resolve()).catch(this.#logger.error); + Promise.resolve()).catch(e => this.#logger.error(e)); break; case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: call = (this.#userConfig.rebalanceListener.onPartitionsRevoked ? this.#userConfig.rebalanceListener.onPartitionsRevoked(assignment) : - Promise.resolve()).catch(this.#logger.error); + Promise.resolve()).catch(e => this.#logger.error(e)); break; default: call = Promise.reject(`Unexpected rebalanceListener error code ${err.code}`).catch((e) => { @@ -343,10 +352,13 @@ class Consumer { if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { const checkPendingSeeks = this.#pendingSeeks.size !== 0; - if (checkPendingSeeks) + if (checkPendingSeeks && !assignmentModified) assignment = this.#assignAsPerSeekedOffsets(assignment); - this.#internalClient.assign(assignment); + if (this.#internalClient.rebalanceProtocol() === "EAGER") + this.#internalClient.assign(assignment); + else + this.#internalClient.incrementalAssign(assignment); if (checkPendingSeeks) { const offsetsToCommit = assignment @@ -363,7 +375,10 @@ class Consumer { } } else { - this.#internalClient.unassign(); + if (this.#internalClient.rebalanceProtocol() === "EAGER") + this.#internalClient.unassign(); + else + this.#internalClient.incrementalUnassign(assignment); } } catch (e) { // Ignore exceptions if we are not connected @@ -403,8 +418,6 @@ class Consumer { kjsConfig.partitionAssignors.forEach(assignor => { if (typeof assignor !== 'string') throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__INVALID_ARG }); - if (assignor === PartitionAssigners.cooperativeSticky) - throw new error.KafkaJSError(CompatibilityErrorMessages.partitionAssignors(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); }); rdKafkaConfig['partition.assignment.strategy'] = kjsConfig.partitionAssignors.join(','); @@ -1111,6 +1124,18 @@ class Consumer { return this.#flattenTopicPartitions(this.#internalClient.assignments()); } + /** + * Get the type of rebalance protocol used in the consumer group. + * + * @returns "NONE" (if not in a group yet), "COOPERATIVE" or "EAGER". + */ + rebalanceProtocol() { + if (this.#state !== ConsumerState.CONNECTED) { + return "NONE"; + } + return this.#internalClient.rebalanceProtocol(); + } + /** * Fetches all partitions of topic that are assigned to this consumer. * @param {string} topic diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 4d392452..6c0be936 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -202,6 +202,58 @@ Baton KafkaConsumer::Unassign() { return Baton(RdKafka::ERR_NO_ERROR); } +Baton KafkaConsumer::IncrementalAssign(std::vector partitions) { + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE, "KafkaConsumer is disconnected"); + } + + RdKafka::KafkaConsumer* consumer = + dynamic_cast(m_client); + + RdKafka::Error* error = consumer->incremental_assign(partitions); + + if (error == NULL) { + m_partition_cnt += partitions.size(); + // We assume here that there are no duplicate assigns and just transfer. + m_partitions.insert(m_partitions.end(), partitions.begin(), partitions.end()); + } else { + // If we're in error, destroy it, otherwise, don't (since we're using them). + RdKafka::TopicPartition::destroy(partitions); + } + + return rdkafkaErrorToBaton(error); +} + +Baton KafkaConsumer::IncrementalUnassign(std::vector partitions) { + if (!IsClosing() && !IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + RdKafka::KafkaConsumer* consumer = + dynamic_cast(m_client); + + RdKafka::Error* error = consumer->incremental_unassign(partitions); + + if (error == NULL) { + // For now, use two for loops. Make more efficient if needed at a later point. + for (unsigned int i = 0; i < partitions.size(); i++) { + for (unsigned int j = 0; j < m_partitions.size(); j++) { + if (partitions[i]->partition() == m_partitions[j]->partition() && + partitions[i]->topic() == m_partitions[j]->topic()) { + m_partitions.erase(m_partitions.begin() + j); + m_partition_cnt--; + break; + } + } + } + } + + // Destroy the old list of partitions since we are no longer using it + RdKafka::TopicPartition::destroy(m_partitions); + + return rdkafkaErrorToBaton(error); +} + Baton KafkaConsumer::Commit(std::vector toppars) { if (!IsConnected()) { return Baton(RdKafka::ERR__STATE); @@ -470,6 +522,17 @@ Baton KafkaConsumer::RefreshAssignments() { } } +std::string KafkaConsumer::RebalanceProtocol() { + if (!IsConnected()) { + return std::string("NONE"); + } + + RdKafka::KafkaConsumer* consumer = + dynamic_cast(m_client); + + return consumer->rebalance_protocol(); +} + std::string KafkaConsumer::Name() { if (!IsConnected()) { return std::string(""); @@ -530,7 +593,10 @@ void KafkaConsumer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "position", NodePosition); Nan::SetPrototypeMethod(tpl, "assign", NodeAssign); Nan::SetPrototypeMethod(tpl, "unassign", NodeUnassign); + Nan::SetPrototypeMethod(tpl, "incrementalAssign", NodeIncrementalAssign); + Nan::SetPrototypeMethod(tpl, "incrementalUnassign", NodeIncrementalUnassign); Nan::SetPrototypeMethod(tpl, "assignments", NodeAssignments); + Nan::SetPrototypeMethod(tpl, "rebalanceProtocol", NodeRebalanceProtocol); Nan::SetPrototypeMethod(tpl, "commit", NodeCommit); Nan::SetPrototypeMethod(tpl, "commitSync", NodeCommitSync); @@ -702,6 +768,12 @@ NAN_METHOD(KafkaConsumer::NodeAssignments) { Conversion::TopicPartition::ToV8Array(consumer->m_partitions)); } +NAN_METHOD(KafkaConsumer::NodeRebalanceProtocol) { + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); + std::string protocol = consumer->RebalanceProtocol(); + info.GetReturnValue().Set(Nan::New(protocol).ToLocalChecked()); +} + NAN_METHOD(KafkaConsumer::NodeAssign) { Nan::HandleScope scope; @@ -780,6 +852,125 @@ NAN_METHOD(KafkaConsumer::NodeUnassign) { info.GetReturnValue().Set(Nan::True()); } +NAN_METHOD(KafkaConsumer::NodeIncrementalAssign) { + Nan::HandleScope scope; + + if (info.Length() < 1 || !info[0]->IsArray()) { + // Just throw an exception + return Nan::ThrowError("Need to specify an array of partitions"); + } + + v8::Local partitions = info[0].As(); + std::vector topic_partitions; + + for (unsigned int i = 0; i < partitions->Length(); ++i) { + v8::Local partition_obj_value; + if (!( + Nan::Get(partitions, i).ToLocal(&partition_obj_value) && + partition_obj_value->IsObject())) { + Nan::ThrowError("Must pass topic-partition objects"); + } + + v8::Local partition_obj = partition_obj_value.As(); + + // Got the object + int64_t partition = GetParameter(partition_obj, "partition", -1); + std::string topic = GetParameter(partition_obj, "topic", ""); + + if (!topic.empty()) { + RdKafka::TopicPartition* part; + + if (partition < 0) { + part = Connection::GetPartition(topic); + } else { + part = Connection::GetPartition(topic, partition); + } + + // Set the default value to offset invalid. If provided, we will not set + // the offset. + int64_t offset = GetParameter( + partition_obj, "offset", RdKafka::Topic::OFFSET_INVALID); + if (offset != RdKafka::Topic::OFFSET_INVALID) { + part->set_offset(offset); + } + + topic_partitions.push_back(part); + } + } + + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); + + // Hand over the partitions to the consumer. + Baton b = consumer->IncrementalAssign(topic_partitions); + + if (b.err() != RdKafka::ERR_NO_ERROR) { + v8::Local errorObject = b.ToObject(); + Nan::ThrowError(errorObject); + } + + info.GetReturnValue().Set(Nan::True()); +} + +NAN_METHOD(KafkaConsumer::NodeIncrementalUnassign) { + Nan::HandleScope scope; + + if (info.Length() < 1 || !info[0]->IsArray()) { + // Just throw an exception + return Nan::ThrowError("Need to specify an array of partitions"); + } + + v8::Local partitions = info[0].As(); + std::vector topic_partitions; + + for (unsigned int i = 0; i < partitions->Length(); ++i) { + v8::Local partition_obj_value; + if (!( + Nan::Get(partitions, i).ToLocal(&partition_obj_value) && + partition_obj_value->IsObject())) { + Nan::ThrowError("Must pass topic-partition objects"); + } + + v8::Local partition_obj = partition_obj_value.As(); + + // Got the object + int64_t partition = GetParameter(partition_obj, "partition", -1); + std::string topic = GetParameter(partition_obj, "topic", ""); + + if (!topic.empty()) { + RdKafka::TopicPartition* part; + + if (partition < 0) { + part = Connection::GetPartition(topic); + } else { + part = Connection::GetPartition(topic, partition); + } + + // Set the default value to offset invalid. If provided, we will not set + // the offset. + int64_t offset = GetParameter( + partition_obj, "offset", RdKafka::Topic::OFFSET_INVALID); + if (offset != RdKafka::Topic::OFFSET_INVALID) { + part->set_offset(offset); + } + + topic_partitions.push_back(part); + } + } + + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); + + // Hand over the partitions to the consumer. + Baton b = consumer->IncrementalUnassign(topic_partitions); + + if (b.err() != RdKafka::ERR_NO_ERROR) { + v8::Local errorObject = b.ToObject(); + Nan::ThrowError(errorObject); + } + + info.GetReturnValue().Set(Nan::True()); +} + + NAN_METHOD(KafkaConsumer::NodeUnsubscribe) { Nan::HandleScope scope; diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index 316ee901..91be6b90 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -74,6 +74,11 @@ class KafkaConsumer : public Connection { Baton Assign(std::vector); Baton Unassign(); + Baton IncrementalAssign(std::vector); + Baton IncrementalUnassign(std::vector); + + std::string RebalanceProtocol(); + Baton Seek(const RdKafka::TopicPartition &partition, int timeout_ms); std::string Name(); @@ -106,7 +111,10 @@ class KafkaConsumer : public Connection { static NAN_METHOD(NodeDisconnect); static NAN_METHOD(NodeAssign); static NAN_METHOD(NodeUnassign); + static NAN_METHOD(NodeIncrementalAssign); + static NAN_METHOD(NodeIncrementalUnassign); static NAN_METHOD(NodeAssignments); + static NAN_METHOD(NodeRebalanceProtocol); static NAN_METHOD(NodeUnsubscribe); static NAN_METHOD(NodeCommit); static NAN_METHOD(NodeCommitSync); From 755dafbc725d8075ea3bebb4d0eb4cdebe1f6546 Mon Sep 17 00:00:00 2001 From: Milind Luthra Date: Wed, 14 Feb 2024 12:59:31 +0530 Subject: [PATCH 083/224] Add windows package build --- .semaphore/semaphore.yml | 79 +++++++++++++++++++++++++++++++++++++++- README.md | 2 +- deps/windows-install.py | 6 ++- lib/client.js | 2 +- package.json | 3 +- 5 files changed, 87 insertions(+), 5 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 3a35008b..c460b7a7 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -29,7 +29,7 @@ global_job_config: - cd deps/librdkafka - git fetch origin - cd ../../ - - export MKL_DEBUG=1 + - cache clear blocks: - name: "Linux amd64 (musl): Build and test" @@ -303,6 +303,82 @@ blocks: - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Windows x64: Release" + dependencies: [ ] + # run: + # when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-windows + env_vars: + # Disable vcpkg telemetry + - name: VCPKG_DISABLE_METRICS + value: 'yes' + - name: ARCHITECTURE + value: "x64" + - name: PLATFORM + value: "win32" + - name: LIBC + value: "unknown" + prologue: + commands: + # The semaphore agent already comes with an installed version of node. We, however, need to use a different + # version of node for the release (as many as we need to cover all the different ABIs). + # The node installer does not allow us to downgrade, so we need to uninstall the current version. + # The method below isn't particularly robust (as it depends on the particular format of the URL), but it + # works and can be easily fixed if it breaks (the node --version in the below jobs can be checked if there are + # any issues in the build). + - $env:InstalledMajor = (Get-Command node).Version.Major + - $env:InstalledMinor = (Get-Command node).Version.Minor + - $env:InstalledBuild = (Get-Command node).Version.Build + - $env:InstalledVersion = "v${env:InstalledMajor}.${env:InstalledMinor}.${env:InstalledBuild}" + - echo "https://nodejs.org/dist/${env:InstalledVersion}/node-${env:InstalledVersion}-x64.msi" + - Invoke-WebRequest "https://nodejs.org/dist/${env:InstalledVersion}/node-${env:InstalledVersion}-x64.msi" -OutFile node_old.msi + - msiexec /qn /l* node-old-log.txt /uninstall node_old.msi + - cat .\node-old-log.txt + jobs: + - name: "Release: LTS:18" + commands: + - Invoke-WebRequest "https://nodejs.org/download/release/v18.19.0/node-v18.19.0-x64.msi" -OutFile node.msi + - msiexec /qn /l* node-log.txt /i node.msi + - cat .\node-log.txt + - node --version + - pip install setuptools + - $env:NODE_ABI = 108 + - $env:ARTIFACT_KEY = "confluent-kafka-javascript-${env:SEMAPHORE_GIT_TAG_NAME}-node-v${env:NODE_ABI}-${env:PLATFORM}-${env:LIBC}-${env:ARCHITECTURE}.tar.gz" + - echo "$env:ARTIFACT_KEY" + - 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/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - artifact push workflow "build/stage/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" --destination "releases/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - Invoke-WebRequest "https://nodejs.org/dist/v20.11.0/node-v20.11.0-x64.msi" -OutFile node.msi + - msiexec /qn /l* node-log.txt /i node.msi + - node --version + - pip install setuptools + - $env:NODE_ABI = 115 + - $env:ARTIFACT_KEY = "confluent-kafka-javascript-${env:SEMAPHORE_GIT_TAG_NAME}-node-v${env:NODE_ABI}-${env:PLATFORM}-${env:LIBC}-${env:ARCHITECTURE}.tar.gz" + - echo "$env:ARTIFACT_KEY" + - 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/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - artifact push workflow "build/stage/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" --destination "releases/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - Invoke-WebRequest "https://nodejs.org/dist/v21.6.1/node-v21.6.1-x64.msi" -OutFile node.msi + - msiexec /qn /l* node-log.txt /i node.msi + - node --version + - pip install setuptools + - $env:NODE_ABI = 120 + - $env:ARTIFACT_KEY = "confluent-kafka-javascript-${env:SEMAPHORE_GIT_TAG_NAME}-node-v${env:NODE_ABI}-${env:PLATFORM}-${env:LIBC}-${env:ARCHITECTURE}.tar.gz" + - echo "$env:ARTIFACT_KEY" + - 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/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - artifact push workflow "build/stage/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" --destination "releases/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - name: 'Packaging: tar all release artifacts' dependencies: - 'Linux amd64: Release' @@ -310,6 +386,7 @@ blocks: - 'Linux amd64 musl: Release' - 'Linux arm64 musl: Release' - 'macOS arm64/m1: Release' + - 'Windows x64: Release' run: when: "tag =~ '^v[0-9]\\.'" task: diff --git a/README.md b/README.md index 1ba0401e..ef96b0de 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ The following configurations are supported for this early access preview: Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: ```bash -$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-javascript.git#v0.1.7-devel" +$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-javascript.git#v0.1.8-devel" ``` Yarn and pnpm support is experimental. diff --git a/deps/windows-install.py b/deps/windows-install.py index f282df63..635141b8 100644 --- a/deps/windows-install.py +++ b/deps/windows-install.py @@ -4,8 +4,12 @@ import os import glob +# Note: we're using the key 'librdkafka_win' for now since NuGet packages are not +# available for librdkafka on master. This will be changed to 'librdkafka' once +# we're out of early access, and we use the latest version of librdkafka rather +# than master. with open('../package.json') as f: - librdkafkaVersion = json.load(f)['librdkafka'] + librdkafkaVersion = json.load(f)['librdkafka_win'] librdkafkaWinSufix = '7' if librdkafkaVersion == '0.11.5' else ''; depsPrecompiledDir = '../deps/precompiled' diff --git a/lib/client.js b/lib/client.js index 1c29594a..a93dc21a 100644 --- a/lib/client.js +++ b/lib/client.js @@ -14,7 +14,7 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); -const bindingVersion = 'v0.1.7-devel'; +const bindingVersion = 'v0.1.8-devel'; var LibrdKafkaError = require('./error'); diff --git a/package.json b/package.json index 2214513a..d5aa2791 100644 --- a/package.json +++ b/package.json @@ -1,8 +1,9 @@ { "name": "confluent-kafka-javascript", - "version": "v0.1.7-devel", + "version": "v0.1.8-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", + "librdkafka_win": "2.3.0", "main": "lib/index.js", "types": "types/index.d.ts", "scripts": { From d92187521c661c8ee0d13da78f2fc42bcec8145f Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 09:38:43 +0530 Subject: [PATCH 084/224] Add tests for incremental rebalancing --- .../consumer/incrementalRebalance.spec.js | 176 ++++++++++++++++++ 1 file changed, 176 insertions(+) create mode 100644 test/promisified/consumer/incrementalRebalance.spec.js diff --git a/test/promisified/consumer/incrementalRebalance.spec.js b/test/promisified/consumer/incrementalRebalance.spec.js new file mode 100644 index 00000000..f62e7483 --- /dev/null +++ b/test/promisified/consumer/incrementalRebalance.spec.js @@ -0,0 +1,176 @@ +jest.setTimeout(30000); + +const { waitFor, + secureRandom, + createTopic, + createProducer, + createConsumer, } = require("../testhelpers"); +const { PartitionAssigners, ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Consumer > incremental rebalance', () => { + let consumer; + let groupId, topicName; + + const consumerConfig = { + groupId, + partitionAssigners: [PartitionAssigners.cooperativeSticky], + }; + + beforeEach(async () => { + topicName = `test-topic1-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}` + consumer = null; + await createTopic({ topic: topicName, partitions: 2 }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + }); + + it('returns protocol name', async () => { + consumer = createConsumer(consumerConfig); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + expect(consumer.rebalanceProtocol()).toEqual('COOPERATIVE'); + }); + + it('calls rebalance callback', async () => { + let assigns = 0; + let revokes = 0; + const rebalanceCallback = function (err, assignment) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assigns++; + expect(assignment.length).toBe(2); + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + revokes++; + expect(assignment.length).toBe(2); + } else { + // It's either assign or revoke and nothing else. + jest.fail('Unexpected error code'); + } + } + + + consumer = createConsumer(consumerConfig, { + 'rebalance_cb': rebalanceCallback, + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(assigns).toBe(1); + expect(consumer.assignment().length).toBe(2); + + await consumer.disconnect(); + consumer = null; + expect(revokes).toBe(1); + expect(assigns).toBe(1); + }); + + it('allows changing the assignment', async () => { + let assigns = 0; + const rebalanceCallback = function (err, assignment) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assigns++; + expect(assignment.length).toBe(2); + assignment = [assignment[0]]; + return assignment; + } else { + // It's either assign or revoke and nothing else. + expect(err.code).toBe(ErrorCodes.ERR__REVOKE_PARTITIONS); + } + } + + + consumer = createConsumer(consumerConfig, { + 'rebalance_cb': rebalanceCallback, + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(assigns).toBe(1); + expect(consumer.assignment().length).toBe(1); + }); + + it('is actually incremental', async () => { + let expectedAssignmentCount = 0; + const rebalanceCallback = (err, assignment) => { + /* Empty assignments are ignored, they're a rebalance for the synchronization barrier. */ + if (assignment.length === 0) + return; + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + expect(assignment.length).toBe(expectedAssignmentCount); + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + expect(assignment.length).toBe(expectedAssignmentCount); + } else { + // It's either assign or revoke and nothing else. + jest.fail('Unexpected error code'); + } + } + + /* First consumer joins and gets all partitions. */ + expectedAssignmentCount = 2; + consumer = createConsumer(consumerConfig, { + 'rebalance_cb': rebalanceCallback, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(consumer.assignment().length).toBe(2); + + /* Second consumer joins and gets one partition. */ + expectedAssignmentCount = 1; + const consumer2 = createConsumer(consumerConfig, { + 'rebalance_cb': rebalanceCallback, + }); + + await consumer2.connect(); + await consumer2.subscribe({ topic: topicName }); + consumer2.run({ eachMessage: async () => { } }); + await waitFor(() => consumer2.assignment().length > 0, () => null, 1000); + expect(consumer.assignment().length).toBe(1); + expect(consumer2.assignment().length).toBe(1); + + await consumer2.disconnect(); + }); + + it('works with promisified handler', async () => { + let assigns = 0; + let revokes = 0; + + consumer = createConsumer(consumerConfig, { + rebalanceListener: { + onPartitionsAssigned: async (assignment) => { + assigns++; + expect(assignment.length).toBe(2); + }, + onPartitionsRevoked: async (assignment) => { + revokes++; + expect(assignment.length).toBe(2); + } + }, + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(assigns).toBe(1); + expect(consumer.assignment().length).toBe(2); + + await consumer.disconnect(); + consumer = null; + expect(revokes).toBe(1); + expect(assigns).toBe(1); + }); +}); \ No newline at end of file From a9850415ddb9a593e1b31f20ec4c9054d5a3fe08 Mon Sep 17 00:00:00 2001 From: Nusair Haq <136121717+nhaq-confluent@users.noreply.github.com> Date: Thu, 15 Feb 2024 23:12:08 -0500 Subject: [PATCH 085/224] Update README.md (#24) --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index ef96b0de..b1a047cb 100644 --- a/README.md +++ b/README.md @@ -20,8 +20,8 @@ We invite you to raise issues to highlight any feedback you may have. Within the early-access, only **basic produce and consume functionality** as well as the ability to **create and delete topics** are supported. All other admin client functionality is coming in future releases. See [INTRODUCTION.md](INTRODUCTION.md) for more details on what is supported. -To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). - +To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). **DISCLAIMER:** Although it is compatible with **confluent-kafka-javascript**, Confluent does not own or maintain kafkajs/confluent-schema-registry, and the use and functionality of the library should be considered "as is". + ## Requirements @@ -34,7 +34,7 @@ The following configurations are supported for this early access preview: Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: ```bash -$ npm install "git+ssh://git@github.com/confluentinc/confluent-kafka-javascript.git#v0.1.8-devel" +$ npm install @confluentinc/confluent-kafka-javascript ``` Yarn and pnpm support is experimental. From 924d0d3eaa86d847410e237e0c16c746bdf7c3bf Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 10:01:01 +0530 Subject: [PATCH 086/224] Add copyright notices and change imports --- INTRODUCTION.md | 6 +++--- MIGRATION.md | 6 +++--- README.md | 10 +++++----- examples/consumer.js | 2 +- examples/eos.js | 2 +- examples/kafkajs/admin.js | 2 +- examples/kafkajs/sr.js | 2 +- examples/producer.js | 2 +- lib/client.js | 3 ++- lib/kafka-consumer.js | 1 + package-lock.json | 8 ++++---- package.json | 4 ++-- src/admin.cc | 1 + src/callbacks.cc | 1 + src/callbacks.h | 1 + src/connection.cc | 1 + src/connection.h | 1 + src/kafka-consumer.cc | 1 + src/kafka-consumer.h | 1 + src/producer.cc | 1 + src/workers.h | 1 + 21 files changed, 34 insertions(+), 23 deletions(-) diff --git a/INTRODUCTION.md b/INTRODUCTION.md index a95a6152..72e8344a 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -25,7 +25,7 @@ This library includes two utility functions for detecting the status of your ins You can get the features supported by your compile of `librdkafka` by reading the variable "features" on the root of the `confluent-kafka-javascript` object. ```js -const Kafka = require('confluent-kafka-javascript'); +const Kafka = require('@confluentinc/kafka-javascript'); console.log(Kafka.features); // #=> [ 'gzip', 'snappy', 'ssl', 'sasl', 'regex', 'lz4' ] @@ -34,7 +34,7 @@ console.log(Kafka.features); You can also get the version of `librdkafka` ```js -const Kafka = require('confluent-kafka-javascript'); +const Kafka = require('@confluentinc/kafka-javascript'); console.log(Kafka.librdkafkaVersion); // #=> 2.3.0 @@ -512,7 +512,7 @@ producer.getMetadata(opts, (err, metadata) => { To create an Admin client, you can do as follows: ```js -const Kafka = require('confluent-kafka-javascript'); +const Kafka = require('@confluentinc/kafka-javascript'); const client = Kafka.AdminClient.create({ 'client.id': 'kafka-admin', diff --git a/MIGRATION.md b/MIGRATION.md index 804e5c37..ff1fdf04 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -11,7 +11,7 @@ ``` to ```javascript - const { Kafka } = require('confluent-kafka-javascript').KafkaJS; + const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; const kafka = new Kafka({ kafkaJS: { brokers: ['kafka1:9092', 'kafka2:9092'], /* ... */ } }); const producer = kafka.producer({ kafkaJS: { /* ... */, } }); ``` @@ -35,7 +35,7 @@ ```diff -const { Kafka } = require('kafkajs'); -+const { Kafka } = require('confluent-kafka-javascript').KafkaJS; ++const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; const kafka = new Kafka({ + kafkaJS: { @@ -371,6 +371,6 @@ Change the import statement, from ``` to ```javascript - const Kafka = require('confluent-kafka-javascript'); + const Kafka = require('@confluentinc/kafka-javascript'); ``` The rest of the functionality should work as usual. \ No newline at end of file diff --git a/README.md b/README.md index b1a047cb..f1aca82a 100644 --- a/README.md +++ b/README.md @@ -20,8 +20,8 @@ We invite you to raise issues to highlight any feedback you may have. Within the early-access, only **basic produce and consume functionality** as well as the ability to **create and delete topics** are supported. All other admin client functionality is coming in future releases. See [INTRODUCTION.md](INTRODUCTION.md) for more details on what is supported. -To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). **DISCLAIMER:** Although it is compatible with **confluent-kafka-javascript**, Confluent does not own or maintain kafkajs/confluent-schema-registry, and the use and functionality of the library should be considered "as is". - +To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). **DISCLAIMER:** Although it is compatible with **confluent-kafka-javascript**, Confluent does not own or maintain kafkajs/confluent-schema-registry, and the use and functionality of the library should be considered "as is". + ## Requirements @@ -34,7 +34,7 @@ The following configurations are supported for this early access preview: Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: ```bash -$ npm install @confluentinc/confluent-kafka-javascript +$ npm install @confluentinc/kafka-javascript ``` Yarn and pnpm support is experimental. @@ -44,8 +44,8 @@ Yarn and pnpm support is experimental. Below is a simple produce example for users migrating from KafkaJS. ```javascript -// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. -const { Kafka } = require("confluent-kafka-javascript").KafkaJS; +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka } = require("@confluentinc/kafka-javascript").KafkaJS; async function producerStart() { const kafka = new Kafka({ diff --git a/examples/consumer.js b/examples/consumer.js index ee92fd6f..0f2019ee 100644 --- a/examples/consumer.js +++ b/examples/consumer.js @@ -1,4 +1,4 @@ -const { Kafka, ErrorCodes } = require('confluent-kafka-javascript').KafkaJS; +const { Kafka, ErrorCodes } = require('@confluentinc/kafka-javascript').KafkaJS; async function consumerStart() { let consumer; diff --git a/examples/eos.js b/examples/eos.js index 5e7debb0..7a96c3c2 100644 --- a/examples/eos.js +++ b/examples/eos.js @@ -1,4 +1,4 @@ -const { Kafka } = require('confluent-kafka-javascript').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function eosStart() { const consumer = new Kafka().consumer({ diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index 5092c491..0b7325e7 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -1,4 +1,4 @@ -// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code // will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 7ecc32bd..44ccb6a1 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -1,4 +1,4 @@ -// require('kafkajs') is replaced with require('confluent-kafka-javascript').KafkaJS. +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code // will typically use 'confluent-kafka-javascript'. const { Kafka } = require('../..').KafkaJS; diff --git a/examples/producer.js b/examples/producer.js index 608b2519..db3c7ced 100644 --- a/examples/producer.js +++ b/examples/producer.js @@ -1,4 +1,4 @@ -const { Kafka } = require('confluent-kafka-javascript').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function producerStart() { const producer = new Kafka().producer({ diff --git a/lib/client.js b/lib/client.js index a93dc21a..133d72e3 100644 --- a/lib/client.js +++ b/lib/client.js @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. @@ -14,7 +15,7 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); -const bindingVersion = 'v0.1.8-devel'; +const bindingVersion = 'v0.1.9-devel'; var LibrdKafkaError = require('./error'); diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 09e01b0c..c1852ecc 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/package-lock.json b/package-lock.json index 6b0cc4b1..8d6a3089 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { - "name": "confluent-kafka-javascript", - "version": "v0.1.7-devel", + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.9-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { - "name": "confluent-kafka-javascript", - "version": "v0.1.7-devel", + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.9-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index d5aa2791..452e00a3 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { - "name": "confluent-kafka-javascript", - "version": "v0.1.8-devel", + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.9-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.3.0", diff --git a/src/admin.cc b/src/admin.cc index 18f53b27..9c6a5865 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/callbacks.cc b/src/callbacks.cc index bbac1333..e3d18e22 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/callbacks.h b/src/callbacks.h index 24d25a62..f244d8c1 100644 --- a/src/callbacks.h +++ b/src/callbacks.h @@ -1,6 +1,7 @@ /* * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/connection.cc b/src/connection.cc index ae19fe50..6c70a785 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/connection.h b/src/connection.h index e5fc5b37..0ad9cea3 100644 --- a/src/connection.h +++ b/src/connection.h @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 6c0be936..1e5cf077 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index 91be6b90..c84c0e34 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/producer.cc b/src/producer.cc index f8c6df93..2349efe3 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/workers.h b/src/workers.h index a6de4de9..3664d9cf 100644 --- a/src/workers.h +++ b/src/workers.h @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2023 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. From 6e199aca0d3b57c537c31352a2c5a17adef1fffc Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 10:02:52 +0530 Subject: [PATCH 087/224] Don't run windows pipeline needlessly --- .semaphore/semaphore.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index c460b7a7..9c29a28e 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -305,8 +305,8 @@ blocks: - name: "Windows x64: Release" dependencies: [ ] - # run: - # when: "tag =~ '^v[0-9]\\.'" + run: + when: "tag =~ '^v[0-9]\\.'" task: agent: machine: From 10f5a0ccf1127474a5ac8cc3a843fd607dad0dac Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 11:59:38 +0530 Subject: [PATCH 088/224] Revert changes to examples --- examples/consumer-flow.md | 76 -------------- examples/consumer.md | 45 --------- examples/docker-alpine.md | 30 ------ examples/high-level-producer.md | 32 ------ examples/kafkajs/admin.js | 2 +- examples/kafkajs/consumer.js | 167 ++++++++++++++++--------------- examples/kafkajs/eos.js | 35 ++++--- examples/kafkajs/producer.js | 28 +++--- examples/kafkajs/sr.js | 2 +- examples/metadata.md | 27 ----- examples/producer-cluster.md | 103 ------------------- examples/producer.md | 73 -------------- package-lock.json | 171 +++++++++++++++----------------- 13 files changed, 206 insertions(+), 585 deletions(-) delete mode 100644 examples/consumer-flow.md delete mode 100644 examples/consumer.md delete mode 100644 examples/docker-alpine.md delete mode 100644 examples/high-level-producer.md delete mode 100644 examples/metadata.md delete mode 100644 examples/producer-cluster.md delete mode 100644 examples/producer.md diff --git a/examples/consumer-flow.md b/examples/consumer-flow.md deleted file mode 100644 index 62e5e31f..00000000 --- a/examples/consumer-flow.md +++ /dev/null @@ -1,76 +0,0 @@ -Connecting to a Kafka Consumer is easy. Let's try to connect to one using -the Flowing implementation - -```js -/* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library - * - * Copyright (c) 2016-2023 Blizzard Entertainment - * - * This software may be modified and distributed under the terms - * of the MIT license. See the LICENSE.txt file for details. - */ - -var Kafka = require('../'); - -var consumer = new Kafka.KafkaConsumer({ - //'debug': 'all', - 'metadata.broker.list': 'localhost:9092', - 'group.id': 'confluent-kafka-js-consumer-flow-example', - 'enable.auto.commit': false -}); - -var topicName = 'test'; - -//logging debug messages, if debug is enabled -consumer.on('event.log', function(log) { - console.log(log); -}); - -//logging all errors -consumer.on('event.error', function(err) { - console.error('Error from consumer'); - console.error(err); -}); - -//counter to commit offsets every numMessages are received -var counter = 0; -var numMessages = 5; - -consumer.on('ready', function(arg) { - console.log('consumer ready.' + JSON.stringify(arg)); - - consumer.subscribe([topicName]); - //start consuming messages - consumer.consume(); -}); - - -consumer.on('data', function(m) { - counter++; - - //committing offsets every numMessages - if (counter % numMessages === 0) { - console.log('calling commit'); - consumer.commit(m); - } - - // Output the actual message contents - console.log(JSON.stringify(m)); - console.log(m.value.toString()); - -}); - -consumer.on('disconnected', function(arg) { - console.log('consumer disconnected. ' + JSON.stringify(arg)); -}); - -//starting the consumer -consumer.connect(); - -//stopping this example after 30s -setTimeout(function() { - consumer.disconnect(); -}, 30000); - -``` diff --git a/examples/consumer.md b/examples/consumer.md deleted file mode 100644 index 4e09cc2f..00000000 --- a/examples/consumer.md +++ /dev/null @@ -1,45 +0,0 @@ -Connecting to a Kafka Consumer is easy. Let's try to connect to one using -the Stream implementation - -```js -/* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library - * - * Copyright (c) 2016-2023 Blizzard Entertainment - * - * This software may be modified and distributed under the terms - * of the MIT license. See the LICENSE.txt file for details. - */ - -var Transform = require('stream').Transform; - -var Kafka = require('../'); - -var stream = Kafka.KafkaConsumer.createReadStream({ - 'metadata.broker.list': 'localhost:9092', - 'group.id': 'librd-test', - 'socket.keepalive.enable': true, - 'enable.auto.commit': false -}, {}, { - topics: 'test', - waitInterval: 0, - objectMode: false -}); - -stream.on('error', function(err) { - if (err) console.log(err); - process.exit(1); -}); - -stream - .pipe(process.stdout); - -stream.on('error', function(err) { - console.log(err); - process.exit(1); -}); - -stream.consumer.on('event.error', function(err) { - console.log(err); -}) -``` diff --git a/examples/docker-alpine.md b/examples/docker-alpine.md deleted file mode 100644 index 9339e191..00000000 --- a/examples/docker-alpine.md +++ /dev/null @@ -1,30 +0,0 @@ -When using docker to install `confluent-kafka-js`, you need to make sure you install appropriate library dependencies. Alpine linux is a lighter weight version of linux and does not come with the same base libraries as other distributions (like glibc). - -You can see some of the differences here: https://linuxacademy.com/blog/cloud/alpine-linux-and-docker/ - -```dockerfile -FROM node:14-alpine - -RUN apk --no-cache add \ - bash \ - g++ \ - ca-certificates \ - lz4-dev \ - musl-dev \ - cyrus-sasl-dev \ - openssl-dev \ - make \ - python3 - -RUN apk add --no-cache --virtual .build-deps gcc zlib-dev libc-dev bsd-compat-headers py-setuptools bash - -# Create app directory -RUN mkdir -p /usr/local/app - -# Move to the app directory -WORKDIR /usr/local/app - -# Install confluent-kafka-js -RUN npm install confluent-kafka-js -# Copy package.json first to check if an npm install is needed -``` diff --git a/examples/high-level-producer.md b/examples/high-level-producer.md deleted file mode 100644 index 2071b822..00000000 --- a/examples/high-level-producer.md +++ /dev/null @@ -1,32 +0,0 @@ -```js -var Kafka = require('../'); - -var producer = new Kafka.HighLevelProducer({ - 'metadata.broker.list': 'localhost:9092', -}); - -// Throw away the keys -producer.setKeySerializer(function(v) { - return new Promise((resolve, reject) => { - setTimeout(() => { - resolve(null); - }, 20); - }); -}); - -// Take the message field -producer.setValueSerializer(function(v) { - return Buffer.from(v.message); -}); - -producer.connect(null, function() { - producer.produce('test', null, { - message: 'alliance4ever', - }, null, Date.now(), function(err, offset) { - // The offset if our acknowledgement level allows us to receive delivery offsets - setImmediate(function() { - producer.disconnect(); - }); - }); -}); -``` diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index 0b7325e7..420adfa3 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -1,6 +1,6 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-javascript'. +// will typically use '@confluentinc/kafka-javascript'. const { Kafka } = require('../..').KafkaJS; async function adminStart() { diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 22d5ccc7..de4b72cf 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,23 +1,29 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use '@confluentinc/kafka-javascript'. +const { Kafka } = require('../..').KafkaJS; async function consumerStart() { - let consumer; - var stopped = false; + let consumer; + var stopped = false; - const kafka = new Kafka({ - brokers: [''], - ssl: true, - connectionTimeout: 5000, - sasl: { - mechanism: 'plain', - username: '', - password: '', - }, - }); + const kafka = new Kafka({ + kafkaJS: { + brokers: ['localhost:9092'], + ssl: true, + connectionTimeout: 5000, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, + } + }); - consumer = kafka.consumer({ + consumer = kafka.consumer({ + kafkaJS: { groupId: 'test-group', + autoCommit: false, rebalanceListener: { onPartitionsAssigned: async (assignment) => { console.log(`Assigned partitions ${JSON.stringify(assignment)}`); @@ -31,89 +37,90 @@ async function consumerStart() { } } }, - rdKafka: { - 'enable.auto.commit': false - } - }); + }, - await consumer.connect(); - console.log("Connected successfully"); + /* Properties from librdkafka can also be used */ + 'auto.commit.interval.ms': 6000, + }); - await consumer.subscribe({ - topics: [ - "topic2" - ] - }) + await consumer.connect(); + console.log("Connected successfully"); - // Batch consumer, commit and seek example - var batch = 0; - consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - console.log({ + await consumer.subscribe({ + topics: [ + "topic2" + ] + }) + + // Batch consumer, commit and seek example + var batch = 0; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }) + + if (++batch % 100 == 0) { + await consumer.seek({ topic, partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }) - - if (++batch % 100 == 0) { - await consumer.seek({ - topic, - partition, - offset: -2 - }); - await consumer.commitOffsets(); - batch = 0; - } - }, - }); + offset: -2 + }); + await consumer.commitOffsets(); + batch = 0; + } + }, + }); - // Pause/Resume example - const pauseResumeLoop = async () => { - let paused = false; - let ticks = 0; - while (!stopped) { - await new Promise((resolve) => setTimeout(resolve, 100)); - if (stopped) - break; + // Pause/Resume example + const pauseResumeLoop = async () => { + let paused = false; + let ticks = 0; + while (!stopped) { + await new Promise((resolve) => setTimeout(resolve, 100)); + if (stopped) + break; - ticks++; - if (ticks == 200) { - ticks = 0; - const assignment = consumer.assignment(); - if (paused) { - console.log(`Resuming partitions ${JSON.stringify(assignment)}`) - consumer.resume(assignment); - } else { - console.log(`Pausing partitions ${JSON.stringify(assignment)}`); - consumer.pause(assignment); - } - paused = !paused; + ticks++; + if (ticks == 200) { + ticks = 0; + const assignment = consumer.assignment(); + if (paused) { + console.log(`Resuming partitions ${JSON.stringify(assignment)}`) + consumer.resume(assignment); + } else { + console.log(`Pausing partitions ${JSON.stringify(assignment)}`); + consumer.pause(assignment); } + paused = !paused; } } + } - if (consumer.assignment) { - // KafkaJS doesn't have assignment() - pauseResumeLoop() - } + if (consumer.assignment()) { + // KafkaJS doesn't have assignment() + pauseResumeLoop() + } - // Disconnect example - const disconnect = () => { - process.off('SIGINT', disconnect); - process.off('SIGTERM', disconnect); - stopped = true; - consumer.commitOffsets() + // Disconnect example + const disconnect = () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + stopped = true; + consumer.commitOffsets() .finally(() => consumer.disconnect() ) .finally(() => console.log("Disconnected successfully") ); - } - process.on('SIGINT', disconnect); - process.on('SIGTERM', disconnect); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); } consumerStart() diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index 6c9f85ca..aebd5c5d 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -1,26 +1,32 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use '@confluentinc/kafka-javascript'. +const { Kafka } = require('../..').KafkaJS; async function eosStart() { const kafka = new Kafka({ - brokers: [''], - ssl: true, - sasl: { - mechanism: 'plain', - username: '', - password: '', + kafkaJS: { + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', + } } }); const consumer = kafka.consumer({ - groupId: 'groupId', - rdKafka: { - "enable.auto.commit": false, - }, + kafkaJS: { + groupId: 'groupId', + autoCommit: false, + } }); const producer = kafka.producer({ - transactionalId: 'txid' + kafkaJS: { + transactionalId: 'txid' + } }); await consumer.connect(); @@ -34,7 +40,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/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index 57c1e6ac..73111590 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -1,14 +1,18 @@ -const { Kafka } = require('../..').KafkaJS -//const { Kafka } = require('kafkajs') +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use '@confluentinc/kafka-javascript'. +const { Kafka } = require('../..').KafkaJS; async function producerStart() { const kafka = new Kafka({ - brokers: [''], - ssl: true, - sasl: { - mechanism: 'plain', - username: '', - password: '', + kafkaJS: { + brokers: [''], + ssl: true, + sasl: { + mechanism: 'plain', + username: '', + password: '', + }, } }); @@ -19,16 +23,16 @@ async function producerStart() { console.log("Connected successfully"); const res = [] - for(let i = 0; i < 50; i++) { + for (let i = 0; i < 50; i++) { res.push(producer.send({ topic: 'topic2', messages: [ - {value: 'v222', partition: 0}, - {value: 'v11', partition: 0, key: 'x'}, + { value: 'v222', partition: 0 }, + { value: 'v11', partition: 0, key: 'x' }, ] })); } - await Promise.allSettled(res); + await Promise.all(res); await producer.disconnect(); diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 44ccb6a1..0b114549 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -1,6 +1,6 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. // Since this example is within the package itself, we use '../..', but code -// will typically use 'confluent-kafka-javascript'. +// will typically use '@confluentinc/kafka-javascript'. const { Kafka } = require('../..').KafkaJS; // Note: The kafkajs/confluent-schema-registry will need to be installed separately to run this example, diff --git a/examples/metadata.md b/examples/metadata.md deleted file mode 100644 index 0fd4c08b..00000000 --- a/examples/metadata.md +++ /dev/null @@ -1,27 +0,0 @@ -```js -/* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library - * - * Copyright (c) 2016-2023 Blizzard Entertainment - * - * This software may be modified and distributed under the terms - * of the MIT license. See the LICENSE.txt file for details. - */ - -var Kafka = require('../'); - -var producer = new Kafka.Producer({ - 'metadata.broker.list': 'localhost:9092', - 'client.id': 'hey', - 'compression.codec': 'snappy' -}); - -producer.connect() - .on('ready', function(i, metadata) { - console.log(i); - console.log(metadata); - }) - .on('event.error', function(err) { - console.log(err); - }); -``` diff --git a/examples/producer-cluster.md b/examples/producer-cluster.md deleted file mode 100644 index be3dee8e..00000000 --- a/examples/producer-cluster.md +++ /dev/null @@ -1,103 +0,0 @@ -```js -/* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library - * - * Copyright (c) 2016-2023 Blizzard Entertainment - * - * This software may be modified and distributed under the terms - * of the MIT license. See the LICENSE.txt file for details. - */ - -var cluster = require('cluster'); -var numCPUs = 6; -var Kafka = require('../'); - -if (cluster.isMaster) { - // Fork workers. - for (var i = 0; i < numCPUs; i++) { - cluster.fork(); - } - - var exited_workers = 0; - - cluster.on('exit', function(worker, code, signal) { - exited_workers++; - if (exited_workers === numCPUs - 1) { - process.exit(); - } - }); -} else { - // Configure client - var producer = new Kafka.Producer({ - 'client.id': 'kafka', - 'metadata.broker.list': 'localhost:9092', - 'compression.codec': 'none', - 'retry.backoff.ms': 200, - 'message.send.max.retries': 10, - 'socket.keepalive.enable': true, - 'queue.buffering.max.messages': 100000, - 'queue.buffering.max.ms': 1000, - 'batch.num.messages': 1000000, - 'dr_cb': true - }); - - producer.setPollInterval(100); - - var total = 0; - var totalSent = 0; - var max = 20000; - var errors = 0; - var started = Date.now(); - - var sendMessage = function() { - var ret = producer.sendMessage({ - topic: 'librdtesting-01', - message: Buffer.from('message ' + total) - }, function() { - }); - total++; - if (total >= max) { - } else { - setImmediate(sendMessage); - } - }; - - var verified_received = 0; - var exitNextTick = false; - var errorsArr = []; - - var t = setInterval(function() { - producer.poll(); - - if (exitNextTick) { - clearInterval(t); - return setTimeout(function() { - console.log('[%d] Received: %d, Errors: %d, Total: %d', process.pid, verified_received, errors, total); - // console.log('[%d] Finished sending %d in %d seconds', process.pid, total, parseInt((Date.now() - started) / 1000)); - if (errors > 0) { - console.error(errorsArr[0]); - return process.exitCode = 1; - } - process.exitCode = 0; - setTimeout(process.exit, 1000); - }, 2000); - } - - if (verified_received + errors === max) { - exitNextTick = true; - } - - }, 1000); - producer.connect() - .on('event.error', function(e) { - errors++; - errorsArr.push(e); - }) - .on('delivery-report', function() { - verified_received++; - }) - .on('ready', sendMessage); - - -} -``` diff --git a/examples/producer.md b/examples/producer.md deleted file mode 100644 index 0518c675..00000000 --- a/examples/producer.md +++ /dev/null @@ -1,73 +0,0 @@ -```js -/* - * confluent-kafka-js - Node.js wrapper for RdKafka C/C++ library - * - * Copyright (c) 2016-2023 Blizzard Entertainment - * - * This software may be modified and distributed under the terms - * of the MIT license. See the LICENSE.txt file for details. - */ - -var Kafka = require('../'); - -var producer = new Kafka.Producer({ - //'debug' : 'all', - 'metadata.broker.list': 'localhost:9092', - 'dr_cb': true //delivery report callback -}); - -var topicName = 'test'; - -//logging debug messages, if debug is enabled -producer.on('event.log', function(log) { - console.log(log); -}); - -//logging all errors -producer.on('event.error', function(err) { - console.error('Error from producer'); - console.error(err); -}); - -//counter to stop this sample after maxMessages are sent -var counter = 0; -var maxMessages = 10; - -producer.on('delivery-report', function(err, report) { - console.log('delivery-report: ' + JSON.stringify(report)); - counter++; -}); - -//Wait for the ready event before producing -producer.on('ready', function(arg) { - console.log('producer ready.' + JSON.stringify(arg)); - - for (var i = 0; i < maxMessages; i++) { - var value = Buffer.from('value-' +i); - var key = "key-"+i; - // if partition is set to -1, librdkafka will use the default partitioner - var partition = -1; - var headers = [ - { header: "header value" } - ] - producer.produce(topicName, partition, value, key, Date.now(), "", headers); - } - - //need to keep polling for a while to ensure the delivery reports are received - var pollLoop = setInterval(function() { - producer.poll(); - if (counter === maxMessages) { - clearInterval(pollLoop); - producer.disconnect(); - } - }, 1000); - -}); - -producer.on('disconnected', function(arg) { - console.log('producer disconnected. ' + JSON.stringify(arg)); -}); - -//starting the producer -producer.connect(); -``` diff --git a/package-lock.json b/package-lock.json index 8d6a3089..79b54884 100644 --- a/package-lock.json +++ b/package-lock.json @@ -894,6 +894,12 @@ "node": ">=8" } }, + "node_modules/@istanbuljs/load-nyc-config/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/@istanbuljs/schema": { "version": "0.1.3", "resolved": "https://registry.npmjs.org/@istanbuljs/schema/-/schema-0.1.3.tgz", @@ -1196,9 +1202,9 @@ } }, "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==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", + "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", "dev": true, "engines": { "node": ">=6.0.0" @@ -1452,9 +1458,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.11.15", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.15.tgz", - "integrity": "sha512-gscmuADZfvNULx1eyirVbr3kVOVZtpQtzKMCZpeSZcN6MfbkRXAR4s9/gsQ4CzxLHw6EStDtKLNtSDL3vbq05A==", + "version": "20.11.19", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.19.tgz", + "integrity": "sha512-7xMnVEcZFu0DikYjWOlRq7NTPETrm7teqUT2WkQjrTIkEgUyyGdWsj/Zg8bEJt5TNklzbPD1X3fqfsHw3SpapQ==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -1853,9 +1859,9 @@ "dev": true }, "node_modules/browserslist": { - "version": "4.22.3", - "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.22.3.tgz", - "integrity": "sha512-UAp55yfwNv0klWNapjs/ktHoguxuQNGnOzxYmfnXIS+8AsRDZkSDxg7R1AX3GKzn078SBI5dzwzj/Yx0Or0e3A==", + "version": "4.23.0", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.0.tgz", + "integrity": "sha512-QW8HiM1shhT2GuzkvklfjcKDiWFXHOeFCIA/huJPwHsslwcydgk7X+z2zXpEijP98UCY7HbubZt5J2Zgvf0CaQ==", "dev": true, "funding": [ { @@ -1872,8 +1878,8 @@ } ], "dependencies": { - "caniuse-lite": "^1.0.30001580", - "electron-to-chromium": "^1.4.648", + "caniuse-lite": "^1.0.30001587", + "electron-to-chromium": "^1.4.668", "node-releases": "^2.0.14", "update-browserslist-db": "^1.0.13" }, @@ -1996,9 +2002,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001582", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001582.tgz", - "integrity": "sha512-vsJG3V5vgfduaQGVxL53uSX/HUzxyr2eA8xCo36OLal7sRcSZbibJtLeh0qja4sFOr/QQGt4opB4tOy+eOgAxg==", + "version": "1.0.30001587", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001587.tgz", + "integrity": "sha512-HMFNotUmLXn71BQxg8cijvqxnIAofforZOwGsxyXJ0qugTdspUF4sPSJ2vhgprHCB996tIDzEq1ubumPDV8ULA==", "dev": true, "funding": [ { @@ -2436,9 +2442,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.653", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.653.tgz", - "integrity": "sha512-wA2A2LQCqnEwQAvwADQq3KpMpNwgAUBnRmrFgRzHnPhbQUFArTR32Ab46f4p0MovDLcg4uqd4nCsN2hTltslpA==", + "version": "1.4.672", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.672.tgz", + "integrity": "sha512-YYCy+goe3UqZqa3MOQCI5Mx/6HdBLzXL/mkbGCEWL3sP3Z1BP9zqAzeD3YEmLZlespYGFtyM8tRp5i2vfaUGCA==", "dev": true }, "node_modules/emittery": { @@ -2498,9 +2504,9 @@ } }, "node_modules/escalade": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", - "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.2.tgz", + "integrity": "sha512-ErCHMCae19vR8vQGe50xIsVomy19rg6gFu3+r3jkEO46suLMWBksvVyoGgQV+jOfl84ZSOSlmv6Gxa89PmTGmA==", "dev": true, "engines": { "node": ">=6" @@ -2746,9 +2752,9 @@ "dev": true }, "node_modules/fastq": { - "version": "1.17.0", - "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.17.0.tgz", - "integrity": "sha512-zGygtijUMT7jnk3h26kUms3BkSDp4IfIKjmnqI2tvx6nuBfiF1UqOxbnLfzdv+apBy+53oaImsKtMw/xYbW+1w==", + "version": "1.17.1", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.17.1.tgz", + "integrity": "sha512-sRVD3lWVIXWg6By68ZN7vho9a1pQcN/WBFaAAsDDFzlJjvoGx0P8z7V1t72grFJfJhu3YPZBuu25f7Kaw2jN1w==", "dev": true, "dependencies": { "reusify": "^1.0.4" @@ -3007,9 +3013,9 @@ "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==", + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.1.tgz", + "integrity": "sha512-1/th4MHjnwncwXsIW6QMzlvYL9kG5e/CpVvLRZe4XPa8TOUNbCELqmvhDmnkNsAjwaG4+I8gJJL0JBvTTLO9qA==", "dev": true, "dependencies": { "function-bind": "^1.1.2" @@ -3190,11 +3196,18 @@ "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/ip-address": { + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/ip-address/-/ip-address-9.0.5.tgz", + "integrity": "sha512-zHtQzGojZXTwZTHQqra+ETKd4Sn3vgi7uBmlPoXVWZqYvuKmtI0l/VZTjqGmJY9x88GGOaZ9+G9ES8hC4T4X8g==", + "dev": true, + "dependencies": { + "jsbn": "1.1.0", + "sprintf-js": "^1.1.3" + }, + "engines": { + "node": ">= 12" + } }, "node_modules/is-arrayish": { "version": "0.2.1", @@ -3996,6 +4009,12 @@ "xmlcreate": "^2.0.4" } }, + "node_modules/jsbn": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-1.1.0.tgz", + "integrity": "sha512-4bYVV3aAMtDTTu4+xsDYa6sy9GyJ69/amsu9sYF2zqjiEoZA5xJi3BrfX3uY+/IekIu7MwdObdbDWpoZdBv3/A==", + "dev": true + }, "node_modules/jsdoc": { "version": "4.0.2", "resolved": "https://registry.npmjs.org/jsdoc/-/jsdoc-4.0.2.tgz", @@ -4509,9 +4528,9 @@ } }, "node_modules/mocha": { - "version": "10.2.0", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.2.0.tgz", - "integrity": "sha512-IDY7fl/BecMwFHzoqF2sg/SHHANeBoMMXFlS9r0OXKDssYE1M5O43wUY/9BVPeIvfH2zmEbBfseqN9gBQZzXkg==", + "version": "10.3.0", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.3.0.tgz", + "integrity": "sha512-uF2XJs+7xSLsrmIvn37i/wnc91nw7XjOQB8ccyx5aEgdnohr7n+rEiZP23WkCYHjilR6+EboEnbq/ZQDz4LSbg==", "dev": true, "dependencies": { "ansi-colors": "4.1.1", @@ -4521,13 +4540,12 @@ "diff": "5.0.0", "escape-string-regexp": "4.0.0", "find-up": "5.0.0", - "glob": "7.2.0", + "glob": "8.1.0", "he": "1.2.0", "js-yaml": "4.1.0", "log-symbols": "4.1.0", "minimatch": "5.0.1", "ms": "2.1.3", - "nanoid": "3.3.3", "serialize-javascript": "6.0.0", "strip-json-comments": "3.1.1", "supports-color": "8.1.1", @@ -4542,10 +4560,15 @@ }, "engines": { "node": ">= 14.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/mochajs" + } + }, + "node_modules/mocha/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/mocha/node_modules/cliui": { @@ -4560,37 +4583,24 @@ } }, "node_modules/mocha/node_modules/glob": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", - "integrity": "sha512-lmLf6gtyrPq8tTjSmrO94wBeQbFR3HbLHbuyD69wuyQkImp2hWqMGB47OX65FBkPffO641IP9jWa1z4ivqG26Q==", + "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": "^3.0.4", - "once": "^1.3.0", - "path-is-absolute": "^1.0.0" + "minimatch": "^5.0.1", + "once": "^1.3.0" }, "engines": { - "node": "*" + "node": ">=12" }, "funding": { "url": "https://github.com/sponsors/isaacs" } }, - "node_modules/mocha/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==", - "dev": true, - "dependencies": { - "brace-expansion": "^1.1.7" - }, - "engines": { - "node": "*" - } - }, "node_modules/mocha/node_modules/minimatch": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.0.1.tgz", @@ -4603,15 +4613,6 @@ "node": ">=10" } }, - "node_modules/mocha/node_modules/minimatch/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/mocha/node_modules/ms": { "version": "2.1.3", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", @@ -4661,18 +4662,6 @@ "resolved": "https://registry.npmjs.org/nan/-/nan-2.18.0.tgz", "integrity": "sha512-W7tfG7vMOGtD30sHoZSSc/JVYiyDPEyQVso/Zz+/uQd0B0L46gtC+pHha5FFMRpil6fm/AoEcRWyOVi4+E/f8w==" }, - "node_modules/nanoid": { - "version": "3.3.3", - "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.3.tgz", - "integrity": "sha512-p1sjXuopFs0xg+fPASzQ28agW1oHD7xDsd9Xkf3T15H3c/cifrFHVwrh74PdoklAPi+i7MdRsE47vm2r6JoB+w==", - "dev": true, - "bin": { - "nanoid": "bin/nanoid.cjs" - }, - "engines": { - "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", @@ -5437,9 +5426,9 @@ "optional": true }, "node_modules/semver": { - "version": "7.5.4", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.4.tgz", - "integrity": "sha512-1bCSESV6Pv+i21Hvpxp3Dx+pSD8lIPt8uVjRrxAUt/nbswYc+tK6Y2btiULjd4+fnq15PX+nqQDC7Oft7WkwcA==", + "version": "7.6.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.0.tgz", + "integrity": "sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==", "dependencies": { "lru-cache": "^6.0.0" }, @@ -5532,16 +5521,16 @@ } }, "node_modules/socks": { - "version": "2.7.1", - "resolved": "https://registry.npmjs.org/socks/-/socks-2.7.1.tgz", - "integrity": "sha512-7maUZy1N7uo6+WVEX6psASxtNlKaNVMlGQKkG/63nEDdLOWNbiUMoLK7X4uYoLhQstau72mLgfEWcXcwsaHbYQ==", + "version": "2.7.3", + "resolved": "https://registry.npmjs.org/socks/-/socks-2.7.3.tgz", + "integrity": "sha512-vfuYK48HXCTFD03G/1/zkIls3Ebr2YNa4qU9gHDZdblHLiqhJrJGkY3+0Nx0JpN9qBhJbVObc1CNciT1bIZJxw==", "dev": true, "dependencies": { - "ip": "^2.0.0", + "ip-address": "^9.0.5", "smart-buffer": "^4.2.0" }, "engines": { - "node": ">= 10.13.0", + "node": ">= 10.0.0", "npm": ">= 3.0.0" } }, @@ -5579,9 +5568,9 @@ } }, "node_modules/sprintf-js": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", - "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==", + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.1.3.tgz", + "integrity": "sha512-Oo+0REFV59/rz3gfJNKQiBlwfHaSESl1pcGyABQsnnIfWOFt6JNj5gCog2U6MLZ//IGYD+nA8nI+mTShREReaA==", "dev": true }, "node_modules/ssri": { From c1f290603772f4c33c4f13574bcd52d74f9e89e0 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 12:18:59 +0530 Subject: [PATCH 089/224] Fix certain consumer examples --- examples/consumer.js | 25 +++++++++++-------------- examples/kafkajs/consumer.js | 2 +- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/examples/consumer.js b/examples/consumer.js index 0f2019ee..188442ea 100644 --- a/examples/consumer.js +++ b/examples/consumer.js @@ -47,7 +47,7 @@ async function consumerStart() { // Initialization consumer = new Kafka().consumer({ 'bootstrap.servers': 'localhost:9092', - 'group.id': 'test-group3', + 'group.id': 'test-group', 'auto.offset.reset': 'earliest', 'enable.partition.eof': 'true', 'rebalance_cb': (err, assignment) => { @@ -70,20 +70,17 @@ async function consumerStart() { console.log("Connected successfully"); await consumer.subscribe({ topics: ["test-topic"] }); - // Start the consumer. - while (!stopped) { - const message = await consumer.consume(1000); - if (!message) { - continue; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); } - console.log({ - topic: message.topic, - partition: message.partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }); - } + }); } consumerStart(); diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index de4b72cf..07d249bc 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -48,7 +48,7 @@ async function consumerStart() { await consumer.subscribe({ topics: [ - "topic2" + "test-topic" ] }) From 4db6a6f05ceb64337363731b81136a6eb199c0cf Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 18:02:20 +0530 Subject: [PATCH 090/224] Add exemption for version checks from devel releases --- ci/checks/librdkafka-correct-version.js | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ci/checks/librdkafka-correct-version.js b/ci/checks/librdkafka-correct-version.js index 5b2506e5..011a5481 100644 --- a/ci/checks/librdkafka-correct-version.js +++ b/ci/checks/librdkafka-correct-version.js @@ -61,7 +61,8 @@ function versionAsString(version) { const librdkafkaVersion = parseLibrdkafkaVersion(defines.RD_KAFKA_VERSION); const versionString = versionAsString(librdkafkaVersion); -if (pjs.librdkafka !== versionString) { +// If our version is a devel (early access) version, we might be on master. +if (pjs.librdkafka !== versionString && !pjs.version.includes('devel')) { console.error(`Librdkafka version of ${versionString} does not match package json: ${pjs.librdkafka}`); process.exit(1); } From 81b75dcc21a1232b51f794c76a594e4f971dabc1 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 16 Feb 2024 18:19:10 +0530 Subject: [PATCH 091/224] Version bump to v0.1.10-devel --- .npmignore | 1 + ci/prepublish.js | 3 ++- lib/client.js | 2 +- lib/error.js | 2 +- package.json | 4 ++-- types/config.d.ts | 4 ++-- types/errors.d.ts | 2 +- 7 files changed, 10 insertions(+), 8 deletions(-) diff --git a/.npmignore b/.npmignore index c6793a26..5130a557 100644 --- a/.npmignore +++ b/.npmignore @@ -8,3 +8,4 @@ deps/librdkafka/config.h build .github .vscode +.semaphore \ No newline at end of file diff --git a/ci/prepublish.js b/ci/prepublish.js index 3526be85..c5ed2107 100644 --- a/ci/prepublish.js +++ b/ci/prepublish.js @@ -1,4 +1,5 @@ require('./checks/librdkafka-exists'); require('./checks/librdkafka-correct-version'); require('./librdkafka-defs-generator.js'); -require('./update-version'); +// Temporarily disabled until we move to non-devel versions. +// require('./update-version'); diff --git a/lib/client.js b/lib/client.js index 133d72e3..f26d15de 100644 --- a/lib/client.js +++ b/lib/client.js @@ -15,7 +15,7 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); -const bindingVersion = 'v0.1.9-devel'; +const bindingVersion = 'v0.1.10-devel'; var LibrdKafkaError = require('./error'); diff --git a/lib/error.js b/lib/error.js index 952c80ee..ff7c47a3 100644 --- a/lib/error.js +++ b/lib/error.js @@ -27,7 +27,7 @@ LibrdKafkaError.wrap = errorWrap; * @enum {number} * @constant */ -// ====== Generated from librdkafka 2.3.0 file src-cpp/rdkafkacpp.h ====== +// ====== Generated from librdkafka master file src-cpp/rdkafkacpp.h ====== LibrdKafkaError.codes = { /* Internal errors to rdkafka: */ diff --git a/package.json b/package.json index 452e00a3..88780e23 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.9-devel", + "version": "v0.1.10-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.3.0", @@ -49,4 +49,4 @@ "engines": { "node": ">=18.0.0" } -} +} \ No newline at end of file diff --git a/types/config.d.ts b/types/config.d.ts index e78a4141..930b611f 100644 --- a/types/config.d.ts +++ b/types/config.d.ts @@ -1,4 +1,4 @@ -// ====== Generated from librdkafka 2.3.0 file CONFIGURATION.md ====== +// ====== Generated from librdkafka master file CONFIGURATION.md ====== // Code that generated this is a derivative work of the code from Nam Nguyen // https://gist.github.com/ntgn81/066c2c8ec5b4238f85d1e9168a04e3fb @@ -810,7 +810,7 @@ export interface ConsumerGlobalConfig extends GlobalConfig { "heartbeat.interval.ms"?: number; /** - * Group protocol type. NOTE: Currently, the only supported group protocol type is `consumer`. + * Group protocol type for the `generic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. * * @default consumer */ diff --git a/types/errors.d.ts b/types/errors.d.ts index 439ec72b..333b17dc 100644 --- a/types/errors.d.ts +++ b/types/errors.d.ts @@ -1,4 +1,4 @@ -// ====== Generated from librdkafka 2.3.0 file src-cpp/rdkafkacpp.h ====== +// ====== Generated from librdkafka master file src-cpp/rdkafkacpp.h ====== export const CODES: { ERRORS: { /* Internal errors to rdkafka: */ /** Begin internal error codes (**-200**) */ From 96d7e3dda20c6944f67be14398b554f62262fdce Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 19 Feb 2024 08:58:04 +0530 Subject: [PATCH 092/224] Update dependency information --- README.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index f1aca82a..7c5c6091 100644 --- a/README.md +++ b/README.md @@ -30,8 +30,11 @@ The following configurations are supported for this early access preview: * Any supported version of Node.js (The two LTS versions, 18 and 20, and the latest version, 21). * Linux (x64 and arm64) - both glibc and musl/alpine. * macOS - arm64/m1. +* Windows - x64 (experimentally available in EA). -Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. It can be installed from GitHub: +[A supported version of Python](https://devguide.python.org/versions/) must be available on the system for the installation process. [This is required for the `node-gyp` build tool.](https://github.com/nodejs/node-gyp?tab=readme-ov-file#configuring-python-dependency). + +Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. ```bash $ npm install @confluentinc/kafka-javascript From 00f927a20623141ae2eb1decc2f023292c7a3cbc Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 19 Feb 2024 09:39:44 +0530 Subject: [PATCH 093/224] Update versioning and librdkafka repo URL --- .gitmodules | 2 +- INTRODUCTION.md | 6 +++--- ci/prepublish.js | 3 +-- ci/update-version.js | 22 ++++++---------------- lib/error.js | 2 +- lib/kafka-consumer.js | 2 +- lib/topic.js | 2 +- src/connection.cc | 2 +- 8 files changed, 15 insertions(+), 26 deletions(-) diff --git a/.gitmodules b/.gitmodules index 4d4e7fb9..63dc1ac1 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,3 @@ [submodule "deps/librdkafka"] path = deps/librdkafka - url = https://github.com/edenhill/librdkafka.git + url = https://github.com/confluentinc/librdkafka.git diff --git a/INTRODUCTION.md b/INTRODUCTION.md index 72e8344a..ecbb3c2a 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -2,7 +2,7 @@ ## Configuration -You can pass many configuration options to `librdkafka`. A full list can be found in `librdkafka`'s [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) +You can pass many configuration options to `librdkafka`. A full list can be found in `librdkafka`'s [Configuration.md](https://github.com/confluentinc/librdkafka/blob/v2.3.0/CONFIGURATION.md) Configuration keys that have the suffix `_cb` are designated as callbacks. Some of these keys are informational and you can choose to opt-in (for example, `dr_cb`). Others are callbacks designed to @@ -56,7 +56,7 @@ const producer = new Kafka.Producer({ }); ``` -A `Producer` requires only `metadata.broker.list` (the Kafka brokers) to be created. The values in this list are separated by commas. For other configuration options, see the [Configuration.md](https://github.com/edenhill/librdkafka/blob/v2.3.0/CONFIGURATION.md) file described previously. +A `Producer` requires only `metadata.broker.list` (the Kafka brokers) to be created. The values in this list are separated by commas. For other configuration options, see the [Configuration.md](https://github.com/confluentinc/librdkafka/blob/v2.3.0/CONFIGURATION.md) file described previously. The following example illustrates a list with several `librdkafka` options set. @@ -259,7 +259,7 @@ const consumer = new Kafka.KafkaConsumer({ }, {}); ``` -The first parameter is the global config, while the second parameter is the topic config that gets applied to all subscribed topics. To view a list of all supported configuration properties, see the [Configuration.md](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) file described previously. Look for the `C` and `*` keys. +The first parameter is the global config, while the second parameter is the topic config that gets applied to all subscribed topics. To view a list of all supported configuration properties, see the [Configuration.md](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md) file described previously. Look for the `C` and `*` keys. The `group.id` and `metadata.broker.list` properties are required for a consumer. diff --git a/ci/prepublish.js b/ci/prepublish.js index c5ed2107..3526be85 100644 --- a/ci/prepublish.js +++ b/ci/prepublish.js @@ -1,5 +1,4 @@ require('./checks/librdkafka-exists'); require('./checks/librdkafka-correct-version'); require('./librdkafka-defs-generator.js'); -// Temporarily disabled until we move to non-devel versions. -// require('./update-version'); +require('./update-version'); diff --git a/ci/update-version.js b/ci/update-version.js index f9df0617..50957b74 100644 --- a/ci/update-version.js +++ b/ci/update-version.js @@ -84,24 +84,15 @@ function getBranch(cb) { } function getPackageVersion(tag, branch) { - const baseVersion = `v${tag.major}.${tag.minor}.${tag.patch}`; + let baseVersion = `v${tag.major}.${tag.minor}.${tag.patch}`; - console.log(`Package version is "${baseVersion}"`); - - // never publish with an suffix - // fixes https://github.com/confluentinc/confluent-kafka-javascript/issues/981 - // baseVersion += '-'; - - // if (tag.commit === 0 && branch === 'master') { - // return baseVersion; - // } - // if (branch !== 'master') { - // baseVersion += (tag.commit + 1 + '.' + branch); - // } else { - // baseVersion += (tag.commit + 1); - // } + // publish with a -devel suffix for EA and RC releases. + if (tag.prerelease.length > 0) { + baseVersion += '-' + tag.prerelease.join('-'); + } + console.log(`Package version is "${baseVersion}"`); return baseVersion; } @@ -114,7 +105,6 @@ getVersion((err, tag) => { if (err) { throw err; } - pjs.version = getPackageVersion(tag, branch); fs.writeFileSync(pjsPath, JSON.stringify(pjs, null, 2)); diff --git a/lib/error.js b/lib/error.js index ff7c47a3..c212a50d 100644 --- a/lib/error.js +++ b/lib/error.js @@ -21,7 +21,7 @@ LibrdKafkaError.wrap = errorWrap; * Enum for identifying errors reported by the library * * You can find this list in the C++ code at - * https://github.com/edenhill/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L148 + * https://github.com/confluentinc/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L148 * * @readonly * @enum {number} diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index c1852ecc..88c07a84 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -655,7 +655,7 @@ KafkaConsumer.prototype.getWatermarkOffsets = function(topic, partition) { * * enable.auto.offset.store must be set to false to use this API, * - * @see https://github.com/edenhill/librdkafka/blob/261371dc0edef4cea9e58a076c8e8aa7dc50d452/src-cpp/rdkafkacpp.h#L1702 + * @see https://github.com/confluentinc/librdkafka/blob/261371dc0edef4cea9e58a076c8e8aa7dc50d452/src-cpp/rdkafkacpp.h#L1702 * * @param {Array.} topicPartitions - Topic partitions with offsets to store offsets for. * @throws {LibrdKafkaError} - Throws when there is no offset stored diff --git a/lib/topic.js b/lib/topic.js index c9089c15..df8d952f 100644 --- a/lib/topic.js +++ b/lib/topic.js @@ -17,7 +17,7 @@ var topicKeyLength = topicKey.length; // Take all of the topic special codes from librdkafka and add them // to the object // You can find this list in the C++ code at -// https://github.com/edenhill/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L1250 +// https://github.com/confluentinc/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L1250 for (var key in librdkafka.topic) { // Skip it if it doesn't start with ErrorCode if (key.indexOf('RdKafka::Topic::') !== 0) { diff --git a/src/connection.cc b/src/connection.cc index 6c70a785..67aba7f0 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -223,7 +223,7 @@ Baton Connection::GetMetadata( return Baton(metadata); } else { // metadata is not set here - // @see https://github.com/edenhill/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L860 + // @see https://github.com/confluentinc/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L860 return Baton(err); } } From 1890c13bc131496e589eba67fdc5d5ba4b864ce6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 19 Feb 2024 09:47:22 +0530 Subject: [PATCH 094/224] Add CHANGELOG.md for v0.1.10-devel --- CHANGELOG.md | 30 ++++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ce4d127d..4a218987 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,20 @@ -# confluent-kafka-javascript v0.1.5-devel - -v0.1.5-devel is a pre-production, early-access release. - -## Features - -1. Pre-built binaries for Linux (both amd64 and arm64, both musl and glibc), for macOS (m1), for node versions 18, 20 and 21. -2. Promisified API for Consumer, Producer and Admin Client. -3. Allow passing topic configuration properties via the global configuration block. -4. Remove dependencies with security issues. +# confluent-kafka-javascript v0.1.10-devel + +v0.1.10-devel is a pre-production, early-access release. + +## Features + +1. Pre-built binaries for Windows (x64) added on an experimental basis. + + +# confluent-kafka-javascript v0.1.9-devel + +v0.1.9-devel is a pre-production, early-access release. + +## Features + +1. Pre-built binaries for Linux (both amd64 and arm64, both musl and glibc), for macOS (m1), for node versions 18, 20 and 21. +2. Promisified API for Consumer, Producer and Admin Client. +3. Allow passing topic configuration properties via the global configuration block. +4. Remove dependencies with security issues. +5. Support for the Cooperative Sticky assignor. From 1cef0f3feb762943ec4a0dd3562460104ba827c7 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 19 Feb 2024 14:40:41 +0530 Subject: [PATCH 095/224] Bump package.json version after release --- package-lock.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/package-lock.json b/package-lock.json index 79b54884..4432d592 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.9-devel", + "version": "v0.1.10-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.9-devel", + "version": "v0.1.10-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { From 2e5f34f839b1f67878454c1472aa550b000ff23b Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 19 Feb 2024 14:42:27 +0530 Subject: [PATCH 096/224] Add confluent-cloud examples --- examples/consumer-confluent-cloud.js | 56 ++++++++++++++ examples/kafkajs/consumer-confluent-cloud.js | 74 +++++++++++++++++++ examples/kafkajs/producer-confluent-cloud.js | 46 ++++++++++++ .../node-rdkafka/consumer-confluent-cloud.md | 53 +++++++++++++ .../high-level-producer-confluent-cloud.md | 40 ++++++++++ examples/producer-confluent-cloud.js | 44 +++++++++++ 6 files changed, 313 insertions(+) create mode 100644 examples/consumer-confluent-cloud.js create mode 100644 examples/kafkajs/consumer-confluent-cloud.js create mode 100644 examples/kafkajs/producer-confluent-cloud.js create mode 100644 examples/node-rdkafka/consumer-confluent-cloud.md create mode 100644 examples/node-rdkafka/high-level-producer-confluent-cloud.md create mode 100644 examples/producer-confluent-cloud.js diff --git a/examples/consumer-confluent-cloud.js b/examples/consumer-confluent-cloud.js new file mode 100644 index 00000000..5881ac2d --- /dev/null +++ b/examples/consumer-confluent-cloud.js @@ -0,0 +1,56 @@ +const { Kafka, ErrorCodes } = require('@confluentinc/kafka-javascript').KafkaJS; + +async function consumerStart() { + let consumer; + let stopped = false; + + const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; + const CLUSTER_API_KEY = 'your_cluster_api_key_here'; + const CLUSTER_API_SECRET = 'your_cluster_api_secret_here'; + + // Set up signals for a graceful shutdown. + const disconnect = () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + stopped = true; + consumer.commitOffsets() + .finally(() => + consumer.disconnect() + ) + .finally(() => + console.log("Disconnected successfully") + ); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); + + // Initialization + consumer = new Kafka().consumer({ + 'bootstrap.servers': `${CLUSTER_BOOTSTRAP_URL}`, + 'security.protocol': 'SASL_SSL', + 'sasl.mechanisms': 'PLAIN', + 'sasl.username': `${CLUSTER_API_KEY}`, + 'sasl.password': `${CLUSTER_API_SECRET}`, + 'group.id': 'test-group', + 'auto.offset.reset': 'earliest', + 'enable.partition.eof': 'true', + }); + + await consumer.connect(); + console.log("Connected successfully"); + await consumer.subscribe({ topics: ["test-topic"] }); + + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); + } + }); +} + +consumerStart(); diff --git a/examples/kafkajs/consumer-confluent-cloud.js b/examples/kafkajs/consumer-confluent-cloud.js new file mode 100644 index 00000000..4e9d5ea0 --- /dev/null +++ b/examples/kafkajs/consumer-confluent-cloud.js @@ -0,0 +1,74 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use '@confluentinc/kafka-javascript'. +const { Kafka } = require('../..').KafkaJS; + +async function consumerStart() { + let consumer; + var stopped = false; + + const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; + const CLUSTER_API_KEY = 'your_cluster_api_key_here'; + const CLUSTER_API_SECRET = 'your_cluster_api_secret_here'; + + const kafka = new Kafka({ + kafkaJS: { + brokers: [`${CLUSTER_BOOTSTRAP_URL}`], + ssl: true, + sasl: { + mechanism: 'plain', + username: `${CLUSTER_API_KEY}`, + password: `${CLUSTER_API_SECRET}`, + }, + } + }); + + consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + }, + + /* Properties from librdkafka can also be used */ + 'auto.commit.interval.ms': 6000, + }); + + await consumer.connect(); + console.log("Connected successfully"); + + await consumer.subscribe({ + topics: [ + "test-topic" + ] + }) + + // Start consuming messages. + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); + }, + }); + + // Disconnect example + const disconnect = () => { + process.off('SIGINT', disconnect); + process.off('SIGTERM', disconnect); + stopped = true; + consumer.commitOffsets() + .finally(() => + consumer.disconnect() + ) + .finally(() => + console.log("Disconnected successfully") + ); + } + process.on('SIGINT', disconnect); + process.on('SIGTERM', disconnect); +} + +consumerStart(); diff --git a/examples/kafkajs/producer-confluent-cloud.js b/examples/kafkajs/producer-confluent-cloud.js new file mode 100644 index 00000000..b4b8eb28 --- /dev/null +++ b/examples/kafkajs/producer-confluent-cloud.js @@ -0,0 +1,46 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +// Since this example is within the package itself, we use '../..', but code +// will typically use '@confluentinc/kafka-javascript'. +const { Kafka } = require('../..').KafkaJS; + +async function producerStart() { + const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; + const CLUSTER_API_KEY = 'your_cluster_api_key_here'; + const CLUSTER_API_SECRET = 'your_cluster_api_secret_here'; + + const kafka = new Kafka({ + kafkaJS: { + brokers: [`${CLUSTER_BOOTSTRAP_URL}`], + ssl: true, + sasl: { + mechanism: 'plain', + username: `${CLUSTER_API_KEY}`, + password: `${CLUSTER_API_SECRET}`, + }, + } + }); + + const producer = kafka.producer(); + + await producer.connect(); + + console.log("Connected successfully"); + + const res = [] + for (let i = 0; i < 50; i++) { + res.push(producer.send({ + topic: 'test-topic', + messages: [ + { value: 'v222', partition: 0 }, + { value: 'v11', partition: 0, key: 'x' }, + ] + })); + } + await Promise.all(res); + + await producer.disconnect(); + + console.log("Disconnected successfully"); +} + +producerStart(); diff --git a/examples/node-rdkafka/consumer-confluent-cloud.md b/examples/node-rdkafka/consumer-confluent-cloud.md new file mode 100644 index 00000000..629825ee --- /dev/null +++ b/examples/node-rdkafka/consumer-confluent-cloud.md @@ -0,0 +1,53 @@ +Connecting to a Kafka Consumer is easy. Let's try to connect to one using +the Stream implementation + +```js +/* + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2023 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +const Transform = require('stream').Transform; + +const Kafka = require('../'); + +const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; +const CLUSTER_API_KEY = 'your_cluster_api_key_here'; +const CLUSTER_API_SECRET = 'your_cluster_api_secret_here'; + +const stream = Kafka.KafkaConsumer.createReadStream({ + 'bootstrap.servers': `${CLUSTER_BOOTSTRAP_URL}`, + 'group.id': 'test-group', + 'socket.keepalive.enable': true, + 'enable.auto.commit': false, + 'security.protocol': 'SASL_SSL', + 'sasl.mechanisms': 'PLAIN', + 'sasl.username': `${CLUSTER_API_KEY}`, + 'sasl.password': `${CLUSTER_API_SECRET}`, +}, {}, { + topics: 'test', + waitInterval: 0, + objectMode: false +}); + +stream.on('error', function(err) { + if (err) console.log(err); + process.exit(1); +}); + +stream + .pipe(process.stdout); + +stream.on('error', function(err) { + console.log(err); + process.exit(1); +}); + +stream.consumer.on('event.error', function(err) { + console.log(err); +}) +``` diff --git a/examples/node-rdkafka/high-level-producer-confluent-cloud.md b/examples/node-rdkafka/high-level-producer-confluent-cloud.md new file mode 100644 index 00000000..e3b8f9aa --- /dev/null +++ b/examples/node-rdkafka/high-level-producer-confluent-cloud.md @@ -0,0 +1,40 @@ +```js +const Kafka = require('../'); + +const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; +const CLUSTER_API_KEY = 'your_cluster_api_key_here'; +const CLUSTER_API_SECRET = 'your_cluster_api_secret_here'; + +const producer = new Kafka.HighLevelProducer({ + 'bootstrap.servers': `${CLUSTER_BOOTSTRAP_URL}`, + 'security.protocol': 'SASL_SSL', + 'sasl.mechanisms': 'PLAIN', + 'sasl.username': `${CLUSTER_API_KEY}`, + 'sasl.password': `${CLUSTER_API_SECRET}`, +}); + +// Throw away the keys +producer.setKeySerializer(function(v) { + return new Promise((resolve, reject) => { + setTimeout(() => { + resolve(null); + }, 20); + }); +}); + +// Take the message field +producer.setValueSerializer(function(v) { + return Buffer.from(v.message); +}); + +producer.connect(null, function() { + producer.produce('test', null, { + message: 'alliance4ever', + }, null, Date.now(), function(err, offset) { + // The offset if our acknowledgement level allows us to receive delivery offsets + setImmediate(function() { + producer.disconnect(); + }); + }); +}); +``` diff --git a/examples/producer-confluent-cloud.js b/examples/producer-confluent-cloud.js new file mode 100644 index 00000000..94438e8f --- /dev/null +++ b/examples/producer-confluent-cloud.js @@ -0,0 +1,44 @@ +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; + +async function producerStart() { + const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; + const CLUSTER_API_KEY = 'your_cluster_api_key_here'; + const CLUSTER_API_SECRET = 'your_cluster_api_secret_here'; + + const producer = new Kafka().producer({ + 'bootstrap.servers': `${CLUSTER_BOOTSTRAP_URL}`, + 'security.protocol': 'SASL_SSL', + 'sasl.mechanisms': 'PLAIN', + 'sasl.username': `${CLUSTER_API_KEY}`, + 'sasl.password': `${CLUSTER_API_SECRET}`, + 'acks': 'all', + }); + + await producer.connect(); + console.log("Connected successfully"); + + const res = [] + for (let i = 0; i < 50; i++) { + res.push(producer.send({ + topic: 'test-topic', + messages: [ + { value: 'v222', partition: 1 }, + { value: 'v11', partition: 0, key: 'x' }, + ] + })); + } + + const produceRecords = await Promise.all(res); + + // Produce records is an array of delivery reports for each call to `send`. + // In case `messages` contains more than one message to the same topic-partition, only the last + // delivery report is included in the array. + console.log("Produced messages, first delivery report:\n" + JSON.stringify(produceRecords[0], null, 2)); + console.log("Produced messages, last delivery report:\n" + JSON.stringify(produceRecords[produceRecords.length - 1], null, 2)); + + await producer.disconnect(); + + console.log("Disconnected successfully"); +} + +producerStart(); From 76bc8ca5d4a17c587967cb328a32b93c59235d75 Mon Sep 17 00:00:00 2001 From: Maxwell Lang <59022944+maxwlang@users.noreply.github.com> Date: Thu, 22 Feb 2024 00:09:43 -0600 Subject: [PATCH 097/224] Delete LICENCE.node-rdkafka-txt (#30) Empty license file --- LICENCE.node-rdkafka-txt | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 LICENCE.node-rdkafka-txt diff --git a/LICENCE.node-rdkafka-txt b/LICENCE.node-rdkafka-txt deleted file mode 100644 index e69de29b..00000000 From a6cc7a8b9e4fdd98e70406c3fa378a812207c572 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 5 Mar 2024 12:16:43 +0530 Subject: [PATCH 098/224] Add eachBatch API compability --- CHANGELOG.md | 9 + MIGRATION.md | 4 +- lib/kafkajs/_common.js | 2 - lib/kafkajs/_consumer.js | 136 ++++++++++++-- .../consumer/consumeMessages.spec.js | 172 +++++++++++++++++- test/promisified/consumer/pause.spec.js | 6 +- 6 files changed, 298 insertions(+), 31 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4a218987..0fb8e71a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +# confluent-kafka-javascript vNext + +v0.1.11-devel is a pre-production, early-access release. + +## Features + +1. Add support for `eachBatch` in the Consumer API (partial support for API compatibility). + + # confluent-kafka-javascript v0.1.10-devel v0.1.10-devel is a pre-production, early-access release. diff --git a/MIGRATION.md b/MIGRATION.md index ff1fdf04..04652506 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -26,7 +26,7 @@ Rather, it must be configured in the top-level configuration while creating the consumer. - `autoCommit` and `autoCommitInterval` are not set on a per-run() basis. Rather, they must be configured in the top-level configuration while creating the consumer. - - `autoCommitThreshold` and `eachBatch` are not supported. + - `autoCommitThreshold` is not supported and `eachBatch` batch size never exceeds 1. - For errors: Check the `error.code` rather than the error `name` or `type`. 4. A more exhaustive list of semantic and configuration differences is [presented below](#common). @@ -282,7 +282,7 @@ producerRun().then(consumerRun).catch(console.error); ``` - The `heartbeat()` no longer needs to be called by the user in the `eachMessage callback`. Heartbeats are automatically managed by librdkafka. - The `partitionsConsumedConcurrently` property is not supported at the moment. - - The `eachBatch` method is not supported. + - An API compatible version of `eachBatch` is available, but the batch size never exceeds 1. The property `eachBatchAutoResolve` is supported. * `commitOffsets`: - Does not yet support sending metadata for topic partitions being committed. * `seek`: diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index b0b0b584..3d263649 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -244,8 +244,6 @@ const CompatibilityErrorMessages = Object.freeze({ "The property 'autoCommitThreshold' is not supported by run.\n", runOptionsRunConcurrently: () => "The property 'partitionsConsumedConcurrently' is not currently supported by run\n", - runOptionsEachBatch: () => - "The property 'eachBatch' is not currently supported by run\n", }); /** diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 3807f7b5..1c964584 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -633,6 +633,71 @@ class Consumer { }; } + /** + * Converts a message returned by node-rdkafka into a message that can be used by the eachBatch callback. + * @param {import("../..").Message} message + * @returns {import("../../types/kafkajs").EachBatchPayload} + * @note Unlike the KafkaJS consumer, a batch here is for API compatibility only. It is always a single message. + */ + #createBatchPayload(message) { + let key = message.key; + if (typeof key === 'string') { + key = Buffer.from(key); + } + + let timestamp = message.timestamp ? String(message.timestamp) : ''; + + let headers; + if (message.headers) { + headers = {} + for (const [key, value] of Object.entries(message.headers)) { + if (!Object.hasOwn(headers, key)) { + headers[key] = value; + } else if (headers[key].constructor === Array) { + headers[key].push(value); + } else { + headers[key] = [headers[key], value]; + } + } + } + + const messageConverted = { + key, + value: message.value, + timestamp, + attributes: 0, + offset: String(message.offset), + size: message.size, + headers + }; + + const batch = { + topic: message.topic, + partition: message.partition, + highWatermark: '-1001', // Invalid - we don't fetch it + messages: [messageConverted], + isEmpty: () => false, + firstOffset: () => messageConverted.offset, + lastOffset: () => messageConverted.offset, + offsetLag: () => notImplemented(), + offsetLagLow: () => notImplemented(), + }; + + const returnPayload = { + batch, + _messageResolved: false, + resolveOffset: () => { returnPayload._messageResolved = true; }, + heartbeat: async () => { /* no op */ }, + pause: this.pause.bind(this, [{ topic: message.topic, partitions: [message.partition] }]), + commitOffsetsIfNecessary: async () => { /* no op */ }, + uncommittedOffsets: () => notImplemented(), + isRunning: () => this.#running, + isStale: () => false, + }; + + return returnPayload; + } + /** * Consumes a single message from the internal consumer. * @returns {Promise} a promise that resolves to a single message. @@ -814,10 +879,6 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - if (Object.hasOwn(config, 'eachBatch')) { - throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsEachBatch(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); - } - if (Object.hasOwn(config, 'partitionsConsumedConcurrently')) { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsPartitionsConsumedConcurrently(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } @@ -827,6 +888,11 @@ class Consumer { } this.#running = true; + /* Batches are auto resolved by default. */ + if (!Object.hasOwn(config, 'eachBatchAutoResolve')) { + config.eachBatchAutoResolve = true; + } + /* We deliberately don't await this. */ this.#runInternal(config); } @@ -860,14 +926,14 @@ class Consumer { } /* 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 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) { @@ -880,16 +946,48 @@ class Consumer { } let eachMessageProcessed = false; + let payload; + if (config.eachMessage) { + payload = this.#createPayload(m); + } else { + payload = this.#createBatchPayload(m); + } try { - await config.eachMessage( - this.#createPayload(m) - ); - eachMessageProcessed = true; + if (config.eachMessage) { + await config.eachMessage(payload); + eachMessageProcessed = true; + } else { + await config.eachBatch(payload); + if (config.eachBatchAutoResolve) { + eachMessageProcessed = true; + } else { + eachMessageProcessed = payload._messageResolved; + } + } } catch (e) { - /* It's not only possible, but expected that an error will be thrown by eachMessage. + /* It's not only possible, but expected that an error will be thrown by eachMessage or eachBatch. * 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). */ + * is encouraged. To meet the API contract, we seek one offset backward (which + * means seeking to the message offset). + * However, we don't do this inside the catch, but just outside it. This is because throwing an + * error is not the only case where we might want to seek back. We might want to seek back + * if the user has not called `resolveOffset` manually in case of using eachBatch without + * eachBatchAutoResolve being set. + * + * So - do nothing but a debug log, but at this point eachMessageProcessed needs to be false unless + * the user has explicitly marked it as true. + */ + this.#logger.debug(`Consumer encountered error while processing message. Error details: ${JSON.stringify(e)}. The same message may be reprocessed.`); + + /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed + * despite an error is if the user says so, and the user can use resolveOffsets for both the possible + * values eachBatchAutoResolve can take. */ + if (config.eachBatch) + eachMessageProcessed = payload._messageResolved + } + + /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ + if (!eachMessageProcessed) { await this.seek({ topic: m.topic, partition: m.partition, diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 440644eb..b22f600c 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -85,6 +85,168 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) }); + it.each([[true], [false]])('consumes messages using eachBatch', async (isAutoResolve) => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + consumer.run({ + eachBatchAutoResolve: isAutoResolve, + eachBatch: async event => { + // Match the message format to be checked easily later. + event.batch.messages = event.batch.messages.map(msg => ({ + message: msg, + topic: event.batch.topic, + partition: event.batch.partition, + })); + messagesConsumed.push(...event.batch.messages); + + // If we're not auto-resolving, we need to resolve the offsets manually. + if (!isAutoResolve) + event.resolveOffset(); + } + }); + + 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: String(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: String(messages.length - 1), + }), + }) + ) + + // check if all offsets are present + expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) + }); + + it('is able to reconsume messages after not resolving it', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + let messageSeen = false; + const messagesConsumed = []; + consumer.run({ + eachBatchAutoResolve: false, + eachBatch: async event => { + expect(event.batch.messages.length).toEqual(1); + expect(event.batch.messages[0].offset).toEqual('0'); + expect(event.batch.topic).toEqual(topicName); + expect(event.batch.partition).toEqual(0); + + if (!messageSeen) { + messageSeen = true; + return; + } + messagesConsumed.push(...event.batch.messages); + + // Since we're not auto-resolving, we need to resolve the offsets manually. + event.resolveOffset(); + } + }); + + const messages = Array(1) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + + await producer.send({ topic: topicName, messages }) + await waitForMessages(messagesConsumed, { number: messages.length }); + }); + + it.each([[true], [false]])('is able to reconsume messages when an error is thrown', async (isAutoResolve) => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + let messageSeen = false; + const messagesConsumed = []; + consumer.run({ + eachBatchAutoResolve: isAutoResolve, + eachBatch: async event => { + expect(event.batch.messages.length).toEqual(1); + expect(event.batch.messages[0].offset).toEqual('0'); + expect(event.batch.topic).toEqual(topicName); + expect(event.batch.partition).toEqual(0); + + if (!messageSeen) { + messageSeen = true; + throw new Error('a new error.'); + } + messagesConsumed.push(...event.batch.messages); + } + }); + + const messages = Array(1) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + + await producer.send({ topic: topicName, messages }) + await waitForMessages(messagesConsumed, { number: messages.length }); + }); + + it.each([[true], [false]])('does not reconsume resolved messages even on error', async (isAutoResolve) => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + consumer.run({ + eachBatchAutoResolve: isAutoResolve, + eachBatch: async event => { + messagesConsumed.push(...event.batch.messages); + // Resolve offsets irrespective of the value of eachBatchAutoResolve. + event.resolveOffset(); + throw new Error('a new error.'); + } + }); + + const messages = Array(2) + .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].key.toString()).toBe(messages[0].key); + expect(messagesConsumed[1].key.toString()).toBe(messages[1].key); + }); + /* Skip until concurrency support for eachMessage is added. */ it.skip('consumes messages concurrently', async () => { const partitionsConsumedConcurrently = 2 @@ -175,8 +337,8 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { ]) }); - /* Skip as it uses eachBatch */ - it.skip('commits the last offsets processed before stopping', async () => { + /* Skip as it uses consuimer events. */ + it('commits the last offsets processed before stopping', async () => { jest.spyOn(cluster, 'refreshMetadataIfNecessary') await Promise.all([admin.connect(), consumer.connect(), producer.connect()]) @@ -300,7 +462,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) }); - /* Skip as it uses eachBatch */ + /* Skip as the current implementation will never fetch more than 1 message. */ it.skip('resolves a batch as stale when seek was called while processing it', async () => { consumer = createConsumer({ groupId, @@ -347,7 +509,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) }); - /* Skip as it uses eachBatch */ + /* Skip as it uses consumer events */ it.skip('skips messages fetched while seek was called', async () => { consumer = createConsumer({ cluster: createCluster(), @@ -398,7 +560,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { }); }); - /* Skip as it uses eachBatch */ + /* Skip as it uses consumer events */ it.skip('discards messages received when pausing while fetch is in-flight', async () => { consumer = createConsumer({ cluster: createCluster(), diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index 6e64c195..0ad75ac6 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -187,8 +187,7 @@ describe('Consumer', () => { 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 () => { + it('pauses when pausing via the eachBatch callback', async () => { await consumer.connect() await producer.connect() const originalMessages = [0, 0, 0, 1].map(partition => { @@ -238,6 +237,7 @@ describe('Consumer', () => { }) await waitForConsumerToJoinGroup(consumer) await waitForMessages(messagesConsumed, { number: 5 }) + expect(messagesConsumed.length).toEqual(5); expect(consumer.paused()).toContainEqual({ topic: topics[0], partitions: [0] }) expect(consumer.paused()).toContainEqual({ topic: topics[1], partitions: [1] }) shouldPause = false @@ -246,7 +246,7 @@ describe('Consumer', () => { expect(consumer.paused()).toEqual([]) expect(messagesConsumed).toContainEqual({ topic: 0, message: 1 }) expect(messagesConsumed).toContainEqual({ topic: 1, message: 3 }) - }); + }, 10000); it('does not fetch messages for the paused topic', async () => { await consumer.connect(); From fc907bdeead1834da9f8dad7afa06ca8e4d994b2 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 6 Mar 2024 10:33:03 +0530 Subject: [PATCH 099/224] Add static membership test --- .../consumer/groupInstanceId.spec.js | 136 ++++++++++++++++++ 1 file changed, 136 insertions(+) create mode 100644 test/promisified/consumer/groupInstanceId.spec.js diff --git a/test/promisified/consumer/groupInstanceId.spec.js b/test/promisified/consumer/groupInstanceId.spec.js new file mode 100644 index 00000000..c905d95a --- /dev/null +++ b/test/promisified/consumer/groupInstanceId.spec.js @@ -0,0 +1,136 @@ +jest.setTimeout(30000); + +const { waitFor, + secureRandom, + createTopic, + createConsumer, + sleep, } = require("../testhelpers"); +const { ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Consumer with static membership', () => { + let consumer; + let groupId, topicName; + + const consumerConfig = { + groupId, + }; + + beforeEach(async () => { + topicName = `test-topic1-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}` + consumer = null; + await createTopic({ topic: topicName, partitions: 2 }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + }); + + it('does not rebalance after disconnect', async () => { + let assigns = 0; + let revokes = 0; + const rebalanceCallback = function (err, assignment) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assigns++; + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + revokes++; + } else { + // It's either assign or revoke and nothing else. + jest.fail('Unexpected error code'); + } + } + + // Create and start two consumers. + consumer = createConsumer(consumerConfig, { + 'group.instance.id': 'instance-1', + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + const consumer2 = createConsumer(consumerConfig, { + 'rebalance_cb': rebalanceCallback, + 'group.instance.id': 'instance-2', + }); + await consumer2.connect(); + await consumer2.subscribe({ topic: topicName }); + consumer2.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer2.assignment().length > 0, () => null, 1000); + expect(assigns).toBe(1); + + // Disconnect one consumer and reconnect it. It should not cause a rebalance in the other. + await consumer.disconnect(); + + consumer = createConsumer(consumerConfig, { + 'group.instance.id': 'instance-1', + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(assigns).toBe(1); + expect(revokes).toBe(0); + + await consumer.disconnect(); + await consumer2.disconnect(); + consumer = null; + }); + + it('does rebalance after session timeout', async () => { + let assigns = 0; + let revokes = 0; + const rebalanceCallback = function (err, assignment) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assigns++; + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + revokes++; + } else { + // It's either assign or revoke and nothing else. + jest.fail('Unexpected error code'); + } + } + + // Create and start two consumers. + consumer = createConsumer(consumerConfig, { + 'group.instance.id': 'instance-1', + 'session.timeout.ms': '10000', + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + const consumer2 = createConsumer(consumerConfig, { + 'rebalance_cb': rebalanceCallback, + 'group.instance.id': 'instance-2', + 'session.timeout.ms': '10000', + }); + await consumer2.connect(); + await consumer2.subscribe({ topic: topicName }); + consumer2.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer2.assignment().length > 0, () => null, 1000); + expect(assigns).toBe(1); + + // Disconnect one consumer and reconnect it. It should cause a rebalance after session timeout. + await consumer.disconnect(); + consumer = null; + + await sleep(8000); + + // Session timeout (10s) hasn't kicked in yet - we have slept for just 8s. + expect(consumer2.assignment().length).toBe(1); + + await waitFor(() => consumer2.assignment().length === 2, () => null, 1000); + expect(consumer2.assignment().length).toBe(2); + expect(assigns).toBe(2); + expect(revokes).toBe(1); + + await consumer2.disconnect(); + }); +}); \ No newline at end of file From a347f7e56b906d0ef32a02c21fe7be3a1ff7e490 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Mar 2024 12:48:00 +0530 Subject: [PATCH 100/224] Skip consumer test which uses events --- test/promisified/consumer/consumeMessages.spec.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index b22f600c..9228bb49 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -338,7 +338,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { }); /* Skip as it uses consuimer events. */ - it('commits the last offsets processed before stopping', async () => { + it.skip('commits the last offsets processed before stopping', async () => { jest.spyOn(cluster, 'refreshMetadataIfNecessary') await Promise.all([admin.connect(), consumer.connect(), producer.connect()]) From 7960d7723f825f57af5ff7071ce28fe40af73077 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Mar 2024 13:52:11 +0530 Subject: [PATCH 101/224] Add ListGroups to admin client --- index.d.ts | 20 ++++ lib/admin.js | 73 ++++++++++-- lib/kafkajs/_admin.js | 29 ++++- lib/kafkajs/_kafka.js | 10 +- src/admin.cc | 113 +++++++++++++++++++ src/admin.h | 7 ++ src/common.cc | 123 ++++++++++++++++++++- src/common.h | 32 ++++-- src/errors.cc | 27 ++++- src/errors.h | 5 + src/kafka-consumer.cc | 5 +- src/workers.cc | 55 +++++++++ src/workers.h | 22 ++++ test/promisified/admin/list_groups.spec.js | 91 +++++++++++++++ types/kafkajs.d.ts | 52 ++++++--- 15 files changed, 618 insertions(+), 46 deletions(-) create mode 100644 test/promisified/admin/list_groups.spec.js diff --git a/index.d.ts b/index.d.ts index c4ee556d..5ad6dd9e 100644 --- a/index.d.ts +++ b/index.d.ts @@ -333,6 +333,22 @@ export interface NewTopic { } | { [cfg: string]: string; }; } +export enum ConsumerGroupStates { + UNKNOWN = 0, + PREPARING_REBALANCE = 1, + COMPLETING_REBALANCE = 2, + STABLE = 3, + DEAD = 4, + EMPTY = 5, +} + +export interface GroupOverview { + groupId: string; + protocolType: string; + isSimpleConsumerGroup: boolean; + state: ConsumerGroupStates; +} + export interface IAdminClient { createTopic(topic: NewTopic, cb?: (err: LibrdKafkaError) => void): void; createTopic(topic: NewTopic, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; @@ -343,6 +359,10 @@ export interface IAdminClient { createPartitions(topic: string, desiredPartitions: number, cb?: (err: LibrdKafkaError) => void): void; createPartitions(topic: string, desiredPartitions: number, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; + listGroups(cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; + listGroups(options: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }, + cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; + disconnect(): void; } diff --git a/lib/admin.js b/lib/admin.js index 120a5353..df7d7c95 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -2,14 +2,30 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * 2024 Confluent, Inc * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. */ 'use strict'; +/* TODO: Think of a way to fetch these from within librdkafka instead of this + * hardcoded list. + * New additions won't be automatically added to this list. + */ +const ConsumerGroupStates = Object.seal({ + UNKNOWN: 0, + PREPARING_REBALANCE: 1, + COMPLETING_REBALANCE: 2, + STABLE: 3, + DEAD: 4, + EMPTY: 5, +}) + + module.exports = { create: createAdminClient, + ConsumerGroupStates, }; var Client = require('./client'); @@ -89,7 +105,7 @@ function AdminClient(conf) { * * Unlike the other connect methods, this one is synchronous. */ -AdminClient.prototype.connect = function() { +AdminClient.prototype.connect = function () { LibrdKafkaError.wrap(this._client.connect(), true); this._isConnected = true; }; @@ -100,7 +116,7 @@ AdminClient.prototype.connect = function() { * This is a synchronous method, but all it does is clean up * some memory and shut some threads down */ -AdminClient.prototype.disconnect = function() { +AdminClient.prototype.disconnect = function () { LibrdKafkaError.wrap(this._client.disconnect(), true); this._isConnected = false; }; @@ -112,7 +128,7 @@ AdminClient.prototype.disconnect = function() { * @param {number} timeout - Number of milliseconds to wait while trying to create the topic. * @param {function} cb - The callback to be executed when finished */ -AdminClient.prototype.createTopic = function(topic, timeout, cb) { +AdminClient.prototype.createTopic = function (topic, timeout, cb) { if (!this._isConnected) { throw new Error('Client is disconnected'); } @@ -126,7 +142,7 @@ AdminClient.prototype.createTopic = function(topic, timeout, cb) { timeout = 5000; } - this._client.createTopic(topic, timeout, function(err) { + this._client.createTopic(topic, timeout, function (err) { if (err) { if (cb) { cb(LibrdKafkaError.create(err)); @@ -147,7 +163,7 @@ AdminClient.prototype.createTopic = function(topic, timeout, cb) { * @param {number} timeout - Number of milliseconds to wait while trying to delete the topic. * @param {function} cb - The callback to be executed when finished */ -AdminClient.prototype.deleteTopic = function(topic, timeout, cb) { +AdminClient.prototype.deleteTopic = function (topic, timeout, cb) { if (!this._isConnected) { throw new Error('Client is disconnected'); } @@ -161,7 +177,7 @@ AdminClient.prototype.deleteTopic = function(topic, timeout, cb) { timeout = 5000; } - this._client.deleteTopic(topic, timeout, function(err) { + this._client.deleteTopic(topic, timeout, function (err) { if (err) { if (cb) { cb(LibrdKafkaError.create(err)); @@ -184,7 +200,7 @@ AdminClient.prototype.deleteTopic = function(topic, timeout, cb) { * @param {number} timeout - Number of milliseconds to wait while trying to create the partitions. * @param {function} cb - The callback to be executed when finished */ -AdminClient.prototype.createPartitions = function(topic, totalPartitions, timeout, cb) { +AdminClient.prototype.createPartitions = function (topic, totalPartitions, timeout, cb) { if (!this._isConnected) { throw new Error('Client is disconnected'); } @@ -198,7 +214,7 @@ AdminClient.prototype.createPartitions = function(topic, totalPartitions, timeou timeout = 5000; } - this._client.createPartitions(topic, totalPartitions, timeout, function(err) { + this._client.createPartitions(topic, totalPartitions, timeout, function (err) { if (err) { if (cb) { cb(LibrdKafkaError.create(err)); @@ -211,3 +227,44 @@ AdminClient.prototype.createPartitions = function(topic, totalPartitions, timeou } }); }; + +/** + * List consumer groups. + * @param {any} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @param {import("../").ConsumerGroupStates[]?} options.matchConsumerGroupStates - + * A list of consumer group states to match. May be unset, fetches all states (default: unset). + * @param {function} cb - The callback to be executed when finished. + * + * Valid ways to call this function: + * listGroups(cb) + * listGroups(options, cb) + */ +AdminClient.prototype.listGroups = function (options, cb) { + if (!this._isConnected) { + throw new Error('Client is disconnected'); + } + + if (typeof options === 'function') { + cb = options; + options = {}; + } + + if (!options) { + options = {}; + } + + this._client.listGroups(options, function (err, groups) { + if (err) { + if (cb) { + cb(LibrdKafkaError.create(err)); + } + return; + } + + if (cb) { + cb(null, groups); + } + }); +} \ No newline at end of file diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 4b976e3c..45d1e95b 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -231,6 +231,33 @@ class Admin { ); } + /** + * List consumer groups. + * + * @param {object?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @param {import("../../types/kafkajs").ConsumerGroupStates[]?} options.matchConsumerGroupStates - + * A list of consumer group states to match. May be unset, fetches all states (default: unset). + * @returns {Promise<{ groups: import("../../types/kafkajs").GroupOverview[], errors: import("../../types/kafkajs").LibrdKafkaError[] }>} + * Resolves with the list of consumer groups, rejects on error. + */ + async listGroups(options = {}) { + if (this.#state !== AdminState.CONNECTED) { + throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE }); + } + + return new Promise((resolve, reject) => { + this.#internalClient.listGroups(options, (err, groups) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } else { + resolve(groups); + } + }); + }); + } + } -module.exports = { Admin } +module.exports = { Admin, ConsumerGroupStates: RdKafka.AdminClient.ConsumerGroupStates } diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 6196cb2a..51b0866f 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,6 +1,6 @@ const { Producer, CompressionTypes } = require('./_producer'); const { Consumer, PartitionAssigners } = require('./_consumer'); -const { Admin } = require('./_admin'); +const { Admin, ConsumerGroupStates } = require('./_admin'); const error = require('./_error'); const { logLevel, checkIfKafkaJsKeysPresent, CompatibilityErrorMessages } = require('./_common'); @@ -80,4 +80,10 @@ class Kafka { } } -module.exports = { Kafka, ...error, logLevel, PartitionAssigners, PartitionAssignors: PartitionAssigners, CompressionTypes }; +module.exports = { + Kafka, + ...error, logLevel, + PartitionAssigners, + PartitionAssignors: PartitionAssigners, + CompressionTypes, + ConsumerGroupStates }; diff --git a/src/admin.cc b/src/admin.cc index 9c6a5865..0251b17f 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -87,6 +87,9 @@ void AdminClient::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "deleteTopic", NodeDeleteTopic); Nan::SetPrototypeMethod(tpl, "createPartitions", NodeCreatePartitions); + // Consumer group related operations + Nan::SetPrototypeMethod(tpl, "listGroups", NodeListGroups); + Nan::SetPrototypeMethod(tpl, "connect", NodeConnect); Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect); Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); @@ -179,6 +182,9 @@ rd_kafka_event_t* PollForEvent( rd_kafka_event_type(event_response) != event_type && attempts > 0); + // TODO: change this function so a type mismatch leads to an INVALID_TYPE + // error rather than a null event. A null event is treated as a timeout, which + // isn't true all the time. // If this isn't the type of response we want, or if we do not have a response // type, bail out with a null if (event_response == NULL || @@ -421,6 +427,69 @@ Baton AdminClient::CreatePartitions( } } +Baton AdminClient::ListGroups( + bool is_match_states_set, + std::vector &match_states, int timeout_ms, + /* out */ rd_kafka_event_t **event_response) { + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + { + scoped_shared_write_lock lock(m_connection_lock); + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + // Make admin options to establish that we are listing groups + rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new( + m_client->c_ptr(), RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS); + + if (is_match_states_set) { + rd_kafka_error_t *error = + rd_kafka_AdminOptions_set_match_consumer_group_states( + options, &match_states[0], match_states.size()); + if (error) { + return Baton::BatonFromErrorAndDestroy(error); + } + } + + // Create queue just for this operation. + rd_kafka_queue_t *topic_rkqu = rd_kafka_queue_new(m_client->c_ptr()); + + rd_kafka_ListConsumerGroups(m_client->c_ptr(), options, topic_rkqu); + + // Poll for an event by type in that queue + // DON'T destroy the event. It is the out parameter, and ownership is + // the caller's. + *event_response = PollForEvent( + topic_rkqu, RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT, timeout_ms); + + // Destroy the queue since we are done with it. + rd_kafka_queue_destroy(topic_rkqu); + + // Destroy the options we just made because we polled already + rd_kafka_AdminOptions_destroy(options); + + // If we got no response from that operation, this is a failure + // likely due to time out + if (*event_response == NULL) { + return Baton(RdKafka::ERR__TIMED_OUT); + } + + // Now we can get the error code from the event + if (rd_kafka_event_error(*event_response)) { + // If we had a special error code, get out of here with it + const rd_kafka_resp_err_t errcode = rd_kafka_event_error(*event_response); + return Baton(static_cast(errcode)); + } + + // At this point, event_response contains the result, which needs + // to be parsed/converted by the caller. + return Baton(RdKafka::ERR_NO_ERROR); + } +} + void AdminClient::ActivateDispatchers() { // Listen to global config m_gconfig->listen(); @@ -600,4 +669,48 @@ NAN_METHOD(AdminClient::NodeCreatePartitions) { return info.GetReturnValue().Set(Nan::Null()); } +/** + * List Consumer Groups. + */ +NAN_METHOD(AdminClient::NodeListGroups) { + Nan::HandleScope scope; + + if (info.Length() < 2 || !info[1]->IsFunction()) { + // Just throw an exception + return Nan::ThrowError("Need to specify a callback"); + } + + if (!info[0]->IsObject()) { + return Nan::ThrowError("Must provide options object"); + } + + v8::Local config = info[0].As(); + + // Create the final callback object + v8::Local cb = info[1].As(); + Nan::Callback *callback = new Nan::Callback(cb); + AdminClient *client = ObjectWrap::Unwrap(info.This()); + + // Get the timeout - default 5000. + int timeout_ms = GetParameter(config, "timeout", 5000); + + // Get the match states, or not if they are unset. + std::vector match_states; + v8::Local matchConsumerGroupStatesKey = + Nan::New("matchConsumerGroupStates").ToLocalChecked(); + bool is_match_states_set = + Nan::Has(config, matchConsumerGroupStatesKey).FromMaybe(false); + v8::Local match_states_array; + + if (is_match_states_set) { + match_states_array = GetParameter>( + config, "matchConsumerGroupStates", match_states_array); + match_states = Conversion::Admin::FromV8GroupStateArray(match_states_array); + } + + // Queue the work. + Nan::AsyncQueueWorker(new Workers::AdminClientListGroups( + callback, client, is_match_states_set, match_states, timeout_ms)); +} + } // namespace NodeKafka diff --git a/src/admin.h b/src/admin.h index a44ce1b2..6e4ebd8e 100644 --- a/src/admin.h +++ b/src/admin.h @@ -51,6 +51,10 @@ class AdminClient : public Connection { Baton CreatePartitions(rd_kafka_NewPartitions_t* topic, int timeout_ms); // Baton AlterConfig(rd_kafka_NewTopic_t* topic, int timeout_ms); // Baton DescribeConfig(rd_kafka_NewTopic_t* topic, int timeout_ms); + Baton ListGroups(bool is_match_states_set, + std::vector& match_states, + int timeout_ms, + rd_kafka_event_t** event_response); protected: static Nan::Persistent constructor; @@ -68,6 +72,9 @@ class AdminClient : public Connection { static NAN_METHOD(NodeDeleteTopic); static NAN_METHOD(NodeCreatePartitions); + // Consumer group operations + static NAN_METHOD(NodeListGroups); + static NAN_METHOD(NodeConnect); static NAN_METHOD(NodeDisconnect); }; diff --git a/src/common.cc b/src/common.cc index 836b8b0f..6e4d928c 100644 --- a/src/common.cc +++ b/src/common.cc @@ -142,9 +142,24 @@ std::vector v8ArrayToStringVector(v8::Local parameter) { return newItem; } +template<> v8::Local GetParameter >( + v8::Local object, std::string field_name, v8::Local def) { + v8::Local field = Nan::New(field_name.c_str()).ToLocalChecked(); + + if (Nan::Has(object, field).FromMaybe(false)) { + v8::Local maybeArray = Nan::Get(object, field).ToLocalChecked(); + if (maybeArray->IsArray()) { + v8::Local parameter = maybeArray.As(); + return parameter; + } + } + + return def; +} + namespace Conversion { -namespace Topic { +namespace Util { std::vector ToStringVector(v8::Local parameter) { std::vector newItem; @@ -185,14 +200,31 @@ v8::Local ToV8Array(std::vector parameter) { for (size_t i = 0; i < parameter.size(); i++) { std::string topic = parameter[i]; - Nan::Set(newItem, i, Nan::New(topic).ToLocalChecked()); } return newItem; } -} // namespace Topic +/** + * @brief Converts a list of rd_kafka_error_t* into a v8 array of RdKafkaError + * objects. + */ +v8::Local ToV8Array(const rd_kafka_error_t** error_list, + size_t error_cnt) { + v8::Local errors = Nan::New(); + + for (size_t i = 0; i < error_cnt; i++) { + RdKafka::ErrorCode code = + static_cast(rd_kafka_error_code(error_list[i])); + std::string msg = std::string(rd_kafka_error_string(error_list[i])); + Nan::Set(errors, i, RdKafkaError(code, msg)); + } + + return errors; +} + +} // namespace Util namespace TopicPartition { @@ -568,6 +600,91 @@ rd_kafka_NewTopic_t** FromV8TopicObjectArray(v8::Local) { return NULL; } +/** + * @brief Converts a v8 array of group states into a vector of + * rd_kafka_consumer_group_state_t. + */ +std::vector FromV8GroupStateArray( + v8::Local array) { + v8::Local parameter = array.As(); + std::vector returnVec; + if (parameter->Length() >= 1) { + for (unsigned int i = 0; i < parameter->Length(); i++) { + v8::Local v; + if (!Nan::Get(parameter, i).ToLocal(&v)) { + continue; + } + Nan::Maybe maybeT = Nan::To(v); + if (maybeT.IsNothing()) { + continue; + } + int64_t state_number = maybeT.FromJust(); + if (state_number >= RD_KAFKA_CONSUMER_GROUP_STATE__CNT) { + continue; + } + returnVec.push_back( + static_cast(state_number)); + } + } + return returnVec; +} + +/** + * @brief Converts a rd_kafka_ListConsumerGroups_result_t* into a v8 object. + */ +v8::Local FromListConsumerGroupsResult( + const rd_kafka_ListConsumerGroups_result_t* result) { + /* Return object type: + { + groups: { + groupId: string, + protocolType: string, + isSimpleConsumerGroup: boolean, + state: ConsumerGroupState (internally a number) + }[], + errors: LibrdKafkaError[] + } + */ + v8::Local returnObject = Nan::New(); + + size_t error_cnt; + const rd_kafka_error_t** error_list = + rd_kafka_ListConsumerGroups_result_errors(result, &error_cnt); + Nan::Set(returnObject, Nan::New("errors").ToLocalChecked(), + Conversion::Util::ToV8Array(error_list, error_cnt)); + + v8::Local groups = Nan::New(); + size_t groups_cnt; + const rd_kafka_ConsumerGroupListing_t** groups_list = + rd_kafka_ListConsumerGroups_result_valid(result, &groups_cnt); + + for (size_t i = 0; i < groups_cnt; i++) { + const rd_kafka_ConsumerGroupListing_t* group = groups_list[i]; + v8::Local groupObject = Nan::New(); + + Nan::Set(groupObject, Nan::New("groupId").ToLocalChecked(), + Nan::New(rd_kafka_ConsumerGroupListing_group_id(group)) + .ToLocalChecked()); + + bool is_simple = + rd_kafka_ConsumerGroupListing_is_simple_consumer_group(group); + Nan::Set(groupObject, Nan::New("isSimpleConsumerGroup").ToLocalChecked(), + Nan::New(is_simple)); + + std::string protocol_type = is_simple ? "simple" : ""; + Nan::Set(groupObject, Nan::New("protocolType").ToLocalChecked(), + Nan::New(protocol_type).ToLocalChecked()); + + Nan::Set(groupObject, Nan::New("state").ToLocalChecked(), + Nan::New(rd_kafka_ConsumerGroupListing_state(group))); + + Nan::Set(groups, i, groupObject); + } + + Nan::Set(returnObject, Nan::New("groups").ToLocalChecked(), groups); + return returnObject; +} + } // namespace Admin } // namespace Conversion diff --git a/src/common.h b/src/common.h index 956223a6..42886a73 100644 --- a/src/common.h +++ b/src/common.h @@ -34,6 +34,8 @@ template<> std::string GetParameter( v8::Local, std::string, std::string); template<> std::vector GetParameter >( v8::Local, std::string, std::vector); +template<> v8::Local GetParameter >( + v8::Local, std::string, v8::Local); // template int GetParameter(v8::Local v8ArrayToStringVector(v8::Local); @@ -90,17 +92,27 @@ class scoped_shared_read_lock { namespace Conversion { -namespace Admin { - // Topics from topic object, or topic object array - rd_kafka_NewTopic_t* FromV8TopicObject( - v8::Local, std::string &errstr); // NOLINT - rd_kafka_NewTopic_t** FromV8TopicObjectArray(v8::Local); -} +namespace Util { +std::vector ToStringVector(v8::Local); +v8::Local ToV8Array(std::vector); +v8::Local ToV8Array(const rd_kafka_error_t **error_list, + size_t error_cnt); +} // namespace Util -namespace Topic { - std::vector ToStringVector(v8::Local); - v8::Local ToV8Array(std::vector); -} // namespace Topic +namespace Admin { +// Topics from topic object, or topic object array +rd_kafka_NewTopic_t *FromV8TopicObject(v8::Local, + std::string &errstr); +rd_kafka_NewTopic_t **FromV8TopicObjectArray(v8::Local); + +// ListGroups: request +std::vector FromV8GroupStateArray( + v8::Local); + +// ListGroups: response +v8::Local FromListConsumerGroupsResult( + const rd_kafka_ListConsumerGroups_result_t *); +} // namespace Admin namespace TopicPartition { diff --git a/src/errors.cc b/src/errors.cc index 18830268..1faa726b 100644 --- a/src/errors.cc +++ b/src/errors.cc @@ -13,8 +13,8 @@ namespace NodeKafka { -v8::Local RdKafkaError(const RdKafka::ErrorCode &err, std::string errstr) { // NOLINT - // +v8::Local RdKafkaError(const RdKafka::ErrorCode &err, + const std::string &errstr) { int code = static_cast(err); v8::Local ret = Nan::New(); @@ -28,7 +28,8 @@ v8::Local RdKafkaError(const RdKafka::ErrorCode &err, std::string er } v8::Local RdKafkaError(const RdKafka::ErrorCode &err) { - return RdKafkaError(err, RdKafka::err2str(err)); + std::string errstr = RdKafka::err2str(err); + return RdKafkaError(err, errstr); } v8::Local RdKafkaError(const RdKafka::ErrorCode &err, std::string errstr, @@ -68,6 +69,26 @@ Baton::Baton(const RdKafka::ErrorCode &code, std::string errstr, bool isFatal, m_isTxnRequiresAbort = isTxnRequiresAbort; } +/** + * Creates a Baton from an rd_kafka_error_t* and destroys it. + */ +Baton Baton::BatonFromErrorAndDestroy(rd_kafka_error_t *error) { + std::string errstr = rd_kafka_error_string(error); + RdKafka::ErrorCode err = + static_cast(rd_kafka_error_code(error)); + rd_kafka_error_destroy(error); + return Baton(err, errstr); +} + +/** + * Creates a Baton from an RdKafka::Error* and deletes it. + */ +Baton Baton::BatonFromErrorAndDestroy(RdKafka::Error *error) { + std::string errstr = error->str(); + RdKafka::ErrorCode err = error->code(); + delete error; + return Baton(err, errstr); +} v8::Local Baton::ToObject() { if (m_errstr.empty()) { diff --git a/src/errors.h b/src/errors.h index fc593022..23acb1fd 100644 --- a/src/errors.h +++ b/src/errors.h @@ -28,6 +28,9 @@ class Baton { explicit Baton(const RdKafka::ErrorCode &, std::string, bool isFatal, bool isRetriable, bool isTxnRequiresAbort); + static Baton BatonFromErrorAndDestroy(rd_kafka_error_t *error); + static Baton BatonFromErrorAndDestroy(RdKafka::Error *error); + template T data() { return static_cast(m_data); } @@ -48,6 +51,8 @@ class Baton { }; v8::Local RdKafkaError(const RdKafka::ErrorCode &); +v8::Local RdKafkaError(const RdKafka::ErrorCode &, + const std::string &); } // namespace NodeKafka diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 1e5cf077..3d519c5d 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -719,7 +719,7 @@ NAN_METHOD(KafkaConsumer::NodeSubscription) { std::vector * topics = b.data*>(); - info.GetReturnValue().Set(Conversion::Topic::ToV8Array(*topics)); + info.GetReturnValue().Set(Conversion::Util::ToV8Array(*topics)); delete topics; } @@ -1079,7 +1079,8 @@ NAN_METHOD(KafkaConsumer::NodeSubscribe) { KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); v8::Local topicsArray = info[0].As(); - std::vector topics = Conversion::Topic::ToStringVector(topicsArray); // NOLINT + std::vector topics = + Conversion::Util::ToStringVector(topicsArray); Baton b = consumer->Subscribe(topics); diff --git a/src/workers.cc b/src/workers.cc index acc5e249..d16eca18 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -1240,5 +1240,60 @@ void AdminClientCreatePartitions::HandleErrorCallback() { callback->Call(argc, argv); } +/** + * @brief List consumer groups in an asynchronous worker. + * + * This callback will list consumer groups. + * + */ +AdminClientListGroups::AdminClientListGroups( + Nan::Callback* callback, AdminClient* client, bool is_match_states_set, + std::vector& match_states, + const int& timeout_ms) + : ErrorAwareWorker(callback), + m_client(client), + m_is_match_states_set(is_match_states_set), + m_match_states(match_states), + m_timeout_ms(timeout_ms) {} + +AdminClientListGroups::~AdminClientListGroups() { + if (this->m_event_response) { + rd_kafka_event_destroy(this->m_event_response); + } +} + +void AdminClientListGroups::Execute() { + Baton b = m_client->ListGroups(m_is_match_states_set, m_match_states, + m_timeout_ms, &m_event_response); + if (b.err() != RdKafka::ERR_NO_ERROR) { + SetErrorBaton(b); + } +} + +void AdminClientListGroups::HandleOKCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 2; + v8::Local argv[argc]; + + argv[0] = Nan::Null(); + + const rd_kafka_ListConsumerGroups_result_t* result = + rd_kafka_event_ListConsumerGroups_result(m_event_response); + + argv[1] = Conversion::Admin::FromListConsumerGroupsResult(result); + + callback->Call(argc, argv); +} + +void AdminClientListGroups::HandleErrorCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 1; + v8::Local argv[argc] = {GetErrorObject()}; + + callback->Call(argc, argv); +} + } // namespace Workers } // namespace NodeKafka diff --git a/src/workers.h b/src/workers.h index 3664d9cf..2844240a 100644 --- a/src/workers.h +++ b/src/workers.h @@ -508,6 +508,28 @@ class AdminClientCreatePartitions : public ErrorAwareWorker { const int m_timeout_ms; }; +/** + * @brief List consumer groups on a remote broker cluster. + */ +class AdminClientListGroups : public ErrorAwareWorker { + public: + AdminClientListGroups(Nan::Callback *, NodeKafka::AdminClient *, bool, + std::vector &, + const int &); + ~AdminClientListGroups(); + + void Execute(); + void HandleOKCallback(); + void HandleErrorCallback(); + + private: + NodeKafka::AdminClient *m_client; + const bool m_is_match_states_set; + std::vector m_match_states; + const int m_timeout_ms; + rd_kafka_event_t *m_event_response; +}; + } // namespace Workers } // namespace NodeKafka diff --git a/test/promisified/admin/list_groups.spec.js b/test/promisified/admin/list_groups.spec.js new file mode 100644 index 00000000..192e203d --- /dev/null +++ b/test/promisified/admin/list_groups.spec.js @@ -0,0 +1,91 @@ +jest.setTimeout(30000); + +const { + createConsumer, + secureRandom, + createTopic, + waitFor, + createAdmin, +} = require('../testhelpers'); +const { ConsumerGroupStates, ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Admin > listGroups', () => { + let topicName, groupId, consumer, admin; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; + + consumer = createConsumer({ + groupId, + fromBeginning: true, + }); + + await createTopic({ topic: topicName, partitions: 2 }); + + admin = createAdmin({}); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()); + admin && (await admin.disconnect()); + }); + + it('should timeout', async () => { + await admin.connect(); + + await expect(admin.listGroups({ timeout: 0 })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__TIMED_OUT + ); + }); + + it('should list consumer groups', async () => { + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + await consumer.run({ eachMessage: async () => {} }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + await admin.connect(); + let listGroupsResult = await admin.listGroups(); + expect(listGroupsResult.errors).toEqual([]); + expect(listGroupsResult.groups).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + groupId, + isSimpleConsumerGroup: false, + protocolType: '', + state: ConsumerGroupStates.STABLE, + }), + ]) + ); + + // Disconnect the consumer to make the group EMPTY. + await consumer.disconnect(); + consumer = null; + + listGroupsResult = await admin.listGroups(); + expect(listGroupsResult.errors).toEqual([]); + expect(listGroupsResult.groups).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + groupId, + isSimpleConsumerGroup: false, + protocolType: '', + state: ConsumerGroupStates.EMPTY, + }), + ]) + ); + + // Consumer group should not show up if filtering by non-empty groups + // using state matching. + listGroupsResult = await admin.listGroups({ + matchConsumerGroupStates: [ ConsumerGroupStates.STABLE, + ConsumerGroupStates.PREPARING_REBALANCE, + ConsumerGroupStates.COMPLETING_REBALANCE, ] }); + expect(listGroupsResult.errors).toEqual([]); + expect(listGroupsResult.groups.map(group => group.groupId)).not.toContain(groupId); + }); +}); + diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 767d92f5..3bbd2d6e 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,5 +1,10 @@ import * as tls from 'tls' import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig, TopicConfig } from './config' +import { ConsumerGroupStates, GroupOverview, LibrdKafkaError } from '../index' + +// Admin API related interfaces, types etc; and Error types are common, so +// just re-export them from here too. +export { ConsumerGroupStates, GroupOverview, LibrdKafkaError } from '../index' export type BrokersFunction = () => string[] | Promise @@ -180,14 +185,6 @@ export interface AdminConstructorConfig extends GlobalConfig { kafkaJS?: AdminConfig; } -export interface ITopicConfig { - topic: string - numPartitions?: number - replicationFactor?: number - replicaAssignment?: ReplicaAssignment[] - configEntries?: IResourceConfigEntry[] -} - export interface ReplicaAssignment { partition: number replicas: Array @@ -444,20 +441,12 @@ export type MemberDescription = { memberMetadata: Buffer } -export type ConsumerGroupState = - | 'Unknown' - | 'PreparingRebalance' - | 'CompletingRebalance' - | 'Stable' - | 'Dead' - | 'Empty' - export type GroupDescription = { groupId: string members: MemberDescription[] protocol: string protocolType: string - state: ConsumerGroupState + state: ConsumerGroupStates } export type Consumer = { @@ -545,3 +534,32 @@ export type Consumer = { readonly events: ConsumerEvents } +export interface ITopicConfig { + topic: string + numPartitions?: number + replicationFactor?: number + replicaAssignment?: ReplicaAssignment[] + configEntries?: IResourceConfigEntry[] +} + +export interface ITopicPartitionConfig { + topic: string + count: number + assignments?: Array> +} + +export type Admin = { + connect(): Promise + disconnect(): Promise + createTopics(options: { + validateOnly?: boolean + waitForLeaders?: boolean + timeout?: number + topics: ITopicConfig[] + }): Promise + deleteTopics(options: { topics: string[]; timeout?: number }): Promise + listGroups(options?: { + timeout?: number, + matchConsumerGroupStates?: ConsumerGroupStates[] + }): Promise<{ groups: GroupOverview[], errors: LibrdKafkaError[] }> +} From 9e3aa359d0cb2aa7af76bc816baa533493e87bc9 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Mar 2024 13:52:22 +0530 Subject: [PATCH 102/224] Update package-lock.json --- package-lock.json | 172 +++++++++++++++++++++------------------------- 1 file changed, 79 insertions(+), 93 deletions(-) diff --git a/package-lock.json b/package-lock.json index 4432d592..8f8473fd 100644 --- a/package-lock.json +++ b/package-lock.json @@ -39,13 +39,13 @@ } }, "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==", + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.3.0.tgz", + "integrity": "sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw==", "dev": true, "dependencies": { - "@jridgewell/gen-mapping": "^0.3.0", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.24" }, "engines": { "node": ">=6.0.0" @@ -145,9 +145,9 @@ } }, "node_modules/@babel/core": { - "version": "7.23.9", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.9.tgz", - "integrity": "sha512-5q0175NOjddqpvvzU+kDiSOAk4PfdO6FvwCWoQ6RO7rTzEe8vlo+4HVfcnAREhD4npMs0e9uZypjTwzZPCf/cw==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.24.0.tgz", + "integrity": "sha512-fQfkg0Gjkza3nf0c7/w6Xf34BW4YvzNfACRLmmb7XRLa6XHdR+K9AlJlxneFfWYf6uhOzuzZVTjF/8KfndZANw==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", @@ -155,11 +155,11 @@ "@babel/generator": "^7.23.6", "@babel/helper-compilation-targets": "^7.23.6", "@babel/helper-module-transforms": "^7.23.3", - "@babel/helpers": "^7.23.9", - "@babel/parser": "^7.23.9", - "@babel/template": "^7.23.9", - "@babel/traverse": "^7.23.9", - "@babel/types": "^7.23.9", + "@babel/helpers": "^7.24.0", + "@babel/parser": "^7.24.0", + "@babel/template": "^7.24.0", + "@babel/traverse": "^7.24.0", + "@babel/types": "^7.24.0", "convert-source-map": "^2.0.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -289,9 +289,9 @@ } }, "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==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.0.tgz", + "integrity": "sha512-9cUznXMG0+FxRuJfvL82QlTqIzhVW9sL0KjMPHhAOOvpQGL8QtdxnBKILjBqxlHyliz0yCa1G903ZXI/FuHy2w==", "dev": true, "engines": { "node": ">=6.9.0" @@ -349,14 +349,14 @@ } }, "node_modules/@babel/helpers": { - "version": "7.23.9", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.9.tgz", - "integrity": "sha512-87ICKgU5t5SzOT7sBMfCOZQ2rHjRU+Pcb9BoILMYz600W6DkVRLFBPwQ18gwUVvggqXivaUakpnxWQGbpywbBQ==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.24.0.tgz", + "integrity": "sha512-ulDZdc0Aj5uLc5nETsa7EPx2L7rM0YJM8r7ck7U73AXi7qOV44IHHRAYZHY6iU1rr3C5N4NtTmMRUJP6kwCWeA==", "dev": true, "dependencies": { - "@babel/template": "^7.23.9", - "@babel/traverse": "^7.23.9", - "@babel/types": "^7.23.9" + "@babel/template": "^7.24.0", + "@babel/traverse": "^7.24.0", + "@babel/types": "^7.24.0" }, "engines": { "node": ">=6.9.0" @@ -448,9 +448,9 @@ } }, "node_modules/@babel/parser": { - "version": "7.23.9", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.9.tgz", - "integrity": "sha512-9tcKgqKbs3xGJ+NtKF2ndOBBLVwPjl1SHxPQkd36r3Dlirw3xWUeGaTbqr7uGZcTaxkVNwc+03SVP7aCdWrTlA==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.24.0.tgz", + "integrity": "sha512-QuP/FxEAzMSjXygs8v4N9dvdXzEHN4W1oF3PxuWAtPo08UdM17u89RDMgjLn/mlc56iM0HlLmVkO/wgR+rDgHg==", "dev": true, "bin": { "parser": "bin/babel-parser.js" @@ -637,23 +637,23 @@ } }, "node_modules/@babel/template": { - "version": "7.23.9", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.23.9.tgz", - "integrity": "sha512-+xrD2BWLpvHKNmX2QbpdpsBaWnRxahMwJjO+KZk2JOElj5nSmKezyS1B4u+QbHMTX69t4ukm6hh9lsYQ7GHCKA==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.24.0.tgz", + "integrity": "sha512-Bkf2q8lMB0AFpX0NFEqSbx1OkTHf0f+0j82mkw+ZpzBnkk7e9Ql0891vlfgi+kHwOk8tQjiQHpqh4LaSa0fKEA==", "dev": true, "dependencies": { "@babel/code-frame": "^7.23.5", - "@babel/parser": "^7.23.9", - "@babel/types": "^7.23.9" + "@babel/parser": "^7.24.0", + "@babel/types": "^7.24.0" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/traverse": { - "version": "7.23.9", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.9.tgz", - "integrity": "sha512-I/4UJ9vs90OkBtY6iiiTORVMyIhJ4kAVmsKo9KFc8UOxMeUfi2hvtIBsET5u9GizXE6/GFSuKCTNfgCswuEjRg==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.24.0.tgz", + "integrity": "sha512-HfuJlI8qq3dEDmNU5ChzzpZRWq+oxCZQyMzIMEqLho+AQnhMnKQUzH6ydo3RBl/YjPCuk68Y6s0Gx0AeyULiWw==", "dev": true, "dependencies": { "@babel/code-frame": "^7.23.5", @@ -662,8 +662,8 @@ "@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.9", - "@babel/types": "^7.23.9", + "@babel/parser": "^7.24.0", + "@babel/types": "^7.24.0", "debug": "^4.3.1", "globals": "^11.1.0" }, @@ -681,9 +681,9 @@ } }, "node_modules/@babel/types": { - "version": "7.23.9", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.23.9.tgz", - "integrity": "sha512-dQjSq/7HaSjRM43FFGnv5keM2HsxpmyV1PfaSVm0nzzjwwTmjOe6J4bC8e3+pTEIgHaHj+1ZlLThRJ2auc/w1Q==", + "version": "7.24.0", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.24.0.tgz", + "integrity": "sha512-+j7a5c253RfKh8iABBhywc8NSfP5LURe7Uh4qpsh6jc+aLJguvmIUBdjSdEMQv2bENrCR5MfRdjGo7vzS/ob7w==", "dev": true, "dependencies": { "@babel/helper-string-parser": "^7.23.4", @@ -1188,14 +1188,14 @@ } }, "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==", + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", + "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", "dev": true, "dependencies": { - "@jridgewell/set-array": "^1.0.1", + "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/trace-mapping": "^0.3.24" }, "engines": { "node": ">=6.0.0" @@ -1211,9 +1211,9 @@ } }, "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==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", + "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", "dev": true, "engines": { "node": ">=6.0.0" @@ -1226,9 +1226,9 @@ "dev": true }, "node_modules/@jridgewell/trace-mapping": { - "version": "0.3.22", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.22.tgz", - "integrity": "sha512-Wf963MzWtA2sjrNt+g18IAln9lKnlRp+K2eH4jjIoF1wYeq3aMREpG09xhlhdzS0EjwU7qmUJYangWa+151vZw==", + "version": "0.3.25", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", + "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", "dev": true, "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -1458,9 +1458,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.11.19", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.19.tgz", - "integrity": "sha512-7xMnVEcZFu0DikYjWOlRq7NTPETrm7teqUT2WkQjrTIkEgUyyGdWsj/Zg8bEJt5TNklzbPD1X3fqfsHw3SpapQ==", + "version": "20.11.25", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.25.tgz", + "integrity": "sha512-TBHyJxk2b7HceLVGFcpAUjsa5zIdsPWlR6XHfyGzd0SFu+/NFgQgMAl96MSDZgQDvJAvV6BKsFOrt6zIL09JDw==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -2002,9 +2002,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001587", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001587.tgz", - "integrity": "sha512-HMFNotUmLXn71BQxg8cijvqxnIAofforZOwGsxyXJ0qugTdspUF4sPSJ2vhgprHCB996tIDzEq1ubumPDV8ULA==", + "version": "1.0.30001596", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001596.tgz", + "integrity": "sha512-zpkZ+kEr6We7w63ORkoJ2pOfBwBkY/bJrG/UZ90qNb45Isblu8wzDgevEOrRL1r9dWayHjYiiyCMEXPn4DweGQ==", "dev": true, "funding": [ { @@ -2442,9 +2442,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.672", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.672.tgz", - "integrity": "sha512-YYCy+goe3UqZqa3MOQCI5Mx/6HdBLzXL/mkbGCEWL3sP3Z1BP9zqAzeD3YEmLZlespYGFtyM8tRp5i2vfaUGCA==", + "version": "1.4.698", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.698.tgz", + "integrity": "sha512-f9iZD1t3CLy1AS6vzM5EKGa6p9pRcOeEFXRFbaG2Ta+Oe7MkfRQ3fsvPYidzHe1h4i0JvIvpcY55C+B6BZNGtQ==", "dev": true }, "node_modules/emittery": { @@ -2838,9 +2838,9 @@ } }, "node_modules/flatted": { - "version": "3.2.9", - "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.2.9.tgz", - "integrity": "sha512-36yxDn5H7OFZQla0/jFJmbIKTdZAQHngCedGxiMmpNfEZM0sdEeT+WczLQrjK6D7o2aiyLYDnkw0R3JK0Qv1RQ==", + "version": "3.3.1", + "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.3.1.tgz", + "integrity": "sha512-X8cqMLLie7KsNUDSdzeN8FYK9rEt4Dt67OsG/DNGnYTSDBG4uFAJFBnUeiV+zCVAvwFy56IjM9sH51jVaEhNxw==", "dev": true }, "node_modules/fs-minipass": { @@ -2859,20 +2859,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", @@ -3356,14 +3342,14 @@ } }, "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==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.2.tgz", + "integrity": "sha512-1WUsZ9R1lA0HtBSohTkm39WTPlNKSJ5iFk7UwqXkBLoHQT+hfqPsfsTDVuZdKGaBwn7din9bS7SsnoAr943hvw==", "dev": true, "dependencies": { - "@babel/core": "^7.12.3", - "@babel/parser": "^7.14.7", - "@istanbuljs/schema": "^0.1.2", + "@babel/core": "^7.23.9", + "@babel/parser": "^7.23.9", + "@istanbuljs/schema": "^0.1.3", "istanbul-lib-coverage": "^3.2.0", "semver": "^7.5.4" }, @@ -3415,9 +3401,9 @@ } }, "node_modules/istanbul-reports": { - "version": "3.1.6", - "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-3.1.6.tgz", - "integrity": "sha512-TLgnMkKg3iTDsQ9PbPTdpfAK2DzjF9mqUG7RMgcQl8oFjad8ob4laGxv5XV5U9MAfx8D6tSJiUyuAwzLicaxlg==", + "version": "3.1.7", + "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-3.1.7.tgz", + "integrity": "sha512-BewmUXImeuRk2YY0PVbxgKAysvhRPUQE0h5QRM++nVWyubKGV0l8qQ5op8+B2DOmwSe63Jivj0BjkPQVf8fP5g==", "dev": true, "dependencies": { "html-escaper": "^2.0.0", @@ -4658,9 +4644,9 @@ "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", - "integrity": "sha512-W7tfG7vMOGtD30sHoZSSc/JVYiyDPEyQVso/Zz+/uQd0B0L46gtC+pHha5FFMRpil6fm/AoEcRWyOVi4+E/f8w==" + "version": "2.19.0", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.19.0.tgz", + "integrity": "sha512-nO1xXxfh/RWNxfd/XPfbIfFk5vgLsAxUR9y5O0cHMJu/AW9U95JLXqthYHjEp+8gQ5p96K9jUp8nbVOxCdRbtw==" }, "node_modules/natural-compare": { "version": "1.4.0", @@ -5521,9 +5507,9 @@ } }, "node_modules/socks": { - "version": "2.7.3", - "resolved": "https://registry.npmjs.org/socks/-/socks-2.7.3.tgz", - "integrity": "sha512-vfuYK48HXCTFD03G/1/zkIls3Ebr2YNa4qU9gHDZdblHLiqhJrJGkY3+0Nx0JpN9qBhJbVObc1CNciT1bIZJxw==", + "version": "2.8.1", + "resolved": "https://registry.npmjs.org/socks/-/socks-2.8.1.tgz", + "integrity": "sha512-B6w7tkwNid7ToxjZ08rQMT8M9BJAf8DKx8Ft4NivzH0zBUfd6jldGcisJn/RLgxcX3FPNDdNQCUEMMT79b+oCQ==", "dev": true, "dependencies": { "ip-address": "^9.0.5", @@ -5818,9 +5804,9 @@ } }, "node_modules/typescript": { - "version": "5.3.3", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.3.3.tgz", - "integrity": "sha512-pXWcraxM0uxAS+tN0AG/BF2TyqmHO014Z070UsJ+pFvYuRSq8KH8DmWpnbXe0pEPDHXZV3FcAbJkijJ5oNEnWw==", + "version": "5.4.2", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.2.tgz", + "integrity": "sha512-+2/g0Fds1ERlP6JsakQQDXjZdZMM+rqpamFZJEKh4kwTIn3iDkgKtby0CeNd5ATNZ4Ry1ax15TMx0W2V+miizQ==", "dev": true, "bin": { "tsc": "bin/tsc", From 1777f4da2d7c82615b0823669867ec8f008174f2 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 15 Mar 2024 13:54:56 +0530 Subject: [PATCH 103/224] Update copyright headers --- lib/admin.js | 2 +- src/admin.h | 1 + src/common.cc | 1 + src/common.h | 1 + src/errors.cc | 1 + src/errors.h | 1 + src/workers.cc | 1 + 7 files changed, 7 insertions(+), 1 deletion(-) diff --git a/lib/admin.js b/lib/admin.js index df7d7c95..d7273f1d 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -2,7 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment - * 2024 Confluent, Inc + * (c) 2024 Confluent, Inc * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/admin.h b/src/admin.h index 6e4ebd8e..1b2258d6 100644 --- a/src/admin.h +++ b/src/admin.h @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/common.cc b/src/common.cc index 6e4d928c..a3d61042 100644 --- a/src/common.cc +++ b/src/common.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/common.h b/src/common.h index 42886a73..0359bc23 100644 --- a/src/common.h +++ b/src/common.h @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/errors.cc b/src/errors.cc index 1faa726b..f708f79c 100644 --- a/src/errors.cc +++ b/src/errors.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/errors.h b/src/errors.h index 23acb1fd..2b4f3464 100644 --- a/src/errors.h +++ b/src/errors.h @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. diff --git a/src/workers.cc b/src/workers.cc index d16eca18..f714f280 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. From 3ea556cd6c60f3bd14f583dd861fc4973f6e7fd5 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 22 Mar 2024 11:19:32 +0530 Subject: [PATCH 104/224] Add AdminAPI for describeGroups --- index.d.ts | 58 +++- lib/admin.js | 61 +++- lib/kafkajs/_admin.js | 33 +- lib/kafkajs/_kafka.js | 5 +- src/admin.cc | 130 +++++++- src/admin.h | 4 + src/common.cc | 297 +++++++++++++++++- src/common.h | 17 +- src/workers.cc | 52 +++ src/workers.h | 21 ++ .../promisified/admin/describe_groups.spec.js | 136 ++++++++ test/promisified/admin/list_groups.spec.js | 4 +- types/kafkajs.d.ts | 24 +- 13 files changed, 795 insertions(+), 47 deletions(-) create mode 100644 test/promisified/admin/describe_groups.spec.js diff --git a/index.d.ts b/index.d.ts index 5ad6dd9e..eff5f4fb 100644 --- a/index.d.ts +++ b/index.d.ts @@ -72,6 +72,7 @@ export interface WatermarkOffsets{ export interface TopicPartition { topic: string; partition: number; + error?: LibrdKafkaError; } export interface TopicPartitionOffset extends TopicPartition{ @@ -349,6 +350,56 @@ export interface GroupOverview { state: ConsumerGroupStates; } +export enum AclOperationTypes { + UNKNOWN = 0, + ANY = 1, + ALL = 2, + READ = 3, + WRITE = 4, + CREATE = 5, + DELETE = 6, + ALTER = 7, + DESCRIBE = 8, + CLUSTER_ACTION = 9, + DESCRIBE_CONFIGS = 10, + ALTER_CONFIGS = 11, + IDEMPOTENT_WRITE = 12, +} + +export type MemberDescription = { + clientHost: string + clientId: string + memberId: string + memberAssignment: Buffer + memberMetadata: Buffer + groupInstanceId?: string, + assignment: TopicPartition[] +} + +export type Node = { + id: number + host: string + port: number + rack?: string +} + +export type GroupDescription = { + groupId: string + error?: LibrdKafkaError + members: MemberDescription[] + protocol: string + isSimpleConsumerGroup: boolean; + protocolType: string + partitionAssignor: string + state: ConsumerGroupStates + coordinator: Node + authorizedOperations?: AclOperationTypes[] +} + +export type GroupDescriptions = { + groups: GroupDescription[], +} + export interface IAdminClient { createTopic(topic: NewTopic, cb?: (err: LibrdKafkaError) => void): void; createTopic(topic: NewTopic, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; @@ -360,9 +411,14 @@ export interface IAdminClient { createPartitions(topic: string, desiredPartitions: number, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; listGroups(cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; - listGroups(options: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }, + listGroups(options?: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }, cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; + describeGroups(groupIds: string[], cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; + describeGroups(groupIds: string[], + options?: { timeout?: number, includeAuthorizedOperations?: boolean }, + cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; + disconnect(): void; } diff --git a/lib/admin.js b/lib/admin.js index d7273f1d..717dc450 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -20,12 +20,28 @@ const ConsumerGroupStates = Object.seal({ STABLE: 3, DEAD: 4, EMPTY: 5, -}) +}); +const AclOperationTypes = Object.seal({ + UNKNOWN: 0, + ANY: 1, + ALL: 2, + READ: 3, + WRITE: 4, + CREATE: 5, + DELETE: 6, + ALTER: 7, + DESCRIBE: 8, + CLUSTER_ACTION: 9, + DESCRIBE_CONFIGS: 10, + ALTER_CONFIGS: 11, + IDEMPOTENT_WRITE: 12, +}); module.exports = { create: createAdminClient, ConsumerGroupStates, + AclOperationTypes, }; var Client = require('./client'); @@ -267,4 +283,45 @@ AdminClient.prototype.listGroups = function (options, cb) { cb(null, groups); } }); -} \ No newline at end of file +} + +/** + * Describe consumer groups. + * @param {string[]} groups - The names of the groups to describe. + * @param {any?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @param {boolean?} options.includeAuthorizedOperations - If true, include operations allowed on the group by the calling client (default: false). + * @param {function} cb - The callback to be executed when finished. + * + * Valid ways to call this function: + * describeGroups(groups, cb) + * describeGroups(groups, options, cb) + */ +AdminClient.prototype.describeGroups = function (groups, options, cb) { + if (!this._isConnected) { + throw new Error('Client is disconnected'); + } + + if (typeof options === 'function') { + cb = options; + options = {}; + } + + if (!options) { + options = {}; + } + + this._client.describeGroups(groups, options, function (err, descriptions) { + if (err) { + if (cb) { + cb(LibrdKafkaError.create(err)); + } + return; + } + + if (cb) { + cb(null, descriptions); + } + }); +} diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 45d1e95b..bf2a5a2d 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -106,7 +106,7 @@ class Admin { this.#state = AdminState.CONNECTING; - const config = this.#config(); + const config = this.#config(); return new Promise((resolve, reject) => { try { @@ -258,6 +258,35 @@ class Admin { }); } + /** + * Describe consumer groups. + * + * @param {string[]} groups - The names of the groups to describe. + * @param {object?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @param {boolean?} options.includeAuthorizedOperations - If true, include operations allowed on the group by the calling client (default: false). + * @returns {Promise} + */ + async describeGroups(groups, options = {}) { + if (this.#state !== AdminState.CONNECTED) { + throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE }); + } + + return new Promise((resolve, reject) => { + this.#internalClient.describeGroups(groups, options, (err, descriptions) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } else { + resolve(descriptions); + } + }); + }); + } } -module.exports = { Admin, ConsumerGroupStates: RdKafka.AdminClient.ConsumerGroupStates } +module.exports = { + Admin, + ConsumerGroupStates: RdKafka.AdminClient.ConsumerGroupStates, + AclOperationTypes: RdKafka.AdminClient.AclOperationTypes +} diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 51b0866f..49a97d17 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,6 +1,6 @@ const { Producer, CompressionTypes } = require('./_producer'); const { Consumer, PartitionAssigners } = require('./_consumer'); -const { Admin, ConsumerGroupStates } = require('./_admin'); +const { Admin, ConsumerGroupStates, AclOperationTypes } = require('./_admin'); const error = require('./_error'); const { logLevel, checkIfKafkaJsKeysPresent, CompatibilityErrorMessages } = require('./_common'); @@ -86,4 +86,5 @@ module.exports = { PartitionAssigners, PartitionAssignors: PartitionAssigners, CompressionTypes, - ConsumerGroupStates }; + ConsumerGroupStates, + AclOperationTypes }; diff --git a/src/admin.cc b/src/admin.cc index 0251b17f..de57485b 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -89,6 +89,7 @@ void AdminClient::Init(v8::Local exports) { // Consumer group related operations Nan::SetPrototypeMethod(tpl, "listGroups", NodeListGroups); + Nan::SetPrototypeMethod(tpl, "describeGroups", NodeDescribeGroups); Nan::SetPrototypeMethod(tpl, "connect", NodeConnect); Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect); @@ -455,18 +456,88 @@ Baton AdminClient::ListGroups( } // Create queue just for this operation. - rd_kafka_queue_t *topic_rkqu = rd_kafka_queue_new(m_client->c_ptr()); + rd_kafka_queue_t *rkqu = rd_kafka_queue_new(m_client->c_ptr()); - rd_kafka_ListConsumerGroups(m_client->c_ptr(), options, topic_rkqu); + rd_kafka_ListConsumerGroups(m_client->c_ptr(), options, rkqu); // Poll for an event by type in that queue // DON'T destroy the event. It is the out parameter, and ownership is // the caller's. *event_response = PollForEvent( - topic_rkqu, RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT, timeout_ms); + rkqu, RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT, timeout_ms); // Destroy the queue since we are done with it. - rd_kafka_queue_destroy(topic_rkqu); + rd_kafka_queue_destroy(rkqu); + + // Destroy the options we just made because we polled already + rd_kafka_AdminOptions_destroy(options); + + // If we got no response from that operation, this is a failure + // likely due to time out + if (*event_response == NULL) { + return Baton(RdKafka::ERR__TIMED_OUT); + } + + // Now we can get the error code from the event + if (rd_kafka_event_error(*event_response)) { + // If we had a special error code, get out of here with it + const rd_kafka_resp_err_t errcode = rd_kafka_event_error(*event_response); + return Baton(static_cast(errcode)); + } + + // At this point, event_response contains the result, which needs + // to be parsed/converted by the caller. + return Baton(RdKafka::ERR_NO_ERROR); + } +} + +Baton AdminClient::DescribeGroups(std::vector &groups, + bool include_authorized_operations, + int timeout_ms, + /* out */ rd_kafka_event_t **event_response) { + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + { + scoped_shared_write_lock lock(m_connection_lock); + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + // Make admin options to establish that we are describing groups + rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new( + m_client->c_ptr(), RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + if (include_authorized_operations) { + rd_kafka_error_t *error = + rd_kafka_AdminOptions_set_include_authorized_operations( + options, include_authorized_operations); + if (error) { + return Baton::BatonFromErrorAndDestroy(error); + } + } + + // Create queue just for this operation. + rd_kafka_queue_t *rkqu = rd_kafka_queue_new(m_client->c_ptr()); + + // Construct a char** to pass to librdkafka. Avoid too many allocations. + std::vector c_groups(groups.size()); + for (size_t i = 0; i < groups.size(); i++) { + c_groups[i] = groups[i].c_str(); + } + + rd_kafka_DescribeConsumerGroups(m_client->c_ptr(), &c_groups[0], + groups.size(), options, rkqu); + + // Poll for an event by type in that queue + // DON'T destroy the event. It is the out parameter, and ownership is + // the caller's. + *event_response = PollForEvent( + rkqu, RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, timeout_ms); + + // Destroy the queue since we are done with it. + rd_kafka_queue_destroy(rkqu); // Destroy the options we just made because we polled already rd_kafka_AdminOptions_destroy(options); @@ -696,10 +767,10 @@ NAN_METHOD(AdminClient::NodeListGroups) { // Get the match states, or not if they are unset. std::vector match_states; - v8::Local matchConsumerGroupStatesKey = + v8::Local match_consumer_group_states_key = Nan::New("matchConsumerGroupStates").ToLocalChecked(); bool is_match_states_set = - Nan::Has(config, matchConsumerGroupStatesKey).FromMaybe(false); + Nan::Has(config, match_consumer_group_states_key).FromMaybe(false); v8::Local match_states_array; if (is_match_states_set) { @@ -713,4 +784,51 @@ NAN_METHOD(AdminClient::NodeListGroups) { callback, client, is_match_states_set, match_states, timeout_ms)); } +/** + * Describe Consumer Groups. + */ +NAN_METHOD(AdminClient::NodeDescribeGroups) { + Nan::HandleScope scope; + + if (info.Length() < 3 || !info[2]->IsFunction()) { + // Just throw an exception + return Nan::ThrowError("Need to specify a callback"); + } + + if (!info[0]->IsArray()) { + return Nan::ThrowError("Must provide group name array"); + } + + if (!info[1]->IsObject()) { + return Nan::ThrowError("Must provide options object"); + } + + // Get list of group names to describe. + v8::Local group_names = info[0].As(); + if (group_names->Length() == 0) { + return Nan::ThrowError("Must provide at least one group name"); + } + std::vector group_names_vector = + v8ArrayToStringVector(group_names); + + v8::Local config = info[1].As(); + + // Get the timeout - default 5000. + int timeout_ms = GetParameter(config, "timeout", 5000); + + // Get whether to include authorized operations - default false. + bool include_authorized_operations = + GetParameter(config, "includeAuthorizedOperations", false); + + // Create the final callback object + v8::Local cb = info[2].As(); + Nan::Callback *callback = new Nan::Callback(cb); + AdminClient *client = ObjectWrap::Unwrap(info.This()); + + // Queue the work. + Nan::AsyncQueueWorker(new Workers::AdminClientDescribeGroups( + callback, client, group_names_vector, include_authorized_operations, + timeout_ms)); +} + } // namespace NodeKafka diff --git a/src/admin.h b/src/admin.h index 1b2258d6..2f6a982f 100644 --- a/src/admin.h +++ b/src/admin.h @@ -56,6 +56,9 @@ class AdminClient : public Connection { std::vector& match_states, int timeout_ms, rd_kafka_event_t** event_response); + Baton DescribeGroups(std::vector& groups, + bool include_authorized_operations, int timeout_ms, + rd_kafka_event_t** event_response); protected: static Nan::Persistent constructor; @@ -75,6 +78,7 @@ class AdminClient : public Connection { // Consumer group operations static NAN_METHOD(NodeListGroups); + static NAN_METHOD(NodeDescribeGroups); static NAN_METHOD(NodeConnect); static NAN_METHOD(NodeDisconnect); diff --git a/src/common.cc b/src/common.cc index a3d61042..0bdc149f 100644 --- a/src/common.cc +++ b/src/common.cc @@ -225,7 +225,52 @@ v8::Local ToV8Array(const rd_kafka_error_t** error_list, return errors; } -} // namespace Util +/** + * @brief Converts a rd_kafka_Node_t* into a v8 object. + */ +v8::Local ToV8Object(const rd_kafka_Node_t* node) { + /* Return object type + { + id: number + host: string + port: number + rack?: string + } + */ + v8::Local obj = Nan::New(); + + Nan::Set(obj, Nan::New("id").ToLocalChecked(), + Nan::New(rd_kafka_Node_id(node))); + Nan::Set(obj, Nan::New("host").ToLocalChecked(), + Nan::New(rd_kafka_Node_host(node)).ToLocalChecked()); + Nan::Set(obj, Nan::New("port").ToLocalChecked(), + Nan::New(rd_kafka_Node_port(node))); + + const char* rack = rd_kafka_Node_rack(node); + if (rack) { + Nan::Set(obj, Nan::New("rack").ToLocalChecked(), + Nan::New(rack).ToLocalChecked()); + } + + return obj; +} + +/** + * @brief Converts a list of rd_kafka_AclOperation_t into a v8 array. + */ +v8::Local ToV8Array( + const rd_kafka_AclOperation_t* authorized_operations, + size_t authorized_operations_cnt) { + v8::Local array = Nan::New(); + + for (size_t i = 0; i < authorized_operations_cnt; i++) { + Nan::Set(array, i, Nan::New(authorized_operations[i])); + } + + return array; +} + +} // namespace Util namespace TopicPartition { @@ -233,6 +278,10 @@ namespace TopicPartition { * @brief RdKafka::TopicPartition vector to a v8 Array * * @see v8ArrayToTopicPartitionVector + * @note This method returns a v8 array of a mix of topic partition + * objects and errors. For a more uniform return type of + * topic partitions (which have an internal error property), + * use `ToTopicPartitionV8Array(const rd_kafka_topic_partition_list_t*, bool)`. */ v8::Local ToV8Array( std::vector & topic_partition_list) { // NOLINT @@ -242,6 +291,8 @@ v8::Local ToV8Array( RdKafka::TopicPartition* topic_partition = topic_partition_list[topic_partition_i]; + // TODO: why do we set the entire array element to be an error rather adding + // an error field to TopicPartition? Or create a TopicPartitionError? if (topic_partition->err() != RdKafka::ErrorCode::ERR_NO_ERROR) { Nan::Set(array, topic_partition_i, Nan::Error(Nan::New(RdKafka::err2str(topic_partition->err())) @@ -267,7 +318,49 @@ v8::Local ToV8Array( return array; } +/** + * @brief Converts a rd_kafka_topic_partition_list_t* into a list of v8 objects. + * + * @param topic_partition_list The list of topic partitions to convert. + * @param include_offset Whether to include the offset in the output. + * @returns [{topic: string, partition: number, offset?: number, error?: + * LibrdKafkaError}] + * + * @note Contains error within the topic partitions object, and not as separate + * array elements, unlike the `ToV8Array(std::vector & + * topic_partition_list)`. + */ +v8::Local ToTopicPartitionV8Array( + const rd_kafka_topic_partition_list_t* topic_partition_list, + bool include_offset) { + v8::Local array = Nan::New(); + for (int topic_partition_i = 0; topic_partition_i < topic_partition_list->cnt; + topic_partition_i++) { + rd_kafka_topic_partition_t topic_partition = + topic_partition_list->elems[topic_partition_i]; + v8::Local obj = Nan::New(); + + Nan::Set(obj, Nan::New("partition").ToLocalChecked(), + Nan::New(topic_partition.partition)); + Nan::Set(obj, Nan::New("topic").ToLocalChecked(), + Nan::New(topic_partition.topic).ToLocalChecked()); + + if (topic_partition.err != RD_KAFKA_RESP_ERR_NO_ERROR) { + v8::Local error = NodeKafka::RdKafkaError( + static_cast(topic_partition.err)); + Nan::Set(obj, Nan::New("error").ToLocalChecked(), error); + } + + if (include_offset) { + Nan::Set(obj, Nan::New("offset").ToLocalChecked(), + Nan::New(topic_partition.offset)); + } + + Nan::Set(array, topic_partition_i, obj); + } + return array; +} /** * @brief v8 Array of topic partitions to RdKafka::TopicPartition vector @@ -530,21 +623,17 @@ rd_kafka_NewTopic_t* FromV8TopicObject( int num_partitions = GetParameter(object, "num_partitions", 0); int replication_factor = GetParameter(object, "replication_factor", 0); - // Too slow to allocate this every call but admin api - // shouldn't be called that often - char* errbuf = reinterpret_cast(malloc(100)); - size_t errstr_size = 100; + char errbuf[512]; rd_kafka_NewTopic_t* new_topic = rd_kafka_NewTopic_new( topic_name.c_str(), num_partitions, replication_factor, errbuf, - errstr_size); + sizeof(errbuf)); if (new_topic == NULL) { - errstr = std::string(errbuf, errstr_size); - free(errbuf); + errstr = std::string(errbuf); return NULL; } @@ -592,8 +681,6 @@ rd_kafka_NewTopic_t* FromV8TopicObject( } } - // Free it again cuz we malloc'd it. - // free(errbuf); return new_topic; } @@ -672,7 +759,7 @@ v8::Local FromListConsumerGroupsResult( Nan::Set(groupObject, Nan::New("isSimpleConsumerGroup").ToLocalChecked(), Nan::New(is_simple)); - std::string protocol_type = is_simple ? "simple" : ""; + std::string protocol_type = is_simple ? "simple" : "consumer"; Nan::Set(groupObject, Nan::New("protocolType").ToLocalChecked(), Nan::New(protocol_type).ToLocalChecked()); @@ -686,6 +773,194 @@ v8::Local FromListConsumerGroupsResult( return returnObject; } +/** + * @brief Converts a rd_kafka_MemberDescription_t* into a v8 object. + */ +v8::Local FromMemberDescription( + const rd_kafka_MemberDescription_t* member) { + /* Return object type: + { + clientHost: string + clientId: string + memberId: string + memberAssignment: Buffer // will be always null + memberMetadata: Buffer // will be always null + groupInstanceId: string + assignment: { + topicPartitions: TopicPartition[] + }, + } + */ + v8::Local returnObject = Nan::New(); + + // clientHost + Nan::Set(returnObject, Nan::New("clientHost").ToLocalChecked(), + Nan::New(rd_kafka_MemberDescription_host(member)) + .ToLocalChecked()); + + // clientId + Nan::Set(returnObject, Nan::New("clientId").ToLocalChecked(), + Nan::New(rd_kafka_MemberDescription_client_id(member)) + .ToLocalChecked()); + + // memberId + Nan::Set(returnObject, Nan::New("memberId").ToLocalChecked(), + Nan::New(rd_kafka_MemberDescription_consumer_id(member)) + .ToLocalChecked()); + + // memberAssignment - not passed to user, always null + Nan::Set(returnObject, Nan::New("memberAssignment").ToLocalChecked(), + Nan::Null()); + + // memberMetadata - not passed to user, always null + Nan::Set(returnObject, Nan::New("memberMetadata").ToLocalChecked(), + Nan::Null()); + + // groupInstanceId + const char* group_instance_id = + rd_kafka_MemberDescription_group_instance_id(member); + if (group_instance_id) { + Nan::Set(returnObject, Nan::New("groupInstanceId").ToLocalChecked(), + Nan::New(group_instance_id).ToLocalChecked()); + } + + // assignment + const rd_kafka_MemberAssignment_t* assignment = + rd_kafka_MemberDescription_assignment(member); + const rd_kafka_topic_partition_list_t* partitions = + rd_kafka_MemberAssignment_partitions(assignment); + v8::Local topicPartitions = + Conversion::TopicPartition::ToTopicPartitionV8Array(partitions, false); + v8::Local assignmentObject = Nan::New(); + Nan::Set(assignmentObject, Nan::New("topicPartitions").ToLocalChecked(), + topicPartitions); + Nan::Set(returnObject, Nan::New("assignment").ToLocalChecked(), + assignmentObject); + + return returnObject; +} + +/** + * @brief Converts a rd_kafka_ConsumerGroupDescription_t* into a v8 object. + */ +v8::Local FromConsumerGroupDescription( + const rd_kafka_ConsumerGroupDescription_t* desc) { + /* Return object type: + { + groupId: string, + error: LibrdKafkaError, + members: MemberDescription[], + protocol: string + isSimpleConsumerGroup: boolean + protocolType: string + partitionAssignor: string + state: ConsumerGroupState - internally a number + coordinator: Node + authorizedOperations: AclOperationType[] - internally numbers + } + */ + v8::Local returnObject = Nan::New(); + + // groupId + Nan::Set( + returnObject, Nan::New("groupId").ToLocalChecked(), + Nan::New(rd_kafka_ConsumerGroupDescription_group_id(desc)) + .ToLocalChecked()); + + // error + const rd_kafka_error_t* error = rd_kafka_ConsumerGroupDescription_error(desc); + if (error) { + RdKafka::ErrorCode code = + static_cast(rd_kafka_error_code(error)); + std::string msg = std::string(rd_kafka_error_string(error)); + Nan::Set(returnObject, Nan::New("error").ToLocalChecked(), + RdKafkaError(code, msg)); + } + + // members + v8::Local members = Nan::New(); + size_t member_cnt = rd_kafka_ConsumerGroupDescription_member_count(desc); + for (size_t i = 0; i < member_cnt; i++) { + const rd_kafka_MemberDescription_t* member = + rd_kafka_ConsumerGroupDescription_member(desc, i); + Nan::Set(members, i, FromMemberDescription(member)); + } + Nan::Set(returnObject, Nan::New("members").ToLocalChecked(), members); + + // isSimpleConsumerGroup + bool is_simple = + rd_kafka_ConsumerGroupDescription_is_simple_consumer_group(desc); + Nan::Set(returnObject, Nan::New("isSimpleConsumerGroup").ToLocalChecked(), + Nan::New(is_simple)); + + // protocolType + std::string protocolType = is_simple ? "simple" : "consumer"; + Nan::Set(returnObject, Nan::New("protocolType").ToLocalChecked(), + Nan::New(protocolType).ToLocalChecked()); + + // protocol + Nan::Set(returnObject, Nan::New("protocol").ToLocalChecked(), + Nan::New( + rd_kafka_ConsumerGroupDescription_partition_assignor(desc)) + .ToLocalChecked()); + + // partitionAssignor + Nan::Set(returnObject, Nan::New("partitionAssignor").ToLocalChecked(), + Nan::New( + rd_kafka_ConsumerGroupDescription_partition_assignor(desc)) + .ToLocalChecked()); + + // state + Nan::Set(returnObject, Nan::New("state").ToLocalChecked(), + Nan::New(rd_kafka_ConsumerGroupDescription_state(desc))); + + // coordinator + const rd_kafka_Node_t* coordinator = + rd_kafka_ConsumerGroupDescription_coordinator(desc); + if (coordinator) { + v8::Local coordinatorObject = + Conversion::Util::ToV8Object(coordinator); + Nan::Set(returnObject, Nan::New("coordinator").ToLocalChecked(), + coordinatorObject); + } + + // authorizedOperations + size_t authorized_operations_cnt; + const rd_kafka_AclOperation_t* authorized_operations = + rd_kafka_ConsumerGroupDescription_authorized_operations( + desc, &authorized_operations_cnt); + if (authorized_operations) { + Nan::Set(returnObject, Nan::New("authorizedOperations").ToLocalChecked(), + Conversion::Util::ToV8Array(authorized_operations, + authorized_operations_cnt)); + } + + return returnObject; +} + +/** + * @brief Converts a rd_kafka_DescribeConsumerGroups_result_t* into a v8 object. + */ +v8::Local FromDescribeConsumerGroupsResult( + const rd_kafka_DescribeConsumerGroups_result_t* result) { + /* Return object type: + { groups: GroupDescription[] } + */ + v8::Local returnObject = Nan::New(); + v8::Local groups = Nan::New(); + size_t groups_cnt; + const rd_kafka_ConsumerGroupDescription_t** groups_list = + rd_kafka_DescribeConsumerGroups_result_groups(result, &groups_cnt); + + for (size_t i = 0; i < groups_cnt; i++) { + const rd_kafka_ConsumerGroupDescription_t* group = groups_list[i]; + Nan::Set(groups, i, FromConsumerGroupDescription(group)); + } + + Nan::Set(returnObject, Nan::New("groups").ToLocalChecked(), groups); + return returnObject; +} + } // namespace Admin } // namespace Conversion diff --git a/src/common.h b/src/common.h index 0359bc23..9011e1f1 100644 --- a/src/common.h +++ b/src/common.h @@ -98,6 +98,9 @@ std::vector ToStringVector(v8::Local); v8::Local ToV8Array(std::vector); v8::Local ToV8Array(const rd_kafka_error_t **error_list, size_t error_cnt); +v8::Local ToV8Array(const rd_kafka_AclOperation_t *, size_t); + +v8::Local ToV8Object(const rd_kafka_Node_t *); } // namespace Util namespace Admin { @@ -113,12 +116,22 @@ std::vector FromV8GroupStateArray( // ListGroups: response v8::Local FromListConsumerGroupsResult( const rd_kafka_ListConsumerGroups_result_t *); + +// DescribeGroups: response +v8::Local FromMemberDescription( + const rd_kafka_MemberDescription_t *member); +v8::Local FromConsumerGroupDescription( + const rd_kafka_ConsumerGroupDescription_t *desc); +v8::Local FromDescribeConsumerGroupsResult( + const rd_kafka_DescribeConsumerGroups_result_t *); } // namespace Admin namespace TopicPartition { -v8::Local ToV8Array(std::vector &); -RdKafka::TopicPartition * FromV8Object(v8::Local); +v8::Local ToV8Array(std::vector &); +v8::Local ToTopicPartitionV8Array( + const rd_kafka_topic_partition_list_t *, bool include_offset); +RdKafka::TopicPartition *FromV8Object(v8::Local); std::vector FromV8Array(const v8::Local &); // NOLINT } // namespace TopicPartition diff --git a/src/workers.cc b/src/workers.cc index f714f280..13374d8a 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -1296,5 +1296,57 @@ void AdminClientListGroups::HandleErrorCallback() { callback->Call(argc, argv); } +/** + * @brief Describe consumer groups in an asynchronous worker. + * + * This callback will list consumer groups. + * + */ +AdminClientDescribeGroups::AdminClientDescribeGroups( + Nan::Callback* callback, NodeKafka::AdminClient* client, + std::vector& groups, bool include_authorized_operations, + const int& timeout_ms) + : ErrorAwareWorker(callback), + m_client(client), + m_groups(groups), + m_include_authorized_operations(include_authorized_operations), + m_timeout_ms(timeout_ms) {} + +AdminClientDescribeGroups::~AdminClientDescribeGroups() { + if (this->m_event_response) { + rd_kafka_event_destroy(this->m_event_response); + } +} + +void AdminClientDescribeGroups::Execute() { + Baton b = m_client->DescribeGroups(m_groups, m_include_authorized_operations, + m_timeout_ms, &m_event_response); + if (b.err() != RdKafka::ERR_NO_ERROR) { + SetErrorBaton(b); + } +} + +void AdminClientDescribeGroups::HandleOKCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 2; + v8::Local argv[argc]; + + argv[0] = Nan::Null(); + argv[1] = Conversion::Admin::FromDescribeConsumerGroupsResult( + rd_kafka_event_DescribeConsumerGroups_result(m_event_response)); + + callback->Call(argc, argv); +} + +void AdminClientDescribeGroups::HandleErrorCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 1; + v8::Local argv[argc] = {GetErrorObject()}; + + callback->Call(argc, argv); +} + } // namespace Workers } // namespace NodeKafka diff --git a/src/workers.h b/src/workers.h index 2844240a..826206ea 100644 --- a/src/workers.h +++ b/src/workers.h @@ -530,6 +530,27 @@ class AdminClientListGroups : public ErrorAwareWorker { rd_kafka_event_t *m_event_response; }; +/** + * @brief Describe consumer groups on a remote broker cluster. + */ +class AdminClientDescribeGroups : public ErrorAwareWorker { + public: + AdminClientDescribeGroups(Nan::Callback *, NodeKafka::AdminClient *, + std::vector &, bool, const int &); + ~AdminClientDescribeGroups(); + + void Execute(); + void HandleOKCallback(); + void HandleErrorCallback(); + + private: + NodeKafka::AdminClient *m_client; + std::vector m_groups; + const bool m_include_authorized_operations; + const int m_timeout_ms; + rd_kafka_event_t *m_event_response; +}; + } // namespace Workers } // namespace NodeKafka diff --git a/test/promisified/admin/describe_groups.spec.js b/test/promisified/admin/describe_groups.spec.js new file mode 100644 index 00000000..0981d12d --- /dev/null +++ b/test/promisified/admin/describe_groups.spec.js @@ -0,0 +1,136 @@ +jest.setTimeout(30000); + +const { + createConsumer, + secureRandom, + createTopic, + waitFor, + createAdmin, + sleep, +} = require('../testhelpers'); +const { ConsumerGroupStates, ErrorCodes, AclOperationTypes } = require('../../../lib').KafkaJS; + +describe('Admin > describeGroups', () => { + let topicName, groupId, consumer, admin; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; + + consumer = createConsumer({ + groupId, + fromBeginning: true, + clientId: 'test-client-id', + }, { + 'group.instance.id': 'test-instance-id', + 'session.timeout.ms': 10000, + 'partition.assignment.strategy': 'roundrobin', + }); + + await createTopic({ topic: topicName, partitions: 2 }); + + admin = createAdmin({}); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()); + admin && (await admin.disconnect()); + }); + + it('should timeout', async () => { + await admin.connect(); + + await expect(admin.describeGroups(['not-a-real-group'], { timeout: 0 })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__TIMED_OUT + ); + }); + + it('should not accept empty or null groups array', async () => { + await admin.connect(); + + await expect(admin.describeGroups([])).rejects.toHaveProperty( + 'message', + 'Must provide at least one group name' + ); + + await expect(admin.describeGroups(null)).rejects.toHaveProperty( + 'message', + 'Must provide group name array' + ); + }); + + it('should describe consumer groups', async () => { + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + await consumer.run({ eachMessage: async () => { } }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + await admin.connect(); + let describeGroupsResult = await admin.describeGroups( + [groupId], { includeAuthorizedOperations: true }); + expect(describeGroupsResult.groups.length).toEqual(1); + expect(describeGroupsResult.groups[0]).toEqual( + expect.objectContaining({ + groupId, + protocol: 'roundrobin', + partitionAssignor: 'roundrobin', + isSimpleConsumerGroup: false, + protocolType: 'consumer', + state: ConsumerGroupStates.STABLE, + coordinator: expect.objectContaining({ + id: expect.any(Number), + host: expect.any(String), + port: expect.any(Number), + }), + authorizedOperations: expect.arrayContaining([AclOperationTypes.READ, AclOperationTypes.DESCRIBE]), + members: expect.arrayContaining([ + expect.objectContaining({ + clientHost: expect.any(String), + clientId: 'test-client-id', + memberId: expect.any(String), + memberAssignment: null, + memberMetadata: null, + groupInstanceId: 'test-instance-id', + assignment: { + topicPartitions:[ + expect.objectContaining({ topic: topicName, partition: 0 }), + expect.objectContaining({ topic: topicName, partition: 1 }), + ], + } + }), + ]), + }) + ); + + // Disconnect the consumer to make the group EMPTY. + await consumer.disconnect(); + consumer = null; + + // Wait so that session.timeout.ms expires and the group becomes EMPTY. + await sleep(12000); + + // Don't include authorized operations this time. + describeGroupsResult = await admin.describeGroups([groupId]); + expect(describeGroupsResult.groups.length).toEqual(1); + expect(describeGroupsResult.groups[0]).toEqual( + expect.objectContaining({ + groupId, + protocol: '', + partitionAssignor: '', + isSimpleConsumerGroup: false, + protocolType: 'consumer', + state: ConsumerGroupStates.EMPTY, + coordinator: expect.objectContaining({ + id: expect.any(Number), + host: expect.any(String), + port: expect.any(Number), + }), + members: [], + }) + ); + expect(describeGroupsResult.groups[0].authorizedOperations).toBeUndefined(); + }); +}); + diff --git a/test/promisified/admin/list_groups.spec.js b/test/promisified/admin/list_groups.spec.js index 192e203d..29479eaf 100644 --- a/test/promisified/admin/list_groups.spec.js +++ b/test/promisified/admin/list_groups.spec.js @@ -55,7 +55,7 @@ describe('Admin > listGroups', () => { expect.objectContaining({ groupId, isSimpleConsumerGroup: false, - protocolType: '', + protocolType: 'consumer', state: ConsumerGroupStates.STABLE, }), ]) @@ -72,7 +72,7 @@ describe('Admin > listGroups', () => { expect.objectContaining({ groupId, isSimpleConsumerGroup: false, - protocolType: '', + protocolType: 'consumer', state: ConsumerGroupStates.EMPTY, }), ]) diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 3bbd2d6e..5de2112a 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,10 +1,10 @@ import * as tls from 'tls' import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig, TopicConfig } from './config' -import { ConsumerGroupStates, GroupOverview, LibrdKafkaError } from '../index' +import { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescription, GroupDescriptions } from '../index' // Admin API related interfaces, types etc; and Error types are common, so // just re-export them from here too. -export { ConsumerGroupStates, GroupOverview, LibrdKafkaError } from '../index' +export { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescriptions } from '../index' export type BrokersFunction = () => string[] | Promise @@ -409,7 +409,6 @@ export interface TopicOffsets { partitions: PartitionOffset[] } - export interface PartitionOffset { partition: number offset: string @@ -433,22 +432,6 @@ export interface OffsetsByTopicPartition { topics: TopicOffsets[] } -export type MemberDescription = { - clientHost: string - clientId: string - memberId: string - memberAssignment: Buffer - memberMetadata: Buffer -} - -export type GroupDescription = { - groupId: string - members: MemberDescription[] - protocol: string - protocolType: string - state: ConsumerGroupStates -} - export type Consumer = { connect(): Promise disconnect(): Promise @@ -562,4 +545,7 @@ export type Admin = { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }): Promise<{ groups: GroupOverview[], errors: LibrdKafkaError[] }> + describeGroups( + groups: string[], + options?: {timeout?: number, includeAuthorizedOperations?: boolean }): Promise } From a69206df542fd4e50692527f86589259d39fa1e6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 22 Mar 2024 14:57:50 +0530 Subject: [PATCH 105/224] Add AdminAPI for deleteGroups --- index.d.ts | 11 ++ lib/admin.js | 40 ++++++ lib/kafkajs/_admin.js | 24 ++++ src/admin.cc | 126 +++++++++++++++++++ src/admin.h | 3 + src/common.cc | 45 +++++++ src/common.h | 4 + src/workers.cc | 59 ++++++++- src/workers.h | 21 ++++ test/promisified/admin/delete_groups.spec.js | 103 +++++++++++++++ types/kafkajs.d.ts | 5 +- 11 files changed, 438 insertions(+), 3 deletions(-) create mode 100644 test/promisified/admin/delete_groups.spec.js diff --git a/index.d.ts b/index.d.ts index eff5f4fb..cbc4d3f5 100644 --- a/index.d.ts +++ b/index.d.ts @@ -400,6 +400,12 @@ export type GroupDescriptions = { groups: GroupDescription[], } +export type DeleteGroupsResult = { + groupId: string + errorCode?: number + error?: LibrdKafkaError +} + export interface IAdminClient { createTopic(topic: NewTopic, cb?: (err: LibrdKafkaError) => void): void; createTopic(topic: NewTopic, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; @@ -419,6 +425,11 @@ export interface IAdminClient { options?: { timeout?: number, includeAuthorizedOperations?: boolean }, cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; + deleteGroups(groupIds: string[], cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void; + deleteGroups(groupIds: string[], + options?: { timeout?: number }, + cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void; + disconnect(): void; } diff --git a/lib/admin.js b/lib/admin.js index 717dc450..c27cbd6d 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -325,3 +325,43 @@ AdminClient.prototype.describeGroups = function (groups, options, cb) { } }); } + +/** + * Delete consumer groups. + * @param {string[]} groups - The names of the groups to delete. + * @param {any?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @param {function} cb - The callback to be executed when finished. + * + * Valid ways to call this function: + * deleteGroups(groups, cb) + * deleteGroups(groups, options, cb) + */ +AdminClient.prototype.deleteGroups = function (groups, options, cb) { + if (!this._isConnected) { + throw new Error('Client is disconnected'); + } + + if (typeof options === 'function') { + cb = options; + options = {}; + } + + if (!options) { + options = {}; + } + + this._client.deleteGroups(groups, options, function (err, reports) { + if (err) { + if (cb) { + cb(LibrdKafkaError.create(err)); + } + return; + } + + if (cb) { + cb(null, reports); + } + }); +} diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index bf2a5a2d..12f1af8d 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -283,6 +283,30 @@ class Admin { }); }); } + + /** + * Delete consumer groups. + * @param {string[]} groups - The names of the groups to delete. + * @param {any?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @returns {Promise} + */ + async deleteGroups(groups, options = {}) { + if (this.#state !== AdminState.CONNECTED) { + throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE }); + } + + return new Promise((resolve, reject) => { + this.#internalClient.deleteGroups(groups, options, (err, reports) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } else { + resolve(reports); + } + }); + }); + } } module.exports = { diff --git a/src/admin.cc b/src/admin.cc index de57485b..cfedd282 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -90,6 +90,7 @@ void AdminClient::Init(v8::Local exports) { // Consumer group related operations Nan::SetPrototypeMethod(tpl, "listGroups", NodeListGroups); Nan::SetPrototypeMethod(tpl, "describeGroups", NodeDescribeGroups); + Nan::SetPrototypeMethod(tpl, "deleteGroups", NodeDeleteGroups); Nan::SetPrototypeMethod(tpl, "connect", NodeConnect); Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect); @@ -446,6 +447,13 @@ Baton AdminClient::ListGroups( rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new( m_client->c_ptr(), RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS); + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_AdminOptions_set_request_timeout( + options, timeout_ms, errstr, sizeof(errstr)); + if (err != RD_KAFKA_RESP_ERR_NO_ERROR) { + return Baton(static_cast(err), errstr); + } + if (is_match_states_set) { rd_kafka_error_t *error = rd_kafka_AdminOptions_set_match_consumer_group_states( @@ -509,6 +517,13 @@ Baton AdminClient::DescribeGroups(std::vector &groups, rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new( m_client->c_ptr(), RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_AdminOptions_set_request_timeout( + options, timeout_ms, errstr, sizeof(errstr)); + if (err != RD_KAFKA_RESP_ERR_NO_ERROR) { + return Baton(static_cast(err), errstr); + } + if (include_authorized_operations) { rd_kafka_error_t *error = rd_kafka_AdminOptions_set_include_authorized_operations( @@ -561,6 +576,67 @@ Baton AdminClient::DescribeGroups(std::vector &groups, } } +Baton AdminClient::DeleteGroups(rd_kafka_DeleteGroup_t **group_list, + size_t group_cnt, int timeout_ms, + /* out */ rd_kafka_event_t **event_response) { + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + { + scoped_shared_write_lock lock(m_connection_lock); + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE); + } + + // Make admin options to establish that we are deleting groups + rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new( + m_client->c_ptr(), RD_KAFKA_ADMIN_OP_DELETEGROUPS); + + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_AdminOptions_set_request_timeout( + options, timeout_ms, errstr, sizeof(errstr)); + if (err != RD_KAFKA_RESP_ERR_NO_ERROR) { + return Baton(static_cast(err), errstr); + } + + // Create queue just for this operation. + rd_kafka_queue_t *rkqu = rd_kafka_queue_new(m_client->c_ptr()); + + rd_kafka_DeleteGroups(m_client->c_ptr(), group_list, group_cnt, options, + rkqu); + + // Poll for an event by type in that queue + // DON'T destroy the event. It is the out parameter, and ownership is + // the caller's. + *event_response = + PollForEvent(rkqu, RD_KAFKA_EVENT_DELETEGROUPS_RESULT, timeout_ms); + + // Destroy the queue since we are done with it. + rd_kafka_queue_destroy(rkqu); + + // Destroy the options we just made because we polled already + rd_kafka_AdminOptions_destroy(options); + + // If we got no response from that operation, this is a failure + // likely due to time out + if (*event_response == NULL) { + return Baton(RdKafka::ERR__TIMED_OUT); + } + + // Now we can get the error code from the event + if (rd_kafka_event_error(*event_response)) { + // If we had a special error code, get out of here with it + const rd_kafka_resp_err_t errcode = rd_kafka_event_error(*event_response); + return Baton(static_cast(errcode)); + } + + // At this point, event_response contains the result, which needs + // to be parsed/converted by the caller. + return Baton(RdKafka::ERR_NO_ERROR); + } +} + void AdminClient::ActivateDispatchers() { // Listen to global config m_gconfig->listen(); @@ -831,4 +907,54 @@ NAN_METHOD(AdminClient::NodeDescribeGroups) { timeout_ms)); } +/** + * Delete Consumer Groups. + */ +NAN_METHOD(AdminClient::NodeDeleteGroups) { + Nan::HandleScope scope; + + if (info.Length() < 3 || !info[2]->IsFunction()) { + // Just throw an exception + return Nan::ThrowError("Need to specify a callback"); + } + + if (!info[0]->IsArray()) { + return Nan::ThrowError("Must provide group name array"); + } + + if (!info[1]->IsObject()) { + return Nan::ThrowError("Must provide options object"); + } + + // Get list of group names to delete, and convert it into an + // rd_kafka_DeleteGroup_t array. + v8::Local group_names = info[0].As(); + if (group_names->Length() == 0) { + return Nan::ThrowError("Must provide at least one group name"); + } + std::vector group_names_vector = + v8ArrayToStringVector(group_names); + + // The ownership of this array is transferred to the worker. + rd_kafka_DeleteGroup_t **group_list = static_cast( + malloc(sizeof(rd_kafka_DeleteGroup_t *) * group_names_vector.size())); + for (size_t i = 0; i < group_names_vector.size(); i++) { + group_list[i] = rd_kafka_DeleteGroup_new(group_names_vector[i].c_str()); + } + + v8::Local config = info[1].As(); + + // Get the timeout - default 5000. + int timeout_ms = GetParameter(config, "timeout", 5000); + + // Create the final callback object + v8::Local cb = info[2].As(); + Nan::Callback *callback = new Nan::Callback(cb); + AdminClient *client = ObjectWrap::Unwrap(info.This()); + + // Queue the work. + Nan::AsyncQueueWorker(new Workers::AdminClientDeleteGroups( + callback, client, group_list, group_names_vector.size(), timeout_ms)); +} + } // namespace NodeKafka diff --git a/src/admin.h b/src/admin.h index 2f6a982f..d931072c 100644 --- a/src/admin.h +++ b/src/admin.h @@ -59,6 +59,8 @@ class AdminClient : public Connection { Baton DescribeGroups(std::vector& groups, bool include_authorized_operations, int timeout_ms, rd_kafka_event_t** event_response); + Baton DeleteGroups(rd_kafka_DeleteGroup_t** group_list, size_t group_cnt, + int timeout_ms, rd_kafka_event_t** event_response); protected: static Nan::Persistent constructor; @@ -79,6 +81,7 @@ class AdminClient : public Connection { // Consumer group operations static NAN_METHOD(NodeListGroups); static NAN_METHOD(NodeDescribeGroups); + static NAN_METHOD(NodeDeleteGroups); static NAN_METHOD(NodeConnect); static NAN_METHOD(NodeDisconnect); diff --git a/src/common.cc b/src/common.cc index 0bdc149f..62542bf1 100644 --- a/src/common.cc +++ b/src/common.cc @@ -961,6 +961,51 @@ v8::Local FromDescribeConsumerGroupsResult( return returnObject; } +/** + * @brief Converts a rd_kafka_DeleteGroups_result_t* into a v8 array. +*/ +v8::Local FromDeleteGroupsResult( + const rd_kafka_DeleteGroups_result_t* result) { + /* Return object type: + [{ + groupId: string + errorCode?: number + error?: LibrdKafkaError + }] + */ + v8::Local returnArray = Nan::New(); + size_t result_cnt; + const rd_kafka_group_result_t** results = + rd_kafka_DeleteGroups_result_groups(result, &result_cnt); + + for (size_t i = 0; i < result_cnt; i++) { + const rd_kafka_group_result_t* group_result = results[i]; + v8::Local group_object = Nan::New(); + + Nan::Set(group_object, Nan::New("groupId").ToLocalChecked(), + Nan::New(rd_kafka_group_result_name(group_result)) + .ToLocalChecked()); + + const rd_kafka_error_t* error = rd_kafka_group_result_error(group_result); + if (!error) { + Nan::Set(group_object, Nan::New("errorCode").ToLocalChecked(), + Nan::New(RD_KAFKA_RESP_ERR_NO_ERROR)); + } else { + RdKafka::ErrorCode code = + static_cast(rd_kafka_error_code(error)); + const char* msg = rd_kafka_error_string(error); + + Nan::Set(group_object, Nan::New("errorCode").ToLocalChecked(), + Nan::New(code)); + Nan::Set(group_object, Nan::New("error").ToLocalChecked(), + RdKafkaError(code, msg)); + } + Nan::Set(returnArray, i, group_object); + } + + return returnArray; +} + } // namespace Admin } // namespace Conversion diff --git a/src/common.h b/src/common.h index 9011e1f1..cccaef60 100644 --- a/src/common.h +++ b/src/common.h @@ -124,6 +124,10 @@ v8::Local FromConsumerGroupDescription( const rd_kafka_ConsumerGroupDescription_t *desc); v8::Local FromDescribeConsumerGroupsResult( const rd_kafka_DescribeConsumerGroups_result_t *); + +// DeleteGroups: Response +v8::Local FromDeleteGroupsResult( + const rd_kafka_DeleteGroups_result_t *); } // namespace Admin namespace TopicPartition { diff --git a/src/workers.cc b/src/workers.cc index 13374d8a..d414e4dc 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -1299,7 +1299,7 @@ void AdminClientListGroups::HandleErrorCallback() { /** * @brief Describe consumer groups in an asynchronous worker. * - * This callback will list consumer groups. + * This callback will describe consumer groups. * */ AdminClientDescribeGroups::AdminClientDescribeGroups( @@ -1348,5 +1348,62 @@ void AdminClientDescribeGroups::HandleErrorCallback() { callback->Call(argc, argv); } +/** + * @brief Delete consumer groups in an asynchronous worker. + * + * This callback will delete consumer groups. + * + */ +AdminClientDeleteGroups::AdminClientDeleteGroups( + Nan::Callback* callback, NodeKafka::AdminClient* client, + rd_kafka_DeleteGroup_t **group_list, + size_t group_cnt, + const int& timeout_ms) + : ErrorAwareWorker(callback), + m_client(client), + m_group_list(group_list), + m_group_cnt(group_cnt), + m_timeout_ms(timeout_ms) {} + +AdminClientDeleteGroups::~AdminClientDeleteGroups() { + if (m_group_list) { + rd_kafka_DeleteGroup_destroy_array(m_group_list, m_group_cnt); + free(m_group_list); + } + + if (this->m_event_response) { + rd_kafka_event_destroy(this->m_event_response); + } +} + +void AdminClientDeleteGroups::Execute() { + Baton b = m_client->DeleteGroups(m_group_list, m_group_cnt, m_timeout_ms, &m_event_response); + if (b.err() != RdKafka::ERR_NO_ERROR) { + SetErrorBaton(b); + } +} + +void AdminClientDeleteGroups::HandleOKCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 2; + v8::Local argv[argc]; + + argv[0] = Nan::Null(); + argv[1] = Conversion::Admin::FromDeleteGroupsResult(rd_kafka_event_DeleteGroups_result(m_event_response)); + + callback->Call(argc, argv); +} + +void AdminClientDeleteGroups::HandleErrorCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 1; + v8::Local argv[argc] = {GetErrorObject()}; + + callback->Call(argc, argv); +} + + } // namespace Workers } // namespace NodeKafka diff --git a/src/workers.h b/src/workers.h index 826206ea..f85add26 100644 --- a/src/workers.h +++ b/src/workers.h @@ -551,6 +551,27 @@ class AdminClientDescribeGroups : public ErrorAwareWorker { rd_kafka_event_t *m_event_response; }; +/** + * @brief Delete consumer groups on a remote broker cluster. + */ +class AdminClientDeleteGroups : public ErrorAwareWorker { + public: + AdminClientDeleteGroups(Nan::Callback *, NodeKafka::AdminClient *, + rd_kafka_DeleteGroup_t **, size_t, const int &); + ~AdminClientDeleteGroups(); + + void Execute(); + void HandleOKCallback(); + void HandleErrorCallback(); + + private: + NodeKafka::AdminClient *m_client; + rd_kafka_DeleteGroup_t **m_group_list; + size_t m_group_cnt; + const int m_timeout_ms; + rd_kafka_event_t *m_event_response; +}; + } // namespace Workers } // namespace NodeKafka diff --git a/test/promisified/admin/delete_groups.spec.js b/test/promisified/admin/delete_groups.spec.js new file mode 100644 index 00000000..6c308d91 --- /dev/null +++ b/test/promisified/admin/delete_groups.spec.js @@ -0,0 +1,103 @@ +jest.setTimeout(30000); + +const { + createConsumer, + secureRandom, + createTopic, + waitFor, + createAdmin, +} = require('../testhelpers'); +const { ConsumerGroupStates, ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Admin > deleteGroups', () => { + let topicName, groupId, consumer, admin; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; + + consumer = createConsumer({ + groupId, + fromBeginning: true, + }); + + await createTopic({ topic: topicName, partitions: 2 }); + + admin = createAdmin({}); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()); + admin && (await admin.disconnect()); + }); + + it('should timeout', async () => { + await admin.connect(); + + await expect(admin.deleteGroups(['invalid-group'], { timeout: 0 })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__TIMED_OUT + ); + }); + + it('should delete empty consumer groups', async () => { + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + await consumer.run({ eachMessage: async () => {} }); + + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + await admin.connect(); + let listGroupsResult = await admin.listGroups(); + expect(listGroupsResult.errors).toEqual([]); + expect(listGroupsResult.groups.map(group => group.groupId)).toContain(groupId); + + // Delete when the group is not empty - it should fail. + let deleteResult = await admin.deleteGroups([groupId]); + expect(deleteResult).toEqual([ + expect.objectContaining({ + groupId, + error: expect.objectContaining({ + code: ErrorCodes.ERR_NON_EMPTY_GROUP, + }), + errorCode: ErrorCodes.ERR_NON_EMPTY_GROUP, + }), + ]); + + // Disconnect the consumer to make the group EMPTY. + await consumer.disconnect(); + consumer = null; + + listGroupsResult = await admin.listGroups(); + expect(listGroupsResult.errors).toEqual([]); + expect(listGroupsResult.groups.map(group => group.groupId)).toContain(groupId); + + // Delete the empty consumer group. + deleteResult = await admin.deleteGroups([groupId]); + expect(deleteResult).toEqual([ + expect.objectContaining({ + groupId, + errorCode: ErrorCodes.ERR_NO_ERROR, + }), + ]); + + // Cross-verify the deletion. + listGroupsResult = await admin.listGroups(); + expect(listGroupsResult.errors).toEqual([]); + expect(listGroupsResult.groups.map(group => group.groupId)).not.toContain(groupId); + + // Deleting the group again should fail. + deleteResult = await admin.deleteGroups([groupId]); + expect(deleteResult).toEqual([ + expect.objectContaining({ + groupId, + error: expect.objectContaining({ + code: ErrorCodes.ERR_GROUP_ID_NOT_FOUND, + }), + errorCode: ErrorCodes.ERR_GROUP_ID_NOT_FOUND, + }), + ]); + + }); +}); + diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 5de2112a..64a6e48e 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,10 +1,10 @@ import * as tls from 'tls' import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig, TopicConfig } from './config' -import { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescription, GroupDescriptions } from '../index' +import { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescription, GroupDescriptions, DeleteGroupsResult } from '../index' // Admin API related interfaces, types etc; and Error types are common, so // just re-export them from here too. -export { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescriptions } from '../index' +export { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescriptions, DeleteGroupsResult } from '../index' export type BrokersFunction = () => string[] | Promise @@ -548,4 +548,5 @@ export type Admin = { describeGroups( groups: string[], options?: {timeout?: number, includeAuthorizedOperations?: boolean }): Promise + deleteGroups(groupIds: string[], options?: { timeout?: number }): Promise } From 31626251ebbd59472caeca8757d699cbad570532 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 22 Mar 2024 15:03:22 +0530 Subject: [PATCH 106/224] Version bump to 0.1.11-devel --- CHANGELOG.md | 3 ++- lib/client.js | 2 +- package-lock.json | 4 ++-- package.json | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0fb8e71a..4b351196 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,11 @@ -# confluent-kafka-javascript vNext +# confluent-kafka-javascript v0.1.11-devel v0.1.11-devel is a pre-production, early-access release. ## Features 1. Add support for `eachBatch` in the Consumer API (partial support for API compatibility). +2. Add support for `listGroups`, `describeGroups` and `deleteGroups` in the Admin API. # confluent-kafka-javascript v0.1.10-devel diff --git a/lib/client.js b/lib/client.js index f26d15de..97950c93 100644 --- a/lib/client.js +++ b/lib/client.js @@ -15,7 +15,7 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); -const bindingVersion = 'v0.1.10-devel'; +const bindingVersion = 'v0.1.11-devel'; var LibrdKafkaError = require('./error'); diff --git a/package-lock.json b/package-lock.json index 8f8473fd..e928ac96 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.10-devel", + "version": "v0.1.11-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.10-devel", + "version": "v0.1.11-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index 88780e23..61f9d21a 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.10-devel", + "version": "v0.1.11-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.3.0", From 419d47f5dec80abaef1fdc9c6637c3162dd0c920 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 26 Mar 2024 08:40:37 +0530 Subject: [PATCH 107/224] Changes in package-lock.json due to updates --- package-lock.json | 237 +++++++++++++++++----------------------------- 1 file changed, 85 insertions(+), 152 deletions(-) diff --git a/package-lock.json b/package-lock.json index e928ac96..14a68451 100644 --- a/package-lock.json +++ b/package-lock.json @@ -52,113 +52,42 @@ } }, "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==", + "version": "7.24.2", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.2.tgz", + "integrity": "sha512-y5+tLQyV8pg3fsiln67BVLD1P13Eg4lh5RW9mF0zUuvLrv9uIQ4MCL+CRT+FTsBlBjcIan6PGsLcBN0m3ClUyQ==", "dev": true, "dependencies": { - "@babel/highlight": "^7.23.4", - "chalk": "^2.4.2" + "@babel/highlight": "^7.24.2", + "picocolors": "^1.0.0" }, "engines": { "node": ">=6.9.0" } }, - "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, - "dependencies": { - "color-convert": "^1.9.0" - }, - "engines": { - "node": ">=4" - } - }, - "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, - "dependencies": { - "ansi-styles": "^3.2.1", - "escape-string-regexp": "^1.0.5", - "supports-color": "^5.3.0" - }, - "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/@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/@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, - "engines": { - "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/@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": { - "has-flag": "^3.0.0" - }, - "engines": { - "node": ">=4" - } - }, "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==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.24.1.tgz", + "integrity": "sha512-Pc65opHDliVpRHuKfzI+gSA4zcgr65O4cl64fFJIWEEh8JoHIHh0Oez1Eo8Arz8zq/JhgKodQaxEwUPRtZylVA==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/core": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.24.0.tgz", - "integrity": "sha512-fQfkg0Gjkza3nf0c7/w6Xf34BW4YvzNfACRLmmb7XRLa6XHdR+K9AlJlxneFfWYf6uhOzuzZVTjF/8KfndZANw==", + "version": "7.24.3", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.24.3.tgz", + "integrity": "sha512-5FcvN1JHw2sHJChotgx8Ek0lyuh4kCKelgMTTqhYJJtloNvUfpAFMeNQUtdlIaktwrSV9LtCdqwk48wL2wBacQ==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.23.5", - "@babel/generator": "^7.23.6", + "@babel/code-frame": "^7.24.2", + "@babel/generator": "^7.24.1", "@babel/helper-compilation-targets": "^7.23.6", "@babel/helper-module-transforms": "^7.23.3", - "@babel/helpers": "^7.24.0", - "@babel/parser": "^7.24.0", + "@babel/helpers": "^7.24.1", + "@babel/parser": "^7.24.1", "@babel/template": "^7.24.0", - "@babel/traverse": "^7.24.0", + "@babel/traverse": "^7.24.1", "@babel/types": "^7.24.0", "convert-source-map": "^2.0.0", "debug": "^4.1.0", @@ -184,14 +113,14 @@ } }, "node_modules/@babel/generator": { - "version": "7.23.6", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.23.6.tgz", - "integrity": "sha512-qrSfCYxYQB5owCmGLbl8XRpX1ytXlpueOb0N0UmQwA073KZxejgQTzAmJezxvpwQD9uGtK2shHdi55QT+MbjIw==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.24.1.tgz", + "integrity": "sha512-DfCRfZsBcrPEHUfuBMgbJ1Ut01Y/itOs+hY2nFLgqsqXd52/iSiVq5TITtUasIUgm+IIKdY2/1I7auiQOEeC9A==", "dev": true, "dependencies": { - "@babel/types": "^7.23.6", - "@jridgewell/gen-mapping": "^0.3.2", - "@jridgewell/trace-mapping": "^0.3.17", + "@babel/types": "^7.24.0", + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.25", "jsesc": "^2.5.1" }, "engines": { @@ -258,12 +187,12 @@ } }, "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==", + "version": "7.24.3", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.3.tgz", + "integrity": "sha512-viKb0F9f2s0BCS22QSF308z/+1YWKV/76mwt61NBzS5izMzDPwdq1pTrzf+Li3npBWX9KdQbkeCt1jSAM7lZqg==", "dev": true, "dependencies": { - "@babel/types": "^7.22.15" + "@babel/types": "^7.24.0" }, "engines": { "node": ">=6.9.0" @@ -322,9 +251,9 @@ } }, "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==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.1.tgz", + "integrity": "sha512-2ofRCjnnA9y+wk8b9IAREroeUP02KHp431N2mhKniy2yKIDKpbrHv9eXwm8cBeWQYcJmzv5qKCu65P47eCF7CQ==", "dev": true, "engines": { "node": ">=6.9.0" @@ -349,13 +278,13 @@ } }, "node_modules/@babel/helpers": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.24.0.tgz", - "integrity": "sha512-ulDZdc0Aj5uLc5nETsa7EPx2L7rM0YJM8r7ck7U73AXi7qOV44IHHRAYZHY6iU1rr3C5N4NtTmMRUJP6kwCWeA==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.24.1.tgz", + "integrity": "sha512-BpU09QqEe6ZCHuIHFphEFgvNSrubve1FtyMton26ekZ85gRGi6LrTF7zArARp2YvyFxloeiRmtSCq5sjh1WqIg==", "dev": true, "dependencies": { "@babel/template": "^7.24.0", - "@babel/traverse": "^7.24.0", + "@babel/traverse": "^7.24.1", "@babel/types": "^7.24.0" }, "engines": { @@ -363,14 +292,15 @@ } }, "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==", + "version": "7.24.2", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.2.tgz", + "integrity": "sha512-Yac1ao4flkTxTteCDZLEvdxg2fZfz1v8M4QpaGypq/WPDqg3ijHYbDfs+LG5hvzSoqaSZ9/Z9lKSP3CjZjv+pA==", "dev": true, "dependencies": { "@babel/helper-validator-identifier": "^7.22.20", "chalk": "^2.4.2", - "js-tokens": "^4.0.0" + "js-tokens": "^4.0.0", + "picocolors": "^1.0.0" }, "engines": { "node": ">=6.9.0" @@ -448,9 +378,9 @@ } }, "node_modules/@babel/parser": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.24.0.tgz", - "integrity": "sha512-QuP/FxEAzMSjXygs8v4N9dvdXzEHN4W1oF3PxuWAtPo08UdM17u89RDMgjLn/mlc56iM0HlLmVkO/wgR+rDgHg==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.24.1.tgz", + "integrity": "sha512-Zo9c7N3xdOIQrNip7Lc9wvRPzlRtovHVE4lkz8WEDr7uYh/GMQhSiIgFxGIArRHYdJE5kxtZjAf8rT0xhdLCzg==", "dev": true, "bin": { "parser": "bin/babel-parser.js" @@ -520,12 +450,12 @@ } }, "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==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.1.tgz", + "integrity": "sha512-2eCtxZXf+kbkMIsXS4poTvT4Yu5rXiRa+9xGVT56raghjmBTKMpFNc9R4IDiB4emao9eO22Ox7CxuJG7BgExqA==", "dev": true, "dependencies": { - "@babel/helper-plugin-utils": "^7.22.5" + "@babel/helper-plugin-utils": "^7.24.0" }, "engines": { "node": ">=6.9.0" @@ -622,12 +552,12 @@ } }, "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==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.1.tgz", + "integrity": "sha512-Yhnmvy5HZEnHUty6i++gcfH1/l68AHnItFHnaCv6hn9dNh0hQvvQJsxpi4BMBFN5DLeHBuucT/0DgzXif/OyRw==", "dev": true, "dependencies": { - "@babel/helper-plugin-utils": "^7.22.5" + "@babel/helper-plugin-utils": "^7.24.0" }, "engines": { "node": ">=6.9.0" @@ -651,18 +581,18 @@ } }, "node_modules/@babel/traverse": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.24.0.tgz", - "integrity": "sha512-HfuJlI8qq3dEDmNU5ChzzpZRWq+oxCZQyMzIMEqLho+AQnhMnKQUzH6ydo3RBl/YjPCuk68Y6s0Gx0AeyULiWw==", + "version": "7.24.1", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.24.1.tgz", + "integrity": "sha512-xuU6o9m68KeqZbQuDt2TcKSxUw/mrsvavlEqQ1leZ/B+C9tk6E4sRWy97WaXgvq5E+nU3cXMxv3WKOCanVMCmQ==", "dev": true, "dependencies": { - "@babel/code-frame": "^7.23.5", - "@babel/generator": "^7.23.6", + "@babel/code-frame": "^7.24.1", + "@babel/generator": "^7.24.1", "@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.24.0", + "@babel/parser": "^7.24.1", "@babel/types": "^7.24.0", "debug": "^4.3.1", "globals": "^11.1.0" @@ -1458,9 +1388,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.11.25", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.25.tgz", - "integrity": "sha512-TBHyJxk2b7HceLVGFcpAUjsa5zIdsPWlR6XHfyGzd0SFu+/NFgQgMAl96MSDZgQDvJAvV6BKsFOrt6zIL09JDw==", + "version": "20.11.30", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.30.tgz", + "integrity": "sha512-dHM6ZxwlmuZaRmUPfv1p+KrdD1Dci04FbdEm/9wEMouFqxYoFl5aMkt0VMAUtYRQDyYvD41WJLukhq/ha3YuTw==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -1809,12 +1739,15 @@ "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==", + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.3.0.tgz", + "integrity": "sha512-Ceh+7ox5qe7LJuLHoY0feh3pHuUDHAcRUeyL2VYghZwfpkNIy/+8Ocg0a3UuSoYzavmylwuLWQOf3hl0jjMMIw==", "dev": true, "engines": { "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, "node_modules/bindings": { @@ -2002,9 +1935,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001596", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001596.tgz", - "integrity": "sha512-zpkZ+kEr6We7w63ORkoJ2pOfBwBkY/bJrG/UZ90qNb45Isblu8wzDgevEOrRL1r9dWayHjYiiyCMEXPn4DweGQ==", + "version": "1.0.30001600", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001600.tgz", + "integrity": "sha512-+2S9/2JFhYmYaDpZvo0lKkfvuKIglrx68MwOBqMGHhQsNkLjB5xtc/TGoEPs+MxjSyN/72qer2g97nzR641mOQ==", "dev": true, "funding": [ { @@ -2339,9 +2272,9 @@ "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==", + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.3.tgz", + "integrity": "sha512-bwy0MGW55bG41VqxxypOsdSdGqLwXPI/focwgTYCFMbdUiBAxLg9CFzG08sz2aqzknwiX7Hkl0bQENjg8iLByw==", "engines": { "node": ">=8" } @@ -2442,9 +2375,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.698", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.698.tgz", - "integrity": "sha512-f9iZD1t3CLy1AS6vzM5EKGa6p9pRcOeEFXRFbaG2Ta+Oe7MkfRQ3fsvPYidzHe1h4i0JvIvpcY55C+B6BZNGtQ==", + "version": "1.4.716", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.716.tgz", + "integrity": "sha512-t/MXMzFKQC3UfMDpw7V5wdB/UAB8dWx4hEsy+fpPYJWW3gqh3u5T1uXp6vR+H6dGCPBxkRo+YBcapBLvbGQHRw==", "dev": true }, "node_modules/emittery": { @@ -2999,9 +2932,9 @@ "integrity": "sha512-8Rf9Y83NBReMnx0gFzA8JImQACstCYWUplepDa9xprwwtmgEZUF0h/i5xSA625zB/I37EtrswSST6OXxwaaIJQ==" }, "node_modules/hasown": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.1.tgz", - "integrity": "sha512-1/th4MHjnwncwXsIW6QMzlvYL9kG5e/CpVvLRZe4XPa8TOUNbCELqmvhDmnkNsAjwaG4+I8gJJL0JBvTTLO9qA==", + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.2.tgz", + "integrity": "sha512-0hJU9SCPvmMzIBdZFqNPXWa6dqh7WdH0cII9y+CyS8rG3nL48Bclra9HmKhVVUHyPWNH5Y7xDwAB7bfgSjkUMQ==", "dev": true, "dependencies": { "function-bind": "^1.1.2" @@ -5182,9 +5115,9 @@ } }, "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==", + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.1.0.tgz", + "integrity": "sha512-bVWawvoZoBYpp6yIoQtQXHZjmz35RSVHnUOTefl8Vcjr8snTPY1wnpSPMWekcFwbxI6gtmT7rSYPFvz71ldiOA==", "dev": true, "funding": [ { @@ -5690,9 +5623,9 @@ } }, "node_modules/tar": { - "version": "6.2.0", - "resolved": "https://registry.npmjs.org/tar/-/tar-6.2.0.tgz", - "integrity": "sha512-/Wo7DcT0u5HUV486xg675HtjNd3BXZ6xDbzsCUZPt5iw8bTQ63bP0Raut3mvro9u+CUyq7YQd8Cx55fsZXxqLQ==", + "version": "6.2.1", + "resolved": "https://registry.npmjs.org/tar/-/tar-6.2.1.tgz", + "integrity": "sha512-DZ4yORTwrbTj/7MZYq2w+/ZFdI6OZ/f9SFHR+71gIVUZhOQPHzVCLpvRnPgyaMpfWxxk/4ONva3GQSyNIKRv6A==", "dependencies": { "chownr": "^2.0.0", "fs-minipass": "^2.0.0", @@ -5804,9 +5737,9 @@ } }, "node_modules/typescript": { - "version": "5.4.2", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.2.tgz", - "integrity": "sha512-+2/g0Fds1ERlP6JsakQQDXjZdZMM+rqpamFZJEKh4kwTIn3iDkgKtby0CeNd5ATNZ4Ry1ax15TMx0W2V+miizQ==", + "version": "5.4.3", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.3.tgz", + "integrity": "sha512-KrPd3PKaCLr78MalgiwJnA25Nm8HAmdwN3mYUYZgG/wizIo9EainNVQI9/yDavtVFRN2h3k8uf3GLHuhDMgEHg==", "dev": true, "bin": { "tsc": "bin/tsc", From d6a963856564485bcf3184d78af12ff7dc7e0c92 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 26 Mar 2024 08:41:21 +0530 Subject: [PATCH 108/224] Add AdminAPI for listTopics --- CHANGELOG.md | 9 ++++ index.d.ts | 3 ++ lib/admin.js | 56 ++++++++++++++++++++++ lib/kafkajs/_admin.js | 24 ++++++++++ src/admin.cc | 1 + test/promisified/admin/list_topics.spec.js | 44 +++++++++++++++++ types/kafkajs.d.ts | 1 + 7 files changed, 138 insertions(+) create mode 100644 test/promisified/admin/list_topics.spec.js diff --git a/CHANGELOG.md b/CHANGELOG.md index 4b351196..51ea4e1a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +# confluent-kafka-javascript vNext + +v0.1.12-devel is a pre-production, early-access release. + +## Features + +1. Add support for `listTopics` in the Admin API. + + # confluent-kafka-javascript v0.1.11-devel v0.1.11-devel is a pre-production, early-access release. diff --git a/index.d.ts b/index.d.ts index cbc4d3f5..6cf753c4 100644 --- a/index.d.ts +++ b/index.d.ts @@ -416,6 +416,9 @@ export interface IAdminClient { createPartitions(topic: string, desiredPartitions: number, cb?: (err: LibrdKafkaError) => void): void; createPartitions(topic: string, desiredPartitions: number, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; + listTopics(cb?: (err: LibrdKafkaError, topics: string[]) => any): void; + listTopics(options?: { timeout?: number }, cb?: (err: LibrdKafkaError, topics: string[]) => any): void; + listGroups(cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; listGroups(options?: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }, cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; diff --git a/lib/admin.js b/lib/admin.js index c27cbd6d..a30cf2e6 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -365,3 +365,59 @@ AdminClient.prototype.deleteGroups = function (groups, options, cb) { } }); } + +/** + * List topics. + * + * @param {any?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @param {function} cb - The callback to be executed when finished. + * + * Valid ways to call this function: + * listTopics(cb) + * listTopics(options, cb) + */ +AdminClient.prototype.listTopics = function (options, cb) { + if (!this._isConnected) { + throw new Error('Client is disconnected'); + } + + if (typeof options === 'function') { + cb = options; + options = {}; + } + + if (!options) { + options = {}; + } + + // Always set allTopics to true, since we need a list. + options.allTopics = true; + if (!Object.hasOwn(options, 'timeout')) { + options.timeout = 5000; + } + + // This definitely isn't the fastest way to list topics as + // this makes a pretty large metadata request. But for the sake + // of AdminAPI, this is okay. + this._client.getMetadata(options, function (err, metadata) { + if (err) { + if (cb) { + cb(LibrdKafkaError.create(err)); + } + return; + } + + const topics = [] + if (metadata.topics) { + for (const topic of metadata.topics) { + topics.push(topic.name); + } + } + + if (cb) { + cb(null, topics); + } + }); +} diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 12f1af8d..30b2f7e8 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -307,6 +307,30 @@ class Admin { }); }); } + + /** + * List topics. + * + * @param {any?} options + * @param {number?} options.timeout - The request timeout in milliseconds. + * May be unset (default: 5000) + * @returns {Promise} + */ + async listTopics(options = {}) { + if (this.#state !== AdminState.CONNECTED) { + throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE }); + } + + return new Promise((resolve, reject) => { + this.#internalClient.listTopics(options, (err, topics) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } else { + resolve(topics); + } + }); + }); + } } module.exports = { diff --git a/src/admin.cc b/src/admin.cc index cfedd282..11f04141 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -95,6 +95,7 @@ void AdminClient::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "connect", NodeConnect); Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect); Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); + Nan::SetPrototypeMethod(tpl, "getMetadata", NodeGetMetadata); constructor.Reset( (tpl->GetFunction(Nan::GetCurrentContext())).ToLocalChecked()); diff --git a/test/promisified/admin/list_topics.spec.js b/test/promisified/admin/list_topics.spec.js new file mode 100644 index 00000000..1e0dd638 --- /dev/null +++ b/test/promisified/admin/list_topics.spec.js @@ -0,0 +1,44 @@ +jest.setTimeout(30000); + +const { + secureRandom, + createTopic, + waitFor, + createAdmin, +} = require('../testhelpers'); +const { ConsumerGroupStates, ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Admin > listTopics', () => { + let topicNames, admin; + + beforeEach(async () => { + topicNames = [`test-topic-${secureRandom()}`, `test-topic-${secureRandom()}`]; + + await createTopic({ topic: topicNames[0], }); + await createTopic({ topic: topicNames[1] }); + + admin = createAdmin({}); + }); + + afterEach(async () => { + admin && (await admin.disconnect()); + }); + + it('should timeout', async () => { + await admin.connect(); + + await expect(admin.listTopics({ timeout: 1 })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__TIMED_OUT + ); + }); + + it('should list consumer topics', async () => { + await admin.connect(); + const listTopicsResult = await admin.listTopics(); + expect(listTopicsResult).toEqual( + expect.arrayContaining(topicNames) + ); + }); +}); + diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 64a6e48e..856f6c40 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -541,6 +541,7 @@ export type Admin = { topics: ITopicConfig[] }): Promise deleteTopics(options: { topics: string[]; timeout?: number }): Promise + listTopics(options?: { timeout?: number }): Promise listGroups(options?: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] From ae6525d346479b32c765b7440fc5a09b0db9f1b3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 2 Apr 2024 09:05:52 +0530 Subject: [PATCH 109/224] Make AdminClient inherit from Client --- lib/admin.js | 10 ++++++++-- src/admin.cc | 3 +++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/lib/admin.js b/lib/admin.js index a30cf2e6..a54c817d 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -48,7 +48,9 @@ var Client = require('./client'); var util = require('util'); var Kafka = require('../librdkafka'); var LibrdKafkaError = require('./error'); -var shallowCopy = require('./util').shallowCopy; +var { shallowCopy, bindingVersion } = require('./util'); + +util.inherits(AdminClient, Client); /** * Create a new AdminClient for making topics, partitions, and more. @@ -108,7 +110,7 @@ function AdminClient(conf) { * for the topic. */ - this._client = new Kafka.AdminClient(conf); + Client.call(this, conf, Kafka.AdminClient); this._isConnected = false; this.globalConfig = conf; } @@ -122,6 +124,7 @@ function AdminClient(conf) { * Unlike the other connect methods, this one is synchronous. */ AdminClient.prototype.connect = function () { + this._client.configureCallbacks(true, this._cb_configs); LibrdKafkaError.wrap(this._client.connect(), true); this._isConnected = true; }; @@ -135,6 +138,9 @@ AdminClient.prototype.connect = function () { AdminClient.prototype.disconnect = function () { LibrdKafkaError.wrap(this._client.disconnect(), true); this._isConnected = false; + // The AdminClient doesn't provide a callback. So we can't + // wait for completion. + this._client.configureCallbacks(false, this._cb_configs); }; /** diff --git a/src/admin.cc b/src/admin.cc index 11f04141..2e166f76 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -82,6 +82,9 @@ void AdminClient::Init(v8::Local exports) { tpl->SetClassName(Nan::New("AdminClient").ToLocalChecked()); tpl->InstanceTemplate()->SetInternalFieldCount(1); + // Inherited from NodeKafka::Connection + Nan::SetPrototypeMethod(tpl, "configureCallbacks", NodeConfigureCallbacks); + // Admin client operations Nan::SetPrototypeMethod(tpl, "createTopic", NodeCreateTopic); Nan::SetPrototypeMethod(tpl, "deleteTopic", NodeDeleteTopic); From 98c78e7c682348ee532770ecb6a972e907e0b7ee Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 2 Apr 2024 09:11:48 +0530 Subject: [PATCH 110/224] Fix aliasing bug between NodeKafka::Conf and ConfImpl --- lib/client.js | 2 +- lib/util.js | 2 ++ src/config.cc | 79 +++++++++++++++++++++++++++++---------------------- src/config.h | 16 +++++++++-- 4 files changed, 61 insertions(+), 38 deletions(-) diff --git a/lib/client.js b/lib/client.js index 97950c93..2e219a3c 100644 --- a/lib/client.js +++ b/lib/client.js @@ -15,7 +15,7 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); -const bindingVersion = 'v0.1.11-devel'; +const bindingVersion = require('./util').bindingVersion; var LibrdKafkaError = require('./error'); diff --git a/lib/util.js b/lib/util.js index c8bee360..2956ac2a 100644 --- a/lib/util.js +++ b/lib/util.js @@ -27,3 +27,5 @@ util.shallowCopy = function (obj) { util.isObject = function (obj) { return obj && typeof obj === 'object'; }; + +util.bindingVersion = 'v0.1.11-devel'; diff --git a/src/config.cc b/src/config.cc index b8d883c7..5da7345e 100644 --- a/src/config.cc +++ b/src/config.cc @@ -81,17 +81,8 @@ Conf * Conf::create(RdKafka::Conf::ConfType type, v8::Local object, return NULL; } } else { - v8::Local cb = value.As(); - rdconf->ConfigureCallback(string_key, cb, true, errstr); - if (!errstr.empty()) { - delete rdconf; - return NULL; - } - rdconf->ConfigureCallback(string_key, cb, false, errstr); - if (!errstr.empty()) { - delete rdconf; - return NULL; - } + // Do nothing - Connection::NodeConfigureCallbacks will handle this for each + // of the three client types. } } @@ -100,56 +91,76 @@ Conf * Conf::create(RdKafka::Conf::ConfType type, v8::Local object, void Conf::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add, std::string &errstr) { if (string_key.compare("rebalance_cb") == 0) { + NodeKafka::Callbacks::Rebalance *rebalance = rebalance_cb(); if (add) { - if (this->m_rebalance_cb == NULL) { - this->m_rebalance_cb = new NodeKafka::Callbacks::Rebalance(); + if (rebalance == NULL) { + rebalance = new NodeKafka::Callbacks::Rebalance(); + this->set(string_key, rebalance, errstr); } - this->m_rebalance_cb->dispatcher.AddCallback(cb); - this->set(string_key, this->m_rebalance_cb, errstr); + rebalance->dispatcher.AddCallback(cb); + this->set(string_key, rebalance, errstr); } else { - if (this->m_rebalance_cb != NULL) { - this->m_rebalance_cb->dispatcher.RemoveCallback(cb); + if (rebalance == NULL) { + rebalance->dispatcher.RemoveCallback(cb); + this->set(string_key, rebalance, errstr); } } } else if (string_key.compare("offset_commit_cb") == 0) { + NodeKafka::Callbacks::OffsetCommit *offset_commit = offset_commit_cb(); if (add) { - if (this->m_offset_commit_cb == NULL) { - this->m_offset_commit_cb = new NodeKafka::Callbacks::OffsetCommit(); + if (offset_commit == NULL) { + offset_commit = new NodeKafka::Callbacks::OffsetCommit(); + this->set(string_key, offset_commit, errstr); } - this->m_offset_commit_cb->dispatcher.AddCallback(cb); - this->set(string_key, this->m_offset_commit_cb, errstr); + offset_commit->dispatcher.AddCallback(cb); } else { - if (this->m_offset_commit_cb != NULL) { - this->m_offset_commit_cb->dispatcher.RemoveCallback(cb); + if (offset_commit != NULL) { + offset_commit->dispatcher.RemoveCallback(cb); } } } } void Conf::listen() { - if (m_rebalance_cb) { - m_rebalance_cb->dispatcher.Activate(); + NodeKafka::Callbacks::Rebalance *rebalance = rebalance_cb(); + if (rebalance) { + rebalance->dispatcher.Activate(); } - if (m_offset_commit_cb) { - m_offset_commit_cb->dispatcher.Activate(); + NodeKafka::Callbacks::OffsetCommit *offset_commit = offset_commit_cb(); + if (offset_commit) { + offset_commit->dispatcher.Activate(); } } void Conf::stop() { - if (m_rebalance_cb) { - m_rebalance_cb->dispatcher.Deactivate(); + NodeKafka::Callbacks::Rebalance *rebalance = rebalance_cb(); + if (rebalance) { + rebalance->dispatcher.Deactivate(); + } + + NodeKafka::Callbacks::OffsetCommit *offset_commit = offset_commit_cb(); + if (offset_commit) { + offset_commit->dispatcher.Deactivate(); } +} + +Conf::~Conf() {} - if (m_offset_commit_cb) { - m_offset_commit_cb->dispatcher.Deactivate(); +NodeKafka::Callbacks::Rebalance* Conf::rebalance_cb() const { + RdKafka::RebalanceCb *cb = NULL; + if (this->get(cb) != RdKafka::Conf::CONF_OK) { + return NULL; } + return static_cast(cb); } -Conf::~Conf() { - if (m_rebalance_cb) { - delete m_rebalance_cb; +NodeKafka::Callbacks::OffsetCommit* Conf::offset_commit_cb() const { + RdKafka::OffsetCommitCb *cb = NULL; + if (this->get(cb) != RdKafka::Conf::CONF_OK) { + return NULL; } + return static_cast(cb); } } // namespace NodeKafka diff --git a/src/config.h b/src/config.h index cea57302..a10c9535 100644 --- a/src/config.h +++ b/src/config.h @@ -33,9 +33,19 @@ class Conf : public RdKafka::Conf { void stop(); void ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add, std::string &errstr); - protected: - NodeKafka::Callbacks::Rebalance * m_rebalance_cb = NULL; - NodeKafka::Callbacks::OffsetCommit * m_offset_commit_cb = NULL; + + private: + NodeKafka::Callbacks::Rebalance* rebalance_cb() const; + NodeKafka::Callbacks::OffsetCommit *offset_commit_cb() const; + + // NOTE: Do NOT add any members to this class. + // Internally, to get an instance of this class, we just cast RdKafka::Conf* that we + // obtain from RdKafka::Conf::create(). However, that's internally an instance of a sub-class, + // ConfImpl. + // This means that any members here are aliased to that with the wrong name (for example, the first + // member of this class, if it's a pointer, will be aliased to consume_cb_ in the ConfImpl, and + // and changing one will change the other!) + // TODO: Just don't inherit from RdKafka::Conf, and instead have a member of type RdKafka::Conf*. }; } // namespace NodeKafka From 0629955f5b463b1d8020444f07c45a65d5efa3dc Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 2 Apr 2024 09:28:52 +0530 Subject: [PATCH 111/224] Fix issue with cooperative assignor when rebalance_cb is a boolean --- lib/kafka-consumer.js | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 88c07a84..48ee30f7 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -62,9 +62,17 @@ function KafkaConsumer(conf, topicConf) { // That's it try { if (err.code === -175 /*ERR__ASSIGN_PARTITIONS*/) { - self.assign(assignment); + if (self.rebalanceProtocol() === 'COOPERATIVE') { + self.incrementalAssign(assignment); + } else { + self.assign(assignment); + } } else if (err.code === -174 /*ERR__REVOKE_PARTITIONS*/) { - self.unassign(); + if (self.rebalanceProtocol() === 'COOPERATIVE') { + self.incrementalUnassign(assignment); + } else { + self.unassign(); + } } } catch (e) { // Ignore exceptions if we are not connected From 85e60d8d774647e3a39ef59da53eb3df364bcfc1 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 4 Apr 2024 15:29:23 +0530 Subject: [PATCH 112/224] Fix memory leak: dispatcher async handle --- src/callbacks.cc | 8 +++++++- src/callbacks.h | 1 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/callbacks.cc b/src/callbacks.cc index e3d18e22..00462268 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -74,10 +74,16 @@ void Dispatcher::Activate() { } } +void Dispatcher::AsyncHandleCloseCallback(uv_handle_t *handle) { + uv_async_t *a = reinterpret_cast(handle); + delete a; +} + // Should be able to run this regardless of whether it is active or not void Dispatcher::Deactivate() { if (async) { - uv_close(reinterpret_cast(async), NULL); + uv_close(reinterpret_cast(async), + Dispatcher::AsyncHandleCloseCallback); async = NULL; } } diff --git a/src/callbacks.h b/src/callbacks.h index f244d8c1..34dfec11 100644 --- a/src/callbacks.h +++ b/src/callbacks.h @@ -53,6 +53,7 @@ class Dispatcher { static_cast(async->data); dispatcher->Flush(); } + static void AsyncHandleCloseCallback(uv_handle_t *); uv_async_t *async; }; From fd13955ed1dc61e92c11d37a74c443615731540c Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 4 Apr 2024 15:29:45 +0530 Subject: [PATCH 113/224] Fix memory leak: ConfImpl within Conf --- src/config.cc | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/config.cc b/src/config.cc index 5da7345e..0c980d7c 100644 --- a/src/config.cc +++ b/src/config.cc @@ -145,7 +145,11 @@ void Conf::stop() { } } -Conf::~Conf() {} +Conf::~Conf() { + // Delete the rdconf object, since that's what we are internally. + RdKafka::Conf *rdconf = static_cast(this); + delete rdconf; +} NodeKafka::Callbacks::Rebalance* Conf::rebalance_cb() const { RdKafka::RebalanceCb *cb = NULL; From 34e4bd3ddf744839d1a722be884ab67ba4df3386 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 5 Apr 2024 11:36:20 +0530 Subject: [PATCH 114/224] Add OAUTHBEARER token refresh callback --- lib/client.js | 36 ++++++++++- lib/util.js | 24 +++++++ src/admin.cc | 28 +++++++- src/callbacks.cc | 33 +++++++++- src/callbacks.h | 17 +++++ src/common.cc | 20 ++++++ src/common.h | 1 + src/config.cc | 50 ++++++++++++++- src/config.h | 21 +++--- src/connection.cc | 144 ++++++++++++++++++++++++++++++++++++++++++ src/connection.h | 10 ++- src/kafka-consumer.cc | 14 ++++ src/producer.cc | 15 ++++- src/workers.cc | 18 ++++-- 14 files changed, 409 insertions(+), 22 deletions(-) diff --git a/lib/client.js b/lib/client.js index 2e219a3c..9f13d93d 100644 --- a/lib/client.js +++ b/lib/client.js @@ -15,7 +15,7 @@ var util = require('util'); var Kafka = require('../librdkafka.js'); var assert = require('assert'); -const bindingVersion = require('./util').bindingVersion; +const { bindingVersion, dictToStringList } = require('./util'); var LibrdKafkaError = require('./error'); @@ -100,6 +100,40 @@ function Client(globalConf, SubClientType, topicConf) { }.bind(this); } + if (Object.hasOwn(this._cb_configs.global, 'oauthbearer_token_refresh_cb')) { + const savedCallback = this._cb_configs.global.oauthbearer_token_refresh_cb; + this._cb_configs.global.oauthbearer_token_refresh_cb = (oauthbearer_config) => { + if (this._isDisconnecting) { + // Don't call the callback if we're in the middle of disconnecting. + // This is especially important when the credentials are wrong, and + // we might want to disconnect without ever completing connection. + return; + } + savedCallback(oauthbearer_config, (err, token) => { + try { + if (err) { + throw err; + } + let { tokenValue, lifetime, principal, extensions } = token; + + // If the principal isn't there, set an empty principal. + if (!principal) { + principal = ''; + } + + // Convert extensions from a Map/object to a list that librdkafka expects. + extensions = dictToStringList(extensions); + + this._client.setOAuthBearerToken(tokenValue, lifetime, principal, extensions); + } catch (e) { + e.message = "oauthbearer_token_refresh_cb: " + e.message; + this._client.setOAuthBearerTokenFailure(e.message); + this.emit('error', e); + } + }); + } + } + this.metrics = {}; this._isConnected = false; this.errorCounter = 0; diff --git a/lib/util.js b/lib/util.js index 2956ac2a..e33d0e00 100644 --- a/lib/util.js +++ b/lib/util.js @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. @@ -28,4 +29,27 @@ util.isObject = function (obj) { return obj && typeof obj === 'object'; }; +// Convert Map or object to a list of [key, value, key, value...]. +util.dictToStringList = function (mapOrObject) { + let list = null; + if (mapOrObject && (mapOrObject instanceof Map)) { + list = + Array + .from(mapOrObject).reduce((acc, [key, value]) => { + acc.push(key, value); + return acc; + }, []) + .map(v => String(v)); + } else if (util.isObject(mapOrObject)) { + list = + Object + .entries(mapOrObject).reduce((acc, [key, value]) => { + acc.push(key, value); + return acc; + }, []) + .map(v => String(v)); + } + return list; +}; + util.bindingVersion = 'v0.1.11-devel'; diff --git a/src/admin.cc b/src/admin.cc index 2e166f76..5a364e6b 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -39,14 +39,28 @@ AdminClient::~AdminClient() { } Baton AdminClient::Connect() { - std::string errstr; + if (IsConnected()) { + return Baton(RdKafka::ERR_NO_ERROR); + } + + Baton baton = setupSaslOAuthBearerConfig(); + if (baton.err() != RdKafka::ERR_NO_ERROR) { + return baton; + } + // Activate the dispatchers before the connection, as some callbacks may run + // on the background thread. + // We will deactivate them if the connection fails. + ActivateDispatchers(); + + std::string errstr; { scoped_shared_write_lock lock(m_connection_lock); m_client = RdKafka::Producer::create(m_gconfig, errstr); } if (!m_client || !errstr.empty()) { + DeactivateDispatchers(); return Baton(RdKafka::ERR__STATE, errstr); } @@ -54,7 +68,12 @@ Baton AdminClient::Connect() { rkqu = rd_kafka_queue_new(m_client->c_ptr()); } - return Baton(RdKafka::ERR_NO_ERROR); + baton = setupSaslOAuthBearerBackgroundQueue(); + if (baton.err() != RdKafka::ERR_NO_ERROR) { + DeactivateDispatchers(); + } + + return baton; } Baton AdminClient::Disconnect() { @@ -66,6 +85,8 @@ Baton AdminClient::Disconnect() { rkqu = NULL; } + DeactivateDispatchers(); + delete m_client; m_client = NULL; } @@ -99,6 +120,9 @@ void AdminClient::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect); Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); Nan::SetPrototypeMethod(tpl, "getMetadata", NodeGetMetadata); + Nan::SetPrototypeMethod(tpl, "setOAuthBearerToken", NodeSetOAuthBearerToken); + Nan::SetPrototypeMethod(tpl, "setOAuthBearerTokenFailure", + NodeSetOAuthBearerTokenFailure); constructor.Reset( (tpl->GetFunction(Nan::GetCurrentContext())).ToLocalChecked()); diff --git a/src/callbacks.cc b/src/callbacks.cc index 00462268..a3a90148 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -10,9 +10,7 @@ #include #include -#include -#include "src/callbacks.h" #include "src/kafka-consumer.h" using v8::Local; @@ -547,6 +545,37 @@ void OffsetCommit::offset_commit_cb(RdKafka::ErrorCode err, dispatcher.Execute(); } +// OAuthBearerTokenRefresh callback +void OAuthBearerTokenRefreshDispatcher::Add( + const std::string &oauthbearer_config) { + scoped_mutex_lock lock(async_lock); + m_oauthbearer_config = oauthbearer_config; +} + +void OAuthBearerTokenRefreshDispatcher::Flush() { + Nan::HandleScope scope; + + const unsigned int argc = 1; + + std::string oauthbearer_config; + { + scoped_mutex_lock lock(async_lock); + oauthbearer_config = m_oauthbearer_config; + m_oauthbearer_config.clear(); + } + + v8::Local argv[argc] = {}; + argv[0] = Nan::New(oauthbearer_config.c_str()).ToLocalChecked(); + + Dispatch(argc, argv); +} + +void OAuthBearerTokenRefresh::oauthbearer_token_refresh_cb( + RdKafka::Handle *handle, const std::string &oauthbearer_config) { + dispatcher.Add(oauthbearer_config); + dispatcher.Execute(); +} + // Partitioner callback Partitioner::Partitioner() {} diff --git a/src/callbacks.h b/src/callbacks.h index 34dfec11..315a1df6 100644 --- a/src/callbacks.h +++ b/src/callbacks.h @@ -248,6 +248,23 @@ class OffsetCommit : public RdKafka::OffsetCommitCb { v8::Persistent m_cb; }; +class OAuthBearerTokenRefreshDispatcher : public Dispatcher { + public: + OAuthBearerTokenRefreshDispatcher(){}; + ~OAuthBearerTokenRefreshDispatcher(){}; + void Add(const std::string &oauthbearer_config); + void Flush(); + + private: + std::string m_oauthbearer_config; +}; + +class OAuthBearerTokenRefresh : public RdKafka::OAuthBearerTokenRefreshCb { + public: + void oauthbearer_token_refresh_cb(RdKafka::Handle *, const std::string &); + OAuthBearerTokenRefreshDispatcher dispatcher; +}; + class Partitioner : public RdKafka::PartitionerCb { public: Partitioner(); diff --git a/src/common.cc b/src/common.cc index 62542bf1..fe0b3894 100644 --- a/src/common.cc +++ b/src/common.cc @@ -143,6 +143,26 @@ std::vector v8ArrayToStringVector(v8::Local parameter) { return newItem; } +std::list v8ArrayToStringList(v8::Local parameter) { + std::list newItem; + if (parameter->Length() >= 1) { + for (unsigned int i = 0; i < parameter->Length(); i++) { + v8::Local v; + if (!Nan::Get(parameter, i).ToLocal(&v)) { + continue; + } + Nan::MaybeLocal p = Nan::To(v); + if (p.IsEmpty()) { + continue; + } + Nan::Utf8String pVal(p.ToLocalChecked()); + std::string pString(*pVal); + newItem.push_back(pString); + } + } + return newItem; +} + template<> v8::Local GetParameter >( v8::Local object, std::string field_name, v8::Local def) { v8::Local field = Nan::New(field_name.c_str()).ToLocalChecked(); diff --git a/src/common.h b/src/common.h index cccaef60..c8c979dc 100644 --- a/src/common.h +++ b/src/common.h @@ -39,6 +39,7 @@ template<> v8::Local GetParameter >( v8::Local, std::string, v8::Local); // template int GetParameter(v8::Local v8ArrayToStringVector(v8::Local); +std::list v8ArrayToStringList(v8::Local); class scoped_mutex_lock { public: diff --git a/src/config.cc b/src/config.cc index 0c980d7c..a781fdf7 100644 --- a/src/config.cc +++ b/src/config.cc @@ -81,8 +81,8 @@ Conf * Conf::create(RdKafka::Conf::ConfType type, v8::Local object, return NULL; } } else { - // Do nothing - Connection::NodeConfigureCallbacks will handle this for each - // of the three client types. + // Do nothing - NodeConfigureCallbacks will handle this for each + // of the three client types, called from within JavaScript. } } @@ -118,6 +118,23 @@ void Conf::ConfigureCallback(const std::string &string_key, const v8::Localdispatcher.RemoveCallback(cb); } } + } else if (string_key.compare("oauthbearer_token_refresh_cb") == 0) { + NodeKafka::Callbacks::OAuthBearerTokenRefresh *oauthbearer_token_refresh = + oauthbearer_token_refresh_cb(); + if (add) { + if (oauthbearer_token_refresh == NULL) { + oauthbearer_token_refresh = + new NodeKafka::Callbacks::OAuthBearerTokenRefresh(); + this->set(string_key, oauthbearer_token_refresh, errstr); + } + oauthbearer_token_refresh->dispatcher.AddCallback(cb); + } else { + if (oauthbearer_token_refresh != NULL) { + oauthbearer_token_refresh->dispatcher.RemoveCallback(cb); + } + } + } else { + errstr = "Invalid callback type"; } } @@ -131,6 +148,12 @@ void Conf::listen() { if (offset_commit) { offset_commit->dispatcher.Activate(); } + + NodeKafka::Callbacks::OAuthBearerTokenRefresh *oauthbearer_token_refresh = + oauthbearer_token_refresh_cb(); + if (oauthbearer_token_refresh) { + oauthbearer_token_refresh->dispatcher.Activate(); + } } void Conf::stop() { @@ -143,6 +166,12 @@ void Conf::stop() { if (offset_commit) { offset_commit->dispatcher.Deactivate(); } + + NodeKafka::Callbacks::OAuthBearerTokenRefresh *oauthbearer_token_refresh = + oauthbearer_token_refresh_cb(); + if (oauthbearer_token_refresh) { + oauthbearer_token_refresh->dispatcher.Deactivate(); + } } Conf::~Conf() { @@ -167,4 +196,21 @@ NodeKafka::Callbacks::OffsetCommit* Conf::offset_commit_cb() const { return static_cast(cb); } +NodeKafka::Callbacks::OAuthBearerTokenRefresh * +Conf::oauthbearer_token_refresh_cb() const { + RdKafka::OAuthBearerTokenRefreshCb *cb = NULL; + if (this->get(cb) != RdKafka::Conf::CONF_OK) { + return NULL; + } + return static_cast(cb); +} + +bool Conf::is_sasl_oauthbearer() const { + std::string sasl_mechanism; + if (this->get("sasl.mechanisms", sasl_mechanism) != RdKafka::Conf::CONF_OK) { + return false; + } + return sasl_mechanism.compare("OAUTHBEARER") == 0; +} + } // namespace NodeKafka diff --git a/src/config.h b/src/config.h index a10c9535..f0ffdba4 100644 --- a/src/config.h +++ b/src/config.h @@ -34,18 +34,23 @@ class Conf : public RdKafka::Conf { void ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add, std::string &errstr); + bool is_sasl_oauthbearer() const; + private: - NodeKafka::Callbacks::Rebalance* rebalance_cb() const; + NodeKafka::Callbacks::Rebalance *rebalance_cb() const; NodeKafka::Callbacks::OffsetCommit *offset_commit_cb() const; + NodeKafka::Callbacks::OAuthBearerTokenRefresh *oauthbearer_token_refresh_cb() + const; // NOTE: Do NOT add any members to this class. - // Internally, to get an instance of this class, we just cast RdKafka::Conf* that we - // obtain from RdKafka::Conf::create(). However, that's internally an instance of a sub-class, - // ConfImpl. - // This means that any members here are aliased to that with the wrong name (for example, the first - // member of this class, if it's a pointer, will be aliased to consume_cb_ in the ConfImpl, and - // and changing one will change the other!) - // TODO: Just don't inherit from RdKafka::Conf, and instead have a member of type RdKafka::Conf*. + // Internally, to get an instance of this class, we just cast RdKafka::Conf* + // that we obtain from RdKafka::Conf::create(). However, that's internally an + // instance of a sub-class, ConfImpl. This means that any members here are + // aliased to that with the wrong name (for example, the first member of this + // class, if it's a pointer, will be aliased to consume_cb_ in the ConfImpl, + // and and changing one will change the other!) + // TODO: Just don't inherit from RdKafka::Conf, and instead have a member of + // type RdKafka::Conf*. }; } // namespace NodeKafka diff --git a/src/connection.cc b/src/connection.cc index 67aba7f0..12697dc3 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -80,6 +80,34 @@ Baton Connection::rdkafkaErrorToBaton(RdKafka::Error* error) { } } +// If OAUTHBEARER authentication is set up, then push the callbacks onto the +// SASL queue so we don't need to keep polling. This method should be called +// before the client is created. +Baton Connection::setupSaslOAuthBearerConfig() { + if (!m_gconfig->is_sasl_oauthbearer()) { + return Baton(RdKafka::ERR_NO_ERROR); + } + + std::string errstr; + if (m_gconfig->enable_sasl_queue(true, errstr) != RdKafka::Conf::CONF_OK) { + return Baton(RdKafka::ERR__STATE, errstr); + } + + return Baton(RdKafka::ERR_NO_ERROR); +} + +// If OAUTHBEARER authentication is set up, then handle the callbacks on +// the background thread. This method should be called after the client is +// created and only if `setupSaslOAuthBearerConfig` is called earlier. +Baton Connection::setupSaslOAuthBearerBackgroundQueue() { + if (!m_gconfig->is_sasl_oauthbearer()) { + return Baton(RdKafka::ERR_NO_ERROR); + } + + RdKafka::Error* error = m_client->sasl_background_callbacks_enable(); + return rdkafkaErrorToBaton(error); +} + RdKafka::TopicPartition* Connection::GetPartition(std::string &topic) { return RdKafka::TopicPartition::create(topic, RdKafka::Topic::PARTITION_UA); } @@ -247,6 +275,49 @@ Baton Connection::SetSaslCredentials( return rdkafkaErrorToBaton(error); } +Baton Connection::SetOAuthBearerToken( + const std::string& value, int64_t lifetime_ms, + const std::string& principal_name, + const std::list& extensions) { + RdKafka::ErrorCode error_code; + std::string errstr; + + if (IsConnected()) { + scoped_shared_read_lock lock(m_connection_lock); + if (IsConnected()) { + error_code = m_client->oauthbearer_set_token( + value, lifetime_ms, principal_name, extensions, errstr); + } else { + return Baton(RdKafka::ERR__STATE); + } + } else { + return Baton(RdKafka::ERR__STATE); + } + + if (error_code != RdKafka::ERR_NO_ERROR) { + return Baton(error_code, errstr); + } + + return Baton(error_code); +} + +Baton Connection::SetOAuthBearerTokenFailure(const std::string& errstr) { + RdKafka::ErrorCode error_code; + + if (IsConnected()) { + scoped_shared_read_lock lock(m_connection_lock); + if (IsConnected()) { + error_code = m_client->oauthbearer_set_token_failure(errstr); + } else { + return Baton(RdKafka::ERR__STATE); + } + } else { + return Baton(RdKafka::ERR__STATE); + } + + return Baton(error_code); +} + void Connection::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add) { if (string_key.compare("event_cb") == 0) { if (add) { @@ -485,4 +556,77 @@ NAN_METHOD(Connection::NodeConfigureCallbacks) { info.GetReturnValue().Set(Nan::True()); } +NAN_METHOD(Connection::NodeSetOAuthBearerToken) { + if (!info[0]->IsString()) { + Nan::ThrowError("1st parameter must be a token string"); + return; + } + + if (!info[1]->IsNumber()) { + Nan::ThrowError("2nd parameter must be a lifetime_ms number"); + return; + } + + if (!info[2]->IsString()) { + Nan::ThrowError("3rd parameter must be a principal_name string"); + return; + } + + if (!info[3]->IsNullOrUndefined() && !info[3]->IsArray()) { + Nan::ThrowError("4th parameter must be an extensions array or null"); + return; + } + + // Get string pointer for the token + Nan::Utf8String tokenUtf8(Nan::To(info[0]).ToLocalChecked()); + std::string token(*tokenUtf8); + + // Get the lifetime_ms + int64_t lifetime_ms = Nan::To(info[1]).FromJust(); + + // Get string pointer for the principal_name + Nan::Utf8String principal_nameUtf8( + Nan::To(info[2]).ToLocalChecked()); + std::string principal_name(*principal_nameUtf8); + + // Get the extensions (if any) + std::list extensions; + if (!info[3]->IsNullOrUndefined()) { + v8::Local extensionsArray = info[3].As(); + extensions = v8ArrayToStringList(extensionsArray); + } + + Connection* obj = ObjectWrap::Unwrap(info.This()); + Baton b = + obj->SetOAuthBearerToken(token, lifetime_ms, principal_name, extensions); + + if (b.err() != RdKafka::ERR_NO_ERROR) { + v8::Local errorObject = b.ToObject(); + return Nan::ThrowError(errorObject); + } + + info.GetReturnValue().Set(Nan::Null()); +} + +NAN_METHOD(Connection::NodeSetOAuthBearerTokenFailure) { + if (!info[0]->IsString()) { + Nan::ThrowError("1st parameter must be an error string"); + return; + } + + // Get string pointer for the error string + Nan::Utf8String errstrUtf8(Nan::To(info[0]).ToLocalChecked()); + std::string errstr(*errstrUtf8); + + Connection* obj = ObjectWrap::Unwrap(info.This()); + Baton b = obj->SetOAuthBearerTokenFailure(errstr); + + if (b.err() != RdKafka::ERR_NO_ERROR) { + v8::Local errorObject = b.ToObject(); + return Nan::ThrowError(errorObject); + } + + info.GetReturnValue().Set(Nan::Null()); +} + } // namespace NodeKafka diff --git a/src/connection.h b/src/connection.h index 0ad9cea3..47fe20b3 100644 --- a/src/connection.h +++ b/src/connection.h @@ -57,6 +57,9 @@ class Connection : public Nan::ObjectWrap { Baton QueryWatermarkOffsets(std::string, int32_t, int64_t*, int64_t*, int); Baton OffsetsForTimes(std::vector &, int); Baton SetSaslCredentials(std::string, std::string); + Baton SetOAuthBearerToken(const std::string&, int64_t, const std::string&, + const std::list&); + Baton SetOAuthBearerTokenFailure(const std::string&); RdKafka::Handle* GetClient(); @@ -76,7 +79,10 @@ class Connection : public Nan::ObjectWrap { static Nan::Persistent constructor; static void New(const Nan::FunctionCallbackInfo& info); - Baton rdkafkaErrorToBaton(RdKafka::Error* error); + static Baton rdkafkaErrorToBaton(RdKafka::Error* error); + + Baton setupSaslOAuthBearerConfig(); + Baton setupSaslOAuthBearerBackgroundQueue(); bool m_has_been_disconnected; bool m_is_closing; @@ -94,6 +100,8 @@ class Connection : public Nan::ObjectWrap { static NAN_METHOD(NodeQueryWatermarkOffsets); static NAN_METHOD(NodeOffsetsForTimes); static NAN_METHOD(NodeSetSaslCredentials); + static NAN_METHOD(NodeSetOAuthBearerToken); + static NAN_METHOD(NodeSetOAuthBearerTokenFailure); }; } // namespace NodeKafka diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 3d519c5d..1a05abe1 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -48,6 +48,11 @@ Baton KafkaConsumer::Connect() { return Baton(RdKafka::ERR_NO_ERROR); } + Baton baton = setupSaslOAuthBearerConfig(); + if (baton.err() != RdKafka::ERR_NO_ERROR) { + return baton; + } + std::string errstr; { scoped_shared_write_lock lock(m_connection_lock); @@ -58,6 +63,11 @@ Baton KafkaConsumer::Connect() { return Baton(RdKafka::ERR__STATE, errstr); } + baton = setupSaslOAuthBearerBackgroundQueue(); + if (baton.err() != RdKafka::ERR_NO_ERROR) { + return baton; + } + if (m_partitions.size() > 0) { m_client->resume(m_partitions); } @@ -569,6 +579,9 @@ void KafkaConsumer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "offsetsForTimes", NodeOffsetsForTimes); Nan::SetPrototypeMethod(tpl, "getWatermarkOffsets", NodeGetWatermarkOffsets); Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); + Nan::SetPrototypeMethod(tpl, "setOAuthBearerToken", NodeSetOAuthBearerToken); + Nan::SetPrototypeMethod(tpl, "setOAuthBearerTokenFailure", + NodeSetOAuthBearerTokenFailure); /* * @brief Methods exposed to do with message retrieval @@ -644,6 +657,7 @@ void KafkaConsumer::New(const Nan::FunctionCallbackInfo& info) { } } + // TODO: fix this - this memory is leaked. KafkaConsumer* consumer = new KafkaConsumer(gconfig, tconfig); // Wrap it diff --git a/src/producer.cc b/src/producer.cc index 2349efe3..82da58d7 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -73,6 +73,9 @@ void Producer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "queryWatermarkOffsets", NodeQueryWatermarkOffsets); // NOLINT Nan::SetPrototypeMethod(tpl, "poll", NodePoll); Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); + Nan::SetPrototypeMethod(tpl, "setOAuthBearerToken", NodeSetOAuthBearerToken); + Nan::SetPrototypeMethod(tpl, "setOAuthBearerTokenFailure", + NodeSetOAuthBearerTokenFailure); /* * @brief Methods exposed to do with message production @@ -176,25 +179,33 @@ Baton Producer::Connect() { } std::string errstr; + + Baton baton = setupSaslOAuthBearerConfig(); + if (baton.err() != RdKafka::ERR_NO_ERROR) { + return baton; + } + { scoped_shared_read_lock lock(m_connection_lock); m_client = RdKafka::Producer::create(m_gconfig, errstr); } if (!m_client) { - // @todo implement errstr into this somehow return Baton(RdKafka::ERR__STATE, errstr); } - return Baton(RdKafka::ERR_NO_ERROR); + baton = setupSaslOAuthBearerBackgroundQueue(); + return baton; } void Producer::ActivateDispatchers() { + m_gconfig->listen(); // From global config. m_event_cb.dispatcher.Activate(); // From connection m_dr_cb.dispatcher.Activate(); } void Producer::DeactivateDispatchers() { + m_gconfig->stop(); // From global config. m_event_cb.dispatcher.Deactivate(); // From connection m_dr_cb.dispatcher.Deactivate(); } diff --git a/src/workers.cc b/src/workers.cc index d414e4dc..c5f5b757 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -200,6 +200,11 @@ ProducerConnect::ProducerConnect(Nan::Callback *callback, Producer* producer): ProducerConnect::~ProducerConnect() {} void ProducerConnect::Execute() { + // Activate the dispatchers before the connection, as some callbacks may run + // on the background thread. + // We will deactivate them if the connection fails. + producer->ActivateDispatchers(); + Baton b = producer->Connect(); if (b.err() != RdKafka::ERR_NO_ERROR) { @@ -218,15 +223,14 @@ void ProducerConnect::HandleOKCallback() { v8::Local argv[argc] = { Nan::Null(), obj}; - // Activate the dispatchers - producer->ActivateDispatchers(); - callback->Call(argc, argv); } void ProducerConnect::HandleErrorCallback() { Nan::HandleScope scope; + producer->DeactivateDispatchers(); + const unsigned int argc = 1; v8::Local argv[argc] = { GetErrorObject() }; @@ -558,6 +562,11 @@ KafkaConsumerConnect::KafkaConsumerConnect(Nan::Callback *callback, KafkaConsumerConnect::~KafkaConsumerConnect() {} void KafkaConsumerConnect::Execute() { + // Activate the dispatchers before the connection, as some callbacks may run + // on the background thread. + // We will deactivate them if the connection fails. + consumer->ActivateDispatchers(); + Baton b = consumer->Connect(); // consumer->Wait(); @@ -577,7 +586,6 @@ void KafkaConsumerConnect::HandleOKCallback() { Nan::New(consumer->Name()).ToLocalChecked()); v8::Local argv[argc] = { Nan::Null(), obj }; - consumer->ActivateDispatchers(); callback->Call(argc, argv); } @@ -585,6 +593,8 @@ void KafkaConsumerConnect::HandleOKCallback() { void KafkaConsumerConnect::HandleErrorCallback() { Nan::HandleScope scope; + consumer->DeactivateDispatchers(); + const unsigned int argc = 1; v8::Local argv[argc] = { Nan::Error(ErrorMessage()) }; From 94760606bbe167020793dc1d411841c39d7eda49 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 5 Apr 2024 11:37:06 +0530 Subject: [PATCH 115/224] Add examples and tests for non-promisified OAUTHBEARER token refresh callback --- e2e/oauthbearer_cb.spec.js | 82 ++++++++ .../oauthbearer_callback_authentication.js | 82 ++++++++ .../package-lock.json | 187 ++++++++++++++++++ .../package.json | 15 ++ 4 files changed, 366 insertions(+) create mode 100644 e2e/oauthbearer_cb.spec.js create mode 100644 examples/node-rdkafka/oauthbearer_callback_authentication/oauthbearer_callback_authentication.js create mode 100644 examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json create mode 100644 examples/node-rdkafka/oauthbearer_callback_authentication/package.json diff --git a/e2e/oauthbearer_cb.spec.js b/e2e/oauthbearer_cb.spec.js new file mode 100644 index 00000000..100599a9 --- /dev/null +++ b/e2e/oauthbearer_cb.spec.js @@ -0,0 +1,82 @@ +/* + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library + * + * Copyright (c) 2024 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +var Kafka = require('../'); +var t = require('assert'); + +var eventListener = require('./listener'); + +var kafkaBrokerList = process.env.KAFKA_HOST || 'localhost:9092'; + +describe('Client with oauthbearer_cb', function () { + const oauthbearer_config = 'key=value'; + let oauthbearer_cb_called = 0; + let oauthbearer_cb = function (config, cb) { + console.log("Called oauthbearer_cb with given config: " + config); + t.equal(config, oauthbearer_config); + oauthbearer_cb_called++; + + // The broker is not expected to be configured for oauthbearer authentication. + // We just want to make sure that token refresh callback is triggered. + cb(new Error('oauthbearer_cb error'), null); + }; + + const commonConfig = { + 'metadata.broker.list': kafkaBrokerList, + 'debug': 'all', + 'security.protocol': 'SASL_PLAINTEXT', + 'sasl.mechanisms': 'OAUTHBEARER', + 'oauthbearer_token_refresh_cb': oauthbearer_cb, + 'sasl.oauthbearer.config': oauthbearer_config, + } + + const checkClient = function (client, done) { + eventListener(client); + + client.on('error', function (e) { + t.match(e.message, /oauthbearer_cb error/); + }); + + client.connect(); + + // We don't actually expect the connection to succeed, but we want to + // make sure that the oauthbearer_cb is called so give it a couple seconds. + setTimeout(() => { + client.disconnect(); + client = null; + t.equal(oauthbearer_cb_called >= 1, true); + done(); + }, 2000); + } + + beforeEach(function (done) { + oauthbearer_cb_called = 0; + done(); + }); + + it('as producer', function (done) { + let producer = new Kafka.Producer(commonConfig); + checkClient(producer, done); + producer = null; + }).timeout(2500); + + it('as consumer', function (done) { + const config = Object.assign({ 'group.id': 'gid' }, commonConfig); + let consumer = new Kafka.KafkaConsumer(config); + checkClient(consumer, done); + consumer = null; + }).timeout(2500); + + it('as admin', function (done) { + let admin = new Kafka.AdminClient.create(commonConfig); + checkClient(admin, done); + admin = null; + }).timeout(2500); + +}); diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/oauthbearer_callback_authentication.js b/examples/node-rdkafka/oauthbearer_callback_authentication/oauthbearer_callback_authentication.js new file mode 100644 index 00000000..e587b506 --- /dev/null +++ b/examples/node-rdkafka/oauthbearer_callback_authentication/oauthbearer_callback_authentication.js @@ -0,0 +1,82 @@ +const Kafka = require('@confluentinc/kafka-javascript'); +var jwt = require('jsonwebtoken'); + +// This example uses the Producer for demonstration purposes. +// It is the same whether you use a Consumer/AdminClient. + +function token_refresh(oauthbearer_config /* string - passed from config */, cb) { + console.log("Called token_refresh with given config: " + oauthbearer_config); + // At this point, we can use the information in the token, make + // some API calls, fetch something from a file... + // For the illustration, everything is hard-coded. + const principal = 'admin'; + // In seconds - needed by jsonwebtoken library + const exp_seconds = Math.floor(Date.now() / 1000) + (60 * 60); + // In milliseconds - needed by kafka-javascript. + const exp_ms = exp_seconds * 1000; + + // For illustration, we're not signing our JWT (algorithm: none). + // For production uses-cases, it should be signed. + const tokenValue = jwt.sign( + { 'sub': principal, exp: exp_seconds, 'scope': 'requiredScope' }, '', { algorithm: 'none' }); + + // SASL extensions can be passed as Map or key/value pairs in an object. + const extensions = { + traceId: '123' + }; + + // The callback is called with the new token, its lifetime, and the principal. + // The extensions are optional and may be omitted. + console.log("Finished token_refresh, triggering callback: with tokenValue: " + + tokenValue.slice(0, 10) + "..., lifetime: " + exp_ms + + ", principal: " + principal + ", extensions: " + JSON.stringify(extensions)); + cb( + // If no token could be fetched or an error occurred, a new Error can be + // and passed as the first parameter and the second parameter omitted. + null, + { tokenValue, lifetime: exp_ms, principal, extensions }); +} + +function run() { + const producer = new Kafka.Producer({ + 'metadata.broker.list': 'localhost:60125', + 'dr_cb': true, + // 'debug': 'all' + + // Config important for OAUTHBEARER: + 'security.protocol': 'SASL_PLAINTEXT', + 'sasl.mechanisms': 'OAUTHBEARER', + 'sasl.oauthbearer.config': 'someConfigPropertiesKey=value', + 'oauthbearer_token_refresh_cb': token_refresh, + }); + + producer.connect(); + + producer.on('event.log', (event) => { + console.log(event); + }); + + producer.on('ready', () => { + console.log('Producer is ready!'); + producer.setPollInterval(1000); + console.log("Producing message."); + producer.produce( + 'topic', + null, // partition - let partitioner choose + Buffer.from('messageValue'), + 'messageKey', + ); + }); + + producer.on('error', (err) => { + console.error("Encountered error in producer: " + err.message); + }); + + producer.on('delivery-report', function (err, report) { + console.log('delivery-report: ' + JSON.stringify(report)); + // since we just want to produce one message, close shop. + producer.disconnect(); + }); +} + +run(); diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json new file mode 100644 index 00000000..d3d70cce --- /dev/null +++ b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json @@ -0,0 +1,187 @@ +{ + "name": "oauthbearer_callback_authentication", + "version": "1.0.0", + "lockfileVersion": 3, + "requires": true, + "packages": { + "": { + "name": "oauthbearer_callback_authentication", + "version": "1.0.0", + "license": "MIT", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../../..", + "jsonwebtoken": "^9.0.2" + } + }, + "../../..": { + "version": "v0.1.11-devel", + "hasInstallScript": true, + "license": "MIT", + "dependencies": { + "@mapbox/node-pre-gyp": "^1.0.11", + "bindings": "^1.3.1", + "nan": "^2.17.0" + }, + "devDependencies": { + "@types/node": "^20.4.5", + "bluebird": "^3.5.3", + "eslint": "8.54.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", + "jshint": "^2.13.6", + "mocha": "^10.2.0", + "node-gyp": "^9.3.1", + "typescript": "^5.1.6" + }, + "engines": { + "node": ">=18.0.0" + } + }, + "node_modules/@confluentinc/kafka-javascript": { + "resolved": "../../..", + "link": true + }, + "node_modules/buffer-equal-constant-time": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz", + "integrity": "sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA==" + }, + "node_modules/ecdsa-sig-formatter": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", + "integrity": "sha512-nagl3RYrbNv6kQkeJIpt6NJZy8twLB/2vtz6yN9Z4vRKHN4/QZJIEbqohALSgwKdnksuY3k5Addp5lg8sVoVcQ==", + "dependencies": { + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jsonwebtoken": { + "version": "9.0.2", + "resolved": "https://registry.npmjs.org/jsonwebtoken/-/jsonwebtoken-9.0.2.tgz", + "integrity": "sha512-PRp66vJ865SSqOlgqS8hujT5U4AOgMfhrwYIuIhfKaoSCZcirrmASQr8CX7cUg+RMih+hgznrjp99o+W4pJLHQ==", + "dependencies": { + "jws": "^3.2.2", + "lodash.includes": "^4.3.0", + "lodash.isboolean": "^3.0.3", + "lodash.isinteger": "^4.0.4", + "lodash.isnumber": "^3.0.3", + "lodash.isplainobject": "^4.0.6", + "lodash.isstring": "^4.0.1", + "lodash.once": "^4.0.0", + "ms": "^2.1.1", + "semver": "^7.5.4" + }, + "engines": { + "node": ">=12", + "npm": ">=6" + } + }, + "node_modules/jwa": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/jwa/-/jwa-1.4.1.tgz", + "integrity": "sha512-qiLX/xhEEFKUAJ6FiBMbes3w9ATzyk5W7Hvzpa/SLYdxNtng+gcurvrI7TbACjIXlsJyr05/S1oUhZrc63evQA==", + "dependencies": { + "buffer-equal-constant-time": "1.0.1", + "ecdsa-sig-formatter": "1.0.11", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jws": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/jws/-/jws-3.2.2.tgz", + "integrity": "sha512-YHlZCB6lMTllWDtSPHz/ZXTsi8S00usEV6v1tjq8tOUZzw7DpSDWVXjXDre6ed1w/pd495ODpHZYSdkRTsa0HA==", + "dependencies": { + "jwa": "^1.4.1", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/lodash.includes": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/lodash.includes/-/lodash.includes-4.3.0.tgz", + "integrity": "sha512-W3Bx6mdkRTGtlJISOvVD/lbqjTlPPUDTMnlXZFnVwi9NKJ6tiAk6LVdlhZMm17VZisqhKcgzpO5Wz91PCt5b0w==" + }, + "node_modules/lodash.isboolean": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/lodash.isboolean/-/lodash.isboolean-3.0.3.tgz", + "integrity": "sha512-Bz5mupy2SVbPHURB98VAcw+aHh4vRV5IPNhILUCsOzRmsTmSQ17jIuqopAentWoehktxGd9e/hbIXq980/1QJg==" + }, + "node_modules/lodash.isinteger": { + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/lodash.isinteger/-/lodash.isinteger-4.0.4.tgz", + "integrity": "sha512-DBwtEWN2caHQ9/imiNeEA5ys1JoRtRfY3d7V9wkqtbycnAmTvRRmbHKDV4a0EYc678/dia0jrte4tjYwVBaZUA==" + }, + "node_modules/lodash.isnumber": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/lodash.isnumber/-/lodash.isnumber-3.0.3.tgz", + "integrity": "sha512-QYqzpfwO3/CWf3XP+Z+tkQsfaLL/EnUlXWVkIk5FUPc4sBdTehEqZONuyRt2P67PXAk+NXmTBcc97zw9t1FQrw==" + }, + "node_modules/lodash.isplainobject": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/lodash.isplainobject/-/lodash.isplainobject-4.0.6.tgz", + "integrity": "sha512-oSXzaWypCMHkPC3NvBEaPHf0KsA5mvPrOPgQWDsbg8n7orZ290M0BmC/jgRZ4vcJ6DTAhjrsSYgdsW/F+MFOBA==" + }, + "node_modules/lodash.isstring": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/lodash.isstring/-/lodash.isstring-4.0.1.tgz", + "integrity": "sha512-0wJxfxH1wgO3GrbuP+dTTk7op+6L41QCXbGINEmD+ny/G/eCqGzxyCsh7159S+mgDDcoarnBw6PC1PS5+wUGgw==" + }, + "node_modules/lodash.once": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz", + "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==" + }, + "node_modules/lru-cache": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", + "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", + "dependencies": { + "yallist": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "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==" + }, + "node_modules/safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", + "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/semver": { + "version": "7.6.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.0.tgz", + "integrity": "sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==", + "dependencies": { + "lru-cache": "^6.0.0" + }, + "bin": { + "semver": "bin/semver.js" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + } + } +} diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/package.json b/examples/node-rdkafka/oauthbearer_callback_authentication/package.json new file mode 100644 index 00000000..c5f50a42 --- /dev/null +++ b/examples/node-rdkafka/oauthbearer_callback_authentication/package.json @@ -0,0 +1,15 @@ +{ + "name": "oauthbearer_callback_authentication", + "version": "1.0.0", + "description": "", + "main": "oauthbearer_callback_authentication.js", + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1" + }, + "author": "", + "license": "MIT", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../../..", + "jsonwebtoken": "^9.0.2" + } +} From 127567f6f336ed610ed3c084c85476714dcdd85b Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 5 Apr 2024 13:28:17 +0530 Subject: [PATCH 116/224] Allow token refresh cb to be an async function --- e2e/oauthbearer_cb.spec.js | 141 +++++++++++++++++++++---------------- lib/client.js | 19 ++++- 2 files changed, 97 insertions(+), 63 deletions(-) diff --git a/e2e/oauthbearer_cb.spec.js b/e2e/oauthbearer_cb.spec.js index 100599a9..10c440ee 100644 --- a/e2e/oauthbearer_cb.spec.js +++ b/e2e/oauthbearer_cb.spec.js @@ -14,69 +14,88 @@ var eventListener = require('./listener'); var kafkaBrokerList = process.env.KAFKA_HOST || 'localhost:9092'; -describe('Client with oauthbearer_cb', function () { - const oauthbearer_config = 'key=value'; - let oauthbearer_cb_called = 0; - let oauthbearer_cb = function (config, cb) { - console.log("Called oauthbearer_cb with given config: " + config); - t.equal(config, oauthbearer_config); - oauthbearer_cb_called++; - - // The broker is not expected to be configured for oauthbearer authentication. - // We just want to make sure that token refresh callback is triggered. - cb(new Error('oauthbearer_cb error'), null); - }; - - const commonConfig = { - 'metadata.broker.list': kafkaBrokerList, - 'debug': 'all', - 'security.protocol': 'SASL_PLAINTEXT', - 'sasl.mechanisms': 'OAUTHBEARER', - 'oauthbearer_token_refresh_cb': oauthbearer_cb, - 'sasl.oauthbearer.config': oauthbearer_config, - } - - const checkClient = function (client, done) { - eventListener(client); - - client.on('error', function (e) { - t.match(e.message, /oauthbearer_cb error/); - }); +const oauthbearer_config = 'key=value'; +let oauthbearer_cb_called = 0; + +let oauthbearer_cb_callback = function (config, cb) { + console.log("Called oauthbearer_cb with given config: " + config); + t.equal(config, oauthbearer_config); + oauthbearer_cb_called++; + + // The broker is not expected to be configured for oauthbearer authentication. + // We just want to make sure that token refresh callback is triggered. + cb(new Error('oauthbearer_cb error'), null); +}; + +let oauthbearer_cb_async = async function (config) { + console.log("Called oauthbearer_cb with given config: " + config); + t.equal(config, oauthbearer_config); + oauthbearer_cb_called++; + + // The broker is not expected to be configured for oauthbearer authentication. + // We just want to make sure that token refresh callback is triggered. + throw new Error('oauthbearer_cb error'); +}; + +for (const oauthbearer_cb of [oauthbearer_cb_async, oauthbearer_cb_callback]) { + describe('Client with ' + (oauthbearer_cb.name), function () { + + const commonConfig = { + 'metadata.broker.list': kafkaBrokerList, + 'debug': 'all', + 'security.protocol': 'SASL_PLAINTEXT', + 'sasl.mechanisms': 'OAUTHBEARER', + 'oauthbearer_token_refresh_cb': oauthbearer_cb, + 'sasl.oauthbearer.config': oauthbearer_config, + } + + const checkClient = function (client, done, useCb) { + eventListener(client); - client.connect(); + client.on('error', function (e) { + t.match(e.message, /oauthbearer_cb error/); + }); - // We don't actually expect the connection to succeed, but we want to - // make sure that the oauthbearer_cb is called so give it a couple seconds. - setTimeout(() => { - client.disconnect(); - client = null; - t.equal(oauthbearer_cb_called >= 1, true); + // The default timeout for the connect is 30s, so even if we + // call disconnect() midway, the test ends up being at least 30s. + client.connect({timeout: 2000}); + + // We don't actually expect the connection to succeed, but we want to + // make sure that the oauthbearer_cb is called so give it a couple seconds. + setTimeout(() => { + t.equal(oauthbearer_cb_called >= 1, true); + client.disconnect(() => { + done(); + }); + client = null; + if (!useCb) // for admin client, where disconnect is sync. + done(); + }, 2000); + } + + beforeEach(function (done) { + oauthbearer_cb_called = 0; done(); - }, 2000); - } + }); - beforeEach(function (done) { - oauthbearer_cb_called = 0; - done(); - }); + it('as producer', function (done) { + let producer = new Kafka.Producer(commonConfig); + checkClient(producer, done, true); + producer = null; + }).timeout(5000); - it('as producer', function (done) { - let producer = new Kafka.Producer(commonConfig); - checkClient(producer, done); - producer = null; - }).timeout(2500); - - it('as consumer', function (done) { - const config = Object.assign({ 'group.id': 'gid' }, commonConfig); - let consumer = new Kafka.KafkaConsumer(config); - checkClient(consumer, done); - consumer = null; - }).timeout(2500); - - it('as admin', function (done) { - let admin = new Kafka.AdminClient.create(commonConfig); - checkClient(admin, done); - admin = null; - }).timeout(2500); - -}); + it('as consumer', function (done) { + const config = Object.assign({ 'group.id': 'gid' }, commonConfig); + let consumer = new Kafka.KafkaConsumer(config); + checkClient(consumer, done, true); + consumer = null; + }).timeout(5000); + + it('as admin', function (done) { + let admin = new Kafka.AdminClient.create(commonConfig); + checkClient(admin, done, false); + admin = null; + }).timeout(5000); + + }); +} diff --git a/lib/client.js b/lib/client.js index 9f13d93d..02f3702a 100644 --- a/lib/client.js +++ b/lib/client.js @@ -109,7 +109,10 @@ function Client(globalConf, SubClientType, topicConf) { // we might want to disconnect without ever completing connection. return; } - savedCallback(oauthbearer_config, (err, token) => { + + // This sets the token or error within librdkafka, and emits any + // errors on the emitter. + const postProcessTokenRefresh = (err, token) => { try { if (err) { throw err; @@ -130,7 +133,19 @@ function Client(globalConf, SubClientType, topicConf) { this._client.setOAuthBearerTokenFailure(e.message); this.emit('error', e); } - }); + }; + const returnPromise = savedCallback(oauthbearer_config, postProcessTokenRefresh); + + // If it looks like a promise, and quacks like a promise, it is a promise + // (or an async function). We expect the callback NOT to have been called + // in such a case. + if (returnPromise && (typeof returnPromise.then === 'function')) { + returnPromise.then((token) => { + postProcessTokenRefresh(null, token); + }).catch(err => { + postProcessTokenRefresh(err); + }); + } } } From 510584b1baaae615ec7c32430568a9b98ed56632 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Apr 2024 13:05:48 +0530 Subject: [PATCH 117/224] Add OAUTHBEARER support to promisified API --- MIGRATION.md | 44 ++++++++++++------------ index.d.ts | 6 +++- lib/admin.js | 12 ++++++- lib/kafkajs/_admin.js | 37 ++++++++++++++++++-- lib/kafkajs/_common.js | 74 ++++++++++++++++++++++++++++++---------- lib/kafkajs/_consumer.js | 1 + lib/kafkajs/_producer.js | 5 +-- lib/producer.js | 3 +- src/admin.cc | 1 + src/connection.cc | 17 +++++++-- src/connection.h | 7 ++-- src/kafka-consumer.cc | 7 ---- src/kafka-consumer.h | 2 -- src/producer.cc | 8 ----- src/producer.h | 2 -- types/kafkajs.d.ts | 8 +++++ 16 files changed, 163 insertions(+), 71 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 04652506..57f76faf 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -89,27 +89,27 @@ producerRun().then(consumerRun).catch(console.error); Each allowed config property is discussed in the table below. If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. - | Property | Default Value | Comment | - |-------------------------------|--------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| - | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | - | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified outside the kafkaJS block.** | - | **sasl** | - | An optional object of the form `{ mechanism: 'plain' or 'scram-sha-256' or 'scram-sha-512', username: string, password: string }`. **Additional authentication types are not yet supported.** | - | clientId | "rdkafka" | An optional string used to identify the client. | - | **connectionTimeout** | 1000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | - | **authenticationTimeout** | 10000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | - | **reauthenticationThreshold** | **80% of connections.max.reauth.ms** | **No longer checked, the default is always used.** | - | requestTimeout | 30000 | number of milliseconds for a network request to timeout. | - | **enforceRequestTimeout** | true | When set to false, `requestTimeout` is set to 5 minutes. **This cannot be completely disabled.** | - | retry | object | Properties individually discussed below. | - | retry.maxRetryTime | 30000 | maximum time to backoff a retry, in milliseconds. | - | retry.initialRetryTime | 300 | minimum time to backoff a retry, in milliseconds | - | **retry.retries** | 5 | Total cap on the number of retries. **Applicable only to Produce requests.** | - | **retry.factor** | 0.2 | Randomization factor (jitter) for backoff. **Cannot be changed**. | - | **retry.multiplier** | 2 | Multiplier for exponential factor of backoff. **Cannot be changed.** | - | **retry.restartOnFailure** | true | Consumer only. **Cannot be changed**. Consumer will always make an attempt to restart. | - | logLevel | `logLevel.INFO` | Decides the severity level of the logger created by the underlying library. A logger created with the `INFO` level will not be able to log `DEBUG` messages later. | - | **socketFactory** | null | **No longer supported.** | - | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | + | Property | Default Value | Comment | + |-------------------------------|--------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | + | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified outside the kafkaJS block.** | + | **sasl** | - | An optional object of the form `{ mechanism: 'plain' | 'scram-sha-256' | 'scram-sha-512', username: string, password: string }` or `{ mechanism: 'oauthbearer', oauthBearerProvider: function }`. Note that for OAUTHBEARER based authentication, the provider function must return lifetime (in ms), and principal name along with token value. **Additional authentication types are not supported.** | + | clientId | "rdkafka" | An optional string used to identify the client. | + | **connectionTimeout** | 1000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | + | **authenticationTimeout** | 10000 | This timeout is not enforced individually, but a sum of `connectionTimeout` and `authenticationTimeout` is enforced together. | + | **reauthenticationThreshold** | **80% of connections.max.reauth.ms** | **No longer checked, the default is always used.** | + | requestTimeout | 30000 | number of milliseconds for a network request to timeout. | + | **enforceRequestTimeout** | true | When set to false, `requestTimeout` is set to 5 minutes. **This cannot be completely disabled.** | + | retry | object | Properties individually discussed below. | + | retry.maxRetryTime | 30000 | maximum time to backoff a retry, in milliseconds. | + | retry.initialRetryTime | 300 | minimum time to backoff a retry, in milliseconds | + | **retry.retries** | 5 | Total cap on the number of retries. **Applicable only to Produce requests.** | + | **retry.factor** | 0.2 | Randomization factor (jitter) for backoff. **Cannot be changed**. | + | **retry.multiplier** | 2 | Multiplier for exponential factor of backoff. **Cannot be changed.** | + | **retry.restartOnFailure** | true | Consumer only. **Cannot be changed**. Consumer will always make an attempt to restart. | + | logLevel | `logLevel.INFO` | Decides the severity level of the logger created by the underlying library. A logger created with the `INFO` level will not be able to log `DEBUG` messages later. | + | **socketFactory** | null | **No longer supported.** | + | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | ### Producer @@ -195,7 +195,7 @@ producerRun().then(consumerRun).catch(console.error); | Property | Default Value | Comment | |--------------------------|-----------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | groupId | null | A mandatory string denoting consumer group name that this consumer is a part of. | - | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range, roundRobin, and cooperativeSticky assignors is provided. Custom assignors are not supported. | + | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range, roundRobin, and cooperativeSticky assignors is provided. Custom assignors are not supported. | | **partitionAssignors** | `[PartitionAssignors.roundRobin]` | Alias for `partitionAssigners` | | **rebalanceTimeout** | **300000** | The maximum allowed time for each member to join the group once a rebalance has begun. Note, that setting this value *also* changes the max poll interval. Message processing in `eachMessage` must not take more than this time. | | heartbeatInterval | 3000 | The expected time in milliseconds between heartbeats to the consumer coordinator. | diff --git a/index.d.ts b/index.d.ts index 6cf753c4..b58d1eeb 100644 --- a/index.d.ts +++ b/index.d.ts @@ -436,8 +436,12 @@ export interface IAdminClient { disconnect(): void; } +export type EventHandlers = { + [event_key: string]: (...args: any[]) => void; +}; + export abstract class AdminClient { - static create(conf: GlobalConfig): IAdminClient; + static create(conf: GlobalConfig, eventHandlers?: EventHandlers): IAdminClient; } export type RdKafka = { diff --git a/lib/admin.js b/lib/admin.js index a54c817d..6c7968ce 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -58,10 +58,19 @@ util.inherits(AdminClient, Client); * This is a factory method because it immediately starts an * active handle with the brokers. * + * @param {object} conf - Key value pairs to configure the admin client + * @param {object} eventHandlers - optional key value pairs of event handlers to attach to the client + * */ -function createAdminClient(conf) { +function createAdminClient(conf, eventHandlers) { var client = new AdminClient(conf); + if (eventHandlers && typeof eventHandlers === 'object') { + for (const key in eventHandlers) { + client.on(key, eventHandlers[key]); + } + } + // Wrap the error so we throw if it failed with some context LibrdKafkaError.wrap(client.connect(), true); @@ -127,6 +136,7 @@ AdminClient.prototype.connect = function () { this._client.configureCallbacks(true, this._cb_configs); LibrdKafkaError.wrap(this._client.connect(), true); this._isConnected = true; + this.emit('ready', { name: this._client.name() }); }; /** diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 30b2f7e8..730a9af4 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -50,6 +50,12 @@ class Admin { */ #logger = new DefaultLogger(); + /** + * connectPromiseFunc is the set of promise functions used to resolve/reject the connect() promise. + * @type {{resolve: Function, reject: Function}|{}} + */ + #connectPromiseFunc = null; + /** * @constructor * @param {import("../../types/kafkajs").AdminConstructorConfig} config @@ -95,6 +101,29 @@ class Admin { return rdKafkaConfig; } + #readyCb() { + if (this.#state !== AdminState.CONNECTING) { + /* 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 = AdminState.CONNECTED; + + // Resolve the promise. + this.#connectPromiseFunc['resolve'](); + } + + /** + * Callback for the event.error event, either fails the initial connect(), or logs the error. + * @param {Error} err + */ + #errorCb(err) { + if (this.#state === AdminState.CONNECTING) { + this.#connectPromiseFunc['reject'](err); + } else { + this.#logger.error(err); + } + } + /** * Set up the client and connect to the bootstrap brokers. * @returns {Promise} Resolves when connection is complete, rejects on error. @@ -111,9 +140,11 @@ class Admin { return new Promise((resolve, reject) => { try { /* AdminClient creation is a synchronous operation for node-rdkafka */ - this.#internalClient = RdKafka.AdminClient.create(config); - this.#state = AdminState.CONNECTED; - resolve(); + this.#connectPromiseFunc = { resolve, reject }; + this.#internalClient = RdKafka.AdminClient.create(config, { + 'error': this.#errorCb.bind(this), + 'ready': this.#readyCb.bind(this), + }); } catch (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); } diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 3d263649..f6d8a43e 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -168,12 +168,12 @@ const CompatibilityErrorMessages = Object.freeze({ brokerString: () => "The 'brokers' property must be an array of strings.\n" + "For example: ['kafka:9092', 'kafka2:9093']\n", - saslOauthbearerUnsupported: () => - "SASL mechanism OAUTHBEARER is not supported yet.", saslUnsupportedMechanism: (mechanism) => `SASL mechanism ${mechanism} is not supported.`, saslUsernamePasswordString: (mechanism) => `The 'sasl.username' and 'sasl.password' properties must be strings and must be present for the mechanism ${mechanism}.`, + saslOauthBearerProvider: () => + `The 'oauthBearerProvider' property must be a function.`, sslObject: () => "The 'ssl' property must be a boolean. Any additional configuration must be provided outside the kafkaJS block.\n" + "Before: \n" + @@ -280,27 +280,65 @@ function kafkaJSToRdKafkaConfig(config) { const mechanism = sasl.mechanism.toUpperCase(); if (mechanism === 'OAUTHBEARER') { - throw new error.KafkaJSError(CompatibilityErrorMessages.saslOauthbearerUnsupported(), { - code: error.ErrorCodes.ERR__NOT_IMPLEMENTED, - }); - } - - /* The mechanism must be PLAIN or SCRAM. */ - if (mechanism !== 'PLAIN' && !mechanism.startsWith('SCRAM')) { + rdkafkaConfig["sasl.mechanism"] = mechanism; + if (Object.hasOwn(sasl, "oauthBearerProvider")) { + if (typeof sasl.oauthBearerProvider !== 'function') { + throw new error.KafkaJSError(CompatibilityErrorMessages.saslOauthBearerProvider(), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + rdkafkaConfig['oauthbearer_token_refresh_cb'] = function (oauthbearer_config) { + return sasl.oauthBearerProvider(oauthbearer_config) + .then((token) => { + if (!Object.hasOwn(token, 'value')) { + throw new error.KafkaJSError('Token must have a value property.', { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } else if (!Object.hasOwn(token, 'principal')) { + throw new error.KafkaJSError('Token must have a principal property.', { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } else if (!Object.hasOwn(token, 'lifetime')) { + throw new error.KafkaJSError('Token must have a lifetime property.', { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + + // Recast token into a value expected by node-rdkafka's callback. + const setToken = { + tokenValue: token.value, + extensions: token.extensions, + principal: token.principal, + lifetime: token.lifetime, + }; + return setToken; + }) + .catch(err => { + if (!(err instanceof Error)) { + err = new Error(err); + } + throw err; + }); + } + } + /* It's a valid case (unlike in KafkaJS) for oauthBearerProvider to be + * null, because librdkafka provides an unsecured token provider for + * non-prod usecases. So don't do anything in that case. */ + } else if (mechanism === 'PLAIN' || mechanism.startsWith('SCRAM')) { + if (typeof sasl.username !== "string" || typeof sasl.password !== "string") { + throw new error.KafkaJSError(CompatibilityErrorMessages.saslUsernamePasswordString(mechanism), { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + rdkafkaConfig["sasl.mechanism"] = mechanism; + rdkafkaConfig["sasl.username"] = sasl.username; + rdkafkaConfig["sasl.password"] = sasl.password; + } else { throw new error.KafkaJSError(CompatibilityErrorMessages.saslUnsupportedMechanism(mechanism), { code: error.ErrorCodes.ERR__INVALID_ARG, }); } - if (typeof sasl.username !== "string" || typeof sasl.password !== "string") { - throw new error.KafkaJSError(CompatibilityErrorMessages.saslUsernamePasswordString(mechanism), { - code: error.ErrorCodes.ERR__INVALID_ARG, - }); - } - - rdkafkaConfig["sasl.mechanism"] = mechanism; - rdkafkaConfig["sasl.username"] = sasl.username; - rdkafkaConfig["sasl.password"] = sasl.password; withSASL = true; } diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 1c964584..bc164ee7 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -790,6 +790,7 @@ class Consumer { this.#state = ConsumerState.CONNECTING; this.#internalClient = new RdKafka.KafkaConsumer(rdKafkaConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); + this.#internalClient.on('error', this.#errorCb.bind(this)); this.#internalClient.on('event.error', this.#errorCb.bind(this)); this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 12f42d87..f38d3697 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -321,6 +321,7 @@ class Producer { this.#internalClient = new RdKafka.Producer(rdKafkaConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); this.#internalClient.on('event.error', this.#errorCb.bind(this)); + this.#internalClient.on('error', this.#errorCb.bind(this)); this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); return new Promise((resolve, reject) => { @@ -614,10 +615,10 @@ class Producer { throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsAcks('sendBatch'), { code: error.ErrorCodes.ERR__INVALID_ARG }); } if (Object.hasOwn(sendOptions, 'timeout')) { - throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsTimeout('sendBatch'), { code: error.ErrorCodes.ERR__INVALID_ARG }); + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsTimeout('timeout'), { code: error.ErrorCodes.ERR__INVALID_ARG }); } if (Object.hasOwn(sendOptions, 'compression')) { - throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsCompression('sendBatch'), { code: error.ErrorCodes.ERR__INVALID_ARG }); + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOptionsCompression('compression'), { code: error.ErrorCodes.ERR__INVALID_ARG }); } if (sendOptions.topicMessages !== null && !Array.isArray(sendOptions.topicMessages)) { diff --git a/lib/producer.js b/lib/producer.js index ddada902..5993a23d 100644 --- a/lib/producer.js +++ b/lib/producer.js @@ -2,6 +2,7 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. @@ -70,7 +71,7 @@ function Producer(conf, topicConf) { delete conf.dr_cb; delete conf.dr_msg_cb; - // client is an initialized consumer object + // client is an initialized producer object // @see NodeKafka::Producer::Init Client.call(this, conf, Kafka.Producer, topicConf); diff --git a/src/admin.cc b/src/admin.cc index 5a364e6b..39bfd76b 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -105,6 +105,7 @@ void AdminClient::Init(v8::Local exports) { // Inherited from NodeKafka::Connection Nan::SetPrototypeMethod(tpl, "configureCallbacks", NodeConfigureCallbacks); + Nan::SetPrototypeMethod(tpl, "name", NodeName); // Admin client operations Nan::SetPrototypeMethod(tpl, "createTopic", NodeCreateTopic); diff --git a/src/connection.cc b/src/connection.cc index 12697dc3..a7f26c29 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -116,11 +116,11 @@ RdKafka::TopicPartition* Connection::GetPartition(std::string &topic, int partit return RdKafka::TopicPartition::create(topic, partition); } -bool Connection::IsConnected() { +bool Connection::IsConnected() const { return !m_is_closing && m_client != NULL; } -bool Connection::IsClosing() { +bool Connection::IsClosing() const { return m_client != NULL && m_is_closing; } @@ -128,6 +128,13 @@ RdKafka::Handle* Connection::GetClient() { return m_client; } +std::string Connection::Name() const { + if (!IsConnected()) { + return std::string(""); + } + return std::string(m_client->name()); +} + Baton Connection::CreateTopic(std::string topic_name) { return CreateTopic(topic_name, NULL); } @@ -629,4 +636,10 @@ NAN_METHOD(Connection::NodeSetOAuthBearerTokenFailure) { info.GetReturnValue().Set(Nan::Null()); } +NAN_METHOD(Connection::NodeName) { + Connection* obj = ObjectWrap::Unwrap(info.This()); + std::string name = obj->Name(); + info.GetReturnValue().Set(Nan::New(name).ToLocalChecked()); +} + } // namespace NodeKafka diff --git a/src/connection.h b/src/connection.h index 47fe20b3..8d859a2a 100644 --- a/src/connection.h +++ b/src/connection.h @@ -47,8 +47,8 @@ namespace NodeKafka { class Connection : public Nan::ObjectWrap { public: - bool IsConnected(); - bool IsClosing(); + bool IsConnected() const; + bool IsClosing() const; // Baton Baton CreateTopic(std::string); @@ -73,6 +73,8 @@ class Connection : public Nan::ObjectWrap { virtual void ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add); + std::string Name() const; + protected: Connection(Conf*, Conf*); ~Connection(); @@ -102,6 +104,7 @@ class Connection : public Nan::ObjectWrap { static NAN_METHOD(NodeSetSaslCredentials); static NAN_METHOD(NodeSetOAuthBearerToken); static NAN_METHOD(NodeSetOAuthBearerTokenFailure); + static NAN_METHOD(NodeName); }; } // namespace NodeKafka diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 1a05abe1..061c6306 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -544,13 +544,6 @@ std::string KafkaConsumer::RebalanceProtocol() { return consumer->rebalance_protocol(); } -std::string KafkaConsumer::Name() { - if (!IsConnected()) { - return std::string(""); - } - return std::string(m_client->name()); -} - Nan::Persistent KafkaConsumer::constructor; void KafkaConsumer::Init(v8::Local exports) { diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index c84c0e34..7a9b523a 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -82,8 +82,6 @@ class KafkaConsumer : public Connection { Baton Seek(const RdKafka::TopicPartition &partition, int timeout_ms); - std::string Name(); - Baton Subscribe(std::vector); Baton Consume(int timeout_ms); diff --git a/src/producer.cc b/src/producer.cc index 82da58d7..191d9592 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -165,14 +165,6 @@ v8::Local Producer::NewInstance(v8::Local arg) { return scope.Escape(instance); } - -std::string Producer::Name() { - if (!IsConnected()) { - return std::string(""); - } - return std::string(m_client->name()); -} - Baton Producer::Connect() { if (IsConnected()) { return Baton(RdKafka::ERR_NO_ERROR); diff --git a/src/producer.h b/src/producer.h index 39dcbefe..1e7854e1 100644 --- a/src/producer.h +++ b/src/producer.h @@ -75,8 +75,6 @@ class Producer : public Connection { int64_t timestamp, void* opaque, RdKafka::Headers* headers); - std::string Name(); - void ActivateDispatchers(); void DeactivateDispatchers(); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 856f6c40..8360f222 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -12,10 +12,18 @@ export type Mechanism = { mechanism: string } +export interface OauthbearerProviderResponse { + value: string, + principal: string, + lifetime: number, // Lifetime must be in milliseconds. + extensions?: Map | { [key: string]: string }, +} + type SASLMechanismOptionsMap = { plain: { username: string; password: string } 'scram-sha-256': { username: string; password: string } 'scram-sha-512': { username: string; password: string } + oauthbearer: { oauthBearerProvider: () => Promise } } export type SASLMechanism = keyof SASLMechanismOptionsMap From e11fcfb508b34e164dc5226c730394da1ff447e1 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Apr 2024 13:06:34 +0530 Subject: [PATCH 118/224] Add examples and tests for OAUTHBEARER (promisified API) --- .../oauthbearer_callback_authentication.js | 65 ++++++ .../package-lock.json | 188 ++++++++++++++++++ .../package.json | 16 ++ test/promisified/oauthbearer_cb.spec.js | 80 ++++++++ 4 files changed, 349 insertions(+) create mode 100644 examples/kafkajs/oauthbearer_calback_authentication/oauthbearer_callback_authentication.js create mode 100644 examples/kafkajs/oauthbearer_calback_authentication/package-lock.json create mode 100644 examples/kafkajs/oauthbearer_calback_authentication/package.json create mode 100644 test/promisified/oauthbearer_cb.spec.js diff --git a/examples/kafkajs/oauthbearer_calback_authentication/oauthbearer_callback_authentication.js b/examples/kafkajs/oauthbearer_calback_authentication/oauthbearer_callback_authentication.js new file mode 100644 index 00000000..3fa9ee4b --- /dev/null +++ b/examples/kafkajs/oauthbearer_calback_authentication/oauthbearer_callback_authentication.js @@ -0,0 +1,65 @@ +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; +var jwt = require('jsonwebtoken'); + +// This example uses the Producer for demonstration purposes. +// It is the same whether you use a Consumer/AdminClient. + +async function token_refresh(oauthbearer_config /* string - passed from config */) { + console.log("Called token_refresh with given config: " + oauthbearer_config); + // At this point, we can use the information in the token, make + // some API calls, fetch something from a file... + // For the illustration, everything is hard-coded. + const principal = 'admin'; + // In seconds - needed by jsonwebtoken library + const exp_seconds = Math.floor(Date.now() / 1000) + (60 * 60); + // In milliseconds - needed by kafka-javascript. + const exp_ms = exp_seconds * 1000; + + // For illustration, we're not signing our JWT (algorithm: none). + // For production uses-cases, it should be signed. + const value = jwt.sign( + { 'sub': principal, exp: exp_seconds, 'scope': 'requiredScope' }, '', { algorithm: 'none' }); + + // SASL extensions can be passed as Map or key/value pairs in an object. + const extensions = { + traceId: '123' + }; + + // The callback is called with the new token, its lifetime, and the principal. + // The extensions are optional and may be omitted. + console.log("Finished token_refresh, triggering callback: with value: " + + value.slice(0, 10) + "..., lifetime: " + exp_ms + + ", principal: " + principal + ", extensions: " + JSON.stringify(extensions)); + + // If no token could be fetched or an error occurred, an Error can be thrown instead. + return { value, lifetime: exp_ms, principal, extensions }; +} + +async function run() { + const kafka = new Kafka({}); + const producer = kafka.producer({ + kafkaJS: { + brokers: ['localhost:46611'], + sasl: { + mechanism: 'oauthbearer', + oauthBearerProvider: token_refresh, + }, + }, + 'sasl.oauthbearer.config': 'someConfigPropertiesKey=value', + }); + + await producer.connect(); + console.log("Producer connected"); + + const deliveryReport = await producer.send({ + topic: 'topic', + messages: [ + { value: 'Hello world!' }, + ], + }); + console.log("Producer sent message", deliveryReport); + + await producer.disconnect(); +} + +run().catch(console.error); \ No newline at end of file diff --git a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json new file mode 100644 index 00000000..ffe3c834 --- /dev/null +++ b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json @@ -0,0 +1,188 @@ +{ + "name": "oauthbearer_calback_authentication", + "version": "1.0.0", + "lockfileVersion": 3, + "requires": true, + "packages": { + "": { + "name": "oauthbearer_calback_authentication", + "version": "1.0.0", + "license": "ISC", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../../..", + "jsonwebtoken": "^9.0.2" + } + }, + "../../..": { + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.11-devel", + "hasInstallScript": true, + "license": "MIT", + "dependencies": { + "@mapbox/node-pre-gyp": "^1.0.11", + "bindings": "^1.3.1", + "nan": "^2.17.0" + }, + "devDependencies": { + "@types/node": "^20.4.5", + "bluebird": "^3.5.3", + "eslint": "8.54.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", + "jshint": "^2.13.6", + "mocha": "^10.2.0", + "node-gyp": "^9.3.1", + "typescript": "^5.1.6" + }, + "engines": { + "node": ">=18.0.0" + } + }, + "node_modules/@confluentinc/kafka-javascript": { + "resolved": "../../..", + "link": true + }, + "node_modules/buffer-equal-constant-time": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz", + "integrity": "sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA==" + }, + "node_modules/ecdsa-sig-formatter": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", + "integrity": "sha512-nagl3RYrbNv6kQkeJIpt6NJZy8twLB/2vtz6yN9Z4vRKHN4/QZJIEbqohALSgwKdnksuY3k5Addp5lg8sVoVcQ==", + "dependencies": { + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jsonwebtoken": { + "version": "9.0.2", + "resolved": "https://registry.npmjs.org/jsonwebtoken/-/jsonwebtoken-9.0.2.tgz", + "integrity": "sha512-PRp66vJ865SSqOlgqS8hujT5U4AOgMfhrwYIuIhfKaoSCZcirrmASQr8CX7cUg+RMih+hgznrjp99o+W4pJLHQ==", + "dependencies": { + "jws": "^3.2.2", + "lodash.includes": "^4.3.0", + "lodash.isboolean": "^3.0.3", + "lodash.isinteger": "^4.0.4", + "lodash.isnumber": "^3.0.3", + "lodash.isplainobject": "^4.0.6", + "lodash.isstring": "^4.0.1", + "lodash.once": "^4.0.0", + "ms": "^2.1.1", + "semver": "^7.5.4" + }, + "engines": { + "node": ">=12", + "npm": ">=6" + } + }, + "node_modules/jwa": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/jwa/-/jwa-1.4.1.tgz", + "integrity": "sha512-qiLX/xhEEFKUAJ6FiBMbes3w9ATzyk5W7Hvzpa/SLYdxNtng+gcurvrI7TbACjIXlsJyr05/S1oUhZrc63evQA==", + "dependencies": { + "buffer-equal-constant-time": "1.0.1", + "ecdsa-sig-formatter": "1.0.11", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jws": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/jws/-/jws-3.2.2.tgz", + "integrity": "sha512-YHlZCB6lMTllWDtSPHz/ZXTsi8S00usEV6v1tjq8tOUZzw7DpSDWVXjXDre6ed1w/pd495ODpHZYSdkRTsa0HA==", + "dependencies": { + "jwa": "^1.4.1", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/lodash.includes": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/lodash.includes/-/lodash.includes-4.3.0.tgz", + "integrity": "sha512-W3Bx6mdkRTGtlJISOvVD/lbqjTlPPUDTMnlXZFnVwi9NKJ6tiAk6LVdlhZMm17VZisqhKcgzpO5Wz91PCt5b0w==" + }, + "node_modules/lodash.isboolean": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/lodash.isboolean/-/lodash.isboolean-3.0.3.tgz", + "integrity": "sha512-Bz5mupy2SVbPHURB98VAcw+aHh4vRV5IPNhILUCsOzRmsTmSQ17jIuqopAentWoehktxGd9e/hbIXq980/1QJg==" + }, + "node_modules/lodash.isinteger": { + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/lodash.isinteger/-/lodash.isinteger-4.0.4.tgz", + "integrity": "sha512-DBwtEWN2caHQ9/imiNeEA5ys1JoRtRfY3d7V9wkqtbycnAmTvRRmbHKDV4a0EYc678/dia0jrte4tjYwVBaZUA==" + }, + "node_modules/lodash.isnumber": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/lodash.isnumber/-/lodash.isnumber-3.0.3.tgz", + "integrity": "sha512-QYqzpfwO3/CWf3XP+Z+tkQsfaLL/EnUlXWVkIk5FUPc4sBdTehEqZONuyRt2P67PXAk+NXmTBcc97zw9t1FQrw==" + }, + "node_modules/lodash.isplainobject": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/lodash.isplainobject/-/lodash.isplainobject-4.0.6.tgz", + "integrity": "sha512-oSXzaWypCMHkPC3NvBEaPHf0KsA5mvPrOPgQWDsbg8n7orZ290M0BmC/jgRZ4vcJ6DTAhjrsSYgdsW/F+MFOBA==" + }, + "node_modules/lodash.isstring": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/lodash.isstring/-/lodash.isstring-4.0.1.tgz", + "integrity": "sha512-0wJxfxH1wgO3GrbuP+dTTk7op+6L41QCXbGINEmD+ny/G/eCqGzxyCsh7159S+mgDDcoarnBw6PC1PS5+wUGgw==" + }, + "node_modules/lodash.once": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz", + "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==" + }, + "node_modules/lru-cache": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", + "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", + "dependencies": { + "yallist": "^4.0.0" + }, + "engines": { + "node": ">=10" + } + }, + "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==" + }, + "node_modules/safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", + "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/semver": { + "version": "7.6.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.0.tgz", + "integrity": "sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==", + "dependencies": { + "lru-cache": "^6.0.0" + }, + "bin": { + "semver": "bin/semver.js" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + } + } +} diff --git a/examples/kafkajs/oauthbearer_calback_authentication/package.json b/examples/kafkajs/oauthbearer_calback_authentication/package.json new file mode 100644 index 00000000..0dbf5ff8 --- /dev/null +++ b/examples/kafkajs/oauthbearer_calback_authentication/package.json @@ -0,0 +1,16 @@ +{ + "name": "oauthbearer_calback_authentication", + "version": "1.0.0", + "description": "", + "main": "index.js", + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1" + }, + "keywords": [], + "author": "", + "license": "ISC", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../../..", + "jsonwebtoken": "^9.0.2" + } +} diff --git a/test/promisified/oauthbearer_cb.spec.js b/test/promisified/oauthbearer_cb.spec.js new file mode 100644 index 00000000..d33f4eb5 --- /dev/null +++ b/test/promisified/oauthbearer_cb.spec.js @@ -0,0 +1,80 @@ +// minimum 30s are needed for the connect timeouts of consumer/producer +jest.setTimeout(35000); + +const { + createProducer, + sleep, + createConsumer, + createAdmin, +} = require('./testhelpers'); + +describe('Client > oauthbearer callback', () => { + let oauthbearer_cb_called = 0; + const oauthbearer_config = 'key=value'; + const providerCb = async (config) => { + expect(config).toEqual(oauthbearer_config); + oauthbearer_cb_called++; + throw new Error('oauthbearer_cb error'); + }; + + beforeEach(async () => { + oauthbearer_cb_called = 0; + }) + + it('works for producer', + async () => { + const client = createProducer({ + sasl: { + mechanism: 'OAUTHBEARER', + oauthBearerProvider: providerCb, + } + }, { + 'sasl.oauthbearer.config': oauthbearer_config, + }); + + await expect(client.connect()).rejects.toThrow('oauthbearer_cb error'); + expect(oauthbearer_cb_called).toEqual(1); + await client.disconnect(); + } + ); + + it('works for consumer', + async () => { + const client = createConsumer({ + groupId: 'gid', + sasl: { + mechanism: 'OAUTHBEARER', + oauthBearerProvider: providerCb, + } + }, { + 'sasl.oauthbearer.config': oauthbearer_config, + }); + + await expect(client.connect()).rejects.toThrow('oauthbearer_cb error'); + expect(oauthbearer_cb_called).toEqual(1); + await client.disconnect(); + } + ); + + it('works for admin', + async () => { + const client = createAdmin({ + sasl: { + mechanism: 'OAUTHBEARER', + oauthBearerProvider: providerCb, + } + }, { + 'sasl.oauthbearer.config': oauthbearer_config, + }); + + // Unlike others, there is no actual connection establishment + // within the admin client, so we can't test for the error here. + await expect(client.connect()).resolves.toBeUndefined(); + + await sleep(2000); // Wait for the callback to be called + expect(oauthbearer_cb_called).toEqual(1); + await client.disconnect(); + } + ); + +}) From 82fce5544c7cadb5cfdf02b2b11bc662775f9a1c Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Apr 2024 17:09:55 +0530 Subject: [PATCH 119/224] Make examples a sub-package to fix the imports --- examples/kafkajs/admin.js | 4 +- examples/kafkajs/consumer-confluent-cloud.js | 4 +- examples/kafkajs/consumer.js | 4 +- examples/kafkajs/eos.js | 4 +- examples/kafkajs/producer-confluent-cloud.js | 4 +- examples/kafkajs/producer.js | 4 +- examples/kafkajs/sr.js | 4 +- examples/package-lock.json | 44 ++++++++++++++++++++ examples/package.json | 14 +++++++ 9 files changed, 65 insertions(+), 21 deletions(-) create mode 100644 examples/package-lock.json create mode 100644 examples/package.json diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js index 420adfa3..2e5d5f73 100644 --- a/examples/kafkajs/admin.js +++ b/examples/kafkajs/admin.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function adminStart() { const kafka = new Kafka({ diff --git a/examples/kafkajs/consumer-confluent-cloud.js b/examples/kafkajs/consumer-confluent-cloud.js index 4e9d5ea0..aa10b0c7 100644 --- a/examples/kafkajs/consumer-confluent-cloud.js +++ b/examples/kafkajs/consumer-confluent-cloud.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function consumerStart() { let consumer; diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 07d249bc..85e5c072 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function consumerStart() { let consumer; diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index aebd5c5d..78e0d5f5 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function eosStart() { const kafka = new Kafka({ diff --git a/examples/kafkajs/producer-confluent-cloud.js b/examples/kafkajs/producer-confluent-cloud.js index b4b8eb28..0e57ff5b 100644 --- a/examples/kafkajs/producer-confluent-cloud.js +++ b/examples/kafkajs/producer-confluent-cloud.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function producerStart() { const CLUSTER_BOOTSTRAP_URL = 'your_cluster_url_here'; diff --git a/examples/kafkajs/producer.js b/examples/kafkajs/producer.js index 73111590..87f7ef11 100644 --- a/examples/kafkajs/producer.js +++ b/examples/kafkajs/producer.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function producerStart() { const kafka = new Kafka({ diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 0b114549..a640f651 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -1,7 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -// Since this example is within the package itself, we use '../..', but code -// will typically use '@confluentinc/kafka-javascript'. -const { Kafka } = require('../..').KafkaJS; +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; // Note: The kafkajs/confluent-schema-registry will need to be installed separately to run this example, // as it isn't a dependency of confluent-kafka-javascript. diff --git a/examples/package-lock.json b/examples/package-lock.json new file mode 100644 index 00000000..cdc9bbba --- /dev/null +++ b/examples/package-lock.json @@ -0,0 +1,44 @@ +{ + "name": "examples", + "version": "1.0.0", + "lockfileVersion": 3, + "requires": true, + "packages": { + "": { + "name": "examples", + "version": "1.0.0", + "license": "MIT", + "dependencies": { + "@confluentinc/kafka-javascript": "file:.." + } + }, + "..": { + "version": "v0.1.11-devel", + "hasInstallScript": true, + "license": "MIT", + "dependencies": { + "@mapbox/node-pre-gyp": "^1.0.11", + "bindings": "^1.3.1", + "nan": "^2.17.0" + }, + "devDependencies": { + "@types/node": "^20.4.5", + "bluebird": "^3.5.3", + "eslint": "8.54.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", + "jshint": "^2.13.6", + "mocha": "^10.2.0", + "node-gyp": "^9.3.1", + "typescript": "^5.1.6" + }, + "engines": { + "node": ">=18.0.0" + } + }, + "node_modules/@confluentinc/kafka-javascript": { + "resolved": "..", + "link": true + } + } +} diff --git a/examples/package.json b/examples/package.json new file mode 100644 index 00000000..b8eb8b2a --- /dev/null +++ b/examples/package.json @@ -0,0 +1,14 @@ +{ + "name": "examples", + "version": "1.0.0", + "description": "", + "main": "index.js", + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1" + }, + "author": "", + "license": "MIT", + "dependencies": { + "@confluentinc/kafka-javascript": "file:.." + } +} From c793ae9c3af68fe898c10fa2fb874047d09382b7 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Apr 2024 11:07:05 +0530 Subject: [PATCH 120/224] Fix types and add typescript examples --- examples/typescript/.gitignore | 1 + examples/typescript/kafkajs.ts | 82 +++++ examples/typescript/node-rdkafka.ts | 82 +++++ examples/typescript/package-lock.json | 108 ++++++ examples/typescript/package.json | 19 ++ index.d.ts | 466 +------------------------- types/kafkajs.d.ts | 24 +- types/rdkafka.d.ts | 459 +++++++++++++++++++++++++ 8 files changed, 773 insertions(+), 468 deletions(-) create mode 100644 examples/typescript/.gitignore create mode 100644 examples/typescript/kafkajs.ts create mode 100644 examples/typescript/node-rdkafka.ts create mode 100644 examples/typescript/package-lock.json create mode 100644 examples/typescript/package.json create mode 100644 types/rdkafka.d.ts diff --git a/examples/typescript/.gitignore b/examples/typescript/.gitignore new file mode 100644 index 00000000..a6c7c285 --- /dev/null +++ b/examples/typescript/.gitignore @@ -0,0 +1 @@ +*.js diff --git a/examples/typescript/kafkajs.ts b/examples/typescript/kafkajs.ts new file mode 100644 index 00000000..3b158b35 --- /dev/null +++ b/examples/typescript/kafkajs.ts @@ -0,0 +1,82 @@ +import { KafkaJS } from '@confluentinc/kafka-javascript'; + +const bootstrapServer = ''; + +async function runProducer() { + const kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [bootstrapServer], + }, + }); + + const producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + + await producer.connect(); + + await producer.send({ + topic: 'test-topic', + messages: [ + { value: 'Hello World!', key: 'key1' }, + ], + }); + + await producer.disconnect(); +} + +async function runConsumer() { + const kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [bootstrapServer], + }, + }); + + const consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group' + Math.random(), + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: 'test-topic' }); + + await consumer.run({ + eachMessage: async ({ message }) => { + console.log({ + key: message.key ? message.key.toString() : null, + value: message.value ? message.value.toString() : null, + }); + }, + }); + + await new Promise((resolve) => setTimeout(resolve, 30000)); + await consumer.disconnect(); +} + +async function runAdminClient() { + const kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [bootstrapServer], + }, + }); + + const admin = kafka.admin() + await admin.connect(); + + await admin.createTopics({ topics: [{ topic: 'test-topic' }] }); + console.log("Created topic"); + + await admin.disconnect(); +} + +runAdminClient() + .then(runProducer) + .then(runConsumer) + .catch(console.error); \ No newline at end of file diff --git a/examples/typescript/node-rdkafka.ts b/examples/typescript/node-rdkafka.ts new file mode 100644 index 00000000..81d7cde1 --- /dev/null +++ b/examples/typescript/node-rdkafka.ts @@ -0,0 +1,82 @@ +import * as RdKafka from '@confluentinc/kafka-javascript'; + +const bootstrapServers = ''; + +function runProducer() { + const producer = new RdKafka.Producer({ + 'bootstrap.servers': bootstrapServers, + 'dr_msg_cb': true, + }); + + producer.connect(); + + producer.on('ready', () => { + console.log("Producer is ready"); + producer.setPollInterval(100); + producer.produce('test-topic', null, Buffer.from('Hello World!'), null, Date.now()); + }); + + producer.on('event.error', (err) => { + console.error(err); + }); + + producer.on('delivery-report', (err, report) => { + console.log("Delivery report received:"); + console.log({err, report}); + producer.disconnect(err => { + if (err) + console.log("Error disconnecting producer ", err); + console.log("Disconnected producer"); + }); + }); +} + +function runConsumer() { + const consumer = new RdKafka.KafkaConsumer({ + 'group.id': 'test-group', + 'bootstrap.servers': bootstrapServers, + }, { + 'auto.offset.reset': 'earliest', + }); + + consumer.connect(); + + consumer.on('ready', () => { + console.log("Consumer is ready"); + consumer.subscribe(['test-topic']); + consumer.consume(); + }); + + consumer.on('data', (data) => { + console.log("Received data"); + console.log(data); + }); + + consumer.on('event.error', (err) => { + console.error(err); + }); + + setTimeout(() => consumer.disconnect(), 30000); +} + +function runAdminClient() { + const admin = RdKafka.AdminClient.create({ + "bootstrap.servers": bootstrapServers, + }); + + admin.createTopic({ topic: "test-topic", num_partitions: 1, replication_factor: 1 }, (err) => { + if (err) { + console.error(err); + admin.disconnect(); + return; + } + console.log("Created topic"); + admin.disconnect(); + }); + +} + +// As an example, run each with some time gap to allow the prior one to finish. +runAdminClient(); +setTimeout(runProducer, 5000); +setTimeout(runConsumer, 25000); \ No newline at end of file diff --git a/examples/typescript/package-lock.json b/examples/typescript/package-lock.json new file mode 100644 index 00000000..48c8d652 --- /dev/null +++ b/examples/typescript/package-lock.json @@ -0,0 +1,108 @@ +{ + "name": "typescript", + "version": "1.0.0", + "lockfileVersion": 3, + "requires": true, + "packages": { + "": { + "name": "typescript", + "version": "1.0.0", + "license": "MIT", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../..", + "node-rdkafka": "^2.18.0", + "typescript": "^5.4.4" + }, + "devDependencies": { + "@types/node": "^20.12.5" + } + }, + "../..": { + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.11-devel", + "hasInstallScript": true, + "license": "MIT", + "dependencies": { + "@mapbox/node-pre-gyp": "^1.0.11", + "bindings": "^1.3.1", + "nan": "^2.17.0" + }, + "devDependencies": { + "@types/node": "^20.4.5", + "bluebird": "^3.5.3", + "eslint": "8.54.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", + "jshint": "^2.13.6", + "mocha": "^10.2.0", + "node-gyp": "^9.3.1", + "typescript": "^5.1.6" + }, + "engines": { + "node": ">=18.0.0" + } + }, + "node_modules/@confluentinc/kafka-javascript": { + "resolved": "../..", + "link": true + }, + "node_modules/@types/node": { + "version": "20.12.5", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.5.tgz", + "integrity": "sha512-BD+BjQ9LS/D8ST9p5uqBxghlN+S42iuNxjsUGjeZobe/ciXzk2qb1B6IXc6AnRLS+yFJRpN2IPEHMzwspfDJNw==", + "dev": true, + "dependencies": { + "undici-types": "~5.26.4" + } + }, + "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/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/nan": { + "version": "2.19.0", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.19.0.tgz", + "integrity": "sha512-nO1xXxfh/RWNxfd/XPfbIfFk5vgLsAxUR9y5O0cHMJu/AW9U95JLXqthYHjEp+8gQ5p96K9jUp8nbVOxCdRbtw==" + }, + "node_modules/node-rdkafka": { + "version": "2.18.0", + "resolved": "https://registry.npmjs.org/node-rdkafka/-/node-rdkafka-2.18.0.tgz", + "integrity": "sha512-jYkmO0sPvjesmzhv1WFOO4z7IMiAFpThR6/lcnFDWgSPkYL95CtcuVNo/R5PpjujmqSgS22GMkL1qvU4DTAvEQ==", + "hasInstallScript": true, + "dependencies": { + "bindings": "^1.3.1", + "nan": "^2.17.0" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/typescript": { + "version": "5.4.4", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.4.tgz", + "integrity": "sha512-dGE2Vv8cpVvw28v8HCPqyb08EzbBURxDpuhJvTrusShUfGnhHBafDsLdS1EhhxyL6BJQE+2cT3dDPAv+MQ6oLw==", + "bin": { + "tsc": "bin/tsc", + "tsserver": "bin/tsserver" + }, + "engines": { + "node": ">=14.17" + } + }, + "node_modules/undici-types": { + "version": "5.26.5", + "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-5.26.5.tgz", + "integrity": "sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==", + "dev": true + } + } +} diff --git a/examples/typescript/package.json b/examples/typescript/package.json new file mode 100644 index 00000000..20db593a --- /dev/null +++ b/examples/typescript/package.json @@ -0,0 +1,19 @@ +{ + "name": "typescript", + "version": "1.0.0", + "description": "", + "main": "index.js", + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1" + }, + "author": "", + "license": "MIT", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../..", + "node-rdkafka": "^2.18.0", + "typescript": "^5.4.4" + }, + "devDependencies": { + "@types/node": "^20.12.5" + } +} diff --git a/index.d.ts b/index.d.ts index b58d1eeb..8cc4d295 100644 --- a/index.d.ts +++ b/index.d.ts @@ -1,463 +1,3 @@ -import { Readable, ReadableOptions, Writable, WritableOptions } from 'stream'; -import { EventEmitter } from 'events'; -import { - GlobalConfig, - TopicConfig, - ConsumerGlobalConfig, - ConsumerTopicConfig, - ProducerGlobalConfig, - ProducerTopicConfig, -} from './types/config'; - -export * from './types/config'; -export * from './types/errors'; -import { Kafka } from './types/kafkajs'; -import * as errors from './types/errors'; - -export interface LibrdKafkaError { - message: string; - code: number; - errno: number; - origin: string; - stack?: string; - isFatal?: boolean; - isRetriable?: boolean; - isTxnRequiresAbort?: boolean; -} - -export interface ReadyInfo { - name: string; -} - -export interface ClientMetrics { - connectionOpened: number; -} - -export interface MetadataOptions { - topic?: string; - allTopics?: boolean; - timeout?: number; -} - -export interface BrokerMetadata { - id: number; - host: string; - port: number; -} - -export interface PartitionMetadata { - id: number; - leader: number; - replicas: number[]; - isrs: number[]; -} - -export interface TopicMetadata { - name: string; - partitions: PartitionMetadata[]; -} - -export interface Metadata { - orig_broker_id: number; - orig_broker_name: string; - topics: TopicMetadata[]; - brokers: BrokerMetadata[]; -} - -export interface WatermarkOffsets{ - lowOffset: number; - highOffset: number; -} - -export interface TopicPartition { - topic: string; - partition: number; - error?: LibrdKafkaError; -} - -export interface TopicPartitionOffset extends TopicPartition{ - offset: number; -} - -export type TopicPartitionTime = TopicPartitionOffset; - -export type EofEvent = TopicPartitionOffset; - -export type Assignment = TopicPartition | TopicPartitionOffset; - -export interface DeliveryReport extends TopicPartitionOffset { - value?: MessageValue; - size: number; - key?: MessageKey; - timestamp?: number; - opaque?: any; -} - -export type NumberNullUndefined = number | null | undefined; - -export type MessageKey = Buffer | string | null | undefined; -export type MessageHeader = { [key: string]: string | Buffer }; -export type MessageValue = Buffer | null; -export type SubscribeTopic = string | RegExp; -export type SubscribeTopicList = SubscribeTopic[]; - -export interface Message extends TopicPartitionOffset { - value: MessageValue; - size: number; - topic: string; - key?: MessageKey; - timestamp?: number; - headers?: MessageHeader[]; - opaque?: any; -} - -export interface ReadStreamOptions extends ReadableOptions { - topics: SubscribeTopicList | SubscribeTopic | ((metadata: Metadata) => SubscribeTopicList); - waitInterval?: number; - fetchSize?: number; - objectMode?: boolean; - highWaterMark?: number; - autoClose?: boolean; - streamAsBatch?: boolean; - connectOptions?: any; -} - -export interface WriteStreamOptions extends WritableOptions { - encoding?: string; - objectMode?: boolean; - topic?: string; - autoClose?: boolean; - pollInterval?: number; - connectOptions?: any; -} - -export interface ProducerStream extends Writable { - producer: Producer; - connect(metadataOptions?: MetadataOptions): void; - close(cb?: () => void): void; -} - -export interface ConsumerStream extends Readable { - consumer: KafkaConsumer; - connect(options: ConsumerGlobalConfig): void; - close(cb?: () => void): void; -} - -type KafkaClientEvents = 'disconnected' | 'ready' | 'connection.failure' | 'event.error' | 'event.stats' | 'event.log' | 'event.event' | 'event.throttle'; -type KafkaConsumerEvents = 'data' | 'partition.eof' | 'rebalance' | 'rebalance.error' | 'subscribed' | 'unsubscribed' | 'unsubscribe' | 'offset.commit' | KafkaClientEvents; -type KafkaProducerEvents = 'delivery-report' | KafkaClientEvents; - -type EventListenerMap = { - // ### Client - // connectivity events - 'disconnected': (metrics: ClientMetrics) => void, - 'ready': (info: ReadyInfo, metadata: Metadata) => void, - 'connection.failure': (error: LibrdKafkaError, metrics: ClientMetrics) => void, - // event messages - 'event.error': (error: LibrdKafkaError) => void, - 'event.stats': (eventData: any) => void, - 'event.log': (eventData: any) => void, - 'event.event': (eventData: any) => void, - 'event.throttle': (eventData: any) => void, - // ### Consumer only - // domain events - 'data': (arg: Message) => void, - 'partition.eof': (arg: EofEvent) => void, - 'rebalance': (err: LibrdKafkaError, assignments: TopicPartition[]) => void, - 'rebalance.error': (err: Error) => void, - // connectivity events - 'subscribed': (topics: SubscribeTopicList) => void, - 'unsubscribe': () => void, - 'unsubscribed': () => void, - // offsets - 'offset.commit': (error: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void, - // ### Producer only - // delivery - 'delivery-report': (error: LibrdKafkaError, report: DeliveryReport) => void, -} - -type EventListener = K extends keyof EventListenerMap ? EventListenerMap[K] : never; - -export abstract class Client extends EventEmitter { - constructor(globalConf: GlobalConfig, SubClientType: any, topicConf: TopicConfig); - - connect(metadataOptions?: MetadataOptions, cb?: (err: LibrdKafkaError, data: Metadata) => any): this; - - getClient(): any; - - connectedTime(): number; - - getLastError(): LibrdKafkaError; - - disconnect(cb?: (err: any, data: ClientMetrics) => any): this; - disconnect(timeout: number, cb?: (err: any, data: ClientMetrics) => any): this; - - isConnected(): boolean; - - getMetadata(metadataOptions?: MetadataOptions, cb?: (err: LibrdKafkaError, data: Metadata) => any): any; - - queryWatermarkOffsets(topic: string, partition: number, timeout: number, cb?: (err: LibrdKafkaError, offsets: WatermarkOffsets) => any): any; - queryWatermarkOffsets(topic: string, partition: number, cb?: (err: LibrdKafkaError, offsets: WatermarkOffsets) => any): any; - - setSaslCredentials(username: string, password: string): void; - - on(event: E, listener: EventListener): this; - once(event: E, listener: EventListener): this; -} - -export class KafkaConsumer extends Client { - constructor(conf: ConsumerGlobalConfig | ConsumerTopicConfig, topicConf?: ConsumerTopicConfig); - - assign(assignments: Assignment[]): this; - - assignments(): Assignment[]; - - commit(topicPartition: TopicPartitionOffset | TopicPartitionOffset[]): this; - commit(): this; - - commitMessage(msg: TopicPartitionOffset): this; - - commitMessageSync(msg: TopicPartitionOffset): this; - - commitSync(topicPartition: TopicPartitionOffset | TopicPartitionOffset[]): this; - - committed(toppars: TopicPartition[], timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void): this; - committed(timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void): this; - - consume(number: number, cb?: (err: LibrdKafkaError, messages: Message[]) => void): void; - consume(cb: (err: LibrdKafkaError, messages: Message[]) => void): void; - consume(): void; - - getWatermarkOffsets(topic: string, partition: number): WatermarkOffsets; - - offsetsStore(topicPartitions: TopicPartitionOffset[]): any; - - pause(topicPartitions: TopicPartition[]): any; - - position(toppars?: TopicPartition[]): TopicPartitionOffset[]; - - resume(topicPartitions: TopicPartition[]): any; - - seek(toppar: TopicPartitionOffset, timeout: number | null, cb: (err: LibrdKafkaError) => void): this; - - setDefaultConsumeTimeout(timeoutMs: number): void; - - setDefaultConsumeLoopTimeoutDelay(timeoutMs: number): void; - - subscribe(topics: SubscribeTopicList): this; - - subscription(): string[]; - - unassign(): this; - - unsubscribe(): this; - - offsetsForTimes(topicPartitions: TopicPartitionTime[], timeout: number, cb?: (err: LibrdKafkaError, offsets: TopicPartitionOffset[]) => any): void; - offsetsForTimes(topicPartitions: TopicPartitionTime[], cb?: (err: LibrdKafkaError, offsets: TopicPartitionOffset[]) => any): void; - - static createReadStream(conf: ConsumerGlobalConfig, topicConfig: ConsumerTopicConfig, streamOptions: ReadStreamOptions | number): ConsumerStream; -} - -export class Producer extends Client { - constructor(conf: ProducerGlobalConfig | ProducerTopicConfig, topicConf?: ProducerTopicConfig); - - flush(timeout?: NumberNullUndefined, cb?: (err: LibrdKafkaError) => void): this; - - poll(): this; - - produce(topic: string, partition: NumberNullUndefined, message: MessageValue, key?: MessageKey, timestamp?: NumberNullUndefined, opaque?: any, headers?: MessageHeader[]): any; - - setPollInterval(interval: number): this; - - static createWriteStream(conf: ProducerGlobalConfig, topicConf: ProducerTopicConfig, streamOptions: WriteStreamOptions): ProducerStream; - - initTransactions(cb: (err: LibrdKafkaError) => void): void; - initTransactions(timeout: number, cb: (err: LibrdKafkaError) => void): void; - beginTransaction(cb: (err: LibrdKafkaError) => void): void; - commitTransaction(cb: (err: LibrdKafkaError) => void): void; - commitTransaction(timeout: number, cb: (err: LibrdKafkaError) => void): void; - abortTransaction(cb: (err: LibrdKafkaError) => void): void; - abortTransaction(timeout: number, cb: (err: LibrdKafkaError) => void): void; - sendOffsetsToTransaction(offsets: TopicPartitionOffset[], consumer: KafkaConsumer, cb: (err: LibrdKafkaError) => void): void; - sendOffsetsToTransaction(offsets: TopicPartitionOffset[], consumer: KafkaConsumer, timeout: number, cb: (err: LibrdKafkaError) => void): void; -} - -export class HighLevelProducer extends Producer { - produce(topic: string, partition: NumberNullUndefined, message: any, key: any, timestamp: NumberNullUndefined, callback: (err: any, offset?: NumberNullUndefined) => void): any; - produce(topic: string, partition: NumberNullUndefined, message: any, key: any, timestamp: NumberNullUndefined, headers: MessageHeader[], callback: (err: any, offset?: NumberNullUndefined) => void): any; - - setKeySerializer(serializer: (key: any, cb: (err: any, key: MessageKey) => void) => void): void; - setKeySerializer(serializer: (key: any) => MessageKey | Promise): void; - setValueSerializer(serializer: (value: any, cb: (err: any, value: MessageValue) => void) => void): void; - setValueSerializer(serializer: (value: any) => MessageValue | Promise): void; -} - -export const features: string[]; - -export const librdkafkaVersion: string; - -export function createReadStream(conf: ConsumerGlobalConfig, topicConf: ConsumerTopicConfig, streamOptions: ReadStreamOptions | number): ConsumerStream; - -export function createWriteStream(conf: ProducerGlobalConfig, topicConf: ProducerTopicConfig, streamOptions: WriteStreamOptions): ProducerStream; - -export interface NewTopic { - topic: string; - num_partitions: number; - replication_factor: number; - config?: { - 'cleanup.policy'?: 'delete' | 'compact' | 'delete,compact' | 'compact,delete'; - 'compression.type'?: 'gzip' | 'snappy' | 'lz4' | 'zstd' | 'uncompressed' | 'producer'; - 'delete.retention.ms'?: string; - 'file.delete.delay.ms'?: string; - 'flush.messages'?: string; - 'flush.ms'?: string; - 'follower.replication.throttled.replicas'?: string; - 'index.interval.bytes'?: string; - 'leader.replication.throttled.replicas'?: string; - 'max.compaction.lag.ms'?: string; - 'max.message.bytes'?: string; - 'message.format.version'?: string; - 'message.timestamp.difference.max.ms'?: string; - 'message.timestamp.type'?: string; - 'min.cleanable.dirty.ratio'?: string; - 'min.compaction.lag.ms'?: string; - 'min.insync.replicas'?: string; - 'preallocate'?: string; - 'retention.bytes'?: string; - 'retention.ms'?: string; - 'segment.bytes'?: string; - 'segment.index.bytes'?: string; - 'segment.jitter.ms'?: string; - 'segment.ms'?: string; - 'unclean.leader.election.enable'?: string; - 'message.downconversion.enable'?: string; - } | { [cfg: string]: string; }; -} - -export enum ConsumerGroupStates { - UNKNOWN = 0, - PREPARING_REBALANCE = 1, - COMPLETING_REBALANCE = 2, - STABLE = 3, - DEAD = 4, - EMPTY = 5, -} - -export interface GroupOverview { - groupId: string; - protocolType: string; - isSimpleConsumerGroup: boolean; - state: ConsumerGroupStates; -} - -export enum AclOperationTypes { - UNKNOWN = 0, - ANY = 1, - ALL = 2, - READ = 3, - WRITE = 4, - CREATE = 5, - DELETE = 6, - ALTER = 7, - DESCRIBE = 8, - CLUSTER_ACTION = 9, - DESCRIBE_CONFIGS = 10, - ALTER_CONFIGS = 11, - IDEMPOTENT_WRITE = 12, -} - -export type MemberDescription = { - clientHost: string - clientId: string - memberId: string - memberAssignment: Buffer - memberMetadata: Buffer - groupInstanceId?: string, - assignment: TopicPartition[] -} - -export type Node = { - id: number - host: string - port: number - rack?: string -} - -export type GroupDescription = { - groupId: string - error?: LibrdKafkaError - members: MemberDescription[] - protocol: string - isSimpleConsumerGroup: boolean; - protocolType: string - partitionAssignor: string - state: ConsumerGroupStates - coordinator: Node - authorizedOperations?: AclOperationTypes[] -} - -export type GroupDescriptions = { - groups: GroupDescription[], -} - -export type DeleteGroupsResult = { - groupId: string - errorCode?: number - error?: LibrdKafkaError -} - -export interface IAdminClient { - createTopic(topic: NewTopic, cb?: (err: LibrdKafkaError) => void): void; - createTopic(topic: NewTopic, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; - - deleteTopic(topic: string, cb?: (err: LibrdKafkaError) => void): void; - deleteTopic(topic: string, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; - - createPartitions(topic: string, desiredPartitions: number, cb?: (err: LibrdKafkaError) => void): void; - createPartitions(topic: string, desiredPartitions: number, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; - - listTopics(cb?: (err: LibrdKafkaError, topics: string[]) => any): void; - listTopics(options?: { timeout?: number }, cb?: (err: LibrdKafkaError, topics: string[]) => any): void; - - listGroups(cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; - listGroups(options?: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }, - cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; - - describeGroups(groupIds: string[], cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; - describeGroups(groupIds: string[], - options?: { timeout?: number, includeAuthorizedOperations?: boolean }, - cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; - - deleteGroups(groupIds: string[], cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void; - deleteGroups(groupIds: string[], - options?: { timeout?: number }, - cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void; - - disconnect(): void; -} - -export type EventHandlers = { - [event_key: string]: (...args: any[]) => void; -}; - -export abstract class AdminClient { - static create(conf: GlobalConfig, eventHandlers?: EventHandlers): IAdminClient; -} - -export type RdKafka = { - Consumer: KafkaConsumer, - Producer: Producer, - HighLevelProducer: HighLevelProducer, - AdminClient: AdminClient, - KafkaConsumer: KafkaConsumer, - createReadStream: typeof KafkaConsumer.createReadStream, - createWriteStream: typeof Producer.createWriteStream, - CODES: typeof errors.CODES, - Topic: (name: string) => string, - features: typeof features, - librdkafkaVersion: typeof librdkafkaVersion, -} - -export type KafkaJS = { - Kafka: Kafka -} +export * from './types/rdkafka'; +export * as RdKafka from './types/rdkafka'; +export * as KafkaJS from './types/kafkajs'; \ No newline at end of file diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 8360f222..75a7390a 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,10 +1,9 @@ -import * as tls from 'tls' -import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig, TopicConfig } from './config' -import { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescription, GroupDescriptions, DeleteGroupsResult } from '../index' +import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig } from './config' +import { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescription, GroupDescriptions, DeleteGroupsResult } from './rdkafka' // Admin API related interfaces, types etc; and Error types are common, so // just re-export them from here too. -export { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescriptions, DeleteGroupsResult } from '../index' +export { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescriptions, DeleteGroupsResult } from './rdkafka' export type BrokersFunction = () => string[] | Promise @@ -55,6 +54,9 @@ export interface ProducerConfig { transactionalId?: string transactionTimeout?: number maxInFlightRequests?: number + acks?: number + compression?: CompressionTypes + timeout?: number rdKafka?: { topicConfig?: ProducerTopicConfig, globalConfig?: ProducerGlobalConfig } } @@ -124,6 +126,7 @@ export class Kafka { constructor(config: CommonConstructorConfig) producer(config?: ProducerConstructorConfig): Producer consumer(config: ConsumerConstructorConfig): Consumer + admin(config?: AdminConstructorConfig): Admin } type Sender = { @@ -144,6 +147,12 @@ export interface RetryOptions { restartOnFailure?: (e: Error) => Promise } +export enum PartitionAssigners { + roundRobin = 'roundRobin', + range = 'range', + cooperativeSticky = 'cooperative-sticky' +} + export interface ConsumerConfig { groupId: string metadataMaxAge?: number @@ -159,6 +168,11 @@ export interface ConsumerConfig { maxInFlightRequests?: number readUncommitted?: boolean rackId?: string + fromBeginning?: boolean + autoCommit?: boolean + autoCommitInterval?: number, + partitionAssigners?: PartitionAssigners[] + partitionAssignors?: PartitionAssigners[] rdKafka?: { topicConfig?: ConsumerTopicConfig, globalConfig?: ConsumerGlobalConfig } } @@ -443,7 +457,7 @@ export interface OffsetsByTopicPartition { export type Consumer = { connect(): Promise disconnect(): Promise - subscribe(subscription: ConsumerSubscribeTopics): Promise + subscribe(subscription: ConsumerSubscribeTopics | ConsumerSubscribeTopic): Promise stop(): Promise run(config?: ConsumerRunConfig): Promise commitOffsets(topicPartitions: Array): Promise diff --git a/types/rdkafka.d.ts b/types/rdkafka.d.ts new file mode 100644 index 00000000..4c9150e9 --- /dev/null +++ b/types/rdkafka.d.ts @@ -0,0 +1,459 @@ +import { Readable, ReadableOptions, Writable, WritableOptions } from 'stream'; +import { EventEmitter } from 'events'; +import { + GlobalConfig, + TopicConfig, + ConsumerGlobalConfig, + ConsumerTopicConfig, + ProducerGlobalConfig, + ProducerTopicConfig, +} from './config'; + +export * from './config'; +export * from './errors'; +import { Kafka } from './kafkajs'; +import * as errors from './errors'; + +export interface LibrdKafkaError { + message: string; + code: number; + errno: number; + origin: string; + stack?: string; + isFatal?: boolean; + isRetriable?: boolean; + isTxnRequiresAbort?: boolean; +} + +export interface ReadyInfo { + name: string; +} + +export interface ClientMetrics { + connectionOpened: number; +} + +export interface MetadataOptions { + topic?: string; + allTopics?: boolean; + timeout?: number; +} + +export interface BrokerMetadata { + id: number; + host: string; + port: number; +} + +export interface PartitionMetadata { + id: number; + leader: number; + replicas: number[]; + isrs: number[]; +} + +export interface TopicMetadata { + name: string; + partitions: PartitionMetadata[]; +} + +export interface Metadata { + orig_broker_id: number; + orig_broker_name: string; + topics: TopicMetadata[]; + brokers: BrokerMetadata[]; +} + +export interface WatermarkOffsets{ + lowOffset: number; + highOffset: number; +} + +export interface TopicPartition { + topic: string; + partition: number; + error?: LibrdKafkaError; +} + +export interface TopicPartitionOffset extends TopicPartition{ + offset: number; +} + +export type TopicPartitionTime = TopicPartitionOffset; + +export type EofEvent = TopicPartitionOffset; + +export type Assignment = TopicPartition | TopicPartitionOffset; + +export interface DeliveryReport extends TopicPartitionOffset { + value?: MessageValue; + size: number; + key?: MessageKey; + timestamp?: number; + opaque?: any; +} + +export type NumberNullUndefined = number | null | undefined; + +export type MessageKey = Buffer | string | null | undefined; +export type MessageHeader = { [key: string]: string | Buffer }; +export type MessageValue = Buffer | null; +export type SubscribeTopic = string | RegExp; +export type SubscribeTopicList = SubscribeTopic[]; + +export interface Message extends TopicPartitionOffset { + value: MessageValue; + size: number; + topic: string; + key?: MessageKey; + timestamp?: number; + headers?: MessageHeader[]; + opaque?: any; +} + +export interface ReadStreamOptions extends ReadableOptions { + topics: SubscribeTopicList | SubscribeTopic | ((metadata: Metadata) => SubscribeTopicList); + waitInterval?: number; + fetchSize?: number; + objectMode?: boolean; + highWaterMark?: number; + autoClose?: boolean; + streamAsBatch?: boolean; + connectOptions?: any; +} + +export interface WriteStreamOptions extends WritableOptions { + encoding?: string; + objectMode?: boolean; + topic?: string; + autoClose?: boolean; + pollInterval?: number; + connectOptions?: any; +} + +export interface ProducerStream extends Writable { + producer: Producer; + connect(metadataOptions?: MetadataOptions): void; + close(cb?: () => void): void; +} + +export interface ConsumerStream extends Readable { + consumer: KafkaConsumer; + connect(options: ConsumerGlobalConfig): void; + close(cb?: () => void): void; +} + +type KafkaClientEvents = 'disconnected' | 'ready' | 'connection.failure' | 'event.error' | 'event.stats' | 'event.log' | 'event.event' | 'event.throttle'; +type KafkaConsumerEvents = 'data' | 'partition.eof' | 'rebalance' | 'rebalance.error' | 'subscribed' | 'unsubscribed' | 'unsubscribe' | 'offset.commit' | KafkaClientEvents; +type KafkaProducerEvents = 'delivery-report' | KafkaClientEvents; + +type EventListenerMap = { + // ### Client + // connectivity events + 'disconnected': (metrics: ClientMetrics) => void, + 'ready': (info: ReadyInfo, metadata: Metadata) => void, + 'connection.failure': (error: LibrdKafkaError, metrics: ClientMetrics) => void, + // event messages + 'event.error': (error: LibrdKafkaError) => void, + 'event.stats': (eventData: any) => void, + 'event.log': (eventData: any) => void, + 'event.event': (eventData: any) => void, + 'event.throttle': (eventData: any) => void, + // ### Consumer only + // domain events + 'data': (arg: Message) => void, + 'partition.eof': (arg: EofEvent) => void, + 'rebalance': (err: LibrdKafkaError, assignments: TopicPartition[]) => void, + 'rebalance.error': (err: Error) => void, + // connectivity events + 'subscribed': (topics: SubscribeTopicList) => void, + 'unsubscribe': () => void, + 'unsubscribed': () => void, + // offsets + 'offset.commit': (error: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void, + // ### Producer only + // delivery + 'delivery-report': (error: LibrdKafkaError, report: DeliveryReport) => void, +} + +type EventListener = K extends keyof EventListenerMap ? EventListenerMap[K] : never; + +export abstract class Client extends EventEmitter { + constructor(globalConf: GlobalConfig, SubClientType: any, topicConf: TopicConfig); + + connect(metadataOptions?: MetadataOptions, cb?: (err: LibrdKafkaError, data: Metadata) => any): this; + + getClient(): any; + + connectedTime(): number; + + getLastError(): LibrdKafkaError; + + disconnect(cb?: (err: any, data: ClientMetrics) => any): this; + disconnect(timeout: number, cb?: (err: any, data: ClientMetrics) => any): this; + + isConnected(): boolean; + + getMetadata(metadataOptions?: MetadataOptions, cb?: (err: LibrdKafkaError, data: Metadata) => any): any; + + queryWatermarkOffsets(topic: string, partition: number, timeout: number, cb?: (err: LibrdKafkaError, offsets: WatermarkOffsets) => any): any; + queryWatermarkOffsets(topic: string, partition: number, cb?: (err: LibrdKafkaError, offsets: WatermarkOffsets) => any): any; + + setSaslCredentials(username: string, password: string): void; + + on(event: E, listener: EventListener): this; + once(event: E, listener: EventListener): this; +} + +export class KafkaConsumer extends Client { + constructor(conf: ConsumerGlobalConfig | ConsumerTopicConfig, topicConf?: ConsumerTopicConfig); + + assign(assignments: Assignment[]): this; + + assignments(): Assignment[]; + + commit(topicPartition: TopicPartitionOffset | TopicPartitionOffset[]): this; + commit(): this; + + commitMessage(msg: TopicPartitionOffset): this; + + commitMessageSync(msg: TopicPartitionOffset): this; + + commitSync(topicPartition: TopicPartitionOffset | TopicPartitionOffset[]): this; + + committed(toppars: TopicPartition[], timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void): this; + committed(timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void): this; + + consume(number: number, cb?: (err: LibrdKafkaError, messages: Message[]) => void): void; + consume(cb: (err: LibrdKafkaError, messages: Message[]) => void): void; + consume(): void; + + getWatermarkOffsets(topic: string, partition: number): WatermarkOffsets; + + offsetsStore(topicPartitions: TopicPartitionOffset[]): any; + + pause(topicPartitions: TopicPartition[]): any; + + position(toppars?: TopicPartition[]): TopicPartitionOffset[]; + + resume(topicPartitions: TopicPartition[]): any; + + seek(toppar: TopicPartitionOffset, timeout: number | null, cb: (err: LibrdKafkaError) => void): this; + + setDefaultConsumeTimeout(timeoutMs: number): void; + + setDefaultConsumeLoopTimeoutDelay(timeoutMs: number): void; + + subscribe(topics: SubscribeTopicList): this; + + subscription(): string[]; + + unassign(): this; + + unsubscribe(): this; + + offsetsForTimes(topicPartitions: TopicPartitionTime[], timeout: number, cb?: (err: LibrdKafkaError, offsets: TopicPartitionOffset[]) => any): void; + offsetsForTimes(topicPartitions: TopicPartitionTime[], cb?: (err: LibrdKafkaError, offsets: TopicPartitionOffset[]) => any): void; + + static createReadStream(conf: ConsumerGlobalConfig, topicConfig: ConsumerTopicConfig, streamOptions: ReadStreamOptions | number): ConsumerStream; +} + +export class Producer extends Client { + constructor(conf: ProducerGlobalConfig | ProducerTopicConfig, topicConf?: ProducerTopicConfig); + + flush(timeout?: NumberNullUndefined, cb?: (err: LibrdKafkaError) => void): this; + + poll(): this; + + produce(topic: string, partition: NumberNullUndefined, message: MessageValue, key?: MessageKey, timestamp?: NumberNullUndefined, opaque?: any, headers?: MessageHeader[]): any; + + setPollInterval(interval: number): this; + + static createWriteStream(conf: ProducerGlobalConfig, topicConf: ProducerTopicConfig, streamOptions: WriteStreamOptions): ProducerStream; + + initTransactions(cb: (err: LibrdKafkaError) => void): void; + initTransactions(timeout: number, cb: (err: LibrdKafkaError) => void): void; + beginTransaction(cb: (err: LibrdKafkaError) => void): void; + commitTransaction(cb: (err: LibrdKafkaError) => void): void; + commitTransaction(timeout: number, cb: (err: LibrdKafkaError) => void): void; + abortTransaction(cb: (err: LibrdKafkaError) => void): void; + abortTransaction(timeout: number, cb: (err: LibrdKafkaError) => void): void; + sendOffsetsToTransaction(offsets: TopicPartitionOffset[], consumer: KafkaConsumer, cb: (err: LibrdKafkaError) => void): void; + sendOffsetsToTransaction(offsets: TopicPartitionOffset[], consumer: KafkaConsumer, timeout: number, cb: (err: LibrdKafkaError) => void): void; +} + +export class HighLevelProducer extends Producer { + produce(topic: string, partition: NumberNullUndefined, message: any, key: any, timestamp: NumberNullUndefined, callback: (err: any, offset?: NumberNullUndefined) => void): any; + produce(topic: string, partition: NumberNullUndefined, message: any, key: any, timestamp: NumberNullUndefined, headers: MessageHeader[], callback: (err: any, offset?: NumberNullUndefined) => void): any; + + setKeySerializer(serializer: (key: any, cb: (err: any, key: MessageKey) => void) => void): void; + setKeySerializer(serializer: (key: any) => MessageKey | Promise): void; + setValueSerializer(serializer: (value: any, cb: (err: any, value: MessageValue) => void) => void): void; + setValueSerializer(serializer: (value: any) => MessageValue | Promise): void; +} + +export const features: string[]; + +export const librdkafkaVersion: string; + +export function createReadStream(conf: ConsumerGlobalConfig, topicConf: ConsumerTopicConfig, streamOptions: ReadStreamOptions | number): ConsumerStream; + +export function createWriteStream(conf: ProducerGlobalConfig, topicConf: ProducerTopicConfig, streamOptions: WriteStreamOptions): ProducerStream; + +export interface NewTopic { + topic: string; + num_partitions: number; + replication_factor: number; + config?: { + 'cleanup.policy'?: 'delete' | 'compact' | 'delete,compact' | 'compact,delete'; + 'compression.type'?: 'gzip' | 'snappy' | 'lz4' | 'zstd' | 'uncompressed' | 'producer'; + 'delete.retention.ms'?: string; + 'file.delete.delay.ms'?: string; + 'flush.messages'?: string; + 'flush.ms'?: string; + 'follower.replication.throttled.replicas'?: string; + 'index.interval.bytes'?: string; + 'leader.replication.throttled.replicas'?: string; + 'max.compaction.lag.ms'?: string; + 'max.message.bytes'?: string; + 'message.format.version'?: string; + 'message.timestamp.difference.max.ms'?: string; + 'message.timestamp.type'?: string; + 'min.cleanable.dirty.ratio'?: string; + 'min.compaction.lag.ms'?: string; + 'min.insync.replicas'?: string; + 'preallocate'?: string; + 'retention.bytes'?: string; + 'retention.ms'?: string; + 'segment.bytes'?: string; + 'segment.index.bytes'?: string; + 'segment.jitter.ms'?: string; + 'segment.ms'?: string; + 'unclean.leader.election.enable'?: string; + 'message.downconversion.enable'?: string; + } | { [cfg: string]: string; }; +} + +export enum ConsumerGroupStates { + UNKNOWN = 0, + PREPARING_REBALANCE = 1, + COMPLETING_REBALANCE = 2, + STABLE = 3, + DEAD = 4, + EMPTY = 5, +} + +export interface GroupOverview { + groupId: string; + protocolType: string; + isSimpleConsumerGroup: boolean; + state: ConsumerGroupStates; +} + +export enum AclOperationTypes { + UNKNOWN = 0, + ANY = 1, + ALL = 2, + READ = 3, + WRITE = 4, + CREATE = 5, + DELETE = 6, + ALTER = 7, + DESCRIBE = 8, + CLUSTER_ACTION = 9, + DESCRIBE_CONFIGS = 10, + ALTER_CONFIGS = 11, + IDEMPOTENT_WRITE = 12, +} + +export type MemberDescription = { + clientHost: string + clientId: string + memberId: string + memberAssignment: Buffer + memberMetadata: Buffer + groupInstanceId?: string, + assignment: TopicPartition[] +} + +export type Node = { + id: number + host: string + port: number + rack?: string +} + +export type GroupDescription = { + groupId: string + error?: LibrdKafkaError + members: MemberDescription[] + protocol: string + isSimpleConsumerGroup: boolean; + protocolType: string + partitionAssignor: string + state: ConsumerGroupStates + coordinator: Node + authorizedOperations?: AclOperationTypes[] +} + +export type GroupDescriptions = { + groups: GroupDescription[], +} + +export type DeleteGroupsResult = { + groupId: string + errorCode?: number + error?: LibrdKafkaError +} + +export interface IAdminClient { + createTopic(topic: NewTopic, cb?: (err: LibrdKafkaError) => void): void; + createTopic(topic: NewTopic, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; + + deleteTopic(topic: string, cb?: (err: LibrdKafkaError) => void): void; + deleteTopic(topic: string, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; + + createPartitions(topic: string, desiredPartitions: number, cb?: (err: LibrdKafkaError) => void): void; + createPartitions(topic: string, desiredPartitions: number, timeout?: number, cb?: (err: LibrdKafkaError) => void): void; + + listTopics(cb?: (err: LibrdKafkaError, topics: string[]) => any): void; + listTopics(options?: { timeout?: number }, cb?: (err: LibrdKafkaError, topics: string[]) => any): void; + + listGroups(cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; + listGroups(options?: { timeout?: number, matchConsumerGroupStates?: ConsumerGroupStates[] }, + cb?: (err: LibrdKafkaError, result: { groups: GroupOverview[], errors: LibrdKafkaError[] }) => any): void; + + describeGroups(groupIds: string[], cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; + describeGroups(groupIds: string[], + options?: { timeout?: number, includeAuthorizedOperations?: boolean }, + cb?: (err: LibrdKafkaError, result: GroupDescriptions) => any): void; + + deleteGroups(groupIds: string[], cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void; + deleteGroups(groupIds: string[], + options?: { timeout?: number }, + cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void; + + disconnect(): void; +} + +export type EventHandlers = { + [event_key: string]: (...args: any[]) => void; +}; + +export abstract class AdminClient { + static create(conf: GlobalConfig, eventHandlers?: EventHandlers): IAdminClient; +} + +export type RdKafka = { + Consumer: KafkaConsumer, + Producer: Producer, + HighLevelProducer: HighLevelProducer, + AdminClient: AdminClient, + KafkaConsumer: KafkaConsumer, + createReadStream: typeof KafkaConsumer.createReadStream, + createWriteStream: typeof Producer.createWriteStream, + CODES: typeof errors.CODES, + Topic: (name: string) => string, + features: typeof features, + librdkafkaVersion: typeof librdkafkaVersion, +} From 5bc9ce9db9bdbb59bc2c03106d7e4f46243aeed5 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Apr 2024 21:17:31 +0530 Subject: [PATCH 121/224] Bump version for 0.1.12-devel --- CHANGELOG.md | 12 ++++++- .../package-lock.json | 2 +- .../package-lock.json | 2 +- examples/package-lock.json | 3 +- examples/typescript/package-lock.json | 34 +------------------ examples/typescript/package.json | 1 - lib/util.js | 2 +- package-lock.json | 4 +-- package.json | 2 +- 9 files changed, 20 insertions(+), 42 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 51ea4e1a..2ebafa9f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,20 @@ -# confluent-kafka-javascript vNext +# confluent-kafka-javascript v0.1.12-devel v0.1.12-devel is a pre-production, early-access release. ## Features 1. Add support for `listTopics` in the Admin API. +2. Add support for OAUTHBEARER token refresh callback for both promisified and non promisified API. + +## Bug Fixes + +1. Fix aliasing bug between `NodeKafka::Conf` and `RdKafka::ConfImpl`. +2. Fix issue where `assign/unassign` were called instead of `incrementalAssign/incrementalUnassign` while using + the Cooperative Sticky assigner, and setting the `rebalance_cb` as a boolean rather than as a function. +3. Fix memory leaks in Dispatcher and Conf (both leaked memory at client close). +4. Fix type definitions and make `KafkaJS` and `RdKafka` separate namespaces, while maintaining compatibility + with node-rdkafka's type definitions. # confluent-kafka-javascript v0.1.11-devel diff --git a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json index ffe3c834..4fe0bdb2 100644 --- a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json +++ b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json @@ -15,7 +15,7 @@ }, "../../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.11-devel", + "version": "v0.1.12-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json index d3d70cce..3bb7a621 100644 --- a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json +++ b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json @@ -14,7 +14,7 @@ } }, "../../..": { - "version": "v0.1.11-devel", + "version": "v0.1.12-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/package-lock.json b/examples/package-lock.json index cdc9bbba..31c75172 100644 --- a/examples/package-lock.json +++ b/examples/package-lock.json @@ -13,7 +13,8 @@ } }, "..": { - "version": "v0.1.11-devel", + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.12-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/typescript/package-lock.json b/examples/typescript/package-lock.json index 48c8d652..99897150 100644 --- a/examples/typescript/package-lock.json +++ b/examples/typescript/package-lock.json @@ -10,7 +10,6 @@ "license": "MIT", "dependencies": { "@confluentinc/kafka-javascript": "file:../..", - "node-rdkafka": "^2.18.0", "typescript": "^5.4.4" }, "devDependencies": { @@ -19,7 +18,7 @@ }, "../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.11-devel", + "version": "v0.1.12-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { @@ -55,37 +54,6 @@ "undici-types": "~5.26.4" } }, - "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/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/nan": { - "version": "2.19.0", - "resolved": "https://registry.npmjs.org/nan/-/nan-2.19.0.tgz", - "integrity": "sha512-nO1xXxfh/RWNxfd/XPfbIfFk5vgLsAxUR9y5O0cHMJu/AW9U95JLXqthYHjEp+8gQ5p96K9jUp8nbVOxCdRbtw==" - }, - "node_modules/node-rdkafka": { - "version": "2.18.0", - "resolved": "https://registry.npmjs.org/node-rdkafka/-/node-rdkafka-2.18.0.tgz", - "integrity": "sha512-jYkmO0sPvjesmzhv1WFOO4z7IMiAFpThR6/lcnFDWgSPkYL95CtcuVNo/R5PpjujmqSgS22GMkL1qvU4DTAvEQ==", - "hasInstallScript": true, - "dependencies": { - "bindings": "^1.3.1", - "nan": "^2.17.0" - }, - "engines": { - "node": ">=6.0.0" - } - }, "node_modules/typescript": { "version": "5.4.4", "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.4.tgz", diff --git a/examples/typescript/package.json b/examples/typescript/package.json index 20db593a..33a10079 100644 --- a/examples/typescript/package.json +++ b/examples/typescript/package.json @@ -10,7 +10,6 @@ "license": "MIT", "dependencies": { "@confluentinc/kafka-javascript": "file:../..", - "node-rdkafka": "^2.18.0", "typescript": "^5.4.4" }, "devDependencies": { diff --git a/lib/util.js b/lib/util.js index e33d0e00..37b99ca4 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.11-devel'; +util.bindingVersion = 'v0.1.12-devel'; diff --git a/package-lock.json b/package-lock.json index 14a68451..6fd62a6a 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.11-devel", + "version": "v0.1.12-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.11-devel", + "version": "v0.1.12-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index 61f9d21a..e7c645c9 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.11-devel", + "version": "v0.1.12-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.3.0", From fe26fed18c1cc3c6219f52781d6e660fe2829308 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 9 Apr 2024 21:27:50 +0530 Subject: [PATCH 122/224] Add docker login step to CI --- .semaphore/semaphore.yml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 9c29a28e..e80de063 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -38,6 +38,9 @@ blocks: agent: machine: type: s1-prod-ubuntu20-04-amd64-2 + prologue: + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' jobs: - name: "Build from source and test for musl" commands: @@ -49,6 +52,9 @@ blocks: agent: machine: type: s1-prod-ubuntu20-04-arm64-1 + prologue: + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' jobs: - name: "Build from source and test for musl" commands: @@ -110,6 +116,9 @@ blocks: value: "linux" - name: LIBC value: "glibc" + prologue: + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' jobs: - name: "Release: LTS:18" commands: @@ -154,6 +163,9 @@ blocks: value: "linux" - name: LIBC value: "glibc" + prologue: + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' jobs: - name: "Release: LTS:18" commands: @@ -198,6 +210,9 @@ blocks: value: "linux" - name: LIBC value: "musl" + prologue: + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' jobs: - name: "Release: LTS:18" commands: @@ -236,6 +251,9 @@ blocks: value: "linux" - name: LIBC value: "musl" + prologue: + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' jobs: - name: "Release: LTS:18" commands: From 56bf6ab012e78bb3e2b7096ad5c3f866c5ecf26b Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 10 Apr 2024 08:43:20 +0530 Subject: [PATCH 123/224] Update release binary download path --- package.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/package.json b/package.json index e7c645c9..6c2d5d09 100644 --- a/package.json +++ b/package.json @@ -18,7 +18,7 @@ "module_name": "confluent-kafka-javascript", "module_path": "./build/{configuration}/", "package_name": "{module_name}-v{version}-{node_abi}-{platform}-{libc}-{arch}.tar.gz", - "host": "https://confluent-kafka-js.s3.us-west-1.amazonaws.com/releases/download/", + "host": "https://github.com/confluentinc/confluent-kafka-javascript/releases/download/", "remote_path": "v{version}" }, "keywords": [ From 9a7530ddfcc0514ccbefab46c5521e6c43a8ed85 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 17 Apr 2024 09:43:18 +0530 Subject: [PATCH 124/224] Update QUICKSTART.md to reflect removed consume() API --- QUICKSTART.md | 44 +++++++++++++++++++++++--------------------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/QUICKSTART.md b/QUICKSTART.md index 1a027246..1dddd66b 100644 --- a/QUICKSTART.md +++ b/QUICKSTART.md @@ -1,7 +1,7 @@ # Basic Producer Example ```javascript -const { Kafka } = require('../..').KafkaJS +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function producerStart() { const producer = new Kafka().producer({ @@ -27,38 +27,40 @@ producerStart(); # Basic Consumer Example ```javascript -const { Kafka } = require('../..').KafkaJS +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; async function consumerStart() { - const consumer = new Kafka().consumer({ + let consumer; + let stopped = false; + + // Initialization + consumer = new Kafka().consumer({ 'bootstrap.servers': '', 'group.id': 'test', 'auto.offset.reset': 'earliest', }); await consumer.connect(); - - await consumer.subscribe({ topics: [ "topic" ] }); - - let stopped = false; - while (!stopped) { - const message = await consumer.consume(1000); - if (!message) { - continue; + await consumer.subscribe({ topics: ["topic"] }); + + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + console.log({ + topic, + partition, + offset: message.offset, + key: message.key?.toString(), + value: message.value.toString(), + }); } - console.log({ - topic: message.topic, - partition: message.partition, - offset: message.offset, - key: message.key?.toString(), - value: message.value.toString(), - }); + }); - // Update stopped whenever we're done consuming. - // stopped = true; + // Update stopped whenever we're done consuming. + // The update can be in another async function or scheduled with setTimeout etc. + while(!stopped) { + await new Promise(resolve => setTimeout(resolve, 1000)); } - // Disconnect and clean up. await consumer.disconnect(); } From 9989ae38fe8eb2c31b2650481a61cdbc56014488 Mon Sep 17 00:00:00 2001 From: Confluent Jenkins Bot Date: Wed, 17 Apr 2024 20:56:33 -0700 Subject: [PATCH 125/224] chore: update repo semaphore project (#21) Co-authored-by: ConfluentSemaphore <40306929+ConfluentSemaphore@users.noreply.github.com> --- .semaphore/project_public.yml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 .semaphore/project_public.yml diff --git a/.semaphore/project_public.yml b/.semaphore/project_public.yml new file mode 100644 index 00000000..fb83ce58 --- /dev/null +++ b/.semaphore/project_public.yml @@ -0,0 +1,25 @@ +# 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-javascript + description: "" +spec: + visibility: public + repository: + url: git@github.com:confluentinc/confluent-kafka-javascript.git + run_on: + - forked_pull_requests + pipeline_file: .semaphore/semaphore.yml + integration_type: github_app + status: + pipeline_files: + - path: .semaphore/semaphore.yml + level: pipeline + forked_pull_requests: + allowed_contributors: + - "ConfluentSemaphore" From dccab40404624dd505d3bdaf0e77f75f0d496a7f Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 19 Apr 2024 15:27:29 +0530 Subject: [PATCH 126/224] Update service.yaml for repository name --- service.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/service.yml b/service.yml index ec678454..c63b80c2 100644 --- a/service.yml +++ b/service.yml @@ -5,6 +5,7 @@ git: enable: true github: enable: true + repo_name: confluentinc/confluent-kafka-javascript codeowners: enable: true semaphore: From 6118b6664eb303a6df4adfb7a5e9fe118a18599d Mon Sep 17 00:00:00 2001 From: Martijn Imhoff Date: Mon, 22 Apr 2024 07:24:55 +0200 Subject: [PATCH 127/224] fix: memory leak in incremental assign (#35) * fix: memory leak in incremental assign * Update src/kafka-consumer.cc Co-authored-by: Milind L * docs: add comment about delete toppars --------- Co-authored-by: Milind L --- src/kafka-consumer.cc | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 061c6306..7d917e35 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -245,12 +245,15 @@ Baton KafkaConsumer::IncrementalUnassign(std::vector p RdKafka::Error* error = consumer->incremental_unassign(partitions); + std::vector delete_partitions; + if (error == NULL) { // For now, use two for loops. Make more efficient if needed at a later point. for (unsigned int i = 0; i < partitions.size(); i++) { for (unsigned int j = 0; j < m_partitions.size(); j++) { if (partitions[i]->partition() == m_partitions[j]->partition() && partitions[i]->topic() == m_partitions[j]->topic()) { + delete_partitions.push_back(m_partitions[j]); m_partitions.erase(m_partitions.begin() + j); m_partition_cnt--; break; @@ -260,7 +263,11 @@ Baton KafkaConsumer::IncrementalUnassign(std::vector p } // Destroy the old list of partitions since we are no longer using it - RdKafka::TopicPartition::destroy(m_partitions); + RdKafka::TopicPartition::destroy(delete_partitions); + + // Destroy the partition args since those are only used to lookup the partitions + // that needed to be deleted. + RdKafka::TopicPartition::destroy(partitions); return rdkafkaErrorToBaton(error); } From 08ecfc353b0060bcd071df5d0400aea01b7bdf58 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 23 Apr 2024 15:40:28 +0530 Subject: [PATCH 128/224] Add optimization where the size of messages in our subscription is less than the consumer cache size --- lib/kafka-consumer.js | 18 +++++++++++++++++- lib/kafkajs/_consumer.js | 3 +++ src/kafka-consumer.cc | 20 +++++++++++++++++--- src/workers.cc | 14 ++++++++++++-- src/workers.h | 3 ++- 5 files changed, 51 insertions(+), 7 deletions(-) diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 48ee30f7..cf65659b 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -20,6 +20,7 @@ var TopicPartition = require('./topic-partition'); var shallowCopy = require('./util').shallowCopy; var DEFAULT_CONSUME_LOOP_TIMEOUT_DELAY = 500; var DEFAULT_CONSUME_TIME_OUT = 1000; +const DEFAULT_IS_TIMEOUT_ONLY_FOR_FIRST_MESSAGE = false; util.inherits(KafkaConsumer, Client); /** @@ -142,6 +143,7 @@ function KafkaConsumer(conf, topicConf) { this._consumeTimeout = DEFAULT_CONSUME_TIME_OUT; this._consumeLoopTimeoutDelay = DEFAULT_CONSUME_LOOP_TIMEOUT_DELAY; + this._consumeIsTimeoutOnlyForFirstMessage = DEFAULT_IS_TIMEOUT_ONLY_FOR_FIRST_MESSAGE; } /** @@ -160,6 +162,20 @@ KafkaConsumer.prototype.setDefaultConsumeLoopTimeoutDelay = function(intervalMs) this._consumeLoopTimeoutDelay = intervalMs; }; +/** + * If true: + * In consume(number, cb), we will wait for `timeoutMs` for the first message to be fetched. + * Subsequent messages will not be waited for and will be fetched (upto `number`) if already ready. + * + * If false: + * In consume(number, cb), we will wait for upto `timeoutMs` for each message to be fetched. + * + * @param {boolean} isTimeoutOnlyForFirstMessage + */ +KafkaConsumer.prototype.setDefaultIsTimeoutOnlyForFirstMessage = function(isTimeoutOnlyForFirstMessage) { + this._consumeIsTimeoutOnlyForFirstMessage = isTimeoutOnlyForFirstMessage; +}; + /** * Get a stream representation of this KafkaConsumer * @@ -512,7 +528,7 @@ KafkaConsumer.prototype._consumeLoop = function(timeoutMs, cb) { KafkaConsumer.prototype._consumeNum = function(timeoutMs, numMessages, cb) { var self = this; - this._client.consume(timeoutMs, numMessages, function(err, messages, eofEvents) { + this._client.consume(timeoutMs, numMessages, this._consumeIsTimeoutOnlyForFirstMessage, function(err, messages, eofEvents) { if (err) { err = LibrdKafkaError.create(err); if (cb) { diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index bc164ee7..1c085d89 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -573,6 +573,9 @@ class Consumer { } this.#state = ConsumerState.CONNECTED; + /* Slight optimization for cases where the size of messages in our subscription is less than the cache size. */ + this.#internalClient.setDefaultIsTimeoutOnlyForFirstMessage(false); + // Resolve the promise. this.#connectPromiseFunc['resolve'](); } diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 7d917e35..4338bdcf 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -1330,7 +1330,11 @@ NAN_METHOD(KafkaConsumer::NodeConsume) { } if (info[1]->IsNumber()) { - if (!info[2]->IsFunction()) { + if (!info[2]->IsBoolean()) { + return Nan::ThrowError("Need to specify a boolean"); + } + + if (!info[3]->IsFunction()) { return Nan::ThrowError("Need to specify a callback"); } @@ -1344,12 +1348,22 @@ NAN_METHOD(KafkaConsumer::NodeConsume) { numMessages = numMessagesMaybe.FromJust(); } + v8::Local isTimeoutOnlyForFirstMessageBoolean = info[2].As(); + Nan::Maybe isTimeoutOnlyForFirstMessageMaybe = Nan::To(isTimeoutOnlyForFirstMessageBoolean); // NOLINT + + bool isTimeoutOnlyForFirstMessage; + if (isTimeoutOnlyForFirstMessageMaybe.IsNothing()) { + return Nan::ThrowError("Parameter must be a boolean"); + } else { + isTimeoutOnlyForFirstMessage = isTimeoutOnlyForFirstMessageMaybe.FromJust(); + } + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); - v8::Local cb = info[2].As(); + v8::Local cb = info[3].As(); Nan::Callback *callback = new Nan::Callback(cb); Nan::AsyncQueueWorker( - new Workers::KafkaConsumerConsumeNum(callback, consumer, numMessages, timeout_ms)); // NOLINT + new Workers::KafkaConsumerConsumeNum(callback, consumer, numMessages, timeout_ms, isTimeoutOnlyForFirstMessage)); // NOLINT } else { if (!info[1]->IsFunction()) { diff --git a/src/workers.cc b/src/workers.cc index c5f5b757..af924717 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -808,11 +808,13 @@ void KafkaConsumerConsumeLoop::HandleErrorCallback() { KafkaConsumerConsumeNum::KafkaConsumerConsumeNum(Nan::Callback *callback, KafkaConsumer* consumer, const uint32_t & num_messages, - const int & timeout_ms) : + const int & timeout_ms, + bool timeout_only_for_first_message) : ErrorAwareWorker(callback), m_consumer(consumer), m_num_messages(num_messages), - m_timeout_ms(timeout_ms) {} + m_timeout_ms(timeout_ms), + m_timeout_only_for_first_message(timeout_only_for_first_message) {} KafkaConsumerConsumeNum::~KafkaConsumerConsumeNum() {} @@ -849,6 +851,14 @@ void KafkaConsumerConsumeNum::Execute() { break; case RdKafka::ERR_NO_ERROR: m_messages.push_back(b.data()); + + // This allows getting ready messages, while not waiting for new ones. + // This is useful when we want to get the as many messages as possible + // within the timeout but not wait if we already have one or more messages. + if (m_timeout_only_for_first_message) { + timeout_ms = 1; + } + break; default: // Set the error for any other errors and break diff --git a/src/workers.h b/src/workers.h index f85add26..7da14599 100644 --- a/src/workers.h +++ b/src/workers.h @@ -441,7 +441,7 @@ class KafkaConsumerSeek : public ErrorAwareWorker { class KafkaConsumerConsumeNum : public ErrorAwareWorker { public: KafkaConsumerConsumeNum(Nan::Callback*, NodeKafka::KafkaConsumer*, - const uint32_t &, const int &); + const uint32_t &, const int &, bool); ~KafkaConsumerConsumeNum(); void Execute(); @@ -451,6 +451,7 @@ class KafkaConsumerConsumeNum : public ErrorAwareWorker { NodeKafka::KafkaConsumer * m_consumer; const uint32_t m_num_messages; const int m_timeout_ms; + const bool m_timeout_only_for_first_message; std::vector m_messages; }; From ed7226bacbb06af0d3ee9fa864e954c2c35e7539 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 23 Apr 2024 15:40:57 +0530 Subject: [PATCH 129/224] Add example to measure performance --- .../performance/performance-promisified.js | 149 ++++++++++++++++++ 1 file changed, 149 insertions(+) create mode 100644 examples/performance/performance-promisified.js diff --git a/examples/performance/performance-promisified.js b/examples/performance/performance-promisified.js new file mode 100644 index 00000000..b7b74054 --- /dev/null +++ b/examples/performance/performance-promisified.js @@ -0,0 +1,149 @@ +const { Kafka, CompressionTypes, ErrorCodes } = require('../../').KafkaJS; +const { randomBytes } = require('crypto'); +const { hrtime } = require('process'); + +async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { + let totalMessagesSent = 0; + let totalBytesSent = 0; + + const message = { + value: randomBytes(msgSize), + } + + const messages = Array(batchSize).fill(message); + + const kafka = new Kafka({ + 'client.id': 'kafka-test-performance', + 'metadata.broker.list': brokers, + 'compression.codec': compression, + }); + + const producer = kafka.producer(); + await producer.connect(); + + console.log('Sending ' + warmupMessages + ' warmup messages.'); + while (warmupMessages > 0) { + await producer.send({ + topic, + messages, + }); + warmupMessages -= batchSize; + } + console.log('Sent warmup messages'); + + // Now that warmup is done, start measuring... + let startTime; + let promises = []; + startTime = hrtime(); + let messagesDispatched = 0; + + // The double while-loop allows us to send a bunch of messages and then + // await them all at once. We need the second while loop to keep sending + // in case of queue full errors, which surface only on awaiting. + while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + promises.push(producer.send({ + topic, + messages, + }).then(() => { + totalMessagesSent += batchSize; + totalBytesSent += batchSize * msgSize; + }).catch((err) => { + if (err.code === ErrorCodes.ERR__QUEUE_FULL) { + /* do nothing, just send them again */ + messagesDispatched -= batchSize; + } else { + console.error(err); + throw err; + } + })); + messagesDispatched += batchSize; + } + await Promise.all(promises); + } + console.log({messagesDispatched, totalMessageCnt}) + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + let rate = (totalBytesSent / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Sent ${totalMessagesSent} messages, ${totalBytesSent} bytes; rate is ${rate} MB/s`); + + await producer.disconnect(); + return rate; +} + +async function runConsumer(brokers, topic, totalMessageCnt) { + const kafka = new Kafka({ + 'client.id': 'kafka-test-performance', + 'metadata.broker.list': brokers, + }); + + const consumer = kafka.consumer({ + 'group.id': 'test-group' + Math.random(), + 'enable.auto.commit': false, + 'auto.offset.reset': 'earliest', + }); + await consumer.connect(); + await consumer.subscribe({ topic }); + + let messagesReceived = 0; + let totalMessageSize = 0; + let startTime; + let rate; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + messagesReceived++; + totalMessageSize += message.value.length; + if (messagesReceived === 1) { + consumer.pause([{ topic }]); + } else if (messagesReceived === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + // } else if (messagesReceived % 100 == 0) { + // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); + } + } + }); + + // Wait until the first message is received + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived > 0) { + clearInterval(interval); + resolve(); + } + }, 100); + }); + + console.log("Starting consumer.") + + totalMessageSize = 0; + startTime = hrtime(); + consumer.resume([{ topic }]); + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived >= totalMessageCnt) { + clearInterval(interval); + resolve(); + } + }, 1000); + }); + + await consumer.disconnect(); + return rate; +} + +const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; +const topic = process.env.KAFKA_TOPIC || 'test-topic'; +const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; +const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; +const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; +const compression = process.env.COMPRESSION || CompressionTypes.NONE; +const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); + +runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression).then(async (producerRate) => { + const consumerRate = await runConsumer(brokers, topic, messageCount); + console.log(producerRate, consumerRate); +}); From 938df93ff1c6b84f2665e2b6542bb1c3275eb7a4 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 25 Apr 2024 15:12:48 +0530 Subject: [PATCH 130/224] Add storeOffsets and fix typings --- lib/kafkajs/_consumer.js | 24 +++ .../consumer/consumerCacheTests.spec.js | 1 - test/promisified/consumer/store.spec.js | 154 ++++++++++++++++++ types/kafkajs.d.ts | 2 + 4 files changed, 180 insertions(+), 1 deletion(-) create mode 100644 test/promisified/consumer/store.spec.js diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 1c085d89..515909cb 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1062,6 +1062,30 @@ class Consumer { // return m ?? null; } + /** + * Store offsets for the given topic partitions. + * + * Stored offsets will be commited automatically at a later point if autoCommit is enabled. + * Otherwise, they will be committed when commitOffsets is called without arguments. + * + * enable.auto.offset.store must be set to false to use this API. + * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]?} topicPartitions + */ + storeOffsets(topicPartitions) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Store can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + if (!this.#userManagedStores) { + throw new error.KafkaJSError( + 'Store can only be called when enable.auto.offset.store is explicitly set to false.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + const topicPartitionsRdKafka = topicPartitions.map( + topicPartitionOffsetToRdKafka); + this.#internalClient.offsetsStore(topicPartitionsRdKafka); + } + async #commitOffsetsUntilNoStateErr(offsetsToCommit) { let err = { code: error.ErrorCodes.ERR_NO_ERROR }; do { diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index 56033c60..a8791079 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -1,6 +1,5 @@ jest.setTimeout(30000) -const { is } = require('bluebird'); const { secureRandom, createTopic, diff --git a/test/promisified/consumer/store.spec.js b/test/promisified/consumer/store.spec.js new file mode 100644 index 00000000..3ee056d9 --- /dev/null +++ b/test/promisified/consumer/store.spec.js @@ -0,0 +1,154 @@ +jest.setTimeout(30000) + +const { + secureRandom, + createTopic, + waitFor, + createProducer, + createConsumer, + sleep, +} = require('../testhelpers'); +const { ErrorCodes } = require('../../../lib').KafkaJS; + +describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}` + + await createTopic({ topic: topicName, partitions: 3 }) + + producer = createProducer({}); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: isAutoCommit, + autoCommitInterval: 500, + }, { + 'enable.auto.offset.store': false, + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }); + + it('should not work if enable.auto.offset.store = true', async () => { + let assignment = []; + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }, { + /* Set to true manually - the default value with kafkaJS block is false. */ + 'enable.auto.offset.store': true, + 'rebalance_cb': function (err, asg) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assignment = asg; + } + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async () => { + } + }); + await waitFor(() => assignment.length > 0, () => null, 1000); + expect( + () => consumer.storeOffsets([{ topic: topicName, partition: 0, offset: '10' }]) + ).toThrow(/Store can only be called when enable.auto.offset.store is explicitly set to false/); + }); + + it('should not work if enable.auto.offset.store is unset', async () => { + let assignment = []; + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }, { + /* Set to true manually - the default value with kafkaJS block is false. */ + 'rebalance_cb': function (err, asg) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assignment = asg; + } + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async () => { + } + }); + await waitFor(() => assignment.length > 0, () => null, 1000); + expect( + () => consumer.storeOffsets([{ topic: topicName, partition: 0, offset: '10' }]) + ).toThrow(/Store can only be called when enable.auto.offset.store is explicitly set to false/); + }); + + it('should commit stored offsets', async () => { + /* Evenly distribute 30 messages across 3 partitions */ + let i = 0; + const messages = Array(3 * 10) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.connect(); + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + let msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + msgCount++; + const offset = (Number(message.offset) + 1).toString(); + expect(() => consumer.storeOffsets([{ topic, partition, offset }])).not.toThrow(); + } + }); + await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); + expect(msgCount).toEqual(30); + + if (!isAutoCommit) + await expect(consumer.commitOffsets()).resolves.toBeUndefined(); + else + await sleep(1000); /* Wait for auto-commit */ + + await consumer.disconnect(); + + /* Send 30 more messages */ + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + + msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async ({ message }) => { + msgCount++; + } + }) + /* Only the extra 30 messages should come to us */ + await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); + await sleep(1000); + expect(msgCount).toEqual(30); + }); + +}); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 75a7390a..2feec790 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -136,6 +136,7 @@ type Sender = { export type Producer = Sender & { connect(): Promise disconnect(): Promise + flush(args?: {timeout?: number}): Promise } export interface RetryOptions { @@ -460,6 +461,7 @@ export type Consumer = { subscribe(subscription: ConsumerSubscribeTopics | ConsumerSubscribeTopic): Promise stop(): Promise run(config?: ConsumerRunConfig): Promise + storeOffsets(topicPartitions: Array): void commitOffsets(topicPartitions: Array): Promise seek(topicPartitionOffset: TopicPartitionOffset): Promise describeGroup(): Promise From 788a9723af1044f4c70fe0be149c350133cdada7 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 6 May 2024 15:28:11 +0530 Subject: [PATCH 131/224] Update comments and fix small issues --- lib/kafkajs/_common.js | 2 +- lib/kafkajs/_consumer.js | 4 ++-- lib/kafkajs/_producer.js | 21 +++++++++++++++---- .../producer/concurrentTransaction.spec.js | 1 + 4 files changed, 21 insertions(+), 7 deletions(-) diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index f6d8a43e..11182b01 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -188,7 +188,7 @@ const CompatibilityErrorMessages = Object.freeze({ socketFactory: () => "The socketFactory property is not supported.", logLevelName: (setLevel) => - "The log level must be one of: " + logLevel.keys().join(", ") + ", was " + setLevel, + "The log level must be one of: " + Object.keys(logLevel).join(", ") + ", was " + setLevel, reauthenticationThreshold: () => "Reauthentication threshold cannot be set, and reauthentication is automated when 80% of connections.max.reauth.ms is reached.", unsupportedKey: (key) => diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 515909cb..2010d27f 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1069,7 +1069,7 @@ class Consumer { * Otherwise, they will be committed when commitOffsets is called without arguments. * * enable.auto.offset.store must be set to false to use this API. - * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]?} topicPartitions + * @param {import("../../types/kafkajs").TopicPartitionOffset[]?} topicPartitions */ storeOffsets(topicPartitions) { if (this.#state !== ConsumerState.CONNECTED) { @@ -1099,7 +1099,7 @@ class Consumer { /** * Commit offsets for the given topic partitions. If topic partitions are not specified, commits all offsets. - * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]?} topicPartitions + * @param {import("../../types/kafkajs").TopicPartitionOffset[]?} topicPartitions * @returns {Promise} a promise that resolves when the offsets have been committed. */ async commitOffsets(topicPartitions = null) { diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index f38d3697..163fdc1b 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -501,8 +501,19 @@ class Producer { } /** - * send(record: ProducerRecord): Promise + * Check if there is an ongoing transaction. + * + * NOTE: Since Producer itself represents a transaction, and there is no distinct + * type for a transaction, this method exists on the producer. + * @returns {boolean} true if there is an ongoing transaction, false otherwise. + */ + isActive() { + return this.#ongoingTransaction; + } + /** + * Sends a record of messages to a specific topic. + * * @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. */ @@ -572,7 +583,7 @@ class Producer { const topicPartitionRecordMetadata = new Map(); for (const recordMetadata of recordMetadataArr) { const key = `${recordMetadata.topicName},${recordMetadata.partition}`; - if (recordMetadata.baseOffset == null || !topicPartitionRecordMetadata.has(key)) { + if (recordMetadata.baseOffset === null || !topicPartitionRecordMetadata.has(key)) { topicPartitionRecordMetadata.set(key, recordMetadata); continue; } @@ -580,7 +591,7 @@ class Producer { const currentRecordMetadata = topicPartitionRecordMetadata.get(key); // Don't overwrite a null baseOffset - if (currentRecordMetadata.baseOffset == null) { + if (currentRecordMetadata.baseOffset === null) { continue; } @@ -598,7 +609,9 @@ class Producer { } /** - * sendBatch(batch: ProducerBatch): Promise + * Sends a record of messages to various topics. + * + * NOTE: This method is identical to calling send() repeatedly and waiting on all the return values together. * @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. */ diff --git a/test/promisified/producer/concurrentTransaction.spec.js b/test/promisified/producer/concurrentTransaction.spec.js index 09ca0175..d530c1a7 100644 --- a/test/promisified/producer/concurrentTransaction.spec.js +++ b/test/promisified/producer/concurrentTransaction.spec.js @@ -34,6 +34,7 @@ describe('Producer > Transactional producer', () => { producer1 = newProducer(); await producer1.connect(); const transaction1 = await producer1.transaction(); + expect(transaction1.isActive()).toBe(true); await transaction1.send({ topic: topicName, messages: [message] }); // Producer 2 starts with the same transactional id to cause the concurrent transactions error From d9958cf12feca91103715b0e744a79c5f458ee6d Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 6 May 2024 15:55:02 +0530 Subject: [PATCH 132/224] Reconcile createTopics type and behaviour --- lib/kafkajs/_admin.js | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 730a9af4..1fe71467 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -207,7 +207,7 @@ class Admin { /** * 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. + * @returns {Promise} Resolves true when the topics are created, false if topic exists already, rejects on error. */ async createTopics(options) { if (this.#state !== AdminState.CONNECTED) { @@ -229,9 +229,13 @@ class Admin { .map(topicConfig => new Promise((resolve, reject) => { this.#internalClient.createTopic(topicConfig, options.timeout ?? 5000, (err) => { if (err) { + if (err.code === error.ErrorCodes.ERR_TOPIC_ALREADY_EXISTS) { + resolve(false); + return; + } reject(createKafkaJsErrorFromLibRdKafkaError(err)); } else { - resolve(); + resolve(true); } }); })); From eb30bde83bb010398b8992ac3f7bf277caf82666 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 6 May 2024 16:02:50 +0530 Subject: [PATCH 133/224] Reconcile types and MIGRATION.md with library --- MIGRATION.md | 81 ++++---- types/kafkajs.d.ts | 475 ++++++++++++--------------------------------- 2 files changed, 174 insertions(+), 382 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 57f76faf..b82f19cc 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -17,6 +17,7 @@ ``` 2. Try running your program. In case a migration is needed, an informative error will be thrown. + If you're using Typescript, some of these changes will be caught at compile time. 3. The most common expected changes to the code are: - For the **producer**: `acks`, `compression` and `timeout` are not set on a per-send() basis. @@ -26,7 +27,8 @@ Rather, it must be configured in the top-level configuration while creating the consumer. - `autoCommit` and `autoCommitInterval` are not set on a per-run() basis. Rather, they must be configured in the top-level configuration while creating the consumer. - - `autoCommitThreshold` is not supported and `eachBatch` batch size never exceeds 1. + - `autoCommitThreshold` is not supported. + - `eachBatch`'s batch size never exceeds 1. - For errors: Check the `error.code` rather than the error `name` or `type`. 4. A more exhaustive list of semantic and configuration differences is [presented below](#common). @@ -91,7 +93,7 @@ producerRun().then(consumerRun).catch(console.error); | Property | Default Value | Comment | |-------------------------------|--------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| - | **brokers** | null | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | + | **brokers** | - | A list of strings, representing the bootstrap brokers. **A function is no longer allowed as an argument for this.** | | **ssl** | false | A boolean, set to true if ssl needs to be enabled. **Additional properties like CA, certificate, key, etc. need to be specified outside the kafkaJS block.** | | **sasl** | - | An optional object of the form `{ mechanism: 'plain' | 'scram-sha-256' | 'scram-sha-512', username: string, password: string }` or `{ mechanism: 'oauthbearer', oauthBearerProvider: function }`. Note that for OAUTHBEARER based authentication, the provider function must return lifetime (in ms), and principal name along with token value. **Additional authentication types are not supported.** | | clientId | "rdkafka" | An optional string used to identify the client. | @@ -134,7 +136,7 @@ producerRun().then(consumerRun).catch(console.error); | **maxInFlightRequests** | null | Maximum number of in-flight requests **per broker connection**. If not set, it is practically unbounded (same as KafkaJS). | | transactionalId | null | If set, turns this into a transactional producer with this identifier. This also automatically sets `idempotent` to true. | | **acks** | -1 | The number of required acks before a Produce succeeds. **This is set on a per-producer level, not on a per `send` level**. -1 denotes it will wait for all brokers in the in-sync replica set. | - | **compression** | CompressionTypes.NONE | Compression codec for Produce messages. **This is set on a per-producer level, not on a per `send` level**. It must be a key of the object CompressionType, namely GZIP, SNAPPY, LZ4, ZSTD or NONE. | + | **compression** | CompressionTypes.NONE | Compression codec for Produce messages. **This is set on a per-producer level, not on a per `send` level**. It must be a key of CompressionType, namely GZIP, SNAPPY, LZ4, ZSTD or NONE. | | **timeout** | 30000 | The ack timeout of the producer request in milliseconds. This value is only enforced by the broker. **This is set on a per-producer level, not on a per `send` level**. | | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | @@ -192,26 +194,26 @@ producerRun().then(consumerRun).catch(console.error); Each allowed config property is discussed in the table below. If there is any change in semantics or the default values, the property and the change is **highlighted in bold**. - | Property | Default Value | Comment | - |--------------------------|-----------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| - | groupId | null | A mandatory string denoting consumer group name that this consumer is a part of. | - | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range, roundRobin, and cooperativeSticky assignors is provided. Custom assignors are not supported. | - | **partitionAssignors** | `[PartitionAssignors.roundRobin]` | Alias for `partitionAssigners` | - | **rebalanceTimeout** | **300000** | The maximum allowed time for each member to join the group once a rebalance has begun. Note, that setting this value *also* changes the max poll interval. Message processing in `eachMessage` must not take more than this time. | - | heartbeatInterval | 3000 | The expected time in milliseconds between heartbeats to the consumer coordinator. | - | metadataMaxAge | 5 minutes | Time in milliseconds after which to refresh metadata for known topics | - | allowAutoTopicCreation | true | Determines if a topic should be created if it doesn't exist while consuming. | - | **maxBytesPerPartition** | 1048576 (1MB) | determines how many bytes can be fetched in one request from a single partition. There is a change in semantics, this size grows dynamically if a single message larger than this is encountered, and the client does not get stuck. | - | minBytes | 1 | Minimum number of bytes the broker responds with (or wait until `maxWaitTimeInMs`) | - | maxBytes | 10485760 (10MB) | Maximum number of bytes the broker responds with. | - | **retry** | object | Identical to `retry` in the common configuration. This takes precedence over the common config retry. | - | readUncommitted | false | If true, consumer will read transactional messages which have not been committed. | - | **maxInFlightRequests** | null | Maximum number of in-flight requests **per broker connection**. If not set, it is practically unbounded (same as KafkaJS). | - | rackId | null | Can be set to an arbitrary string which will be used for fetch-from-follower if set up on the cluster. | - | **fromBeginning** | false | If there is initial offset in offset store or the desired offset is out of range, and this is true, we consume the earliest possible offset. **This is set on a per-consumer level, not on a per `subscribe` level**. | - | **autoCommit** | true | Whether to periodically auto-commit offsets to the broker while consuming. **This is set on a per-consumer level, not on a per `run` level**. | - | **autoCommitInterval** | 5000 | Offsets are committed periodically at this interval, if autoCommit is true. **This is set on a per-consumer level, not on a per `run` level. The default value is changed to 5 seconds.**. | - | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | + | Property | Default Value | Comment | + |--------------------------|-----------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | groupId | - | A mandatory string denoting consumer group name that this consumer is a part of. | + | **partitionAssigners** | `[PartitionAssigners.roundRobin]` | Support for range, roundRobin, and cooperativeSticky assignors is provided. Custom assignors are not supported. | + | **partitionAssignors** | `[PartitionAssignors.roundRobin]` | Alias for `partitionAssigners` | + | **rebalanceTimeout** | **300000** | The maximum allowed time for each member to join the group once a rebalance has begun. Note, that setting this value *also* changes the max poll interval. Message processing in `eachMessage/eachBatch` must not take more than this time. | + | heartbeatInterval | 3000 | The expected time in milliseconds between heartbeats to the consumer coordinator. | + | metadataMaxAge | 5 minutes | Time in milliseconds after which to refresh metadata for known topics | + | allowAutoTopicCreation | true | Determines if a topic should be created if it doesn't exist while consuming. | + | **maxBytesPerPartition** | 1048576 (1MB) | determines how many bytes can be fetched in one request from a single partition. There is a change in semantics, this size grows dynamically if a single message larger than this is encountered, and the client does not get stuck. | + | minBytes | 1 | Minimum number of bytes the broker responds with (or wait until `maxWaitTimeInMs`) | + | maxBytes | 10485760 (10MB) | Maximum number of bytes the broker responds with. | + | **retry** | object | Identical to `retry` in the common configuration. This takes precedence over the common config retry. | + | readUncommitted | false | If true, consumer will read transactional messages which have not been committed. | + | **maxInFlightRequests** | null | Maximum number of in-flight requests **per broker connection**. If not set, it is practically unbounded (same as KafkaJS). | + | rackId | null | Can be set to an arbitrary string which will be used for fetch-from-follower if set up on the cluster. | + | **fromBeginning** | false | If there is initial offset in offset store or the desired offset is out of range, and this is true, we consume the earliest possible offset. **This is set on a per-consumer level, not on a per `subscribe` level**. | + | **autoCommit** | true | Whether to periodically auto-commit offsets to the broker while consuming. **This is set on a per-consumer level, not on a per `run` level**. | + | **autoCommitInterval** | 5000 | Offsets are committed periodically at this interval, if autoCommit is true. **This is set on a per-consumer level, not on a per `run` level. The default value is changed to 5 seconds.**. | + | outer config | {} | The configuration outside the kafkaJS block can contain any of the keys present in the [librdkafka CONFIGURATION table](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). | #### Semantic and Per-Method Changes @@ -280,36 +282,43 @@ producerRun().then(consumerRun).catch(console.error); eachMessage: someFunc, }); ``` - - The `heartbeat()` no longer needs to be called by the user in the `eachMessage callback`. Heartbeats are automatically managed by librdkafka. + - The `heartbeat()` no longer needs to be called by the user in the `eachMessage/eachBatch` callback. Heartbeats are automatically managed by librdkafka. - The `partitionsConsumedConcurrently` property is not supported at the moment. - - An API compatible version of `eachBatch` is available, but the batch size never exceeds 1. The property `eachBatchAutoResolve` is supported. + - An API compatible version of `eachBatch` is available, but the batch size never exceeds 1. + The property `eachBatchAutoResolve` is supported. Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, + and within the returned batch, `offsetLag` and `offsetLagLow` are unsupported, and `commitOffsetsIfNecessary` is a no-op. * `commitOffsets`: - Does not yet support sending metadata for topic partitions being committed. + - If called with no arguments, it commits all offsets passed to the user (or the stored offsets, if manually handling offset storage using `consumer.storeOffsets`). * `seek`: - The restriction to call seek only after `run` is removed. It can be called any time. * `pause` and `resume`: - These methods 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. + In practice, this means it can be called whenever `consumer.assignment()` has a non-zero size, or within the `eachMessage/eachBatch` callback. * `stop` is not yet supported, and the user must disconnect the consumer. ### 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. +The admin-client only has support for a limited subset of methods, with more to be added. + + * 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. + * The `listTopics` method is supported with an additional `timeout` option. + * The `listGroups` method is supported with additional `timeout` and `matchConsumerGroupStates` options. + A number of additional properties have been added to the returned groups, and a list of errors within the returned object. + * The `describeGroups` method is supported with additional `timeout` and `includeAuthorizedOperations` options. + A number of additional properties have been added to the returned groups. + * The `deleteGroups` method is supported with an additional `timeout` option. ### Using the Schema Registry In case you are using the Schema Registry client at `kafkajs/confluent-schema-registry`, you will not need to make any changes to the usage. -An example is made available [here](../examples/kafkajs/sr.js). +An example is made available [here](./examples/kafkajs/sr.js). ### Error Handling - **Action**: Convert any checks based on `instanceof` and `error.name` or to error - checks based on `error.code` or `error.type`. + Convert any checks based on `instanceof` and `error.name` or to error checks based on `error.code` or `error.type`. **Example**: ```javascript @@ -324,7 +333,7 @@ An example is made available [here](../examples/kafkajs/sr.js). } ``` - **Comprehensive Changes**: + **Error Type Changes**: Some possible subtypes of `KafkaJSError` have been removed, and additional information has been added into `KafkaJSError`. diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 2feec790..e182c4b6 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,15 +1,21 @@ -import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig } from './config' -import { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescription, GroupDescriptions, DeleteGroupsResult } from './rdkafka' +import { ConsumerGlobalConfig, GlobalConfig, ProducerGlobalConfig } from './config' +import { + ConsumerGroupStates, + GroupOverview, + LibrdKafkaError, + GroupDescriptions, + DeleteGroupsResult +} from './rdkafka' // Admin API related interfaces, types etc; and Error types are common, so // just re-export them from here too. -export { ConsumerGroupStates, GroupOverview, LibrdKafkaError, GroupDescriptions, DeleteGroupsResult } from './rdkafka' - -export type BrokersFunction = () => string[] | Promise - -export type Mechanism = { - mechanism: string -} +export { + ConsumerGroupStates, + GroupOverview, + LibrdKafkaError, + GroupDescriptions, + DeleteGroupsResult +} from './rdkafka' export interface OauthbearerProviderResponse { value: string, @@ -31,22 +37,69 @@ type SASLMechanismOptions = T extends SASLMechanism : never export type SASLOptions = SASLMechanismOptions +export interface RetryOptions { + maxRetryTime?: number + initialRetryTime?: number + retries?: number +} + +export enum logLevel { + NOTHING = 0, + ERROR = 1, + WARN = 2, + INFO = 3, + DEBUG = 4, +} + +export type Logger = { + info: (message: string, extra?: object) => void + error: (message: string, extra?: object) => void + warn: (message: string, extra?: object) => void + debug: (message: string, extra?: object) => void + + namespace: (namespace: string, logLevel?: logLevel) => Logger + setLogLevel: (logLevel: logLevel) => void +} + export interface KafkaConfig { - brokers: string[] | BrokersFunction - ssl?: boolean - sasl?: SASLOptions | Mechanism + brokers: string[], + ssl?: boolean, + sasl?: SASLOptions, clientId?: string connectionTimeout?: number authenticationTimeout?: number - reauthenticationThreshold?: number requestTimeout?: number - enforceRequestTimeout?: boolean + enforceRequestTimeout?: boolean, + retry?: RetryOptions, + logLevel?: logLevel, } export interface CommonConstructorConfig extends GlobalConfig { kafkaJS?: KafkaConfig; } +export class Kafka { + constructor(config: CommonConstructorConfig) + producer(config?: ProducerConstructorConfig): Producer + consumer(config: ConsumerConstructorConfig): Consumer + admin(config?: AdminConstructorConfig): Admin +} + +type Client = { + connect(): Promise + disconnect(): Promise + logger(): Logger + setSaslCredentialProvider(authInfo: { username: string, password: string }): void +} + +export enum CompressionTypes { + None = 'none', + GZIP = 'gzip', + Snappy = 'snappy', + LZ4 = 'lz4', + ZSTD = 'zstd', +} + export interface ProducerConfig { metadataMaxAge?: number allowAutoTopicCreation?: boolean @@ -56,8 +109,8 @@ export interface ProducerConfig { maxInFlightRequests?: number acks?: number compression?: CompressionTypes - timeout?: number - rdKafka?: { topicConfig?: ProducerTopicConfig, globalConfig?: ProducerGlobalConfig } + timeout?: number, + retry?: RetryOptions, } export interface ProducerConstructorConfig extends ProducerGlobalConfig { @@ -76,27 +129,9 @@ export interface Message { timestamp?: string } -export enum CompressionTypes { - None = 0, - GZIP = 1, - Snappy = 2, - LZ4 = 3, - ZSTD = 4, -} - -export var CompressionCodecs: { - [CompressionTypes.GZIP]: () => any - [CompressionTypes.Snappy]: () => any - [CompressionTypes.LZ4]: () => any - [CompressionTypes.ZSTD]: () => any -} - export interface ProducerRecord { topic: string messages: Message[] - acks?: number - timeout?: number - compression?: CompressionTypes } export interface TopicMessages { @@ -105,9 +140,6 @@ export interface TopicMessages { } export interface ProducerBatch { - acks?: number - timeout?: number - compression?: CompressionTypes topicMessages?: TopicMessages[] } @@ -122,34 +154,29 @@ export type RecordMetadata = { logStartOffset?: string } -export class Kafka { - constructor(config: CommonConstructorConfig) - producer(config?: ProducerConstructorConfig): Producer - consumer(config: ConsumerConstructorConfig): Consumer - admin(config?: AdminConstructorConfig): Admin -} +export type Transaction = Producer; -type Sender = { +export type Producer = Client & { send(record: ProducerRecord): Promise -} + sendBatch(batch: ProducerBatch): Promise + flush(args?: { timeout?: number }): Promise -export type Producer = Sender & { - connect(): Promise - disconnect(): Promise - flush(args?: {timeout?: number}): Promise + // Transactional producer-only methods. + transaction(): Promise + commit(): Promise + abort(): Promise + sendOffsets(args: { consumerGroupId?: string, consumer?: Consumer, topics: TopicOffsets[] }): Promise + isActive(): boolean } -export interface RetryOptions { - maxRetryTime?: number - initialRetryTime?: number - factor?: number - multiplier?: number - retries?: number - restartOnFailure?: (e: Error) => Promise +export enum PartitionAssigners { + roundRobin = 'roundrobin', + range = 'range', + cooperativeSticky = 'cooperative-sticky' } -export enum PartitionAssigners { - roundRobin = 'roundRobin', +export enum PartitionAssignors { + roundRobin = 'roundrobin', range = 'range', cooperativeSticky = 'cooperative-sticky' } @@ -164,7 +191,7 @@ export interface ConsumerConfig { minBytes?: number maxBytes?: number maxWaitTimeInMs?: number - retry?: RetryOptions & { restartOnFailure?: (err: Error) => Promise } + retry?: RetryOptions, allowAutoTopicCreation?: boolean maxInFlightRequests?: number readUncommitted?: boolean @@ -172,188 +199,14 @@ export interface ConsumerConfig { fromBeginning?: boolean autoCommit?: boolean autoCommitInterval?: number, - partitionAssigners?: PartitionAssigners[] - partitionAssignors?: PartitionAssigners[] - rdKafka?: { topicConfig?: ConsumerTopicConfig, globalConfig?: ConsumerGlobalConfig } + partitionAssigners?: PartitionAssigners[], + partitionAssignors?: PartitionAssignors[], } export interface ConsumerConstructorConfig extends ConsumerGlobalConfig { kafkaJS?: ConsumerConfig; } -export type ConsumerEvents = { - HEARTBEAT: 'consumer.heartbeat' - COMMIT_OFFSETS: 'consumer.commit_offsets' - GROUP_JOIN: 'consumer.group_join' - FETCH_START: 'consumer.fetch_start' - FETCH: 'consumer.fetch' - START_BATCH_PROCESS: 'consumer.start_batch_process' - END_BATCH_PROCESS: 'consumer.end_batch_process' - CONNECT: 'consumer.connect' - DISCONNECT: 'consumer.disconnect' - STOP: 'consumer.stop' - CRASH: 'consumer.crash' - REBALANCING: 'consumer.rebalancing' - RECEIVED_UNSUBSCRIBED_TOPICS: 'consumer.received_unsubscribed_topics' - REQUEST: 'consumer.network.request' - REQUEST_TIMEOUT: 'consumer.network.request_timeout' - REQUEST_QUEUE_SIZE: 'consumer.network.request_queue_size' -} - -export interface AdminConfig { - retry?: RetryOptions -} - -export interface AdminConstructorConfig extends GlobalConfig { - kafkaJS?: AdminConfig; -} - -export interface ReplicaAssignment { - partition: number - replicas: Array -} - -export interface IResourceConfigEntry { - name: string - value: string -} - -export enum logLevel { - NOTHING = 0, - ERROR = 1, - WARN = 2, - INFO = 4, - DEBUG = 5, -} - -export type Logger = { - info: (message: string, extra?: object) => void - error: (message: string, extra?: object) => void - warn: (message: string, extra?: object) => void - debug: (message: string, extra?: object) => void - - namespace: (namespace: string, logLevel?: logLevel) => Logger - setLogLevel: (logLevel: logLevel) => void -} - -type ValueOf = T[keyof T] - -export interface InstrumentationEvent { - id: string - type: string - timestamp: number - payload: T -} - -export type RemoveInstrumentationEventListener = () => void - -export type ConsumerFetchStartEvent = InstrumentationEvent<{ nodeId: number }> -export type ConsumerFetchEvent = InstrumentationEvent<{ - numberOfBatches: number - duration: number - nodeId: number -}> - -export type ConsumerHeartbeatEvent = InstrumentationEvent<{ - groupId: string - memberId: string - groupGenerationId: number -}> - -export type ConsumerCommitOffsetsEvent = InstrumentationEvent<{ - groupId: string - memberId: string - groupGenerationId: number - topics: TopicOffsets[] -}> - -export interface IMemberAssignment { - [key: string]: number[] -} - -export type ConsumerGroupJoinEvent = InstrumentationEvent<{ - duration: number - groupId: string - isLeader: boolean - leaderId: string - groupProtocol: string - memberId: string - memberAssignment: IMemberAssignment -}> - -interface IBatchProcessEvent { - topic: string - partition: number - highWatermark: string - offsetLag: string - offsetLagLow: string - batchSize: number - firstOffset: string - lastOffset: string -} - -export type ConsumerStartBatchProcessEvent = InstrumentationEvent - -export type ConsumerEndBatchProcessEvent = InstrumentationEvent< - IBatchProcessEvent & { duration: number } -> - -export type ConnectEvent = InstrumentationEvent - -export type DisconnectEvent = InstrumentationEvent - -export type ConsumerCrashEvent = InstrumentationEvent<{ - error: Error - groupId: string - restart: boolean -}> - -export type ConsumerRebalancingEvent = InstrumentationEvent<{ - groupId: string - memberId: string -}> - -export type ConsumerReceivedUnsubcribedTopicsEvent = InstrumentationEvent<{ - groupId: string - generationId: number - memberId: string - assignedTopics: string[] - topicsSubscribed: string[] - topicsNotSubscribed: string[] -}> - -export type RequestEvent = InstrumentationEvent<{ - apiKey: number - apiName: string - apiVersion: number - broker: string - clientId: string - correlationId: number - createdAt: number - duration: number - pendingDuration: number - sentAt: number - size: number -}> - -export type RequestTimeoutEvent = InstrumentationEvent<{ - apiKey: number - apiName: string - apiVersion: number - broker: string - clientId: string - correlationId: number - createdAt: number - pendingDuration: number - sentAt: number -}> - -export type RequestQueueSizeEvent = InstrumentationEvent<{ - broker: string - clientId: string - queueSize: number -}> - interface MessageSetEntry { key: Buffer | null value: Buffer | null @@ -382,8 +235,6 @@ export type Batch = { isEmpty(): boolean firstOffset(): string | null lastOffset(): string - offsetLag(): string - offsetLagLow(): string } export type KafkaMessage = MessageSetEntry | RecordBatchEntry @@ -396,13 +247,22 @@ export interface EachMessagePayload { pause(): () => void } +export interface PartitionOffset { + partition: number + offset: string +} + +export interface TopicOffsets { + topic: string + partitions: PartitionOffset[] +} + export interface EachBatchPayload { batch: Batch resolveOffset(offset: string): void heartbeat(): Promise pause(): () => void - commitOffsetsIfNecessary(offsets?: Offsets): Promise - uncommittedOffsets(): OffsetsByTopicPartition + commitOffsetsIfNecessary(): Promise isRunning(): boolean isStale(): boolean } @@ -411,30 +271,17 @@ export type EachBatchHandler = (payload: EachBatchPayload) => Promise export type EachMessageHandler = (payload: EachMessagePayload) => Promise -export type ConsumerSubscribeTopic = { topic: string | RegExp; fromBeginning?: boolean, replace?: boolean } +/** + * @deprecated Replaced by ConsumerSubscribeTopics + */ +export type ConsumerSubscribeTopic = { topic: string | RegExp; replace?: boolean } -export type ConsumerSubscribeTopics = { topics: (string | RegExp)[]; fromBeginning?: boolean, replace?: boolean } +export type ConsumerSubscribeTopics = { topics: (string | RegExp)[]; replace?: boolean } export type ConsumerRunConfig = { - autoCommit?: boolean - autoCommitInterval?: number | null - autoCommitThreshold?: number | null - partitionsConsumedConcurrently?: number + eachBatchAutoResolve?: boolean, eachMessage?: EachMessageHandler -} - -export interface Offsets { - topics: TopicOffsets[] -} - -export interface TopicOffsets { - topic: string - partitions: PartitionOffset[] -} - -export interface PartitionOffset { - partition: number - offset: string + eachBatch?: EachBatchHandler } export type TopicPartitions = { topic: string; partitions: number[] } @@ -455,112 +302,48 @@ export interface OffsetsByTopicPartition { topics: TopicOffsets[] } -export type Consumer = { - connect(): Promise - disconnect(): Promise +export type Consumer = Client & { subscribe(subscription: ConsumerSubscribeTopics | ConsumerSubscribeTopic): Promise stop(): Promise run(config?: ConsumerRunConfig): Promise - storeOffsets(topicPartitions: Array): void - commitOffsets(topicPartitions: Array): Promise + storeOffsets(topicPartitions: Array): void + commitOffsets(topicPartitions?: Array): Promise seek(topicPartitionOffset: TopicPartitionOffset): Promise - describeGroup(): Promise pause(topics: Array<{ topic: string; partitions?: number[] }>): void paused(): TopicPartitions[] - assignment(): TopicPartitions[] resume(topics: Array<{ topic: string; partitions?: number[] }>): void - on( - eventName: ConsumerEvents['HEARTBEAT'], - listener: (event: ConsumerHeartbeatEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['COMMIT_OFFSETS'], - listener: (event: ConsumerCommitOffsetsEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['GROUP_JOIN'], - listener: (event: ConsumerGroupJoinEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['FETCH_START'], - listener: (event: ConsumerFetchStartEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['FETCH'], - listener: (event: ConsumerFetchEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['START_BATCH_PROCESS'], - listener: (event: ConsumerStartBatchProcessEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['END_BATCH_PROCESS'], - listener: (event: ConsumerEndBatchProcessEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['CONNECT'], - listener: (event: ConnectEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['DISCONNECT'], - listener: (event: DisconnectEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['STOP'], - listener: (event: InstrumentationEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['CRASH'], - listener: (event: ConsumerCrashEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['REBALANCING'], - listener: (event: ConsumerRebalancingEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['RECEIVED_UNSUBSCRIBED_TOPICS'], - listener: (event: ConsumerReceivedUnsubcribedTopicsEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['REQUEST'], - listener: (event: RequestEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['REQUEST_TIMEOUT'], - listener: (event: RequestTimeoutEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ConsumerEvents['REQUEST_QUEUE_SIZE'], - listener: (event: RequestQueueSizeEvent) => void - ): RemoveInstrumentationEventListener - on( - eventName: ValueOf, - listener: (event: InstrumentationEvent) => void - ): RemoveInstrumentationEventListener - logger(): Logger - readonly events: ConsumerEvents + assignment(): TopicPartitions[] +} + +export interface AdminConfig { + retry?: RetryOptions +} + +export interface AdminConstructorConfig extends GlobalConfig { + kafkaJS?: AdminConfig; +} + +export interface ReplicaAssignment { + partition: number + replicas: Array +} + +export interface IResourceConfigEntry { + name: string + value: string } export interface ITopicConfig { topic: string numPartitions?: number replicationFactor?: number - replicaAssignment?: ReplicaAssignment[] configEntries?: IResourceConfigEntry[] } -export interface ITopicPartitionConfig { - topic: string - count: number - assignments?: Array> -} - export type Admin = { connect(): Promise disconnect(): Promise createTopics(options: { - validateOnly?: boolean - waitForLeaders?: boolean timeout?: number topics: ITopicConfig[] }): Promise @@ -572,6 +355,6 @@ export type Admin = { }): Promise<{ groups: GroupOverview[], errors: LibrdKafkaError[] }> describeGroups( groups: string[], - options?: {timeout?: number, includeAuthorizedOperations?: boolean }): Promise + options?: { timeout?: number, includeAuthorizedOperations?: boolean }): Promise deleteGroups(groupIds: string[], options?: { timeout?: number }): Promise } From 9cd609be1798c55e1b68d7b5195c75ef80701a0e Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 7 May 2024 09:32:21 +0530 Subject: [PATCH 134/224] Run integration tests on semaphore --- .semaphore/semaphore.yml | 6 +++++ docker-compose.yml | 47 ++++++++++++++++++++-------------------- 2 files changed, 30 insertions(+), 23 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index e80de063..31b8850e 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -97,6 +97,12 @@ blocks: - name: "Test" commands: - make test + - name: "Promisified Tests" + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' + - docker compose up -d && sleep 30 + - docker ps + - npx jest --forceExit --no-colors --runInBand --ci test/promisified/producer/ - name: "ESLint" commands: - npx eslint lib/kafkajs diff --git a/docker-compose.yml b/docker-compose.yml index abe29df2..911dfb16 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -1,23 +1,24 @@ ---- -zookeeper: - image: confluentinc/cp-zookeeper - ports: - - "2181:2181" - environment: - ZOOKEEPER_CLIENT_PORT: 2181 - ZOOKEEPER_TICK_TIME: 2000 -kafka: - image: confluentinc/cp-kafka - links: - - zookeeper - ports: - - "9092:9092" - environment: - KAFKA_BROKER_ID: 1 - KAFKA_ZOOKEEPER_CONNECT: 'zookeeper:2181' - KAFKA_ADVERTISED_LISTENERS: 'PLAINTEXT://localhost:9092' - KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 - KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 - KAFKA_DEFAULT_REPLICATION_FACTOR: 1 - KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 +version: '3' +services: + zookeeper: + image: confluentinc/cp-zookeeper + environment: + ZOOKEEPER_CLIENT_PORT: 2181 + kafka: + image: confluentinc/cp-kafka + restart: always + depends_on: + - zookeeper + ports: + - 9092:9092 + environment: + KAFKA_BROKER_ID: 0 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT_HOST:PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT_HOST://localhost:9092 + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT_HOST + KAFKA_SASL_ENABLED_MECHANISMS: PLAIN + KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 18000 + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 From 62a2bc5348de8fa0b21513c3ae3767dd727b7c9f Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 7 May 2024 17:59:30 +0530 Subject: [PATCH 135/224] Bump version to 0.1.13-devel --- CHANGELOG.md | 15 +++++++++++++++ .../package-lock.json | 2 +- .../package-lock.json | 3 ++- examples/package-lock.json | 2 +- examples/typescript/package-lock.json | 2 +- lib/util.js | 2 +- package-lock.json | 4 ++-- package.json | 4 ++-- 8 files changed, 25 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2ebafa9f..e2034c4f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,18 @@ +# confluent-kafka-javascript v0.1.13-devel + +v0.1.13-devel is a pre-production, early-access release. + +## Features + +1. Add support for `storeOffsets` in the consumer API. +2. Add optimization while consuming, in cases where the size of messages pending in our subscription is less than the consumer cache size. + +## Bug Fixes + +1. Fix memory leak in incremental assign (@martijnimhoff, #35). +2. Fix various issues with typings, and reconcile typings, JavaScript code, and MIGRATION.md to be consistent. + + # confluent-kafka-javascript v0.1.12-devel v0.1.12-devel is a pre-production, early-access release. diff --git a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json index 4fe0bdb2..884b1a7f 100644 --- a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json +++ b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json @@ -15,7 +15,7 @@ }, "../../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.12-devel", + "version": "v0.1.13-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json index 3bb7a621..2eaa0ebe 100644 --- a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json +++ b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json @@ -14,7 +14,8 @@ } }, "../../..": { - "version": "v0.1.12-devel", + "name": "@confluentinc/kafka-javascript", + "version": "v0.1.13-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/package-lock.json b/examples/package-lock.json index 31c75172..997e5510 100644 --- a/examples/package-lock.json +++ b/examples/package-lock.json @@ -14,7 +14,7 @@ }, "..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.12-devel", + "version": "v0.1.13-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/typescript/package-lock.json b/examples/typescript/package-lock.json index 99897150..c666072e 100644 --- a/examples/typescript/package-lock.json +++ b/examples/typescript/package-lock.json @@ -18,7 +18,7 @@ }, "../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.12-devel", + "version": "v0.1.13-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/lib/util.js b/lib/util.js index 37b99ca4..5f35a802 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.12-devel'; +util.bindingVersion = 'v0.1.13-devel'; diff --git a/package-lock.json b/package-lock.json index 6fd62a6a..1f4c6c10 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.12-devel", + "version": "v0.1.13-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.12-devel", + "version": "v0.1.13-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index 6c2d5d09..d42b5490 100644 --- a/package.json +++ b/package.json @@ -1,9 +1,9 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.12-devel", + "version": "v0.1.13-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", - "librdkafka_win": "2.3.0", + "librdkafka_win": "2.4.0", "main": "lib/index.js", "types": "types/index.d.ts", "scripts": { From a036bc395fbb619d1295a4094acb8891dee04c99 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 8 May 2024 10:32:16 +0530 Subject: [PATCH 136/224] Increase available memory for tests --- .semaphore/semaphore.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 31b8850e..de032782 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -101,8 +101,8 @@ blocks: commands: - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' - docker compose up -d && sleep 30 - - docker ps - - npx jest --forceExit --no-colors --runInBand --ci test/promisified/producer/ + - export NODE_OPTIONS='--max-old-space-size=1536' + - npx jest --forceExit --no-colors --ci test/promisified/ - name: "ESLint" commands: - npx eslint lib/kafkajs From 8fe910798273d8fee75596820c3ea9af6bedb79b Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 8 May 2024 11:48:07 +0530 Subject: [PATCH 137/224] Increase available time for tests --- .semaphore/semaphore.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index de032782..48239bd9 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -15,7 +15,7 @@ auto_cancel: when: "branch != 'master'" execution_time_limit: - hours: 1 + hours: 3 queue: - when: "branch != 'master'" @@ -102,7 +102,7 @@ blocks: - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' - docker compose up -d && sleep 30 - export NODE_OPTIONS='--max-old-space-size=1536' - - npx jest --forceExit --no-colors --ci test/promisified/ + - npx jest --forceExit --no-colors --ci test/promisified/admin/delete_groups.spec.js test/promisified/consumer/pause.spec.js - name: "ESLint" commands: - npx eslint lib/kafkajs From 178d395f43450e2f2d4baeda5fd59780e0e1c937 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 10 May 2024 16:05:05 +0530 Subject: [PATCH 138/224] Add metadata to offset commits --- e2e/consumer.spec.js | 66 +++++++++++++++++++++++++----------- e2e/topicUtils.js | 81 ++++++++++++++++++++++++++++++++++++++++++++ src/common.cc | 27 ++++++++++++++- 3 files changed, 153 insertions(+), 21 deletions(-) create mode 100644 e2e/topicUtils.js diff --git a/e2e/consumer.spec.js b/e2e/consumer.spec.js index 15044cea..c408ad45 100644 --- a/e2e/consumer.spec.js +++ b/e2e/consumer.spec.js @@ -10,17 +10,20 @@ var t = require('assert'); var crypto = require('crypto'); var eventListener = require('./listener'); +const { createTopics, deleteTopics } = require('./topicUtils'); var KafkaConsumer = require('../').KafkaConsumer; var kafkaBrokerList = process.env.KAFKA_HOST || 'localhost:9092'; -var topic = 'test'; describe('Consumer', function() { var gcfg; + let topic; + let createdTopics = []; - beforeEach(function() { + beforeEach(function(done) { var grp = 'kafka-mocha-grp-' + crypto.randomBytes(20).toString('hex'); + topic = 'test' + crypto.randomBytes(20).toString('hex'); gcfg = { 'bootstrap.servers': kafkaBrokerList, 'group.id': grp, @@ -28,6 +31,12 @@ describe('Consumer', function() { 'rebalance_cb': true, 'enable.auto.commit': false }; + createTopics([{topic, num_partitions: 1, replication_factor: 1}], kafkaBrokerList, done); + createdTopics.push(topic); + }); + + after(function(done) { + deleteTopics(createdTopics, kafkaBrokerList, done); }); describe('commit', function() { @@ -94,34 +103,50 @@ describe('Consumer', function() { t.equal(position.length, 0); }); - it('after assign, should get committed array without offsets ', function(done) { - consumer.assign([{topic:topic, partition:0}]); - // Defer this for a second - setTimeout(function() { - consumer.committed(null, 1000, function(err, committed) { - t.ifError(err); - t.equal(committed.length, 1); - t.equal(typeof committed[0], 'object', 'TopicPartition should be an object'); - t.deepStrictEqual(committed[0].partition, 0); - t.equal(committed[0].offset, undefined); - done(); - }); + it('after assign, should get committed array without offsets ', function (done) { + consumer.assign([{ topic: topic, partition: 0 }]); + consumer.committed(null, 1000, function (err, committed) { + t.ifError(err); + t.equal(committed.length, 1); + t.equal(typeof committed[0], 'object', 'TopicPartition should be an object'); + t.deepStrictEqual(committed[0].partition, 0); + t.equal(committed[0].offset, undefined); + done(); }, 1000); }); - it('after assign and commit, should get committed offsets', function(done) { + it('after assign and commit, should get committed offsets with same metadata', function(done) { consumer.assign([{topic:topic, partition:0}]); - consumer.commitSync({topic:topic, partition:0, offset:1000}); + consumer.commitSync({topic:topic, partition:0, offset:1000, metadata: 'A string with unicode ǂ'}); consumer.committed(null, 1000, function(err, committed) { t.ifError(err); t.equal(committed.length, 1); t.equal(typeof committed[0], 'object', 'TopicPartition should be an object'); t.deepStrictEqual(committed[0].partition, 0); t.deepStrictEqual(committed[0].offset, 1000); + t.deepStrictEqual(committed[0].metadata, 'A string with unicode ǂ'); done(); }); }); + it('after assign and commit, a different consumer should get the same committed offsets and metadata', function(done) { + consumer.assign([{topic:topic, partition:0}]); + consumer.commitSync({topic:topic, partition:0, offset:1000, metadata: 'A string with unicode ǂ'}); + + let consumer2 = new KafkaConsumer(gcfg, {}); + consumer2.connect({ timeout: 2000 }, function (err, info) { + consumer2.committed([{ topic, partition: 0 }], 1000, function (err, committed) { + t.ifError(err); + t.equal(committed.length, 1); + t.equal(typeof committed[0], 'object', 'TopicPartition should be an object'); + t.deepStrictEqual(committed[0].partition, 0); + t.deepStrictEqual(committed[0].offset, 1000); + t.deepStrictEqual(committed[0].metadata, 'A string with unicode ǂ'); + consumer2.disconnect(done); + }); + }); + }); + it('after assign, before consume, position should return an array without offsets', function(done) { consumer.assign([{topic:topic, partition:0}]); var position = consumer.position(); @@ -154,7 +179,7 @@ describe('Consumer', function() { consumer.connect({ timeout: 2000 }, function(err, info) { t.ifError(err); consumer.assign([{ - topic: 'test', + topic, partition: 0, offset: 0 }]); @@ -172,7 +197,7 @@ describe('Consumer', function() { it('should be able to seek', function(cb) { consumer.seek({ - topic: 'test', + topic, partition: 0, offset: 0 }, 1, function(err) { @@ -183,7 +208,7 @@ describe('Consumer', function() { it('should be able to seek with a timeout of 0', function(cb) { consumer.seek({ - topic: 'test', + topic, partition: 0, offset: 0 }, 0, function(err) { @@ -217,7 +242,7 @@ describe('Consumer', function() { t.equal(0, consumer.subscription().length); consumer.subscribe([topic]); t.equal(1, consumer.subscription().length); - t.equal('test', consumer.subscription()[0]); + t.equal(topic, consumer.subscription()[0]); t.equal(0, consumer.assignments().length); }); @@ -308,6 +333,7 @@ describe('Consumer', function() { consumer.subscribe([topic]); + consumer.setDefaultConsumeTimeout(500); // Topic might not have any messages. consumer.consume(1, function(err, messages) { t.ifError(err); diff --git a/e2e/topicUtils.js b/e2e/topicUtils.js new file mode 100644 index 00000000..dab62568 --- /dev/null +++ b/e2e/topicUtils.js @@ -0,0 +1,81 @@ +/* + * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library + * Copyright (c) 2024 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +module.exports = { createTopics, deleteTopics }; + +var Kafka = require('../'); + +// Create topics and wait for them to be created in the metadata. +function createTopics(topics, brokerList, cb) { + const client = Kafka.AdminClient.create({ + 'client.id': 'kafka-test-admin-client', + 'metadata.broker.list': brokerList, + }); + let promises = []; + for (const topic of topics) { + client.createTopic(topic, (err) => { + promises.push(new Promise((resolve, reject) => { + if (err && err.code !== Kafka.CODES.ERR_TOPIC_ALREADY_EXISTS) { + reject(err); + } + resolve(); + })); + }); + } + + Promise.all(promises).then(() => { + let interval = setInterval(() => { + client.listTopics((err, topicList) => { + if (err) { + client.disconnect(); + clearInterval(interval); + cb(err); + return; + } + for (const topic of topics) { + if (!topicList.includes(topic.topic)) { + return; + } + } + client.disconnect(); + clearInterval(interval); + cb(); + }); + }, 100); + }).catch((err) => { + client.disconnect(); + cb(err); + }); +} + +// Delete topics. +function deleteTopics(topics, brokerList, cb) { + const client = Kafka.AdminClient.create({ + 'client.id': 'kafka-test-admin-client', + 'metadata.broker.list': brokerList, + }); + let promises = []; + for (const topic of topics) { + client.deleteTopic(topic, (err) => { + promises.push(new Promise((resolve, reject) => { + if (err && err.code !== Kafka.CODES.ERR_UNKNOWN_TOPIC_OR_PART) { + reject(err); + } + resolve(); + })); + }); + } + + Promise.all(promises).then(() => { + client.disconnect(); + cb(); + }).catch((err) => { + client.disconnect(); + cb(err); + }); +} diff --git a/src/common.cc b/src/common.cc index fe0b3894..6a436f8d 100644 --- a/src/common.cc +++ b/src/common.cc @@ -325,6 +325,16 @@ v8::Local ToV8Array( Nan::Set(obj, Nan::New("offset").ToLocalChecked(), Nan::New(topic_partition->offset())); } + + // If present, size >= 1, since it will include at least the null terminator. + if (topic_partition->get_metadata().size() > 0) { + Nan::Set(obj, Nan::New("metadata").ToLocalChecked(), + Nan::New( + reinterpret_cast(topic_partition->get_metadata().data()), + topic_partition->get_metadata().size() - 1) // null terminator is not required by the constructor. + .ToLocalChecked()); + } + Nan::Set(obj, Nan::New("partition").ToLocalChecked(), Nan::New(topic_partition->partition())); Nan::Set(obj, Nan::New("topic").ToLocalChecked(), @@ -428,7 +438,22 @@ RdKafka::TopicPartition * FromV8Object(v8::Local topic_partition) { return NULL; } - return RdKafka::TopicPartition::create(topic, partition, offset); + RdKafka::TopicPartition *toppar = RdKafka::TopicPartition::create(topic, partition, offset); + + v8::Local metadataKey = Nan::New("metadata").ToLocalChecked(); + if (Nan::Has(topic_partition, metadataKey).FromMaybe(false)) { + v8::Local metadataValue = Nan::Get(topic_partition, metadataKey).ToLocalChecked(); + + if (metadataValue->IsString()) { + Nan::Utf8String metadataValueUtf8Str(metadataValue.As()); + std::string metadataValueStr(*metadataValueUtf8Str); + std::vector metadataVector(metadataValueStr.begin(), metadataValueStr.end()); + metadataVector.push_back('\0'); // The null terminator is not included in the iterator. + toppar->set_metadata(metadataVector); + } + } + + return toppar; } } // namespace TopicPartition From bda98c6ddce4e59fcffe4bad1fe7512caaee4b7a Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 13 May 2024 10:01:17 +0530 Subject: [PATCH 139/224] Add types for logger and loglevel to configs --- types/kafkajs.d.ts | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index e182c4b6..04669cab 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -72,6 +72,7 @@ export interface KafkaConfig { enforceRequestTimeout?: boolean, retry?: RetryOptions, logLevel?: logLevel, + logger?: Logger, } export interface CommonConstructorConfig extends GlobalConfig { @@ -111,6 +112,8 @@ export interface ProducerConfig { compression?: CompressionTypes timeout?: number, retry?: RetryOptions, + logLevel?: logLevel, + logger?: Logger, } export interface ProducerConstructorConfig extends ProducerGlobalConfig { @@ -192,6 +195,8 @@ export interface ConsumerConfig { maxBytes?: number maxWaitTimeInMs?: number retry?: RetryOptions, + logLevel?: logLevel, + logger?: Logger, allowAutoTopicCreation?: boolean maxInFlightRequests?: number readUncommitted?: boolean @@ -317,6 +322,8 @@ export type Consumer = Client & { export interface AdminConfig { retry?: RetryOptions + logLevel?: logLevel, + logger?: Logger, } export interface AdminConstructorConfig extends GlobalConfig { From 357cdb3250f5ee5a3cbc6bccc76687d65d062b6f Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 15 May 2024 14:44:27 +0530 Subject: [PATCH 140/224] Add Producer poll from librdkafka background thread --- MIGRATION.md | 16 +++++++++++++ lib/kafkajs/_producer.js | 32 +------------------------ lib/producer.js | 17 ++++++++++++++ src/producer.cc | 51 +++++++++++++++++++++++++++++++++++++++- src/producer.h | 3 +++ types/rdkafka.d.ts | 1 + 6 files changed, 88 insertions(+), 32 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index b82f19cc..089f39ed 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -181,6 +181,22 @@ producerRun().then(consumerRun).catch(console.error); messages: [ /* ... */ ], }); ``` + - It's recommended to send a number of messages without awaiting them, and then calling `flush` to ensure all messages are sent, rather than awaiting each message. This is more efficient. + Example: + ```javascript + const kafka = new Kafka({ kafkaJS: { /* ... */ }}); + const producer = kafka.producer(); + await producer.connect(); + for (/*...*/) producer.send({ /* ... */}); + await producer.flush({timeout: 5000}); + ``` + + However, in case it is desired to await every message, `linger.ms` should be set to 0, to ensure that the default batching behaviour does not cause a delay in awaiting messages. + Example: + ```javascript + const kafka = new Kafka({ kafkaJS: { /* ... */ }}); + const producer = kafka.producer({ 'linger.ms': 0 }); + ``` * A transactional producer (with a `transactionId`) set, **cannot** send messages without initiating a transaction using `producer.transaction()`. diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 163fdc1b..76860b81 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -13,7 +13,6 @@ const { kafkaJSToRdKafkaConfig, const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); -const { hrtime } = require('process'); const ProducerState = Object.freeze({ INIT: 0, @@ -33,8 +32,6 @@ const CompressionTypes = Object.freeze({ ZSTD: 'zstd', }); -const producerPollIntervalMs = 500; - class Producer { /** * The config supplied by the user. @@ -78,14 +75,6 @@ class Producer { */ #logger = new DefaultLogger(); - /** - * Stores the time of the last poll. - * In case we are producing in a tight loop, the interval timer will not - * fire, and we won't poll. By maintaining the last poll time, we can - * poll at the end of send() and sendBatch(). - */ - #lastPollTime = hrtime(); - /** * @constructor * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig @@ -277,16 +266,7 @@ class Producer { this.#state = ProducerState.CONNECTED; - /* Start a loop to poll. the queues. */ - const pollInterval = setInterval(() => { - if (this.#state >= ProducerState.DISCONNECTING) { - clearInterval(pollInterval); - return; - } - this.#internalClient.poll(); - this.#lastPollTime = hrtime(); - }, producerPollIntervalMs); - + this.#internalClient.setPollInBackground(true); this.#internalClient.on('delivery-report', this.#deliveryCallback.bind(this)); // Resolve the promise. @@ -564,16 +544,6 @@ class Producer { reject(err); } })); - - /* Poll if we haven't polled in a while. This can be the case if we're producing - * in a tight loop without awaiting the produce. */ - const elapsed = hrtime(this.#lastPollTime); - const elapsedInNanos = elapsed[0] * 1e9 + elapsed[1]; - if (elapsedInNanos > producerPollIntervalMs * 1000) { - this.#lastPollTime = hrtime(); - this.#internalClient.poll(); - } - } /* The delivery report will be handled by the delivery-report event handler, and we can simply wait for it here. */ diff --git a/lib/producer.js b/lib/producer.js index 5993a23d..99b892ae 100644 --- a/lib/producer.js +++ b/lib/producer.js @@ -230,6 +230,23 @@ Producer.prototype.setPollInterval = function(interval) { return this; }; +/** + * Set automatic polling for events on the librdkafka background thread. + * + * This provides several advantages over `setPollInterval`, as the polling + * does not happen on the event loop, but on the C thread spawned by librdkafka, + * and can be more efficient for high-throughput producers. + * + * @param {boolean} set Whether to poll in the background or not. + * @note If set = true, this will disable any polling interval set by `setPollInterval`. + */ +Producer.prototype.setPollInBackground = function(set) { + if (set) { + this.setPollInterval(0); // Clear poll interval from JS. + } + this._client.setPollInBackground(set); +} + /** * Flush the producer * diff --git a/src/producer.cc b/src/producer.cc index 191d9592..79add38f 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -33,7 +33,8 @@ namespace NodeKafka { Producer::Producer(Conf* gconfig, Conf* tconfig): Connection(gconfig, tconfig), m_dr_cb(), - m_partitioner_cb() { + m_partitioner_cb(), + m_is_background_polling(false) { std::string errstr; if (m_tconfig) @@ -72,6 +73,7 @@ void Producer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "getMetadata", NodeGetMetadata); Nan::SetPrototypeMethod(tpl, "queryWatermarkOffsets", NodeQueryWatermarkOffsets); // NOLINT Nan::SetPrototypeMethod(tpl, "poll", NodePoll); + Nan::SetPrototypeMethod(tpl, "setPollInBackground", NodeSetPollInBackground); Nan::SetPrototypeMethod(tpl, "setSaslCredentials", NodeSetSaslCredentials); Nan::SetPrototypeMethod(tpl, "setOAuthBearerToken", NodeSetOAuthBearerToken); Nan::SetPrototypeMethod(tpl, "setOAuthBearerTokenFailure", @@ -330,9 +332,39 @@ Baton Producer::Produce(void* message, size_t size, std::string topic, } void Producer::Poll() { + // We're not allowed to call poll when we have forwarded the main + // queue to the background queue, as that would indirectly poll + // the background queue. However, that's not allowed by librdkafka. + if (m_is_background_polling) { + return; + } m_client->poll(0); } +Baton Producer::SetPollInBackground(bool set) { + scoped_shared_read_lock lock(m_connection_lock); + rd_kafka_t* rk = this->m_client->c_ptr(); + if (!IsConnected()) { + return Baton(RdKafka::ERR__STATE, "Producer is disconnected"); + } + + if (set && !m_is_background_polling) { + m_is_background_polling = true; + rd_kafka_queue_t* main_q = rd_kafka_queue_get_main(rk); + rd_kafka_queue_t* background_q = rd_kafka_queue_get_background(rk); + rd_kafka_queue_forward(main_q, background_q); + rd_kafka_queue_destroy(main_q); + rd_kafka_queue_destroy(background_q); + } else if (!set && m_is_background_polling) { + m_is_background_polling = false; + rd_kafka_queue_t* main_q = rd_kafka_queue_get_main(rk); + rd_kafka_queue_forward(main_q, NULL); + rd_kafka_queue_destroy(main_q); + } + + return Baton(RdKafka::ERR_NO_ERROR); +} + void Producer::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add) { if (string_key.compare("delivery_cb") == 0) { if (add) { @@ -686,6 +718,23 @@ NAN_METHOD(Producer::NodePoll) { } } +NAN_METHOD(Producer::NodeSetPollInBackground) { + Nan::HandleScope scope; + if (info.Length() < 1 || !info[0]->IsBoolean()) { + // Just throw an exception + return Nan::ThrowError( + "Need to specify a boolean for setting or unsetting"); + } + bool set = Nan::To(info[0]).FromJust(); + + Producer* producer = ObjectWrap::Unwrap(info.This()); + Baton b = producer->SetPollInBackground(set); + if (b.err() != RdKafka::ERR_NO_ERROR) { + return Nan::ThrowError(b.errstr().c_str()); + } + info.GetReturnValue().Set(b.ToObject()); +} + Baton Producer::Flush(int timeout_ms) { RdKafka::ErrorCode response_code; if (IsConnected()) { diff --git a/src/producer.h b/src/producer.h index 1e7854e1..974afe85 100644 --- a/src/producer.h +++ b/src/producer.h @@ -54,6 +54,7 @@ class Producer : public Connection { Baton Connect(); void Disconnect(); void Poll(); + Baton SetPollInBackground(bool); #if RD_KAFKA_VERSION > 0x00090200 Baton Flush(int timeout_ms); #endif @@ -103,6 +104,7 @@ class Producer : public Connection { static NAN_METHOD(NodeConnect); static NAN_METHOD(NodeDisconnect); static NAN_METHOD(NodePoll); + static NAN_METHOD(NodeSetPollInBackground); #if RD_KAFKA_VERSION > 0x00090200 static NAN_METHOD(NodeFlush); #endif @@ -114,6 +116,7 @@ class Producer : public Connection { Callbacks::Delivery m_dr_cb; Callbacks::Partitioner m_partitioner_cb; + bool m_is_background_polling; }; } // namespace NodeKafka diff --git a/types/rdkafka.d.ts b/types/rdkafka.d.ts index 4c9150e9..76d7e23c 100644 --- a/types/rdkafka.d.ts +++ b/types/rdkafka.d.ts @@ -268,6 +268,7 @@ export class Producer extends Client { produce(topic: string, partition: NumberNullUndefined, message: MessageValue, key?: MessageKey, timestamp?: NumberNullUndefined, opaque?: any, headers?: MessageHeader[]): any; setPollInterval(interval: number): this; + setPollInBackground(set: boolean): void; static createWriteStream(conf: ProducerGlobalConfig, topicConf: ProducerTopicConfig, streamOptions: WriteStreamOptions): ProducerStream; From 3f9f7faffab0a26672996bd7fd0178b93d6f9e1f Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 16 May 2024 16:51:19 +0530 Subject: [PATCH 141/224] Enable optimization for less-than-cache-size messages in consumer --- lib/kafkajs/_consumer.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 2010d27f..435d5905 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -574,7 +574,7 @@ class Consumer { this.#state = ConsumerState.CONNECTED; /* Slight optimization for cases where the size of messages in our subscription is less than the cache size. */ - this.#internalClient.setDefaultIsTimeoutOnlyForFirstMessage(false); + this.#internalClient.setDefaultIsTimeoutOnlyForFirstMessage(true); // Resolve the promise. this.#connectPromiseFunc['resolve'](); From 35e893cfdb63efc6133a0033da535a778ac3e796 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 17 May 2024 15:06:10 +0530 Subject: [PATCH 142/224] Fix issues with header conversions --- e2e/both.spec.js | 22 +++++++++- examples/consumer.js | 1 + examples/producer.js | 15 ++++++- examples/typescript/kafkajs.ts | 10 ++++- lib/kafkajs/_common.js | 10 +++-- lib/kafkajs/_consumer.js | 16 ++++---- src/producer.cc | 35 ++++++++++++---- .../consumer/consumeMessages.spec.js | 40 +++++++++++++++++++ 8 files changed, 126 insertions(+), 23 deletions(-) diff --git a/e2e/both.spec.js b/e2e/both.spec.js index 5b376dec..62983179 100644 --- a/e2e/both.spec.js +++ b/e2e/both.spec.js @@ -424,12 +424,30 @@ describe('Consumer/Producer', function() { var headers = [ { key1: 'value1' }, { key2: Buffer.from('value2') }, - { key3: 100 }, - { key4: 10.1 }, ]; run_headers_test(done, headers); }); + it('should not be able to produce any non-string and non-buffer headers: consumeLoop', function(done) { + producer.setPollInterval(10); + + const headerCases = [ + [ { key: 10 } ], + [ { key: null }], + [ { key: undefined }], + ]; + for (const headerCase of headerCases) { + const buffer = Buffer.from('value'); + const key = 'key'; + t.throws( + () => producer.produce(topic, null, buffer, key, null, "", headerCase), + 'must be string or buffer' + ); + } + + done(); + }); + it('should be able to produce and consume messages: empty buffer key and empty value', function(done) { var emptyString = ''; var key = Buffer.from(emptyString); diff --git a/examples/consumer.js b/examples/consumer.js index 188442ea..f6437d3d 100644 --- a/examples/consumer.js +++ b/examples/consumer.js @@ -75,6 +75,7 @@ async function consumerStart() { console.log({ topic, partition, + headers: message.headers, offset: message.offset, key: message.key?.toString(), value: message.value.toString(), diff --git a/examples/producer.js b/examples/producer.js index db3c7ced..440cbcb8 100644 --- a/examples/producer.js +++ b/examples/producer.js @@ -14,8 +14,19 @@ async function producerStart() { res.push(producer.send({ topic: 'test-topic', messages: [ - { value: 'v222', partition: 1 }, - { value: 'v11', partition: 0, key: 'x' }, + { + value: 'v1', + partition: 0, + key: 'x', + headers: { + 'header1': ['h1v1', 'h1v2'], + 'header3': 'h3v3', + } + }, + { + value: 'v2', + key: 'y', + } ] })); } diff --git a/examples/typescript/kafkajs.ts b/examples/typescript/kafkajs.ts index 3b158b35..faf11afb 100644 --- a/examples/typescript/kafkajs.ts +++ b/examples/typescript/kafkajs.ts @@ -22,7 +22,14 @@ async function runProducer() { await producer.send({ topic: 'test-topic', messages: [ - { value: 'Hello World!', key: 'key1' }, + { + value: 'Hello World!', + key: 'key1', + headers: { + 'header1': 'value1', + 'header2': [Buffer.from('value2'), 'value3'] + } + }, ], }); @@ -52,6 +59,7 @@ async function runConsumer() { console.log({ key: message.key ? message.key.toString() : null, value: message.value ? message.value.toString() : null, + headers: message.headers, }); }, }); diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 11182b01..c24de9ce 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -559,12 +559,16 @@ function convertToRdKafkaHeaders(kafkaJSHeaders) { const headers = []; for (const [key, value] of Object.entries(kafkaJSHeaders)) { - if (value.constructor === Array) { + if (value && value.constructor === Array) { for (const v of value) { - headers.push({ key, value: v }); + const header = {}; + header[key] = v; + headers.push(header); } } else { - headers.push({ key, value }); + const header = {}; + header[key] = value; + headers.push(header); } } return headers; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 435d5905..eb4a9897 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -608,13 +608,15 @@ class Consumer { let headers; if (message.headers) { headers = {} - for (const [key, value] of Object.entries(message.headers)) { - if (!Object.hasOwn(headers, key)) { - headers[key] = value; - } else if (headers[key].constructor === Array) { - headers[key].push(value); - } else { - headers[key] = [headers[key], value]; + for (const header of message.headers) { + for (const [key, value] of Object.entries(header)) { + if (!Object.hasOwn(headers, key)) { + headers[key] = value; + } else if (headers[key].constructor === Array) { + headers[key].push(value); + } else { + headers[key] = [headers[key], value]; + } } } } diff --git a/src/producer.cc b/src/producer.cc index 79add38f..63b02a57 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -595,18 +595,37 @@ NAN_METHOD(Producer::NodeProduce) { v8::Local props = header->GetOwnPropertyNames( Nan::GetCurrentContext()).ToLocalChecked(); - Nan::MaybeLocal v8Key = Nan::To( - Nan::Get(props, 0).ToLocalChecked()); - Nan::MaybeLocal v8Value = Nan::To( - Nan::Get(header, v8Key.ToLocalChecked()).ToLocalChecked()); + // TODO: Other properties in the list of properties should not be + // ignored, but they are. This is a bug, need to handle it either in JS + // or here. + Nan::MaybeLocal v8Key = + Nan::To(Nan::Get(props, 0).ToLocalChecked()); + + // The key must be a string. + if (v8Key.IsEmpty()) { + Nan::ThrowError("Header key must be a string"); + } Nan::Utf8String uKey(v8Key.ToLocalChecked()); std::string key(*uKey); - Nan::Utf8String uValue(v8Value.ToLocalChecked()); - std::string value(*uValue); - headers.push_back( - RdKafka::Headers::Header(key, value.c_str(), value.size())); + // Valid types for the header are string or buffer. + // Other types will throw an error. + v8::Local v8Value = + Nan::Get(header, v8Key.ToLocalChecked()).ToLocalChecked(); + + if (node::Buffer::HasInstance(v8Value)) { + const char* value = node::Buffer::Data(v8Value); + const size_t value_len = node::Buffer::Length(v8Value); + headers.push_back(RdKafka::Headers::Header(key, value, value_len)); + } else if (v8Value->IsString()) { + Nan::Utf8String uValue(v8Value); + std::string value(*uValue); + headers.push_back( + RdKafka::Headers::Header(key, value.c_str(), value.size())); + } else { + Nan::ThrowError("Header value must be a string or buffer"); + } } } } diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 9228bb49..ede7bbe6 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -85,6 +85,46 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) }); + it('consume messages with headers', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + + const messages = [ { + value: `value-${secureRandom}`, + headers: { + 'header-1': 'value-1', + 'header-2': 'value-2', + 'header-3': [ 'value-3-1', 'value-3-2', Buffer.from([1,0,1,0,1]) ], + 'header-4': Buffer.from([1,0,1,0,1]), + } + } ] + + 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({ + value: Buffer.from(messages[0].value), + offset: '0', + headers: { + // Headers are always returned as Buffers from the broker. + 'header-1': Buffer.from('value-1'), + 'header-2': Buffer.from('value-2'), + 'header-3': [ Buffer.from('value-3-1'), Buffer.from('value-3-2'), Buffer.from([1,0,1,0,1]) ], + 'header-4': Buffer.from([1,0,1,0,1]), + } + }), + }) + ) + }); + it.each([[true], [false]])('consumes messages using eachBatch', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); From 9612dff8b7028a90dcef67c99afd734abcffb6bf Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 17 May 2024 15:58:38 +0530 Subject: [PATCH 143/224] Bump version to 0.1.14-devel --- CHANGELOG.md | 17 +++++++++++++++++ .../package-lock.json | 2 +- .../package-lock.json | 2 +- examples/package-lock.json | 2 +- examples/typescript/package-lock.json | 2 +- lib/util.js | 2 +- package-lock.json | 4 ++-- package.json | 2 +- 8 files changed, 25 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e2034c4f..fd0bf1de 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,20 @@ +# confluent-kafka-javascript v0.1.14-devel + +v0.1.14-devel is a pre-production, early-access release. + +## Features + +1. Add metadata to offset commit and offset store (non-promisified API). +2. Add types for logger and loglevel to configuration. +3. Add Producer polling from background thread. This improves performance for cases when send is awaited on. +4. Enable consume optimization from v0.1.13-devel (Features #2) by default for the promisified API. + +## Bug Fixes + +1. Fix issues with the header conversions from promisified API to the non-promisified API to match + the type signature and allow Buffers to be passed as header values in the C++ layer. + + # confluent-kafka-javascript v0.1.13-devel v0.1.13-devel is a pre-production, early-access release. diff --git a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json index 884b1a7f..c87e88b0 100644 --- a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json +++ b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json @@ -15,7 +15,7 @@ }, "../../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json index 2eaa0ebe..83278c82 100644 --- a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json +++ b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json @@ -15,7 +15,7 @@ }, "../../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/package-lock.json b/examples/package-lock.json index 997e5510..aa944dc2 100644 --- a/examples/package-lock.json +++ b/examples/package-lock.json @@ -14,7 +14,7 @@ }, "..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/examples/typescript/package-lock.json b/examples/typescript/package-lock.json index c666072e..a90bd306 100644 --- a/examples/typescript/package-lock.json +++ b/examples/typescript/package-lock.json @@ -18,7 +18,7 @@ }, "../..": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/lib/util.js b/lib/util.js index 5f35a802..52023cdf 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.13-devel'; +util.bindingVersion = 'v0.1.14-devel'; diff --git a/package-lock.json b/package-lock.json index 1f4c6c10..525fd4a1 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index d42b5490..68490cba 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.13-devel", + "version": "v0.1.14-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.4.0", From 618d6bf9cad546015ced0044440ec416d652965e Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 17 May 2024 18:11:56 +0530 Subject: [PATCH 144/224] Add latest node version and update librdkafka on each build --- .semaphore/semaphore.yml | 59 ++++++++++++++++++++++++++++++++++++++++ README.md | 2 +- deps/librdkafka | 2 +- 3 files changed, 61 insertions(+), 2 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 48239bd9..fa00dccf 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -28,6 +28,7 @@ global_job_config: - git submodule update --init --recursive - cd deps/librdkafka - git fetch origin + - git pull origin master - cd ../../ - cache clear @@ -153,6 +154,15 @@ blocks: - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 22" + commands: + - sem-version node 22.2.0 + - export NODE_ABI=127 + - export ARTIFACT_KEY="confluent-kafka-javascript-${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 workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Linux arm64: Release" dependencies: [ ] @@ -200,6 +210,15 @@ blocks: - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 22" + commands: + - sem-version node 22.2.0 + - export NODE_ABI=127 + - export ARTIFACT_KEY="confluent-kafka-javascript-${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 workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Linux amd64 musl: Release" dependencies: [ ] @@ -241,6 +260,15 @@ blocks: - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 22" + commands: + - sem-version node 22.2.0 + - export NODE_ABI=127 + - export ARTIFACT_KEY="confluent-kafka-javascript-${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 workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Linux arm64 musl: Release" dependencies: [ ] @@ -282,6 +310,15 @@ blocks: - docker run -v "$(pwd):/v" node:21-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 22" + commands: + - sem-version node 22.2.0 + - export NODE_ABI=127 + - export ARTIFACT_KEY="confluent-kafka-javascript-${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 workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "macOS arm64/m1: Release" dependencies: [ ] @@ -326,6 +363,15 @@ blocks: - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 22" + commands: + - sem-version node 22.2.0 + - export NODE_ABI=127 + - export ARTIFACT_KEY="confluent-kafka-javascript-${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 workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Windows x64: Release" dependencies: [ ] @@ -402,6 +448,19 @@ blocks: - npx node-pre-gyp package - ls "build/stage/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" - artifact push workflow "build/stage/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" --destination "releases/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - name: "Release: latest: 22" + commands: + - Invoke-WebRequest "https://nodejs.org/dist/v22.2.0/node-v22.2.0-x64.msi" -OutFile node.msi + - msiexec /qn /l* node-log.txt /i node.msi + - node --version + - pip install setuptools + - $env:NODE_ABI = 127 + - $env:ARTIFACT_KEY = "confluent-kafka-javascript-${env:SEMAPHORE_GIT_TAG_NAME}-node-v${env:NODE_ABI}-${env:PLATFORM}-${env:LIBC}-${env:ARCHITECTURE}.tar.gz" + - echo "$env:ARTIFACT_KEY" + - 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/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" + - artifact push workflow "build/stage/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" --destination "releases/${env:SEMAPHORE_GIT_TAG_NAME}/${env:ARTIFACT_KEY}" - name: 'Packaging: tar all release artifacts' dependencies: diff --git a/README.md b/README.md index 7c5c6091..8565e0ff 100644 --- a/README.md +++ b/README.md @@ -27,7 +27,7 @@ To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry] The following configurations are supported for this early access preview: -* Any supported version of Node.js (The two LTS versions, 18 and 20, and the latest version, 21). +* Any supported version of Node.js (The two LTS versions, 18 and 20, and the latest versions, 21 and 22). * Linux (x64 and arm64) - both glibc and musl/alpine. * macOS - arm64/m1. * Windows - x64 (experimentally available in EA). diff --git a/deps/librdkafka b/deps/librdkafka index e03d3bb9..2587cac7 160000 --- a/deps/librdkafka +++ b/deps/librdkafka @@ -1 +1 @@ -Subproject commit e03d3bb91ed92a38f38d9806b8d8deffe78a1de5 +Subproject commit 2587cac70f83fced42c51f921bed325a434f5bc7 From 7bf9da68404553926f51d7b892c5e9354acebdb9 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 17 May 2024 18:50:41 +0530 Subject: [PATCH 145/224] Bump version to 0.1.15-devel --- .gitignore | 2 + .semaphore/semaphore.yml | 12 +- CHANGELOG.md | 9 + .../package-lock.json | 188 ------------------ .../package-lock.json | 188 ------------------ examples/package-lock.json | 45 ----- examples/typescript/package-lock.json | 76 ------- lib/util.js | 2 +- package-lock.json | 4 +- package.json | 2 +- 10 files changed, 19 insertions(+), 509 deletions(-) delete mode 100644 examples/kafkajs/oauthbearer_calback_authentication/package-lock.json delete mode 100644 examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json delete mode 100644 examples/package-lock.json delete mode 100644 examples/typescript/package-lock.json diff --git a/.gitignore b/.gitignore index 1b6d3a6a..8ac250f5 100644 --- a/.gitignore +++ b/.gitignore @@ -5,6 +5,8 @@ npm-debug.log docs +examples/**/package-lock.json + deps/* !deps/*.gyp !deps/windows-install.* diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index fa00dccf..e7bb13ad 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -262,11 +262,9 @@ blocks: - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: latest: 22" commands: - - sem-version node 22.2.0 - export NODE_ABI=127 - - export ARTIFACT_KEY="confluent-kafka-javascript-${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 + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:22-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" @@ -312,11 +310,9 @@ blocks: - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - name: "Release: latest: 22" commands: - - sem-version node 22.2.0 - export NODE_ABI=127 - - export ARTIFACT_KEY="confluent-kafka-javascript-${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 + - export ARTIFACT_KEY="confluent-kafka-javascript-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-linux-${LIBC}-${ARCHITECTURE}.tar.gz" + - docker run -v "$(pwd):/v" node:22-alpine /v/.semaphore/build-docker.sh - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - artifact push workflow "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" diff --git a/CHANGELOG.md b/CHANGELOG.md index fd0bf1de..312f66d6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +# confluent-kafka-javascript v0.1.15-devel + +v0.1.15-devel is a pre-production, early-access release. + +## Features + +1. Add Node v22 builds and bump librdkafka version on each version bump of this library. + + # confluent-kafka-javascript v0.1.14-devel v0.1.14-devel is a pre-production, early-access release. diff --git a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json b/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json deleted file mode 100644 index c87e88b0..00000000 --- a/examples/kafkajs/oauthbearer_calback_authentication/package-lock.json +++ /dev/null @@ -1,188 +0,0 @@ -{ - "name": "oauthbearer_calback_authentication", - "version": "1.0.0", - "lockfileVersion": 3, - "requires": true, - "packages": { - "": { - "name": "oauthbearer_calback_authentication", - "version": "1.0.0", - "license": "ISC", - "dependencies": { - "@confluentinc/kafka-javascript": "file:../../..", - "jsonwebtoken": "^9.0.2" - } - }, - "../../..": { - "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", - "hasInstallScript": true, - "license": "MIT", - "dependencies": { - "@mapbox/node-pre-gyp": "^1.0.11", - "bindings": "^1.3.1", - "nan": "^2.17.0" - }, - "devDependencies": { - "@types/node": "^20.4.5", - "bluebird": "^3.5.3", - "eslint": "8.54.0", - "jest": "^29.7.0", - "jsdoc": "^4.0.2", - "jshint": "^2.13.6", - "mocha": "^10.2.0", - "node-gyp": "^9.3.1", - "typescript": "^5.1.6" - }, - "engines": { - "node": ">=18.0.0" - } - }, - "node_modules/@confluentinc/kafka-javascript": { - "resolved": "../../..", - "link": true - }, - "node_modules/buffer-equal-constant-time": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz", - "integrity": "sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA==" - }, - "node_modules/ecdsa-sig-formatter": { - "version": "1.0.11", - "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", - "integrity": "sha512-nagl3RYrbNv6kQkeJIpt6NJZy8twLB/2vtz6yN9Z4vRKHN4/QZJIEbqohALSgwKdnksuY3k5Addp5lg8sVoVcQ==", - "dependencies": { - "safe-buffer": "^5.0.1" - } - }, - "node_modules/jsonwebtoken": { - "version": "9.0.2", - "resolved": "https://registry.npmjs.org/jsonwebtoken/-/jsonwebtoken-9.0.2.tgz", - "integrity": "sha512-PRp66vJ865SSqOlgqS8hujT5U4AOgMfhrwYIuIhfKaoSCZcirrmASQr8CX7cUg+RMih+hgznrjp99o+W4pJLHQ==", - "dependencies": { - "jws": "^3.2.2", - "lodash.includes": "^4.3.0", - "lodash.isboolean": "^3.0.3", - "lodash.isinteger": "^4.0.4", - "lodash.isnumber": "^3.0.3", - "lodash.isplainobject": "^4.0.6", - "lodash.isstring": "^4.0.1", - "lodash.once": "^4.0.0", - "ms": "^2.1.1", - "semver": "^7.5.4" - }, - "engines": { - "node": ">=12", - "npm": ">=6" - } - }, - "node_modules/jwa": { - "version": "1.4.1", - "resolved": "https://registry.npmjs.org/jwa/-/jwa-1.4.1.tgz", - "integrity": "sha512-qiLX/xhEEFKUAJ6FiBMbes3w9ATzyk5W7Hvzpa/SLYdxNtng+gcurvrI7TbACjIXlsJyr05/S1oUhZrc63evQA==", - "dependencies": { - "buffer-equal-constant-time": "1.0.1", - "ecdsa-sig-formatter": "1.0.11", - "safe-buffer": "^5.0.1" - } - }, - "node_modules/jws": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/jws/-/jws-3.2.2.tgz", - "integrity": "sha512-YHlZCB6lMTllWDtSPHz/ZXTsi8S00usEV6v1tjq8tOUZzw7DpSDWVXjXDre6ed1w/pd495ODpHZYSdkRTsa0HA==", - "dependencies": { - "jwa": "^1.4.1", - "safe-buffer": "^5.0.1" - } - }, - "node_modules/lodash.includes": { - "version": "4.3.0", - "resolved": "https://registry.npmjs.org/lodash.includes/-/lodash.includes-4.3.0.tgz", - "integrity": "sha512-W3Bx6mdkRTGtlJISOvVD/lbqjTlPPUDTMnlXZFnVwi9NKJ6tiAk6LVdlhZMm17VZisqhKcgzpO5Wz91PCt5b0w==" - }, - "node_modules/lodash.isboolean": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/lodash.isboolean/-/lodash.isboolean-3.0.3.tgz", - "integrity": "sha512-Bz5mupy2SVbPHURB98VAcw+aHh4vRV5IPNhILUCsOzRmsTmSQ17jIuqopAentWoehktxGd9e/hbIXq980/1QJg==" - }, - "node_modules/lodash.isinteger": { - "version": "4.0.4", - "resolved": "https://registry.npmjs.org/lodash.isinteger/-/lodash.isinteger-4.0.4.tgz", - "integrity": "sha512-DBwtEWN2caHQ9/imiNeEA5ys1JoRtRfY3d7V9wkqtbycnAmTvRRmbHKDV4a0EYc678/dia0jrte4tjYwVBaZUA==" - }, - "node_modules/lodash.isnumber": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/lodash.isnumber/-/lodash.isnumber-3.0.3.tgz", - "integrity": "sha512-QYqzpfwO3/CWf3XP+Z+tkQsfaLL/EnUlXWVkIk5FUPc4sBdTehEqZONuyRt2P67PXAk+NXmTBcc97zw9t1FQrw==" - }, - "node_modules/lodash.isplainobject": { - "version": "4.0.6", - "resolved": "https://registry.npmjs.org/lodash.isplainobject/-/lodash.isplainobject-4.0.6.tgz", - "integrity": "sha512-oSXzaWypCMHkPC3NvBEaPHf0KsA5mvPrOPgQWDsbg8n7orZ290M0BmC/jgRZ4vcJ6DTAhjrsSYgdsW/F+MFOBA==" - }, - "node_modules/lodash.isstring": { - "version": "4.0.1", - "resolved": "https://registry.npmjs.org/lodash.isstring/-/lodash.isstring-4.0.1.tgz", - "integrity": "sha512-0wJxfxH1wgO3GrbuP+dTTk7op+6L41QCXbGINEmD+ny/G/eCqGzxyCsh7159S+mgDDcoarnBw6PC1PS5+wUGgw==" - }, - "node_modules/lodash.once": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz", - "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==" - }, - "node_modules/lru-cache": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", - "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=10" - } - }, - "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==" - }, - "node_modules/safe-buffer": { - "version": "5.2.1", - "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", - "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", - "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/semver": { - "version": "7.6.0", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.0.tgz", - "integrity": "sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==", - "dependencies": { - "lru-cache": "^6.0.0" - }, - "bin": { - "semver": "bin/semver.js" - }, - "engines": { - "node": ">=10" - } - }, - "node_modules/yallist": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", - "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" - } - } -} diff --git a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json b/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json deleted file mode 100644 index 83278c82..00000000 --- a/examples/node-rdkafka/oauthbearer_callback_authentication/package-lock.json +++ /dev/null @@ -1,188 +0,0 @@ -{ - "name": "oauthbearer_callback_authentication", - "version": "1.0.0", - "lockfileVersion": 3, - "requires": true, - "packages": { - "": { - "name": "oauthbearer_callback_authentication", - "version": "1.0.0", - "license": "MIT", - "dependencies": { - "@confluentinc/kafka-javascript": "file:../../..", - "jsonwebtoken": "^9.0.2" - } - }, - "../../..": { - "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", - "hasInstallScript": true, - "license": "MIT", - "dependencies": { - "@mapbox/node-pre-gyp": "^1.0.11", - "bindings": "^1.3.1", - "nan": "^2.17.0" - }, - "devDependencies": { - "@types/node": "^20.4.5", - "bluebird": "^3.5.3", - "eslint": "8.54.0", - "jest": "^29.7.0", - "jsdoc": "^4.0.2", - "jshint": "^2.13.6", - "mocha": "^10.2.0", - "node-gyp": "^9.3.1", - "typescript": "^5.1.6" - }, - "engines": { - "node": ">=18.0.0" - } - }, - "node_modules/@confluentinc/kafka-javascript": { - "resolved": "../../..", - "link": true - }, - "node_modules/buffer-equal-constant-time": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz", - "integrity": "sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA==" - }, - "node_modules/ecdsa-sig-formatter": { - "version": "1.0.11", - "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", - "integrity": "sha512-nagl3RYrbNv6kQkeJIpt6NJZy8twLB/2vtz6yN9Z4vRKHN4/QZJIEbqohALSgwKdnksuY3k5Addp5lg8sVoVcQ==", - "dependencies": { - "safe-buffer": "^5.0.1" - } - }, - "node_modules/jsonwebtoken": { - "version": "9.0.2", - "resolved": "https://registry.npmjs.org/jsonwebtoken/-/jsonwebtoken-9.0.2.tgz", - "integrity": "sha512-PRp66vJ865SSqOlgqS8hujT5U4AOgMfhrwYIuIhfKaoSCZcirrmASQr8CX7cUg+RMih+hgznrjp99o+W4pJLHQ==", - "dependencies": { - "jws": "^3.2.2", - "lodash.includes": "^4.3.0", - "lodash.isboolean": "^3.0.3", - "lodash.isinteger": "^4.0.4", - "lodash.isnumber": "^3.0.3", - "lodash.isplainobject": "^4.0.6", - "lodash.isstring": "^4.0.1", - "lodash.once": "^4.0.0", - "ms": "^2.1.1", - "semver": "^7.5.4" - }, - "engines": { - "node": ">=12", - "npm": ">=6" - } - }, - "node_modules/jwa": { - "version": "1.4.1", - "resolved": "https://registry.npmjs.org/jwa/-/jwa-1.4.1.tgz", - "integrity": "sha512-qiLX/xhEEFKUAJ6FiBMbes3w9ATzyk5W7Hvzpa/SLYdxNtng+gcurvrI7TbACjIXlsJyr05/S1oUhZrc63evQA==", - "dependencies": { - "buffer-equal-constant-time": "1.0.1", - "ecdsa-sig-formatter": "1.0.11", - "safe-buffer": "^5.0.1" - } - }, - "node_modules/jws": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/jws/-/jws-3.2.2.tgz", - "integrity": "sha512-YHlZCB6lMTllWDtSPHz/ZXTsi8S00usEV6v1tjq8tOUZzw7DpSDWVXjXDre6ed1w/pd495ODpHZYSdkRTsa0HA==", - "dependencies": { - "jwa": "^1.4.1", - "safe-buffer": "^5.0.1" - } - }, - "node_modules/lodash.includes": { - "version": "4.3.0", - "resolved": "https://registry.npmjs.org/lodash.includes/-/lodash.includes-4.3.0.tgz", - "integrity": "sha512-W3Bx6mdkRTGtlJISOvVD/lbqjTlPPUDTMnlXZFnVwi9NKJ6tiAk6LVdlhZMm17VZisqhKcgzpO5Wz91PCt5b0w==" - }, - "node_modules/lodash.isboolean": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/lodash.isboolean/-/lodash.isboolean-3.0.3.tgz", - "integrity": "sha512-Bz5mupy2SVbPHURB98VAcw+aHh4vRV5IPNhILUCsOzRmsTmSQ17jIuqopAentWoehktxGd9e/hbIXq980/1QJg==" - }, - "node_modules/lodash.isinteger": { - "version": "4.0.4", - "resolved": "https://registry.npmjs.org/lodash.isinteger/-/lodash.isinteger-4.0.4.tgz", - "integrity": "sha512-DBwtEWN2caHQ9/imiNeEA5ys1JoRtRfY3d7V9wkqtbycnAmTvRRmbHKDV4a0EYc678/dia0jrte4tjYwVBaZUA==" - }, - "node_modules/lodash.isnumber": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/lodash.isnumber/-/lodash.isnumber-3.0.3.tgz", - "integrity": "sha512-QYqzpfwO3/CWf3XP+Z+tkQsfaLL/EnUlXWVkIk5FUPc4sBdTehEqZONuyRt2P67PXAk+NXmTBcc97zw9t1FQrw==" - }, - "node_modules/lodash.isplainobject": { - "version": "4.0.6", - "resolved": "https://registry.npmjs.org/lodash.isplainobject/-/lodash.isplainobject-4.0.6.tgz", - "integrity": "sha512-oSXzaWypCMHkPC3NvBEaPHf0KsA5mvPrOPgQWDsbg8n7orZ290M0BmC/jgRZ4vcJ6DTAhjrsSYgdsW/F+MFOBA==" - }, - "node_modules/lodash.isstring": { - "version": "4.0.1", - "resolved": "https://registry.npmjs.org/lodash.isstring/-/lodash.isstring-4.0.1.tgz", - "integrity": "sha512-0wJxfxH1wgO3GrbuP+dTTk7op+6L41QCXbGINEmD+ny/G/eCqGzxyCsh7159S+mgDDcoarnBw6PC1PS5+wUGgw==" - }, - "node_modules/lodash.once": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz", - "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==" - }, - "node_modules/lru-cache": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", - "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=10" - } - }, - "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==" - }, - "node_modules/safe-buffer": { - "version": "5.2.1", - "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", - "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", - "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/semver": { - "version": "7.6.0", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.0.tgz", - "integrity": "sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==", - "dependencies": { - "lru-cache": "^6.0.0" - }, - "bin": { - "semver": "bin/semver.js" - }, - "engines": { - "node": ">=10" - } - }, - "node_modules/yallist": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", - "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" - } - } -} diff --git a/examples/package-lock.json b/examples/package-lock.json deleted file mode 100644 index aa944dc2..00000000 --- a/examples/package-lock.json +++ /dev/null @@ -1,45 +0,0 @@ -{ - "name": "examples", - "version": "1.0.0", - "lockfileVersion": 3, - "requires": true, - "packages": { - "": { - "name": "examples", - "version": "1.0.0", - "license": "MIT", - "dependencies": { - "@confluentinc/kafka-javascript": "file:.." - } - }, - "..": { - "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", - "hasInstallScript": true, - "license": "MIT", - "dependencies": { - "@mapbox/node-pre-gyp": "^1.0.11", - "bindings": "^1.3.1", - "nan": "^2.17.0" - }, - "devDependencies": { - "@types/node": "^20.4.5", - "bluebird": "^3.5.3", - "eslint": "8.54.0", - "jest": "^29.7.0", - "jsdoc": "^4.0.2", - "jshint": "^2.13.6", - "mocha": "^10.2.0", - "node-gyp": "^9.3.1", - "typescript": "^5.1.6" - }, - "engines": { - "node": ">=18.0.0" - } - }, - "node_modules/@confluentinc/kafka-javascript": { - "resolved": "..", - "link": true - } - } -} diff --git a/examples/typescript/package-lock.json b/examples/typescript/package-lock.json deleted file mode 100644 index a90bd306..00000000 --- a/examples/typescript/package-lock.json +++ /dev/null @@ -1,76 +0,0 @@ -{ - "name": "typescript", - "version": "1.0.0", - "lockfileVersion": 3, - "requires": true, - "packages": { - "": { - "name": "typescript", - "version": "1.0.0", - "license": "MIT", - "dependencies": { - "@confluentinc/kafka-javascript": "file:../..", - "typescript": "^5.4.4" - }, - "devDependencies": { - "@types/node": "^20.12.5" - } - }, - "../..": { - "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", - "hasInstallScript": true, - "license": "MIT", - "dependencies": { - "@mapbox/node-pre-gyp": "^1.0.11", - "bindings": "^1.3.1", - "nan": "^2.17.0" - }, - "devDependencies": { - "@types/node": "^20.4.5", - "bluebird": "^3.5.3", - "eslint": "8.54.0", - "jest": "^29.7.0", - "jsdoc": "^4.0.2", - "jshint": "^2.13.6", - "mocha": "^10.2.0", - "node-gyp": "^9.3.1", - "typescript": "^5.1.6" - }, - "engines": { - "node": ">=18.0.0" - } - }, - "node_modules/@confluentinc/kafka-javascript": { - "resolved": "../..", - "link": true - }, - "node_modules/@types/node": { - "version": "20.12.5", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.5.tgz", - "integrity": "sha512-BD+BjQ9LS/D8ST9p5uqBxghlN+S42iuNxjsUGjeZobe/ciXzk2qb1B6IXc6AnRLS+yFJRpN2IPEHMzwspfDJNw==", - "dev": true, - "dependencies": { - "undici-types": "~5.26.4" - } - }, - "node_modules/typescript": { - "version": "5.4.4", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.4.tgz", - "integrity": "sha512-dGE2Vv8cpVvw28v8HCPqyb08EzbBURxDpuhJvTrusShUfGnhHBafDsLdS1EhhxyL6BJQE+2cT3dDPAv+MQ6oLw==", - "bin": { - "tsc": "bin/tsc", - "tsserver": "bin/tsserver" - }, - "engines": { - "node": ">=14.17" - } - }, - "node_modules/undici-types": { - "version": "5.26.5", - "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-5.26.5.tgz", - "integrity": "sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==", - "dev": true - } - } -} diff --git a/lib/util.js b/lib/util.js index 52023cdf..78b0bac1 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.14-devel'; +util.bindingVersion = 'v0.1.15-devel'; diff --git a/package-lock.json b/package-lock.json index 525fd4a1..140e483e 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", + "version": "v0.1.15-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", + "version": "v0.1.15-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index 68490cba..0b3e3090 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.14-devel", + "version": "v0.1.15-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.4.0", From 9e803e0f72f568b6ccb101e2803299fe46579620 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 20 May 2024 14:07:54 +0530 Subject: [PATCH 146/224] Add committed() and metadata storage to commit/store for promisified API Adds tests Fixes types --- lib/kafkajs/_common.js | 36 +++++- lib/kafkajs/_consumer.js | 41 +++++- test/promisified/consumer/commit.spec.js | 152 +++++++++++++++++++++++ test/promisified/consumer/store.spec.js | 66 ++++++++++ types/kafkajs.d.ts | 5 +- types/rdkafka.d.ts | 14 ++- 6 files changed, 300 insertions(+), 14 deletions(-) create mode 100644 test/promisified/consumer/commit.spec.js diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index c24de9ce..b6a8db52 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -490,8 +490,6 @@ function checkIfKafkaJsKeysPresent(propertyType, config) { return null; } - - /** * Converts a topicPartitionOffset from KafkaJS to a format that can be used by node-rdkafka. * @param {import("../../types/kafkajs").TopicPartitionOffset} tpo @@ -506,6 +504,38 @@ function topicPartitionOffsetToRdKafka(tpo) { }; } +/** + * Converts a topicPartitionOffset from KafkaJS to a format that can be used by node-rdkafka. + * Includes metadata. + * + * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata} tpo + * @returns {import("../../types/rdkafka").TopicPartitionOffsetAndMetadata} + */ +function topicPartitionOffsetMetadataToRdKafka(tpo) { + return { + topic: tpo.topic, + partition: tpo.partition, + offset: tpo.offset ? Number(tpo.offset) : null, + metadata: tpo.metadata, + }; +} + +/** + * Converts a topicPartitionOffset from node-rdkafka to a format that can be used by KafkaJS. + * Includes metadata. + * + * @param {import("../../types/rdkafka").TopicPartitionOffsetAndMetadata} tpo + * @returns {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata} + */ +function topicPartitionOffsetMetadataToKafkaJS(tpo) { + return { + topic: tpo.topic, + partition: tpo.partition, + offset: tpo.offset ? tpo.offset.toString() : null, + metadata: tpo.metadata, + }; +} + /** * Convert a librdkafka error from node-rdkafka into a KafkaJSError. * @param {import("../error")} librdKafkaError to convert from. @@ -659,6 +689,8 @@ async function acquireOrLog(lock, logger) { module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, + topicPartitionOffsetMetadataToRdKafka, + topicPartitionOffsetMetadataToKafkaJS, createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders, notImplemented, diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index eb4a9897..7f78a993 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -4,6 +4,8 @@ const RdKafka = require('../rdkafka'); const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, + topicPartitionOffsetMetadataToRdKafka, + topicPartitionOffsetMetadataToKafkaJS, createKafkaJsErrorFromLibRdKafkaError, notImplemented, loggerTrampoline, @@ -754,7 +756,7 @@ class Consumer { /** * 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). + * @returns {import("../../types/rdkafka").TopicPartition[]} a list of (topic, partition). */ #flattenTopicPartitions(topics) { const ret = []; @@ -865,8 +867,6 @@ class Consumer { /** * Starts consumer polling. This method returns immediately. * @param {import("../../types/kafkajs").ConsumerRunConfig} config - * @note This should be used only when in KafkaJS compatibility mode (ie. with kafkaJS blocks in the config). - * If not in KafkaJS compatibility mode, this is deprecated, use consume() instead as the preferred method. */ async run(config) { if (this.#state !== ConsumerState.CONNECTED) { @@ -1084,7 +1084,7 @@ class Consumer { } const topicPartitionsRdKafka = topicPartitions.map( - topicPartitionOffsetToRdKafka); + topicPartitionOffsetMetadataToRdKafka); this.#internalClient.offsetsStore(topicPartitionsRdKafka); } @@ -1114,7 +1114,7 @@ class Consumer { this.#internalClient.commitSync(); } else { const topicPartitionsRdKafka = topicPartitions.map( - topicPartitionOffsetToRdKafka); + topicPartitionOffsetMetadataToRdKafka); this.#internalClient.commitSync(topicPartitionsRdKafka); } } catch (e) { @@ -1124,6 +1124,37 @@ class Consumer { } } + /** + * Fetch committed offsets for the given topic partitions. + * + * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]} topicPartitions - + * the topic partitions to check for committed offsets. Defaults to all assigned partitions. + * @param {number} timeout - timeout in ms. Defaults to infinite (-1). + * @returns {Promise} a promise that resolves to the committed offsets. + */ + async committed(topicPartitions = null, timeout = -1) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Committed can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + if (!topicPartitions) { + topicPartitions = this.assignment(); + } + + const topicPartitionsRdKafka = topicPartitions.map( + topicPartitionOffsetToRdKafka); + + return new Promise((resolve, reject) => { + this.#internalClient.committed(topicPartitionsRdKafka, timeout, (err, offsets) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + return; + } + resolve(offsets.map(topicPartitionOffsetMetadataToKafkaJS)); + }); + }); + } + /** * 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. diff --git a/test/promisified/consumer/commit.spec.js b/test/promisified/consumer/commit.spec.js new file mode 100644 index 00000000..b766b39c --- /dev/null +++ b/test/promisified/consumer/commit.spec.js @@ -0,0 +1,152 @@ +jest.setTimeout(30000) + +const { + secureRandom, + createTopic, + waitFor, + createProducer, + createConsumer, + sleep, +} = require('../testhelpers'); + +describe('Consumer commit', () => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}` + + await createTopic({ topic: topicName, partitions: 3 }) + + producer = createProducer({}); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: false, + autoCommitInterval: 500, + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }); + + it('should commit offsets', async () => { + /* Evenly distribute 30 messages across 3 partitions */ + let i = 0; + const messages = Array(3 * 10) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.connect(); + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + let msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + msgCount++; + const offset = (Number(message.offset) + 1).toString(); + await expect(() => consumer.commitOffsets([{ topic, partition, offset }])).not.toThrow(); + } + }); + await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); + expect(msgCount).toEqual(30); + + await consumer.disconnect(); + + /* Send 30 more messages */ + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + + msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async ({ message }) => { + msgCount++; + } + }) + /* Only the extra 30 messages should come to us */ + await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); + await sleep(1000); + expect(msgCount).toEqual(30); + }); + + it('should commit offsets with metadata', async () => { + /* Evenly distribute 30 messages across 3 partitions */ + let i = 0; + const messages = Array(3 * 10) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.connect(); + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + let msgCount = 0; + const metadata = 'unicode-metadata-😊'; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + msgCount++; + const offset = (Number(message.offset) + 1).toString(); + await expect(() => consumer.commitOffsets([{ topic, partition, offset, metadata }])).not.toThrow(); + } + }); + await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); + expect(msgCount).toEqual(30); + + let committed = await consumer.committed(null, 5000); + expect(committed).toEqual([ + { topic: topicName, partition: 0, offset: '10', metadata }, + { topic: topicName, partition: 1, offset: '10', metadata }, + { topic: topicName, partition: 2, offset: '10', metadata } + ]); + + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + + msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: 'not-a-real-topic-name' }) + + /* At this point, we're not actually assigned anything, but we should be able to fetch + * the stored offsets and metadata anyway since we're of the same consumer group. */ + committed = await consumer.committed([ + { topic: topicName, partition: 0 }, + { topic: topicName, partition: 1 }, + { topic: topicName, partition: 2 }, + ]); + expect(committed).toEqual([ + { topic: topicName, partition: 0, offset: '10', metadata }, + { topic: topicName, partition: 1, offset: '10', metadata }, + { topic: topicName, partition: 2, offset: '10', metadata } + ]); + }); + +}); diff --git a/test/promisified/consumer/store.spec.js b/test/promisified/consumer/store.spec.js index 3ee056d9..dcc54d52 100644 --- a/test/promisified/consumer/store.spec.js +++ b/test/promisified/consumer/store.spec.js @@ -151,4 +151,70 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { expect(msgCount).toEqual(30); }); + it('should commit stored offsets with metadata', async () => { + /* Evenly distribute 30 messages across 3 partitions */ + let i = 0; + const messages = Array(3 * 10) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.connect(); + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + let msgCount = 0; + const metadata = 'unicode-metadata-😊'; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + msgCount++; + const offset = (Number(message.offset) + 1).toString(); + expect(() => consumer.storeOffsets([{ topic, partition, offset, metadata }])).not.toThrow(); + } + }); + await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); + expect(msgCount).toEqual(30); + + if (!isAutoCommit) + await expect(consumer.commitOffsets()).resolves.toBeUndefined(); + else + await sleep(1000); /* Wait for auto-commit */ + + let committed = await consumer.committed(null, 5000); + expect(committed).toEqual([ + { topic: topicName, partition: 0, offset: '10', metadata }, + { topic: topicName, partition: 1, offset: '10', metadata }, + { topic: topicName, partition: 2, offset: '10', metadata } + ]); + + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + + msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: 'not-a-real-topic-name' }) + + /* At this point, we're not actually assigned anything, but we should be able to fetch + * the stored offsets and metadata anyway since we're of the same consumer group. */ + committed = await consumer.committed([ + { topic: topicName, partition: 0 }, + { topic: topicName, partition: 1 }, + { topic: topicName, partition: 2 } + ]); + expect(committed).toEqual([ + { topic: topicName, partition: 0, offset: '10', metadata }, + { topic: topicName, partition: 1, offset: '10', metadata }, + { topic: topicName, partition: 2, offset: '10', metadata } + ]); + }); + }); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 04669cab..0402ab79 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -311,8 +311,9 @@ export type Consumer = Client & { subscribe(subscription: ConsumerSubscribeTopics | ConsumerSubscribeTopic): Promise stop(): Promise run(config?: ConsumerRunConfig): Promise - storeOffsets(topicPartitions: Array): void - commitOffsets(topicPartitions?: Array): Promise + storeOffsets(topicPartitions: Array): void + commitOffsets(topicPartitions?: Array): Promise + committed(topicPartitions?: Array, timeout?: number): Promise seek(topicPartitionOffset: TopicPartitionOffset): Promise pause(topics: Array<{ topic: string; partitions?: number[] }>): void paused(): TopicPartitions[] diff --git a/types/rdkafka.d.ts b/types/rdkafka.d.ts index 76d7e23c..47eb3e80 100644 --- a/types/rdkafka.d.ts +++ b/types/rdkafka.d.ts @@ -79,6 +79,10 @@ export interface TopicPartitionOffset extends TopicPartition{ offset: number; } +export interface TopicPartitionOffsetAndMetadata extends TopicPartitionOffset { + metadata?: string | null; +} + export type TopicPartitionTime = TopicPartitionOffset; export type EofEvent = TopicPartitionOffset; @@ -212,17 +216,17 @@ export class KafkaConsumer extends Client { assignments(): Assignment[]; - commit(topicPartition: TopicPartitionOffset | TopicPartitionOffset[]): this; + commit(topicPartition: TopicPartitionOffsetAndMetadata | TopicPartitionOffsetAndMetadata[]): this; commit(): this; commitMessage(msg: TopicPartitionOffset): this; commitMessageSync(msg: TopicPartitionOffset): this; - commitSync(topicPartition: TopicPartitionOffset | TopicPartitionOffset[]): this; + commitSync(topicPartition: TopicPartitionOffsetAndMetadata | TopicPartitionOffsetAndMetadata[]): this; - committed(toppars: TopicPartition[], timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void): this; - committed(timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffset[]) => void): this; + committed(toppars: TopicPartition[], timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffsetAndMetadata[]) => void): this; + committed(timeout: number, cb: (err: LibrdKafkaError, topicPartitions: TopicPartitionOffsetAndMetadata[]) => void): this; consume(number: number, cb?: (err: LibrdKafkaError, messages: Message[]) => void): void; consume(cb: (err: LibrdKafkaError, messages: Message[]) => void): void; @@ -230,7 +234,7 @@ export class KafkaConsumer extends Client { getWatermarkOffsets(topic: string, partition: number): WatermarkOffsets; - offsetsStore(topicPartitions: TopicPartitionOffset[]): any; + offsetsStore(topicPartitions: TopicPartitionOffsetAndMetadata[]): any; pause(topicPartitions: TopicPartition[]): any; From c4a4e25f0fcc9ffb1df32d61fad38849298f2735 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 21 May 2024 15:05:59 +0530 Subject: [PATCH 147/224] Add leaderEpoch fields to toppars --- README.md | 4 +- lib/kafka-consumer.js | 6 ++- lib/kafkajs/_common.js | 3 ++ lib/kafkajs/_consumer.js | 6 ++- src/common.cc | 43 +++++++++++++++++-- test/promisified/consumer/commit.spec.js | 16 +++---- .../consumer/consumerCacheTests.spec.js | 10 +++-- test/promisified/consumer/store.spec.js | 15 ++++--- types/kafkajs.d.ts | 3 ++ types/rdkafka.d.ts | 3 +- 10 files changed, 81 insertions(+), 28 deletions(-) diff --git a/README.md b/README.md index 8565e0ff..609f68be 100644 --- a/README.md +++ b/README.md @@ -32,10 +32,10 @@ The following configurations are supported for this early access preview: * macOS - arm64/m1. * Windows - x64 (experimentally available in EA). -[A supported version of Python](https://devguide.python.org/versions/) must be available on the system for the installation process. [This is required for the `node-gyp` build tool.](https://github.com/nodejs/node-gyp?tab=readme-ov-file#configuring-python-dependency). - Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. +In case your system configuration is not within the supported ones, [a supported version of Python](https://devguide.python.org/versions/) must be available on the system for the installation process. [This is required for the `node-gyp` build tool.](https://github.com/nodejs/node-gyp?tab=readme-ov-file#configuring-python-dependency). + ```bash $ npm install @confluentinc/kafka-javascript ``` diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index cf65659b..6efaa631 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -605,7 +605,8 @@ KafkaConsumer.prototype.commitMessage = function(msg) { var topicPartition = { topic: msg.topic, partition: msg.partition, - offset: msg.offset + 1 + offset: msg.offset + 1, + leaderEpoch: msg.leaderEpoch }; this._errorWrap(this._client.commit(topicPartition), true); @@ -640,7 +641,8 @@ KafkaConsumer.prototype.commitMessageSync = function(msg) { var topicPartition = { topic: msg.topic, partition: msg.partition, - offset: msg.offset + 1 + offset: msg.offset + 1, + leaderEpoch: msg.leaderEpoch, }; this._errorWrap(this._client.commitSync(topicPartition), true); diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index b6a8db52..39e535d8 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -501,6 +501,7 @@ function topicPartitionOffsetToRdKafka(tpo) { topic: tpo.topic, partition: tpo.partition, offset: Number(tpo.offset), + leaderEpoch: tpo.leaderEpoch, }; } @@ -517,6 +518,7 @@ function topicPartitionOffsetMetadataToRdKafka(tpo) { partition: tpo.partition, offset: tpo.offset ? Number(tpo.offset) : null, metadata: tpo.metadata, + leaderEpoch: tpo.leaderEpoch, }; } @@ -533,6 +535,7 @@ function topicPartitionOffsetMetadataToKafkaJS(tpo) { partition: tpo.partition, offset: tpo.offset ? tpo.offset.toString() : null, metadata: tpo.metadata, + leaderEpoch: tpo.leaderEpoch }; } diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 7f78a993..14479265 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -633,6 +633,7 @@ class Consumer { attributes: 0, offset: String(message.offset), size: message.size, + leaderEpoch: message.leaderEpoch, headers }, heartbeat: async () => { /* no op */ }, @@ -675,6 +676,7 @@ class Consumer { attributes: 0, offset: String(message.offset), size: message.size, + leaderEpoch: message.leaderEpoch, headers }; @@ -1005,7 +1007,9 @@ class Consumer { if (eachMessageProcessed) { try { if (!this.#userManagedStores) { - this.#internalClient.offsetsStore([{ topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1 }]); + this.#internalClient.offsetsStore([{ + topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch + }]); } this.#lastConsumedOffsets.set(`${m.topic}|${m.partition}`, Number(m.offset) + 1); } catch (e) { diff --git a/src/common.cc b/src/common.cc index 6a436f8d..98947b03 100644 --- a/src/common.cc +++ b/src/common.cc @@ -341,6 +341,12 @@ v8::Local ToV8Array( Nan::New(topic_partition->topic().c_str()) .ToLocalChecked()); + int leader_epoch = topic_partition->get_leader_epoch(); + if (leader_epoch >= 0) { + Nan::Set(obj, Nan::New("leaderEpoch").ToLocalChecked(), + Nan::New(leader_epoch)); + } + Nan::Set(array, topic_partition_i, obj); } } @@ -387,6 +393,13 @@ v8::Local ToTopicPartitionV8Array( Nan::New(topic_partition.offset)); } + int leader_epoch = + rd_kafka_topic_partition_get_leader_epoch(&topic_partition); + if (leader_epoch >= 0) { + Nan::Set(obj, Nan::New("leaderEpoch").ToLocalChecked(), + Nan::New(leader_epoch)); + } + Nan::Set(array, topic_partition_i, obj); } return array; @@ -440,19 +453,35 @@ RdKafka::TopicPartition * FromV8Object(v8::Local topic_partition) { RdKafka::TopicPartition *toppar = RdKafka::TopicPartition::create(topic, partition, offset); - v8::Local metadataKey = Nan::New("metadata").ToLocalChecked(); + v8::Local metadataKey = Nan::New("metadata").ToLocalChecked(); if (Nan::Has(topic_partition, metadataKey).FromMaybe(false)) { - v8::Local metadataValue = Nan::Get(topic_partition, metadataKey).ToLocalChecked(); + v8::Local metadataValue = + Nan::Get(topic_partition, metadataKey).ToLocalChecked(); if (metadataValue->IsString()) { Nan::Utf8String metadataValueUtf8Str(metadataValue.As()); std::string metadataValueStr(*metadataValueUtf8Str); - std::vector metadataVector(metadataValueStr.begin(), metadataValueStr.end()); - metadataVector.push_back('\0'); // The null terminator is not included in the iterator. + std::vector metadataVector(metadataValueStr.begin(), + metadataValueStr.end()); + metadataVector.push_back( + '\0'); // The null terminator is not included in the iterator. toppar->set_metadata(metadataVector); } } + toppar->set_leader_epoch(-1); + v8::Local leaderEpochKey = + Nan::New("leaderEpoch").ToLocalChecked(); + if (Nan::Has(topic_partition, leaderEpochKey).FromMaybe(false)) { + v8::Local leaderEpochValue = + Nan::Get(topic_partition, leaderEpochKey).ToLocalChecked(); + + if (leaderEpochValue->IsNumber()) { + int32_t leaderEpoch = Nan::To(leaderEpochValue).FromJust(); + toppar->set_leader_epoch(leaderEpoch); + } + } + return toppar; } @@ -643,6 +672,12 @@ v8::Local ToV8Object(RdKafka::Message *message, Nan::Set(pack, Nan::New("timestamp").ToLocalChecked(), Nan::New(message->timestamp().timestamp)); + int32_t leader_epoch = message->leader_epoch(); + if (leader_epoch >= 0) { + Nan::Set(pack, Nan::New("leaderEpoch").ToLocalChecked(), + Nan::New(leader_epoch)); + } + return pack; } else { return RdKafkaError(message->err()); diff --git a/test/promisified/consumer/commit.spec.js b/test/promisified/consumer/commit.spec.js index b766b39c..1684f138 100644 --- a/test/promisified/consumer/commit.spec.js +++ b/test/promisified/consumer/commit.spec.js @@ -110,7 +110,8 @@ describe('Consumer commit', () => { eachMessage: async ({ topic, partition, message }) => { msgCount++; const offset = (Number(message.offset) + 1).toString(); - await expect(() => consumer.commitOffsets([{ topic, partition, offset, metadata }])).not.toThrow(); + const leaderEpoch = message.leaderEpoch; + await expect(() => consumer.commitOffsets([{ topic, partition, offset, metadata, leaderEpoch }])).not.toThrow(); } }); await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); @@ -118,9 +119,9 @@ describe('Consumer commit', () => { let committed = await consumer.committed(null, 5000); expect(committed).toEqual([ - { topic: topicName, partition: 0, offset: '10', metadata }, - { topic: topicName, partition: 1, offset: '10', metadata }, - { topic: topicName, partition: 2, offset: '10', metadata } + { topic: topicName, partition: 0, offset: '10', metadata, leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 1, offset: '10', metadata, leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 2, offset: '10', metadata, leaderEpoch: expect.any(Number) } ]); await consumer.disconnect(); @@ -143,10 +144,9 @@ describe('Consumer commit', () => { { topic: topicName, partition: 2 }, ]); expect(committed).toEqual([ - { topic: topicName, partition: 0, offset: '10', metadata }, - { topic: topicName, partition: 1, offset: '10', metadata }, - { topic: topicName, partition: 2, offset: '10', metadata } + { topic: topicName, partition: 0, offset: '10', metadata,leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 1, offset: '10', metadata,leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 2, offset: '10', metadata,leaderEpoch: expect.any(Number) } ]); }); - }); diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index a8791079..4f071e29 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -210,6 +210,7 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { fromBeginning: true, rebalanceTimeout: 10000, sessionTimeout: 10000, + autoCommitInterval: 1000, clientId: "impatientConsumer", autoCommit: isAutoCommit, }); @@ -254,8 +255,12 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { await producer.send({ topic: topicName, messages }) /* Wait for the messages - some of them, before starting the - * second consumer. */ - await waitForMessages(messagesConsumed, { number: 1024 }); + * second consumer. + * FIXME: This can get a bit flaky depending on the system, as sometimes + * the impatientConsumer consumes all the messages before consumer1TryingToJoin + * can be set to true */ + await waitForMessages(messagesConsumed, { number: 1024, delay: 100 }); + consumer1TryingToJoin = true; await consumer.connect(); await consumer.subscribe({ topic: topicName }); @@ -265,7 +270,6 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { consumer1Messages.push(event); } }); - consumer1TryingToJoin = true; await waitFor(() => consumer.assignment().length > 0, () => null); consumer1TryingToJoin = false; diff --git a/test/promisified/consumer/store.spec.js b/test/promisified/consumer/store.spec.js index dcc54d52..f69b9fc6 100644 --- a/test/promisified/consumer/store.spec.js +++ b/test/promisified/consumer/store.spec.js @@ -173,7 +173,8 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { eachMessage: async ({ topic, partition, message }) => { msgCount++; const offset = (Number(message.offset) + 1).toString(); - expect(() => consumer.storeOffsets([{ topic, partition, offset, metadata }])).not.toThrow(); + const leaderEpoch = message.leaderEpoch; + expect(() => consumer.storeOffsets([{ topic, partition, offset, metadata, leaderEpoch }])).not.toThrow(); } }); await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); @@ -186,9 +187,9 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { let committed = await consumer.committed(null, 5000); expect(committed).toEqual([ - { topic: topicName, partition: 0, offset: '10', metadata }, - { topic: topicName, partition: 1, offset: '10', metadata }, - { topic: topicName, partition: 2, offset: '10', metadata } + { topic: topicName, partition: 0, offset: '10', metadata, leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 1, offset: '10', metadata, leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 2, offset: '10', metadata, leaderEpoch: expect.any(Number) } ]); await consumer.disconnect(); @@ -211,9 +212,9 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { { topic: topicName, partition: 2 } ]); expect(committed).toEqual([ - { topic: topicName, partition: 0, offset: '10', metadata }, - { topic: topicName, partition: 1, offset: '10', metadata }, - { topic: topicName, partition: 2, offset: '10', metadata } + { topic: topicName, partition: 0, offset: '10', metadata, leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 1, offset: '10', metadata, leaderEpoch: expect.any(Number) }, + { topic: topicName, partition: 2, offset: '10', metadata, leaderEpoch: expect.any(Number) } ]); }); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 0402ab79..9ba640b4 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -220,6 +220,7 @@ interface MessageSetEntry { offset: string size: number headers?: never + leaderEpoch?: number } interface RecordBatchEntry { @@ -230,6 +231,7 @@ interface RecordBatchEntry { offset: string headers: IHeaders size?: never + leaderEpoch?: number } export type Batch = { @@ -294,6 +296,7 @@ export type TopicPartitions = { topic: string; partitions: number[] } export type TopicPartition = { topic: string partition: number + leaderEpoch?: number } export type TopicPartitionOffset = TopicPartition & { offset: string diff --git a/types/rdkafka.d.ts b/types/rdkafka.d.ts index 47eb3e80..3f481d72 100644 --- a/types/rdkafka.d.ts +++ b/types/rdkafka.d.ts @@ -73,9 +73,10 @@ export interface TopicPartition { topic: string; partition: number; error?: LibrdKafkaError; + leaderEpoch?: number; } -export interface TopicPartitionOffset extends TopicPartition{ +export interface TopicPartitionOffset extends TopicPartition { offset: number; } From 39bd9f9e8feef08f8e4c99d00788344255f2a0d1 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 4 Jun 2024 05:23:11 +0200 Subject: [PATCH 148/224] Add data-governance to code owners for schema registry clients (#52) * Add data-governance to code owners for schema registry clients * Fix ownership --- .github/CODEOWNERS | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 786e3d5c..e91fadf4 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1 +1 @@ -* @confluentinc/clients +* @confluentinc/clients @confluentinc/data-governance From d14794460d4a474e8e8ed0452b492f3754a5e1a0 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 5 Jun 2024 11:43:12 +0530 Subject: [PATCH 149/224] Fix deprecation warning --- lib/error.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/error.js b/lib/error.js index c212a50d..b0fdb426 100644 --- a/lib/error.js +++ b/lib/error.js @@ -405,7 +405,7 @@ function LibrdKafkaError(e) { this.origin = 'kafka'; } Error.captureStackTrace(this, this.constructor); - } else if (!util.isError(e)) { + } else if (!(Object.prototype.toString(e) === "[object Error]" || e instanceof Error)) { // This is the better way this.message = e.message; this.code = e.code; From 82b57d5b4194f82361a793f2fe6b0085cd204491 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 5 Jun 2024 11:43:36 +0530 Subject: [PATCH 150/224] Separate eachMessage and eachBatch internal consume loop --- lib/kafkajs/_consumer.js | 138 ++++++++++++++++++++++++++++++++++----- 1 file changed, 121 insertions(+), 17 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 14479265..4f54a365 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -902,11 +902,16 @@ class Consumer { } /* We deliberately don't await this. */ - this.#runInternal(config); + if (config.eachMessage) { + this.#runInternalEachMessage(config); + } else { + this.#runInternalEachBatch(config); + } } - /* Internal polling loop. It accepts the same config object that `run` accepts. */ - async #runInternal(config) { + /* Internal polling loop. + * It accepts the same config object that `run` accepts, but config.eachMessage must be set. */ + async #runInternalEachMessage(config) { while (this.#state === ConsumerState.CONNECTED) { /* We need to acquire a lock here, because we need to ensure that we don't @@ -954,26 +959,125 @@ class Consumer { } let eachMessageProcessed = false; - let payload; - if (config.eachMessage) { - payload = this.#createPayload(m); - } else { - payload = this.#createBatchPayload(m); + const payload = this.#createPayload(m); + try { + await config.eachMessage(payload); + eachMessageProcessed = true; + } 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 (which + * means seeking to the message offset). + * However, we don't do this inside the catch, but just outside it. This is because throwing an + * error is not the only case where we might want to seek back. + * + * So - do nothing but a debug log, but at this point eachMessageProcessed is false. + */ + this.#logger.debug(`Consumer encountered error while processing message. Error details: ${JSON.stringify(e)}. The same message may be reprocessed.`); + } + + /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ + if (!eachMessageProcessed) { + await this.seek({ + topic: m.topic, + partition: m.partition, + offset: m.offset, + }); + } + + /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ + if (eachMessageProcessed) { + try { + if (!this.#userManagedStores) { + this.#internalClient.offsetsStore([{ + topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch + }]); + } + this.#lastConsumedOffsets.set(`${m.topic}|${m.partition}`, Number(m.offset) + 1); + } catch (e) { + /* Not much we can do, except log the error. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); + } + } + + /* 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. + */ + + /* Release the lock so that any pending disconnect can go through. */ + await this.#lock.release(); + } + } + + /* Internal polling loop. + * It accepts the same config object that `run` accepts, but config.eachBatch must be set. */ + async #runInternalEachBatch(config) { + while (this.#state === ConsumerState.CONNECTED) { + + /* We need to acquire a lock here, because we need to ensure that we don't + * disconnect while in the middle of processing a message. */ + if (!(await acquireOrLog(this.#lock, this.#logger))) + continue; + + /* Invalidate the message cache if needed. */ + if (this.#messageCache.isStale()) { + await this.#clearCacheAndResetPositions(true); + await this.#lock.release(); + continue; + } + + const m = await this.#consumeSingleCached().catch(e => { + /* Since this error cannot be exposed to the user in the current situation, just log and retry. + * This is due to restartOnFailure being set to always true. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${JSON.stringify(e)}`); + }); + + if (!m) { + await this.#lock.release(); + 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. */ + this.#lock.release(); + continue; + } } + + let eachMessageProcessed = false; + const payload = this.#createBatchPayload(m); try { - if (config.eachMessage) { - await config.eachMessage(payload); + await config.eachBatch(payload); + if (config.eachBatchAutoResolve) { eachMessageProcessed = true; } else { - await config.eachBatch(payload); - if (config.eachBatchAutoResolve) { - eachMessageProcessed = true; - } else { - eachMessageProcessed = payload._messageResolved; - } + eachMessageProcessed = payload._messageResolved; } } catch (e) { - /* It's not only possible, but expected that an error will be thrown by eachMessage or eachBatch. + /* It's not only possible, but expected that an error will be thrown by eachBatch. * 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 (which * means seeking to the message offset). From 16d9547a88c5c11c2522c2696a0e2ed133286564 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 10 Jun 2024 13:21:54 +0530 Subject: [PATCH 151/224] Update performance example with more cases --- examples/performance/package.json | 16 +++ .../performance/performance-consolidated.js | 55 ++++++++++ ...omisified.js => performance-primitives.js} | 101 +++++++++++++++--- 3 files changed, 155 insertions(+), 17 deletions(-) create mode 100644 examples/performance/package.json create mode 100644 examples/performance/performance-consolidated.js rename examples/performance/{performance-promisified.js => performance-primitives.js} (60%) diff --git a/examples/performance/package.json b/examples/performance/package.json new file mode 100644 index 00000000..151d0816 --- /dev/null +++ b/examples/performance/package.json @@ -0,0 +1,16 @@ +{ + "name": "performance", + "version": "1.0.0", + "main": "performance-promisified.js", + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1" + }, + "keywords": [], + "author": "", + "license": "ISC", + "description": "", + "dependencies": { + "@confluentinc/kafka-javascript": "file:../..", + "kafkajs": "^2.2.4" + } +} diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js new file mode 100644 index 00000000..3ebd4b0c --- /dev/null +++ b/examples/performance/performance-consolidated.js @@ -0,0 +1,55 @@ +const { runProducer, runConsumer, runConsumeTransformProduce } = require('./performance-primitives'); + +const { CompressionTypes } = require('../../').KafkaJS; + +const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; +const topic = process.env.KAFKA_TOPIC || 'test-topic'; +const topic2 = process.env.KAFKA_TOPIC2 || 'test-topic2'; +const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; +const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; +const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; +const compression = process.env.COMPRESSION || CompressionTypes.NONE; +const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); + +(async function () { + const producer = process.argv.includes('--producer'); + const consumer = process.argv.includes('--consumer'); + const ctp = process.argv.includes('--ctp'); + const all = process.argv.includes('--all'); + + if (producer || all) { + console.log("=== Running Basic Producer Performance Test:") + console.log(` Brokers: ${brokers}`); + console.log(` Topic: ${topic}`); + console.log(` Message Count: ${messageCount}`); + console.log(` Message Size: ${messageSize}`); + console.log(` Batch Size: ${batchSize}`); + console.log(` Compression: ${compression}`); + console.log(` Warmup Messages: ${warmupMessages}`); + const producerRate = await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + console.log("=== Producer Rate: ", producerRate); + } + + if (consumer || all) { + // If user runs this without --producer then they are responsible for seeding the topic. + console.log("=== Running Basic Consumer Performance Test:") + console.log(` Brokers: ${brokers}`); + console.log(` Topic: ${topic}`); + console.log(` Message Count: ${messageCount}`); + const consumerRate = await runConsumer(brokers, topic, messageCount); + console.log("=== Consumer Rate: ", consumerRate); + } + + if (ctp || all) { + console.log("=== Running Consume-Transform-Produce Performance Test:") + console.log(` Brokers: ${brokers}`); + console.log(` ConsumeTopic: ${topic}`); + console.log(` ProduceTopic: ${topic2}`); + console.log(` Message Count: ${messageCount}`); + // Seed the topic with messages + await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, messageCount); + console.log("=== Consume-Transform-Produce Rate: ", ctpRate); + } + +})(); \ No newline at end of file diff --git a/examples/performance/performance-promisified.js b/examples/performance/performance-primitives.js similarity index 60% rename from examples/performance/performance-promisified.js rename to examples/performance/performance-primitives.js index b7b74054..af5a0e64 100644 --- a/examples/performance/performance-promisified.js +++ b/examples/performance/performance-primitives.js @@ -1,7 +1,13 @@ -const { Kafka, CompressionTypes, ErrorCodes } = require('../../').KafkaJS; +const { Kafka, ErrorCodes } = require('../../').KafkaJS; const { randomBytes } = require('crypto'); const { hrtime } = require('process'); +module.exports = { + runProducer, + runConsumer, + runConsumeTransformProduce, +}; + async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { let totalMessagesSent = 0; let totalBytesSent = 0; @@ -61,7 +67,6 @@ async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessa } await Promise.all(promises); } - console.log({messagesDispatched, totalMessageCnt}) let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; let rate = (totalBytesSent / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ @@ -81,7 +86,7 @@ async function runConsumer(brokers, topic, totalMessageCnt) { 'group.id': 'test-group' + Math.random(), 'enable.auto.commit': false, 'auto.offset.reset': 'earliest', - }); + }); await consumer.connect(); await consumer.subscribe({ topic }); @@ -101,8 +106,8 @@ async function runConsumer(brokers, topic, totalMessageCnt) { rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); consumer.pause([{ topic }]); - // } else if (messagesReceived % 100 == 0) { - // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); + // } else if (messagesReceived % 100 == 0) { + // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); } } }); @@ -135,15 +140,77 @@ async function runConsumer(brokers, topic, totalMessageCnt) { return rate; } -const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; -const topic = process.env.KAFKA_TOPIC || 'test-topic'; -const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; -const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; -const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; -const compression = process.env.COMPRESSION || CompressionTypes.NONE; -const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); - -runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression).then(async (producerRate) => { - const consumerRate = await runConsumer(brokers, topic, messageCount); - console.log(producerRate, consumerRate); -}); +async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, totalMessageCnt) { + const kafka = new Kafka({ + 'client.id': 'kafka-test-performance', + 'metadata.broker.list': brokers, + }); + + const producer = kafka.producer({ + /* We want things to be flushed immediately as we'll be awaiting this. */ + 'linger.ms': 0 + }); + await producer.connect(); + + const consumer = kafka.consumer({ + 'group.id': 'test-group' + Math.random(), + 'enable.auto.commit': false, + 'auto.offset.reset': 'earliest', + }); + await consumer.connect(); + await consumer.subscribe({ topic: consumeTopic }); + + let messagesReceived = 0; + let totalMessageSize = 0; + let startTime; + let rate; + consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + await producer.send({ + topic: produceTopic, + messages: [{ value: message.value }], + }) + messagesReceived++; + totalMessageSize += message.value.length; + if (messagesReceived === 1) { + consumer.pause([{ topic }]); + } else if (messagesReceived === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd, transformed and sent ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + // } else if (messagesReceived % 1 == 0) { + // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); + } + } + }); + + // Wait until the first message is received + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived > 0) { + clearInterval(interval); + resolve(); + } + }, 100); + }); + + console.log("Starting consume-transform-produce.") + + totalMessageSize = 0; + startTime = hrtime(); + consumer.resume([{ topic: consumeTopic }]); + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived >= totalMessageCnt) { + clearInterval(interval); + resolve(); + } + }, 1000); + }); + + await consumer.disconnect(); + await producer.disconnect(); + return rate; +} From 9d3130f148bf851700cfa4306c42326f90aba6ba Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 10 Jun 2024 16:10:10 +0530 Subject: [PATCH 152/224] Add per-partition cache with global expiry --- lib/kafkajs/_common.js | 10 + lib/kafkajs/_consumer.js | 225 ++++++++++++++---- .../consumer/consumerCacheTests.spec.js | 12 +- 3 files changed, 194 insertions(+), 53 deletions(-) diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 39e535d8..e59513fc 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -689,6 +689,15 @@ async function acquireOrLog(lock, logger) { return false; } +/** + * Creates a key for maps from a topicPartition object. + * @param {{topic: string, partition: number}} topicPartition Any object which can be treated as a topic partition. + * @returns {string} The created key. + */ +function partitionKey(topicPartition) { + return topicPartition.topic + '|'+ (topicPartition.partition); +} + module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, @@ -707,4 +716,5 @@ module.exports = { checkIfKafkaJsKeysPresent, Lock, acquireOrLog, + partitionKey, }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 4f54a365..2da28bf1 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -15,7 +15,8 @@ const { checkAllowedKeys, logLevel, Lock, - acquireOrLog + acquireOrLog, + partitionKey, } = require('./_common'); const { Buffer } = require('buffer'); const { hrtime } = require('process'); @@ -36,28 +37,21 @@ const PartitionAssigners = Object.freeze({ /** - * MessageCache represents a cache of messages that have been consumed, - * but not yet passed to the user. - * It has a dynamic capacity, increased or decreased based on requirement. + * A PerPartitionMessageCache is a cache for messages for a single partition. */ -class MessageCache { +class PerPartitionMessageCache { /* The cache is a list of messages. */ cache = []; - /* The maximum size of the cache. Set to 1 initially. */ - maxSize = 1; /* Index of next element to be fetched in the cache. */ - currentIndex = this.maxSize; + currentIndex = 0; /* Whether the cache is stale. */ stale = false; - /* Number of times the cache has been requested to be increased in size. */ - increaseCount = 0; - /* Last cached time */ - cachedTime = hrtime(); - /* Expiry duration for this cache */ - expiryDurationMs = 500; - constructor(expiryDurationMs) { - this.expiryDurationMs = expiryDurationMs; + /** + * Returns the number of total elements in the cache. + */ + size() { + return this.cache.length; } /** @@ -65,18 +59,107 @@ class MessageCache { */ clear() { this.cache = []; - this.maxSize = 1; - this.currentIndex = this.maxSize; + this.currentIndex = 0; this.stale = false; + } + + /** + * Adds a message to the cache. + */ + add(message) { + this.cache.push(message); + } + + /** + * Returns whether the cache is stale. + */ + isStale() { + return this.stale; + } + + /** + * @returns The next element in the cache or null if none exists. + * @warning Does not check for staleness. + */ + next() { + return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; + } +} + + +/** + * MessageCache defines a dynamically sized cache for messages. + * Internally, it uses PerPartitionMessageCache to store messages for each partition. + * The capacity is increased or decreased according to whether the last fetch of messages + * was less than the current capacity or saturated the current capacity. + */ +class MessageCache { + + constructor(expiryDurationMs) { + /* Per partition cache list containing non-empty PPCs */ + this.ppcList = []; + /* Map of topic+partition to PerPartitionMessageCache. */ + this.tpToPpc = new Map(); + /* Index of the current PPC in the ppcList. */ + this.currentPpc = 0; + /* Maximum size of the cache. (Capacity) */ + this.maxSize = 1; + /* Number of times the size has been increased in a row, used for accounting for maxSize. */ this.increaseCount = 0; + /* Last cached time */ this.cachedTime = hrtime(); + /* Whether the cache is stale. */ + this.stale = false; + /* Expiry duration for this cache */ + this.expiryDurationMs = expiryDurationMs; + } + + addTopicPartitions(topicPartitions) { + if (this.ppcList.length !== 0) { + throw new Error('Cannot add topic partitions to a non-empty cache.'); + } + for (const topicPartition of topicPartitions) { + const key = partitionKey(topicPartition); + this.tpToPpc.set(key, new PerPartitionMessageCache()); + } + } + + removeTopicPartitions(topicPartitions = null) { + if (this.ppcList.length !== 0) { + throw new Error('Cannot remove topic partitions from a non-empty cache.'); + } + + if (topicPartitions === null) { + this.tpToPpc.clear(); + return; + } + for (const topicPartition of assignment) { + const key = partitionKey(topicPartition); + this.tpToPpc.delete(key); + } + } + + /** + * Returns whether the cache is stale. + */ + isStale() { + if (this.stale) + return true; + + const cacheTime = hrtime(this.cachedTime); + const cacheTimeMs = Math.floor(cacheTime[0] * 1000 + cacheTime[1] / 1000000); + this.stale = cacheTimeMs > this.expiryDurationMs; + + // TODO: ideally, local staleness should not lead to global staleness. + // But for now, make it so because seeking to stored offset on local staleness is tricky. + this.stale = this.stale || this.ppcList.some(cache => cache.isStale()); + return this.stale; } /** * Request a size increase. * It increases the size by 2x, but only if the size is less than 1024, * only if the size has been requested to be increased twice in a row. - * @returns */ increaseMaxSize() { if (this.maxSize === 1024) @@ -101,33 +184,67 @@ class MessageCache { } /** - * Sets cache and resets all the indices and timer. - * @param {*} messages + * Add a single message to the cache. */ - setCache(messages) { - this.cache = messages; - this.currentIndex = 1; - this.cachedTime = hrtime(); + #add(message) { + const key = partitionKey(message) + const cache = this.tpToPpc.get(key); + cache.add(message); + if (cache.size() === 1) { + this.ppcList.push(cache); + } } /** - * @returns The next element in the cache or null if none exists. - * @warning Does not check for staleness. + * Adds many messages into the cache, partitioning them as per their toppar. */ - next() { - return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; + addMessages(messages) { + this.stale = false; + this.cachedTime = hrtime(); + this.currentPpc = 0; + for (const message of messages) + this.#add(message); + + // TODO: add ppcList sort step. + // Rationale: ideally it's best to consume in the ascending order of timestamps. } - /* Whether the cache is stale. */ - isStale() { - if (this.stale) - return true; + /** + * Returns the next element in the cache, or null if none exists. + * + * If the current PPC is exhausted, it moves to the next PPC. + * If all PPCs are exhausted, it returns null. + * @warning Does not check for staleness. That is left up to the user. + */ + next() { + if (this.currentPpc >= this.ppcList.length) { + return null; + } - const cacheTime = hrtime(this.cachedTime); - const cacheTimeMs = Math.floor(cacheTime[0] * 1000 + cacheTime[1] / 1000000); - return cacheTimeMs > this.expiryDurationMs; + let next = null; + while (next === null && this.currentPpc < this.ppcList.length) { + next = this.ppcList[this.currentPpc].next(); + if (next !== null) + break; + this.currentPpc++; + } + return next; // Caller is responsible for triggering fetch logic here if next == null. } + /** + * Clears cache completely. + */ + clear() { + for (const cache of this.ppcList) { + cache.clear(); + } + this.ppcList = []; + this.currentPpc = 0; + this.maxSize = 1; + this.increaseCount = 0; + this.stale = false; + this.cachedTime = hrtime(); + } } class Consumer { @@ -254,7 +371,7 @@ class Consumer { const assignment = this.assignment(); const seekPromises = []; for (const topicPartitionOffset of assignment) { - const key = `${topicPartitionOffset.topic}|${topicPartitionOffset.partition}`; + const key = partitionKey(topicPartitionOffset); if (!this.#lastConsumedOffsets.has(key)) continue; @@ -376,11 +493,19 @@ class Consumer { } } + // Populate per-partion caches. + // For cooperative sticky, just add the newly recieved partitions. + // If it's eager, it's already empty, so we can add all the partitions. + this.#messageCache.addTopicPartitions(assignment); + } else { - if (this.#internalClient.rebalanceProtocol() === "EAGER") + if (this.#internalClient.rebalanceProtocol() === "EAGER") { this.#internalClient.unassign(); - else + this.#messageCache.removeTopicPartitions(); + } else { this.#internalClient.incrementalUnassign(assignment); + this.#messageCache.removeTopicPartitions(assignment); + } } } catch (e) { // Ignore exceptions if we are not connected @@ -724,8 +849,8 @@ class Consumer { reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } - this.#messageCache.setCache(messages); - const message = messages[0]; + this.#messageCache.addMessages(messages); + const message = this.#messageCache.next(); if (messages.length === this.#messageCache.maxSize) { this.#messageCache.increaseMaxSize(); } else { @@ -930,7 +1055,7 @@ class Consumer { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) - this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${JSON.stringify(e)}`); + this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${e} : ${e.stack}`); }); if (!m) { @@ -973,7 +1098,7 @@ class Consumer { * * So - do nothing but a debug log, but at this point eachMessageProcessed is false. */ - this.#logger.debug(`Consumer encountered error while processing message. Error details: ${JSON.stringify(e)}. The same message may be reprocessed.`); + this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); } /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ @@ -993,7 +1118,7 @@ class Consumer { topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch }]); } - this.#lastConsumedOffsets.set(`${m.topic}|${m.partition}`, Number(m.offset) + 1); + this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); } catch (e) { /* Not much we can do, except log the error. */ if (this.#logger) @@ -1089,7 +1214,7 @@ class Consumer { * So - do nothing but a debug log, but at this point eachMessageProcessed needs to be false unless * the user has explicitly marked it as true. */ - this.#logger.debug(`Consumer encountered error while processing message. Error details: ${JSON.stringify(e)}. The same message may be reprocessed.`); + this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed * despite an error is if the user says so, and the user can use resolveOffsets for both the possible @@ -1115,7 +1240,7 @@ class Consumer { topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch }]); } - this.#lastConsumedOffsets.set(`${m.topic}|${m.partition}`, Number(m.offset) + 1); + this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); } catch (e) { /* Not much we can do, except log the error. */ if (this.#logger) @@ -1273,7 +1398,7 @@ class Consumer { for (let i = 0; i < assignment.length; i++) { const topicPartition = assignment[i]; - const key = `${topicPartition.topic}|${topicPartition.partition}`; + const key = partitionKey(topicPartition); if (!this.#pendingSeeks.has(key)) continue; @@ -1303,7 +1428,7 @@ class Consumer { let invalidateMessage = false; for (const topicPartition of assignment) { - const key = `${topicPartition.topic}|${topicPartition.partition}`; + const key = partitionKey(topicPartition); if (!this.#pendingSeeks.has(key)) continue; @@ -1372,7 +1497,7 @@ class Consumer { } this.#checkPendingSeeks = true; - this.#pendingSeeks.set(`${rdKafkaTopicPartitionOffset.topic}|${rdKafkaTopicPartitionOffset.partition}`, rdKafkaTopicPartitionOffset.offset); + this.#pendingSeeks.set(partitionKey(rdKafkaTopicPartitionOffset), rdKafkaTopicPartitionOffset.offset); } async describeGroup() { @@ -1441,6 +1566,8 @@ class Consumer { return; } this.#internalClient.pause(topics); + + // TODO: make this staleness per-partition, not on a global cache level. this.#messageCache.stale = true; topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index 4f071e29..d46d382f 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -202,13 +202,16 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { await consumer2.disconnect(); }); - it('does not hold up polling', async () => { - /* This consumer has a low max.poll.interval.ms */ + it('does not hold up polling for non-message events', async () => { + /* Even if the cache is full of messages, we should still be polling for + * non-message events like rebalances, etc. Internally, this is to make sure that + * we call poll() at least once within max.poll.interval.ms even if the cache is + * still full. This depends on us expiring the cache on time. */ const impatientConsumer = createConsumer({ groupId, maxWaitTimeInMs: 100, fromBeginning: true, - rebalanceTimeout: 10000, + rebalanceTimeout: 10000, /* also changes max.poll.interval.ms */ sessionTimeout: 10000, autoCommitInterval: 1000, clientId: "impatientConsumer", @@ -234,7 +237,8 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { ]); /* When the second consumer is joining, deliberately slow down message consumption. - * We should still have a rebalance very soon, since we must expire the cache and + * This is so the cache remains full. + * We should still have a rebalance very soon, since we will expire the cache and * trigger a rebalance before max.poll.interval.ms. */ if (consumer1TryingToJoin) { From a728b4a17a43173b07cc763b80dd22fa32c8b654 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 12 Jun 2024 12:47:47 +0530 Subject: [PATCH 153/224] Add per-partition cache expiry logic --- lib/kafkajs/_consumer.js | 315 +++++++-------------------------- lib/kafkajs/_consumer_cache.js | 298 +++++++++++++++++++++++++++++++ 2 files changed, 361 insertions(+), 252 deletions(-) create mode 100644 lib/kafkajs/_consumer_cache.js diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 2da28bf1..0875f6a3 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -19,7 +19,7 @@ const { partitionKey, } = require('./_common'); const { Buffer } = require('buffer'); -const { hrtime } = require('process'); +const MessageCache = require('./_consumer_cache'); const ConsumerState = Object.freeze({ INIT: 0, @@ -35,218 +35,6 @@ const PartitionAssigners = Object.freeze({ cooperativeSticky: 'cooperative-sticky', }); - -/** - * A PerPartitionMessageCache is a cache for messages for a single partition. - */ -class PerPartitionMessageCache { - /* The cache is a list of messages. */ - cache = []; - /* Index of next element to be fetched in the cache. */ - currentIndex = 0; - /* Whether the cache is stale. */ - stale = false; - - /** - * Returns the number of total elements in the cache. - */ - size() { - return this.cache.length; - } - - /** - * Clears the cache. - */ - clear() { - this.cache = []; - this.currentIndex = 0; - this.stale = false; - } - - /** - * Adds a message to the cache. - */ - add(message) { - this.cache.push(message); - } - - /** - * Returns whether the cache is stale. - */ - isStale() { - return this.stale; - } - - /** - * @returns The next element in the cache or null if none exists. - * @warning Does not check for staleness. - */ - next() { - return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; - } -} - - -/** - * MessageCache defines a dynamically sized cache for messages. - * Internally, it uses PerPartitionMessageCache to store messages for each partition. - * The capacity is increased or decreased according to whether the last fetch of messages - * was less than the current capacity or saturated the current capacity. - */ -class MessageCache { - - constructor(expiryDurationMs) { - /* Per partition cache list containing non-empty PPCs */ - this.ppcList = []; - /* Map of topic+partition to PerPartitionMessageCache. */ - this.tpToPpc = new Map(); - /* Index of the current PPC in the ppcList. */ - this.currentPpc = 0; - /* Maximum size of the cache. (Capacity) */ - this.maxSize = 1; - /* Number of times the size has been increased in a row, used for accounting for maxSize. */ - this.increaseCount = 0; - /* Last cached time */ - this.cachedTime = hrtime(); - /* Whether the cache is stale. */ - this.stale = false; - /* Expiry duration for this cache */ - this.expiryDurationMs = expiryDurationMs; - } - - addTopicPartitions(topicPartitions) { - if (this.ppcList.length !== 0) { - throw new Error('Cannot add topic partitions to a non-empty cache.'); - } - for (const topicPartition of topicPartitions) { - const key = partitionKey(topicPartition); - this.tpToPpc.set(key, new PerPartitionMessageCache()); - } - } - - removeTopicPartitions(topicPartitions = null) { - if (this.ppcList.length !== 0) { - throw new Error('Cannot remove topic partitions from a non-empty cache.'); - } - - if (topicPartitions === null) { - this.tpToPpc.clear(); - return; - } - for (const topicPartition of assignment) { - const key = partitionKey(topicPartition); - this.tpToPpc.delete(key); - } - } - - /** - * Returns whether the cache is stale. - */ - isStale() { - if (this.stale) - return true; - - const cacheTime = hrtime(this.cachedTime); - const cacheTimeMs = Math.floor(cacheTime[0] * 1000 + cacheTime[1] / 1000000); - this.stale = cacheTimeMs > this.expiryDurationMs; - - // TODO: ideally, local staleness should not lead to global staleness. - // But for now, make it so because seeking to stored offset on local staleness is tricky. - this.stale = this.stale || this.ppcList.some(cache => cache.isStale()); - return this.stale; - } - - /** - * Request a size increase. - * It increases the size by 2x, but only if the size is less than 1024, - * only if the size has been requested to be increased twice in a row. - */ - increaseMaxSize() { - if (this.maxSize === 1024) - return; - - this.increaseCount++; - if (this.increaseCount <= 1) - return; - - this.maxSize = Math.min(this.maxSize << 1, 1024); - this.increaseCount = 0; - } - - /** - * Request a size decrease. - * It decreases the size to 80% of the last received size, with a minimum of 1. - * @param {number} recvdSize - the number of messages received in the last poll. - */ - decreaseMaxSize(recvdSize) { - this.maxSize = Math.max(Math.floor((recvdSize * 8) / 10), 1); - this.increaseCount = 0; - } - - /** - * Add a single message to the cache. - */ - #add(message) { - const key = partitionKey(message) - const cache = this.tpToPpc.get(key); - cache.add(message); - if (cache.size() === 1) { - this.ppcList.push(cache); - } - } - - /** - * Adds many messages into the cache, partitioning them as per their toppar. - */ - addMessages(messages) { - this.stale = false; - this.cachedTime = hrtime(); - this.currentPpc = 0; - for (const message of messages) - this.#add(message); - - // TODO: add ppcList sort step. - // Rationale: ideally it's best to consume in the ascending order of timestamps. - } - - /** - * Returns the next element in the cache, or null if none exists. - * - * If the current PPC is exhausted, it moves to the next PPC. - * If all PPCs are exhausted, it returns null. - * @warning Does not check for staleness. That is left up to the user. - */ - next() { - if (this.currentPpc >= this.ppcList.length) { - return null; - } - - let next = null; - while (next === null && this.currentPpc < this.ppcList.length) { - next = this.ppcList[this.currentPpc].next(); - if (next !== null) - break; - this.currentPpc++; - } - return next; // Caller is responsible for triggering fetch logic here if next == null. - } - - /** - * Clears cache completely. - */ - clear() { - for (const cache of this.ppcList) { - cache.clear(); - } - this.ppcList = []; - this.currentPpc = 0; - this.maxSize = 1; - this.increaseCount = 0; - this.stale = false; - this.cachedTime = hrtime(); - } -} - class Consumer { /** * The config supplied by the user. @@ -312,7 +100,6 @@ class Consumer { /** * A map of topic+partition to the offset that was last consumed. * The keys are of the type "|". - * This is only populated when we're in the kafkaJS compatibility mode. * @type {Map} */ #lastConsumedOffsets = new Map(); @@ -358,25 +145,25 @@ class Consumer { } /** - * Clear the message cache. - * For simplicity, this always clears the entire message cache rather than being selective. + * Clear the message cache, and reset to stored positions. * - * @param {boolean} seek - whether to seek to the stored offsets after clearing the cache. - * this should be set to true if partitions are retained after this operation. + * @param {Array<{topic: string, partition: number}>|null} topicPartitions to clear the cache for, if null, then clear all assigned. */ - async #clearCacheAndResetPositions(seek = true) { - /* Seek to stored offset for each topic partition so that if - * we've gotten further along then they have, we can come back. */ - if (seek) { - const assignment = this.assignment(); - const seekPromises = []; - for (const topicPartitionOffset of assignment) { - const key = partitionKey(topicPartitionOffset); - if (!this.#lastConsumedOffsets.has(key)) - continue; + async #clearCacheAndResetPositions(topicPartitions = null) { + /* Seek to stored offset for each topic partition. It's possible that we've + * consumed messages upto N from the internalClient, but the user has stale'd the cache + * after consuming just k (< N) messages. We seek to k+1. */ + + const clearPartitions = topicPartitions ? topicPartitions : this.assignment(); + const seekPromises = []; + for (const topicPartitionOffset of clearPartitions) { + const key = partitionKey(topicPartitionOffset); + if (!this.#lastConsumedOffsets.has(key)) + continue; - /* Fire off a seek */ - const seekPromise = new Promise((resolve, reject) => this.#internalClient.seek({ + /* Fire off a seek */ + const seekPromise = new Promise((resolve, reject) => { + this.#internalClient.seek({ topic: topicPartitionOffset.topic, partition: topicPartitionOffset.partition, offset: +this.#lastConsumedOffsets.get(key) @@ -386,18 +173,24 @@ class Consumer { } else { resolve(); } - })); - seekPromises.push(seekPromise); - } + }); - /* TODO: we should cry more about this and render the consumer unusable. */ - await Promise.all(seekPromises).catch(err => this.#logger.error("Seek error. This is effectively a fatal error:" + err)); + this.#lastConsumedOffsets.delete(key); + }); + seekPromises.push(seekPromise); } - /* Clear the cache. */ - this.#messageCache.clear(); - /* Clear the offsets - no need to keep them around. */ - this.#lastConsumedOffsets.clear(); + /* TODO: we should cry more about this and render the consumer unusable. */ + await Promise.all(seekPromises).catch(err => this.#logger.error("Seek error. This is effectively a fatal error:" + err)); + + + /* Clear the cache and stored offsets. + * We need to do this only if topicPartitions = null (global cache expiry). + * This is because in case of a local cache expiry, MessageCache handles + * skipping that (and clearing that later before getting new messages). */ + if (!topicPartitions) { + this.#messageCache.clear(); + } } /** @@ -1044,9 +837,14 @@ class Consumer { if (!(await acquireOrLog(this.#lock, this.#logger))) continue; - /* Invalidate the message cache if needed. */ - if (this.#messageCache.isStale()) { - await this.#clearCacheAndResetPositions(true); + /* Invalidate the message cache if needed */ + const locallyStale = this.#messageCache.popLocallyStale(); + if (this.#messageCache.isStale()) { /* global staleness */ + await this.#clearCacheAndResetPositions(); + await this.#lock.release(); + continue; + } else if (locallyStale.length !== 0) { /* local staleness */ + await this.#clearCacheAndResetPositions(locallyStale); await this.#lock.release(); continue; } @@ -1153,9 +951,14 @@ class Consumer { if (!(await acquireOrLog(this.#lock, this.#logger))) continue; - /* Invalidate the message cache if needed. */ - if (this.#messageCache.isStale()) { - await this.#clearCacheAndResetPositions(true); + /* Invalidate the message cache if needed */ + const locallyStale = this.#messageCache.popLocallyStale(); + if (this.#messageCache.isStale()) { /* global staleness */ + await this.#clearCacheAndResetPositions(); + await this.#lock.release(); + continue; + } else if (locallyStale.length !== 0) { /* local staleness */ + await this.#clearCacheAndResetPositions(locallyStale); await this.#lock.release(); continue; } @@ -1441,14 +1244,21 @@ class Consumer { offset }; - /* We need a complete reset of the cache if we're seeking to a different offset even for one partition. - * At a later point, this may be improved at the cost of added complexity of maintaining message generation, - * or else purging the cache of just those partitions which are seeked. */ - await this.#clearCacheAndResetPositions(true); + /* The ideal sequence of events here is to: + * 1. Mark the cache as stale so we don't consume from it any further. + * 2. Call clearCacheAndResetPositions() for the topic partition, which is supposed + * to be called after each cache invalidation. + * + * However, what (2) does is to pop lastConsumedOffsets[topic partition], and seeks to + * the said popped value. Seeking is redundant since we seek here anyway. So, we can skip + * the seek by just clearing the lastConsumedOffsets[topic partition]. + */ + this.#messageCache.markStale([topicPartition]); + this.#lastConsumedOffsets.delete(key); /* 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. */ - /* TODO: is it work awaiting seeks to finish? */ + /* TODO: is it worth awaiting seeks to finish? */ this.#internalClient.seek(topicPartitionOffset, 0, err => err ? this.#logger.error(err) : null); offsetsToCommit.push({ topic: topicPartition.topic, @@ -1567,8 +1377,9 @@ class Consumer { } this.#internalClient.pause(topics); - // TODO: make this staleness per-partition, not on a global cache level. - this.#messageCache.stale = true; + /* Mark the messages in the cache as stale, runInternal* will deal with + * making it unusable. */ + this.#messageCache.markStale(topics); topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js new file mode 100644 index 00000000..5abd28c1 --- /dev/null +++ b/lib/kafkajs/_consumer_cache.js @@ -0,0 +1,298 @@ +const { hrtime } = require('process'); +const { + partitionKey, +} = require('./_common'); + +/** + * A PerPartitionMessageCache is a cache for messages for a single partition. + */ +class PerPartitionMessageCache { + /* The cache is a list of messages. */ + cache = []; + /* Index of next element to be fetched in the cache. */ + currentIndex = 0; + /* Whether the cache is stale. */ + stale = false; + + /** + * Returns the number of total elements in the cache. + */ + size() { + return this.cache.length; + } + + /** + * Clears the cache. + */ + clear() { + this.cache = []; + this.currentIndex = 0; + this.stale = false; + } + + /** + * Adds a message to the cache. + */ + add(message) { + this.cache.push(message); + } + + /** + * Returns whether the cache is stale. + */ + isStale() { + return this.stale; + } + + /** + * @returns The next element in the cache or null if none exists. + * @warning Does not check for staleness. + */ + next() { + return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; + } +} + + +/** + * MessageCache defines a dynamically sized cache for messages. + * Internally, it uses PerPartitionMessageCache to store messages for each partition. + * The capacity is increased or decreased according to whether the last fetch of messages + * was less than the current capacity or saturated the current capacity. + */ +class MessageCache { + + constructor(expiryDurationMs) { + /* Per partition cache list containing non-empty PPCs */ + this.ppcList = []; + /* Map of topic+partition to PerPartitionMessageCache. */ + this.tpToPpc = new Map(); + /* Index of the current PPC in the ppcList. */ + this.currentPpc = 0; + /* Maximum size of the cache. (Capacity) */ + this.maxSize = 1; + /* Number of times the size has been increased in a row, used for accounting for maxSize. */ + this.increaseCount = 0; + /* Last cached time */ + this.cachedTime = hrtime(); + /* Whether the cache is stale. */ + this.stale = false; + /* Expiry duration for this cache */ + this.expiryDurationMs = expiryDurationMs; + /* A list of caches which have been marked stale since the last call to popLocallyStale or addMessages. */ + this.locallyStaleCaches = []; + } + + /** + * Add a set of topic partitions to the cache (empty PPCs). + * Pre-conditions: ppcList must be empty (cache is inactive) + */ + addTopicPartitions(topicPartitions) { + if (this.ppcList.length !== 0) { + throw new Error('Cannot add topic partitions to a non-empty cache.'); + } + for (const topicPartition of topicPartitions) { + const key = partitionKey(topicPartition); + this.tpToPpc.set(key, new PerPartitionMessageCache()); + } + } + + /** + * Remove a set of topic partitions from the cache. + * If topicPartitions is null, removes everything. + * Pre-conditions: ppcList must be empty (cache is inactive) + */ + removeTopicPartitions(topicPartitions = null) { + if (this.ppcList.length !== 0) { + throw new Error('Cannot remove topic partitions from a non-empty cache.'); + } + + if (topicPartitions === null) { + this.tpToPpc.clear(); + return; + } + for (const topicPartition of topicPartitions) { + const key = partitionKey(topicPartition); + this.tpToPpc.delete(key); + } + } + + /** + * Returns whether the cache is globally stale. + */ + isStale() { + if (this.stale) + return true; + + const cacheTime = hrtime(this.cachedTime); + const cacheTimeMs = Math.floor(cacheTime[0] * 1000 + cacheTime[1] / 1000000); + this.stale = cacheTimeMs > this.expiryDurationMs; + + return this.stale; + } + + /** + * If there are any locally stale caches, return them, and clear + * the list of locally stale caches. + */ + popLocallyStale() { + if (this.locallyStaleCaches.length > 0) { + const locallyStale = this.locallyStaleCaches; + this.locallyStaleCaches = []; + return locallyStale; + } + return []; + } + + /** + * Mark a set of topic partitions 'stale'. + * If no topic partitions are provided, marks the entire cache as stale globally. + * + * Pre-conditions: toppars must be in tpToPpc, may or may not be in ppcList. + * Post-conditions: PPCs marked stale, locally stale caches updated to contain said toppars. + */ + markStale(topicPartitions = null) { + if (!topicPartitions) { + this.stale = true; + return; + } + + for (const topicPartition of topicPartitions) { + const key = partitionKey(topicPartition); + const cache = this.tpToPpc.get(key); + if (!cache) + continue; + + if (!cache.stale) { + /* Newly stale cache, so add it into list of such caches. */ + this.locallyStaleCaches.push(topicPartition); + } + cache.stale = true; + } + } + + /** + * Request a size increase. + * It increases the size by 2x, but only if the size is less than 1024, + * only if the size has been requested to be increased twice in a row. + */ + increaseMaxSize() { + if (this.maxSize === 1024) + return; + + this.increaseCount++; + if (this.increaseCount <= 1) + return; + + this.maxSize = Math.min(this.maxSize << 1, 1024); + this.increaseCount = 0; + } + + /** + * Request a size decrease. + * It decreases the size to 80% of the last received size, with a minimum of 1. + * @param {number} recvdSize - the number of messages received in the last poll. + */ + decreaseMaxSize(recvdSize) { + this.maxSize = Math.max(Math.floor((recvdSize * 8) / 10), 1); + this.increaseCount = 0; + } + + /** + * Add a single message to a PPC. + * Pre-conditions: PPC does not have stale messages. + * Post-conditions: PPC is unstale, ppcList contains all caches with messages in them. + */ + #add(message) { + const key = partitionKey(message) + const cache = this.tpToPpc.get(key); + cache.add(message); + if (cache.size() === 1) { + this.ppcList.push(cache); + /* Just in case this cache was marked stale by pause or seek, we unstale it now + * that there are fresh messages in here. It is possible because markStale() can + * mark toppar caches as stale without checking if they're in ppcList. */ + cache.stale = false; + } + } + + /** + * Adds many messages into the cache, partitioning them as per their toppar. + * Pre-conditions: no locally stale caches with messages in them. + * Post-conditions: all caches are unstale, (todo: ppcList is sorted by timestamp). + */ + addMessages(messages) { + /* There will be caches in the ppcList which are either stale, or have + * run out of messages. We need to clear them, else #add() will not add + * them back to the ppcList since they're not empty. */ + this.ppcList.forEach(cache => cache.clear()); + this.currentPpc = 0; + this.ppcList = []; + + if (this.locallyStaleCaches.length !== 0 && this.locallyStaleCaches.some(tp => { + const key = partitionKey(tp); + return this.tpToPpc.get(key).size() !== 0; + })) { + console.error('Locally stale caches should have been cleared before adding messages: ', this.locallyStaleCaches); + throw new Error('Locally stale caches should have been cleared before adding messages.'); + } + + this.stale = false; + this.cachedTime = hrtime(); + + for (const message of messages) + this.#add(message); + + // TODO: add ppcList sort step. + // Rationale: ideally it's best to consume in the ascending order of timestamps. + } + + /** + * Returns the next element in the cache, or null if none exists. + * + * If the current PPC is exhausted, it moves to the next PPC. + * If all PPCs are exhausted, it returns null. + * @warning Does not check for global staleness. That is left up to the user. + * Skips locally stale messages. + */ + next() { + if (this.currentPpc >= this.ppcList.length) { + return null; + } + + let next = null; + while (next === null && this.currentPpc < this.ppcList.length) { + if (this.ppcList[this.currentPpc].isStale()) { + this.currentPpc++; + continue; + } + + next = this.ppcList[this.currentPpc].next(); + if (next !== null) + break; + this.currentPpc++; + } + return next; // Caller is responsible for triggering fetch logic here if next == null. + } + + /** + * Clears the cache completely. + * This resets it to a base state, and reduces the capacity of the cache back to 1. + * Pre-conditions: none + * Post-conditions: maxSize = 1, all caches are unstale, ppcList is empty, locallyStaleCaches is empty. + */ + clear() { + for (const cache of this.ppcList) { + cache.clear(); + } + this.ppcList = []; + this.currentPpc = 0; + this.maxSize = 1; + this.increaseCount = 0; + this.stale = false; + this.cachedTime = hrtime(); + this.locallyStaleCaches = []; + } +} + +module.exports = MessageCache; From 28d2253cdaac27dca3dfb98bc8a9091cdb7090b7 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 13 Jun 2024 11:29:57 +0530 Subject: [PATCH 154/224] Allow cache to disburse multiple messages at once --- LICENSE.heap-js | 36 + lib/kafkajs/_consumer.js | 34 +- lib/kafkajs/_consumer_cache.js | 82 +- lib/kafkajs/_heap.js | 2353 ++++++++++++++++++++++++++++++++ 4 files changed, 2483 insertions(+), 22 deletions(-) create mode 100644 LICENSE.heap-js create mode 100644 lib/kafkajs/_heap.js diff --git a/LICENSE.heap-js b/LICENSE.heap-js new file mode 100644 index 00000000..a2a9eba1 --- /dev/null +++ b/LICENSE.heap-js @@ -0,0 +1,36 @@ +Code from https://github.com/ignlg/heap-js/ | Commit Hash: a3bb403 | dist/heap-js.es5.js + +Code used in lib/kafkajs/_heap.js + +---- + + +BSD 3-Clause License + +Copyright (c) 2017, Ignacio Lago +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 0875f6a3..0db4284f 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -353,7 +353,7 @@ class Consumer { if (Object.hasOwn(kjsConfig, 'rebalanceTimeout')) { /* In librdkafka, we use the max poll interval as the rebalance timeout as well. */ - rdKafkaConfig['max.poll.interval.ms'] = kjsConfig.rebalanceTimeout; + rdKafkaConfig['max.poll.interval.ms'] = +kjsConfig.rebalanceTimeout; } else { rdKafkaConfig['max.poll.interval.ms'] = 300000; /* librdkafka default */ } @@ -627,15 +627,24 @@ class Consumer { /** * Consumes a single message from the internal consumer. + * @param {number} savedIndex - the index of the message in the cache to return. * @returns {Promise} a promise that resolves to a single message. * @note this method caches messages as well, but returns only a single message. */ - async #consumeSingleCached() { - const msg = this.#messageCache.next(); + async #consumeSingleCached(savedIndex) { + const msg = this.#messageCache.next(savedIndex); if (msg) { return msg; } + // TODO: Add this block for concurrency + // if (!msg) { + // // it's possible that we get msg = null, but that's because partitionConcurrency + // // exceeds the number of partitions containing messages. So in this case, + // // we should not call for new fetches, rather, try to focus on what we have left. + // return null; + // } + return new Promise((resolve, reject) => { this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { if (err) { @@ -712,7 +721,8 @@ class Consumer { } const rdKafkaConfig = this.#config(); - this.#messageCache = new MessageCache(Math.floor(rdKafkaConfig['max.poll.interval.ms'] * 0.8)); + const maxPollInterval = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; + this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), 1); this.#state = ConsumerState.CONNECTING; this.#internalClient = new RdKafka.KafkaConsumer(rdKafkaConfig); @@ -830,6 +840,7 @@ class Consumer { /* Internal polling loop. * It accepts the same config object that `run` accepts, but config.eachMessage must be set. */ async #runInternalEachMessage(config) { + let savedIdx = -1; while (this.#state === ConsumerState.CONNECTED) { /* We need to acquire a lock here, because we need to ensure that we don't @@ -840,16 +851,20 @@ class Consumer { /* Invalidate the message cache if needed */ const locallyStale = this.#messageCache.popLocallyStale(); if (this.#messageCache.isStale()) { /* global staleness */ + // TODO: await all concurrent promises for eachMessage here. await this.#clearCacheAndResetPositions(); await this.#lock.release(); continue; } else if (locallyStale.length !== 0) { /* local staleness */ + // TODO: is it correct to await some concurrent promises for eachMessage here? + // to be safe we can do it, but I don't think we really need to do that for + // correctness. await this.#clearCacheAndResetPositions(locallyStale); await this.#lock.release(); continue; } - const m = await this.#consumeSingleCached().catch(e => { + const m = await this.#consumeSingleCached(savedIdx).catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) @@ -857,9 +872,13 @@ class Consumer { }); if (!m) { + // await all concurrency related promises right here if this is null, if any such promise exists. + // see note in consumeSingleCached + savedIdx = -1; await this.#lock.release(); continue; } + savedIdx = m.index; /* TODO: add partitionsConsumedConcurrently-based concurrency here. * If we maintain a map of topic partitions to promises, and a counter, @@ -944,6 +963,7 @@ class Consumer { /* Internal polling loop. * It accepts the same config object that `run` accepts, but config.eachBatch must be set. */ async #runInternalEachBatch(config) { + let savedIdx = -1; while (this.#state === ConsumerState.CONNECTED) { /* We need to acquire a lock here, because we need to ensure that we don't @@ -963,7 +983,7 @@ class Consumer { continue; } - const m = await this.#consumeSingleCached().catch(e => { + const m = await this.#consumeSingleCached(savedIdx).catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) @@ -971,9 +991,11 @@ class Consumer { }); if (!m) { + savedIdx = -1; await this.#lock.release(); continue; } + savedIdx = m.index; /* TODO: add partitionsConsumedConcurrently-based concurrency here. * If we maintain a map of topic partitions to promises, and a counter, diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 5abd28c1..6bd345b5 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -2,6 +2,7 @@ const { hrtime } = require('process'); const { partitionKey, } = require('./_common'); +const { Heap } = require('./_heap'); /** * A PerPartitionMessageCache is a cache for messages for a single partition. @@ -62,13 +63,13 @@ class PerPartitionMessageCache { */ class MessageCache { - constructor(expiryDurationMs) { + constructor(expiryDurationMs, maxConcurrency) { /* Per partition cache list containing non-empty PPCs */ this.ppcList = []; /* Map of topic+partition to PerPartitionMessageCache. */ this.tpToPpc = new Map(); /* Index of the current PPC in the ppcList. */ - this.currentPpc = 0; + this.currentPpcTODO_remove_this = 0; /* Maximum size of the cache. (Capacity) */ this.maxSize = 1; /* Number of times the size has been increased in a row, used for accounting for maxSize. */ @@ -81,6 +82,15 @@ class MessageCache { this.expiryDurationMs = expiryDurationMs; /* A list of caches which have been marked stale since the last call to popLocallyStale or addMessages. */ this.locallyStaleCaches = []; + /* Max allowed concurrency */ + this.maxConcurrency = maxConcurrency; + /* Contains a list of indices of ppcList from which we are allowed to consume. */ + this.indices = new Heap(); + /* Largest ppc index we are allowed to consume from (inclusive). */ + this.maxIndicesIndex = 0; + /* Contains a list of indices of ppcList from which we have sent a message returned through next, but + * the user has not returned the index back to us via next(idx) */ + this.pendingIndices = new Set(); } /** @@ -226,7 +236,6 @@ class MessageCache { * run out of messages. We need to clear them, else #add() will not add * them back to the ppcList since they're not empty. */ this.ppcList.forEach(cache => cache.clear()); - this.currentPpc = 0; this.ppcList = []; if (this.locallyStaleCaches.length !== 0 && this.locallyStaleCaches.some(tp => { @@ -245,6 +254,15 @@ class MessageCache { // TODO: add ppcList sort step. // Rationale: ideally it's best to consume in the ascending order of timestamps. + + /* Reset the indices and pendingIndices because ppcList is being created newly. */ + this.indices.clear(); + if (this.pendingIndices.size > 0) console.error('addMessages: pendingIndices = ', this.pendingIndices, console.trace()); + this.pendingIndices.clear(); + this.maxIndicesIndex = Math.min(this.maxConcurrency, this.ppcList.length - 1); + for (let i = 0; i <= this.maxIndicesIndex; i++) { + this.indices.push(i); + } } /** @@ -252,27 +270,56 @@ class MessageCache { * * If the current PPC is exhausted, it moves to the next PPC. * If all PPCs are exhausted, it returns null. + * @param {number} idx - after a consumer has consumed a message, it must return the index back to us via this parameter. + * otherwise, no messages from that topic partition will be consumed. + * @returns {Object} - the next message in the cache, or null if none exists. An `index` field is added to the message. * @warning Does not check for global staleness. That is left up to the user. * Skips locally stale messages. + * The topicPartition, if provided, MUST be one such that the user has fetched + * the message from the same topicPartition earlier. + * @note Whenever making changes to this function, ensure that you benchmark perf. */ - next() { - if (this.currentPpc >= this.ppcList.length) { - return null; + next(idx = -1) { + let index = idx; + if (!this.pendingIndices.has(index)) { + /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible + * that we ran out of messages and fetched a new batch. So we just discard what the user is + * returning to us. */ + index = -1; + } else { + this.pendingIndices.delete(index); } - let next = null; - while (next === null && this.currentPpc < this.ppcList.length) { - if (this.ppcList[this.currentPpc].isStale()) { - this.currentPpc++; + if (index === -1) { + if (this.indices.size() === 0) + return null; + index = this.indices.pop(); // index cannot be undefined here since indices.size > 0 + } + + while (true) { + const next = this.ppcList[index].next(); + if (this.ppcList[index].isStale() || next === null) { + /* If the current PPC is stale or empty, then we move on to the next one. + * It is equally valid to choose any PPC available within this.indices, or else + * move on to the next PPC (maxIndicesIndex + 1) if available. + * We prefer the second option a bit more since we don't have to do a heap operation. */ + const toAdd = this.maxIndicesIndex + 1; + if (toAdd < this.ppcList.length) { + this.maxIndicesIndex = toAdd; + index = toAdd; + } else if (!this.indices.isEmpty()) { + index = this.indices.pop() + } else { + break; // nothing left. + } continue; } - next = this.ppcList[this.currentPpc].next(); - if (next !== null) - break; - this.currentPpc++; + this.pendingIndices.add(index); + next.index = index; + return next; } - return next; // Caller is responsible for triggering fetch logic here if next == null. + return null; // Caller is responsible for triggering fetch logic here if next == null. } /** @@ -286,12 +333,15 @@ class MessageCache { cache.clear(); } this.ppcList = []; - this.currentPpc = 0; this.maxSize = 1; this.increaseCount = 0; this.stale = false; this.cachedTime = hrtime(); this.locallyStaleCaches = []; + this.indices.clear(); + // if (this.pendingIndices.size > 0) console.log('clear: pendingIndices = ', this.pendingIndices, console.); + this.pendingIndices.clear(); + this.currentIndex = 0; } } diff --git a/lib/kafkajs/_heap.js b/lib/kafkajs/_heap.js new file mode 100644 index 00000000..cd486661 --- /dev/null +++ b/lib/kafkajs/_heap.js @@ -0,0 +1,2353 @@ +/** +Code from https://github.com/ignlg/heap-js/ | Commit Hash: a3bb403 | dist/heap-js.es5.js + + +---- + + +BSD 3-Clause License + +Copyright (c) 2017, Ignacio Lago +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +*/ + +var __awaiter = (undefined && undefined.__awaiter) || function (thisArg, _arguments, P, generator) { + function adopt(value) { return value instanceof P ? value : new P(function (resolve) { resolve(value); }); } + return new (P || (P = Promise))(function (resolve, reject) { + function fulfilled(value) { try { step(generator.next(value)); } catch (e) { reject(e); } } + function rejected(value) { try { step(generator["throw"](value)); } catch (e) { reject(e); } } + function step(result) { result.done ? resolve(result.value) : adopt(result.value).then(fulfilled, rejected); } + step((generator = generator.apply(thisArg, _arguments || [])).next()); + }); +}; +var __generator$1 = (undefined && undefined.__generator) || function (thisArg, body) { + var _ = { label: 0, sent: function() { if (t[0] & 1) throw t[1]; return t[1]; }, trys: [], ops: [] }, f, y, t, g; + return g = { next: verb(0), "throw": verb(1), "return": verb(2) }, typeof Symbol === "function" && (g[Symbol.iterator] = function() { return this; }), g; + function verb(n) { return function (v) { return step([n, v]); }; } + function step(op) { + if (f) throw new TypeError("Generator is already executing."); + while (g && (g = 0, op[0] && (_ = 0)), _) try { + if (f = 1, y && (t = op[0] & 2 ? y["return"] : op[0] ? y["throw"] || ((t = y["return"]) && t.call(y), 0) : y.next) && !(t = t.call(y, op[1])).done) return t; + if (y = 0, t) op = [op[0] & 2, t.value]; + switch (op[0]) { + case 0: case 1: t = op; break; + case 4: _.label++; return { value: op[1], done: false }; + case 5: _.label++; y = op[1]; op = [0]; continue; + case 7: op = _.ops.pop(); _.trys.pop(); continue; + default: + if (!(t = _.trys, t = t.length > 0 && t[t.length - 1]) && (op[0] === 6 || op[0] === 2)) { _ = 0; continue; } + if (op[0] === 3 && (!t || (op[1] > t[0] && op[1] < t[3]))) { _.label = op[1]; break; } + if (op[0] === 6 && _.label < t[1]) { _.label = t[1]; t = op; break; } + if (t && _.label < t[2]) { _.label = t[2]; _.ops.push(op); break; } + if (t[2]) _.ops.pop(); + _.trys.pop(); continue; + } + op = body.call(thisArg, _); + } catch (e) { op = [6, e]; y = 0; } finally { f = t = 0; } + if (op[0] & 5) throw op[1]; return { value: op[0] ? op[1] : void 0, done: true }; + } +}; +var __read$1 = (undefined && undefined.__read) || function (o, n) { + var m = typeof Symbol === "function" && o[Symbol.iterator]; + if (!m) return o; + var i = m.call(o), r, ar = [], e; + try { + while ((n === void 0 || n-- > 0) && !(r = i.next()).done) ar.push(r.value); + } + catch (error) { e = { error: error }; } + finally { + try { + if (r && !r.done && (m = i["return"])) m.call(i); + } + finally { if (e) throw e.error; } + } + return ar; +}; +var __spreadArray$1 = (undefined && undefined.__spreadArray) || function (to, from, pack) { + if (pack || arguments.length === 2) for (var i = 0, l = from.length, ar; i < l; i++) { + if (ar || !(i in from)) { + if (!ar) ar = Array.prototype.slice.call(from, 0, i); + ar[i] = from[i]; + } + } + return to.concat(ar || Array.prototype.slice.call(from)); +}; +var __values = (undefined && undefined.__values) || function(o) { + var s = typeof Symbol === "function" && Symbol.iterator, m = s && o[s], i = 0; + if (m) return m.call(o); + if (o && typeof o.length === "number") return { + next: function () { + if (o && i >= o.length) o = void 0; + return { value: o && o[i++], done: !o }; + } + }; + throw new TypeError(s ? "Object is not iterable." : "Symbol.iterator is not defined."); +}; +/** + * Heap + * @type {Class} + */ +var HeapAsync = /** @class */ (function () { + /** + * Heap instance constructor. + * @param {Function} compare Optional comparison function, defaults to Heap.minComparator + */ + function HeapAsync(compare) { + if (compare === void 0) { compare = HeapAsync.minComparator; } + var _this = this; + this.compare = compare; + this.heapArray = []; + this._limit = 0; + /** + * Alias of add + */ + this.offer = this.add; + /** + * Alias of peek + */ + this.element = this.peek; + /** + * Alias of pop + */ + this.poll = this.pop; + /** + * Returns the inverse to the comparison function. + * @return {Number} + */ + this._invertedCompare = function (a, b) { + return _this.compare(a, b).then(function (res) { return -1 * res; }); + }; + } + /* + Static methods + */ + /** + * Gets children indices for given index. + * @param {Number} idx Parent index + * @return {Array(Number)} Array of children indices + */ + HeapAsync.getChildrenIndexOf = function (idx) { + return [idx * 2 + 1, idx * 2 + 2]; + }; + /** + * Gets parent index for given index. + * @param {Number} idx Children index + * @return {Number | undefined} Parent index, -1 if idx is 0 + */ + HeapAsync.getParentIndexOf = function (idx) { + if (idx <= 0) { + return -1; + } + var whichChildren = idx % 2 ? 1 : 2; + return Math.floor((idx - whichChildren) / 2); + }; + /** + * Gets sibling index for given index. + * @param {Number} idx Children index + * @return {Number | undefined} Sibling index, -1 if idx is 0 + */ + HeapAsync.getSiblingIndexOf = function (idx) { + if (idx <= 0) { + return -1; + } + var whichChildren = idx % 2 ? 1 : -1; + return idx + whichChildren; + }; + /** + * Min heap comparison function, default. + * @param {any} a First element + * @param {any} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + HeapAsync.minComparator = function (a, b) { + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + if (a > b) { + return [2 /*return*/, 1]; + } + else if (a < b) { + return [2 /*return*/, -1]; + } + else { + return [2 /*return*/, 0]; + } + }); + }); + }; + /** + * Max heap comparison function. + * @param {any} a First element + * @param {any} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + HeapAsync.maxComparator = function (a, b) { + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + if (b > a) { + return [2 /*return*/, 1]; + } + else if (b < a) { + return [2 /*return*/, -1]; + } + else { + return [2 /*return*/, 0]; + } + }); + }); + }; + /** + * Min number heap comparison function, default. + * @param {Number} a First element + * @param {Number} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + HeapAsync.minComparatorNumber = function (a, b) { + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + return [2 /*return*/, a - b]; + }); + }); + }; + /** + * Max number heap comparison function. + * @param {Number} a First element + * @param {Number} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + HeapAsync.maxComparatorNumber = function (a, b) { + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + return [2 /*return*/, b - a]; + }); + }); + }; + /** + * Default equality function. + * @param {any} a First element + * @param {any} b Second element + * @return {Boolean} True if equal, false otherwise + */ + HeapAsync.defaultIsEqual = function (a, b) { + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + return [2 /*return*/, a === b]; + }); + }); + }; + /** + * Prints a heap. + * @param {HeapAsync} heap Heap to be printed + * @returns {String} + */ + HeapAsync.print = function (heap) { + function deep(i) { + var pi = HeapAsync.getParentIndexOf(i); + return Math.floor(Math.log2(pi + 1)); + } + function repeat(str, times) { + var out = ''; + for (; times > 0; --times) { + out += str; + } + return out; + } + var node = 0; + var lines = []; + var maxLines = deep(heap.length - 1) + 2; + var maxLength = 0; + while (node < heap.length) { + var i = deep(node) + 1; + if (node === 0) { + i = 0; + } + // Text representation + var nodeText = String(heap.get(node)); + if (nodeText.length > maxLength) { + maxLength = nodeText.length; + } + // Add to line + lines[i] = lines[i] || []; + lines[i].push(nodeText); + node += 1; + } + return lines + .map(function (line, i) { + var times = Math.pow(2, maxLines - i) - 1; + return (repeat(' ', Math.floor(times / 2) * maxLength) + + line + .map(function (el) { + // centered + var half = (maxLength - el.length) / 2; + return repeat(' ', Math.ceil(half)) + el + repeat(' ', Math.floor(half)); + }) + .join(repeat(' ', times * maxLength))); + }) + .join('\n'); + }; + /* + Python style + */ + /** + * Converts an array into an array-heap, in place + * @param {Array} arr Array to be modified + * @param {Function} compare Optional compare function + * @return {HeapAsync} For convenience, it returns a Heap instance + */ + HeapAsync.heapify = function (arr, compare) { + return __awaiter(this, void 0, void 0, function () { + var heap; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + heap = new HeapAsync(compare); + heap.heapArray = arr; + return [4 /*yield*/, heap.init()]; + case 1: + _a.sent(); + return [2 /*return*/, heap]; + } + }); + }); + }; + /** + * Extract the peek of an array-heap + * @param {Array} heapArr Array to be modified, should be a heap + * @param {Function} compare Optional compare function + * @return {any} Returns the extracted peek + */ + HeapAsync.heappop = function (heapArr, compare) { + var heap = new HeapAsync(compare); + heap.heapArray = heapArr; + return heap.pop(); + }; + /** + * Pushes a item into an array-heap + * @param {Array} heapArr Array to be modified, should be a heap + * @param {any} item Item to push + * @param {Function} compare Optional compare function + */ + HeapAsync.heappush = function (heapArr, item, compare) { + return __awaiter(this, void 0, void 0, function () { + var heap; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + heap = new HeapAsync(compare); + heap.heapArray = heapArr; + return [4 /*yield*/, heap.push(item)]; + case 1: + _a.sent(); + return [2 /*return*/]; + } + }); + }); + }; + /** + * Push followed by pop, faster + * @param {Array} heapArr Array to be modified, should be a heap + * @param {any} item Item to push + * @param {Function} compare Optional compare function + * @return {any} Returns the extracted peek + */ + HeapAsync.heappushpop = function (heapArr, item, compare) { + var heap = new HeapAsync(compare); + heap.heapArray = heapArr; + return heap.pushpop(item); + }; + /** + * Replace peek with item + * @param {Array} heapArr Array to be modified, should be a heap + * @param {any} item Item as replacement + * @param {Function} compare Optional compare function + * @return {any} Returns the extracted peek + */ + HeapAsync.heapreplace = function (heapArr, item, compare) { + var heap = new HeapAsync(compare); + heap.heapArray = heapArr; + return heap.replace(item); + }; + /** + * Return the `n` most valuable elements of a heap-like Array + * @param {Array} heapArr Array, should be an array-heap + * @param {number} n Max number of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + HeapAsync.heaptop = function (heapArr, n, compare) { + if (n === void 0) { n = 1; } + var heap = new HeapAsync(compare); + heap.heapArray = heapArr; + return heap.top(n); + }; + /** + * Return the `n` least valuable elements of a heap-like Array + * @param {Array} heapArr Array, should be an array-heap + * @param {number} n Max number of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + HeapAsync.heapbottom = function (heapArr, n, compare) { + if (n === void 0) { n = 1; } + var heap = new HeapAsync(compare); + heap.heapArray = heapArr; + return heap.bottom(n); + }; + /** + * Return the `n` most valuable elements of an iterable + * @param {number} n Max number of elements + * @param {Iterable} Iterable Iterable list of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + HeapAsync.nlargest = function (n, iterable, compare) { + return __awaiter(this, void 0, void 0, function () { + var heap; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + heap = new HeapAsync(compare); + heap.heapArray = __spreadArray$1([], __read$1(iterable), false); + return [4 /*yield*/, heap.init()]; + case 1: + _a.sent(); + return [2 /*return*/, heap.top(n)]; + } + }); + }); + }; + /** + * Return the `n` least valuable elements of an iterable + * @param {number} n Max number of elements + * @param {Iterable} Iterable Iterable list of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + HeapAsync.nsmallest = function (n, iterable, compare) { + return __awaiter(this, void 0, void 0, function () { + var heap; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + heap = new HeapAsync(compare); + heap.heapArray = __spreadArray$1([], __read$1(iterable), false); + return [4 /*yield*/, heap.init()]; + case 1: + _a.sent(); + return [2 /*return*/, heap.bottom(n)]; + } + }); + }); + }; + /* + Instance methods + */ + /** + * Adds an element to the heap. Aliases: `offer`. + * Same as: push(element) + * @param {any} element Element to be added + * @return {Boolean} true + */ + HeapAsync.prototype.add = function (element) { + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: return [4 /*yield*/, this._sortNodeUp(this.heapArray.push(element) - 1)]; + case 1: + _a.sent(); + this._applyLimit(); + return [2 /*return*/, true]; + } + }); + }); + }; + /** + * Adds an array of elements to the heap. + * Similar as: push(element, element, ...). + * @param {Array} elements Elements to be added + * @return {Boolean} true + */ + HeapAsync.prototype.addAll = function (elements) { + return __awaiter(this, void 0, void 0, function () { + var i, l; + var _a; + return __generator$1(this, function (_b) { + switch (_b.label) { + case 0: + i = this.length; + (_a = this.heapArray).push.apply(_a, __spreadArray$1([], __read$1(elements), false)); + l = this.length; + _b.label = 1; + case 1: + if (!(i < l)) return [3 /*break*/, 4]; + return [4 /*yield*/, this._sortNodeUp(i)]; + case 2: + _b.sent(); + _b.label = 3; + case 3: + ++i; + return [3 /*break*/, 1]; + case 4: + this._applyLimit(); + return [2 /*return*/, true]; + } + }); + }); + }; + /** + * Return the bottom (lowest value) N elements of the heap. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + HeapAsync.prototype.bottom = function (n) { + if (n === void 0) { n = 1; } + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + if (this.heapArray.length === 0 || n <= 0) { + // Nothing to do + return [2 /*return*/, []]; + } + else if (this.heapArray.length === 1) { + // Just the peek + return [2 /*return*/, [this.heapArray[0]]]; + } + else if (n >= this.heapArray.length) { + // The whole heap + return [2 /*return*/, __spreadArray$1([], __read$1(this.heapArray), false)]; + } + else { + // Some elements + return [2 /*return*/, this._bottomN_push(~~n)]; + } + }); + }); + }; + /** + * Check if the heap is sorted, useful for testing purposes. + * @return {Undefined | Element} Returns an element if something wrong is found, otherwise it's undefined + */ + HeapAsync.prototype.check = function () { + return __awaiter(this, void 0, void 0, function () { + var j, el, children, children_1, children_1_1, ch, e_1_1; + var e_1, _a; + return __generator$1(this, function (_b) { + switch (_b.label) { + case 0: + j = 0; + _b.label = 1; + case 1: + if (!(j < this.heapArray.length)) return [3 /*break*/, 10]; + el = this.heapArray[j]; + children = this.getChildrenOf(j); + _b.label = 2; + case 2: + _b.trys.push([2, 7, 8, 9]); + children_1 = (e_1 = void 0, __values(children)), children_1_1 = children_1.next(); + _b.label = 3; + case 3: + if (!!children_1_1.done) return [3 /*break*/, 6]; + ch = children_1_1.value; + return [4 /*yield*/, this.compare(el, ch)]; + case 4: + if ((_b.sent()) > 0) { + return [2 /*return*/, el]; + } + _b.label = 5; + case 5: + children_1_1 = children_1.next(); + return [3 /*break*/, 3]; + case 6: return [3 /*break*/, 9]; + case 7: + e_1_1 = _b.sent(); + e_1 = { error: e_1_1 }; + return [3 /*break*/, 9]; + case 8: + try { + if (children_1_1 && !children_1_1.done && (_a = children_1.return)) _a.call(children_1); + } + finally { if (e_1) throw e_1.error; } + return [7 /*endfinally*/]; + case 9: + ++j; + return [3 /*break*/, 1]; + case 10: return [2 /*return*/]; + } + }); + }); + }; + /** + * Remove all of the elements from this heap. + */ + HeapAsync.prototype.clear = function () { + this.heapArray = []; + }; + /** + * Clone this heap + * @return {HeapAsync} + */ + HeapAsync.prototype.clone = function () { + var cloned = new HeapAsync(this.comparator()); + cloned.heapArray = this.toArray(); + cloned._limit = this._limit; + return cloned; + }; + /** + * Returns the comparison function. + * @return {Function} + */ + HeapAsync.prototype.comparator = function () { + return this.compare; + }; + /** + * Returns true if this queue contains the specified element. + * @param {any} o Element to be found + * @param {Function} fn Optional comparison function, receives (element, needle) + * @return {Boolean} + */ + HeapAsync.prototype.contains = function (o, fn) { + if (fn === void 0) { fn = HeapAsync.defaultIsEqual; } + return __awaiter(this, void 0, void 0, function () { + var _a, _b, el, e_2_1; + var e_2, _c; + return __generator$1(this, function (_d) { + switch (_d.label) { + case 0: + _d.trys.push([0, 5, 6, 7]); + _a = __values(this.heapArray), _b = _a.next(); + _d.label = 1; + case 1: + if (!!_b.done) return [3 /*break*/, 4]; + el = _b.value; + return [4 /*yield*/, fn(el, o)]; + case 2: + if (_d.sent()) { + return [2 /*return*/, true]; + } + _d.label = 3; + case 3: + _b = _a.next(); + return [3 /*break*/, 1]; + case 4: return [3 /*break*/, 7]; + case 5: + e_2_1 = _d.sent(); + e_2 = { error: e_2_1 }; + return [3 /*break*/, 7]; + case 6: + try { + if (_b && !_b.done && (_c = _a.return)) _c.call(_a); + } + finally { if (e_2) throw e_2.error; } + return [7 /*endfinally*/]; + case 7: return [2 /*return*/, false]; + } + }); + }); + }; + /** + * Initialise a heap, sorting nodes + * @param {Array} array Optional initial state array + */ + HeapAsync.prototype.init = function (array) { + return __awaiter(this, void 0, void 0, function () { + var i; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + if (array) { + this.heapArray = __spreadArray$1([], __read$1(array), false); + } + i = Math.floor(this.heapArray.length); + _a.label = 1; + case 1: + if (!(i >= 0)) return [3 /*break*/, 4]; + return [4 /*yield*/, this._sortNodeDown(i)]; + case 2: + _a.sent(); + _a.label = 3; + case 3: + --i; + return [3 /*break*/, 1]; + case 4: + this._applyLimit(); + return [2 /*return*/]; + } + }); + }); + }; + /** + * Test if the heap has no elements. + * @return {Boolean} True if no elements on the heap + */ + HeapAsync.prototype.isEmpty = function () { + return this.length === 0; + }; + /** + * Get the leafs of the tree (no children nodes) + */ + HeapAsync.prototype.leafs = function () { + if (this.heapArray.length === 0) { + return []; + } + var pi = HeapAsync.getParentIndexOf(this.heapArray.length - 1); + return this.heapArray.slice(pi + 1); + }; + Object.defineProperty(HeapAsync.prototype, "length", { + /** + * Length of the heap. + * @return {Number} + */ + get: function () { + return this.heapArray.length; + }, + enumerable: false, + configurable: true + }); + Object.defineProperty(HeapAsync.prototype, "limit", { + /** + * Get length limit of the heap. + * @return {Number} + */ + get: function () { + return this._limit; + }, + /** + * Set length limit of the heap. + * @return {Number} + */ + set: function (_l) { + this._limit = ~~_l; + this._applyLimit(); + }, + enumerable: false, + configurable: true + }); + /** + * Top node. Aliases: `element`. + * Same as: `top(1)[0]` + * @return {any} Top node + */ + HeapAsync.prototype.peek = function () { + return this.heapArray[0]; + }; + /** + * Extract the top node (root). Aliases: `poll`. + * @return {any} Extracted top node, undefined if empty + */ + HeapAsync.prototype.pop = function () { + return __awaiter(this, void 0, void 0, function () { + var last; + return __generator$1(this, function (_a) { + last = this.heapArray.pop(); + if (this.length > 0 && last !== undefined) { + return [2 /*return*/, this.replace(last)]; + } + return [2 /*return*/, last]; + }); + }); + }; + /** + * Pushes element(s) to the heap. + * @param {...any} elements Elements to insert + * @return {Boolean} True if elements are present + */ + HeapAsync.prototype.push = function () { + var elements = []; + for (var _i = 0; _i < arguments.length; _i++) { + elements[_i] = arguments[_i]; + } + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + if (elements.length < 1) { + return [2 /*return*/, false]; + } + else if (elements.length === 1) { + return [2 /*return*/, this.add(elements[0])]; + } + else { + return [2 /*return*/, this.addAll(elements)]; + } + }); + }); + }; + /** + * Same as push & pop in sequence, but faster + * @param {any} element Element to insert + * @return {any} Extracted top node + */ + HeapAsync.prototype.pushpop = function (element) { + return __awaiter(this, void 0, void 0, function () { + var _a; + return __generator$1(this, function (_b) { + switch (_b.label) { + case 0: return [4 /*yield*/, this.compare(this.heapArray[0], element)]; + case 1: + if (!((_b.sent()) < 0)) return [3 /*break*/, 3]; + _a = __read$1([this.heapArray[0], element], 2), element = _a[0], this.heapArray[0] = _a[1]; + return [4 /*yield*/, this._sortNodeDown(0)]; + case 2: + _b.sent(); + _b.label = 3; + case 3: return [2 /*return*/, element]; + } + }); + }); + }; + /** + * Remove an element from the heap. + * @param {any} o Element to be found + * @param {Function} fn Optional function to compare + * @return {Boolean} True if the heap was modified + */ + HeapAsync.prototype.remove = function (o, fn) { + if (fn === void 0) { fn = HeapAsync.defaultIsEqual; } + return __awaiter(this, void 0, void 0, function () { + var idx, i; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + if (!(this.length > 0)) return [3 /*break*/, 13]; + if (!(o === undefined)) return [3 /*break*/, 2]; + return [4 /*yield*/, this.pop()]; + case 1: + _a.sent(); + return [2 /*return*/, true]; + case 2: + idx = -1; + i = 0; + _a.label = 3; + case 3: + if (!(i < this.heapArray.length)) return [3 /*break*/, 6]; + return [4 /*yield*/, fn(this.heapArray[i], o)]; + case 4: + if (_a.sent()) { + idx = i; + return [3 /*break*/, 6]; + } + _a.label = 5; + case 5: + ++i; + return [3 /*break*/, 3]; + case 6: + if (!(idx >= 0)) return [3 /*break*/, 13]; + if (!(idx === 0)) return [3 /*break*/, 8]; + return [4 /*yield*/, this.pop()]; + case 7: + _a.sent(); + return [3 /*break*/, 12]; + case 8: + if (!(idx === this.length - 1)) return [3 /*break*/, 9]; + this.heapArray.pop(); + return [3 /*break*/, 12]; + case 9: + this.heapArray.splice(idx, 1, this.heapArray.pop()); + return [4 /*yield*/, this._sortNodeUp(idx)]; + case 10: + _a.sent(); + return [4 /*yield*/, this._sortNodeDown(idx)]; + case 11: + _a.sent(); + _a.label = 12; + case 12: return [2 /*return*/, true]; + case 13: return [2 /*return*/, false]; + } + }); + }); + }; + /** + * Pop the current peek value, and add the new item. + * @param {any} element Element to replace peek + * @return {any} Old peek + */ + HeapAsync.prototype.replace = function (element) { + return __awaiter(this, void 0, void 0, function () { + var peek; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + peek = this.heapArray[0]; + this.heapArray[0] = element; + return [4 /*yield*/, this._sortNodeDown(0)]; + case 1: + _a.sent(); + return [2 /*return*/, peek]; + } + }); + }); + }; + /** + * Size of the heap + * @return {Number} + */ + HeapAsync.prototype.size = function () { + return this.length; + }; + /** + * Return the top (highest value) N elements of the heap. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + HeapAsync.prototype.top = function (n) { + if (n === void 0) { n = 1; } + return __awaiter(this, void 0, void 0, function () { + return __generator$1(this, function (_a) { + if (this.heapArray.length === 0 || n <= 0) { + // Nothing to do + return [2 /*return*/, []]; + } + else if (this.heapArray.length === 1 || n === 1) { + // Just the peek + return [2 /*return*/, [this.heapArray[0]]]; + } + else if (n >= this.heapArray.length) { + // The whole peek + return [2 /*return*/, __spreadArray$1([], __read$1(this.heapArray), false)]; + } + else { + // Some elements + return [2 /*return*/, this._topN_push(~~n)]; + } + }); + }); + }; + /** + * Clone the heap's internal array + * @return {Array} + */ + HeapAsync.prototype.toArray = function () { + return __spreadArray$1([], __read$1(this.heapArray), false); + }; + /** + * String output, call to Array.prototype.toString() + * @return {String} + */ + HeapAsync.prototype.toString = function () { + return this.heapArray.toString(); + }; + /** + * Get the element at the given index. + * @param {Number} i Index to get + * @return {any} Element at that index + */ + HeapAsync.prototype.get = function (i) { + return this.heapArray[i]; + }; + /** + * Get the elements of these node's children + * @param {Number} idx Node index + * @return {Array(any)} Children elements + */ + HeapAsync.prototype.getChildrenOf = function (idx) { + var _this = this; + return HeapAsync.getChildrenIndexOf(idx) + .map(function (i) { return _this.heapArray[i]; }) + .filter(function (e) { return e !== undefined; }); + }; + /** + * Get the element of this node's parent + * @param {Number} idx Node index + * @return {any} Parent element + */ + HeapAsync.prototype.getParentOf = function (idx) { + var pi = HeapAsync.getParentIndexOf(idx); + return this.heapArray[pi]; + }; + /** + * Iterator interface + */ + HeapAsync.prototype[Symbol.iterator] = function () { + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + if (!this.length) return [3 /*break*/, 2]; + return [4 /*yield*/, this.pop()]; + case 1: + _a.sent(); + return [3 /*break*/, 0]; + case 2: return [2 /*return*/]; + } + }); + }; + /** + * Returns an iterator. To comply with Java interface. + */ + HeapAsync.prototype.iterator = function () { + return this; + }; + /** + * Limit heap size if needed + */ + HeapAsync.prototype._applyLimit = function () { + if (this._limit && this._limit < this.heapArray.length) { + var rm = this.heapArray.length - this._limit; + // It's much faster than splice + while (rm) { + this.heapArray.pop(); + --rm; + } + } + }; + /** + * Return the bottom (lowest value) N elements of the heap, without corner cases, unsorted + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + HeapAsync.prototype._bottomN_push = function (n) { + return __awaiter(this, void 0, void 0, function () { + var bottomHeap, startAt, parentStartAt, indices, i, arr, i; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + bottomHeap = new HeapAsync(this.compare); + bottomHeap.limit = n; + bottomHeap.heapArray = this.heapArray.slice(-n); + return [4 /*yield*/, bottomHeap.init()]; + case 1: + _a.sent(); + startAt = this.heapArray.length - 1 - n; + parentStartAt = HeapAsync.getParentIndexOf(startAt); + indices = []; + for (i = startAt; i > parentStartAt; --i) { + indices.push(i); + } + arr = this.heapArray; + _a.label = 2; + case 2: + if (!indices.length) return [3 /*break*/, 6]; + i = indices.shift(); + return [4 /*yield*/, this.compare(arr[i], bottomHeap.peek())]; + case 3: + if (!((_a.sent()) > 0)) return [3 /*break*/, 5]; + return [4 /*yield*/, bottomHeap.replace(arr[i])]; + case 4: + _a.sent(); + if (i % 2) { + indices.push(HeapAsync.getParentIndexOf(i)); + } + _a.label = 5; + case 5: return [3 /*break*/, 2]; + case 6: return [2 /*return*/, bottomHeap.toArray()]; + } + }); + }); + }; + /** + * Move a node to a new index, switching places + * @param {Number} j First node index + * @param {Number} k Another node index + */ + HeapAsync.prototype._moveNode = function (j, k) { + var _a; + _a = __read$1([this.heapArray[k], this.heapArray[j]], 2), this.heapArray[j] = _a[0], this.heapArray[k] = _a[1]; + }; + /** + * Move a node down the tree (to the leaves) to find a place where the heap is sorted. + * @param {Number} i Index of the node + */ + HeapAsync.prototype._sortNodeDown = function (i) { + return __awaiter(this, void 0, void 0, function () { + var moveIt, self, getPotentialParent, childrenIdx, bestChildIndex, j, bestChild, _a; + var _this = this; + return __generator$1(this, function (_b) { + switch (_b.label) { + case 0: + moveIt = i < this.heapArray.length - 1; + self = this.heapArray[i]; + getPotentialParent = function (best, j) { return __awaiter(_this, void 0, void 0, function () { + var _a; + return __generator$1(this, function (_b) { + switch (_b.label) { + case 0: + _a = this.heapArray.length > j; + if (!_a) return [3 /*break*/, 2]; + return [4 /*yield*/, this.compare(this.heapArray[j], this.heapArray[best])]; + case 1: + _a = (_b.sent()) < 0; + _b.label = 2; + case 2: + if (_a) { + best = j; + } + return [2 /*return*/, best]; + } + }); + }); }; + _b.label = 1; + case 1: + if (!moveIt) return [3 /*break*/, 8]; + childrenIdx = HeapAsync.getChildrenIndexOf(i); + bestChildIndex = childrenIdx[0]; + j = 1; + _b.label = 2; + case 2: + if (!(j < childrenIdx.length)) return [3 /*break*/, 5]; + return [4 /*yield*/, getPotentialParent(bestChildIndex, childrenIdx[j])]; + case 3: + bestChildIndex = _b.sent(); + _b.label = 4; + case 4: + ++j; + return [3 /*break*/, 2]; + case 5: + bestChild = this.heapArray[bestChildIndex]; + _a = typeof bestChild !== 'undefined'; + if (!_a) return [3 /*break*/, 7]; + return [4 /*yield*/, this.compare(self, bestChild)]; + case 6: + _a = (_b.sent()) > 0; + _b.label = 7; + case 7: + if (_a) { + this._moveNode(i, bestChildIndex); + i = bestChildIndex; + } + else { + moveIt = false; + } + return [3 /*break*/, 1]; + case 8: return [2 /*return*/]; + } + }); + }); + }; + /** + * Move a node up the tree (to the root) to find a place where the heap is sorted. + * @param {Number} i Index of the node + */ + HeapAsync.prototype._sortNodeUp = function (i) { + return __awaiter(this, void 0, void 0, function () { + var moveIt, pi, _a; + return __generator$1(this, function (_b) { + switch (_b.label) { + case 0: + moveIt = i > 0; + _b.label = 1; + case 1: + if (!moveIt) return [3 /*break*/, 4]; + pi = HeapAsync.getParentIndexOf(i); + _a = pi >= 0; + if (!_a) return [3 /*break*/, 3]; + return [4 /*yield*/, this.compare(this.heapArray[pi], this.heapArray[i])]; + case 2: + _a = (_b.sent()) > 0; + _b.label = 3; + case 3: + if (_a) { + this._moveNode(i, pi); + i = pi; + } + else { + moveIt = false; + } + return [3 /*break*/, 1]; + case 4: return [2 /*return*/]; + } + }); + }); + }; + /** + * Return the top (highest value) N elements of the heap, without corner cases, unsorted + * Implementation: push. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + HeapAsync.prototype._topN_push = function (n) { + return __awaiter(this, void 0, void 0, function () { + var topHeap, indices, arr, i; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + topHeap = new HeapAsync(this._invertedCompare); + topHeap.limit = n; + indices = [0]; + arr = this.heapArray; + _a.label = 1; + case 1: + if (!indices.length) return [3 /*break*/, 7]; + i = indices.shift(); + if (!(i < arr.length)) return [3 /*break*/, 6]; + if (!(topHeap.length < n)) return [3 /*break*/, 3]; + return [4 /*yield*/, topHeap.push(arr[i])]; + case 2: + _a.sent(); + indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i)), false)); + return [3 /*break*/, 6]; + case 3: return [4 /*yield*/, this.compare(arr[i], topHeap.peek())]; + case 4: + if (!((_a.sent()) < 0)) return [3 /*break*/, 6]; + return [4 /*yield*/, topHeap.replace(arr[i])]; + case 5: + _a.sent(); + indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i)), false)); + _a.label = 6; + case 6: return [3 /*break*/, 1]; + case 7: return [2 /*return*/, topHeap.toArray()]; + } + }); + }); + }; + /** + * Return the top (highest value) N elements of the heap, without corner cases, unsorted + * Implementation: init + push. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + HeapAsync.prototype._topN_fill = function (n) { + return __awaiter(this, void 0, void 0, function () { + var heapArray, topHeap, branch, indices, i, i; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + heapArray = this.heapArray; + topHeap = new HeapAsync(this._invertedCompare); + topHeap.limit = n; + topHeap.heapArray = heapArray.slice(0, n); + return [4 /*yield*/, topHeap.init()]; + case 1: + _a.sent(); + branch = HeapAsync.getParentIndexOf(n - 1) + 1; + indices = []; + for (i = branch; i < n; ++i) { + indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i).filter(function (l) { return l < heapArray.length; })), false)); + } + if ((n - 1) % 2) { + indices.push(n); + } + _a.label = 2; + case 2: + if (!indices.length) return [3 /*break*/, 6]; + i = indices.shift(); + if (!(i < heapArray.length)) return [3 /*break*/, 5]; + return [4 /*yield*/, this.compare(heapArray[i], topHeap.peek())]; + case 3: + if (!((_a.sent()) < 0)) return [3 /*break*/, 5]; + return [4 /*yield*/, topHeap.replace(heapArray[i])]; + case 4: + _a.sent(); + indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i)), false)); + _a.label = 5; + case 5: return [3 /*break*/, 2]; + case 6: return [2 /*return*/, topHeap.toArray()]; + } + }); + }); + }; + /** + * Return the top (highest value) N elements of the heap, without corner cases, unsorted + * Implementation: heap. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + HeapAsync.prototype._topN_heap = function (n) { + return __awaiter(this, void 0, void 0, function () { + var topHeap, result, i, _a, _b; + return __generator$1(this, function (_c) { + switch (_c.label) { + case 0: + topHeap = this.clone(); + result = []; + i = 0; + _c.label = 1; + case 1: + if (!(i < n)) return [3 /*break*/, 4]; + _b = (_a = result).push; + return [4 /*yield*/, topHeap.pop()]; + case 2: + _b.apply(_a, [(_c.sent())]); + _c.label = 3; + case 3: + ++i; + return [3 /*break*/, 1]; + case 4: return [2 /*return*/, result]; + } + }); + }); + }; + /** + * Return index of the top element + * @param list + */ + HeapAsync.prototype._topIdxOf = function (list) { + return __awaiter(this, void 0, void 0, function () { + var idx, top, i, comp; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + if (!list.length) { + return [2 /*return*/, -1]; + } + idx = 0; + top = list[idx]; + i = 1; + _a.label = 1; + case 1: + if (!(i < list.length)) return [3 /*break*/, 4]; + return [4 /*yield*/, this.compare(list[i], top)]; + case 2: + comp = _a.sent(); + if (comp < 0) { + idx = i; + top = list[i]; + } + _a.label = 3; + case 3: + ++i; + return [3 /*break*/, 1]; + case 4: return [2 /*return*/, idx]; + } + }); + }); + }; + /** + * Return the top element + * @param list + */ + HeapAsync.prototype._topOf = function () { + var list = []; + for (var _i = 0; _i < arguments.length; _i++) { + list[_i] = arguments[_i]; + } + return __awaiter(this, void 0, void 0, function () { + var heap; + return __generator$1(this, function (_a) { + switch (_a.label) { + case 0: + heap = new HeapAsync(this.compare); + return [4 /*yield*/, heap.init(list)]; + case 1: + _a.sent(); + return [2 /*return*/, heap.peek()]; + } + }); + }); + }; + return HeapAsync; +}()); + +var __generator = (undefined && undefined.__generator) || function (thisArg, body) { + var _ = { label: 0, sent: function() { if (t[0] & 1) throw t[1]; return t[1]; }, trys: [], ops: [] }, f, y, t, g; + return g = { next: verb(0), "throw": verb(1), "return": verb(2) }, typeof Symbol === "function" && (g[Symbol.iterator] = function() { return this; }), g; + function verb(n) { return function (v) { return step([n, v]); }; } + function step(op) { + if (f) throw new TypeError("Generator is already executing."); + while (g && (g = 0, op[0] && (_ = 0)), _) try { + if (f = 1, y && (t = op[0] & 2 ? y["return"] : op[0] ? y["throw"] || ((t = y["return"]) && t.call(y), 0) : y.next) && !(t = t.call(y, op[1])).done) return t; + if (y = 0, t) op = [op[0] & 2, t.value]; + switch (op[0]) { + case 0: case 1: t = op; break; + case 4: _.label++; return { value: op[1], done: false }; + case 5: _.label++; y = op[1]; op = [0]; continue; + case 7: op = _.ops.pop(); _.trys.pop(); continue; + default: + if (!(t = _.trys, t = t.length > 0 && t[t.length - 1]) && (op[0] === 6 || op[0] === 2)) { _ = 0; continue; } + if (op[0] === 3 && (!t || (op[1] > t[0] && op[1] < t[3]))) { _.label = op[1]; break; } + if (op[0] === 6 && _.label < t[1]) { _.label = t[1]; t = op; break; } + if (t && _.label < t[2]) { _.label = t[2]; _.ops.push(op); break; } + if (t[2]) _.ops.pop(); + _.trys.pop(); continue; + } + op = body.call(thisArg, _); + } catch (e) { op = [6, e]; y = 0; } finally { f = t = 0; } + if (op[0] & 5) throw op[1]; return { value: op[0] ? op[1] : void 0, done: true }; + } +}; +var __read = (undefined && undefined.__read) || function (o, n) { + var m = typeof Symbol === "function" && o[Symbol.iterator]; + if (!m) return o; + var i = m.call(o), r, ar = [], e; + try { + while ((n === void 0 || n-- > 0) && !(r = i.next()).done) ar.push(r.value); + } + catch (error) { e = { error: error }; } + finally { + try { + if (r && !r.done && (m = i["return"])) m.call(i); + } + finally { if (e) throw e.error; } + } + return ar; +}; +var __spreadArray = (undefined && undefined.__spreadArray) || function (to, from, pack) { + if (pack || arguments.length === 2) for (var i = 0, l = from.length, ar; i < l; i++) { + if (ar || !(i in from)) { + if (!ar) ar = Array.prototype.slice.call(from, 0, i); + ar[i] = from[i]; + } + } + return to.concat(ar || Array.prototype.slice.call(from)); +}; +var toInt = function (n) { return ~~n; }; +/** + * Heap + * @type {Class} + */ +var Heap = /** @class */ (function () { + /** + * Heap instance constructor. + * @param {Function} compare Optional comparison function, defaults to Heap.minComparator + */ + function Heap(compare) { + if (compare === void 0) { compare = Heap.minComparator; } + var _this = this; + this.compare = compare; + this.heapArray = []; + this._limit = 0; + /** + * Alias of {@link add} + * @see add + */ + this.offer = this.add; + /** + * Alias of {@link peek} + * @see peek + */ + this.element = this.peek; + /** + * Alias of {@link pop} + * @see pop + */ + this.poll = this.pop; + /** + * Alias of {@link clear} + * @see clear + */ + this.removeAll = this.clear; + /** + * Returns the inverse to the comparison function. + * @return {Function} + */ + this._invertedCompare = function (a, b) { + return -1 * _this.compare(a, b); + }; + } + /* + Static methods + */ + /** + * Gets children indices for given index. + * @param {Number} idx Parent index + * @return {Array(Number)} Array of children indices + */ + Heap.getChildrenIndexOf = function (idx) { + return [idx * 2 + 1, idx * 2 + 2]; + }; + /** + * Gets parent index for given index. + * @param {Number} idx Children index + * @return {Number | undefined} Parent index, -1 if idx is 0 + */ + Heap.getParentIndexOf = function (idx) { + if (idx <= 0) { + return -1; + } + var whichChildren = idx % 2 ? 1 : 2; + return Math.floor((idx - whichChildren) / 2); + }; + /** + * Gets sibling index for given index. + * @param {Number} idx Children index + * @return {Number | undefined} Sibling index, -1 if idx is 0 + */ + Heap.getSiblingIndexOf = function (idx) { + if (idx <= 0) { + return -1; + } + var whichChildren = idx % 2 ? 1 : -1; + return idx + whichChildren; + }; + /** + * Min heap comparison function, default. + * @param {any} a First element + * @param {any} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + Heap.minComparator = function (a, b) { + if (a > b) { + return 1; + } + else if (a < b) { + return -1; + } + else { + return 0; + } + }; + /** + * Max heap comparison function. + * @param {any} a First element + * @param {any} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + Heap.maxComparator = function (a, b) { + if (b > a) { + return 1; + } + else if (b < a) { + return -1; + } + else { + return 0; + } + }; + /** + * Min number heap comparison function, default. + * @param {Number} a First element + * @param {Number} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + Heap.minComparatorNumber = function (a, b) { + return a - b; + }; + /** + * Max number heap comparison function. + * @param {Number} a First element + * @param {Number} b Second element + * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up + */ + Heap.maxComparatorNumber = function (a, b) { + return b - a; + }; + /** + * Default equality function. + * @param {any} a First element + * @param {any} b Second element + * @return {Boolean} True if equal, false otherwise + */ + Heap.defaultIsEqual = function (a, b) { + return a === b; + }; + /** + * Prints a heap. + * @param {Heap} heap Heap to be printed + * @returns {String} + */ + Heap.print = function (heap) { + function deep(i) { + var pi = Heap.getParentIndexOf(i); + return Math.floor(Math.log2(pi + 1)); + } + function repeat(str, times) { + var out = ''; + for (; times > 0; --times) { + out += str; + } + return out; + } + var node = 0; + var lines = []; + var maxLines = deep(heap.length - 1) + 2; + var maxLength = 0; + while (node < heap.length) { + var i = deep(node) + 1; + if (node === 0) { + i = 0; + } + // Text representation + var nodeText = String(heap.get(node)); + if (nodeText.length > maxLength) { + maxLength = nodeText.length; + } + // Add to line + lines[i] = lines[i] || []; + lines[i].push(nodeText); + node += 1; + } + return lines + .map(function (line, i) { + var times = Math.pow(2, maxLines - i) - 1; + return (repeat(' ', Math.floor(times / 2) * maxLength) + + line + .map(function (el) { + // centered + var half = (maxLength - el.length) / 2; + return repeat(' ', Math.ceil(half)) + el + repeat(' ', Math.floor(half)); + }) + .join(repeat(' ', times * maxLength))); + }) + .join('\n'); + }; + /* + Python style + */ + /** + * Converts an array into an array-heap, in place + * @param {Array} arr Array to be modified + * @param {Function} compare Optional compare function + * @return {Heap} For convenience, it returns a Heap instance + */ + Heap.heapify = function (arr, compare) { + var heap = new Heap(compare); + heap.heapArray = arr; + heap.init(); + return heap; + }; + /** + * Extract the peek of an array-heap + * @param {Array} heapArr Array to be modified, should be a heap + * @param {Function} compare Optional compare function + * @return {any} Returns the extracted peek + */ + Heap.heappop = function (heapArr, compare) { + var heap = new Heap(compare); + heap.heapArray = heapArr; + return heap.pop(); + }; + /** + * Pushes a item into an array-heap + * @param {Array} heapArr Array to be modified, should be a heap + * @param {any} item Item to push + * @param {Function} compare Optional compare function + */ + Heap.heappush = function (heapArr, item, compare) { + var heap = new Heap(compare); + heap.heapArray = heapArr; + heap.push(item); + }; + /** + * Push followed by pop, faster + * @param {Array} heapArr Array to be modified, should be a heap + * @param {any} item Item to push + * @param {Function} compare Optional compare function + * @return {any} Returns the extracted peek + */ + Heap.heappushpop = function (heapArr, item, compare) { + var heap = new Heap(compare); + heap.heapArray = heapArr; + return heap.pushpop(item); + }; + /** + * Replace peek with item + * @param {Array} heapArr Array to be modified, should be a heap + * @param {any} item Item as replacement + * @param {Function} compare Optional compare function + * @return {any} Returns the extracted peek + */ + Heap.heapreplace = function (heapArr, item, compare) { + var heap = new Heap(compare); + heap.heapArray = heapArr; + return heap.replace(item); + }; + /** + * Return the `n` most valuable elements of a heap-like Array + * @param {Array} heapArr Array, should be an array-heap + * @param {number} n Max number of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + Heap.heaptop = function (heapArr, n, compare) { + if (n === void 0) { n = 1; } + var heap = new Heap(compare); + heap.heapArray = heapArr; + return heap.top(n); + }; + /** + * Return the `n` least valuable elements of a heap-like Array + * @param {Array} heapArr Array, should be an array-heap + * @param {number} n Max number of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + Heap.heapbottom = function (heapArr, n, compare) { + if (n === void 0) { n = 1; } + var heap = new Heap(compare); + heap.heapArray = heapArr; + return heap.bottom(n); + }; + /** + * Return the `n` most valuable elements of an iterable + * @param {number} n Max number of elements + * @param {Iterable} Iterable Iterable list of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + Heap.nlargest = function (n, iterable, compare) { + var heap = new Heap(compare); + heap.heapArray = __spreadArray([], __read(iterable), false); + heap.init(); + return heap.top(n); + }; + /** + * Return the `n` least valuable elements of an iterable + * @param {number} n Max number of elements + * @param {Iterable} Iterable Iterable list of elements + * @param {Function} compare Optional compare function + * @return {any} Elements + */ + Heap.nsmallest = function (n, iterable, compare) { + var heap = new Heap(compare); + heap.heapArray = __spreadArray([], __read(iterable), false); + heap.init(); + return heap.bottom(n); + }; + /* + Instance methods + */ + /** + * Adds an element to the heap. Aliases: {@link offer}. + * Same as: {@link push}(element). + * @param {any} element Element to be added + * @return {Boolean} true + */ + Heap.prototype.add = function (element) { + this._sortNodeUp(this.heapArray.push(element) - 1); + this._applyLimit(); + return true; + }; + /** + * Adds an array of elements to the heap. + * Similar as: {@link push}(element, element, ...). + * @param {Array} elements Elements to be added + * @return {Boolean} true + */ + Heap.prototype.addAll = function (elements) { + var _a; + var i = this.length; + (_a = this.heapArray).push.apply(_a, __spreadArray([], __read(elements), false)); + for (var l = this.length; i < l; ++i) { + this._sortNodeUp(i); + } + this._applyLimit(); + return true; + }; + /** + * Return the bottom (lowest value) N elements of the heap. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + Heap.prototype.bottom = function (n) { + if (n === void 0) { n = 1; } + if (this.heapArray.length === 0 || n <= 0) { + // Nothing to do + return []; + } + else if (this.heapArray.length === 1) { + // Just the peek + return [this.heapArray[0]]; + } + else if (n >= this.heapArray.length) { + // The whole heap + return __spreadArray([], __read(this.heapArray), false); + } + else { + // Some elements + return this._bottomN_push(~~n); + } + }; + /** + * Check if the heap is sorted, useful for testing purposes. + * @return {Undefined | Element} Returns an element if something wrong is found, otherwise it's undefined + */ + Heap.prototype.check = function () { + var _this = this; + return this.heapArray.find(function (el, j) { return !!_this.getChildrenOf(j).find(function (ch) { return _this.compare(el, ch) > 0; }); }); + }; + /** + * Remove all of the elements from this heap. + */ + Heap.prototype.clear = function () { + this.heapArray = []; + }; + /** + * Clone this heap + * @return {Heap} + */ + Heap.prototype.clone = function () { + var cloned = new Heap(this.comparator()); + cloned.heapArray = this.toArray(); + cloned._limit = this._limit; + return cloned; + }; + /** + * Returns the comparison function. + * @return {Function} + */ + Heap.prototype.comparator = function () { + return this.compare; + }; + /** + * Returns true if this queue contains the specified element. + * @param {any} o Element to be found + * @param {Function} callbackFn Optional comparison function, receives (element, needle) + * @return {Boolean} + */ + Heap.prototype.contains = function (o, callbackFn) { + if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } + return this.indexOf(o, callbackFn) !== -1; + }; + /** + * Initialize a heap, sorting nodes + * @param {Array} array Optional initial state array + */ + Heap.prototype.init = function (array) { + if (array) { + this.heapArray = __spreadArray([], __read(array), false); + } + for (var i = Math.floor(this.heapArray.length); i >= 0; --i) { + this._sortNodeDown(i); + } + this._applyLimit(); + }; + /** + * Test if the heap has no elements. + * @return {Boolean} True if no elements on the heap + */ + Heap.prototype.isEmpty = function () { + return this.length === 0; + }; + /** + * Get the index of the first occurrence of the element in the heap (using the comparator). + * @param {any} element Element to be found + * @param {Function} callbackFn Optional comparison function, receives (element, needle) + * @return {Number} Index or -1 if not found + */ + Heap.prototype.indexOf = function (element, callbackFn) { + if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } + if (this.heapArray.length === 0) { + return -1; + } + var indexes = []; + var currentIndex = 0; + while (currentIndex < this.heapArray.length) { + var currentElement = this.heapArray[currentIndex]; + if (callbackFn(currentElement, element)) { + return currentIndex; + } + else if (this.compare(currentElement, element) <= 0) { + indexes.push.apply(indexes, __spreadArray([], __read(Heap.getChildrenIndexOf(currentIndex)), false)); + } + currentIndex = indexes.shift() || this.heapArray.length; + } + return -1; + }; + /** + * Get the indexes of the every occurrence of the element in the heap (using the comparator). + * @param {any} element Element to be found + * @param {Function} callbackFn Optional comparison function, receives (element, needle) + * @return {Array} Array of indexes or empty array if not found + */ + Heap.prototype.indexOfEvery = function (element, callbackFn) { + if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } + if (this.heapArray.length === 0) { + return []; + } + var indexes = []; + var foundIndexes = []; + var currentIndex = 0; + while (currentIndex < this.heapArray.length) { + var currentElement = this.heapArray[currentIndex]; + if (callbackFn(currentElement, element)) { + foundIndexes.push(currentIndex); + indexes.push.apply(indexes, __spreadArray([], __read(Heap.getChildrenIndexOf(currentIndex)), false)); + } + else if (this.compare(currentElement, element) <= 0) { + indexes.push.apply(indexes, __spreadArray([], __read(Heap.getChildrenIndexOf(currentIndex)), false)); + } + currentIndex = indexes.shift() || this.heapArray.length; + } + return foundIndexes; + }; + /** + * Get the leafs of the tree (no children nodes). + * See also: {@link getChildrenOf} and {@link bottom}. + * @return {Array} + * @see getChildrenOf + * @see bottom + */ + Heap.prototype.leafs = function () { + if (this.heapArray.length === 0) { + return []; + } + var pi = Heap.getParentIndexOf(this.heapArray.length - 1); + return this.heapArray.slice(pi + 1); + }; + Object.defineProperty(Heap.prototype, "length", { + /** + * Length of the heap. Aliases: {@link size}. + * @return {Number} + * @see size + */ + get: function () { + return this.heapArray.length; + }, + enumerable: false, + configurable: true + }); + Object.defineProperty(Heap.prototype, "limit", { + /** + * Get length limit of the heap. + * Use {@link setLimit} or {@link limit} to set the limit. + * @return {Number} + * @see setLimit + */ + get: function () { + return this._limit; + }, + /** + * Set length limit of the heap. Same as using {@link setLimit}. + * @description If the heap is longer than the limit, the needed amount of leafs are removed. + * @param {Number} _l Limit, defaults to 0 (no limit). Negative, Infinity, or NaN values set the limit to 0. + * @see setLimit + */ + set: function (_l) { + if (_l < 0 || isNaN(_l)) { + // NaN, negative, and Infinity are treated as 0 + this._limit = 0; + } + else { + // truncating a floating-point number to an integer + this._limit = ~~_l; + } + this._applyLimit(); + }, + enumerable: false, + configurable: true + }); + /** + * Set length limit of the heap. + * Same as assigning to {@link limit} but returns NaN if the value was invalid. + * @param {Number} _l Limit. Negative, Infinity, or NaN values set the limit to 0. + * @return {Number} The limit or NaN if the value was negative, or NaN. + * @see limit + */ + Heap.prototype.setLimit = function (_l) { + this.limit = _l; + if (_l < 0 || isNaN(_l)) { + return NaN; + } + else { + return this._limit; + } + }; + /** + * Top node. Aliases: {@link element}. + * Same as: {@link top}(1)[0]. + * @return {any} Top node + * @see top + */ + Heap.prototype.peek = function () { + return this.heapArray[0]; + }; + /** + * Extract the top node (root). Aliases: {@link poll}. + * @return {any} Extracted top node, undefined if empty + */ + Heap.prototype.pop = function () { + var last = this.heapArray.pop(); + if (this.length > 0 && last !== undefined) { + return this.replace(last); + } + return last; + }; + /** + * Pushes element(s) to the heap. + * See also: {@link add} and {@link addAll}. + * @param {...any} elements Elements to insert + * @return {Boolean} True if elements are present + */ + Heap.prototype.push = function () { + var elements = []; + for (var _i = 0; _i < arguments.length; _i++) { + elements[_i] = arguments[_i]; + } + if (elements.length < 1) { + return false; + } + else if (elements.length === 1) { + return this.add(elements[0]); + } + else { + return this.addAll(elements); + } + }; + /** + * Same as push & pop in sequence, but faster + * @param {any} element Element to insert + * @return {any} Extracted top node + */ + Heap.prototype.pushpop = function (element) { + var _a; + if (this.compare(this.heapArray[0], element) < 0) { + _a = __read([this.heapArray[0], element], 2), element = _a[0], this.heapArray[0] = _a[1]; + this._sortNodeDown(0); + } + return element; + }; + /** + * Remove the first occurrence of an element from the heap. + * @param {any} o Element to be found + * @param {Function} callbackFn Optional equality function, receives (element, needle) + * @return {Boolean} True if the heap was modified + */ + Heap.prototype.remove = function (o, callbackFn) { + if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } + if (this.length > 0) { + if (o === undefined) { + this.pop(); + return true; + } + else { + var idx = this.indexOf(o, callbackFn); + if (idx >= 0) { + if (idx === 0) { + this.pop(); + } + else if (idx === this.length - 1) { + this.heapArray.pop(); + } + else { + this.heapArray.splice(idx, 1, this.heapArray.pop()); + this._sortNodeUp(idx); + this._sortNodeDown(idx); + } + return true; + } + } + } + return false; + }; + /** + * Pop the current peek value, and add the new item. + * @param {any} element Element to replace peek + * @return {any} Old peek + */ + Heap.prototype.replace = function (element) { + var peek = this.heapArray[0]; + this.heapArray[0] = element; + this._sortNodeDown(0); + return peek; + }; + /** + * Size of the heap + * @return {Number} + */ + Heap.prototype.size = function () { + return this.length; + }; + /** + * Return the top (highest value) N elements of the heap. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + Heap.prototype.top = function (n) { + if (n === void 0) { n = 1; } + if (this.heapArray.length === 0 || n <= 0) { + // Nothing to do + return []; + } + else if (this.heapArray.length === 1 || n === 1) { + // Just the peek + return [this.heapArray[0]]; + } + else if (n >= this.heapArray.length) { + // The whole peek + return __spreadArray([], __read(this.heapArray), false); + } + else { + // Some elements + return this._topN_push(~~n); + } + }; + /** + * Clone the heap's internal array + * @return {Array} + */ + Heap.prototype.toArray = function () { + return __spreadArray([], __read(this.heapArray), false); + }; + /** + * String output, call to Array.prototype.toString() + * @return {String} + */ + Heap.prototype.toString = function () { + return this.heapArray.toString(); + }; + /** + * Get the element at the given index. + * @param {Number} i Index to get + * @return {any} Element at that index + */ + Heap.prototype.get = function (i) { + return this.heapArray[i]; + }; + /** + * Get the elements of these node's children + * @param {Number} idx Node index + * @return {Array(any)} Children elements + */ + Heap.prototype.getChildrenOf = function (idx) { + var _this = this; + return Heap.getChildrenIndexOf(idx) + .map(function (i) { return _this.heapArray[i]; }) + .filter(function (e) { return e !== undefined; }); + }; + /** + * Get the element of this node's parent + * @param {Number} idx Node index + * @return {any} Parent element + */ + Heap.prototype.getParentOf = function (idx) { + var pi = Heap.getParentIndexOf(idx); + return this.heapArray[pi]; + }; + /** + * Iterator interface + */ + Heap.prototype[Symbol.iterator] = function () { + return __generator(this, function (_a) { + switch (_a.label) { + case 0: + if (!this.length) return [3 /*break*/, 2]; + return [4 /*yield*/, this.pop()]; + case 1: + _a.sent(); + return [3 /*break*/, 0]; + case 2: return [2 /*return*/]; + } + }); + }; + /** + * Returns an iterator. To comply with Java interface. + */ + Heap.prototype.iterator = function () { + return this.toArray(); + }; + /** + * Limit heap size if needed + */ + Heap.prototype._applyLimit = function () { + if (this._limit > 0 && this._limit < this.heapArray.length) { + var rm = this.heapArray.length - this._limit; + // It's much faster than splice + while (rm) { + this.heapArray.pop(); + --rm; + } + } + }; + /** + * Return the bottom (lowest value) N elements of the heap, without corner cases, unsorted + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + Heap.prototype._bottomN_push = function (n) { + // Use an inverted heap + var bottomHeap = new Heap(this.compare); + bottomHeap.limit = n; + bottomHeap.heapArray = this.heapArray.slice(-n); + bottomHeap.init(); + var startAt = this.heapArray.length - 1 - n; + var parentStartAt = Heap.getParentIndexOf(startAt); + var indices = []; + for (var i = startAt; i > parentStartAt; --i) { + indices.push(i); + } + var arr = this.heapArray; + while (indices.length) { + var i = indices.shift(); + if (this.compare(arr[i], bottomHeap.peek()) > 0) { + bottomHeap.replace(arr[i]); + if (i % 2) { + indices.push(Heap.getParentIndexOf(i)); + } + } + } + return bottomHeap.toArray(); + }; + /** + * Move a node to a new index, switching places + * @param {Number} j First node index + * @param {Number} k Another node index + */ + Heap.prototype._moveNode = function (j, k) { + var _a; + _a = __read([this.heapArray[k], this.heapArray[j]], 2), this.heapArray[j] = _a[0], this.heapArray[k] = _a[1]; + }; + /** + * Move a node down the tree (to the leaves) to find a place where the heap is sorted. + * @param {Number} i Index of the node + */ + Heap.prototype._sortNodeDown = function (i) { + var _this = this; + var moveIt = i < this.heapArray.length - 1; + var self = this.heapArray[i]; + var getPotentialParent = function (best, j) { + if (_this.heapArray.length > j && _this.compare(_this.heapArray[j], _this.heapArray[best]) < 0) { + best = j; + } + return best; + }; + while (moveIt) { + var childrenIdx = Heap.getChildrenIndexOf(i); + var bestChildIndex = childrenIdx.reduce(getPotentialParent, childrenIdx[0]); + var bestChild = this.heapArray[bestChildIndex]; + if (typeof bestChild !== 'undefined' && this.compare(self, bestChild) > 0) { + this._moveNode(i, bestChildIndex); + i = bestChildIndex; + } + else { + moveIt = false; + } + } + }; + /** + * Move a node up the tree (to the root) to find a place where the heap is sorted. + * @param {Number} i Index of the node + */ + Heap.prototype._sortNodeUp = function (i) { + var moveIt = i > 0; + while (moveIt) { + var pi = Heap.getParentIndexOf(i); + if (pi >= 0 && this.compare(this.heapArray[pi], this.heapArray[i]) > 0) { + this._moveNode(i, pi); + i = pi; + } + else { + moveIt = false; + } + } + }; + /** + * Return the top (highest value) N elements of the heap, without corner cases, unsorted + * Implementation: push. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + Heap.prototype._topN_push = function (n) { + // Use an inverted heap + var topHeap = new Heap(this._invertedCompare); + topHeap.limit = n; + var indices = [0]; + var arr = this.heapArray; + while (indices.length) { + var i = indices.shift(); + if (i < arr.length) { + if (topHeap.length < n) { + topHeap.push(arr[i]); + indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i)), false)); + } + else if (this.compare(arr[i], topHeap.peek()) < 0) { + topHeap.replace(arr[i]); + indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i)), false)); + } + } + } + return topHeap.toArray(); + }; + /** + * Return the top (highest value) N elements of the heap, without corner cases, unsorted + * Implementation: init + push. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + Heap.prototype._topN_fill = function (n) { + // Use an inverted heap + var heapArray = this.heapArray; + var topHeap = new Heap(this._invertedCompare); + topHeap.limit = n; + topHeap.heapArray = heapArray.slice(0, n); + topHeap.init(); + var branch = Heap.getParentIndexOf(n - 1) + 1; + var indices = []; + for (var i = branch; i < n; ++i) { + indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i).filter(function (l) { return l < heapArray.length; })), false)); + } + if ((n - 1) % 2) { + indices.push(n); + } + while (indices.length) { + var i = indices.shift(); + if (i < heapArray.length) { + if (this.compare(heapArray[i], topHeap.peek()) < 0) { + topHeap.replace(heapArray[i]); + indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i)), false)); + } + } + } + return topHeap.toArray(); + }; + /** + * Return the top (highest value) N elements of the heap, without corner cases, unsorted + * Implementation: heap. + * + * @param {Number} n Number of elements. + * @return {Array} Array of length <= N. + */ + Heap.prototype._topN_heap = function (n) { + var topHeap = this.clone(); + var result = []; + for (var i = 0; i < n; ++i) { + result.push(topHeap.pop()); + } + return result; + }; + /** + * Return index of the top element + * @param list + */ + Heap.prototype._topIdxOf = function (list) { + if (!list.length) { + return -1; + } + var idx = 0; + var top = list[idx]; + for (var i = 1; i < list.length; ++i) { + var comp = this.compare(list[i], top); + if (comp < 0) { + idx = i; + top = list[i]; + } + } + return idx; + }; + /** + * Return the top element + * @param list + */ + Heap.prototype._topOf = function () { + var list = []; + for (var _i = 0; _i < arguments.length; _i++) { + list[_i] = arguments[_i]; + } + var heap = new Heap(this.compare); + heap.init(list); + return heap.peek(); + }; + return Heap; +}()); + +module.exports = { Heap, HeapAsync, Heap, toInt }; From 4ea26f0cbb99ca8f5787a85a8b40ffb8b52b1177 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 17 Jun 2024 11:12:53 +0530 Subject: [PATCH 155/224] Add per-partition concurrency --- lib/kafkajs/_consumer.js | 262 ++++++++++++++++++---------- lib/kafkajs/_consumer_cache.js | 41 ++++- lib/kafkajs/_producer.js | 2 +- test/promisified/unit/cache.spec.js | 238 +++++++++++++++++++++++++ 4 files changed, 441 insertions(+), 102 deletions(-) create mode 100644 test/promisified/unit/cache.spec.js diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 0db4284f..7882677c 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -130,6 +130,24 @@ class Consumer { */ #userManagedStores = false; + /** + * Populated with Promises for each partition that is being processed concurrently. + * Each promise might run eachMessage/eachBatch. + */ + #runningPromises = []; + + /** + * Each message that is consumed has an associated cache index. + * This array maps a an index within runningPromises to the associated cached index. + * ie. runningPromises[i] is associated with the cache index #savedIndexToPromiseIndex[i]. + */ + #savedIndexToPromiseIndex = []; + + /** + * Signals an intent to disconnect the consumer. + */ + #disconnectStarted = false; + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -637,16 +655,17 @@ class Consumer { return msg; } - // TODO: Add this block for concurrency - // if (!msg) { - // // it's possible that we get msg = null, but that's because partitionConcurrency - // // exceeds the number of partitions containing messages. So in this case, - // // we should not call for new fetches, rather, try to focus on what we have left. - // return null; - // } + /* It's possible that we get msg = null, but that's because partitionConcurrency + * exceeds the number of partitions containing messages. So in this case, + * we should not call for new fetches, rather, try to focus on what we have left. + */ + if (!msg && this.#messageCache.pendingSize() !== 0) { + return null; + } return new Promise((resolve, reject) => { this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { + if (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; @@ -721,9 +740,6 @@ class Consumer { } const rdKafkaConfig = this.#config(); - const maxPollInterval = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; - this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), 1); - this.#state = ConsumerState.CONNECTING; this.#internalClient = new RdKafka.KafkaConsumer(rdKafkaConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); @@ -815,10 +831,6 @@ class Consumer { throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsAutoCommitThreshold(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - if (Object.hasOwn(config, 'partitionsConsumedConcurrently')) { - throw new error.KafkaJSError(CompatibilityErrorMessages.runOptionsPartitionsConsumedConcurrently(), { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); - } - if (this.#running) { throw new error.KafkaJSError('Consumer is already running.', { code: error.ErrorCodes.ERR__STATE }); } @@ -829,6 +841,14 @@ class Consumer { config.eachBatchAutoResolve = true; } + if (!Object.hasOwn(config, 'partitionsConsumedConcurrently')) { + config.partitionsConsumedConcurrently = 1; + } + + const rdKafkaConfig = this.#config(); + const maxPollInterval = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; + this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), config.partitionsConsumedConcurrently); + /* We deliberately don't await this. */ if (config.eachMessage) { this.#runInternalEachMessage(config); @@ -837,126 +857,173 @@ class Consumer { } } + /** + * Processes a single message. + * + * @param m Message as obtained from #consumeSingleCached. + * @param config Config as passed to run(). + * @returns {Promise} the cache index of the message that was processed. + */ + async #messageProcessor(m, config) { + let eachMessageProcessed = false; + const payload = this.#createPayload(m); + + try { + await config.eachMessage(payload); + eachMessageProcessed = true; + } 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 (which + * means seeking to the message offset). + * However, we don't do this inside the catch, but just outside it. This is because throwing an + * error is not the only case where we might want to seek back. + * + * So - do nothing but a debug log, but at this point eachMessageProcessed is false. + */ + this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); + } + + /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ + if (!eachMessageProcessed) { + await this.seek({ + topic: m.topic, + partition: m.partition, + offset: m.offset, + }); + } + + /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ + if (eachMessageProcessed) { + try { + if (!this.#userManagedStores) { + this.#internalClient.offsetsStore([{ + topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch + }]); + } + this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); + } catch (e) { + /* Not much we can do, except log the error. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); + } + } + + + /* 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. + * TOOD: this block can probably be common and not per message. */ + if (this.#checkPendingSeeks) + await this.#seekInternal(); + + return m.index; + } + + /** + * Awaits the completion of a single message's processing. + * + * @returns {Promise} the cache index of the message in the cache that was processed. + */ + async waitOne() { + const savedIndex = await Promise.any(this.#runningPromises); + const promiseIndex = this.#savedIndexToPromiseIndex.findIndex(p => p === savedIndex); + if (promiseIndex === -1) { + console.error("Promise not found in runningPromises"); + throw new Error("Promise not found in runningPromises"); + } + this.#runningPromises[promiseIndex] = this.#runningPromises[this.#runningPromises.length - 1]; + this.#savedIndexToPromiseIndex[promiseIndex] = this.#savedIndexToPromiseIndex[this.#savedIndexToPromiseIndex.length - 1]; + this.#runningPromises.pop(); + this.#savedIndexToPromiseIndex.pop(); + + return savedIndex; + } + + /** + * Awaits the completion of all messages that are being processed. + * + * @returns {Promise} a list of cache indices of the messages that were processed. + */ + async waitAll() { + const indices = await Promise.all(this.#runningPromises); + this.#runningPromises = []; + this.#savedIndexToPromiseIndex = []; + return indices; + } + /* Internal polling loop. * It accepts the same config object that `run` accepts, but config.eachMessage must be set. */ async #runInternalEachMessage(config) { - let savedIdx = -1; - while (this.#state === ConsumerState.CONNECTED) { + const concurrency = config.partitionsConsumedConcurrently; + let nextIdx = -1; + while (!(await acquireOrLog(this.#lock, this.#logger))); - /* We need to acquire a lock here, because we need to ensure that we don't - * disconnect while in the middle of processing a message. */ - if (!(await acquireOrLog(this.#lock, this.#logger))) - continue; + while (this.#state === ConsumerState.CONNECTED) { + /* Release lock and cleanup if we intend to disconnect. */ + if (this.#disconnectStarted) { + const indices = await this.waitAll(); + indices.forEach(idx => this.#messageCache.return(idx)); + if (nextIdx !== -1) { + this.#messageCache.return(nextIdx); + } + nextIdx = -1; + this.#lock.release(); + break; + } /* Invalidate the message cache if needed */ const locallyStale = this.#messageCache.popLocallyStale(); if (this.#messageCache.isStale()) { /* global staleness */ - // TODO: await all concurrent promises for eachMessage here. + const indices = await this.waitAll(); + indices.forEach(idx => this.#messageCache.return(idx)); + if (nextIdx !== -1) { + this.#messageCache.return(nextIdx); + } + nextIdx = -1; await this.#clearCacheAndResetPositions(); - await this.#lock.release(); continue; } else if (locallyStale.length !== 0) { /* local staleness */ // TODO: is it correct to await some concurrent promises for eachMessage here? // to be safe we can do it, but I don't think we really need to do that for - // correctness. + // any correctness reason. await this.#clearCacheAndResetPositions(locallyStale); - await this.#lock.release(); continue; } - const m = await this.#consumeSingleCached(savedIdx).catch(e => { + const m = await this.#consumeSingleCached(nextIdx).catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${e} : ${e.stack}`); }); + nextIdx = -1; + if (!m) { - // await all concurrency related promises right here if this is null, if any such promise exists. + // await any concurrency related promises right here if this is null, if any such promise exists. // see note in consumeSingleCached - savedIdx = -1; - await this.#lock.release(); - continue; - } - savedIdx = m.index; - - /* 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. */ - this.#lock.release(); - continue; + if (this.#runningPromises.length) { + nextIdx = await this.waitOne(); } + continue; } - let eachMessageProcessed = false; - const payload = this.#createPayload(m); - try { - await config.eachMessage(payload); - eachMessageProcessed = true; - } 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 (which - * means seeking to the message offset). - * However, we don't do this inside the catch, but just outside it. This is because throwing an - * error is not the only case where we might want to seek back. - * - * So - do nothing but a debug log, but at this point eachMessageProcessed is false. - */ - this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); - } - - /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ - if (!eachMessageProcessed) { - await this.seek({ - topic: m.topic, - partition: m.partition, - offset: m.offset, - }); - } + const p = this.#messageProcessor(m, config); + this.#runningPromises.push(p); + this.#savedIndexToPromiseIndex.push(m.index); - /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ - if (eachMessageProcessed) { - try { - if (!this.#userManagedStores) { - this.#internalClient.offsetsStore([{ - topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch - }]); - } - this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); - } catch (e) { - /* Not much we can do, except log the error. */ - if (this.#logger) - this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); - } + if (this.#runningPromises.length < concurrency) { + continue; } - /* 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(); + nextIdx = await this.waitOne(); /* 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. */ - - /* Release the lock so that any pending disconnect can go through. */ - await this.#lock.release(); } } @@ -1497,6 +1564,7 @@ class Consumer { return; } + this.#disconnectStarted = true; while (!(await acquireOrLog(this.#lock, this.#logger))); /* Just retry... */ this.#state = ConsumerState.DISCONNECTING; diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 6bd345b5..1fe9e7f9 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -93,6 +93,10 @@ class MessageCache { this.pendingIndices = new Set(); } + pendingSize() { + return this.pendingIndices.size; + } + /** * Add a set of topic partitions to the cache (empty PPCs). * Pre-conditions: ppcList must be empty (cache is inactive) @@ -232,6 +236,9 @@ class MessageCache { * Post-conditions: all caches are unstale, (todo: ppcList is sorted by timestamp). */ addMessages(messages) { + if (this.pendingSize() > 0) { + throw new Error(`Cache cannot be added to with ${this.pendingSize()} pending indices.`); + } /* There will be caches in the ppcList which are either stale, or have * run out of messages. We need to clear them, else #add() will not add * them back to the ppcList since they're not empty. */ @@ -263,6 +270,24 @@ class MessageCache { for (let i = 0; i <= this.maxIndicesIndex; i++) { this.indices.push(i); } + // const ppcObj = this.ppcList.map(ppc => ppc.cache.map(c => ({topic: c.topic, partition: c.partition, offset: c.offset}))); + // console.log("addMessages", "\ntpToPPC: ", this.tpToPpc, "\nppcList: ", JSON.stringify(ppcObj, null, 2)); + } + + /** + * Allows returning the cache index of a consumed message without asking for another message. + * @param {number} idx - the index of the message that was consumed. + * @note This is a no-op if the index is not in the pendingIndices set. + */ + return(idx) { + if (!this.pendingIndices.has(idx)) { + /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible + * that we ran out of messages and fetched a new batch. So we just discard what the user is + * returning to us. */ + console.error("Returning unowned index", idx, "to cache. Discarding it."); + } else { + this.pendingIndices.delete(idx); + } } /** @@ -280,19 +305,25 @@ class MessageCache { * @note Whenever making changes to this function, ensure that you benchmark perf. */ next(idx = -1) { + // console.log("next called with ", idx, " and pending indices", this.pendingIndices, " and ppcList ", JSON.stringify(this.ppcList.map(ppc => ({ + // ...ppc, + // cache: ppc.cache.map(c => ({topic: c.topic, partition: c.partition, offset: c.offset})), + // })), null, 2)); let index = idx; - if (!this.pendingIndices.has(index)) { + if (index !== -1 && !this.pendingIndices.has(index)) { /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible * that we ran out of messages and fetched a new batch. So we just discard what the user is * returning to us. */ + console.error("Returning unowned index", idx, "to cache. Discarding it."); index = -1; } else { this.pendingIndices.delete(index); } if (index === -1) { - if (this.indices.size() === 0) + if (this.indices.size() === 0 || this.pendingIndices.size === this.maxConcurrency) { return null; + } index = this.indices.pop(); // index cannot be undefined here since indices.size > 0 } @@ -329,6 +360,10 @@ class MessageCache { * Post-conditions: maxSize = 1, all caches are unstale, ppcList is empty, locallyStaleCaches is empty. */ clear() { + if (this.pendingSize() > 0) { + console.error('clear: pendingIndices = ', this.pendingIndices, console.trace()); + throw new Error(`Cache cannot be cleared with ${this.pendingSize()} pending indices.`); + } for (const cache of this.ppcList) { cache.clear(); } @@ -339,8 +374,6 @@ class MessageCache { this.cachedTime = hrtime(); this.locallyStaleCaches = []; this.indices.clear(); - // if (this.pendingIndices.size > 0) console.log('clear: pendingIndices = ', this.pendingIndices, console.); - this.pendingIndices.clear(); this.currentIndex = 0; } } diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 76860b81..49cb0024 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -25,7 +25,7 @@ const ProducerState = Object.freeze({ }); const CompressionTypes = Object.freeze({ - NONE: 'none', + None: 'none', GZIP: 'gzip', SNAPPY: 'snappy', LZ4: 'lz4', diff --git a/test/promisified/unit/cache.spec.js b/test/promisified/unit/cache.spec.js new file mode 100644 index 00000000..379983c4 --- /dev/null +++ b/test/promisified/unit/cache.spec.js @@ -0,0 +1,238 @@ +const MessageCache = require('../../../lib/kafkajs/_consumer_cache'); + +describe('MessageCache', () => { + const expiryTime = 300000; // Long time. + const toppars = [{ topic: 'topic', partition: 0 }, { topic: 'topic', partition: 1 }, { topic: 'topic', partition: 2 }]; + const messages = + Array(5000) + .fill() + .map((_, i) => ({ topic: 'topic', partition: i % 3, number: i })); + + beforeEach(() => { + }); + + describe("with concurrency", () => { + let cache; + beforeEach(() => { + cache = new MessageCache(expiryTime, 1); + cache.addTopicPartitions(toppars); + }); + + it('caches messages and retrieves them', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + for (let i = 0; i < 90; i++) { + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + } + + /* Results are on a per-partition basis and well-ordered */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + + it('does not allow fetching more than 1 message at a time', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + let next = cache.next(-1); + let savedIndex = next.index; + expect(next).not.toBeNull(); + next = cache.next(-1); + expect(next).toBeNull(); + expect(cache.pendingSize()).toBeGreaterThan(0); + + // Fetch after returning index works. + next = cache.next(savedIndex); + expect(next).not.toBeNull(); + }); + + it('stops fetching from stale partition', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + for (let i = 0; i < 3; i++) { + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + cache.markStale([{topic: next.topic, partition: next.partition}]); + } + + // We should not be able to get anything more. + expect(cache.next(nextIdx)).toBeNull(); + // Nothing should be pending, we've returned everything. + expect(cache.pendingSize()).toBe(0); + // The first 3 messages from different toppars are what we should get. + expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 3))); + }); + + }); + + describe("with concurrency = 2", () => { + let cache; + beforeEach(() => { + cache = new MessageCache(expiryTime, 2); + cache.addTopicPartitions(toppars); + }); + + it('caches messages and retrieves them', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + for (let i = 0; i < 90; i++) { + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + } + + /* Results are on a per-partition basis and well-ordered */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + + it('caches messages and retrieves 2-at-a-time', () => { + const msgs = messages.slice(0, 90).filter(msg => msg.partition !== 3); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdxs = [-1, -1]; + for (let i = 0; i < 30; i++) { + const next0 = cache.next(nextIdxs[0]); + const next1 = cache.next(nextIdxs[1]); + expect(next0).not.toBeNull(); + expect(next1).not.toBeNull(); + receivedMessages.push(next0); + receivedMessages.push(next1); + nextIdxs = [next0.index, next1.index]; + } + + /* Results are on a zig-zag basis. */ + expect(receivedMessages.every((msg, i) => msg.number === receivedMessages.number)); + }); + + it('does not allow fetching more than 2 message at a time', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + let next = cache.next(-1); + let savedIndex = next.index; + expect(next).not.toBeNull(); + next = cache.next(-1); + expect(next).not.toBeNull(); + next = cache.next(-1); + expect(next).toBeNull(); + expect(cache.pendingSize()).toBe(2); + + // Fetch after returning index works. + next = cache.next(savedIndex); + expect(next).not.toBeNull(); + }); + + it('stops fetching from stale partition', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + for (let i = 0; i < 3; i++) { + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + cache.markStale([{topic: next.topic, partition: next.partition}]); + } + + // We should not be able to get anything more. + expect(cache.next(nextIdx)).toBeNull(); + // Nothing should be pending, we've returned everything. + expect(cache.pendingSize()).toBe(0); + // The first 3 messages from different toppars are what we should get. + expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 3))); + }); + + it('one slow processing message should not slow down others', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + const slowMsg = cache.next(nextIdx); + for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + } + + + // We should not be able to get anything more. + expect(cache.next(nextIdx)).toBeNull(); + // The slowMsg should be pending. + expect(cache.pendingSize()).toBe(1); + + /* Messages should be partition-wise and well-ordered. */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + + it('should not be able to handle cache-clearance in the middle of processing', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + const slowMsg = cache.next(nextIdx); + for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + } + + + // We should not be able to get anything more. + expect(cache.next(nextIdx)).toBeNull(); + + // The slowMsg should be pending. + expect(cache.pendingSize()).toBe(1); + + expect(() => cache.clear()).toThrow(); + }); + + it('should not be able to handle message adds in the middle of processing', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + const slowMsg = cache.next(nextIdx); + for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ + const next = cache.next(nextIdx); + expect(next).not.toBeNull(); + receivedMessages.push(next); + nextIdx = next.index; + } + + // We should not be able to get anything more. + expect(cache.next(nextIdx)).toBeNull(); + + // The slowMsg should be pending. + expect(cache.pendingSize()).toBe(1); + + expect(() => cache.addMessages(msgs)).toThrow(); + }); + }); +}); \ No newline at end of file From b140a48caf0ea739e5e09b44316d89798f0ee271 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 17 Jun 2024 15:48:21 +0530 Subject: [PATCH 156/224] Add partition level concurrency to faux-eachBatch --- lib/kafkajs/_consumer.js | 217 +++++++++++++++++++++------------------ 1 file changed, 117 insertions(+), 100 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 7882677c..8b2acdfa 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -665,7 +665,6 @@ class Consumer { return new Promise((resolve, reject) => { this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { - if (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; @@ -882,6 +881,9 @@ class Consumer { * So - do nothing but a debug log, but at this point eachMessageProcessed is false. */ this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); + + /* TODO: log error if error type is not KafkaJSError and if no pause() has been called */ + this.#logger.error(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); } /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ @@ -920,6 +922,82 @@ class Consumer { return m.index; } + /** + * Processes a batch message (a single message as of now). + * + * @param m Message as obtained from #consumeSingleCached. + * @param config Config as passed to run(). + * @returns {Promise} the cache index of the message that was processed. + */ + async #batchProcessor(m, config) { + let eachMessageProcessed = false; + const payload = this.#createBatchPayload(m); + try { + await config.eachBatch(payload); + if (config.eachBatchAutoResolve) { + eachMessageProcessed = true; + } else { + eachMessageProcessed = payload._messageResolved; + } + } catch (e) { + /* It's not only possible, but expected that an error will be thrown by eachBatch. + * 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 (which + * means seeking to the message offset). + * However, we don't do this inside the catch, but just outside it. This is because throwing an + * error is not the only case where we might want to seek back. We might want to seek back + * if the user has not called `resolveOffset` manually in case of using eachBatch without + * eachBatchAutoResolve being set. + * + * So - do nothing but a debug log, but at this point eachMessageProcessed needs to be false unless + * the user has explicitly marked it as true. + */ + this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); + + /* TODO: log error if error type is not KafkaJSError and if no pause() has been called */ + this.#logger.error(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); + + /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed + * despite an error is if the user says so, and the user can use resolveOffsets for both the possible + * values eachBatchAutoResolve can take. */ + if (config.eachBatch) + eachMessageProcessed = payload._messageResolved + } + + /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ + if (!eachMessageProcessed) { + await this.seek({ + topic: m.topic, + partition: m.partition, + offset: m.offset, + }); + } + + /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ + if (eachMessageProcessed) { + try { + if (!this.#userManagedStores) { + this.#internalClient.offsetsStore([{ + topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch + }]); + } + this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); + } catch (e) { + /* Not much we can do, except log the error. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); + } + } + + /* 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(); + + return m.index; + } + /** * Awaits the completion of a single message's processing. * @@ -1030,130 +1108,69 @@ class Consumer { /* Internal polling loop. * It accepts the same config object that `run` accepts, but config.eachBatch must be set. */ async #runInternalEachBatch(config) { - let savedIdx = -1; - while (this.#state === ConsumerState.CONNECTED) { + const concurrency = config.partitionsConsumedConcurrently; + let nextIdx = -1; + while (!(await acquireOrLog(this.#lock, this.#logger))); - /* We need to acquire a lock here, because we need to ensure that we don't - * disconnect while in the middle of processing a message. */ - if (!(await acquireOrLog(this.#lock, this.#logger))) - continue; + while (this.#state === ConsumerState.CONNECTED) { + /* Release lock and cleanup if we intend to disconnect. */ + if (this.#disconnectStarted) { + const indices = await this.waitAll(); + indices.forEach(idx => this.#messageCache.return(idx)); + if (nextIdx !== -1) { + this.#messageCache.return(nextIdx); + } + nextIdx = -1; + this.#lock.release(); + break; + } /* Invalidate the message cache if needed */ const locallyStale = this.#messageCache.popLocallyStale(); if (this.#messageCache.isStale()) { /* global staleness */ + const indices = await this.waitAll(); + indices.forEach(idx => this.#messageCache.return(idx)); + if (nextIdx !== -1) { + this.#messageCache.return(nextIdx); + } + nextIdx = -1; await this.#clearCacheAndResetPositions(); - await this.#lock.release(); continue; } else if (locallyStale.length !== 0) { /* local staleness */ + // TODO: is it correct to await some concurrent promises for eachMessage here? + // to be safe we can do it, but I don't think we really need to do that for + // any correctness reason. await this.#clearCacheAndResetPositions(locallyStale); - await this.#lock.release(); continue; } - const m = await this.#consumeSingleCached(savedIdx).catch(e => { + const m = await this.#consumeSingleCached(nextIdx).catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${JSON.stringify(e)}`); }); - if (!m) { - savedIdx = -1; - await this.#lock.release(); - continue; - } - savedIdx = m.index; - - /* 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. */ - this.#lock.release(); - continue; - } - } + nextIdx = -1; - let eachMessageProcessed = false; - const payload = this.#createBatchPayload(m); - try { - await config.eachBatch(payload); - if (config.eachBatchAutoResolve) { - eachMessageProcessed = true; - } else { - eachMessageProcessed = payload._messageResolved; + if (!m) { + // await any concurrency related promises right here if this is null, if any such promise exists. + // see note in consumeSingleCached + if (this.#runningPromises.length) { + nextIdx = await this.waitOne(); } - } catch (e) { - /* It's not only possible, but expected that an error will be thrown by eachBatch. - * 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 (which - * means seeking to the message offset). - * However, we don't do this inside the catch, but just outside it. This is because throwing an - * error is not the only case where we might want to seek back. We might want to seek back - * if the user has not called `resolveOffset` manually in case of using eachBatch without - * eachBatchAutoResolve being set. - * - * So - do nothing but a debug log, but at this point eachMessageProcessed needs to be false unless - * the user has explicitly marked it as true. - */ - this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); - - /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed - * despite an error is if the user says so, and the user can use resolveOffsets for both the possible - * values eachBatchAutoResolve can take. */ - if (config.eachBatch) - eachMessageProcessed = payload._messageResolved + continue; } - /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ - if (!eachMessageProcessed) { - await this.seek({ - topic: m.topic, - partition: m.partition, - offset: m.offset, - }); - } + const p = this.#batchProcessor(m, config); + this.#runningPromises.push(p); + this.#savedIndexToPromiseIndex.push(m.index); - /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ - if (eachMessageProcessed) { - try { - if (!this.#userManagedStores) { - this.#internalClient.offsetsStore([{ - topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch - }]); - } - this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); - } catch (e) { - /* Not much we can do, except log the error. */ - if (this.#logger) - this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); - } + if (this.#runningPromises.length < concurrency) { + continue; } - /* 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. - */ - - /* Release the lock so that any pending disconnect can go through. */ - await this.#lock.release(); + nextIdx = await this.waitOne(); } } From f2ec06393e65fd744dcbb995228911d7168f8d2d Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 20 Jun 2024 18:58:44 +0530 Subject: [PATCH 157/224] Create persistent workers for per-partition concurrency, prevents excessive Promise spawning --- lib/kafkajs/_consumer.js | 255 +++++++++++++-------------------- lib/kafkajs/_consumer_cache.js | 68 ++++++--- 2 files changed, 150 insertions(+), 173 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 8b2acdfa..f5666aa6 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -131,22 +131,41 @@ class Consumer { #userManagedStores = false; /** - * Populated with Promises for each partition that is being processed concurrently. - * Each promise might run eachMessage/eachBatch. + * Signals an intent to disconnect the consumer. */ - #runningPromises = []; + #disconnectStarted = false; /** - * Each message that is consumed has an associated cache index. - * This array maps a an index within runningPromises to the associated cached index. - * ie. runningPromises[i] is associated with the cache index #savedIndexToPromiseIndex[i]. + * Number of partitions owned by the consumer. + * @note This value may or may not be completely accurate, it's more so a hint for spawning concurrent workers. */ - #savedIndexToPromiseIndex = []; + #partitionCount = 0; /** - * Signals an intent to disconnect the consumer. + * Whether worker termination has been scheduled. */ - #disconnectStarted = false; + #workerTerminationScheduled = false; + + /** + * The worker functions currently running in the consumer. + */ + #workers = []; + + /** + * The number of partitions to consume concurrently as set by the user, or 1. + */ + #concurrency = 1; + + /** + * Whether any call to the internalClient's consume() method is in progress. + */ + #fetchInProgress = false; + + /** + * TODO: remove this or make it a bit more reliable. + * This is a debug property for this branch. + */ + clientId = null; /** * @constructor @@ -217,7 +236,6 @@ class Consumer { * @param {import("../../types").TopicPartition[]} assignment */ #rebalanceCallback(err, assignment) { - // Create the librdkafka error err = LibrdKafkaError.create(err); const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; @@ -276,6 +294,8 @@ class Consumer { * and marked the cache stale. This means that the cache is always expired when a rebalance * is triggered. * This is applicable both for incremental and non-incremental rebalances. + * Multiple consume()s cannot be called together, too, because we make sure that only + * one worker is calling into the internal consumer at a time. */ try { @@ -285,10 +305,13 @@ class Consumer { if (checkPendingSeeks && !assignmentModified) assignment = this.#assignAsPerSeekedOffsets(assignment); - if (this.#internalClient.rebalanceProtocol() === "EAGER") + if (this.#internalClient.rebalanceProtocol() === "EAGER") { this.#internalClient.assign(assignment); - else + this.#partitionCount = assignment.length; + } else { this.#internalClient.incrementalAssign(assignment); + this.#partitionCount += assignment.length; + } if (checkPendingSeeks) { const offsetsToCommit = assignment @@ -313,9 +336,11 @@ class Consumer { if (this.#internalClient.rebalanceProtocol() === "EAGER") { this.#internalClient.unassign(); this.#messageCache.removeTopicPartitions(); + this.#partitionCount = 0; } else { this.#internalClient.incrementalUnassign(assignment); this.#messageCache.removeTopicPartitions(assignment); + this.#partitionCount -= assignment.length; } } } catch (e) { @@ -324,6 +349,18 @@ class Consumer { this.#internalClient.emit('rebalance.error', e); } } + + /** + * Schedule worker termination here, in case the number of workers is not equal to the target concurrency. + * We need to do this so we will respawn workers with the correct concurrency count. + */ + const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); + if (workersToSpawn !== this.#workers.length) { + this.#workerTerminationScheduled = true; + /* We don't need to await the workers here. We are OK if the termination and respawning + * occurs later, since even if we have a few more or few less workers for a while, it's + * not a big deal. */ + } }); } @@ -338,6 +375,8 @@ class Consumer { { code: error.ErrorCodes.ERR__INVALID_ARG }); } const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); + this.clientId = rdKafkaConfig['client.id']; + this.#logger = new DefaultLogger(); /* Consumer specific configuration */ if (Object.hasOwn(kjsConfig, 'groupId')) { @@ -663,8 +702,14 @@ class Consumer { return null; } + if (this.#fetchInProgress) { + return null; + } + + this.#fetchInProgress = true; return new Promise((resolve, reject) => { this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { + this.#fetchInProgress = false; if (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; @@ -846,14 +891,10 @@ class Consumer { const rdKafkaConfig = this.#config(); const maxPollInterval = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; - this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), config.partitionsConsumedConcurrently); + this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), config.partitionsConsumedConcurrently, this.#logger); - /* We deliberately don't await this. */ - if (config.eachMessage) { - this.#runInternalEachMessage(config); - } else { - this.#runInternalEachBatch(config); - } + /* We deliberately don't await this because we want to return from this method immediately. */ + this.#runInternal(config); } /** @@ -960,8 +1001,7 @@ class Consumer { /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed * despite an error is if the user says so, and the user can use resolveOffsets for both the possible * values eachBatchAutoResolve can take. */ - if (config.eachBatch) - eachMessageProcessed = payload._messageResolved + eachMessageProcessed = payload._messageResolved; } /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ @@ -999,68 +1039,25 @@ class Consumer { } /** - * Awaits the completion of a single message's processing. + * Starts a worker to fetch messages/batches from the internal consumer and process them. * - * @returns {Promise} the cache index of the message in the cache that was processed. - */ - async waitOne() { - const savedIndex = await Promise.any(this.#runningPromises); - const promiseIndex = this.#savedIndexToPromiseIndex.findIndex(p => p === savedIndex); - if (promiseIndex === -1) { - console.error("Promise not found in runningPromises"); - throw new Error("Promise not found in runningPromises"); - } - this.#runningPromises[promiseIndex] = this.#runningPromises[this.#runningPromises.length - 1]; - this.#savedIndexToPromiseIndex[promiseIndex] = this.#savedIndexToPromiseIndex[this.#savedIndexToPromiseIndex.length - 1]; - this.#runningPromises.pop(); - this.#savedIndexToPromiseIndex.pop(); - - return savedIndex; - } - - /** - * Awaits the completion of all messages that are being processed. + * A worker runs until it's told to stop. + * Conditions where the worker is told to stop: + * 1. Cache globally stale + * 2. Disconnected initiated + * 3. Rebalance + * 4. Some other worker has started terminating. * - * @returns {Promise} a list of cache indices of the messages that were processed. + * Worker termination acts as a async barrier. */ - async waitAll() { - const indices = await Promise.all(this.#runningPromises); - this.#runningPromises = []; - this.#savedIndexToPromiseIndex = []; - return indices; - } - - /* Internal polling loop. - * It accepts the same config object that `run` accepts, but config.eachMessage must be set. */ - async #runInternalEachMessage(config) { - const concurrency = config.partitionsConsumedConcurrently; + async #worker(config, perMessageProcessor, id) { let nextIdx = -1; - while (!(await acquireOrLog(this.#lock, this.#logger))); - - while (this.#state === ConsumerState.CONNECTED) { - /* Release lock and cleanup if we intend to disconnect. */ - if (this.#disconnectStarted) { - const indices = await this.waitAll(); - indices.forEach(idx => this.#messageCache.return(idx)); - if (nextIdx !== -1) { - this.#messageCache.return(nextIdx); - } - nextIdx = -1; - this.#lock.release(); - break; - } - + while (!this.#workerTerminationScheduled) { /* Invalidate the message cache if needed */ const locallyStale = this.#messageCache.popLocallyStale(); if (this.#messageCache.isStale()) { /* global staleness */ - const indices = await this.waitAll(); - indices.forEach(idx => this.#messageCache.return(idx)); - if (nextIdx !== -1) { - this.#messageCache.return(nextIdx); - } - nextIdx = -1; - await this.#clearCacheAndResetPositions(); - continue; + this.#workerTerminationScheduled = true; + break; } else if (locallyStale.length !== 0) { /* local staleness */ // TODO: is it correct to await some concurrent promises for eachMessage here? // to be safe we can do it, but I don't think we really need to do that for @@ -1079,99 +1076,44 @@ class Consumer { nextIdx = -1; if (!m) { - // await any concurrency related promises right here if this is null, if any such promise exists. - // see note in consumeSingleCached - if (this.#runningPromises.length) { - nextIdx = await this.waitOne(); - } - continue; - } - - const p = this.#messageProcessor(m, config); - this.#runningPromises.push(p); - this.#savedIndexToPromiseIndex.push(m.index); - - if (this.#runningPromises.length < concurrency) { + /* Backoff a little. If m is null, we might be fetching from the internal consumer (fetch in progress), + * and calling consumeSingleCached in a tight loop will help no one. */ + await new Promise((resolve) => setTimeout(resolve, 1)); continue; } - nextIdx = await this.waitOne(); + nextIdx = await perMessageProcessor(m, config); + } - /* 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. - */ + if (nextIdx !== -1) { + this.#messageCache.return(nextIdx); } } - /* Internal polling loop. - * It accepts the same config object that `run` accepts, but config.eachBatch must be set. */ - async #runInternalEachBatch(config) { - const concurrency = config.partitionsConsumedConcurrently; - let nextIdx = -1; + /** + * Internal polling loop. + * Spawns and awaits workers until disconnect is initiated. + */ + async #runInternal(config) { + this.#concurrency = config.partitionsConsumedConcurrently; + const perMessageProcessor = config.eachMessage ? this.#messageProcessor : this.#batchProcessor; + this.#workers = []; while (!(await acquireOrLog(this.#lock, this.#logger))); - while (this.#state === ConsumerState.CONNECTED) { - /* Release lock and cleanup if we intend to disconnect. */ - if (this.#disconnectStarted) { - const indices = await this.waitAll(); - indices.forEach(idx => this.#messageCache.return(idx)); - if (nextIdx !== -1) { - this.#messageCache.return(nextIdx); - } - nextIdx = -1; - this.#lock.release(); - break; - } + while (!this.#disconnectStarted) { + this.#workerTerminationScheduled = false; + const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); + this.#workers = Array(workersToSpawn).fill().map((_, i) => this.#worker(config, perMessageProcessor.bind(this), i)); + await Promise.all(this.#workers); - /* Invalidate the message cache if needed */ - const locallyStale = this.#messageCache.popLocallyStale(); - if (this.#messageCache.isStale()) { /* global staleness */ - const indices = await this.waitAll(); - indices.forEach(idx => this.#messageCache.return(idx)); - if (nextIdx !== -1) { - this.#messageCache.return(nextIdx); - } - nextIdx = -1; + /* One of the possible reasons for the workers to end is that the cache is globally stale. + * We need to take care of expiring it. */ + if (this.#messageCache.isStale()) { await this.#clearCacheAndResetPositions(); - continue; - } else if (locallyStale.length !== 0) { /* local staleness */ - // TODO: is it correct to await some concurrent promises for eachMessage here? - // to be safe we can do it, but I don't think we really need to do that for - // any correctness reason. - await this.#clearCacheAndResetPositions(locallyStale); - continue; } - - const m = await this.#consumeSingleCached(nextIdx).catch(e => { - /* Since this error cannot be exposed to the user in the current situation, just log and retry. - * This is due to restartOnFailure being set to always true. */ - if (this.#logger) - this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${JSON.stringify(e)}`); - }); - - nextIdx = -1; - - if (!m) { - // await any concurrency related promises right here if this is null, if any such promise exists. - // see note in consumeSingleCached - if (this.#runningPromises.length) { - nextIdx = await this.waitOne(); - } - continue; - } - - const p = this.#batchProcessor(m, config); - this.#runningPromises.push(p); - this.#savedIndexToPromiseIndex.push(m.index); - - if (this.#runningPromises.length < concurrency) { - continue; - } - - nextIdx = await this.waitOne(); } + + this.#lock.release(); } /** @@ -1582,6 +1524,7 @@ class Consumer { } this.#disconnectStarted = true; + this.#workerTerminationScheduled = true; while (!(await acquireOrLog(this.#lock, this.#logger))); /* Just retry... */ this.#state = ConsumerState.DISCONNECTING; diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 1fe9e7f9..2149f779 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -63,7 +63,7 @@ class PerPartitionMessageCache { */ class MessageCache { - constructor(expiryDurationMs, maxConcurrency) { + constructor(expiryDurationMs, maxConcurrency, logger) { /* Per partition cache list containing non-empty PPCs */ this.ppcList = []; /* Map of topic+partition to PerPartitionMessageCache. */ @@ -91,6 +91,8 @@ class MessageCache { /* Contains a list of indices of ppcList from which we have sent a message returned through next, but * the user has not returned the index back to us via next(idx) */ this.pendingIndices = new Set(); + /* Logger provided by cache user. Must have 'error' function defined on it. `console` is used by default. */ + this.logger = logger ?? console; } pendingSize() { @@ -102,11 +104,18 @@ class MessageCache { * Pre-conditions: ppcList must be empty (cache is inactive) */ addTopicPartitions(topicPartitions) { - if (this.ppcList.length !== 0) { - throw new Error('Cannot add topic partitions to a non-empty cache.'); + if (this.ppcList.some(ppc => (ppc.currentIndex < ppc.size()) && !ppc.stale)) { + throw new Error('Cannot add topic partitions to a cache which contains unprocessed, unstale elements.'); } for (const topicPartition of topicPartitions) { const key = partitionKey(topicPartition); + const existing = this.tpToPpc.get(key); + /* We're erring on the side of caution by including this check, as in the current model, + * a rebalance occurs only if all the caches are drained. */ + if (existing && existing.cache.length > 0 && !existing.stale && existing.currentIndex < existing.cache.length) { + this.logger.error("Cache already exists for key " + key + " with messages in it."); + throw new Error("Cache already exists for key " + key + " with messages in it."); + } this.tpToPpc.set(key, new PerPartitionMessageCache()); } } @@ -117,16 +126,32 @@ class MessageCache { * Pre-conditions: ppcList must be empty (cache is inactive) */ removeTopicPartitions(topicPartitions = null) { - if (this.ppcList.length !== 0) { - throw new Error('Cannot remove topic partitions from a non-empty cache.'); + if (this.ppcList.some(ppc => (ppc.currentIndex < ppc.size()) && !ppc.stale)) { + throw new Error('Cannot remove topic partitions from a cache which contains unprocessed, unstale elements.'); } if (topicPartitions === null) { + for (const key of this.tpToPpc.keys()) { + const existing = this.tpToPpc.get(key); + /* We're erring on the side of caution by including this check, as in the current model, + * a rebalance occurs only if all the caches are drained. */ + if (existing && existing.cache.length > 0 && !existing.stale && existing.currentIndex < existing.cache.length) { + this.logger.error("Cache already exists for key " + key + " with messages in it."); + throw new Error("Cache already exists for key " + key + " with messages in it."); + } + } this.tpToPpc.clear(); return; } for (const topicPartition of topicPartitions) { const key = partitionKey(topicPartition); + const existing = this.tpToPpc.get(key); + /* We're erring on the side of caution by including this check, as in the current model, + * a rebalance occurs only if all the caches are drained. */ + if (existing && existing.cache.length > 0 && !existing.stale && existing.currentIndex < existing.cache.length) { + this.logger.error("Cache already exists for key " + key + " with messages in it."); + throw new Error("Cache already exists for key " + key + " with messages in it."); + } this.tpToPpc.delete(key); } } @@ -220,6 +245,10 @@ class MessageCache { #add(message) { const key = partitionKey(message) const cache = this.tpToPpc.get(key); + if (!cache) { + this.logger.error("No cache found for message", message); + throw new Error("Inconsistency between fetched message and partition map"); + } cache.add(message); if (cache.size() === 1) { this.ppcList.push(cache); @@ -239,6 +268,7 @@ class MessageCache { if (this.pendingSize() > 0) { throw new Error(`Cache cannot be added to with ${this.pendingSize()} pending indices.`); } + /* There will be caches in the ppcList which are either stale, or have * run out of messages. We need to clear them, else #add() will not add * them back to the ppcList since they're not empty. */ @@ -249,7 +279,7 @@ class MessageCache { const key = partitionKey(tp); return this.tpToPpc.get(key).size() !== 0; })) { - console.error('Locally stale caches should have been cleared before adding messages: ', this.locallyStaleCaches); + logger.error('Locally stale caches should have been cleared before adding messages: ', this.locallyStaleCaches); throw new Error('Locally stale caches should have been cleared before adding messages.'); } @@ -264,14 +294,12 @@ class MessageCache { /* Reset the indices and pendingIndices because ppcList is being created newly. */ this.indices.clear(); - if (this.pendingIndices.size > 0) console.error('addMessages: pendingIndices = ', this.pendingIndices, console.trace()); + if (this.pendingIndices.size > 0) logger.error('addMessages: pendingIndices = ', this.pendingIndices, console.trace()); this.pendingIndices.clear(); this.maxIndicesIndex = Math.min(this.maxConcurrency, this.ppcList.length - 1); for (let i = 0; i <= this.maxIndicesIndex; i++) { this.indices.push(i); } - // const ppcObj = this.ppcList.map(ppc => ppc.cache.map(c => ({topic: c.topic, partition: c.partition, offset: c.offset}))); - // console.log("addMessages", "\ntpToPPC: ", this.tpToPpc, "\nppcList: ", JSON.stringify(ppcObj, null, 2)); } /** @@ -284,9 +312,10 @@ class MessageCache { /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible * that we ran out of messages and fetched a new batch. So we just discard what the user is * returning to us. */ - console.error("Returning unowned index", idx, "to cache. Discarding it."); + this.logger.error("Returning unowned index", idx, "to cache. Discarding it."); } else { this.pendingIndices.delete(idx); + this.indices.add(idx); } } @@ -305,19 +334,17 @@ class MessageCache { * @note Whenever making changes to this function, ensure that you benchmark perf. */ next(idx = -1) { - // console.log("next called with ", idx, " and pending indices", this.pendingIndices, " and ppcList ", JSON.stringify(this.ppcList.map(ppc => ({ - // ...ppc, - // cache: ppc.cache.map(c => ({topic: c.topic, partition: c.partition, offset: c.offset})), - // })), null, 2)); let index = idx; if (index !== -1 && !this.pendingIndices.has(index)) { /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible * that we ran out of messages and fetched a new batch. So we just discard what the user is * returning to us. */ - console.error("Returning unowned index", idx, "to cache. Discarding it."); + this.logger.error("Returning unowned index", idx, "to cache. Discarding it."); index = -1; - } else { + } else if (index !== -1) { this.pendingIndices.delete(index); + /* We don't add the index back to the this.indices here because we're just going to remove it again the + * first thing in the loop below, so it's slightly better to just avoid doing it. */ } if (index === -1) { @@ -327,6 +354,13 @@ class MessageCache { index = this.indices.pop(); // index cannot be undefined here since indices.size > 0 } + /* This loop will always terminate. Why? + * On each iteration: + * 1. We either return (if next is not null). + * 2. We change the PPC index we're interested in, and there are a finite number of PPCs. + * (PPCs don't repeat within the loop since the indices of the PPC are popped from within the + * heap and not put back in, or else a new index is created bounded by ppcList.length). + */ while (true) { const next = this.ppcList[index].next(); if (this.ppcList[index].isStale() || next === null) { @@ -361,7 +395,7 @@ class MessageCache { */ clear() { if (this.pendingSize() > 0) { - console.error('clear: pendingIndices = ', this.pendingIndices, console.trace()); + this.logger.error('clear: pendingIndices = ', this.pendingIndices, logger.trace()); throw new Error(`Cache cannot be cleared with ${this.pendingSize()} pending indices.`); } for (const cache of this.ppcList) { From 85f9b7e44fbf69da3f53158ec58f259067901e6d Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 24 Jun 2024 15:37:14 +0530 Subject: [PATCH 158/224] Fix tests for Per Partition Concurrency --- .../consumer/consumeMessages.spec.js | 196 ++++++++++-------- .../consumer/consumerCacheTests.spec.js | 42 ++-- test/promisified/consumer/pause.spec.js | 6 +- test/promisified/consumer/subscribe.spec.js | 2 +- test/promisified/testhelpers.js | 8 +- 5 files changed, 145 insertions(+), 109 deletions(-) diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index ede7bbe6..479775b8 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -15,15 +15,19 @@ const { generateMessages, } = require('../testhelpers'); -describe.each([[true], [false]])('Consumer', (isAutoCommit) => { +/* All combinations of autoCommit and partitionsConsumedConcurrently */ +const cases = Array(2 * 3).fill().map((_, i) => [i < 3, (i % 3) + 1]).slice(-1); + +describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) => { let topicName, groupId, producer, consumer; + const partitions = 3; beforeEach(async () => { + console.log("Starting:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); topicName = `test-topic-${secureRandom()}` groupId = `consumer-group-id-${secureRandom()}` - await createTopic({ topic: topicName }) - + await createTopic({ topic: topicName, partitions }) producer = createProducer({}); consumer = createConsumer({ @@ -37,6 +41,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { afterEach(async () => { consumer && (await consumer.disconnect()) producer && (await producer.disconnect()) + console.log("Ending:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); }); it('consume messages', async () => { @@ -45,13 +50,16 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { await consumer.subscribe({ topic: topicName }) const messagesConsumed = []; - consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + consumer.run({ + partitionsConsumedConcurrently, + eachMessage: async event => messagesConsumed.push(event) + }); - const messages = Array(100) + const messages = Array(10) .fill() .map(() => { const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + return { key: `key-${value}`, value: `value-${value}`, partition: 0 } }) await producer.send({ topic: topicName, messages }) @@ -76,7 +84,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { message: expect.objectContaining({ key: Buffer.from(messages[messages.length - 1].key), value: Buffer.from(messages[messages.length - 1].value), - offset: '99', + offset: '' + (messagesConsumed.length - 1), }), }) ) @@ -91,17 +99,21 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { await consumer.subscribe({ topic: topicName }) const messagesConsumed = []; - consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + consumer.run({ + partitionsConsumedConcurrently, + eachMessage: async event => messagesConsumed.push(event) + }); - const messages = [ { - value: `value-${secureRandom}`, - headers: { - 'header-1': 'value-1', - 'header-2': 'value-2', - 'header-3': [ 'value-3-1', 'value-3-2', Buffer.from([1,0,1,0,1]) ], - 'header-4': Buffer.from([1,0,1,0,1]), - } - } ] + const messages = [{ + value: `value-${secureRandom}`, + headers: { + 'header-1': 'value-1', + 'header-2': 'value-2', + 'header-3': ['value-3-1', 'value-3-2', Buffer.from([1, 0, 1, 0, 1])], + 'header-4': Buffer.from([1, 0, 1, 0, 1]), + }, + partition: 0, + }] await producer.send({ topic: topicName, messages }) await waitForMessages(messagesConsumed, { number: messages.length }) @@ -117,8 +129,8 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { // Headers are always returned as Buffers from the broker. 'header-1': Buffer.from('value-1'), 'header-2': Buffer.from('value-2'), - 'header-3': [ Buffer.from('value-3-1'), Buffer.from('value-3-2'), Buffer.from([1,0,1,0,1]) ], - 'header-4': Buffer.from([1,0,1,0,1]), + 'header-3': [Buffer.from('value-3-1'), Buffer.from('value-3-2'), Buffer.from([1, 0, 1, 0, 1])], + 'header-4': Buffer.from([1, 0, 1, 0, 1]), } }), }) @@ -132,6 +144,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { const messagesConsumed = []; consumer.run({ + partitionsConsumedConcurrently, eachBatchAutoResolve: isAutoResolve, eachBatch: async event => { // Match the message format to be checked easily later. @@ -148,42 +161,47 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { } }); - const messages = Array(100) + const messages = Array(100 * partitions) .fill() - .map(() => { + .map((_, i) => { const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions } }) 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: String(0), - }), - }) - ) + for (let p = 0; p < partitions; p++) { + const specificPartitionMessages = messagesConsumed.filter(m => m.partition === p); + const specificExpectedMessages = messages.filter(m => m.partition === p); + expect(specificPartitionMessages[0]).toEqual( + expect.objectContaining({ + topic: topicName, + partition: p, + message: expect.objectContaining({ + key: Buffer.from(specificExpectedMessages[0].key), + value: Buffer.from(specificExpectedMessages[0].value), + offset: String(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: String(messages.length - 1), - }), - }) - ) + expect(specificPartitionMessages[specificPartitionMessages.length - 1]).toEqual( + expect.objectContaining({ + topic: topicName, + partition: p, + message: expect.objectContaining({ + key: Buffer.from(specificExpectedMessages[specificExpectedMessages.length - 1].key), + value: Buffer.from(specificExpectedMessages[specificExpectedMessages.length - 1].value), + offset: String(specificExpectedMessages.length - 1), + }), + }) + ); + + // check if all offsets are present + expect(specificPartitionMessages.map(m => m.message.offset)).toEqual(specificExpectedMessages.map((_, i) => `${i}`)) + } - // check if all offsets are present - expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) }); it('is able to reconsume messages after not resolving it', async () => { @@ -194,6 +212,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { let messageSeen = false; const messagesConsumed = []; consumer.run({ + partitionsConsumedConcurrently, eachBatchAutoResolve: false, eachBatch: async event => { expect(event.batch.messages.length).toEqual(1); @@ -216,10 +235,11 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { .fill() .map(() => { const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + return { key: `key-${value}`, value: `value-${value}`, partition: 0 } }) - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); + await waitFor(() => consumer.assignment().length > 0, () => { }, 100); await waitForMessages(messagesConsumed, { number: messages.length }); }); @@ -231,6 +251,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { let messageSeen = false; const messagesConsumed = []; consumer.run({ + partitionsConsumedConcurrently, eachBatchAutoResolve: isAutoResolve, eachBatch: async event => { expect(event.batch.messages.length).toEqual(1); @@ -250,10 +271,10 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { .fill() .map(() => { const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; }) - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await waitForMessages(messagesConsumed, { number: messages.length }); }); @@ -264,6 +285,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { const messagesConsumed = []; consumer.run({ + partitionsConsumedConcurrently, eachBatchAutoResolve: isAutoResolve, eachBatch: async event => { messagesConsumed.push(...event.batch.messages); @@ -277,7 +299,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { .fill() .map(() => { const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + return { key: `key-${value}`, value: `value-${value}`, partition: 0 } }) await producer.send({ topic: topicName, messages }) @@ -287,49 +309,45 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { expect(messagesConsumed[1].key.toString()).toBe(messages[1].key); }); - /* Skip until concurrency support for eachMessage is added. */ - it.skip('consumes messages concurrently', async () => { - const partitionsConsumedConcurrently = 2 + it('consumes messages concurrently where partitionsConsumedConcurrently - partitions = diffConcurrencyPartitions', async () => { + const partitions = 3; + /* We want partitionsConsumedConcurrently to be 2, 3, and 4 rather than 1, 2, and 3 that is tested by the test. */ + const partitionsConsumedConcurrentlyDiff = partitionsConsumedConcurrently + 1; topicName = `test-topic-${secureRandom()}` await createTopic({ topic: topicName, - partitions: partitionsConsumedConcurrently + 1, + partitions: partitions, }) 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 = [] + let inProgress = 0; + let inProgressMaxValue = 0; + const messagesConsumed = []; consumer.run({ - partitionsConsumedConcurrently, + partitionsConsumedConcurrently: partitionsConsumedConcurrentlyDiff, eachMessage: async event => { - await sleep(1) - messagesConsumed.push(event) + inProgress++; + await sleep(1); + messagesConsumed.push(event); + inProgressMaxValue = Math.max(inProgress, inProgressMaxValue) + inProgress--; }, }) - await waitForConsumerToJoinGroup(consumer) + await waitFor(() => consumer.assignment().length > 0, () => { }, 100); - const messages = Array(100) + const messages = Array(1024*9) .fill() - .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } - }) - - await producer.send({ topic: topicName, messages }) - await waitForMessages(messagesConsumed, { number: messages.length }) + .map((_, i) => { + const value = secureRandom(512) + return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions } + }); - expect(hitConcurrencyLimit).toBeTrue() + await producer.send({ topic: topicName, messages }); + await waitForMessages(messagesConsumed, { number: messages.length }); + expect(inProgressMaxValue).toBe(Math.min(partitionsConsumedConcurrentlyDiff, partitions)); }); it('consume GZIP messages', async () => { @@ -346,9 +364,9 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); const key1 = secureRandom(); - const message1 = { key: `key-${key1}`, value: `value-${key1}` }; + const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 }; const key2 = secureRandom(); - const message2 = { key: `key-${key2}`, value: `value-${key2}` }; + const message2 = { key: `key-${key2}`, value: `value-${key2}`, partition: 0 }; await producer.send({ topic: topicName, @@ -477,7 +495,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { .fill() .map(() => { const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + return { key: `key-${value}`, value: `value-${value}`, partition: 0 } }); await consumer.connect(); @@ -704,7 +722,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { await consumer.subscribe({ topic: topicName }); const messagesConsumed = [] - const idempotentMessages = generateMessages({ prefix: 'idempotent' }) + const idempotentMessages = generateMessages({ prefix: 'idempotent', partition: 0 }) consumer.run({ eachMessage: async event => messagesConsumed.push(event), @@ -743,9 +761,9 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { const messagesConsumed = []; - const messages1 = generateMessages({ prefix: 'txn1' }); - const messages2 = generateMessages({ prefix: 'txn2' }); - const nontransactionalMessages1 = generateMessages({ prefix: 'nontransactional1', number: 1 }); + const messages1 = generateMessages({ prefix: 'txn1', partition: 0 }); + const messages2 = generateMessages({ prefix: 'txn2', partition: 0 }); + const nontransactionalMessages1 = generateMessages({ prefix: 'nontransactional1', number: 1, partition: 0 }); consumer.run({ eachMessage: async event => messagesConsumed.push(event), @@ -800,9 +818,9 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { const messagesConsumed = [] - const abortedMessages1 = generateMessages({ prefix: 'aborted-txn-1' }); - const abortedMessages2 = generateMessages({ prefix: 'aborted-txn-2' }); - const committedMessages = generateMessages({ prefix: 'committed-txn', number: 10 }); + const abortedMessages1 = generateMessages({ prefix: 'aborted-txn-1', partition: 0 }); + const abortedMessages2 = generateMessages({ prefix: 'aborted-txn-2', partition: 0 }); + const committedMessages = generateMessages({ prefix: 'committed-txn', number: 10, partition: 0 }); consumer.run({ eachMessage: async event => messagesConsumed.push(event), @@ -858,7 +876,7 @@ describe.each([[true], [false]])('Consumer', (isAutoCommit) => { const messagesConsumed = []; - const abortedMessages = generateMessages({ prefix: 'aborted-txn1' }); + const abortedMessages = generateMessages({ prefix: 'aborted-txn1', partition: 0 }); consumer.run({ eachMessage: async event => messagesConsumed.push(event), diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index d46d382f..d51f9919 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -10,10 +10,14 @@ const { sleep, } = require('../testhelpers'); -describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { +/* All combinations of autoCommit and partitionsConsumedConcurrently */ +const cases = Array(2 * 3).fill().map((_, i) => [i % 2 === 0, (i % 3) + 1]); + +describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumedConcurrently) => { let topicName, groupId, producer, consumer; beforeEach(async () => { + console.log("Starting:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); topicName = `test-topic-${secureRandom()}` groupId = `consumer-group-id-${secureRandom()}` @@ -33,6 +37,7 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { afterEach(async () => { consumer && (await consumer.disconnect()) producer && (await producer.disconnect()) + console.log("Ending:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); }); it('is cleared on pause', async () => { @@ -40,19 +45,21 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { await producer.connect(); await consumer.subscribe({ topic: topicName }) + const msgs = 1024; const messagesConsumed = []; consumer.run({ + partitionsConsumedConcurrently, eachMessage: async event => { messagesConsumed.push(event); - if (event.partition === 0 && (+event.message.offset) === 1023) { + if (event.partition === 0 && (+event.message.offset) === (msgs - 1)) { consumer.pause([{ topic: topicName, partitions: [0] }]); } } }); - /* Evenly distribute 1024*9 messages across 3 partitions */ + /* Evenly distribute msgs*9 messages across 3 partitions */ let i = 0; - const messages = Array(1024 * 9) + const messages = Array(msgs * 9) .fill() .map(() => { const value = secureRandom() @@ -62,20 +69,20 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { await producer.send({ topic: topicName, messages }) // Wait for the messages. - // We consume 1024 messages from partition 0, and 1024*3 from partition 1 and 2. - await waitForMessages(messagesConsumed, { number: 1024 * 7 }); + // We consume msgs*1 messages from partition 0, and msgs*3 from partition 1 and 2. + await waitForMessages(messagesConsumed, { number: msgs * 7 }); // We should not consume even one more message than that. await sleep(1000); - expect(messagesConsumed.length).toEqual(1024 * 7); + expect(messagesConsumed.length).toEqual(msgs * 7); // check if all offsets are present // partition 0 - expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.message.offset)).toEqual(Array(1024).fill().map((_, i) => `${i}`)); + expect(messagesConsumed.filter(m => m.partition === 0).map(m => m.message.offset)).toEqual(Array(msgs).fill().map((_, i) => `${i}`)); // partition 1 - expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.message.offset)).toEqual(Array(msgs * 3).fill().map((_, i) => `${i}`)); // partition 2 - expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.message.offset)).toEqual(Array(msgs * 3).fill().map((_, i) => `${i}`)); }); it('is cleared on seek', async () => { @@ -86,6 +93,7 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { const messagesConsumed = []; let hasBeenSeeked = false; consumer.run({ + partitionsConsumedConcurrently, eachMessage: async event => { messagesConsumed.push(event); if (event.partition === 0 && (+event.message.offset) === 1023 && !hasBeenSeeked) { @@ -130,6 +138,7 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { maxWaitTimeInMs: 100, fromBeginning: true, autoCommit: isAutoCommit, + clientId: "consumer2", }); await consumer.connect(); @@ -142,6 +151,7 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { let consumer2ConsumeRunning = false; consumer.run({ + partitionsConsumedConcurrently, eachMessage: async event => { messagesConsumed.push(event); messagesConsumedConsumer1.push(event); @@ -160,7 +170,8 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { /* Evenly distribute 1024*9 messages across 3 partitions */ let i = 0; - const messages = Array(1024 * 10) + const multiplier = 9; + const messages = Array(1024 * multiplier) .fill() .map(() => { const value = secureRandom() @@ -186,21 +197,21 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { consumer2ConsumeRunning = true; /* Now that both consumers have joined, wait for all msgs to be consumed */ - await waitForMessages(messagesConsumed, { number: 1024 * 10 }); + await waitForMessages(messagesConsumed, { number: 1024 * multiplier }); /* No extra messages should be consumed. */ await sleep(1000); - expect(messagesConsumed.length).toEqual(1024 * 10); + expect(messagesConsumed.length).toEqual(1024 * multiplier); /* Check if all messages were consumed. */ expect(messagesConsumed.map(event => (+event.message.offset)).sort((a, b) => a - b)) - .toEqual(Array(1024 * 10).fill().map((_, i) => Math.floor(i / 3))); + .toEqual(Array(1024 * multiplier).fill().map((_, i) => Math.floor(i / 3))); /* Consumer2 should have consumed at least one message. */ expect(messagesConsumedConsumer2.length).toBeGreaterThan(0); await consumer2.disconnect(); - }); + }, 60000); it('does not hold up polling for non-message events', async () => { /* Even if the cache is full of messages, we should still be polling for @@ -228,6 +239,7 @@ describe.each([[false], [true]])('Consumer message cache', (isAutoCommit) => { let consumer1TryingToJoin = false; impatientConsumer.run({ + partitionsConsumedConcurrently, eachMessage: async event => { messagesConsumed.push(event); impatientConsumerMessages.push(event); diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index 0ad75ac6..18d8dd71 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -15,6 +15,7 @@ describe('Consumer', () => { let groupId, producer, topics; beforeEach(async () => { + console.log("Starting:", expect.getState().currentTestName); topics = [`test-topic1-${secureRandom()}`, `test-topic2-${secureRandom()}`] groupId = `consumer-group-id-${secureRandom()}` @@ -36,6 +37,7 @@ describe('Consumer', () => { afterEach(async () => { consumer && (await consumer.disconnect()) producer && (await producer.disconnect()) + console.log("Ending:", expect.getState().currentTestName); }) describe('when pausing', () => { @@ -140,7 +142,7 @@ describe('Consumer', () => { const messagesConsumed = [] consumer.run({ eachMessage: async event => { - const { topic, message, partition } = event + const { topic, message, partition } = event; const whichTopic = topics.indexOf(topic) const whichMessage = messages.findIndex(m => String(m.key) === String(message.key)) @@ -185,7 +187,7 @@ describe('Consumer', () => { expect(messagesConsumed).toHaveLength(8) expect(messagesConsumed).toContainEqual({ topic: 0, message: 1 }) // partition 0 expect(messagesConsumed).toContainEqual({ topic: 0, message: 3 }) // partition 0 - }, 10000); + }, 15000); it('pauses when pausing via the eachBatch callback', async () => { await consumer.connect() diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index 5d407b2c..34727e0d 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -130,7 +130,7 @@ describe('Consumer', () => { }); consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); - await waitForConsumerToJoinGroup(consumer); + await waitFor(() => consumer.assignment().length > 0, () => null, 100); await producer.connect(); await producer.sendBatch({ diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index bcb1f99f..d7e2f212 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -81,17 +81,21 @@ async function sleep(ms) { } const generateMessages = options => { - const { prefix, number = 100 } = options || {} + const { prefix, number = 100, partition } = options || {} const prefixOrEmpty = prefix ? `-${prefix}` : '' return Array(number) .fill() .map((v, i) => { const value = secureRandom() - return { + const message = { key: `key${prefixOrEmpty}-${i}-${value}`, value: `value${prefixOrEmpty}-${i}-${value}`, + }; + if (partition !== undefined) { + message.partition = partition; } + return message; }) } From 9a2f9c42e4893dc0d1f7ef88d4840c46fa87b944 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 25 Jun 2024 11:23:50 +0530 Subject: [PATCH 159/224] Add message set capability to message cache --- lib/kafkajs/_consumer_cache.js | 80 ++++++++++++++++++ test/promisified/unit/cache.spec.js | 121 ++++++++++++++++++++++++++-- 2 files changed, 196 insertions(+), 5 deletions(-) diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 2149f779..166487f4 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -52,6 +52,26 @@ class PerPartitionMessageCache { next() { return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; } + + /** + * @returns Upto `n` next elements in the cache or an null if none available. + * @warning Does not check for staleness. + */ + nextN(n) { + if (this.currentIndex >= this.cache.length) { + return null; + } + + if (this.currentIndex + n >= this.cache.length) { + const res = this.cache.slice(this.currentIndex); + this.currentIndex = this.cache.length; + return res; + } + + const res = this.cache.slice(this.currentIndex, this.currentIndex + n); + this.currentIndex += n; + return res; + } } @@ -387,6 +407,66 @@ class MessageCache { return null; // Caller is responsible for triggering fetch logic here if next == null. } + /** + * Returns the next `size` elements in the cache as an array, or null if none exists. + * + * @sa next, the behaviour is similar in other aspects. + */ + nextN(idx = -1, size = 1) { + let index = idx; + if (index !== -1 && !this.pendingIndices.has(index)) { + /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible + * that we ran out of messages and fetched a new batch. So we just discard what the user is + * returning to us. */ + this.logger.error("Returning unowned index", idx, "to cache. Discarding it."); + index = -1; + } else if (index !== -1) { + this.pendingIndices.delete(index); + /* We don't add the index back to the this.indices here because we're just going to remove it again the + * first thing in the loop below, so it's slightly better to just avoid doing it. */ + } + + if (index === -1) { + if (this.indices.size() === 0 || this.pendingIndices.size === this.maxConcurrency) { + return null; + } + index = this.indices.pop(); // index cannot be undefined here since indices.size > 0 + } + + /* This loop will always terminate. Why? + * On each iteration: + * 1. We either return (if next is not null). + * 2. We change the PPC index we're interested in, and there are a finite number of PPCs. + * (PPCs don't repeat within the loop since the indices of the PPC are popped from within the + * heap and not put back in, or else a new index is created bounded by ppcList.length). + */ + while (true) { + const next = this.ppcList[index].nextN(size); + if (this.ppcList[index].isStale() || next === null) { + /* If the current PPC is stale or empty, then we move on to the next one. + * It is equally valid to choose any PPC available within this.indices, or else + * move on to the next PPC (maxIndicesIndex + 1) if available. + * We prefer the second option a bit more since we don't have to do a heap operation. */ + const toAdd = this.maxIndicesIndex + 1; + if (toAdd < this.ppcList.length) { + this.maxIndicesIndex = toAdd; + index = toAdd; + } else if (!this.indices.isEmpty()) { + index = this.indices.pop() + } else { + break; // nothing left. + } + continue; + } + + this.pendingIndices.add(index); + /* Arrays are just objects. Setting a property is odd, but not disallowed. */ + next.index = index; + return next; + } + return null; // Caller is responsible for triggering fetch logic here if next == null. + } + /** * Clears the cache completely. * This resets it to a base state, and reduces the capacity of the cache back to 1. diff --git a/test/promisified/unit/cache.spec.js b/test/promisified/unit/cache.spec.js index 379983c4..daf03c60 100644 --- a/test/promisified/unit/cache.spec.js +++ b/test/promisified/unit/cache.spec.js @@ -8,9 +8,6 @@ describe('MessageCache', () => { .fill() .map((_, i) => ({ topic: 'topic', partition: i % 3, number: i })); - beforeEach(() => { - }); - describe("with concurrency", () => { let cache; beforeEach(() => { @@ -37,6 +34,31 @@ describe('MessageCache', () => { expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); }); + it('caches messages and retrieves N of them', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + const expectedFetchedSizes = [11, 11, 8]; + for (let i = 0; i < (90/11); i++) { + /* We choose to fetch 11 messages together rather than 10 so that we can test the case where + * remaining messages > 0 but less than requested size. */ + const next = cache.nextN(nextIdx, 11); + /* There are 30 messages per partition, the first fetch will get 11, the second 11, and the last one + * 8, and then it repeats for each partition. */ + expect(next.length).toBe(expectedFetchedSizes[i % 3]); + expect(next).not.toBeNull(); + receivedMessages.push(...next); + nextIdx = next.index; + } + + /* Results are on a per-partition basis and well-ordered */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + it('does not allow fetching more than 1 message at a time', () => { const msgs = messages.slice(0, 90); cache.addMessages(msgs); @@ -119,8 +141,30 @@ describe('MessageCache', () => { nextIdxs = [next0.index, next1.index]; } - /* Results are on a zig-zag basis. */ - expect(receivedMessages.every((msg, i) => msg.number === receivedMessages.number)); + expect(receivedMessages.length).toBe(60); + expect(receivedMessages.filter(msg => msg.partition === 0).length).toBe(30); + expect(receivedMessages.filter(msg => msg.partition === 1).length).toBe(30); + }); + + it('caches messages and retrieves N of them 2-at-a-time', () => { + const msgs = messages.slice(0, 90).filter(msg => msg.partition !== 3); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdxs = [-1, -1]; + for (let i = 0; i < 30/11; i++) { + const next0 = cache.nextN(nextIdxs[0], 11); + const next1 = cache.nextN(nextIdxs[1], 11); + expect(next0).not.toBeNull(); + expect(next1).not.toBeNull(); + receivedMessages.push(...next0); + receivedMessages.push(...next1); + nextIdxs = [next0.index, next1.index]; + } + + expect(receivedMessages.length).toBe(60); + expect(receivedMessages.filter(msg => msg.partition === 0).length).toBe(30); + expect(receivedMessages.filter(msg => msg.partition === 1).length).toBe(30); }); it('does not allow fetching more than 2 message at a time', () => { @@ -141,6 +185,25 @@ describe('MessageCache', () => { expect(next).not.toBeNull(); }); + + it('does not allow fetching more than 2 message sets at a time', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + let next = cache.nextN(-1, 11); + let savedIndex = next.index; + expect(next).not.toBeNull(); + next = cache.nextN(-1, 11); + expect(next).not.toBeNull(); + next = cache.nextN(-1, 11); + expect(next).toBeNull(); + expect(cache.pendingSize()).toBe(2); + + // Fetch after returning index works. + next = cache.nextN(savedIndex, 11); + expect(next).not.toBeNull(); + }); + it('stops fetching from stale partition', () => { const msgs = messages.slice(0, 90); cache.addMessages(msgs); @@ -163,6 +226,29 @@ describe('MessageCache', () => { expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 3))); }); + it('stops fetching message sets from stale partition', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + for (let i = 0; i < 3; i++) { + const next = cache.nextN(nextIdx, 11); + expect(next).not.toBeNull(); + receivedMessages.push(...next); + nextIdx = next.index; + cache.markStale([{topic: next[0].topic, partition: next[0].partition}]); + } + + // We should not be able to get anything more. + expect(cache.nextN(nextIdx, 11)).toBeNull(); + // Nothing should be pending, we've returned everything. + expect(cache.pendingSize()).toBe(0); + // The first [11, 11, 11] messages from different toppars. + expect(receivedMessages.length).toBe(33); + expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 33))); + }); + it('one slow processing message should not slow down others', () => { const msgs = messages.slice(0, 90); cache.addMessages(msgs); @@ -188,6 +274,31 @@ describe('MessageCache', () => { expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); }); + it('one slow processing message set should not slow down others', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let nextIdx = -1; + const slowMsg = cache.nextN(nextIdx, 11); + for (let i = 0; i < 60/11; i++) { /* 60 - for non-partition 0 msgs */ + const next = cache.nextN(nextIdx, 11); + expect(next).not.toBeNull(); + receivedMessages.push(...next); + nextIdx = next.index; + } + + + // We should not be able to get anything more. + expect(cache.nextN(nextIdx, 11)).toBeNull(); + // The slowMsg should be pending. + expect(cache.pendingSize()).toBe(1); + + /* Messages should be partition-wise and well-ordered. */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + it('should not be able to handle cache-clearance in the middle of processing', () => { const msgs = messages.slice(0, 90); cache.addMessages(msgs); From 2f4bd76825504ce550c6aa6588ce93b93ea43d5b Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 1 Jul 2024 09:18:00 +0530 Subject: [PATCH 160/224] Add naive batching (without resolution handling) --- lib/kafkajs/_consumer.js | 161 +++++++++++++++++++++++++++------------ 1 file changed, 111 insertions(+), 50 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index f5666aa6..19a7945f 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -617,52 +617,60 @@ class Consumer { } /** - * Converts a message returned by node-rdkafka into a message that can be used by the eachBatch callback. - * @param {import("../..").Message} message + * Converts a list of messages returned by node-rdkafka into a message that can be used by the eachBatch callback. + * @param {import("../..").Message[]} messages - must not be empty. Must contain messages from the same topic and partition. * @returns {import("../../types/kafkajs").EachBatchPayload} - * @note Unlike the KafkaJS consumer, a batch here is for API compatibility only. It is always a single message. */ - #createBatchPayload(message) { - let key = message.key; - if (typeof key === 'string') { - key = Buffer.from(key); - } + #createBatchPayload(messages) { + const topic = messages[0].topic; + const partition = messages[0].partition; + + const messagesConverted = []; + for (let i = 0; i < messages.length; i++) { + const message = messages[i]; + let key = message.key; + if (typeof key === 'string') { + key = Buffer.from(key); + } - let timestamp = message.timestamp ? String(message.timestamp) : ''; + let timestamp = message.timestamp ? String(message.timestamp) : ''; - let headers; - if (message.headers) { - headers = {} - for (const [key, value] of Object.entries(message.headers)) { - if (!Object.hasOwn(headers, key)) { - headers[key] = value; - } else if (headers[key].constructor === Array) { - headers[key].push(value); - } else { - headers[key] = [headers[key], value]; + let headers; + if (message.headers) { + headers = {} + for (const [key, value] of Object.entries(message.headers)) { + if (!Object.hasOwn(headers, key)) { + headers[key] = value; + } else if (headers[key].constructor === Array) { + headers[key].push(value); + } else { + headers[key] = [headers[key], value]; + } } } - } - const messageConverted = { - key, - value: message.value, - timestamp, - attributes: 0, - offset: String(message.offset), - size: message.size, - leaderEpoch: message.leaderEpoch, - headers - }; + const messageConverted = { + key, + value: message.value, + timestamp, + attributes: 0, + offset: String(message.offset), + size: message.size, + leaderEpoch: message.leaderEpoch, + headers + }; + + messagesConverted.push(messageConverted); + } const batch = { - topic: message.topic, - partition: message.partition, + topic, + partition, highWatermark: '-1001', // Invalid - we don't fetch it - messages: [messageConverted], + messages: messagesConverted, isEmpty: () => false, - firstOffset: () => messageConverted.offset, - lastOffset: () => messageConverted.offset, + firstOffset: () => messagesConverted[0].offset, + lastOffset: () => messagesConverted[messagesConverted.length - 1].offset, offsetLag: () => notImplemented(), offsetLagLow: () => notImplemented(), }; @@ -672,7 +680,7 @@ class Consumer { _messageResolved: false, resolveOffset: () => { returnPayload._messageResolved = true; }, heartbeat: async () => { /* no op */ }, - pause: this.pause.bind(this, [{ topic: message.topic, partitions: [message.partition] }]), + pause: this.pause.bind(this, [{ topic, partitions: [partition] }]), commitOffsetsIfNecessary: async () => { /* no op */ }, uncommittedOffsets: () => notImplemented(), isRunning: () => this.#running, @@ -685,7 +693,7 @@ class Consumer { /** * Consumes a single message from the internal consumer. * @param {number} savedIndex - the index of the message in the cache to return. - * @returns {Promise} a promise that resolves to a single message. + * @returns {Promise} a promise that resolves to a single message or null. * @note this method caches messages as well, but returns only a single message. */ async #consumeSingleCached(savedIndex) { @@ -726,6 +734,52 @@ class Consumer { }); } + /** + * Consumes a single message from the internal consumer. + * @param {number} savedIndex - the index of the message in the cache to return. + * @param {number} size - the number of messages to fetch. + * @returns {Promise} a promise that resolves to a list of messages or null. + * @note this method caches messages as well. + * @sa #consumeSingleCached + */ + async #consumeCachedN(savedIndex, size) { + const msgs = this.#messageCache.nextN(savedIndex, size); + if (msgs) { + return msgs; + } + + /* It's possible that we get msgs = null, but that's because partitionConcurrency + * exceeds the number of partitions containing messages. So in this case, + * we should not call for new fetches, rather, try to focus on what we have left. + */ + if (!msgs && this.#messageCache.pendingSize() !== 0) { + return null; + } + + if (this.#fetchInProgress) { + return null; + } + + this.#fetchInProgress = true; + return new Promise((resolve, reject) => { + this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { + this.#fetchInProgress = false; + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + return; + } + this.#messageCache.addMessages(messages); + const msgsList = this.#messageCache.nextN(-1, size); + if (messages.length === this.#messageCache.maxSize) { + this.#messageCache.increaseMaxSize(); + } else { + this.#messageCache.decreaseMaxSize(messages.length); + } + resolve(msgsList); + }); + }); + } + /** * Consumes n messages from the internal consumer. * @returns {Promise} a promise that resolves to a list of messages. @@ -964,15 +1018,15 @@ class Consumer { } /** - * Processes a batch message (a single message as of now). + * Processes a batch of messages. * - * @param m Message as obtained from #consumeSingleCached. + * @param ms Messages as obtained from #consumeCachedN. * @param config Config as passed to run(). * @returns {Promise} the cache index of the message that was processed. */ - async #batchProcessor(m, config) { + async #batchProcessor(ms, config) { let eachMessageProcessed = false; - const payload = this.#createBatchPayload(m); + const payload = this.#createBatchPayload(ms); try { await config.eachBatch(payload); if (config.eachBatchAutoResolve) { @@ -1005,23 +1059,25 @@ class Consumer { } /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ + /* TODO: currently we're seeking to just the first offset. Fix this to take care of messages we are resolving. */ if (!eachMessageProcessed) { await this.seek({ - topic: m.topic, - partition: m.partition, - offset: m.offset, + topic: ms[0].topic, + partition: ms[0].partition, + offset: ms[0].offset, }); } /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ + /* TODO: currently we just store the last offset of the batch. Fix it to store the last resolved one + 1. */ if (eachMessageProcessed) { try { if (!this.#userManagedStores) { this.#internalClient.offsetsStore([{ - topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch + topic: ms[ms.length - 1].topic, partition: ms[ms.length - 1].partition, offset: Number(ms[ms.length - 1].offset) + 1, leaderEpoch: ms[ms.length - 1].leaderEpoch }]); } - this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); + this.#lastConsumedOffsets.set(partitionKey(ms[ms.length - 1]), Number(ms[ms.length - 1].offset) + 1); } catch (e) { /* Not much we can do, except log the error. */ if (this.#logger) @@ -1035,7 +1091,7 @@ class Consumer { if (this.#checkPendingSeeks) await this.#seekInternal(); - return m.index; + return ms.index; } /** @@ -1050,7 +1106,7 @@ class Consumer { * * Worker termination acts as a async barrier. */ - async #worker(config, perMessageProcessor, id) { + async #worker(config, perMessageProcessor, fetcher, id) { let nextIdx = -1; while (!this.#workerTerminationScheduled) { /* Invalidate the message cache if needed */ @@ -1066,7 +1122,7 @@ class Consumer { continue; } - const m = await this.#consumeSingleCached(nextIdx).catch(e => { + const m = await fetcher(nextIdx).catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) @@ -1097,13 +1153,18 @@ class Consumer { async #runInternal(config) { this.#concurrency = config.partitionsConsumedConcurrently; const perMessageProcessor = config.eachMessage ? this.#messageProcessor : this.#batchProcessor; + /* TODO: make this dynamic, based on max batch size / size of last message seen. */ + const maxBatchSize = 30; + const fetcher = config.eachMessage + ? (savedIdx) => this.#consumeSingleCached(savedIdx) + : (savedIdx) => this.#consumeCachedN(savedIdx, maxBatchSize); this.#workers = []; while (!(await acquireOrLog(this.#lock, this.#logger))); while (!this.#disconnectStarted) { this.#workerTerminationScheduled = false; const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); - this.#workers = Array(workersToSpawn).fill().map((_, i) => this.#worker(config, perMessageProcessor.bind(this), i)); + this.#workers = Array(workersToSpawn).fill().map((_, i) => this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this), i)); await Promise.all(this.#workers); /* One of the possible reasons for the workers to end is that the cache is globally stale. From b2b546a0a062291c6ea5dd4bdf5c0b9b23761dd6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 2 Jul 2024 16:17:47 +0530 Subject: [PATCH 161/224] Add batch staleness, resolution, and offset management to eachBatch --- lib/kafkajs/_consumer.js | 189 ++++++++++++++++++++++++++++++--------- 1 file changed, 149 insertions(+), 40 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 19a7945f..137020df 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -130,6 +130,11 @@ class Consumer { */ #userManagedStores = false; + /** + * Whether the user has enabled manual offset management (commits). + */ + #autoCommit = false; + /** * Signals an intent to disconnect the consumer. */ @@ -161,6 +166,16 @@ class Consumer { */ #fetchInProgress = false; + /** + * Maps topic-partition key to the batch payload for marking staleness. + * + * Only used with eachBatch. + * NOTE: given that size of this map will never exceed #concurrency, a + * linear search might actually be faster over what will generally be <10 elems. + * But a map makes conceptual sense. Revise at a later point if needed. + */ + #topicPartitionToBatchPayload = new Map(); + /** * TODO: remove this or make it a bit more reliable. * This is a debug property for this branch. @@ -540,6 +555,12 @@ class Consumer { this.#userManagedStores = !rdKafkaConfig['enable.auto.offset.store']; } + if (!Object.hasOwn(rdKafkaConfig, 'enable.auto.commit')) { + this.#autoCommit = true; /* librdkafka default. */ + } else { + this.#autoCommit = rdKafkaConfig['enable.auto.commit']; + } + return rdKafkaConfig; } @@ -616,6 +637,66 @@ class Consumer { }; } + /** + * Method used by #createBatchPayload to resolve offsets. + * Resolution stores the offset into librdkafka if needed, and into the lastConsumedOffsets map + * that we use for seeking to the last consumed offset when forced to clear cache. + * + * @param {*} payload The payload we're creating. This is a method attached to said object. + * @param {*} offsetToResolve The offset to resolve. + * @param {*} leaderEpoch The leader epoch of the message (optional). We expect users to provide it, but for API-compatibility reasons, it's optional. + */ + #eachBatchPayload_resolveOffsets(payload, offsetToResolve, leaderEpoch = -1) { + const offset = +offsetToResolve; + + if (isNaN(offset)) { + /* Not much we can do but throw and log an error. */ + const e = new error.KafkaJSError(`Invalid offset to resolve: ${offsetToResolve}`, { code: error.ErrorCodes.ERR__INVALID_ARG }); + throw e; + } + + /* The user might resolve offset N (< M) after resolving offset M. Given that in librdkafka we can only + * store one offset, store the last possible one. */ + if (offset <= payload._lastResolvedOffset.offset) + return; + + const topic = payload.batch.topic; + const partition = payload.batch.partition; + const key = partitionKey({ topic, partition }); + + payload._lastResolvedOffset = { offset, leaderEpoch }; + + try { + if (!this.#userManagedStores) { + this.#internalClient.offsetsStore([{ + topic, + partition, + offset: offset + 1, + leaderEpoch: leaderEpoch, + }]); + } + this.#lastConsumedOffsets.set(key, offset + 1); + } catch (e) { + /* Not much we can do, except log the error. */ + if (this.#logger) + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${e}:${e.stack}`); + } + } + + /** + * Method used by #createBatchPayload to commit offsets. + */ + async #eachBatchPayload_commitOffsetsIfNecessary() { + if (this.#autoCommit) { + /* librdkafka internally handles committing of whatever we store. + * We don't worry about it here. */ + return; + } + /* If the offsets are being resolved by the user, they've already called resolveOffset() at this point + * We just need to commit the offsets that we've stored. */ + await this.commitOffsets(); + } + /** * Converts a list of messages returned by node-rdkafka into a message that can be used by the eachBatch callback. * @param {import("../..").Message[]} messages - must not be empty. Must contain messages from the same topic and partition. @@ -666,27 +747,31 @@ class Consumer { const batch = { topic, partition, - highWatermark: '-1001', // Invalid - we don't fetch it + highWatermark: '-1001', /* We don't fetch it yet. We can call committed() to fetch it but that might incur network calls. */ messages: messagesConverted, isEmpty: () => false, - firstOffset: () => messagesConverted[0].offset, - lastOffset: () => messagesConverted[messagesConverted.length - 1].offset, + firstOffset: () => (messagesConverted[0].offset).toString(), + lastOffset: () => (messagesConverted[messagesConverted.length - 1].offset).toString(), offsetLag: () => notImplemented(), offsetLagLow: () => notImplemented(), }; const returnPayload = { batch, - _messageResolved: false, - resolveOffset: () => { returnPayload._messageResolved = true; }, + _stale: false, + _lastResolvedOffset: { offset: -1, leaderEpoch: -1 }, heartbeat: async () => { /* no op */ }, pause: this.pause.bind(this, [{ topic, partitions: [partition] }]), - commitOffsetsIfNecessary: async () => { /* no op */ }, - uncommittedOffsets: () => notImplemented(), + commitOffsetsIfNecessary: this.#eachBatchPayload_commitOffsetsIfNecessary.bind(this), isRunning: () => this.#running, - isStale: () => false, + isStale: () => returnPayload._stale, + /* NOTE: Probably never to be implemented. Not sure exactly how we'd compute this + * inexpensively. */ + uncommittedOffsets: () => notImplemented(), }; + returnPayload.resolveOffset = this.#eachBatchPayload_resolveOffsets.bind(this, returnPayload); + return returnPayload; } @@ -1020,20 +1105,33 @@ class Consumer { /** * Processes a batch of messages. * - * @param ms Messages as obtained from #consumeCachedN. + * @param ms Messages as obtained from #consumeCachedN (ms.length !== 0). * @param config Config as passed to run(). * @returns {Promise} the cache index of the message that was processed. */ async #batchProcessor(ms, config) { - let eachMessageProcessed = false; + const key = partitionKey(ms[0]); const payload = this.#createBatchPayload(ms); + + this.#topicPartitionToBatchPayload.set(key, payload); + + let lastOffsetProcessed = { offset: -1, leaderEpoch: -1 }; + const lastOffset = +(ms[ms.length - 1].offset); + const lastLeaderEpoch = ms[ms.length - 1].leaderEpoch; try { await config.eachBatch(payload); - if (config.eachBatchAutoResolve) { - eachMessageProcessed = true; - } else { - eachMessageProcessed = payload._messageResolved; + + /* If the user isn't resolving offsets, we resolve them here. It's significant here to call this method + * because besides updating `payload._lastResolvedOffset`, this method is also storing the offsets to + * librdkafka, and accounting for any cache invalidations. + * Don't bother resolving offsets if payload became stale at some point. We can't know when the payload + * became stale, so either the user has been nice enough to keep resolving messages, or we must seek to + * the first offset to ensure no message loss. */ + if (config.eachBatchAutoResolve && !payload._stale) { + payload.resolveOffset(lastOffset, lastLeaderEpoch); } + + lastOffsetProcessed = payload._lastResolvedOffset; } catch (e) { /* It's not only possible, but expected that an error will be thrown by eachBatch. * This is especially true since the pattern of pause() followed by throwing an error @@ -1053,38 +1151,24 @@ class Consumer { this.#logger.error(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed - * despite an error is if the user says so, and the user can use resolveOffsets for both the possible + * despite an error is if the user says so, and the user can use resolveOffset for both the possible * values eachBatchAutoResolve can take. */ - eachMessageProcessed = payload._messageResolved; + lastOffsetProcessed = payload._lastResolvedOffset; } - /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ - /* TODO: currently we're seeking to just the first offset. Fix this to take care of messages we are resolving. */ - if (!eachMessageProcessed) { + this.#topicPartitionToBatchPayload.delete(key); + + /* If any message is unprocessed, either due to an error or due to the user not marking it processed, we must seek + * back to get it so it can be reprocessed. */ + if (lastOffsetProcessed.offset !== lastOffset) { + const offsetToSeekTo = lastOffsetProcessed.offset === -1 ? ms[0].offset : (lastOffsetProcessed.offset + 1); await this.seek({ topic: ms[0].topic, partition: ms[0].partition, - offset: ms[0].offset, + offset: offsetToSeekTo, }); } - /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ - /* TODO: currently we just store the last offset of the batch. Fix it to store the last resolved one + 1. */ - if (eachMessageProcessed) { - try { - if (!this.#userManagedStores) { - this.#internalClient.offsetsStore([{ - topic: ms[ms.length - 1].topic, partition: ms[ms.length - 1].partition, offset: Number(ms[ms.length - 1].offset) + 1, leaderEpoch: ms[ms.length - 1].leaderEpoch - }]); - } - this.#lastConsumedOffsets.set(partitionKey(ms[ms.length - 1]), Number(ms[ms.length - 1].offset) + 1); - } catch (e) { - /* Not much we can do, except log the error. */ - if (this.#logger) - this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); - } - } - /* 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. */ @@ -1164,8 +1248,18 @@ class Consumer { while (!this.#disconnectStarted) { this.#workerTerminationScheduled = false; const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); - this.#workers = Array(workersToSpawn).fill().map((_, i) => this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this), i)); - await Promise.all(this.#workers); + this.#workers = + Array(workersToSpawn) + .fill() + .map((_, i) => + this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this), i) + .catch(e => { + if (this.#logger) + this.#logger.error(`Worker ${i} encountered an error: ${e}:${e.stack}`); + })); + + /* Best we can do is log errors on worker issues - handled by the catch block above. */ + await Promise.allSettled(this.#workers) /* One of the possible reasons for the workers to end is that the cache is globally stale. * We need to take care of expiring it. */ @@ -1416,7 +1510,17 @@ class Consumer { } this.#checkPendingSeeks = true; - this.#pendingSeeks.set(partitionKey(rdKafkaTopicPartitionOffset), rdKafkaTopicPartitionOffset.offset); + const key = partitionKey(rdKafkaTopicPartitionOffset) + this.#pendingSeeks.set(key, rdKafkaTopicPartitionOffset.offset); + + /* Only for eachBatch: + * Immediately mark the batch it's associated with as stale, even if we don't + * do the actual 'seekInternal' at this time. This is because we need read-after-write + * consistency for eachBatch, and calling seek(toppar) from within eachBatch(toppar) + * should change the result of batch.isStale() immediately. */ + if (this.#topicPartitionToBatchPayload.has(key)) { + this.#topicPartitionToBatchPayload.get(key)._stale = true; + } } async describeGroup() { @@ -1490,6 +1594,11 @@ class Consumer { * making it unusable. */ this.#messageCache.markStale(topics); + /* If anyone's using eachBatch, mark the batch as stale. */ + topics.map(partitionKey) + .filter(key => this.#topicPartitionToBatchPayload.has(key)) + .forEach(key => this.#topicPartitionToBatchPayload.get(key)._stale = true); + topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); return () => this.resume(topics); From 0848e7f428ac06861aa64b15e3eb31e31d465110 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 2 Jul 2024 16:20:55 +0530 Subject: [PATCH 162/224] Update tests for true eachBatch Also cleans up some excessive parametrization. Separates tests into files so they can be parallelized better. wq --- test/promisified/consumer/commit.spec.js | 75 +++ .../consumer/consumeMessages.spec.js | 622 ++---------------- .../consumer/consumerCacheTests.spec.js | 22 +- .../consumer/consumerTransactions.spec.js | 418 ++++++++++++ .../consumer/groupInstanceId.spec.js | 7 +- test/promisified/consumer/seek.spec.js | 241 ++++++- .../producer/concurrentSend.spec.js | 45 ++ 7 files changed, 857 insertions(+), 573 deletions(-) create mode 100644 test/promisified/consumer/consumerTransactions.spec.js create mode 100644 test/promisified/producer/concurrentSend.spec.js diff --git a/test/promisified/consumer/commit.spec.js b/test/promisified/consumer/commit.spec.js index 1684f138..2f28deaf 100644 --- a/test/promisified/consumer/commit.spec.js +++ b/test/promisified/consumer/commit.spec.js @@ -149,4 +149,79 @@ describe('Consumer commit', () => { { topic: topicName, partition: 2, offset: '10', metadata,leaderEpoch: expect.any(Number) } ]); }); + + it.each([[true], [false]])('should commit only resolved offsets while using eachBatch', async (isAutoCommit) => { + /* Evenly distribute 3*30 messages across 3 partitions */ + const numMsgs = 30; + let i = 0; + const messages = Array(3 * numMsgs) + .fill() + .map(() => { + const value = secureRandom() + return { value: `value-${value}`, partition: (i++) % 3 } + }) + + await producer.connect(); + await producer.send({ topic: topicName, messages }) + await producer.flush(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: isAutoCommit, + autoCommitInterval: 500, + }); + + let msgCount = 0; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }) + await consumer.run({ + eachBatchAutoResolve: false, + eachBatch: async ({ batch, resolveOffset, commitOffsetsIfNecessary }) => { + for (const message of batch.messages) { + msgCount++; + if ((+message.offset) < numMsgs/2) { + resolveOffset(message.offset); + } + } + if (!isAutoCommit) + await commitOffsetsIfNecessary(); + } + }); + await waitFor(() => msgCount >= (3 * numMsgs), () => null, { delay: 100 }); + + /* Disconnect should commit any uncommitted offsets */ + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + }); + + await consumer.connect(); + const toppars = Array(3).fill().map((_, i) => ({ topic: topicName, partition: i })); + const committed = await consumer.committed(toppars); + const halfOffset = Math.floor(numMsgs/2).toString(); + expect(committed).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 0, + offset: halfOffset, + }), + expect.objectContaining({ + topic: topicName, + partition: 1, + offset: halfOffset, + }), + expect.objectContaining({ + topic: topicName, + partition: 2, + offset: halfOffset, + }) + ]) + ) + }); }); diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 479775b8..d03d6bce 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -1,8 +1,6 @@ jest.setTimeout(30000) -const { ErrorCodes, CompressionTypes } = require('../../../lib').KafkaJS; - -const { doesNotMatch } = require('assert'); +const { CompressionTypes } = require('../../../lib').KafkaJS; const { secureRandom, createTopic, @@ -12,18 +10,17 @@ const { waitForMessages, waitForConsumerToJoinGroup, sleep, - generateMessages, } = require('../testhelpers'); -/* All combinations of autoCommit and partitionsConsumedConcurrently */ -const cases = Array(2 * 3).fill().map((_, i) => [i < 3, (i % 3) + 1]).slice(-1); +/* All variations of partitionsConsumedConcurrently */ +const cases = Array(3).fill().map((_, i) => [(i % 3) + 1]); -describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) => { +describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { let topicName, groupId, producer, consumer; const partitions = 3; beforeEach(async () => { - console.log("Starting:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); + console.log("Starting:", expect.getState().currentTestName, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); topicName = `test-topic-${secureRandom()}` groupId = `consumer-group-id-${secureRandom()}` @@ -34,14 +31,14 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) groupId, maxWaitTimeInMs: 100, fromBeginning: true, - autoCommit: isAutoCommit, + autoCommit: true, }); }); afterEach(async () => { consumer && (await consumer.disconnect()) producer && (await producer.disconnect()) - console.log("Ending:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); + console.log("Ending:", expect.getState().currentTestName, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); }); it('consume messages', async () => { @@ -157,7 +154,7 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) // If we're not auto-resolving, we need to resolve the offsets manually. if (!isAutoResolve) - event.resolveOffset(); + event.resolveOffset(event.batch.messages[event.batch.messages.length - 1].message.offset); } }); @@ -204,6 +201,52 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) }); + it('partially resolving offsets in eachBatch leads to reconsumption', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + const skippedResolutionForPartition = Array(partitions).fill(false); + const messagesPerPartition = 100; + consumer.run({ + partitionsConsumedConcurrently, + eachBatchAutoResolve: false, + eachBatch: async (event) => { + const partition = event.batch.partition; + let maxOffset = -1; + for (const message of event.batch.messages) { + const offset = +message.offset; + maxOffset = offset; + messagesConsumed.push(message); + /* If we get a message greater than the halfway point, don't resolve it the first time around + * Only resolve it when we see it the second time. */ + if (offset < Math.floor(messagesPerPartition/2) || skippedResolutionForPartition[partition]) { + event.resolveOffset(offset); + } + } + /* If we've completed the first half of messages, then we are now allowed to resolve + * the second half. */ + if (maxOffset >= Math.floor(messagesPerPartition/2)) + skippedResolutionForPartition[partition] = true; + } + }); + + const messages = Array(messagesPerPartition * partitions) + .fill() + .map((_, i) => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions } + }) + + await producer.send({ topic: topicName, messages }); + + /* It's not possible to actually know the exact number of messages without knowing the + * cache growth characteristics, which may change in the future. So just check if there + * is at least 1 message more than we sent. */ + await waitForMessages(messagesConsumed, { number: messages.length + 1 }); + }); + it('is able to reconsume messages after not resolving it', async () => { await consumer.connect(); await producer.connect(); @@ -227,7 +270,7 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) messagesConsumed.push(...event.batch.messages); // Since we're not auto-resolving, we need to resolve the offsets manually. - event.resolveOffset(); + event.resolveOffset(event.batch.messages[event.batch.messages.length - 1].offset); } }); @@ -290,7 +333,7 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) eachBatch: async event => { messagesConsumed.push(...event.batch.messages); // Resolve offsets irrespective of the value of eachBatchAutoResolve. - event.resolveOffset(); + event.resolveOffset(event.batch.messages[event.batch.messages.length - 1].offset); throw new Error('a new error.'); } }); @@ -395,7 +438,7 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) ]) }); - /* Skip as it uses consuimer events. */ + /* Skip as it uses consumer events. */ it.skip('commits the last offsets processed before stopping', async () => { jest.spyOn(cluster, 'refreshMetadataIfNecessary') @@ -482,142 +525,6 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) 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, - fromBeginning: true, - }); - - const messages = Array(10) - .fill() - .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: 0 } - }); - - 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 the current implementation will never fetch more than 1 message. */ - 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, - fromBeginning: true, - autoCommit: isAutoCommit, - }) - - 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 consumer events */ - 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 consumer events */ it.skip('discards messages received when pausing while fetch is in-flight', async () => { consumer = createConsumer({ @@ -702,419 +609,4 @@ describe.each(cases)('Consumer', (isAutoCommit, partitionsConsumedConcurrently) await producer.disconnect(); }); - - describe('transactions', () => { - it('accepts messages from an idempotent producer', async () => { - producer = createProducer({ - idempotent: true, - maxInFlightRequests: 1, - }) - - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - autoCommit: isAutoCommit - }); - - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }); - - const messagesConsumed = [] - const idempotentMessages = generateMessages({ prefix: 'idempotent', partition: 0 }) - - 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, - fromBeginning: true, - autoCommit: isAutoCommit, - }); - - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }); - - const messagesConsumed = []; - - const messages1 = generateMessages({ prefix: 'txn1', partition: 0 }); - const messages2 = generateMessages({ prefix: 'txn2', partition: 0 }); - const nontransactionalMessages1 = generateMessages({ prefix: 'nontransactional1', number: 1, partition: 0 }); - - 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, - fromBeginning: true, - autoCommit: isAutoCommit, - }); - - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }); - - const messagesConsumed = [] - - const abortedMessages1 = generateMessages({ prefix: 'aborted-txn-1', partition: 0 }); - const abortedMessages2 = generateMessages({ prefix: 'aborted-txn-2', partition: 0 }); - const committedMessages = generateMessages({ prefix: 'committed-txn', number: 10, partition: 0 }); - - 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, - fromBeginning: true, - autoCommit: isAutoCommit, - }) - - await consumer.connect(); - await producer.connect(); - await consumer.subscribe({ topic: topicName }); - - const messagesConsumed = []; - - const abortedMessages = generateMessages({ prefix: 'aborted-txn1', partition: 0 }); - - 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 () => { - if (isAutoCommit) { /* only autoCommit: false makes sense for this test. */ - return; - } - // 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, - fromBeginning: true, - autoCommit: 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, - }) - - // 2. 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, - fromBeginning: true, - autoCommit: false, - }); - - 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 () => { - if (isAutoCommit) { /* only autoCommit: false makes sense for this test. */ - return; - } - - // 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, - fromBeginning: true, - autoCommit: 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 }); - - 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 - we cannot stop it, so we recreate it. */ - messagesConsumed = [] - uncommittedOffsetsPerMessage = [] - - await consumer.disconnect(); - - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - autoCommit: false, - }); - - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - - 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/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index d51f9919..f40b775d 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -10,8 +10,13 @@ const { sleep, } = require('../testhelpers'); -/* All combinations of autoCommit and partitionsConsumedConcurrently */ -const cases = Array(2 * 3).fill().map((_, i) => [i % 2 === 0, (i % 3) + 1]); +/* All required combinations of [autoCommit, partitionsConsumedConcurrently] */ +const cases = [ + [true, 1], + [true, 3], + [false, 1], + [false, 3], +]; describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumedConcurrently) => { let topicName, groupId, producer, consumer; @@ -25,7 +30,6 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed producer = createProducer({}); - const common = {}; consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, @@ -133,6 +137,18 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed }); it('is cleared before rebalance', async () => { + /* If another test times out, jest chooses to run this test in parallel with + * the other test. I think this causes an issue with shared groupIds. So to ensure + * the consumers are created with the same groupId, we create them here. + * TODO: verify correctness of theory. It's conjecture... which solves flakiness. */ + let groupId = `consumer-group-id-${secureRandom()}`; + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: isAutoCommit, + }); + const consumer2 = createConsumer({ groupId, maxWaitTimeInMs: 100, diff --git a/test/promisified/consumer/consumerTransactions.spec.js b/test/promisified/consumer/consumerTransactions.spec.js new file mode 100644 index 00000000..4afd8bbf --- /dev/null +++ b/test/promisified/consumer/consumerTransactions.spec.js @@ -0,0 +1,418 @@ +jest.setTimeout(10000); + +const { ErrorCodes } = require('../../../lib').KafkaJS; +const { + secureRandom, + createTopic, + createProducer, + createConsumer, + waitForMessages, + generateMessages, +} = require('../testhelpers'); + +describe('Consumer transactions', () => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}` + + await createTopic({ topic: topicName }) + producer = createProducer({ + idempotent: true, + maxInFlightRequests: 1, + }); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: true, + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }); + + it('accepts messages from an idempotent producer', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = [] + const idempotentMessages = generateMessages({ prefix: 'idempotent', partition: 0 }) + + 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, + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + const messages1 = generateMessages({ prefix: 'txn1', partition: 0 }); + const messages2 = generateMessages({ prefix: 'txn2', partition: 0 }); + const nontransactionalMessages1 = generateMessages({ prefix: 'nontransactional1', number: 1, partition: 0 }); + + 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, + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = [] + + const abortedMessages1 = generateMessages({ prefix: 'aborted-txn-1', partition: 0 }); + const abortedMessages2 = generateMessages({ prefix: 'aborted-txn-2', partition: 0 }); + const committedMessages = generateMessages({ prefix: 'committed-txn', number: 10, partition: 0 }); + + 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, + fromBeginning: true, + autoCommit: true, + }) + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + const abortedMessages = generateMessages({ prefix: 'aborted-txn1', partition: 0 }); + + 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, + fromBeginning: true, + autoCommit: 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, + }) + + // 2. 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, + fromBeginning: true, + autoCommit: false, + }); + + 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, + fromBeginning: true, + autoCommit: 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 }); + + 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 - we cannot stop it, so we recreate it. */ + messagesConsumed = [] + uncommittedOffsetsPerMessage = [] + + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + autoCommit: false, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + 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/) + } + ); +}); \ No newline at end of file diff --git a/test/promisified/consumer/groupInstanceId.spec.js b/test/promisified/consumer/groupInstanceId.spec.js index c905d95a..b3477888 100644 --- a/test/promisified/consumer/groupInstanceId.spec.js +++ b/test/promisified/consumer/groupInstanceId.spec.js @@ -11,13 +11,14 @@ describe('Consumer with static membership', () => { let consumer; let groupId, topicName; - const consumerConfig = { - groupId, - }; + let consumerConfig; beforeEach(async () => { topicName = `test-topic1-${secureRandom()}`; groupId = `consumer-group-id-${secureRandom()}` + consumerConfig = { + groupId, + }; consumer = null; await createTopic({ topic: topicName, partitions: 2 }); }); diff --git a/test/promisified/consumer/seek.spec.js b/test/promisified/consumer/seek.spec.js index f3ba6a47..c400ed3b 100644 --- a/test/promisified/consumer/seek.spec.js +++ b/test/promisified/consumer/seek.spec.js @@ -6,12 +6,15 @@ const { secureRandom, createTopic, waitForMessages, + waitFor, + sleep, } = require('../testhelpers') -describe('Consumer', () => { +describe('Consumer seek >', () => { let topicName, groupId, producer, consumer; beforeEach(async () => { + console.log("Starting:", expect.getState().currentTestName); topicName = `test-topic-${secureRandom()}`; groupId = `consumer-group-id-${secureRandom()}`; @@ -26,6 +29,7 @@ describe('Consumer', () => { afterEach(async () => { consumer && (await consumer.disconnect()); producer && (await producer.disconnect()); + console.log("Ending:", expect.getState().currentTestName); }); describe('when seek offset', () => { @@ -82,7 +86,7 @@ describe('Consumer', () => { }); - describe('When "enable.auto.commit" is false', () => { + describe('when "enable.auto.commit" is false', () => { beforeEach(() => { consumer = createConsumer({ groupId, @@ -342,4 +346,237 @@ describe('Consumer', () => { }); }); }); + + describe('batch staleness >', () => { + it('stops consuming messages after staleness', async () => { + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 500, + fromBeginning: true, + }); + + const messages = Array(10) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}`, partition: 0 } + }); + + 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]) + }); + + it('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, + fromBeginning: true, + autoCommit: true, + }) + + 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, 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 waitFor(() => offsetsConsumed.length >= 2, () => null, { delay: 50 }) + + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + }); + + it('resolves a batch as stale when seek is called from outside eachBatch', async () => { + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 500, + fromBeginning: true, + autoCommit: true, + }) + + const messages = Array(10) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}`, partition: 0 } + }); + + 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, resolveOffset }) => { + for (const message of batch.messages) { + if (isStale()) break; + + offsetsConsumed.push(message.offset) + + /* Slow things down so we can call seek predictably. */ + await sleep(1000); + + resolveOffset(message.offset) + } + }, + }) + + await waitFor(() => offsetsConsumed.length === 1, () => null, { delay: 50 }) + consumer.seek({ topic: topicName, partition: 0, offset: offsetsConsumed[0] }); + + await waitFor(() => offsetsConsumed.length >= 2, () => null, { delay: 50 }) + + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + }); + + it('resolves a batch as stale when pause was called while processing it', async () => { + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 500, + fromBeginning: true, + autoCommit: true, + }) + + const numMessages = 100; + const messages = Array(numMessages) + .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 = []; + + let resume; + consumer.run({ + eachBatchAutoResolve: true, + eachBatch: async ({ batch, isStale, resolveOffset, pause }) => { + for (const message of batch.messages) { + if (isStale()) break; + + offsetsConsumed.push(message.offset) + + if (offsetsConsumed.length === Math.floor(numMessages/2)) { + resume = pause(); + } + + resolveOffset(message.offset); + } + }, + }) + + /* Despite eachBatchAutoResolve being true, it shouldn't resolve offsets on its own. + * However, manual resolution of offsets should still count. */ + await waitFor(() => offsetsConsumed.length >= numMessages/2, () => null, { delay: 50 }); + + resume(); + + /* Since we've properly resolved all offsets before pause, including the offset that we paused at, + * there is no repeat. */ + await waitFor(() => offsetsConsumed.length >= numMessages, () => null, { delay: 50 }) + expect(offsetsConsumed.length).toBe(numMessages); + + expect(+offsetsConsumed[Math.floor(numMessages/2)]).toEqual(+offsetsConsumed[Math.floor(numMessages/2) + 1] - 1) + }); + + /* Skip as it uses consumer events */ + 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) + }); + }); }) diff --git a/test/promisified/producer/concurrentSend.spec.js b/test/promisified/producer/concurrentSend.spec.js new file mode 100644 index 00000000..1fc29b30 --- /dev/null +++ b/test/promisified/producer/concurrentSend.spec.js @@ -0,0 +1,45 @@ +jest.setTimeout(10000); + +const { + secureRandom, + createProducer, + createTopic, +} = require('../testhelpers'); + +describe('Producer', () => { + let producer, topicName, message; + const partitions = 3; + + beforeEach(async () => { + producer = createProducer({ + }, { + 'linger.ms': 0, + }); + + topicName = `test-topic-${secureRandom()}`; + + await createTopic({ topic: topicName, partitions: 3 }); + }) + + afterEach(async () => { + producer && (await producer.disconnect()); + }) + + + it('can send messages concurrently', + async () => { + await producer.connect(); + const sender = async (p) => { + message = { partition: p, value: `value-${secureRandom()}` }; + const report = await producer.send({ topic: topicName, messages: [message] }); + return report; + }; + const reports = await Promise.all(Array(partitions).fill().map((_, i) => sender(i))); + expect(reports.length).toBe(partitions); + for (let i = 0; i < partitions; i++) { + expect(reports[i].length).toBe(1); + expect(reports[i][0].partition).toBe(i); + } + } + ); +}) From ca1e002a3143e53321d8c530942bae51eaf37df4 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Jul 2024 13:34:21 +0530 Subject: [PATCH 163/224] Remove debug-only properties --- lib/kafkajs/_consumer.js | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 137020df..e6770e4a 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -176,12 +176,6 @@ class Consumer { */ #topicPartitionToBatchPayload = new Map(); - /** - * TODO: remove this or make it a bit more reliable. - * This is a debug property for this branch. - */ - clientId = null; - /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -390,7 +384,7 @@ class Consumer { { code: error.ErrorCodes.ERR__INVALID_ARG }); } const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); - this.clientId = rdKafkaConfig['client.id']; + this.#logger = new DefaultLogger(); /* Consumer specific configuration */ @@ -1190,7 +1184,7 @@ class Consumer { * * Worker termination acts as a async barrier. */ - async #worker(config, perMessageProcessor, fetcher, id) { + async #worker(config, perMessageProcessor, fetcher) { let nextIdx = -1; while (!this.#workerTerminationScheduled) { /* Invalidate the message cache if needed */ @@ -1252,7 +1246,7 @@ class Consumer { Array(workersToSpawn) .fill() .map((_, i) => - this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this), i) + this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this)) .catch(e => { if (this.#logger) this.#logger.error(`Worker ${i} encountered an error: ${e}:${e.stack}`); From 717ec67eeca6c9facf8c78f68c28af44b7780095 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Jul 2024 16:20:44 +0530 Subject: [PATCH 164/224] Update MIGRATION.md for eachBatch --- MIGRATION.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 089f39ed..0225819e 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -298,11 +298,15 @@ producerRun().then(consumerRun).catch(console.error); eachMessage: someFunc, }); ``` - - The `heartbeat()` no longer needs to be called by the user in the `eachMessage/eachBatch` callback. Heartbeats are automatically managed by librdkafka. - - The `partitionsConsumedConcurrently` property is not supported at the moment. - - An API compatible version of `eachBatch` is available, but the batch size never exceeds 1. - The property `eachBatchAutoResolve` is supported. Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, - and within the returned batch, `offsetLag` and `offsetLagLow` are unsupported, and `commitOffsetsIfNecessary` is a no-op. + - The `heartbeat()` no longer needs to be called by the user in the `eachMessage/eachBatch` callback. + Heartbeats are automatically managed by librdkafka. + - The `partitionsConsumedConcurrently` is supported by both `eachMessage` and `eachBatch`. + - An API compatible version of `eachBatch` is available, but the batch size calculation is not + as per configured parameters, rather, a constant maximum size is configured internally. This is subject + to change. + The property `eachBatchAutoResolve` is supported. + Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, + and within the returned batch, `offsetLag` and `offsetLagLow` are unsupported. * `commitOffsets`: - Does not yet support sending metadata for topic partitions being committed. - If called with no arguments, it commits all offsets passed to the user (or the stored offsets, if manually handling offset storage using `consumer.storeOffsets`). From d77611508f6130ea3985108ea44eaa909eec949a Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 8 Jul 2024 16:25:10 +0530 Subject: [PATCH 165/224] Bump version --- CHANGELOG.md | 11 +++++++++++ lib/util.js | 2 +- package-lock.json | 4 ++-- package.json | 2 +- 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 312f66d6..1a1d8cc5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,14 @@ +# confluent-kafka-javascript v0.1.16-devel + +v0.1.16-devel is a pre-production, early-access release. + +## Features + +1. Add per-partition concurrency to consumer. +2. Add true `eachBatch` support to consumer. +3. Add a `leaderEpoch` field to the topic partitions where required (listing, committing, etc.). + + # confluent-kafka-javascript v0.1.15-devel v0.1.15-devel is a pre-production, early-access release. diff --git a/lib/util.js b/lib/util.js index 78b0bac1..3e3854c4 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.15-devel'; +util.bindingVersion = 'v0.1.16-devel'; diff --git a/package-lock.json b/package-lock.json index 140e483e..6970c390 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.15-devel", + "version": "v0.1.16-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.15-devel", + "version": "v0.1.16-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index 0b3e3090..28fd63ea 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.15-devel", + "version": "v0.1.16-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", "librdkafka_win": "2.4.0", From 1d9533b236d178048d0ea3b351ec7884134953c3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 6 Aug 2024 09:52:49 +0530 Subject: [PATCH 166/224] Fix linting and Makefile issues (#2) * Add single-offset-storage method for performance In promisified mode, offsets are stored after every single message Calling into FFI is already not cheap and on top of that, the offsetsStore method does a lot of allocations because it needs to make a vector etc. etc. This method change yields a benefit of 8%. Also reduces dynamic casts by storing a reference to the rdkafka consumer. * Remove jshint and expand eslint coverage Many changes within, but only cosmetic: 1. remove jshint as it's rather unmaintained and we don't need 2 linters. 2. update eslint to latest possible version. 3. update eslint coverage to include old API and promisified tests. 4. update makefile accordingly 5. fix lint errors. * Update Makefule, update cpplint to latest version * Comply with cpplint --- .eslintrc.js | 26 - .jshintignore | 2 - .jshintrc | 24 - Makefile | 21 +- cpplint.py | 3313 ++++++++++------- eslint.config.js | 54 + lib/admin.js | 12 +- lib/client.js | 11 +- lib/error.js | 6 +- lib/kafka-consumer.js | 27 +- lib/kafkajs/_admin.js | 4 +- lib/kafkajs/_common.js | 12 +- lib/kafkajs/_consumer.js | 29 +- lib/kafkajs/_consumer_cache.js | 10 +- lib/kafkajs/_error.js | 4 +- lib/kafkajs/_producer.js | 10 +- lib/producer-stream.js | 7 +- lib/producer.js | 8 +- lib/producer/high-level-producer.js | 4 +- lib/rdkafka.js | 2 +- lib/util.js | 2 +- package-lock.json | 1232 +++--- package.json | 9 +- src/admin.cc | 5 +- src/admin.h | 2 +- src/binding.cc | 2 + src/binding.h | 2 +- src/callbacks.cc | 4 + src/callbacks.h | 6 +- src/common.cc | 25 +- src/common.h | 3 +- src/config.cc | 9 +- src/config.h | 7 +- src/connection.cc | 31 +- src/connection.h | 6 +- src/errors.cc | 7 +- src/errors.h | 2 +- src/kafka-consumer.cc | 187 +- src/kafka-consumer.h | 7 +- src/producer.cc | 53 +- src/producer.h | 9 +- src/topic.h | 2 +- src/workers.cc | 63 +- test/promisified/admin/delete_groups.spec.js | 2 +- test/promisified/admin/list_topics.spec.js | 3 +- test/promisified/consumer/commit.spec.js | 54 +- .../consumer/consumeMessages.spec.js | 220 +- .../consumer/consumerCacheTests.spec.js | 52 +- .../consumer/consumerTransactions.spec.js | 80 +- .../consumer/groupInstanceId.spec.js | 12 +- .../consumer/incrementalRebalance.spec.js | 11 +- test/promisified/consumer/pause.spec.js | 228 +- test/promisified/consumer/seek.spec.js | 168 +- test/promisified/consumer/store.spec.js | 46 +- test/promisified/consumer/subscribe.spec.js | 10 +- test/promisified/oauthbearer_cb.spec.js | 4 +- .../producer/concurrentSend.spec.js | 6 +- .../producer/concurrentTransaction.spec.js | 12 +- test/promisified/producer/flush.spec.js | 6 +- .../producer/idempotentProduceMessage.spec.js | 152 +- .../producer/producingToInvalidTopic.spec.js | 14 +- test/promisified/testhelpers.js | 12 +- test/promisified/unit/cache.spec.js | 8 +- 63 files changed, 3414 insertions(+), 2947 deletions(-) delete mode 100644 .eslintrc.js delete mode 100644 .jshintignore delete mode 100644 .jshintrc create mode 100644 eslint.config.js diff --git a/.eslintrc.js b/.eslintrc.js deleted file mode 100644 index 120e6804..00000000 --- a/.eslintrc.js +++ /dev/null @@ -1,26 +0,0 @@ -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/.jshintignore b/.jshintignore deleted file mode 100644 index d08618fd..00000000 --- a/.jshintignore +++ /dev/null @@ -1,2 +0,0 @@ -README.md -lib/kafkajs/* \ No newline at end of file diff --git a/.jshintrc b/.jshintrc deleted file mode 100644 index a97e0dc7..00000000 --- a/.jshintrc +++ /dev/null @@ -1,24 +0,0 @@ -{ - "node": true, - "mocha": true, - "browser": false, - "boss": true, - "curly": true, - "debug": false, - "devel": false, - "eqeqeq": true, - "evil": true, - "forin": false, - "latedef": false, - "noarg": true, - "nonew": true, - "nomen": false, - "onevar": false, - "plusplus": false, - "regexp": false, - "undef": true, - "strict": false, - "white": false, - "eqnull": true, - "esversion": 11 -} diff --git a/Makefile b/Makefile index 3d12a428..75b85441 100644 --- a/Makefile +++ b/Makefile @@ -1,15 +1,9 @@ NODE-GYP ?= node_modules/.bin/node-gyp -# Sick of changing this. Do a check and try to use python 2 if it doesn't work -PYTHON_VERSION_FULL := $(wordlist 2,4,$(subst ., ,$(shell python --version 2>&1))) -PYTHON_VERSION_MAJOR := $(word 1,${PYTHON_VERSION_FULL}) - -ifeq ($(PYTHON_VERSION_MAJOR), 2) PYTHON = python -else -PYTHON = python2 +ifeq (, $(shell command -v python)) + PYTHON = python3 endif - NODE ?= node CPPLINT ?= cpplint.py BUILDTYPE ?= Release @@ -24,9 +18,7 @@ CONFIG_OUTPUTS = \ build/binding.Makefile build/config.gypi CPPLINT_FILES = $(wildcard src/*.cc src/*.h) -CPPLINT_FILTER = -legal/copyright -JSLINT_FILES = lib/*.js test/*.js e2e/*.js lib/kafkajs/*.js -ESLINT_FILES = lib/kafkajs/*.js test/promisified/*.js +CPPLINT_FILTER = -legal/copyright,-readability/todo,-whitespace/indent_namespace,-runtime/references PACKAGE = $(shell node -pe 'require("./package.json").name.split("/")[1]') VERSION = $(shell node -pe 'require("./package.json").version') @@ -40,16 +32,13 @@ endif all: lint lib test e2e -lint: cpplint jslint eslint +lint: cpplint eslint cpplint: @$(PYTHON) $(CPPLINT) --filter=$(CPPLINT_FILTER) $(CPPLINT_FILES) -jslint: node_modules/.dirstamp - @./node_modules/.bin/jshint --verbose $(JSLINT_FILES) - eslint: node_modules/.dirstamp - @./node_modules/.bin/eslint $(ESLINT_FILES) + @./node_modules/.bin/eslint . lib: node_modules/.dirstamp $(CONFIG_OUTPUTS) @PYTHONHTTPSVERIFY=0 $(NODE-GYP) build $(GYPBUILDARGS) diff --git a/cpplint.py b/cpplint.py index 44726248..0e50a882 100644 --- a/cpplint.py +++ b/cpplint.py @@ -42,84 +42,155 @@ """ import codecs +import collections import copy import getopt +import glob +import itertools import math # for log import os import re -import sre_compile import string import sys +import sysconfig import unicodedata +import xml.etree.ElementTree +# if empty, use defaults +_valid_extensions = set([]) + +__VERSION__ = '1.7' _USAGE = """ -Syntax: cpplint.py [--verbose=#] [--output=vs7] [--filter=-x,+y,...] +Syntax: cpplint.py [--verbose=#] [--output=emacs|eclipse|vs7|junit|sed|gsed] + [--filter=-x,+y,...] [--counting=total|toplevel|detailed] [--root=subdir] - [--linelength=digits] + [--repository=path] + [--linelength=digits] [--headers=x,y,...] + [--recursive] + [--exclude=path] + [--extensions=hpp,cpp,...] + [--includeorder=default|standardcfirst] + [--config=filename] + [--quiet] + [--version] [file] ... + Style checker for C/C++ source files. + This is a fork of the Google style checker with minor extensions. + The style guidelines this tries to follow are those in - http://google-styleguide.googlecode.com/svn/trunk/cppguide.xml + https://google.github.io/styleguide/cppguide.html Every problem is given a confidence score from 1-5, with 5 meaning we are certain of the problem, and 1 meaning it could be a legitimate construct. This will miss some errors, and is not a substitute for a code review. - To suppress false-positive errors of a certain category, add a - 'NOLINT(category)' comment to the line. NOLINT or NOLINT(*) - suppresses errors of all categories on that line. + To suppress false-positive errors of certain categories, add a + 'NOLINT(category[, category...])' comment to the line. NOLINT or NOLINT(*) + suppresses errors of all categories on that line. To suppress categories + on the next line use NOLINTNEXTLINE instead of NOLINT. To suppress errors in + a block of code 'NOLINTBEGIN(category[, category...])' comment to a line at + the start of the block and to end the block add a comment with 'NOLINTEND'. + NOLINT blocks are inclusive so any statements on the same line as a BEGIN + or END will have the error suppression applied. The files passed in will be linted; at least one file must be provided. - Default linted extensions are .cc, .cpp, .cu, .cuh and .h. Change the - extensions with the --extensions flag. + Default linted extensions are %s. + Other file types will be ignored. + Change the extensions with the --extensions flag. Flags: - output=vs7 + output=emacs|eclipse|vs7|junit|sed|gsed By default, the output is formatted to ease emacs parsing. Visual Studio - compatible output (vs7) may also be used. Other formats are unsupported. + compatible output (vs7) may also be used. Further support exists for + eclipse (eclipse), and JUnit (junit). XML parsers such as those used + in Jenkins and Bamboo may also be used. + The sed format outputs sed commands that should fix some of the errors. + Note that this requires gnu sed. If that is installed as gsed on your + system (common e.g. on macOS with homebrew) you can use the gsed output + format. Sed commands are written to stdout, not stderr, so you should be + able to pipe output straight to a shell to run the fixes. verbose=# Specify a number 0-5 to restrict errors to certain verbosity levels. + Errors with lower verbosity levels have lower confidence and are more + likely to be false positives. + + quiet + Don't print anything if no errors are found. filter=-x,+y,... Specify a comma-separated list of category-filters to apply: only error messages whose category names pass the filters will be printed. (Category names are printed with the message and look like "[whitespace/indent]".) Filters are evaluated left to right. - "-FOO" and "FOO" means "do not print categories that start with FOO". + "-FOO" means "do not print categories that start with FOO". "+FOO" means "do print categories that start with FOO". Examples: --filter=-whitespace,+whitespace/braces - --filter=whitespace,runtime/printf,+runtime/printf_format + --filter=-whitespace,-runtime/printf,+runtime/printf_format --filter=-,+build/include_what_you_use To see a list of all the categories used in cpplint, pass no arg: --filter= + Filters can directly be limited to files and also line numbers. The + syntax is category:file:line , where line is optional. The filter limitation + works for both + and - and can be combined with ordinary filters: + + Examples: --filter=-whitespace:foo.h,+whitespace/braces:foo.h + --filter=-whitespace,-runtime/printf:foo.h:14,+runtime/printf_format:foo.h + --filter=-,+build/include_what_you_use:foo.h:321 + counting=total|toplevel|detailed The total number of errors found is always printed. If 'toplevel' is provided, then the count of errors in each of the top-level categories like 'build' and 'whitespace' will also be printed. If 'detailed' is provided, then a count - is provided for each category like 'build/class'. + is provided for each category like 'legal/copyright'. + + repository=path + The top level directory of the repository, used to derive the header + guard CPP variable. By default, this is determined by searching for a + path that contains .git, .hg, or .svn. When this flag is specified, the + given path is used instead. This option allows the header guard CPP + variable to remain consistent even if members of a team have different + repository root directories (such as when checking out a subdirectory + with SVN). In addition, users of non-mainstream version control systems + can use this flag to ensure readable header guard CPP variables. + + Examples: + Assuming that Alice checks out ProjectName and Bob checks out + ProjectName/trunk and trunk contains src/chrome/ui/browser.h, then + with no --repository flag, the header guard CPP variable will be: + + Alice => TRUNK_SRC_CHROME_BROWSER_UI_BROWSER_H_ + Bob => SRC_CHROME_BROWSER_UI_BROWSER_H_ + + If Alice uses the --repository=trunk flag and Bob omits the flag or + uses --repository=. then the header guard CPP variable will be: + + Alice => SRC_CHROME_BROWSER_UI_BROWSER_H_ + Bob => SRC_CHROME_BROWSER_UI_BROWSER_H_ root=subdir The root directory used for deriving header guard CPP variable. - By default, the header guard CPP variable is calculated as the relative - path to the directory that contains .git, .hg, or .svn. When this flag - is specified, the relative path is calculated from the specified - directory. If the specified directory does not exist, this flag is - ignored. + This directory is relative to the top level directory of the repository + which by default is determined by searching for a directory that contains + .git, .hg, or .svn but can also be controlled with the --repository flag. + If the specified directory does not exist, this flag is ignored. Examples: - Assuming that src/.git exists, the header guard CPP variables for + Assuming that src is the top level directory of the repository (and + cwd=top/src), the header guard CPP variables for src/chrome/browser/ui/browser.h are: No flag => CHROME_BROWSER_UI_BROWSER_H_ --root=chrome => BROWSER_UI_BROWSER_H_ --root=chrome/browser => UI_BROWSER_H_ + --root=.. => SRC_CHROME_BROWSER_UI_BROWSER_H_ linelength=digits This is the allowed line length for the project. The default value is @@ -128,11 +199,50 @@ Examples: --linelength=120 + recursive + Search for files to lint recursively. Each directory given in the list + of files to be linted is replaced by all files that descend from that + directory. Files with extensions not in the valid extensions list are + excluded. + + exclude=path + Exclude the given path from the list of files to be linted. Relative + paths are evaluated relative to the current directory and shell globbing + is performed. This flag can be provided multiple times to exclude + multiple files. + + Examples: + --exclude=one.cc + --exclude=src/*.cc + --exclude=src/*.cc --exclude=test/*.cc + extensions=extension,extension,... The allowed file extensions that cpplint will check Examples: - --extensions=hpp,cpp + --extensions=%s + + includeorder=default|standardcfirst + For the build/include_order rule, the default is to blindly assume angle + bracket includes with file extension are c-system-headers (default), + even knowing this will have false classifications. + The default is established at google. + standardcfirst means to instead use an allow-list of known c headers and + treat all others as separate group of "other system headers". The C headers + included are those of the C-standard lib and closely related ones. + + config=filename + Search for config files with the specified name instead of CPPLINT.cfg + + headers=x,y,... + The header extensions that cpplint will treat as .h in checks. Values are + automatically added to --extensions list. + (by default, only files with extensions %s will be assumed to be headers) + + Examples: + --headers=%s + --headers=hpp,hxx + --headers=hpp cpplint.py supports per-directory configurations specified in CPPLINT.cfg files. CPPLINT.cfg file can contain a number of key=value pairs. @@ -142,6 +252,8 @@ filter=+filter1,-filter2,... exclude_files=regex linelength=80 + root=subdir + headers=x,y,... "set noparent" option prevents cpplint from traversing directory tree upwards looking for more .cfg files in parent directories. This option @@ -153,16 +265,22 @@ "exclude_files" allows to specify a regular expression to be matched against a file name. If the expression matches, the file is skipped and not run - through liner. + through the linter. "linelength" allows to specify the allowed line length for the project. + The "root" option is similar in function to the --root flag (see example + above). Paths are relative to the directory of the CPPLINT.cfg. + + The "headers" option is similar in function to the --headers flag + (see example above). + CPPLINT.cfg has an effect on files in the same directory and all sub-directories, unless overridden by a nested configuration file. Example file: filter=-build/include_order,+build/include_alpha - exclude_files=.*\.cc + exclude_files=.*\\.cc The above example disables build/include_order warning and enables build/include_alpha as well as excludes all .cc from being @@ -175,17 +293,20 @@ # If you add a new error message with a new category, add it to the list # here! cpplint_unittest.py should tell you if you forget to do this. _ERROR_CATEGORIES = [ - 'build/class', 'build/c++11', + 'build/c++17', 'build/deprecated', 'build/endif_comment', 'build/explicit_make_pair', 'build/forward_decl', 'build/header_guard', 'build/include', + 'build/include_subdir', 'build/include_alpha', 'build/include_order', 'build/include_what_you_use', + 'build/namespaces_headers', + 'build/namespaces_literals', 'build/namespaces', 'build/printf_format', 'build/storage_class', @@ -196,7 +317,6 @@ 'readability/check', 'readability/constructors', 'readability/fn_size', - 'readability/function', 'readability/inheritance', 'readability/multiline_comment', 'readability/multiline_string', @@ -214,7 +334,6 @@ 'runtime/invalid_increment', 'runtime/member_string_references', 'runtime/memset', - 'runtime/indentation_namespace', 'runtime/operator', 'runtime/printf', 'runtime/printf_format', @@ -227,11 +346,13 @@ 'whitespace/comma', 'whitespace/comments', 'whitespace/empty_conditional_body', + 'whitespace/empty_if_body', 'whitespace/empty_loop_body', 'whitespace/end_of_line', 'whitespace/ending_newline', 'whitespace/forcolon', 'whitespace/indent', + 'whitespace/indent_namespace', 'whitespace/line_length', 'whitespace/newline', 'whitespace/operators', @@ -241,10 +362,49 @@ 'whitespace/todo', ] +# keywords to use with --outputs which generate stdout for machine processing +_MACHINE_OUTPUTS = [ + 'junit', + 'sed', + 'gsed' +] + # These error categories are no longer enforced by cpplint, but for backwards- # compatibility they may still appear in NOLINT comments. _LEGACY_ERROR_CATEGORIES = [ + 'build/class', 'readability/streams', + 'readability/function', + ] + +# These prefixes for categories should be ignored since they relate to other +# tools which also use the NOLINT syntax, e.g. clang-tidy. +_OTHER_NOLINT_CATEGORY_PREFIXES = [ + 'clang-analyzer-', + 'abseil-', + 'altera-', + 'android-', + 'boost-', + 'bugprone-', + 'cert-', + 'concurrency-', + 'cppcoreguidelines-', + 'darwin-', + 'fuchsia-', + 'google-', + 'hicpp-', + 'linuxkernel-', + 'llvm-', + 'llvmlibc-', + 'misc-', + 'modernize-', + 'mpi-', + 'objc-', + 'openmp-', + 'performance-', + 'portability-', + 'readability-', + 'zircon-', ] # The default state of the category filter. This is overridden by the --filter= @@ -253,6 +413,16 @@ # All entries here should start with a '-' or '+', as in the --filter= flag. _DEFAULT_FILTERS = ['-build/include_alpha'] +# The default list of categories suppressed for C (not C++) files. +_DEFAULT_C_SUPPRESSED_CATEGORIES = [ + 'readability/casting', + ] + +# The default list of categories suppressed for Linux Kernel files. +_DEFAULT_KERNEL_SUPPRESSED_CATEGORIES = [ + 'whitespace/tab', + ] + # We used to check for high-bit characters, but after much discussion we # decided those were OK, as long as they were in UTF-8 and didn't represent # hard-coded international strings, which belong in a separate i18n file. @@ -265,7 +435,7 @@ 'alloc.h', 'builtinbuf.h', 'bvector.h', - 'complex.h', + # 'complex.h', collides with System C header "complex.h" since C11 'defalloc.h', 'deque.h', 'editbuf.h', @@ -311,7 +481,7 @@ 'tree.h', 'type_traits.h', 'vector.h', - # 17.6.1.2 C++ library headers + # C++ library headers 'algorithm', 'array', 'atomic', @@ -346,6 +516,7 @@ 'random', 'ratio', 'regex', + 'scoped_allocator', 'set', 'sstream', 'stack', @@ -364,7 +535,45 @@ 'utility', 'valarray', 'vector', - # 17.6.1.2 C++ headers for C library facilities + # C++14 headers + 'shared_mutex', + # C++17 headers + 'any', + 'charconv', + 'codecvt', + 'execution', + 'filesystem', + 'memory_resource', + 'optional', + 'string_view', + 'variant', + # C++20 headers + 'barrier', + 'bit', + 'compare', + 'concepts', + 'coroutine', + 'format', + 'latch' + 'numbers', + 'ranges', + 'semaphore', + 'source_location', + 'span', + 'stop_token', + 'syncstream', + 'version', + # C++23 headers + 'expected', + 'flat_map', + 'flat_set', + 'generator', + 'mdspan', + 'print', + 'spanstream', + 'stacktrace', + 'stdfloat', + # C++ headers for C library facilities 'cassert', 'ccomplex', 'cctype', @@ -393,49 +602,242 @@ 'cwctype', ]) +# C headers +_C_HEADERS = frozenset([ + # System C headers + 'assert.h', + 'complex.h', + 'ctype.h', + 'errno.h', + 'fenv.h', + 'float.h', + 'inttypes.h', + 'iso646.h', + 'limits.h', + 'locale.h', + 'math.h', + 'setjmp.h', + 'signal.h', + 'stdalign.h', + 'stdarg.h', + 'stdatomic.h', + 'stdbool.h', + 'stddef.h', + 'stdint.h', + 'stdio.h', + 'stdlib.h', + 'stdnoreturn.h', + 'string.h', + 'tgmath.h', + 'threads.h', + 'time.h', + 'uchar.h', + 'wchar.h', + 'wctype.h', + # C23 headers + 'stdbit.h', + 'stdckdint.h', + # additional POSIX C headers + 'aio.h', + 'arpa/inet.h', + 'cpio.h', + 'dirent.h', + 'dlfcn.h', + 'fcntl.h', + 'fmtmsg.h', + 'fnmatch.h', + 'ftw.h', + 'glob.h', + 'grp.h', + 'iconv.h', + 'langinfo.h', + 'libgen.h', + 'monetary.h', + 'mqueue.h', + 'ndbm.h', + 'net/if.h', + 'netdb.h', + 'netinet/in.h', + 'netinet/tcp.h', + 'nl_types.h', + 'poll.h', + 'pthread.h', + 'pwd.h', + 'regex.h', + 'sched.h', + 'search.h', + 'semaphore.h', + 'setjmp.h', + 'signal.h', + 'spawn.h', + 'strings.h', + 'stropts.h', + 'syslog.h', + 'tar.h', + 'termios.h', + 'trace.h', + 'ulimit.h', + 'unistd.h', + 'utime.h', + 'utmpx.h', + 'wordexp.h', + # additional GNUlib headers + 'a.out.h', + 'aliases.h', + 'alloca.h', + 'ar.h', + 'argp.h', + 'argz.h', + 'byteswap.h', + 'crypt.h', + 'endian.h', + 'envz.h', + 'err.h', + 'error.h', + 'execinfo.h', + 'fpu_control.h', + 'fstab.h', + 'fts.h', + 'getopt.h', + 'gshadow.h', + 'ieee754.h', + 'ifaddrs.h', + 'libintl.h', + 'mcheck.h', + 'mntent.h', + 'obstack.h', + 'paths.h', + 'printf.h', + 'pty.h', + 'resolv.h', + 'shadow.h', + 'sysexits.h', + 'ttyent.h', + # Additional linux glibc headers + 'dlfcn.h', + 'elf.h', + 'features.h', + 'gconv.h', + 'gnu-versions.h', + 'lastlog.h', + 'libio.h', + 'link.h', + 'malloc.h', + 'memory.h', + 'netash/ash.h', + 'netatalk/at.h', + 'netax25/ax25.h', + 'neteconet/ec.h', + 'netipx/ipx.h', + 'netiucv/iucv.h', + 'netpacket/packet.h', + 'netrom/netrom.h', + 'netrose/rose.h', + 'nfs/nfs.h', + 'nl_types.h', + 'nss.h', + 're_comp.h', + 'regexp.h', + 'sched.h', + 'sgtty.h', + 'stab.h', + 'stdc-predef.h', + 'stdio_ext.h', + 'syscall.h', + 'termio.h', + 'thread_db.h', + 'ucontext.h', + 'ustat.h', + 'utmp.h', + 'values.h', + 'wait.h', + 'xlocale.h', + # Hardware specific headers + 'arm_neon.h', + 'emmintrin.h', + 'xmmintin.h', + ]) + +# Folders of C libraries so commonly used in C++, +# that they have parity with standard C libraries. +C_STANDARD_HEADER_FOLDERS = frozenset([ + # standard C library + "sys", + # glibc for linux + "arpa", + "asm-generic", + "bits", + "gnu", + "net", + "netinet", + "protocols", + "rpc", + "rpcsvc", + "scsi", + # linux kernel header + "drm", + "linux", + "misc", + "mtd", + "rdma", + "sound", + "video", + "xen", + ]) + +# Type names +_TYPES = re.compile( + r'^(?:' + # [dcl.type.simple] + r'(char(16_t|32_t)?)|wchar_t|' + r'bool|short|int|long|signed|unsigned|float|double|' + # [support.types] + r'(ptrdiff_t|size_t|max_align_t|nullptr_t)|' + # [cstdint.syn] + r'(u?int(_fast|_least)?(8|16|32|64)_t)|' + r'(u?int(max|ptr)_t)|' + r')$') + # These headers are excluded from [build/include] and [build/include_order] # checks: # - Anything not following google file name conventions (containing an # uppercase character, such as Python.h or nsStringAPI.h, for example). # - Lua headers. -# - rdkafka.cpp header, because it would be located in different directories depending -# on whether it's pulled from librdkafka sources or librdkafka-dev package. _THIRD_PARTY_HEADERS_PATTERN = re.compile( - r'^(?:[^/]*[A-Z][^/]*\.h|lua\.h|lauxlib\.h|lualib\.h|rdkafkacpp\.h)$') + r'^(?:[^/]*[A-Z][^/]*\.h|lua\.h|lauxlib\.h|lualib\.h)$') + +# Pattern for matching FileInfo.BaseName() against test file name +_test_suffixes = ['_test', '_regtest', '_unittest'] +_TEST_FILE_SUFFIX = '(' + '|'.join(_test_suffixes) + r')$' +# Pattern that matches only complete whitespace, possibly across multiple lines. +_EMPTY_CONDITIONAL_BODY_PATTERN = re.compile(r'^\s*$', re.DOTALL) # Assertion macros. These are defined in base/logging.h and -# testing/base/gunit.h. Note that the _M versions need to come first -# for substring matching to work. +# testing/base/public/gunit.h. _CHECK_MACROS = [ 'DCHECK', 'CHECK', - 'EXPECT_TRUE_M', 'EXPECT_TRUE', - 'ASSERT_TRUE_M', 'ASSERT_TRUE', - 'EXPECT_FALSE_M', 'EXPECT_FALSE', - 'ASSERT_FALSE_M', 'ASSERT_FALSE', + 'EXPECT_TRUE', 'ASSERT_TRUE', + 'EXPECT_FALSE', 'ASSERT_FALSE', ] # Replacement macros for CHECK/DCHECK/EXPECT_TRUE/EXPECT_FALSE -_CHECK_REPLACEMENT = dict([(m, {}) for m in _CHECK_MACROS]) +_CHECK_REPLACEMENT = dict([(macro_var, {}) for macro_var in _CHECK_MACROS]) for op, replacement in [('==', 'EQ'), ('!=', 'NE'), ('>=', 'GE'), ('>', 'GT'), ('<=', 'LE'), ('<', 'LT')]: - _CHECK_REPLACEMENT['DCHECK'][op] = 'DCHECK_%s' % replacement - _CHECK_REPLACEMENT['CHECK'][op] = 'CHECK_%s' % replacement - _CHECK_REPLACEMENT['EXPECT_TRUE'][op] = 'EXPECT_%s' % replacement - _CHECK_REPLACEMENT['ASSERT_TRUE'][op] = 'ASSERT_%s' % replacement - _CHECK_REPLACEMENT['EXPECT_TRUE_M'][op] = 'EXPECT_%s_M' % replacement - _CHECK_REPLACEMENT['ASSERT_TRUE_M'][op] = 'ASSERT_%s_M' % replacement + _CHECK_REPLACEMENT['DCHECK'][op] = f'DCHECK_{replacement}' + _CHECK_REPLACEMENT['CHECK'][op] = f'CHECK_{replacement}' + _CHECK_REPLACEMENT['EXPECT_TRUE'][op] = f'EXPECT_{replacement}' + _CHECK_REPLACEMENT['ASSERT_TRUE'][op] = f'ASSERT_{replacement}' for op, inv_replacement in [('==', 'NE'), ('!=', 'EQ'), ('>=', 'LT'), ('>', 'LE'), ('<=', 'GT'), ('<', 'GE')]: - _CHECK_REPLACEMENT['EXPECT_FALSE'][op] = 'EXPECT_%s' % inv_replacement - _CHECK_REPLACEMENT['ASSERT_FALSE'][op] = 'ASSERT_%s' % inv_replacement - _CHECK_REPLACEMENT['EXPECT_FALSE_M'][op] = 'EXPECT_%s_M' % inv_replacement - _CHECK_REPLACEMENT['ASSERT_FALSE_M'][op] = 'ASSERT_%s_M' % inv_replacement + _CHECK_REPLACEMENT['EXPECT_FALSE'][op] = f'EXPECT_{inv_replacement}' + _CHECK_REPLACEMENT['ASSERT_FALSE'][op] = f'ASSERT_{inv_replacement}' # Alternative tokens and their replacements. For full list, see section 2.5 # Alternative tokens [lex.digraph] in the C++ standard. @@ -462,16 +864,17 @@ # False positives include C-style multi-line comments and multi-line strings # but those have always been troublesome for cpplint. _ALT_TOKEN_REPLACEMENT_PATTERN = re.compile( - r'[ =()](' + ('|'.join(_ALT_TOKEN_REPLACEMENT.keys())) + r')(?=[ (]|$)') + r'([ =()])(' + ('|'.join(_ALT_TOKEN_REPLACEMENT.keys())) + r')([ (]|$)') # These constants define types of headers for use with # _IncludeState.CheckNextIncludeOrder(). _C_SYS_HEADER = 1 _CPP_SYS_HEADER = 2 -_LIKELY_MY_HEADER = 3 -_POSSIBLE_MY_HEADER = 4 -_OTHER_HEADER = 5 +_OTHER_SYS_HEADER = 3 +_LIKELY_MY_HEADER = 4 +_POSSIBLE_MY_HEADER = 5 +_OTHER_HEADER = 6 # These constants define the current inline assembly state _NO_ASM = 0 # Outside of inline assembly block @@ -484,8 +887,28 @@ r'(?:\s+(volatile|__volatile__))?' r'\s*[{(]') - -_regexp_compile_cache = {} +# Match strings that indicate we're working on a C (not C++) file. +_SEARCH_C_FILE = re.compile(r'\b(?:LINT_C_FILE|' + r'vim?:\s*.*(\s*|:)filetype=c(\s*|:|$))') + +# Match string that indicates we're working on a Linux Kernel file. +_SEARCH_KERNEL_FILE = re.compile(r'\b(?:LINT_KERNEL_FILE)') + +# Commands for sed to fix the problem +_SED_FIXUPS = { + 'Remove spaces around =': r's/ = /=/', + 'Remove spaces around !=': r's/ != /!=/', + 'Remove space before ( in if (': r's/if (/if(/', + 'Remove space before ( in for (': r's/for (/for(/', + 'Remove space before ( in while (': r's/while (/while(/', + 'Remove space before ( in switch (': r's/switch (/switch(/', + 'Should have a space between // and comment': r's/\/\//\/\/ /', + 'Missing space before {': r's/\([^ ]\){/\1 {/', + 'Tab found, replace by spaces': r's/\t/ /g', + 'Line ends in whitespace. Consider deleting these extra spaces.': r's/\s*$//', + 'You don\'t need a ; after a }': r's/};/}/', + 'Missing space after ,': r's/,\([^ ]\)/, \1/g', +} # {str, set(int)}: a map from error categories to sets of linenumbers # on which those errors are expected and should be suppressed. @@ -494,17 +917,147 @@ # The root directory used for deriving header guard CPP variable. # This is set by --root flag. _root = None +_root_debug = False + +# The top level repository directory. If set, _root is calculated relative to +# this directory instead of the directory containing version control artifacts. +# This is set by the --repository flag. +_repository = None + +# Files to exclude from linting. This is set by the --exclude flag. +_excludes = None + +# Whether to suppress all PrintInfo messages, UNRELATED to --quiet flag +_quiet = False # The allowed line length of files. # This is set by --linelength flag. _line_length = 80 +# This allows to use different include order rule than default +_include_order = "default" + +# This allows different config files to be used +_config_filename = "CPPLINT.cfg" + +# Treat all headers starting with 'h' equally: .h, .hpp, .hxx etc. +# This is set by --headers flag. +_hpp_headers = set([]) + +class ErrorSuppressions: + """Class to track all error suppressions for cpplint""" + + class LineRange: + """Class to represent a range of line numbers for which an error is suppressed""" + def __init__(self, begin, end): + self.begin = begin + self.end = end + + def __str__(self): + return f'[{self.begin}-{self.end}]' + + def __contains__(self, obj): + return self.begin <= obj <= self.end + + def ContainsRange(self, other): + return self.begin <= other.begin and self.end >= other.end + + def __init__(self): + self._suppressions = collections.defaultdict(list) + self._open_block_suppression = None + + def _AddSuppression(self, category, line_range): + suppressed = self._suppressions[category] + if not (suppressed and suppressed[-1].ContainsRange(line_range)): + suppressed.append(line_range) + + def GetOpenBlockStart(self): + """:return: The start of the current open block or `-1` if there is not an open block""" + return self._open_block_suppression.begin if self._open_block_suppression else -1 + + def AddGlobalSuppression(self, category): + """Add a suppression for `category` which is suppressed for the whole file""" + self._AddSuppression(category, self.LineRange(0, math.inf)) + + def AddLineSuppression(self, category, linenum): + """Add a suppression for `category` which is suppressed only on `linenum`""" + self._AddSuppression(category, self.LineRange(linenum, linenum)) + + def StartBlockSuppression(self, category, linenum): + """Start a suppression block for `category` on `linenum`. inclusive""" + if self._open_block_suppression is None: + self._open_block_suppression = self.LineRange(linenum, math.inf) + self._AddSuppression(category, self._open_block_suppression) + + def EndBlockSuppression(self, linenum): + """End the current block suppression on `linenum`. inclusive""" + if self._open_block_suppression: + self._open_block_suppression.end = linenum + self._open_block_suppression = None + + def IsSuppressed(self, category, linenum): + """:return: `True` if `category` is suppressed for `linenum`""" + suppressed = self._suppressions[category] + self._suppressions[None] + return any(linenum in lr for lr in suppressed) + + def HasOpenBlock(self): + """:return: `True` if a block suppression was started but not ended""" + return self._open_block_suppression is not None + + def Clear(self): + """Clear all current error suppressions""" + self._suppressions.clear() + self._open_block_suppression = None + +_error_suppressions = ErrorSuppressions() + +def ProcessHppHeadersOption(val): + global _hpp_headers + try: + _hpp_headers = {ext.strip() for ext in val.split(',')} + except ValueError: + PrintUsage('Header extensions must be comma separated list.') + +def ProcessIncludeOrderOption(val): + if val is None or val == "default": + pass + elif val == "standardcfirst": + global _include_order + _include_order = val + else: + PrintUsage('Invalid includeorder value %s. Expected default|standardcfirst') + +def IsHeaderExtension(file_extension): + return file_extension in GetHeaderExtensions() + +def GetHeaderExtensions(): + if _hpp_headers: + return _hpp_headers + if _valid_extensions: + return {h for h in _valid_extensions if 'h' in h} + return set(['h', 'hh', 'hpp', 'hxx', 'h++', 'cuh']) + # The allowed extensions for file names -# This is set by --extensions flag. -_valid_extensions = set(['cc', 'h', 'cpp', 'cu', 'cuh']) +# This is set by --extensions flag +def GetAllExtensions(): + return GetHeaderExtensions().union(_valid_extensions or set( + ['c', 'cc', 'cpp', 'cxx', 'c++', 'cu'])) + +def ProcessExtensionsOption(val): + global _valid_extensions + try: + extensions = [ext.strip() for ext in val.split(',')] + _valid_extensions = set(extensions) + except ValueError: + PrintUsage('Extensions should be a comma-separated list of values;' + 'for example: extensions=hpp,cpp\n' + f'This could not be parsed: "{val}"') + +def GetNonHeaderExtensions(): + return GetAllExtensions().difference(GetHeaderExtensions()) def ParseNolintSuppressions(filename, raw_line, linenum, error): - """Updates the global list of error-suppressions. + """Updates the global list of line error-suppressions. Parses any NOLINT comments on the current line, updating the global error_suppressions store. Reports an error if the NOLINT comment @@ -516,79 +1069,91 @@ def ParseNolintSuppressions(filename, raw_line, linenum, error): linenum: int, the number of the current line. error: function, an error handler. """ - matched = Search(r'\bNOLINT(NEXTLINE)?\b(\([^)]+\))?', raw_line) + matched = re.search(r'\bNOLINT(NEXTLINE|BEGIN|END)?\b(\([^)]+\))?', raw_line) if matched: - if matched.group(1): - suppressed_line = linenum + 1 - else: - suppressed_line = linenum - category = matched.group(2) - if category in (None, '(*)'): # => "suppress all" - _error_suppressions.setdefault(None, set()).add(suppressed_line) + no_lint_type = matched.group(1) + if no_lint_type == 'NEXTLINE': + def ProcessCategory(category): + _error_suppressions.AddLineSuppression(category, linenum + 1) + elif no_lint_type == 'BEGIN': + if _error_suppressions.HasOpenBlock(): + error(filename, linenum, 'readability/nolint', 5, + f'NONLINT block already defined on line {_error_suppressions.GetOpenBlockStart()}') + + def ProcessCategory(category): + _error_suppressions.StartBlockSuppression(category, linenum) + elif no_lint_type == 'END': + if not _error_suppressions.HasOpenBlock(): + error(filename, linenum, 'readability/nolint', 5, 'Not in a NOLINT block') + + def ProcessCategory(category): + if category is not None: + error(filename, linenum, 'readability/nolint', 5, + f'NOLINT categories not supported in block END: {category}') + _error_suppressions.EndBlockSuppression(linenum) else: - if category.startswith('(') and category.endswith(')'): - category = category[1:-1] + def ProcessCategory(category): + _error_suppressions.AddLineSuppression(category, linenum) + categories = matched.group(2) + if categories in (None, '(*)'): # => "suppress all" + ProcessCategory(None) + elif categories.startswith('(') and categories.endswith(')'): + for category in set(map(lambda c: c.strip(), categories[1:-1].split(','))): if category in _ERROR_CATEGORIES: - _error_suppressions.setdefault(category, set()).add(suppressed_line) + ProcessCategory(category) + elif any(c for c in _OTHER_NOLINT_CATEGORY_PREFIXES if category.startswith(c)): + # Ignore any categories from other tools. + pass elif category not in _LEGACY_ERROR_CATEGORIES: error(filename, linenum, 'readability/nolint', 5, - 'Unknown NOLINT error category: %s' % category) + f'Unknown NOLINT error category: {category}') + +def ProcessGlobalSuppresions(lines): + """Deprecated; use ProcessGlobalSuppressions.""" + ProcessGlobalSuppressions(lines) + +def ProcessGlobalSuppressions(lines): + """Updates the list of global error suppressions. + + Parses any lint directives in the file that have global effect. + + Args: + lines: An array of strings, each representing a line of the file, with the + last element being empty if the file is terminated with a newline. + """ + for line in lines: + if _SEARCH_C_FILE.search(line): + for category in _DEFAULT_C_SUPPRESSED_CATEGORIES: + _error_suppressions.AddGlobalSuppression(category) + if _SEARCH_KERNEL_FILE.search(line): + for category in _DEFAULT_KERNEL_SUPPRESSED_CATEGORIES: + _error_suppressions.AddGlobalSuppression(category) def ResetNolintSuppressions(): """Resets the set of NOLINT suppressions to empty.""" - _error_suppressions.clear() + _error_suppressions.Clear() def IsErrorSuppressedByNolint(category, linenum): """Returns true if the specified error category is suppressed on this line. Consults the global error_suppressions map populated by - ParseNolintSuppressions/ResetNolintSuppressions. + ParseNolintSuppressions/ProcessGlobalSuppressions/ResetNolintSuppressions. Args: category: str, the category of the error. linenum: int, the current line number. Returns: - bool, True iff the error should be suppressed due to a NOLINT comment. - """ - return (linenum in _error_suppressions.get(category, set()) or - linenum in _error_suppressions.get(None, set())) - - -def Match(pattern, s): - """Matches the string with the pattern, caching the compiled regexp.""" - # The regexp compilation caching is inlined in both Match and Search for - # performance reasons; factoring it out into a separate function turns out - # to be noticeably expensive. - if pattern not in _regexp_compile_cache: - _regexp_compile_cache[pattern] = sre_compile.compile(pattern) - return _regexp_compile_cache[pattern].match(s) - - -def ReplaceAll(pattern, rep, s): - """Replaces instances of pattern in a string with a replacement. - - The compiled regex is kept in a cache shared by Match and Search. - - Args: - pattern: regex pattern - rep: replacement text - s: search string - - Returns: - string with replacements made (or original string if no replacements) + bool, True iff the error should be suppressed due to a NOLINT comment, + block suppression or global suppression. """ - if pattern not in _regexp_compile_cache: - _regexp_compile_cache[pattern] = sre_compile.compile(pattern) - return _regexp_compile_cache[pattern].sub(rep, s) + return _error_suppressions.IsSuppressed(category, linenum) -def Search(pattern, s): - """Searches the string for the pattern, caching the compiled regexp.""" - if pattern not in _regexp_compile_cache: - _regexp_compile_cache[pattern] = sre_compile.compile(pattern) - return _regexp_compile_cache[pattern].search(s) +def _IsSourceExtension(s): + """File extension (excluding dot) matches a source file extension.""" + return s in GetNonHeaderExtensions() class _IncludeState(object): @@ -609,11 +1174,13 @@ class _IncludeState(object): _MY_H_SECTION = 1 _C_SECTION = 2 _CPP_SECTION = 3 - _OTHER_H_SECTION = 4 + _OTHER_SYS_SECTION = 4 + _OTHER_H_SECTION = 5 _TYPE_NAMES = { _C_SYS_HEADER: 'C system header', _CPP_SYS_HEADER: 'C++ system header', + _OTHER_SYS_HEADER: 'other system header', _LIKELY_MY_HEADER: 'header this file implements', _POSSIBLE_MY_HEADER: 'header this file may implement', _OTHER_HEADER: 'other header', @@ -623,11 +1190,14 @@ class _IncludeState(object): _MY_H_SECTION: 'a header this file implements', _C_SECTION: 'C system header', _CPP_SECTION: 'C++ system header', + _OTHER_SYS_SECTION: 'other system header', _OTHER_H_SECTION: 'other header', } def __init__(self): self.include_list = [[]] + self._section = None + self._last_header = None self.ResetSection('') def FindHeader(self, header): @@ -698,7 +1268,7 @@ def IsInAlphabeticalOrder(self, clean_lines, linenum, header_path): # If previous line was a blank line, assume that the headers are # intentionally sorted the way they are. if (self._last_header > header_path and - Match(r'^\s*#\s*include\b', clean_lines.elided[linenum - 1])): + re.match(r'^\s*#\s*include\b', clean_lines.elided[linenum - 1])): return False return True @@ -716,9 +1286,8 @@ def CheckNextIncludeOrder(self, header_type): error message describing what's wrong. """ - error_message = ('Found %s after %s' % - (self._TYPE_NAMES[header_type], - self._SECTION_NAMES[self._section])) + error_message = (f'Found {self._TYPE_NAMES[header_type]}' + f' after {self._SECTION_NAMES[self._section]}') last_section = self._section @@ -734,6 +1303,12 @@ def CheckNextIncludeOrder(self, header_type): else: self._last_header = '' return error_message + elif header_type == _OTHER_SYS_HEADER: + if self._section <= self._OTHER_SYS_SECTION: + self._section = self._OTHER_SYS_SECTION + else: + self._last_header = '' + return error_message elif header_type == _LIKELY_MY_HEADER: if self._section <= self._MY_H_SECTION: self._section = self._MY_H_SECTION @@ -768,16 +1343,32 @@ def __init__(self): self._filters_backup = self.filters[:] self.counting = 'total' # In what way are we counting errors? self.errors_by_category = {} # string to int dict storing error counts + self.quiet = False # Suppress non-error messagess? # output format: # "emacs" - format that emacs can parse (default) + # "eclipse" - format that eclipse can parse # "vs7" - format that Microsoft Visual Studio 7 can parse + # "junit" - format that Jenkins, Bamboo, etc can parse + # "sed" - returns a gnu sed command to fix the problem + # "gsed" - like sed, but names the command gsed, e.g. for macOS homebrew users self.output_format = 'emacs' + # For JUnit output, save errors and failures until the end so that they + # can be written into the XML + self._junit_errors = [] + self._junit_failures = [] + def SetOutputFormat(self, output_format): """Sets the output format for errors.""" self.output_format = output_format + def SetQuiet(self, quiet): + """Sets the module's quiet settings, and returns the previous setting.""" + last_quiet = self.quiet + self.quiet = quiet + return last_quiet + def SetVerboseLevel(self, level): """Sets the module's verbosity, and returns the previous setting.""" last_verbose_level = self.verbose_level @@ -815,7 +1406,7 @@ def AddFilters(self, filters): for filt in self.filters: if not (filt.startswith('+') or filt.startswith('-')): raise ValueError('Every filter in --filters must start with + or -' - ' (%s does not)' % filt) + f' ({filt} does not)') def BackupFilters(self): """ Saves the current filter list to backup storage.""" @@ -842,10 +1433,70 @@ def IncrementErrorCount(self, category): def PrintErrorCounts(self): """Print a summary of errors by category, and the total.""" - for category, count in self.errors_by_category.iteritems(): - sys.stderr.write('Category \'%s\' errors found: %d\n' % - (category, count)) - sys.stderr.write('Total errors found: %d\n' % self.error_count) + for category, count in sorted(dict.items(self.errors_by_category)): + self.PrintInfo(f'Category \'{category}\' errors found: {count}\n') + if self.error_count > 0: + self.PrintInfo(f'Total errors found: {self.error_count}\n') + + def PrintInfo(self, message): + # _quiet does not represent --quiet flag. + # Hide infos from stdout to keep stdout pure for machine consumption + if not _quiet and self.output_format not in _MACHINE_OUTPUTS: + sys.stdout.write(message) + + def PrintError(self, message): + if self.output_format == 'junit': + self._junit_errors.append(message) + else: + sys.stderr.write(message) + + def AddJUnitFailure(self, filename, linenum, message, category, confidence): + self._junit_failures.append((filename, linenum, message, category, + confidence)) + + def FormatJUnitXML(self): + num_errors = len(self._junit_errors) + num_failures = len(self._junit_failures) + + testsuite = xml.etree.ElementTree.Element('testsuite') + testsuite.attrib['errors'] = str(num_errors) + testsuite.attrib['failures'] = str(num_failures) + testsuite.attrib['name'] = 'cpplint' + + if num_errors == 0 and num_failures == 0: + testsuite.attrib['tests'] = str(1) + xml.etree.ElementTree.SubElement(testsuite, 'testcase', name='passed') + + else: + testsuite.attrib['tests'] = str(num_errors + num_failures) + if num_errors > 0: + testcase = xml.etree.ElementTree.SubElement(testsuite, 'testcase') + testcase.attrib['name'] = 'errors' + error = xml.etree.ElementTree.SubElement(testcase, 'error') + error.text = '\n'.join(self._junit_errors) + if num_failures > 0: + # Group failures by file + failed_file_order = [] + failures_by_file = {} + for failure in self._junit_failures: + failed_file = failure[0] + if failed_file not in failed_file_order: + failed_file_order.append(failed_file) + failures_by_file[failed_file] = [] + failures_by_file[failed_file].append(failure) + # Create a testcase for each file + for failed_file in failed_file_order: + failures = failures_by_file[failed_file] + testcase = xml.etree.ElementTree.SubElement(testsuite, 'testcase') + testcase.attrib['name'] = failed_file + failure = xml.etree.ElementTree.SubElement(testcase, 'failure') + template = '{0}: {1} [{2}] [{3}]' + texts = [template.format(f[1], f[2], f[3], f[4]) for f in failures] + failure.text = '\n'.join(texts) + + xml_decl = '\n' + return xml_decl + xml.etree.ElementTree.tostring(testsuite, 'utf-8').decode('utf-8') + _cpplint_state = _CppLintState() @@ -859,6 +1510,14 @@ def _SetOutputFormat(output_format): """Sets the module's output format.""" _cpplint_state.SetOutputFormat(output_format) +def _Quiet(): + """Return's the module's quiet setting.""" + return _cpplint_state.quiet + +def _SetQuiet(quiet): + """Set the module's quiet status, and return previous setting.""" + return _cpplint_state.SetQuiet(quiet) + def _VerboseLevel(): """Returns the module's verbosity setting.""" @@ -946,7 +1605,10 @@ def Check(self, error, filename, linenum): filename: The name of the current file. linenum: The number of the line to check. """ - if Match(r'T(EST|est)', self.current_function): + if not self.in_a_function: + return + + if re.match(r'T(EST|est)', self.current_function): base_trigger = self._TEST_TRIGGER else: base_trigger = self._NORMAL_TRIGGER @@ -959,9 +1621,8 @@ def Check(self, error, filename, linenum): error_level = 5 error(filename, linenum, 'readability/fn_size', error_level, 'Small and focused functions are preferred:' - ' %s has %d non-comment lines' - ' (error triggered by exceeding %d lines).' % ( - self.current_function, self.lines_in_function, trigger)) + f' {self.current_function} has {self.lines_in_function} non-comment lines' + f' (error triggered by exceeding {trigger} lines).') def End(self): """Stop analyzing function body.""" @@ -988,12 +1649,12 @@ def FullName(self): return os.path.abspath(self._filename).replace('\\', '/') def RepositoryName(self): - """FullName after removing the local path to the repository. + r"""FullName after removing the local path to the repository. If we have a real absolute path name here we can try to do something smart: detecting the root of the checkout and truncating /path/to/checkout from the name so that we get header guards that don't include things like - "C:\Documents and Settings\..." or "/home/username/..." in them and thus + "C:\\Documents and Settings\\..." or "/home/username/..." in them and thus people on different computers who have checked the source out to different locations won't see bogus errors. """ @@ -1002,6 +1663,20 @@ def RepositoryName(self): if os.path.exists(fullname): project_dir = os.path.dirname(fullname) + # If the user specified a repository path, it exists, and the file is + # contained in it, use the specified repository path + if _repository: + repo = FileInfo(_repository).FullName() + root_dir = project_dir + while os.path.exists(root_dir): + # allow case insensitive compare on Windows + if os.path.normcase(root_dir) == os.path.normcase(repo): + return os.path.relpath(fullname, root_dir).replace('\\', '/') + one_up_dir = os.path.dirname(root_dir) + if one_up_dir == root_dir: + break + root_dir = one_up_dir + if os.path.exists(os.path.join(project_dir, ".svn")): # If there's a .svn file in the current directory, we recursively look # up the directory tree for the top of the SVN checkout @@ -1016,12 +1691,14 @@ def RepositoryName(self): # Not SVN <= 1.6? Try to find a git, hg, or svn top level directory by # searching up from the current path. - root_dir = os.path.dirname(fullname) - while (root_dir != os.path.dirname(root_dir) and - not os.path.exists(os.path.join(root_dir, ".git")) and - not os.path.exists(os.path.join(root_dir, ".hg")) and - not os.path.exists(os.path.join(root_dir, ".svn"))): - root_dir = os.path.dirname(root_dir) + root_dir = current_dir = os.path.dirname(fullname) + while current_dir != os.path.dirname(current_dir): + if (os.path.exists(os.path.join(current_dir, ".git")) or + os.path.exists(os.path.join(current_dir, ".hg")) or + os.path.exists(os.path.join(current_dir, ".svn"))): + root_dir = current_dir + break + current_dir = os.path.dirname(current_dir) if (os.path.exists(os.path.join(root_dir, ".git")) or os.path.exists(os.path.join(root_dir, ".hg")) or @@ -1051,7 +1728,7 @@ def BaseName(self): return self.Split()[1] def Extension(self): - """File extension - text following the final period.""" + """File extension - text following the final period, includes that period.""" return self.Split()[2] def NoExtension(self): @@ -1060,10 +1737,10 @@ def NoExtension(self): def IsSource(self): """File has a source file extension.""" - return self.Extension()[1:] in ('c', 'cc', 'cpp', 'cxx') + return _IsSourceExtension(self.Extension()[1:]) -def _ShouldPrintError(category, confidence, linenum): +def _ShouldPrintError(category, confidence, filename, linenum): """If confidence >= verbose, category passes filter and is not suppressed.""" # There are three ways we might decide not to print an error message: @@ -1077,11 +1754,16 @@ def _ShouldPrintError(category, confidence, linenum): is_filtered = False for one_filter in _Filters(): + filter_cat, filter_file, filter_line = _ParseFilterSelector(one_filter[1:]) + category_match = category.startswith(filter_cat) + file_match = filter_file == "" or filter_file == filename + line_match = filter_line == linenum or filter_line == -1 + if one_filter.startswith('-'): - if category.startswith(one_filter[1:]): + if category_match and file_match and line_match: is_filtered = True elif one_filter.startswith('+'): - if category.startswith(one_filter[1:]): + if category_match and file_match and line_match: is_filtered = False else: assert False # should have been checked for in SetFilter. @@ -1098,9 +1780,9 @@ def Error(filename, linenum, category, confidence, message): that is, how certain we are this is a legitimate style regression, and not a misidentification or a use that's sometimes justified. - False positives can be suppressed by the use of - "cpplint(category)" comments on the offending line. These are - parsed into _error_suppressions. + False positives can be suppressed by the use of "NOLINT(category)" + comments, NOLINTNEXTLINE or in blocks started by NOLINTBEGIN. These + are parsed into _error_suppressions. Args: filename: The name of the file containing the error. @@ -1113,17 +1795,28 @@ def Error(filename, linenum, category, confidence, message): and 1 meaning that it could be a legitimate construct. message: The error message. """ - if _ShouldPrintError(category, confidence, linenum): + if _ShouldPrintError(category, confidence, filename, linenum): _cpplint_state.IncrementErrorCount(category) if _cpplint_state.output_format == 'vs7': - sys.stderr.write('%s(%s): %s [%s] [%d]\n' % ( - filename, linenum, message, category, confidence)) + _cpplint_state.PrintError(f'{filename}({linenum}): error cpplint:' + f' [{category}] {message} [{confidence}]\n') elif _cpplint_state.output_format == 'eclipse': - sys.stderr.write('%s:%s: warning: %s [%s] [%d]\n' % ( - filename, linenum, message, category, confidence)) + sys.stderr.write(f'{filename}:{linenum}: warning:' + f' {message} [{category}] [{confidence}]\n') + elif _cpplint_state.output_format == 'junit': + _cpplint_state.AddJUnitFailure(filename, linenum, message, category, confidence) + elif _cpplint_state.output_format in ['sed', 'gsed']: + if message in _SED_FIXUPS: + sys.stdout.write(f"{_cpplint_state.output_format} -i" + f" '{linenum}{_SED_FIXUPS[message]}' {filename}" + f" # {message} [{category}] [{confidence}]\n") + else: + sys.stderr.write(f'# {filename}:{linenum}: ' + f' "{message}" [{category}] [{confidence}]\n') else: - sys.stderr.write('%s:%s: %s [%s] [%d]\n' % ( - filename, linenum, message, category, confidence)) + final_message = (f'{filename}:{linenum}: ' + f' {message} [{category}] [{confidence}]\n') + sys.stderr.write(final_message) # Matches standard C++ escape sequences per 2.13.2.3 of the C++ standard. @@ -1193,7 +1886,7 @@ def CleanseRawStrings(raw_lines): # Found the end of the string, match leading space for this # line and resume copying the original lines, and also insert # a "" on the last line. - leading_space = Match(r'^(\s*)\S', line) + leading_space = re.match(r'^(\s*)\S', line) line = leading_space.group(1) + '""' + line[end + len(delimiter):] delimiter = None else: @@ -1206,8 +1899,18 @@ def CleanseRawStrings(raw_lines): while delimiter is None: # Look for beginning of a raw string. # See 2.14.15 [lex.string] for syntax. - matched = Match(r'^(.*)\b(?:R|u8R|uR|UR|LR)"([^\s\\()]*)\((.*)$', line) - if matched: + # + # Once we have matched a raw string, we check the prefix of the + # line to make sure that the line is not part of a single line + # comment. It's done this way because we remove raw strings + # before removing comments as opposed to removing comments + # before removing raw strings. This is because there are some + # cpplint checks that requires the comments to be preserved, but + # we don't want to check comments that are inside raw strings. + matched = re.match(r'^(.*?)\b(?:R|u8R|uR|UR|LR)"([^\s\\()]*)\((.*)$', line) + if (matched and + not re.match(r'^([^\'"]|\'(\\.|[^\'])*\'|"(\\.|[^"])*")*//', + matched.group(1))): delimiter = ')' + matched.group(2) + '"' end = matched.group(3).find(delimiter) @@ -1251,7 +1954,7 @@ def FindNextMultiLineCommentEnd(lines, lineix): def RemoveMultiLineCommentsFromRange(lines, begin, end): """Clears a range of lines for multi-line comments.""" - # Having // dummy comments makes the lines non-empty, so we will not get + # Having // comments makes the lines non-empty, so we will not get # unnecessary blank line warnings later in the code. for i in range(begin, end): lines[i] = '/**/' @@ -1289,6 +1992,28 @@ def CleanseComments(line): return _RE_PATTERN_CLEANSE_LINE_C_COMMENTS.sub('', line) +def ReplaceAlternateTokens(line): + """Replace any alternate token by its original counterpart. + + In order to comply with the google rule stating that unary operators should + never be followed by a space, an exception is made for the 'not' and 'compl' + alternate tokens. For these, any trailing space is removed during the + conversion. + + Args: + line: The line being processed. + + Returns: + The line with alternate tokens replaced. + """ + for match in _ALT_TOKEN_REPLACEMENT_PATTERN.finditer(line): + token = _ALT_TOKEN_REPLACEMENT[match.group(2)] + tail = '' if match.group(2) in ['not', 'compl'] and match.group(3) == ' ' \ + else r'\3' + line = re.sub(match.re, rf'\1{token}{tail}', line, count=1) + return line + + class CleansedLines(object): """Holds 4 copies of all lines with different preprocessing applied to them. @@ -1301,15 +2026,17 @@ class CleansedLines(object): """ def __init__(self, lines): + if '-readability/alt_tokens' in _cpplint_state.filters: + for i, line in enumerate(lines): + lines[i] = ReplaceAlternateTokens(line) self.elided = [] self.lines = [] self.raw_lines = lines self.num_lines = len(lines) self.lines_without_raw_strings = CleanseRawStrings(lines) - for linenum in range(len(self.lines_without_raw_strings)): - self.lines.append(CleanseComments( - self.lines_without_raw_strings[linenum])) - elided = self._CollapseStrings(self.lines_without_raw_strings[linenum]) + for line in self.lines_without_raw_strings: + self.lines.append(CleanseComments(line)) + elided = self._CollapseStrings(line) self.elided.append(CleanseComments(elided)) def NumLines(self): @@ -1342,7 +2069,7 @@ def _CollapseStrings(elided): collapsed = '' while True: # Find the first quote character - match = Match(r'^([^\'"]*)([\'"])(.*)$', elided) + match = re.match(r'^([^\'"]*)([\'"])(.*)$', elided) if not match: collapsed += elided break @@ -1367,8 +2094,8 @@ def _CollapseStrings(elided): # correctly as long as there are digits on both sides of the # separator. So we are fine as long as we don't see something # like "0.'3" (gcc 4.9.0 will not allow this literal). - if Search(r'\b(?:0[bBxX]?|[1-9])[0-9a-fA-F]*$', head): - match_literal = Match(r'^((?:\'?[0-9a-zA-Z_])*)(.*)$', "'" + tail) + if re.search(r'\b(?:0[bBxX]?|[1-9])[0-9a-fA-F]*$', head): + match_literal = re.match(r'^((?:\'?[0-9a-zA-Z_])*)(.*)$', "'" + tail) collapsed += head + match_literal.group(1).replace("'", '') elided = match_literal.group(2) else: @@ -1397,7 +2124,7 @@ def FindEndOfExpressionInLine(line, startpos, stack): On finding an unclosed expression: (-1, None) Otherwise: (-1, new stack at end of this line) """ - for i in xrange(startpos, len(line)): + for i in range(startpos, len(line)): char = line[i] if char in '([{': # Found start of parenthesized expression, push to expression stack @@ -1410,7 +2137,7 @@ def FindEndOfExpressionInLine(line, startpos, stack): stack.pop() if not stack: return (-1, None) - elif i > 0 and Search(r'\boperator\s*$', line[0:i]): + elif i > 0 and re.search(r'\boperator\s*$', line[0:i]): # operator<, don't add to stack continue else: @@ -1439,7 +2166,7 @@ def FindEndOfExpressionInLine(line, startpos, stack): # Ignore "->" and operator functions if (i > 0 and - (line[i - 1] == '-' or Search(r'\boperator\s*$', line[0:i - 1]))): + (line[i - 1] == '-' or re.search(r'\boperator\s*$', line[0:i - 1]))): continue # Pop the stack if there is a matching '<'. Otherwise, ignore @@ -1486,7 +2213,7 @@ def CloseExpression(clean_lines, linenum, pos): """ line = clean_lines.elided[linenum] - if (line[pos] not in '({[<') or Match(r'<[<=]', line[pos:]): + if (line[pos] not in '({[<') or re.match(r'<[<=]', line[pos:]): return (line, clean_lines.NumLines(), -1) # Check first line @@ -1534,8 +2261,8 @@ def FindStartOfExpressionInLine(line, endpos, stack): # Ignore it if it's a "->" or ">=" or "operator>" if (i > 0 and (line[i - 1] == '-' or - Match(r'\s>=\s', line[i - 1:]) or - Search(r'\boperator\s*$', line[0:i]))): + re.match(r'\s>=\s', line[i - 1:]) or + re.search(r'\boperator\s*$', line[0:i]))): i -= 1 else: stack.append('>') @@ -1625,8 +2352,8 @@ def CheckForCopyright(filename, lines, error): """Logs an error if no Copyright message appears at the top of the file.""" # We'll say it should occur by line 10. Don't forget there's a - # dummy line at the front. - for line in xrange(1, min(len(lines), 11)): + # placeholder line at the front. + for line in range(1, min(len(lines), 11)): if re.search(r'Copyright', lines[line], re.I): break else: # means no copyright line was found error(filename, 0, 'legal/copyright', 5, @@ -1643,12 +2370,36 @@ def GetIndentLevel(line): Returns: An integer count of leading spaces, possibly zero. """ - indent = Match(r'^( *)\S', line) + indent = re.match(r'^( *)\S', line) if indent: return len(indent.group(1)) else: return 0 +def PathSplitToList(path): + """Returns the path split into a list by the separator. + + Args: + path: An absolute or relative path (e.g. '/a/b/c/' or '../a') + + Returns: + A list of path components (e.g. ['a', 'b', 'c]). + """ + lst = [] + while True: + (head, tail) = os.path.split(path) + if head == path: # absolute paths end + lst.append(head) + break + if tail == path: # relative paths end + lst.append(tail) + break + + path = head + lst.append(tail) + + lst.reverse() + return lst def GetHeaderGuardCPPVariable(filename): """Returns the CPP variable that should be used as a header guard. @@ -1668,11 +2419,62 @@ def GetHeaderGuardCPPVariable(filename): filename = re.sub(r'/\.flymake/([^/]*)$', r'/\1', filename) # Replace 'c++' with 'cpp'. filename = filename.replace('C++', 'cpp').replace('c++', 'cpp') - + fileinfo = FileInfo(filename) file_path_from_root = fileinfo.RepositoryName() - if _root: - file_path_from_root = re.sub('^' + _root + os.sep, '', file_path_from_root) + + def FixupPathFromRoot(): + if _root_debug: + sys.stderr.write(f"\n_root fixup, _root = '{_root}'," + f" repository name = '{fileinfo.RepositoryName()}'\n") + + # Process the file path with the --root flag if it was set. + if not _root: + if _root_debug: + sys.stderr.write("_root unspecified\n") + return file_path_from_root + + def StripListPrefix(lst, prefix): + # f(['x', 'y'], ['w, z']) -> None (not a valid prefix) + if lst[:len(prefix)] != prefix: + return None + # f(['a, 'b', 'c', 'd'], ['a', 'b']) -> ['c', 'd'] + return lst[(len(prefix)):] + + # root behavior: + # --root=subdir , lstrips subdir from the header guard + maybe_path = StripListPrefix(PathSplitToList(file_path_from_root), + PathSplitToList(_root)) + + if _root_debug: + sys.stderr.write(("_root lstrip (maybe_path=%s, file_path_from_root=%s," + + " _root=%s)\n") % (maybe_path, file_path_from_root, _root)) + + if maybe_path: + return os.path.join(*maybe_path) + + # --root=.. , will prepend the outer directory to the header guard + full_path = fileinfo.FullName() + # adapt slashes for windows + root_abspath = os.path.abspath(_root).replace('\\', '/') + + maybe_path = StripListPrefix(PathSplitToList(full_path), + PathSplitToList(root_abspath)) + + if _root_debug: + sys.stderr.write(("_root prepend (maybe_path=%s, full_path=%s, " + + "root_abspath=%s)\n") % (maybe_path, full_path, root_abspath)) + + if maybe_path: + return os.path.join(*maybe_path) + + if _root_debug: + sys.stderr.write(f"_root ignore, returning {file_path_from_root}\n") + + # --root=FAKE_DIR is ignored + return file_path_from_root + + file_path_from_root = FixupPathFromRoot() return re.sub(r'[^a-zA-Z0-9]', '_', file_path_from_root).upper() + '_' @@ -1696,7 +2498,12 @@ def CheckForHeaderGuard(filename, clean_lines, error): # and not the general NOLINT or NOLINT(*) syntax. raw_lines = clean_lines.lines_without_raw_strings for i in raw_lines: - if Search(r'//\s*NOLINT\(build/header_guard\)', i): + if re.search(r'//\s*NOLINT\(build/header_guard\)', i): + return + + # Allow pragma once instead of header guards + for i in raw_lines: + if re.search(r'^\s*#pragma\s+once', i): return cppvar = GetHeaderGuardCPPVariable(filename) @@ -1723,8 +2530,7 @@ def CheckForHeaderGuard(filename, clean_lines, error): if not ifndef or not define or ifndef != define: error(filename, 0, 'build/header_guard', 5, - 'No #ifndef header guard found, suggested CPP variable is: %s' % - cppvar) + f'No #ifndef header guard found, suggested CPP variable is: {cppvar}') return # The guard should be PATH_FILE_H_, but we also allow PATH_FILE_H__ @@ -1737,66 +2543,75 @@ def CheckForHeaderGuard(filename, clean_lines, error): ParseNolintSuppressions(filename, raw_lines[ifndef_linenum], ifndef_linenum, error) error(filename, ifndef_linenum, 'build/header_guard', error_level, - '#ifndef header guard has wrong style, please use: %s' % cppvar) + f'#ifndef header guard has wrong style, please use: {cppvar}') # Check for "//" comments on endif line. ParseNolintSuppressions(filename, raw_lines[endif_linenum], endif_linenum, error) - match = Match(r'#endif\s*//\s*' + cppvar + r'(_)?\b', endif) + match = re.match(r'#endif\s*//\s*' + cppvar + r'(_)?\b', endif) if match: if match.group(1) == '_': # Issue low severity warning for deprecated double trailing underscore error(filename, endif_linenum, 'build/header_guard', 0, - '#endif line should be "#endif // %s"' % cppvar) + f'#endif line should be "#endif // {cppvar}"') return # Didn't find the corresponding "//" comment. If this file does not # contain any "//" comments at all, it could be that the compiler # only wants "/**/" comments, look for those instead. no_single_line_comments = True - for i in xrange(1, len(raw_lines) - 1): + for i in range(1, len(raw_lines) - 1): line = raw_lines[i] - if Match(r'^(?:(?:\'(?:\.|[^\'])*\')|(?:"(?:\.|[^"])*")|[^\'"])*//', line): + if re.match(r'^(?:(?:\'(?:\.|[^\'])*\')|(?:"(?:\.|[^"])*")|[^\'"])*//', line): no_single_line_comments = False break if no_single_line_comments: - match = Match(r'#endif\s*/\*\s*' + cppvar + r'(_)?\s*\*/', endif) + match = re.match(r'#endif\s*/\*\s*' + cppvar + r'(_)?\s*\*/', endif) if match: if match.group(1) == '_': # Low severity warning for double trailing underscore error(filename, endif_linenum, 'build/header_guard', 0, - '#endif line should be "#endif /* %s */"' % cppvar) + f'#endif line should be "#endif /* {cppvar} */"') return # Didn't find anything error(filename, endif_linenum, 'build/header_guard', 5, - '#endif line should be "#endif // %s"' % cppvar) + f'#endif line should be "#endif // {cppvar}"') def CheckHeaderFileIncluded(filename, include_state, error): - """Logs an error if a .cc file does not include its header.""" + """Logs an error if a source file does not include its header.""" # Do not check test files - if filename.endswith('_test.cc') or filename.endswith('_unittest.cc'): - return - fileinfo = FileInfo(filename) - headerfile = filename[0:len(filename) - 2] + 'h' - if not os.path.exists(headerfile): + if re.search(_TEST_FILE_SUFFIX, fileinfo.BaseName()): return - headername = FileInfo(headerfile).RepositoryName() - first_include = 0 - for section_list in include_state.include_list: - for f in section_list: - if headername in f[0] or f[0] in headername: - return - if not first_include: - first_include = f[1] - error(filename, first_include, 'build/include', 5, - '%s should include its header file %s' % (fileinfo.RepositoryName(), - headername)) + first_include = message = None + basefilename = filename[0:len(filename) - len(fileinfo.Extension())] + for ext in GetHeaderExtensions(): + headerfile = basefilename + '.' + ext + if not os.path.exists(headerfile): + continue + headername = FileInfo(headerfile).RepositoryName() + include_uses_unix_dir_aliases = False + for section_list in include_state.include_list: + for f in section_list: + include_text = f[0] + if "./" in include_text: + include_uses_unix_dir_aliases = True + if headername in include_text or include_text in headername: + return + if not first_include: + first_include = f[1] + + message = f'{fileinfo.RepositoryName()} should include its header file {headername}' + if include_uses_unix_dir_aliases: + message += ". Relative paths like . and .. are not allowed." + + if message: + error(filename, first_include, 'build/include', 5, message) def CheckForBadCharacters(filename, lines, error): @@ -1817,7 +2632,7 @@ def CheckForBadCharacters(filename, lines, error): error: The function to call with any errors found. """ for linenum, line in enumerate(lines): - if u'\ufffd' in line: + if '\ufffd' in line: error(filename, linenum, 'readability/utf8', 5, 'Line contains invalid UTF-8 (or Unicode replacement character).') if '\0' in line: @@ -1929,7 +2744,7 @@ def CheckPosixThreading(filename, clean_lines, linenum, error): for single_thread_func, multithread_safe_func, pattern in _THREADING_LIST: # Additional pattern matching check to confirm that this is the # function we are looking for - if Search(pattern, line): + if re.search(pattern, line): error(filename, linenum, 'runtime/threadsafe_fn', 2, 'Consider using ' + multithread_safe_func + '...) instead of ' + single_thread_func + @@ -1949,7 +2764,7 @@ def CheckVlogArguments(filename, clean_lines, linenum, error): error: The function to call with any errors found. """ line = clean_lines.elided[linenum] - if Search(r'\bVLOG\((INFO|ERROR|WARNING|DFATAL|FATAL)\)', line): + if re.search(r'\bVLOG\((INFO|ERROR|WARNING|DFATAL|FATAL)\)', line): error(filename, linenum, 'runtime/vlog', 5, 'VLOG() should be used with numeric verbosity level. ' 'Use LOG() if you want symbolic severity levels.') @@ -1983,23 +2798,24 @@ def CheckInvalidIncrement(filename, clean_lines, linenum, error): def IsMacroDefinition(clean_lines, linenum): - if Search(r'^#define', clean_lines[linenum]): + if re.search(r'^#define', clean_lines[linenum]): return True - if linenum > 0 and Search(r'\\$', clean_lines[linenum - 1]): + if linenum > 0 and re.search(r'\\$', clean_lines[linenum - 1]): return True return False def IsForwardClassDeclaration(clean_lines, linenum): - return Match(r'^\s*(\btemplate\b)*.*class\s+\w+;\s*$', clean_lines[linenum]) + return re.match(r'^\s*(\btemplate\b)*.*class\s+\w+;\s*$', clean_lines[linenum]) class _BlockInfo(object): """Stores information about a generic block of code.""" - def __init__(self, seen_open_brace): + def __init__(self, linenum, seen_open_brace): + self.starting_linenum = linenum self.seen_open_brace = seen_open_brace self.open_parentheses = 0 self.inline_asm = _NO_ASM @@ -2048,17 +2864,16 @@ def IsBlockInfo(self): class _ExternCInfo(_BlockInfo): """Stores information about an 'extern "C"' block.""" - def __init__(self): - _BlockInfo.__init__(self, True) + def __init__(self, linenum): + _BlockInfo.__init__(self, linenum, True) class _ClassInfo(_BlockInfo): """Stores information about a class.""" def __init__(self, name, class_or_struct, clean_lines, linenum): - _BlockInfo.__init__(self, False) + _BlockInfo.__init__(self, linenum, False) self.name = name - self.starting_linenum = linenum self.is_derived = False self.check_namespace_indentation = True if class_or_struct == 'struct': @@ -2088,15 +2903,15 @@ def __init__(self, name, class_or_struct, clean_lines, linenum): def CheckBegin(self, filename, clean_lines, linenum, error): # Look for a bare ':' - if Search('(^|[^:]):($|[^:])', clean_lines.elided[linenum]): + if re.search('(^|[^:]):($|[^:])', clean_lines.elided[linenum]): self.is_derived = True def CheckEnd(self, filename, clean_lines, linenum, error): # If there is a DISALLOW macro, it should appear near the end of # the class. seen_last_thing_in_class = False - for i in xrange(linenum - 1, self.starting_linenum, -1): - match = Search( + for i in range(linenum - 1, self.starting_linenum, -1): + match = re.search( r'\b(DISALLOW_COPY_AND_ASSIGN|DISALLOW_IMPLICIT_CONSTRUCTORS)\(' + self.name + r'\)', clean_lines.elided[i]) @@ -2106,29 +2921,28 @@ def CheckEnd(self, filename, clean_lines, linenum, error): match.group(1) + ' should be the last thing in the class') break - if not Match(r'^\s*$', clean_lines.elided[i]): + if not re.match(r'^\s*$', clean_lines.elided[i]): seen_last_thing_in_class = True # Check that closing brace is aligned with beginning of the class. # Only do this if the closing brace is indented by only whitespaces. # This means we will not check single-line class definitions. - indent = Match(r'^( *)\}', clean_lines.elided[linenum]) + indent = re.match(r'^( *)\}', clean_lines.elided[linenum]) if indent and len(indent.group(1)) != self.class_indent: if self.is_struct: parent = 'struct ' + self.name else: parent = 'class ' + self.name error(filename, linenum, 'whitespace/indent', 3, - 'Closing brace should be aligned with beginning of %s' % parent) + f'Closing brace should be aligned with beginning of {parent}') class _NamespaceInfo(_BlockInfo): """Stores information about a namespace.""" def __init__(self, name, linenum): - _BlockInfo.__init__(self, False) + _BlockInfo.__init__(self, linenum, False) self.name = name or '' - self.starting_linenum = linenum self.check_namespace_indentation = True def CheckEnd(self, filename, clean_lines, linenum, error): @@ -2147,7 +2961,7 @@ def CheckEnd(self, filename, clean_lines, linenum, error): # deciding what these nontrivial things are, so this check is # triggered by namespace size only, which works most of the time. if (linenum - self.starting_linenum < 10 - and not Match(r'};*\s*(//|/\*).*\bnamespace\b', line)): + and not re.match(r'^\s*};*\s*(//|/\*).*\bnamespace\b', line)): return # Look for matching comment at end of namespace. @@ -2164,18 +2978,17 @@ def CheckEnd(self, filename, clean_lines, linenum, error): # expected namespace. if self.name: # Named namespace - if not Match((r'};*\s*(//|/\*).*\bnamespace\s+' + re.escape(self.name) + - r'[\*/\.\\\s]*$'), + if not re.match((r'^\s*};*\s*(//|/\*).*\bnamespace\s+' + + re.escape(self.name) + r'[\*/\.\\\s]*$'), line): error(filename, linenum, 'readability/namespace', 5, - 'Namespace should be terminated with "// namespace %s"' % - self.name) + f'Namespace should be terminated with "// namespace {self.name}"') else: # Anonymous namespace - if not Match(r'};*\s*(//|/\*).*\bnamespace[\*/\.\\\s]*$', line): + if not re.match(r'^\s*};*\s*(//|/\*).*\bnamespace[\*/\.\\\s]*$', line): # If "// namespace anonymous" or "// anonymous namespace (more text)", # mention "// anonymous namespace" as an acceptable form - if Match(r'}.*\b(namespace anonymous|anonymous namespace)\b', line): + if re.match(r'^\s*}.*\b(namespace anonymous|anonymous namespace)\b', line): error(filename, linenum, 'readability/namespace', 5, 'Anonymous namespace should be terminated with "// namespace"' ' or "// anonymous namespace"') @@ -2278,7 +3091,7 @@ def InTemplateArgumentList(self, clean_lines, linenum, pos): while linenum < clean_lines.NumLines(): # Find the earliest character that might indicate a template argument line = clean_lines.elided[linenum] - match = Match(r'^[^{};=\[\]\.<>]*(.)', line[pos:]) + match = re.match(r'^[^{};=\[\]\.<>]*(.)', line[pos:]) if not match: linenum += 1 pos = 0 @@ -2338,11 +3151,11 @@ def UpdatePreprocessor(self, line): Args: line: current line to check. """ - if Match(r'^\s*#\s*(if|ifdef|ifndef)\b', line): + if re.match(r'^\s*#\s*(if|ifdef|ifndef)\b', line): # Beginning of #if block, save the nesting stack here. The saved # stack will allow us to restore the parsing state in the #else case. self.pp_stack.append(_PreprocessorInfo(copy.deepcopy(self.stack))) - elif Match(r'^\s*#\s*(else|elif)\b', line): + elif re.match(r'^\s*#\s*(else|elif)\b', line): # Beginning of #else block if self.pp_stack: if not self.pp_stack[-1].seen_else: @@ -2357,7 +3170,7 @@ def UpdatePreprocessor(self, line): else: # TODO(unknown): unexpected #else, issue warning? pass - elif Match(r'^\s*#\s*endif\b', line): + elif re.match(r'^\s*#\s*endif\b', line): # End of #if or #else blocks. if self.pp_stack: # If we saw an #else, we will need to restore the nesting @@ -2429,7 +3242,7 @@ def Update(self, filename, clean_lines, linenum, error): # declarations even if it weren't followed by a whitespace, this # is so that we don't confuse our namespace checker. The # missing spaces will be flagged by CheckSpacing. - namespace_decl_match = Match(r'^\s*namespace\b\s*([:\w]+)?(.*)$', line) + namespace_decl_match = re.match(r'^\s*namespace\b\s*([:\w]+)?(.*)$', line) if not namespace_decl_match: break @@ -2446,9 +3259,9 @@ def Update(self, filename, clean_lines, linenum, error): # such as in: # class LOCKABLE API Object { # }; - class_decl_match = Match( - r'^(\s*(?:template\s*<[\w\s<>,:]*>\s*)?' - r'(class|struct)\s+(?:[A-Z_]+\s+)*(\w+(?:::\w+)*))' + class_decl_match = re.match( + r'^(\s*(?:template\s*<[\w\s<>,:=]*>\s*)?' + r'(class|struct)\s+(?:[a-zA-Z0-9_]+\s+)*(\w+(?:::\w+)*))' r'(.*)$', line) if (class_decl_match and (not self.stack or self.stack[-1].open_parentheses == 0)): @@ -2476,7 +3289,7 @@ def Update(self, filename, clean_lines, linenum, error): # Update access control if we are inside a class/struct if self.stack and isinstance(self.stack[-1], _ClassInfo): classinfo = self.stack[-1] - access_match = Match( + access_match = re.match( r'^(.*)\b(public|private|protected|signals)(\s+(?:slots\s*)?)?' r':(?:[^:]|$)', line) @@ -2487,7 +3300,7 @@ def Update(self, filename, clean_lines, linenum, error): # check if the keywords are not preceded by whitespaces. indent = access_match.group(1) if (len(indent) != classinfo.class_indent + 1 and - Match(r'^\s*$', indent)): + re.match(r'^\s*$', indent)): if classinfo.is_struct: parent = 'struct ' + classinfo.name else: @@ -2496,13 +3309,13 @@ def Update(self, filename, clean_lines, linenum, error): if access_match.group(3): slots = access_match.group(3) error(filename, linenum, 'whitespace/indent', 3, - '%s%s: should be indented +1 space inside %s' % ( - access_match.group(2), slots, parent)) + f'{access_match.group(2)}{slots}:' + f' should be indented +1 space inside {parent}') # Consume braces or semicolons from what's left of the line while True: # Match first brace, semicolon, or closed parenthesis. - matched = Match(r'^[^{;)}]*([{;)}])(.*)$', line) + matched = re.match(r'^[^{;)}]*([{;)}])(.*)$', line) if not matched: break @@ -2513,10 +3326,10 @@ def Update(self, filename, clean_lines, linenum, error): # stack otherwise. if not self.SeenOpenBrace(): self.stack[-1].seen_open_brace = True - elif Match(r'^extern\s*"[^"]*"\s*\{', line): - self.stack.append(_ExternCInfo()) + elif re.match(r'^extern\s*"[^"]*"\s*\{', line): + self.stack.append(_ExternCInfo(linenum)) else: - self.stack.append(_BlockInfo(True)) + self.stack.append(_BlockInfo(linenum, True)) if _MATCH_ASM.match(line): self.stack[-1].inline_asm = _BLOCK_ASM @@ -2550,28 +3363,6 @@ def InnermostClass(self): return classinfo return None - def CheckCompletedBlocks(self, filename, error): - """Checks that all classes and namespaces have been completely parsed. - - Call this when all lines in a file have been processed. - Args: - filename: The name of the current file. - error: The function to call with any errors found. - """ - # Note: This test can result in false positives if #ifdef constructs - # get in the way of brace matching. See the testBuildClass test in - # cpplint_unittest.py for an example of this. - for obj in self.stack: - if isinstance(obj, _ClassInfo): - error(filename, obj.starting_linenum, 'build/class', 5, - 'Failed to find complete declaration of class %s' % - obj.name) - elif isinstance(obj, _NamespaceInfo): - error(filename, obj.starting_linenum, 'build/namespaces', 5, - 'Failed to find complete declaration of namespace %s' % - obj.name) - - def CheckForNonStandardConstructs(filename, clean_lines, linenum, nesting_state, error): r"""Logs an error if we see certain non-ANSI constructs ignored by gcc-2. @@ -2604,46 +3395,47 @@ def CheckForNonStandardConstructs(filename, clean_lines, linenum, # Remove comments from the line, but leave in strings for now. line = clean_lines.lines[linenum] - if Search(r'printf\s*\(.*".*%[-+ ]?\d*q', line): + if re.search(r'printf\s*\(.*".*%[-+ ]?\d*q', line): error(filename, linenum, 'runtime/printf_format', 3, '%q in format strings is deprecated. Use %ll instead.') - if Search(r'printf\s*\(.*".*%\d+\$', line): + if re.search(r'printf\s*\(.*".*%\d+\$', line): error(filename, linenum, 'runtime/printf_format', 2, '%N$ formats are unconventional. Try rewriting to avoid them.') # Remove escaped backslashes before looking for undefined escapes. line = line.replace('\\\\', '') - if Search(r'("|\').*\\(%|\[|\(|{)', line): + if re.search(r'("|\').*\\(%|\[|\(|{)', line): error(filename, linenum, 'build/printf_format', 3, '%, [, (, and { are undefined character escapes. Unescape them.') # For the rest, work with both comments and strings removed. line = clean_lines.elided[linenum] - if Search(r'\b(const|volatile|void|char|short|int|long' + if re.search(r'\b(const|volatile|void|char|short|int|long' r'|float|double|signed|unsigned' r'|schar|u?int8|u?int16|u?int32|u?int64)' r'\s+(register|static|extern|typedef)\b', line): error(filename, linenum, 'build/storage_class', 5, - 'Storage class (static, extern, typedef, etc) should be first.') + 'Storage-class specifier (static, extern, typedef, etc) should be ' + 'at the beginning of the declaration.') - if Match(r'\s*#\s*endif\s*[^/\s]+', line): + if re.match(r'\s*#\s*endif\s*[^/\s]+', line): error(filename, linenum, 'build/endif_comment', 5, 'Uncommented text after #endif is non-standard. Use a comment.') - if Match(r'\s*class\s+(\w+\s*::\s*)+\w+\s*;', line): + if re.match(r'\s*class\s+(\w+\s*::\s*)+\w+\s*;', line): error(filename, linenum, 'build/forward_decl', 5, 'Inner-style forward declarations are invalid. Remove this line.') - if Search(r'(\w+|[+-]?\d+(\.\d*)?)\s*(<|>)\?=?\s*(\w+|[+-]?\d+)(\.\d*)?', + if re.search(r'(\w+|[+-]?\d+(\.\d*)?)\s*(<|>)\?=?\s*(\w+|[+-]?\d+)(\.\d*)?', line): error(filename, linenum, 'build/deprecated', 3, '>? and = 1 and not noarg_constructor and - len(defaulted_args) >= len(constructor_args) - 1)) + len(defaulted_args) >= len(constructor_args) - 1) or + # variadic arguments with zero or one argument + (len(constructor_args) <= 2 and + len(variadic_args) >= 1)) initializer_list_constructor = bool( onearg_constructor and - Search(r'\bstd\s*::\s*initializer_list\b', constructor_args[0])) + re.search(r'\bstd\s*::\s*initializer_list\b', constructor_args[0])) copy_constructor = bool( onearg_constructor and - Match(r'(const\s+)?%s(\s*<[^>]*>)?(\s+const)?\s*(?:<\w+>\s*)?&' - % re.escape(base_classname), constructor_args[0].strip())) + re.match(r'((const\s+(volatile\s+)?)?|(volatile\s+(const\s+)?))?' + rf'{re.escape(base_classname)}(\s*<[^>]*>)?(\s+const)?\s*(?:<\w+>\s*)?&', + constructor_args[0].strip()) + ) if (not is_marked_explicit and onearg_constructor and not initializer_list_constructor and not copy_constructor): - if defaulted_args: - error(filename, linenum, 'runtime/explicit', 5, + if defaulted_args or variadic_args: + error(filename, linenum, 'runtime/explicit', 4, 'Constructors callable with one argument ' 'should be marked explicit.') else: - error(filename, linenum, 'runtime/explicit', 5, + error(filename, linenum, 'runtime/explicit', 4, 'Single-parameter constructors should be marked explicit.') - elif is_marked_explicit and not onearg_constructor: - if noarg_constructor: - error(filename, linenum, 'runtime/explicit', 5, - 'Zero-parameter constructors should not be marked explicit.') - else: - error(filename, linenum, 'runtime/explicit', 0, - 'Constructors that require multiple arguments ' - 'should not be marked explicit.') def CheckSpacingForFunctionCall(filename, clean_lines, linenum, error): @@ -2756,7 +3543,7 @@ def CheckSpacingForFunctionCall(filename, clean_lines, linenum, error): r'\bfor\s*\((.*)\)\s*{', r'\bwhile\s*\((.*)\)\s*[{;]', r'\bswitch\s*\((.*)\)\s*{'): - match = Search(pattern, line) + match = re.search(pattern, line) if match: fncall = match.group(1) # look inside the parens for function calls break @@ -2775,25 +3562,26 @@ def CheckSpacingForFunctionCall(filename, clean_lines, linenum, error): # Note that we assume the contents of [] to be short enough that # they'll never need to wrap. if ( # Ignore control structures. - not Search(r'\b(if|for|while|switch|return|new|delete|catch|sizeof)\b', + not re.search(r'\b(if|elif|for|while|switch|return|new|delete|catch|sizeof)\b', fncall) and # Ignore pointers/references to functions. - not Search(r' \([^)]+\)\([^)]*(\)|,$)', fncall) and + not re.search(r' \([^)]+\)\([^)]*(\)|,$)', fncall) and # Ignore pointers/references to arrays. - not Search(r' \([^)]+\)\[[^\]]+\]', fncall)): - if Search(r'\w\s*\(\s(?!\s*\\$)', fncall): # a ( used for a fn call + not re.search(r' \([^)]+\)\[[^\]]+\]', fncall)): + if re.search(r'\w\s*\(\s(?!\s*\\$)', fncall): # a ( used for a fn call error(filename, linenum, 'whitespace/parens', 4, 'Extra space after ( in function call') - elif Search(r'\(\s+(?!(\s*\\)|\()', fncall): + elif re.search(r'\(\s+(?!(\s*\\)|\()', fncall): error(filename, linenum, 'whitespace/parens', 2, 'Extra space after (') - if (Search(r'\w\s+\(', fncall) and - not Search(r'#\s*define|typedef|using\s+\w+\s*=', fncall) and - not Search(r'\w\s+\((\w+::)*\*\w+\)\(', fncall) and - not Search(r'\bcase\s+\(', fncall)): + if (re.search(r'\w\s+\(', fncall) and + not re.search(r'_{0,2}asm_{0,2}\s+_{0,2}volatile_{0,2}\s+\(', fncall) and + not re.search(r'#\s*define|typedef|using\s+\w+\s*=', fncall) and + not re.search(r'\w\s+\((\w+::)*\*\w+\)\(', fncall) and + not re.search(r'\bcase\s+\(', fncall)): # TODO(unknown): Space after an operator function seem to be a common # error, silence those for now by restricting them to highest verbosity. - if Search(r'\boperator_*\b', line): + if re.search(r'\boperator_*\b', line): error(filename, linenum, 'whitespace/parens', 0, 'Extra space before ( in function call') else: @@ -2801,10 +3589,10 @@ def CheckSpacingForFunctionCall(filename, clean_lines, linenum, error): 'Extra space before ( in function call') # If the ) is followed only by a newline or a { + newline, assume it's # part of a control statement (if/while/etc), and don't complain - if Search(r'[^)]\s+\)\s*[^{\s]', fncall): + if re.search(r'[^)]\s+\)\s*[^{\s]', fncall): # If the closing parenthesis is preceded by only whitespaces, # try to give a more descriptive error message. - if Search(r'^\s+\)', fncall): + if re.search(r'^\s+\)', fncall): error(filename, linenum, 'whitespace/parens', 2, 'Closing ) should be moved to the previous line') else: @@ -2830,10 +3618,10 @@ def IsBlankLine(line): def CheckForNamespaceIndentation(filename, nesting_state, clean_lines, line, error): is_namespace_indent_item = ( - len(nesting_state.stack) > 1 and - nesting_state.stack[-1].check_namespace_indentation and - isinstance(nesting_state.previous_stack_top, _NamespaceInfo) and - nesting_state.previous_stack_top == nesting_state.stack[-2]) + len(nesting_state.stack) >= 1 and + (isinstance(nesting_state.stack[-1], _NamespaceInfo) or + (isinstance(nesting_state.previous_stack_top, _NamespaceInfo))) + ) if ShouldCheckNamespaceIndentation(nesting_state, is_namespace_indent_item, clean_lines.elided, line): @@ -2846,7 +3634,7 @@ def CheckForFunctionLengths(filename, clean_lines, linenum, """Reports for long function bodies. For an overview why this is done, see: - http://google-styleguide.googlecode.com/svn/trunk/cppguide.xml#Write_Short_Functions + https://google-styleguide.googlecode.com/svn/trunk/cppguide.xml#Write_Short_Functions Uses a simplistic algorithm assuming other style guidelines (especially spacing) are followed. @@ -2870,28 +3658,28 @@ def CheckForFunctionLengths(filename, clean_lines, linenum, starting_func = False regexp = r'(\w(\w|::|\*|\&|\s)*)\(' # decls * & space::name( ... - match_result = Match(regexp, line) + match_result = re.match(regexp, line) if match_result: # If the name is all caps and underscores, figure it's a macro and # ignore it, unless it's TEST or TEST_F. function_name = match_result.group(1).split()[-1] if function_name == 'TEST' or function_name == 'TEST_F' or ( - not Match(r'[A-Z_]+$', function_name)): + not re.match(r'[A-Z_]+$', function_name)): starting_func = True if starting_func: body_found = False - for start_linenum in xrange(linenum, clean_lines.NumLines()): + for start_linenum in range(linenum, clean_lines.NumLines()): start_line = lines[start_linenum] joined_line += ' ' + start_line.lstrip() - if Search(r'(;|})', start_line): # Declarations and trivial functions + if re.search(r'(;|})', start_line): # Declarations and trivial functions body_found = True break # ... ignore - elif Search(r'{', start_line): + if re.search(r'{', start_line): body_found = True - function = Search(r'((\w|:)*)\(', line).group(1) - if Match(r'TEST', function): # Handle TEST... macros - parameter_regexp = Search(r'(\(.*\))', joined_line) + function = re.search(r'((\w|:)*)\(', line).group(1) + if re.match(r'TEST', function): # Handle TEST... macros + parameter_regexp = re.search(r'(\(.*\))', joined_line) if parameter_regexp: # Ignore bad syntax function += parameter_regexp.group(1) else: @@ -2902,10 +3690,10 @@ def CheckForFunctionLengths(filename, clean_lines, linenum, # No body for the function (or evidence of a non-function) was found. error(filename, linenum, 'readability/fn_size', 5, 'Lint failed to find start of function body.') - elif Match(r'^\}\s*$', line): # function end + elif re.match(r'^\}\s*$', line): # function end function_state.Check(error, filename, linenum) function_state.End() - elif not Match(r'^\s*$', line): + elif not re.match(r'^\s*$', line): function_state.Count() # Count non-blank/non-comment lines. @@ -2925,11 +3713,9 @@ def CheckComment(line, filename, linenum, next_line_start, error): commentpos = line.find('//') if commentpos != -1: # Check if the // may be in quotes. If so, ignore it - # Comparisons made explicit for clarity -- pylint: disable=g-explicit-bool-comparison - if (line.count('"', 0, commentpos) - - line.count('\\"', 0, commentpos)) % 2 == 0: # not in quotes + if re.sub(r'\\.', '', line[0:commentpos]).count('"') % 2 == 0: # Allow one space for new scopes, two spaces otherwise: - if (not (Match(r'^.*{ *//', line) and next_line_start == commentpos) and + if (not (re.match(r'^.*{ *//', line) and next_line_start == commentpos) and ((commentpos >= 1 and line[commentpos-1] not in string.whitespace) or (commentpos >= 2 and @@ -2954,7 +3740,8 @@ def CheckComment(line, filename, linenum, next_line_start, error): '"// TODO(my_username): Stuff."') middle_whitespace = match.group(3) - # Comparisons made explicit for correctness -- pylint: disable=g-explicit-bool-comparison + # Comparisons made explicit for correctness + # -- pylint: disable=g-explicit-bool-comparison if middle_whitespace != ' ' and middle_whitespace != '': error(filename, linenum, 'whitespace/todo', 2, 'TODO(my_username) should be followed by a space') @@ -2962,42 +3749,12 @@ def CheckComment(line, filename, linenum, next_line_start, error): # If the comment contains an alphanumeric character, there # should be a space somewhere between it and the // unless # it's a /// or //! Doxygen comment. - if (Match(r'//[^ ]*\w', comment) and - not Match(r'(///|//\!)(\s+|$)', comment)): + if (re.match(r'//[^ ]*\w', comment) and + not re.match(r'(///|//\!)(\s+|$)', comment)): error(filename, linenum, 'whitespace/comments', 4, 'Should have a space between // and comment') -def CheckAccess(filename, clean_lines, linenum, nesting_state, error): - """Checks for improper use of DISALLOW* macros. - - Args: - filename: The name of the current file. - clean_lines: A CleansedLines instance containing the file. - linenum: The number of the line to check. - nesting_state: A NestingState instance which maintains information about - the current stack of nested blocks being parsed. - error: The function to call with any errors found. - """ - line = clean_lines.elided[linenum] # get rid of comments and strings - - matched = Match((r'\s*(DISALLOW_COPY_AND_ASSIGN|' - r'DISALLOW_IMPLICIT_CONSTRUCTORS)'), line) - if not matched: - return - if nesting_state.stack and isinstance(nesting_state.stack[-1], _ClassInfo): - if nesting_state.stack[-1].access != 'private': - error(filename, linenum, 'readability/constructors', 3, - '%s must be in the private: section' % matched.group(1)) - - else: - # Found DISALLOW* macro outside a class declaration, or perhaps it - # was used inside a function when it should have been part of the - # class declaration. We could issue a warning here, but it - # probably resulted in a compiler error already. - pass - - def CheckSpacing(filename, clean_lines, linenum, nesting_state, error): """Checks for the correctness of various spacing issues in the code. @@ -3056,12 +3813,12 @@ def CheckSpacing(filename, clean_lines, linenum, nesting_state, error): # the previous line is indented 6 spaces, which may happen when the # initializers of a constructor do not fit into a 80 column line. exception = False - if Match(r' {6}\w', prev_line): # Initializer list? + if re.match(r' {6}\w', prev_line): # Initializer list? # We are looking for the opening column of initializer list, which # should be indented 4 spaces to cause 6 space indentation afterwards. search_position = linenum-2 while (search_position >= 0 - and Match(r' {6}\w', elided[search_position])): + and re.match(r' {6}\w', elided[search_position])): search_position -= 1 exception = (search_position >= 0 and elided[search_position][:5] == ' :') @@ -3072,9 +3829,9 @@ def CheckSpacing(filename, clean_lines, linenum, nesting_state, error): # or colon (for initializer lists) we assume that it is the last line of # a function header. If we have a colon indented 4 spaces, it is an # initializer list. - exception = (Match(r' {4}\w[^\(]*\)\s*(const\s*)?(\{\s*$|:)', + exception = (re.match(r' {4}\w[^\(]*\)\s*(const\s*)?(\{\s*$|:)', prev_line) - or Match(r' {4}:', prev_line)) + or re.match(r' {4}:', prev_line)) if not exception: error(filename, linenum, 'whitespace/blank_line', 2, @@ -3091,16 +3848,16 @@ def CheckSpacing(filename, clean_lines, linenum, nesting_state, error): if linenum + 1 < clean_lines.NumLines(): next_line = raw[linenum + 1] if (next_line - and Match(r'\s*}', next_line) + and re.match(r'\s*}', next_line) and next_line.find('} else ') == -1): error(filename, linenum, 'whitespace/blank_line', 3, 'Redundant blank line at the end of a code block ' 'should be deleted.') - matched = Match(r'\s*(public|protected|private):', prev_line) + matched = re.match(r'\s*(public|protected|private):', prev_line) if matched: error(filename, linenum, 'whitespace/blank_line', 3, - 'Do not leave a blank line after "%s:"' % matched.group(1)) + f'Do not leave a blank line after "{matched.group(1)}:"') # Next, check comments next_line_start = 0 @@ -3112,16 +3869,17 @@ def CheckSpacing(filename, clean_lines, linenum, nesting_state, error): # get rid of comments and strings line = clean_lines.elided[linenum] - # You shouldn't have spaces before your brackets, except maybe after - # 'delete []' or 'return []() {};' - if Search(r'\w\s+\[', line) and not Search(r'(?:delete|return)\s+\[', line): + # You shouldn't have spaces before your brackets, except for C++11 attributes + # or maybe after 'delete []', 'return []() {};', or 'auto [abc, ...] = ...;'. + if (re.search(r'\w\s+\[(?!\[)', line) and + not re.search(r'(?:auto&?|delete|return)\s+\[', line)): error(filename, linenum, 'whitespace/braces', 5, 'Extra space before [') # In range-based for, we wanted spaces before and after the colon, but # not around "::" tokens that might appear. - if (Search(r'for *\(.*[^:]:[^: ]', line) or - Search(r'for *\(.*[^: ]:[^:]', line)): + if (re.search(r'for *\(.*[^:]:[^: ]', line) or + re.search(r'for *\(.*[^: ]:[^:]', line)): error(filename, linenum, 'whitespace/forcolon', 2, 'Missing space around colon in range-based for loop') @@ -3144,7 +3902,7 @@ def CheckOperatorSpacing(filename, clean_lines, linenum, error): # The replacement is done repeatedly to avoid false positives from # operators that call operators. while True: - match = Match(r'^(.*\boperator\b)(\S+)(\s*\(.*)$', line) + match = re.match(r'^(.*\boperator\b)(\S+)(\s*\(.*)$', line) if match: line = match.group(1) + ('_' * len(match.group(2))) + match.group(3) else: @@ -3154,12 +3912,12 @@ def CheckOperatorSpacing(filename, clean_lines, linenum, error): # Otherwise not. Note we only check for non-spaces on *both* sides; # sometimes people put non-spaces on one side when aligning ='s among # many lines (not that this is behavior that I approve of...) - if ((Search(r'[\w.]=', line) or - Search(r'=[\w.]', line)) - and not Search(r'\b(if|while|for) ', line) + if ((re.search(r'[\w.]=', line) or + re.search(r'=[\w.]', line)) + and not re.search(r'\b(if|while|for) ', line) # Operators taken from [lex.operators] in C++11 standard. - and not Search(r'(>=|<=|==|!=|&=|\^=|\|=|\+=|\*=|\/=|\%=)', line) - and not Search(r'operator=', line)): + and not re.search(r'(>=|<=|==|!=|&=|\^=|\|=|\+=|\*=|\/=|\%=)', line) + and not re.search(r'operator=', line)): error(filename, linenum, 'whitespace/operators', 4, 'Missing spaces around =') @@ -3176,18 +3934,19 @@ def CheckOperatorSpacing(filename, clean_lines, linenum, error): # macro context and don't do any checks. This avoids false # positives. # - # Note that && is not included here. Those are checked separately - # in CheckRValueReference - match = Search(r'[^<>=!\s](==|!=|<=|>=|\|\|)[^<>=!\s,;\)]', line) + # Note that && is not included here. This is because there are too + # many false positives due to RValue references. + match = re.search(r'[^<>=!\s](==|!=|<=|>=|\|\|)[^<>=!\s,;\)]', line) if match: + # TODO: support alternate operators error(filename, linenum, 'whitespace/operators', 3, - 'Missing spaces around %s' % match.group(1)) - elif not Match(r'#.*include', line): + f'Missing spaces around {match.group(1)}') + elif not re.match(r'#.*include', line): # Look for < that is not surrounded by spaces. This is only # triggered if both sides are missing spaces, even though # technically should should flag if at least one side is missing a # space. This is done to avoid some false positives with shifts. - match = Match(r'^(.*[^\s<])<[^\s=<,]', line) + match = re.match(r'^(.*[^\s<])<[^\s=<,]', line) if match: (_, _, end_pos) = CloseExpression( clean_lines, linenum, len(match.group(1))) @@ -3198,7 +3957,7 @@ def CheckOperatorSpacing(filename, clean_lines, linenum, error): # Look for > that is not surrounded by spaces. Similar to the # above, we only trigger if both sides are missing spaces to avoid # false positives with shifts. - match = Match(r'^(.*[^-\s>])>[^\s=>,]', line) + match = re.match(r'^(.*[^-\s>])>[^\s=>,]', line) if match: (_, _, start_pos) = ReverseCloseExpression( clean_lines, linenum, len(match.group(1))) @@ -3211,7 +3970,7 @@ def CheckOperatorSpacing(filename, clean_lines, linenum, error): # # We also allow operators following an opening parenthesis, since # those tend to be macros that deal with operators. - match = Search(r'(operator|[^\s(<])(?:L|UL|ULL|l|ul|ull)?<<([^\s,=<])', line) + match = re.search(r'(operator|[^\s(<])(?:L|UL|LL|ULL|l|ul|ll|ull)?<<([^\s,=<])', line) if (match and not (match.group(1).isdigit() and match.group(2).isdigit()) and not (match.group(1) == 'operator' and match.group(2) == ';')): error(filename, linenum, 'whitespace/operators', 3, @@ -3229,16 +3988,16 @@ def CheckOperatorSpacing(filename, clean_lines, linenum, error): # follows would be part of an identifier, and there should still be # a space separating the template type and the identifier. # type> alpha - match = Search(r'>>[a-zA-Z_]', line) + match = re.search(r'>>[a-zA-Z_]', line) if match: error(filename, linenum, 'whitespace/operators', 3, 'Missing spaces around >>') # There shouldn't be space around unary operators - match = Search(r'(!\s|~\s|[\s]--[\s;]|[\s]\+\+[\s;])', line) + match = re.search(r'(!\s|~\s|[\s]--[\s;]|[\s]\+\+[\s;])', line) if match: error(filename, linenum, 'whitespace/operators', 4, - 'Extra space for operator %s' % match.group(1)) + f'Extra space for operator {match.group(1)}') def CheckParenthesisSpacing(filename, clean_lines, linenum, error): @@ -3253,30 +4012,29 @@ def CheckParenthesisSpacing(filename, clean_lines, linenum, error): line = clean_lines.elided[linenum] # No spaces after an if, while, switch, or for - match = Search(r' (if\(|for\(|while\(|switch\()', line) + match = re.search(r' (if\(|for\(|while\(|switch\()', line) if match: error(filename, linenum, 'whitespace/parens', 5, - 'Missing space before ( in %s' % match.group(1)) + f'Missing space before ( in {match.group(1)}') # For if/for/while/switch, the left and right parens should be # consistent about how many spaces are inside the parens, and # there should either be zero or one spaces inside the parens. # We don't want: "if ( foo)" or "if ( foo )". # Exception: "for ( ; foo; bar)" and "for (foo; bar; )" are allowed. - match = Search(r'\b(if|for|while|switch)\s*' + match = re.search(r'\b(if|for|while|switch)\s*' r'\(([ ]*)(.).*[^ ]+([ ]*)\)\s*{\s*$', line) if match: if len(match.group(2)) != len(match.group(4)): if not (match.group(3) == ';' and len(match.group(2)) == 1 + len(match.group(4)) or - not match.group(2) and Search(r'\bfor\s*\(.*; \)', line)): + not match.group(2) and re.search(r'\bfor\s*\(.*; \)', line)): error(filename, linenum, 'whitespace/parens', 5, - 'Mismatching spaces inside () in %s' % match.group(1)) + f'Mismatching spaces inside () in {match.group(1)}') if len(match.group(2)) not in [0, 1]: error(filename, linenum, 'whitespace/parens', 5, - 'Should have zero or one spaces inside ( and ) in %s' % - match.group(1)) + f'Should have zero or one spaces inside ( and ) in {match.group(1)}') def CheckCommaSpacing(filename, clean_lines, linenum, error): @@ -3301,8 +4059,9 @@ def CheckCommaSpacing(filename, clean_lines, linenum, error): # verify that lines contain missing whitespaces, second pass on raw # lines to confirm that those missing whitespaces are not due to # elided comments. - if (Search(r',[^,\s]', ReplaceAll(r'\boperator\s*,\s*\(', 'F(', line)) and - Search(r',[^,\s]', raw[linenum])): + match = re.search(r',[^,\s]', re.sub(r'\b__VA_OPT__\s*\(,\)', '', + re.sub(r'\boperator\s*,\s*\(', 'F(', line))) + if (match and re.search(r',[^,\s]', raw[linenum])): error(filename, linenum, 'whitespace/comma', 3, 'Missing space after ,') @@ -3310,27 +4069,95 @@ def CheckCommaSpacing(filename, clean_lines, linenum, error): # except for few corner cases # TODO(unknown): clarify if 'if (1) { return 1;}' is requires one more # space after ; - if Search(r';[^\s};\\)/]', line): + if re.search(r';[^\s};\\)/]', line): error(filename, linenum, 'whitespace/semicolon', 3, 'Missing space after ;') -def CheckBracesSpacing(filename, clean_lines, linenum, error): +def _IsType(clean_lines, nesting_state, expr): + """Check if expression looks like a type name, returns true if so. + + Args: + clean_lines: A CleansedLines instance containing the file. + nesting_state: A NestingState instance which maintains information about + the current stack of nested blocks being parsed. + expr: The expression to check. + Returns: + True, if token looks like a type. + """ + # Keep only the last token in the expression + last_word = re.match(r'^.*(\b\S+)$', expr) + if last_word: + token = last_word.group(1) + else: + token = expr + + # Match native types and stdint types + if _TYPES.match(token): + return True + + # Try a bit harder to match templated types. Walk up the nesting + # stack until we find something that resembles a typename + # declaration for what we are looking for. + typename_pattern = (r'\b(?:typename|class|struct)\s+' + re.escape(token) + + r'\b') + block_index = len(nesting_state.stack) - 1 + while block_index >= 0: + if isinstance(nesting_state.stack[block_index], _NamespaceInfo): + return False + + # Found where the opening brace is. We want to scan from this + # line up to the beginning of the function, minus a few lines. + # template + # class C + # : public ... { // start scanning here + last_line = nesting_state.stack[block_index].starting_linenum + + next_block_start = 0 + if block_index > 0: + next_block_start = nesting_state.stack[block_index - 1].starting_linenum + first_line = last_line + while first_line >= next_block_start: + if clean_lines.elided[first_line].find('template') >= 0: + break + first_line -= 1 + if first_line < next_block_start: + # Didn't find any "template" keyword before reaching the next block, + # there are probably no template things to check for this block + block_index -= 1 + continue + + # Look for typename in the specified range + for i in range(first_line, last_line + 1, 1): + if re.search(typename_pattern, clean_lines.elided[i]): + return True + block_index -= 1 + + return False + + +def CheckBracesSpacing(filename, clean_lines, linenum, nesting_state, error): """Checks for horizontal spacing near commas. Args: filename: The name of the current file. clean_lines: A CleansedLines instance containing the file. linenum: The number of the line to check. + nesting_state: A NestingState instance which maintains information about + the current stack of nested blocks being parsed. error: The function to call with any errors found. """ line = clean_lines.elided[linenum] # Except after an opening paren, or after another opening brace (in case of # an initializer list, for instance), you should have spaces before your - # braces. And since you should never have braces at the beginning of a line, - # this is an easy test. - match = Match(r'^(.*[^ ({>]){', line) + # braces when they are delimiting blocks, classes, namespaces etc. + # And since you should never have braces at the beginning of a line, + # this is an easy test. Except that braces used for initialization don't + # follow the same rule; we often don't want spaces before those. + match = re.match(r'^(.*[^ ({>]){', line) + if match: # Try a bit harder to check for brace initialization. This # happens in one of the following forms: @@ -3360,35 +4187,40 @@ def CheckBracesSpacing(filename, clean_lines, linenum, error): # There is a false negative with this approach if people inserted # spurious semicolons, e.g. "if (cond){};", but we will catch the # spurious semicolon with a separate check. + leading_text = match.group(1) (endline, endlinenum, endpos) = CloseExpression( clean_lines, linenum, len(match.group(1))) trailing_text = '' if endpos > -1: trailing_text = endline[endpos:] - for offset in xrange(endlinenum + 1, + for offset in range(endlinenum + 1, min(endlinenum + 3, clean_lines.NumLines() - 1)): trailing_text += clean_lines.elided[offset] - if not Match(r'^[\s}]*[{.;,)<>\]:]', trailing_text): + # We also suppress warnings for `uint64_t{expression}` etc., as the style + # guide recommends brace initialization for integral types to avoid + # overflow/truncation. + if (not re.match(r'^[\s}]*[{.;,)<>\]:]', trailing_text) + and not _IsType(clean_lines, nesting_state, leading_text)): error(filename, linenum, 'whitespace/braces', 5, 'Missing space before {') # Make sure '} else {' has spaces. - if Search(r'}else', line): + if re.search(r'}else', line): error(filename, linenum, 'whitespace/braces', 5, 'Missing space before else') # You shouldn't have a space before a semicolon at the end of the line. # There's a special case for "for" since the style guide allows space before # the semicolon there. - if Search(r':\s*;\s*$', line): + if re.search(r':\s*;\s*$', line): error(filename, linenum, 'whitespace/semicolon', 5, 'Semicolon defining empty statement. Use {} instead.') - elif Search(r'^\s*;\s*$', line): + elif re.search(r'^\s*;\s*$', line): error(filename, linenum, 'whitespace/semicolon', 5, 'Line contains only semicolon. If this should be an empty statement, ' 'use {} instead.') - elif (Search(r'\s+;\s*$', line) and - not Search(r'\bfor\b', line)): + elif (re.search(r'\s+;\s*$', line) and + not re.search(r'\bfor\b', line)): error(filename, linenum, 'whitespace/semicolon', 5, 'Extra space before last semicolon. If this should be an empty ' 'statement, use {} instead.') @@ -3407,410 +4239,10 @@ def IsDecltype(clean_lines, linenum, column): (text, _, start_col) = ReverseCloseExpression(clean_lines, linenum, column) if start_col < 0: return False - if Search(r'\bdecltype\s*$', text[0:start_col]): + if re.search(r'\bdecltype\s*$', text[0:start_col]): return True return False - -def IsTemplateParameterList(clean_lines, linenum, column): - """Check if the token ending on (linenum, column) is the end of template<>. - - Args: - clean_lines: A CleansedLines instance containing the file. - linenum: the number of the line to check. - column: end column of the token to check. - Returns: - True if this token is end of a template parameter list, False otherwise. - """ - (_, startline, startpos) = ReverseCloseExpression( - clean_lines, linenum, column) - if (startpos > -1 and - Search(r'\btemplate\s*$', clean_lines.elided[startline][0:startpos])): - return True - return False - - -def IsRValueType(typenames, clean_lines, nesting_state, linenum, column): - """Check if the token ending on (linenum, column) is a type. - - Assumes that text to the right of the column is "&&" or a function - name. - - Args: - typenames: set of type names from template-argument-list. - clean_lines: A CleansedLines instance containing the file. - nesting_state: A NestingState instance which maintains information about - the current stack of nested blocks being parsed. - linenum: the number of the line to check. - column: end column of the token to check. - Returns: - True if this token is a type, False if we are not sure. - """ - prefix = clean_lines.elided[linenum][0:column] - - # Get one word to the left. If we failed to do so, this is most - # likely not a type, since it's unlikely that the type name and "&&" - # would be split across multiple lines. - match = Match(r'^(.*)(\b\w+|[>*)&])\s*$', prefix) - if not match: - return False - - # Check text following the token. If it's "&&>" or "&&," or "&&...", it's - # most likely a rvalue reference used inside a template. - suffix = clean_lines.elided[linenum][column:] - if Match(r'&&\s*(?:[>,]|\.\.\.)', suffix): - return True - - # Check for known types and end of templates: - # int&& variable - # vector&& variable - # - # Because this function is called recursively, we also need to - # recognize pointer and reference types: - # int* Function() - # int& Function() - if (match.group(2) in typenames or - match.group(2) in ['char', 'char16_t', 'char32_t', 'wchar_t', 'bool', - 'short', 'int', 'long', 'signed', 'unsigned', - 'float', 'double', 'void', 'auto', '>', '*', '&']): - return True - - # If we see a close parenthesis, look for decltype on the other side. - # decltype would unambiguously identify a type, anything else is - # probably a parenthesized expression and not a type. - if match.group(2) == ')': - return IsDecltype( - clean_lines, linenum, len(match.group(1)) + len(match.group(2)) - 1) - - # Check for casts and cv-qualifiers. - # match.group(1) remainder - # -------------- --------- - # const_cast< type&& - # const type&& - # type const&& - if Search(r'\b(?:const_cast\s*<|static_cast\s*<|dynamic_cast\s*<|' - r'reinterpret_cast\s*<|\w+\s)\s*$', - match.group(1)): - return True - - # Look for a preceding symbol that might help differentiate the context. - # These are the cases that would be ambiguous: - # match.group(1) remainder - # -------------- --------- - # Call ( expression && - # Declaration ( type&& - # sizeof ( type&& - # if ( expression && - # while ( expression && - # for ( type&& - # for( ; expression && - # statement ; type&& - # block { type&& - # constructor { expression && - start = linenum - line = match.group(1) - match_symbol = None - while start >= 0: - # We want to skip over identifiers and commas to get to a symbol. - # Commas are skipped so that we can find the opening parenthesis - # for function parameter lists. - match_symbol = Match(r'^(.*)([^\w\s,])[\w\s,]*$', line) - if match_symbol: - break - start -= 1 - line = clean_lines.elided[start] - - if not match_symbol: - # Probably the first statement in the file is an rvalue reference - return True - - if match_symbol.group(2) == '}': - # Found closing brace, probably an indicate of this: - # block{} type&& - return True - - if match_symbol.group(2) == ';': - # Found semicolon, probably one of these: - # for(; expression && - # statement; type&& - - # Look for the previous 'for(' in the previous lines. - before_text = match_symbol.group(1) - for i in xrange(start - 1, max(start - 6, 0), -1): - before_text = clean_lines.elided[i] + before_text - if Search(r'for\s*\([^{};]*$', before_text): - # This is the condition inside a for-loop - return False - - # Did not find a for-init-statement before this semicolon, so this - # is probably a new statement and not a condition. - return True - - if match_symbol.group(2) == '{': - # Found opening brace, probably one of these: - # block{ type&& = ... ; } - # constructor{ expression && expression } - - # Look for a closing brace or a semicolon. If we see a semicolon - # first, this is probably a rvalue reference. - line = clean_lines.elided[start][0:len(match_symbol.group(1)) + 1] - end = start - depth = 1 - while True: - for ch in line: - if ch == ';': - return True - elif ch == '{': - depth += 1 - elif ch == '}': - depth -= 1 - if depth == 0: - return False - end += 1 - if end >= clean_lines.NumLines(): - break - line = clean_lines.elided[end] - # Incomplete program? - return False - - if match_symbol.group(2) == '(': - # Opening parenthesis. Need to check what's to the left of the - # parenthesis. Look back one extra line for additional context. - before_text = match_symbol.group(1) - if linenum > 1: - before_text = clean_lines.elided[linenum - 1] + before_text - before_text = match_symbol.group(1) - - # Patterns that are likely to be types: - # [](type&& - # for (type&& - # sizeof(type&& - # operator=(type&& - # - if Search(r'(?:\]|\bfor|\bsizeof|\boperator\s*\S+\s*)\s*$', before_text): - return True - - # Patterns that are likely to be expressions: - # if (expression && - # while (expression && - # : initializer(expression && - # , initializer(expression && - # ( FunctionCall(expression && - # + FunctionCall(expression && - # + (expression && - # - # The last '+' represents operators such as '+' and '-'. - if Search(r'(?:\bif|\bwhile|[-+=%^(]*>)?\s*$', - match_symbol.group(1)) - if match_func: - # Check for constructors, which don't have return types. - if Search(r'\b(?:explicit|inline)$', match_func.group(1)): - return True - implicit_constructor = Match(r'\s*(\w+)\((?:const\s+)?(\w+)', prefix) - if (implicit_constructor and - implicit_constructor.group(1) == implicit_constructor.group(2)): - return True - return IsRValueType(typenames, clean_lines, nesting_state, linenum, - len(match_func.group(1))) - - # Nothing before the function name. If this is inside a block scope, - # this is probably a function call. - return not (nesting_state.previous_stack_top and - nesting_state.previous_stack_top.IsBlockInfo()) - - if match_symbol.group(2) == '>': - # Possibly a closing bracket, check that what's on the other side - # looks like the start of a template. - return IsTemplateParameterList( - clean_lines, start, len(match_symbol.group(1))) - - # Some other symbol, usually something like "a=b&&c". This is most - # likely not a type. - return False - - -def IsDeletedOrDefault(clean_lines, linenum): - """Check if current constructor or operator is deleted or default. - - Args: - clean_lines: A CleansedLines instance containing the file. - linenum: The number of the line to check. - Returns: - True if this is a deleted or default constructor. - """ - open_paren = clean_lines.elided[linenum].find('(') - if open_paren < 0: - return False - (close_line, _, close_paren) = CloseExpression( - clean_lines, linenum, open_paren) - if close_paren < 0: - return False - return Match(r'\s*=\s*(?:delete|default)\b', close_line[close_paren:]) - - -def IsRValueAllowed(clean_lines, linenum, typenames): - """Check if RValue reference is allowed on a particular line. - - Args: - clean_lines: A CleansedLines instance containing the file. - linenum: The number of the line to check. - typenames: set of type names from template-argument-list. - Returns: - True if line is within the region where RValue references are allowed. - """ - # Allow region marked by PUSH/POP macros - for i in xrange(linenum, 0, -1): - line = clean_lines.elided[i] - if Match(r'GOOGLE_ALLOW_RVALUE_REFERENCES_(?:PUSH|POP)', line): - if not line.endswith('PUSH'): - return False - for j in xrange(linenum, clean_lines.NumLines(), 1): - line = clean_lines.elided[j] - if Match(r'GOOGLE_ALLOW_RVALUE_REFERENCES_(?:PUSH|POP)', line): - return line.endswith('POP') - - # Allow operator= - line = clean_lines.elided[linenum] - if Search(r'\boperator\s*=\s*\(', line): - return IsDeletedOrDefault(clean_lines, linenum) - - # Allow constructors - match = Match(r'\s*(?:[\w<>]+::)*([\w<>]+)\s*::\s*([\w<>]+)\s*\(', line) - if match and match.group(1) == match.group(2): - return IsDeletedOrDefault(clean_lines, linenum) - if Search(r'\b(?:explicit|inline)\s+[\w<>]+\s*\(', line): - return IsDeletedOrDefault(clean_lines, linenum) - - if Match(r'\s*[\w<>]+\s*\(', line): - previous_line = 'ReturnType' - if linenum > 0: - previous_line = clean_lines.elided[linenum - 1] - if Match(r'^\s*$', previous_line) or Search(r'[{}:;]\s*$', previous_line): - return IsDeletedOrDefault(clean_lines, linenum) - - # Reject types not mentioned in template-argument-list - while line: - match = Match(r'^.*?(\w+)\s*&&(.*)$', line) - if not match: - break - if match.group(1) not in typenames: - return False - line = match.group(2) - - # All RValue types that were in template-argument-list should have - # been removed by now. Those were allowed, assuming that they will - # be forwarded. - # - # If there are no remaining RValue types left (i.e. types that were - # not found in template-argument-list), flag those as not allowed. - return line.find('&&') < 0 - - -def GetTemplateArgs(clean_lines, linenum): - """Find list of template arguments associated with this function declaration. - - Args: - clean_lines: A CleansedLines instance containing the file. - linenum: Line number containing the start of the function declaration, - usually one line after the end of the template-argument-list. - Returns: - Set of type names, or empty set if this does not appear to have - any template parameters. - """ - # Find start of function - func_line = linenum - while func_line > 0: - line = clean_lines.elided[func_line] - if Match(r'^\s*$', line): - return set() - if line.find('(') >= 0: - break - func_line -= 1 - if func_line == 0: - return set() - - # Collapse template-argument-list into a single string - argument_list = '' - match = Match(r'^(\s*template\s*)<', clean_lines.elided[func_line]) - if match: - # template-argument-list on the same line as function name - start_col = len(match.group(1)) - _, end_line, end_col = CloseExpression(clean_lines, func_line, start_col) - if end_col > -1 and end_line == func_line: - start_col += 1 # Skip the opening bracket - argument_list = clean_lines.elided[func_line][start_col:end_col] - - elif func_line > 1: - # template-argument-list one line before function name - match = Match(r'^(.*)>\s*$', clean_lines.elided[func_line - 1]) - if match: - end_col = len(match.group(1)) - _, start_line, start_col = ReverseCloseExpression( - clean_lines, func_line - 1, end_col) - if start_col > -1: - start_col += 1 # Skip the opening bracket - while start_line < func_line - 1: - argument_list += clean_lines.elided[start_line][start_col:] - start_col = 0 - start_line += 1 - argument_list += clean_lines.elided[func_line - 1][start_col:end_col] - - if not argument_list: - return set() - - # Extract type names - typenames = set() - while True: - match = Match(r'^[,\s]*(?:typename|class)(?:\.\.\.)?\s+(\w+)(.*)$', - argument_list) - if not match: - break - typenames.add(match.group(1)) - argument_list = match.group(2) - return typenames - - -def CheckRValueReference(filename, clean_lines, linenum, nesting_state, error): - """Check for rvalue references. - - Args: - filename: The name of the current file. - clean_lines: A CleansedLines instance containing the file. - linenum: The number of the line to check. - nesting_state: A NestingState instance which maintains information about - the current stack of nested blocks being parsed. - error: The function to call with any errors found. - """ - # Find lines missing spaces around &&. - # TODO(unknown): currently we don't check for rvalue references - # with spaces surrounding the && to avoid false positives with - # boolean expressions. - line = clean_lines.elided[linenum] - match = Match(r'^(.*\S)&&', line) - if not match: - match = Match(r'(.*)&&\S', line) - if (not match) or '(&&)' in line or Search(r'\boperator\s*$', match.group(1)): - return - - # Either poorly formed && or an rvalue reference, check the context - # to get a more accurate error message. Mostly we want to determine - # if what's to the left of "&&" is a type or not. - typenames = GetTemplateArgs(clean_lines, linenum) - and_pos = len(match.group(1)) - if IsRValueType(typenames, clean_lines, nesting_state, linenum, and_pos): - if not IsRValueAllowed(clean_lines, linenum, typenames): - error(filename, linenum, 'build/c++11', 3, - 'RValue references are an unapproved C++ feature.') - else: - error(filename, linenum, 'whitespace/operators', 3, - 'Missing spaces around &&') - - def CheckSectionSpacing(filename, clean_lines, class_info, linenum, error): """Checks for additional blank line issues related to sections. @@ -3838,7 +4270,7 @@ def CheckSectionSpacing(filename, clean_lines, class_info, linenum, error): linenum <= class_info.starting_linenum): return - matched = Match(r'\s*(public|protected|private):', clean_lines.lines[linenum]) + matched = re.match(r'\s*(public|protected|private):', clean_lines.lines[linenum]) if matched: # Issue warning if the line before public/protected/private was # not a blank line, but don't do this if the previous line contains @@ -3850,20 +4282,20 @@ def CheckSectionSpacing(filename, clean_lines, class_info, linenum, error): # common when defining classes in C macros. prev_line = clean_lines.lines[linenum - 1] if (not IsBlankLine(prev_line) and - not Search(r'\b(class|struct)\b', prev_line) and - not Search(r'\\$', prev_line)): + not re.search(r'\b(class|struct)\b', prev_line) and + not re.search(r'\\$', prev_line)): # Try a bit harder to find the beginning of the class. This is to # account for multi-line base-specifier lists, e.g.: # class Derived # : public Base { end_class_head = class_info.starting_linenum for i in range(class_info.starting_linenum, linenum): - if Search(r'\{\s*$', clean_lines.lines[i]): + if re.search(r'\{\s*$', clean_lines.lines[i]): end_class_head = i break if end_class_head < linenum - 1: error(filename, linenum, 'whitespace/blank_line', 3, - '"%s:" should be preceded by a blank line' % matched.group(1)) + f'"{matched.group(1)}:" should be preceded by a blank line') def GetPreviousNonBlankLine(clean_lines, linenum): @@ -3901,31 +4333,36 @@ def CheckBraces(filename, clean_lines, linenum, error): line = clean_lines.elided[linenum] # get rid of comments and strings - if Match(r'\s*{\s*$', line): + if re.match(r'\s*{\s*$', line): # We allow an open brace to start a line in the case where someone is using # braces in a block to explicitly create a new scope, which is commonly used # to control the lifetime of stack-allocated variables. Braces are also # used for brace initializers inside function calls. We don't detect this # perfectly: we just don't complain if the last non-whitespace character on # the previous non-blank line is ',', ';', ':', '(', '{', or '}', or if the - # previous line starts a preprocessor block. + # previous line starts a preprocessor block. We also allow a brace on the + # following line if it is part of an array initialization and would not fit + # within the 80 character limit of the preceding line. prevline = GetPreviousNonBlankLine(clean_lines, linenum)[0] - if (not Search(r'[,;:}{(]\s*$', prevline) and - not Match(r'\s*#', prevline)): + if (not re.search(r'[,;:}{(]\s*$', prevline) and + not re.match(r'\s*#', prevline) and + not (GetLineWidth(prevline) > _line_length - 2 and '[]' in prevline)): error(filename, linenum, 'whitespace/braces', 4, '{ should almost always be at the end of the previous line') # An else clause should be on the same line as the preceding closing brace. - if Match(r'\s*else\b\s*(?:if\b|\{|$)', line): + if last_wrong := re.match(r'\s*else\b\s*(?:if\b|\{|$)', line): prevline = GetPreviousNonBlankLine(clean_lines, linenum)[0] - if Match(r'\s*}\s*$', prevline): + if re.match(r'\s*}\s*$', prevline): error(filename, linenum, 'whitespace/newline', 4, 'An else should appear on the same line as the preceding }') + else: + last_wrong = False # If braces come on one side of an else, they should be on both. # However, we have to worry about "else if" that spans multiple lines! - if Search(r'else if\s*\(', line): # could be multi-line if - brace_on_left = bool(Search(r'}\s*else if\s*\(', line)) + if re.search(r'else if\s*\(', line): # could be multi-line if + brace_on_left = bool(re.search(r'}\s*else if\s*\(', line)) # find the ( after the if pos = line.find('else if') pos = line.find('(', pos) @@ -3935,19 +4372,29 @@ def CheckBraces(filename, clean_lines, linenum, error): if brace_on_left != brace_on_right: # must be brace after if error(filename, linenum, 'readability/braces', 5, 'If an else has a brace on one side, it should have it on both') - elif Search(r'}\s*else[^{]*$', line) or Match(r'[^}]*else\s*{', line): + # Prevent detection if statement has { and we detected an improper newline after } + elif re.search(r'}\s*else[^{]*$', line) or (re.match(r'[^}]*else\s*{', line) and not last_wrong): error(filename, linenum, 'readability/braces', 5, 'If an else has a brace on one side, it should have it on both') - # Likewise, an else should never have the else clause on the same line - if Search(r'\belse [^\s{]', line) and not Search(r'\belse if\b', line): - error(filename, linenum, 'whitespace/newline', 4, - 'Else clause should never be on same line as else (use 2 lines)') - - # In the same way, a do/while should never be on one line - if Match(r'\s*do [^\s{]', line): - error(filename, linenum, 'whitespace/newline', 4, - 'do/while clauses should not be on a single line') + # No control clauses with braces should have its contents on the same line + # Exclude } which will be covered by empty-block detect + # Exclude ; which may be used by while in a do-while + if keyword := re.search( + r'\b(else if|if|while|for|switch)' # These have parens + r'\s*\(.*\)\s*(?:\[\[(?:un)?likely\]\]\s*)?{\s*[^\s\\};]', line): + error(filename, linenum, 'whitespace/newline', 5, + f'Controlled statements inside brackets of {keyword.group(1)} clause' + ' should be on a separate line') + elif keyword := re.search( + r'\b(else|do|try)' # These don't have parens + r'\s*(?:\[\[(?:un)?likely\]\]\s*)?{\s*[^\s\\}]', line): + error(filename, linenum, 'whitespace/newline', 5, + f'Controlled statements inside brackets of {keyword.group(1)} clause' + ' should be on a separate line') + + # TODO: Err on if...else and do...while statements without braces; + # style guide has changed since the below comment was written # Check single-line if/else bodies. The style guide says 'curly braces are not # required for single-line statements'. We additionally allow multi-line, @@ -3956,21 +4403,21 @@ def CheckBraces(filename, clean_lines, linenum, error): # its line, and the line after that should have an indent level equal to or # lower than the if. We also check for ambiguous if/else nesting without # braces. - if_else_match = Search(r'\b(if\s*\(|else\b)', line) - if if_else_match and not Match(r'\s*#', line): + if_else_match = re.search(r'\b(if\s*(|constexpr)\s*\(|else\b)', line) + if if_else_match and not re.match(r'\s*#', line): if_indent = GetIndentLevel(line) endline, endlinenum, endpos = line, linenum, if_else_match.end() - if_match = Search(r'\bif\s*\(', line) + if_match = re.search(r'\bif\s*(|constexpr)\s*\(', line) if if_match: # This could be a multiline if condition, so find the end first. pos = if_match.end() - 1 (endline, endlinenum, endpos) = CloseExpression(clean_lines, linenum, pos) # Check for an opening brace, either directly after the if or on the next # line. If found, this isn't a single-statement conditional. - if (not Match(r'\s*{', endline[endpos:]) - and not (Match(r'\s*$', endline[endpos:]) + if (not re.match(r'\s*(?:\[\[(?:un)?likely\]\]\s*)?{', endline[endpos:]) + and not (re.match(r'\s*$', endline[endpos:]) and endlinenum < (len(clean_lines.elided) - 1) - and Match(r'\s*{', clean_lines.elided[endlinenum + 1]))): + and re.match(r'\s*{', clean_lines.elided[endlinenum + 1]))): while (endlinenum < len(clean_lines.elided) and ';' not in clean_lines.elided[endlinenum][endpos:]): endlinenum += 1 @@ -3980,11 +4427,11 @@ def CheckBraces(filename, clean_lines, linenum, error): # We allow a mix of whitespace and closing braces (e.g. for one-liner # methods) and a single \ after the semicolon (for macros) endpos = endline.find(';') - if not Match(r';[\s}]*(\\?)$', endline[endpos:]): + if not re.match(r';[\s}]*(\\?)$', endline[endpos:]): # Semicolon isn't the last character, there's something trailing. # Output a warning if the semicolon is not contained inside # a lambda expression. - if not Match(r'^[^{};]*\[[^\[\]]*\][^{}]*\{[^{}]*\}\s*\)*[;,]\s*$', + if not re.match(r'^[^{};]*\[[^\[\]]*\][^{}]*\{[^{}]*\}\s*\)*[;,]\s*$', endline): error(filename, linenum, 'readability/braces', 4, 'If/else bodies with multiple statements require braces') @@ -3995,7 +4442,7 @@ def CheckBraces(filename, clean_lines, linenum, error): # With ambiguous nested if statements, this will error out on the # if that *doesn't* match the else, regardless of whether it's the # inner one or outer one. - if (if_match and Match(r'\s*else\b', next_line) + if (if_match and re.match(r'\s*else\b', next_line) and next_indent != if_indent): error(filename, linenum, 'readability/braces', 4, 'Else clause should be indented at the same level as if. ' @@ -4019,9 +4466,9 @@ def CheckTrailingSemicolon(filename, clean_lines, linenum, error): # Block bodies should not be followed by a semicolon. Due to C++11 # brace initialization, there are more places where semicolons are - # required than not, so we use a whitelist approach to check these - # rather than a blacklist. These are the places where "};" should - # be replaced by just "}": + # required than not, so we explicitly list the allowed rules rather + # than listing the disallowed ones. These are the places where "};" + # should be replaced by just "}": # 1. Some flavor of block following closing parenthesis: # for (;;) {}; # while (...) {}; @@ -4061,7 +4508,7 @@ def CheckTrailingSemicolon(filename, clean_lines, linenum, error): # to namespaces. For now we do not warn for this case. # # Try matching case 1 first. - match = Match(r'^(.*\)\s*)\{', line) + match = re.match(r'^(.*\)\s*)\{', line) if match: # Matched closing parenthesis (case 1). Check the token before the # matching opening parenthesis, and don't warn if it looks like a @@ -4077,42 +4524,44 @@ def CheckTrailingSemicolon(filename, clean_lines, linenum, error): # - INTERFACE_DEF # - EXCLUSIVE_LOCKS_REQUIRED, SHARED_LOCKS_REQUIRED, LOCKS_EXCLUDED: # - # We implement a whitelist of safe macros instead of a blacklist of + # We implement a list of safe macros instead of a list of # unsafe macros, even though the latter appears less frequently in # google code and would have been easier to implement. This is because - # the downside for getting the whitelist wrong means some extra - # semicolons, while the downside for getting the blacklist wrong + # the downside for getting the allowed checks wrong means some extra + # semicolons, while the downside for getting disallowed checks wrong # would result in compile errors. # # In addition to macros, we also don't want to warn on # - Compound literals # - Lambdas - # - alignas specifier with anonymous structs: + # - alignas specifier with anonymous structs + # - decltype closing_brace_pos = match.group(1).rfind(')') opening_parenthesis = ReverseCloseExpression( clean_lines, linenum, closing_brace_pos) if opening_parenthesis[2] > -1: line_prefix = opening_parenthesis[0][0:opening_parenthesis[2]] - macro = Search(r'\b([A-Z_]+)\s*$', line_prefix) - func = Match(r'^(.*\])\s*$', line_prefix) + macro = re.search(r'\b([A-Z_][A-Z0-9_]*)\s*$', line_prefix) + func = re.match(r'^(.*\])\s*$', line_prefix) if ((macro and macro.group(1) not in ( 'TEST', 'TEST_F', 'MATCHER', 'MATCHER_P', 'TYPED_TEST', 'EXCLUSIVE_LOCKS_REQUIRED', 'SHARED_LOCKS_REQUIRED', 'LOCKS_EXCLUDED', 'INTERFACE_DEF')) or - (func and not Search(r'\boperator\s*\[\s*\]', func.group(1))) or - Search(r'\b(?:struct|union)\s+alignas\s*$', line_prefix) or - Search(r'\s+=\s*$', line_prefix)): + (func and not re.search(r'\boperator\s*\[\s*\]', func.group(1))) or + re.search(r'\b(?:struct|union)\s+alignas\s*$', line_prefix) or + re.search(r'\bdecltype$', line_prefix) or + re.search(r'\s+=\s*$', line_prefix)): match = None if (match and opening_parenthesis[1] > 1 and - Search(r'\]\s*$', clean_lines.elided[opening_parenthesis[1] - 1])): + re.search(r'\]\s*$', clean_lines.elided[opening_parenthesis[1] - 1])): # Multi-line lambda-expression match = None else: # Try matching cases 2-3. - match = Match(r'^(.*(?:else|\)\s*const)\s*)\{', line) + match = re.match(r'^(.*(?:else|\)\s*const)\s*)\{', line) if not match: # Try matching cases 4-6. These are always matched on separate lines. # @@ -4123,14 +4572,14 @@ def CheckTrailingSemicolon(filename, clean_lines, linenum, error): # // blank line # } prevline = GetPreviousNonBlankLine(clean_lines, linenum)[0] - if prevline and Search(r'[;{}]\s*$', prevline): - match = Match(r'^(\s*)\{', line) + if prevline and re.search(r'[;{}]\s*$', prevline): + match = re.match(r'^(\s*)\{', line) # Check matching closing brace if match: (endline, endlinenum, endpos) = CloseExpression( clean_lines, linenum, len(match.group(1))) - if endpos > -1 and Match(r'^\s*;', endline[endpos:]): + if endpos > -1 and re.match(r'^\s*;', endline[endpos:]): # Current {} pair is eligible for semicolon check, and we have found # the redundant semicolon, output warning here. # @@ -4138,6 +4587,14 @@ def CheckTrailingSemicolon(filename, clean_lines, linenum, error): # outputting warnings for the matching closing brace, if there are # nested blocks with trailing semicolons, we will get the error # messages in reversed order. + + # We need to check the line forward for NOLINT + raw_lines = clean_lines.raw_lines + ParseNolintSuppressions(filename, raw_lines[endlinenum-1], endlinenum-1, + error) + ParseNolintSuppressions(filename, raw_lines[endlinenum], endlinenum, + error) + error(filename, endlinenum, 'readability/braces', 4, "You don't need a ; after a }") @@ -4159,16 +4616,16 @@ def CheckEmptyBlockBody(filename, clean_lines, linenum, error): # We also check "if" blocks here, since an empty conditional block # is likely an error. line = clean_lines.elided[linenum] - matched = Match(r'\s*(for|while|if)\s*\(', line) + matched = re.match(r'\s*(for|while|if)\s*\(', line) if matched: - # Find the end of the conditional expression + # Find the end of the conditional expression. (end_line, end_linenum, end_pos) = CloseExpression( clean_lines, linenum, line.find('(')) # Output warning if what follows the condition expression is a semicolon. # No warning for all other cases, including whitespace or newline, since we # have a separate check for semicolons preceded by whitespace. - if end_pos >= 0 and Match(r';', end_line[end_pos:]): + if end_pos >= 0 and re.match(r';', end_line[end_pos:]): if matched.group(1) == 'if': error(filename, end_linenum, 'whitespace/empty_conditional_body', 5, 'Empty conditional bodies should use {}') @@ -4176,6 +4633,75 @@ def CheckEmptyBlockBody(filename, clean_lines, linenum, error): error(filename, end_linenum, 'whitespace/empty_loop_body', 5, 'Empty loop bodies should use {} or continue') + # Check for if statements that have completely empty bodies (no comments) + # and no else clauses. + if end_pos >= 0 and matched.group(1) == 'if': + # Find the position of the opening { for the if statement. + # Return without logging an error if it has no brackets. + opening_linenum = end_linenum + opening_line_fragment = end_line[end_pos:] + # Loop until EOF or find anything that's not whitespace or opening {. + while not re.search(r'^\s*\{', opening_line_fragment): + if re.search(r'^(?!\s*$)', opening_line_fragment): + # Conditional has no brackets. + return + opening_linenum += 1 + if opening_linenum == len(clean_lines.elided): + # Couldn't find conditional's opening { or any code before EOF. + return + opening_line_fragment = clean_lines.elided[opening_linenum] + # Set opening_line (opening_line_fragment may not be entire opening line). + opening_line = clean_lines.elided[opening_linenum] + + # Find the position of the closing }. + opening_pos = opening_line_fragment.find('{') + if opening_linenum == end_linenum: + # We need to make opening_pos relative to the start of the entire line. + opening_pos += end_pos + (closing_line, closing_linenum, closing_pos) = CloseExpression( + clean_lines, opening_linenum, opening_pos) + if closing_pos < 0: + return + + # Now construct the body of the conditional. This consists of the portion + # of the opening line after the {, all lines until the closing line, + # and the portion of the closing line before the }. + if (clean_lines.raw_lines[opening_linenum] != + CleanseComments(clean_lines.raw_lines[opening_linenum])): + # Opening line ends with a comment, so conditional isn't empty. + return + if closing_linenum > opening_linenum: + # Opening line after the {. Ignore comments here since we checked above. + bodylist = list(opening_line[opening_pos+1:]) + # All lines until closing line, excluding closing line, with comments. + bodylist.extend(clean_lines.raw_lines[opening_linenum+1:closing_linenum]) + # Closing line before the }. Won't (and can't) have comments. + bodylist.append(clean_lines.elided[closing_linenum][:closing_pos-1]) + body = '\n'.join(bodylist) + else: + # If statement has brackets and fits on a single line. + body = opening_line[opening_pos+1:closing_pos-1] + + # Check if the body is empty + if not _EMPTY_CONDITIONAL_BODY_PATTERN.search(body): + return + # The body is empty. Now make sure there's not an else clause. + current_linenum = closing_linenum + current_line_fragment = closing_line[closing_pos:] + # Loop until EOF or find anything that's not whitespace or else clause. + while re.search(r'^\s*$|^(?=\s*else)', current_line_fragment): + if re.search(r'^(?=\s*else)', current_line_fragment): + # Found an else clause, so don't log an error. + return + current_linenum += 1 + if current_linenum == len(clean_lines.elided): + break + current_line_fragment = clean_lines.elided[current_linenum] + + # The body is empty and there's no else clause until EOF or other code. + error(filename, end_linenum, 'whitespace/empty_if_body', 4, + ('If statement had no body and no else clause')) + def FindCheckMacro(line): """Find a replaceable CHECK-like macro. @@ -4193,7 +4719,7 @@ def FindCheckMacro(line): # to make sure that we are matching the expected CHECK macro, as # opposed to some other macro that happens to contain the CHECK # substring. - matched = Match(r'^(.*\b' + macro + r'\s*)\(', line) + matched = re.match(r'^(.*\b' + macro + r'\s*)\(', line) if not matched: continue return (macro, len(matched.group(1))) @@ -4225,14 +4751,14 @@ def CheckCheck(filename, clean_lines, linenum, error): # If the check macro is followed by something other than a # semicolon, assume users will log their own custom error messages # and don't suggest any replacements. - if not Match(r'\s*;', last_line[end_pos:]): + if not re.match(r'\s*;', last_line[end_pos:]): return if linenum == end_line: expression = lines[linenum][start_pos + 1:end_pos - 1] else: expression = lines[linenum][start_pos + 1:] - for i in xrange(linenum + 1, end_line): + for i in range(linenum + 1, end_line): expression += lines[i] expression += last_line[0:end_pos - 1] @@ -4243,7 +4769,7 @@ def CheckCheck(filename, clean_lines, linenum, error): rhs = '' operator = None while expression: - matched = Match(r'^\s*(<<|<<=|>>|>>=|->\*|->|&&|\|\||' + matched = re.match(r'^\s*(<<|<<=|>>|>>=|->\*|->|&&|\|\||' r'==|!=|>=|>|<=|<|\()(.*)$', expression) if matched: token = matched.group(1) @@ -4277,9 +4803,9 @@ def CheckCheck(filename, clean_lines, linenum, error): # characters at once if possible. Trivial benchmark shows that this # is more efficient when the operands are longer than a single # character, which is generally the case. - matched = Match(r'^([^-=!<>()&|]+)(.*)$', expression) + matched = re.match(r'^([^-=!<>()&|]+)(.*)$', expression) if not matched: - matched = Match(r'^(\s*\S)(.*)$', expression) + matched = re.match(r'^(\s*\S)(.*)$', expression) if not matched: break lhs += matched.group(1) @@ -4303,7 +4829,7 @@ def CheckCheck(filename, clean_lines, linenum, error): lhs = lhs.strip() rhs = rhs.strip() match_constant = r'^([-+]?(\d+|0[xX][0-9a-fA-F]+)[lLuU]{0,3}|".*"|\'.*\')$' - if Match(match_constant, lhs) or Match(match_constant, rhs): + if re.match(match_constant, lhs) or re.match(match_constant, rhs): # Note: since we know both lhs and rhs, we can provide a more # descriptive error message like: # Consider using CHECK_EQ(x, 42) instead of CHECK(x == 42) @@ -4313,9 +4839,8 @@ def CheckCheck(filename, clean_lines, linenum, error): # We are still keeping the less descriptive message because if lhs # or rhs gets long, the error message might become unreadable. error(filename, linenum, 'readability/check', 2, - 'Consider using %s instead of %s(a %s b)' % ( - _CHECK_REPLACEMENT[check_macro][operator], - check_macro, operator)) + f'Consider using {_CHECK_REPLACEMENT[check_macro][operator]}' + f' instead of {check_macro}(a {operator} b)') def CheckAltTokens(filename, clean_lines, linenum, error): @@ -4330,7 +4855,7 @@ def CheckAltTokens(filename, clean_lines, linenum, error): line = clean_lines.elided[linenum] # Avoid preprocessor lines - if Match(r'^\s*#', line): + if re.match(r'^\s*#', line): return # Last ditch effort to avoid multi-line comments. This will not help @@ -4346,8 +4871,8 @@ def CheckAltTokens(filename, clean_lines, linenum, error): for match in _ALT_TOKEN_REPLACEMENT_PATTERN.finditer(line): error(filename, linenum, 'readability/alt_tokens', 2, - 'Use operator %s instead of %s' % ( - _ALT_TOKEN_REPLACEMENT[match.group(1)], match.group(1))) + f'Use operator {_ALT_TOKEN_REPLACEMENT[match.group(2)]}' + f' instead of {match.group(2)}') def GetLineWidth(line): @@ -4360,12 +4885,22 @@ def GetLineWidth(line): The width of the line in column positions, accounting for Unicode combining characters and wide characters. """ - if isinstance(line, unicode): + if isinstance(line, str): width = 0 for uc in unicodedata.normalize('NFC', line): if unicodedata.east_asian_width(uc) in ('W', 'F'): width += 2 elif not unicodedata.combining(uc): + # Issue 337 + # https://mail.python.org/pipermail/python-list/2012-August/628809.html + if (sys.version_info.major, sys.version_info.minor) <= (3, 2): + # https://github.com/python/cpython/blob/2.7/Include/unicodeobject.h#L81 + is_wide_build = sysconfig.get_config_var("Py_UNICODE_SIZE") >= 4 + # https://github.com/python/cpython/blob/2.7/Objects/unicodeobject.c#L564 + is_low_surrogate = 0xDC00 <= ord(uc) <= 0xDFFF + if not is_wide_build and is_low_surrogate: + width -= 1 + width += 1 return width else: @@ -4395,6 +4930,7 @@ def CheckStyle(filename, clean_lines, linenum, file_extension, nesting_state, # raw strings, raw_lines = clean_lines.lines_without_raw_strings line = raw_lines[linenum] + prev = raw_lines[linenum - 1] if linenum > 0 else '' if line.find('\t') != -1: error(filename, linenum, 'whitespace/tab', 1, @@ -4412,32 +4948,37 @@ def CheckStyle(filename, clean_lines, linenum, file_extension, nesting_state, # if(match($0, " <<")) complain = 0; # if(match(prev, " +for \\(")) complain = 0; # if(prevodd && match(prevprev, " +for \\(")) complain = 0; - scope_or_label_pattern = r'\s*\w+\s*:\s*\\?$' + scope_or_label_pattern = r'\s*(?:public|private|protected|signals)(?:\s+(?:slots\s*)?)?:\s*\\?$' classinfo = nesting_state.InnermostClass() initial_spaces = 0 cleansed_line = clean_lines.elided[linenum] while initial_spaces < len(line) and line[initial_spaces] == ' ': initial_spaces += 1 - if line and line[-1].isspace(): - error(filename, linenum, 'whitespace/end_of_line', 4, - 'Line ends in whitespace. Consider deleting these extra spaces.') # There are certain situations we allow one space, notably for # section labels, and also lines containing multi-line raw strings. - elif ((initial_spaces == 1 or initial_spaces == 3) and - not Match(scope_or_label_pattern, cleansed_line) and - not (clean_lines.raw_lines[linenum] != line and - Match(r'^\s*""', line))): + # We also don't check for lines that look like continuation lines + # (of lines ending in double quotes, commas, equals, or angle brackets) + # because the rules for how to indent those are non-trivial. + if (not re.search(r'[",=><] *$', prev) and + (initial_spaces == 1 or initial_spaces == 3) and + not re.match(scope_or_label_pattern, cleansed_line) and + not (clean_lines.raw_lines[linenum] != line and + re.match(r'^\s*""', line))): error(filename, linenum, 'whitespace/indent', 3, 'Weird number of spaces at line-start. ' 'Are you using a 2-space indent?') + if line and line[-1].isspace(): + error(filename, linenum, 'whitespace/end_of_line', 4, + 'Line ends in whitespace. Consider deleting these extra spaces.') + # Check if the line is a header guard. is_header_guard = False - if file_extension == 'h': + if IsHeaderExtension(file_extension): cppvar = GetHeaderGuardCPPVariable(filename) - if (line.startswith('#ifndef %s' % cppvar) or - line.startswith('#define %s' % cppvar) or - line.startswith('#endif // %s' % cppvar)): + if (line.startswith(f'#ifndef {cppvar}') or + line.startswith(f'#define {cppvar}') or + line.startswith(f'#endif // {cppvar}')): is_header_guard = True # #include lines and header guards can be long, since there's no clean way to # split them. @@ -4447,20 +4988,23 @@ def CheckStyle(filename, clean_lines, linenum, file_extension, nesting_state, # # The "$Id:...$" comment may also get very long without it being the # developers fault. + # + # Doxygen documentation copying can get pretty long when using an overloaded + # function declaration if (not line.startswith('#include') and not is_header_guard and - not Match(r'^\s*//.*http(s?)://\S*$', line) and - not Match(r'^// \$Id:.*#[0-9]+ \$$', line)): + not re.match(r'^\s*//.*http(s?)://\S*$', line) and + not re.match(r'^\s*//\s*[^\s]*$', line) and + not re.match(r'^// \$Id:.*#[0-9]+ \$$', line) and + not re.match(r'^\s*/// [@\\](copydoc|copydetails|copybrief) .*$', line)): line_width = GetLineWidth(line) - extended_length = int((_line_length * 1.25)) - if line_width > extended_length: - error(filename, linenum, 'whitespace/line_length', 4, - 'Lines should very rarely be longer than %i characters' % - extended_length) - elif line_width > _line_length: + if line_width > _line_length: error(filename, linenum, 'whitespace/line_length', 2, - 'Lines should be <= %i characters long' % _line_length) + f'Lines should be <= {_line_length} characters long') if (cleansed_line.count(';') > 1 and + # allow simple single line lambdas + not re.match(r'^[^{};]*\[[^\[\]]*\][^{}]*\{[^{}\n\r]*\}', + line) and # for loops are allowed two ;'s (and may run over two lines). cleansed_line.find('for') == -1 and (GetPreviousNonBlankLine(clean_lines, linenum)[0].find('for') == -1 or @@ -4476,14 +5020,12 @@ def CheckStyle(filename, clean_lines, linenum, file_extension, nesting_state, CheckBraces(filename, clean_lines, linenum, error) CheckTrailingSemicolon(filename, clean_lines, linenum, error) CheckEmptyBlockBody(filename, clean_lines, linenum, error) - CheckAccess(filename, clean_lines, linenum, nesting_state, error) CheckSpacing(filename, clean_lines, linenum, nesting_state, error) CheckOperatorSpacing(filename, clean_lines, linenum, error) CheckParenthesisSpacing(filename, clean_lines, linenum, error) CheckCommaSpacing(filename, clean_lines, linenum, error) - CheckBracesSpacing(filename, clean_lines, linenum, error) + CheckBracesSpacing(filename, clean_lines, linenum, nesting_state, error) CheckSpacingForFunctionCall(filename, clean_lines, linenum, error) - CheckRValueReference(filename, clean_lines, linenum, nesting_state, error) CheckCheck(filename, clean_lines, linenum, error) CheckAltTokens(filename, clean_lines, linenum, error) classinfo = nesting_state.InnermostClass() @@ -4519,38 +5061,25 @@ def _DropCommonSuffixes(filename): Returns: The filename with the common suffix removed. """ - for suffix in ('test.cc', 'regtest.cc', 'unittest.cc', - 'inl.h', 'impl.h', 'internal.h'): + for suffix in itertools.chain( + (f"{test_suffix.lstrip('_')}.{ext}" + for test_suffix, ext in itertools.product(_test_suffixes, GetNonHeaderExtensions())), + (f'{suffix}.{ext}' + for suffix, ext in itertools.product(['inl', 'imp', 'internal'], GetHeaderExtensions()))): if (filename.endswith(suffix) and len(filename) > len(suffix) and filename[-len(suffix) - 1] in ('-', '_')): return filename[:-len(suffix) - 1] return os.path.splitext(filename)[0] -def _IsTestFilename(filename): - """Determines if the given filename has a suffix that identifies it as a test. - - Args: - filename: The input filename. - - Returns: - True if 'filename' looks like a test, False otherwise. - """ - if (filename.endswith('_test.cc') or - filename.endswith('_unittest.cc') or - filename.endswith('_regtest.cc')): - return True - else: - return False - - -def _ClassifyInclude(fileinfo, include, is_system): +def _ClassifyInclude(fileinfo, include, used_angle_brackets, include_order="default"): """Figures out what kind of header 'include' is. Args: fileinfo: The current file cpplint is running over. A FileInfo instance. include: The path to a #included file. - is_system: True if the #include used <> rather than "". + used_angle_brackets: True if the #include used <> rather than "". + include_order: "default" or other value allowed in program arguments Returns: One of the _XXX_HEADER constants. @@ -4560,6 +5089,8 @@ def _ClassifyInclude(fileinfo, include, is_system): _C_SYS_HEADER >>> _ClassifyInclude(FileInfo('foo/foo.cc'), 'string', True) _CPP_SYS_HEADER + >>> _ClassifyInclude(FileInfo('foo/foo.cc'), 'foo/foo.h', True, "standardcfirst") + _OTHER_SYS_HEADER >>> _ClassifyInclude(FileInfo('foo/foo.cc'), 'foo/foo.h', False) _LIKELY_MY_HEADER >>> _ClassifyInclude(FileInfo('foo/foo_unknown_extension.cc'), @@ -4570,13 +5101,24 @@ def _ClassifyInclude(fileinfo, include, is_system): """ # This is a list of all standard c++ header files, except # those already checked for above. - is_cpp_h = include in _CPP_HEADERS + is_cpp_header = include in _CPP_HEADERS + + # Mark include as C header if in list or in a known folder for standard-ish C headers. + is_std_c_header = (include_order == "default") or (include in _C_HEADERS + # additional linux glibc header folders + or re.search(rf'(?:{"|".join(C_STANDARD_HEADER_FOLDERS)})\/.*\.h', include)) + + # Headers with C++ extensions shouldn't be considered C system headers + include_ext = os.path.splitext(include)[1] + is_system = used_angle_brackets and include_ext not in ['.hh', '.hpp', '.hxx', '.h++'] if is_system: - if is_cpp_h: + if is_cpp_header: return _CPP_SYS_HEADER - else: + if is_std_c_header: return _C_SYS_HEADER + else: + return _OTHER_SYS_HEADER # If the target file and the include we're checking share a # basename when we drop common extensions, and the include @@ -4584,9 +5126,11 @@ def _ClassifyInclude(fileinfo, include, is_system): target_dir, target_base = ( os.path.split(_DropCommonSuffixes(fileinfo.RepositoryName()))) include_dir, include_base = os.path.split(_DropCommonSuffixes(include)) + target_dir_pub = os.path.normpath(target_dir + '/../public') + target_dir_pub = target_dir_pub.replace('\\', '/') if target_base == include_base and ( include_dir == target_dir or - include_dir == os.path.normpath(target_dir + '/../public')): + include_dir == target_dir_pub): return _LIKELY_MY_HEADER # If the target and include share some initial basename @@ -4628,10 +5172,12 @@ def CheckIncludeLine(filename, clean_lines, linenum, include_state, error): # # We also make an exception for Lua headers, which follow google # naming convention but not the include convention. - match = Match(r'#include\s*"([^/]+\.h)"', line) - if match and not _THIRD_PARTY_HEADERS_PATTERN.match(match.group(1)): - error(filename, linenum, 'build/include', 4, - 'Include the directory when naming .h files') + match = re.match(r'#include\s*"([^/]+\.(.*))"', line) + if match: + if (IsHeaderExtension(match.group(2)) and + not _THIRD_PARTY_HEADERS_PATTERN.match(match.group(1))): + error(filename, linenum, 'build/include_subdir', 4, + 'Include the directory when naming header files') # we shouldn't include a file more than once. actually, there are a # handful of instances where doing so is okay, but in general it's @@ -4639,17 +5185,33 @@ def CheckIncludeLine(filename, clean_lines, linenum, include_state, error): match = _RE_PATTERN_INCLUDE.search(line) if match: include = match.group(2) - is_system = (match.group(1) == '<') + used_angle_brackets = match.group(1) == '<' duplicate_line = include_state.FindHeader(include) if duplicate_line >= 0: error(filename, linenum, 'build/include', 4, - '"%s" already included at %s:%s' % - (include, filename, duplicate_line)) - elif (include.endswith('.cc') and + f'"{include}" already included at {filename}:{duplicate_line}') + return + + for extension in GetNonHeaderExtensions(): + if (include.endswith('.' + extension) and os.path.dirname(fileinfo.RepositoryName()) != os.path.dirname(include)): - error(filename, linenum, 'build/include', 4, - 'Do not include .cc files from other packages') - elif not _THIRD_PARTY_HEADERS_PATTERN.match(include): + error(filename, linenum, 'build/include', 4, + 'Do not include .' + extension + ' files from other packages') + return + + # We DO want to include a 3rd party looking header if it matches the + # filename. Otherwise we get an erroneous error "...should include its + # header" error later. + third_src_header = False + for ext in GetHeaderExtensions(): + basefilename = filename[0:len(filename) - len(fileinfo.Extension())] + headerfile = basefilename + '.' + ext + headername = FileInfo(headerfile).RepositoryName() + if headername in include or include in headername: + third_src_header = True + break + + if third_src_header or not _THIRD_PARTY_HEADERS_PATTERN.match(include): include_state.include_list[-1].append((include, linenum)) # We want to ensure that headers appear in the right order: @@ -4664,16 +5226,16 @@ def CheckIncludeLine(filename, clean_lines, linenum, include_state, error): # track of the highest type seen, and complains if we see a # lower type after that. error_message = include_state.CheckNextIncludeOrder( - _ClassifyInclude(fileinfo, include, is_system)) + _ClassifyInclude(fileinfo, include, used_angle_brackets, _include_order)) if error_message: error(filename, linenum, 'build/include_order', 4, - '%s. Should be: %s.h, c system, c++ system, other.' % - (error_message, fileinfo.BaseName())) + f'{error_message}. Should be: {fileinfo.BaseName()}.h, c system,' + ' c++ system, other.') canonical_include = include_state.CanonicalizeAlphabeticalOrder(include) if not include_state.IsInAlphabeticalOrder( clean_lines, linenum, canonical_include): error(filename, linenum, 'build/include_alpha', 4, - 'Include "%s" not in alphabetical order' % include) + f'Include "{include}" not in alphabetical order') include_state.SetLastHeader(canonical_include) @@ -4703,7 +5265,7 @@ def _GetTextInside(text, start_pattern): # Give opening punctuations to get the matching close-punctuations. matching_punctuation = {'(': ')', '{': '}', '[': ']'} - closing_punctuation = set(matching_punctuation.itervalues()) + closing_punctuation = set(dict.values(matching_punctuation)) # Find the position to start extracting text. match = re.search(start_pattern, text, re.M) @@ -4758,6 +5320,9 @@ def _GetTextInside(text, start_pattern): _RE_PATTERN_CONST_REF_PARAM = ( r'(?:.*\s*\bconst\s*&\s*' + _RE_PATTERN_IDENT + r'|const\s+' + _RE_PATTERN_TYPE + r'\s*&\s*' + _RE_PATTERN_IDENT + r')') +# Stream types. +_RE_PATTERN_REF_STREAM_PARAM = ( + r'(?:.*stream\s*&\s*' + _RE_PATTERN_IDENT + r')') def CheckLanguage(filename, clean_lines, linenum, file_extension, @@ -4790,19 +5355,17 @@ def CheckLanguage(filename, clean_lines, linenum, file_extension, # Reset include state across preprocessor directives. This is meant # to silence warnings for conditional includes. - match = Match(r'^\s*#\s*(if|ifdef|ifndef|elif|else|endif)\b', line) + match = re.match(r'^\s*#\s*(if|ifdef|ifndef|elif|else|endif)\b', line) if match: include_state.ResetSection(match.group(1)) - # Make Windows paths like Unix. - fullname = os.path.abspath(filename).replace('\\', '/') - + # Perform other checks now that we are sure that this is not an include line CheckCasts(filename, clean_lines, linenum, error) CheckGlobalStatic(filename, clean_lines, linenum, error) CheckPrintf(filename, clean_lines, linenum, error) - if file_extension == 'h': + if IsHeaderExtension(file_extension): # TODO(unknown): check that 1-arg constructors are explicit. # How to tell it's a constructor? # (handled in CheckForNonStandardConstructs for now) @@ -4812,15 +5375,15 @@ def CheckLanguage(filename, clean_lines, linenum, file_extension, # Check if people are using the verboten C basic types. The only exception # we regularly allow is "unsigned short port" for port. - if Search(r'\bshort port\b', line): - if not Search(r'\bunsigned short port\b', line): + if re.search(r'\bshort port\b', line): + if not re.search(r'\bunsigned short port\b', line): error(filename, linenum, 'runtime/int', 4, 'Use "unsigned short" for ports, not "short"') else: - match = Search(r'\b(short|long(?! +double)|long long)\b', line) + match = re.search(r'\b(short|long(?! +double)|long long)\b', line) if match: error(filename, linenum, 'runtime/int', 4, - 'Use int16/int64/etc, rather than the C type %s' % match.group(1)) + f'Use int16/int64/etc, rather than the C type {match.group(1)}') # Check if some verboten operator overloading is going on # TODO(unknown): catch out-of-line unary operator&: @@ -4828,13 +5391,13 @@ def CheckLanguage(filename, clean_lines, linenum, file_extension, # int operator&(const X& x) { return 42; } // unary operator& # The trick is it's hard to tell apart from binary operator&: # class Y { int operator&(const Y& x) { return 23; } }; // binary operator& - if Search(r'\boperator\s*&\s*\(\s*\)', line): + if re.search(r'\boperator\s*&\s*\(\s*\)', line): error(filename, linenum, 'runtime/operator', 4, 'Unary operator& is dangerous. Do not use it.') # Check for suspicious usage of "if" like # } if (a == b) { - if Search(r'\}\s*if\s*\(', line): + if re.search(r'\}\s*if\s*\(', line): error(filename, linenum, 'readability/braces', 4, 'Did you mean "else if"? If not, start a new line for "if".') @@ -4847,28 +5410,32 @@ def CheckLanguage(filename, clean_lines, linenum, file_extension, # boy_this_is_a_really_long_variable_that_cannot_fit_on_the_prev_line); printf_args = _GetTextInside(line, r'(?i)\b(string)?printf\s*\(') if printf_args: - match = Match(r'([\w.\->()]+)$', printf_args) + match = re.match(r'([\w.\->()]+)$', printf_args) if match and match.group(1) != '__VA_ARGS__': function_name = re.search(r'\b((?:string)?printf)\s*\(', line, re.I).group(1) error(filename, linenum, 'runtime/printf', 4, - 'Potential format string bug. Do %s("%%s", %s) instead.' - % (function_name, match.group(1))) + 'Potential format string bug. Do' + f' {function_name}("%s", {match.group(1)}) instead.') # Check for potential memset bugs like memset(buf, sizeof(buf), 0). - match = Search(r'memset\s*\(([^,]*),\s*([^,]*),\s*0\s*\)', line) - if match and not Match(r"^''|-?[0-9]+|0x[0-9A-Fa-f]$", match.group(2)): + match = re.search(r'memset\s*\(([^,]*),\s*([^,]*),\s*0\s*\)', line) + if match and not re.match(r"^''|-?[0-9]+|0x[0-9A-Fa-f]$", match.group(2)): error(filename, linenum, 'runtime/memset', 4, - 'Did you mean "memset(%s, 0, %s)"?' - % (match.group(1), match.group(2))) + f'Did you mean "memset({match.group(1)}, 0, {match.group(2)})"?') - if Search(r'\busing namespace\b', line): - error(filename, linenum, 'build/namespaces', 5, - 'Do not use namespace using-directives. ' - 'Use using-declarations instead.') + if re.search(r'\busing namespace\b', line): + if re.search(r'\bliterals\b', line): + error(filename, linenum, 'build/namespaces_literals', 5, + 'Do not use namespace using-directives. ' + 'Use using-declarations instead.') + else: + error(filename, linenum, 'build/namespaces', 5, + 'Do not use namespace using-directives. ' + 'Use using-declarations instead.') # Detect variable-length arrays. - match = Match(r'\s*(.+::)?(\w+) [a-z]\w*\[(.+)];', line) + match = re.match(r'\s*(.+::)?(\w+) [a-z]\w*\[(.+)];', line) if (match and match.group(2) != 'return' and match.group(2) != 'delete' and match.group(3).find(']') == -1): # Split the size using space and arithmetic operators as delimiters. @@ -4882,17 +5449,17 @@ def CheckLanguage(filename, clean_lines, linenum, file_extension, skip_next = False continue - if Search(r'sizeof\(.+\)', tok): continue - if Search(r'arraysize\(\w+\)', tok): continue + if re.search(r'sizeof\(.+\)', tok): continue + if re.search(r'arraysize\(\w+\)', tok): continue tok = tok.lstrip('(') tok = tok.rstrip(')') if not tok: continue - if Match(r'\d+', tok): continue - if Match(r'0[xX][0-9a-fA-F]+', tok): continue - if Match(r'k[A-Z0-9]\w*', tok): continue - if Match(r'(.+::)?k[A-Z0-9]\w*', tok): continue - if Match(r'(.+::)?[A-Z][A-Z0-9_]*', tok): continue + if re.match(r'\d+', tok): continue + if re.match(r'0[xX][0-9a-fA-F]+', tok): continue + if re.match(r'k[A-Z0-9]\w*', tok): continue + if re.match(r'(.+::)?k[A-Z0-9]\w*', tok): continue + if re.match(r'(.+::)?[A-Z][A-Z0-9_]*', tok): continue # A catch all for tricky sizeof cases, including 'sizeof expression', # 'sizeof(*type)', 'sizeof(const type)', 'sizeof(struct StructName)' # requires skipping the next token because we split on ' ' and '*'. @@ -4909,12 +5476,12 @@ def CheckLanguage(filename, clean_lines, linenum, file_extension, # Check for use of unnamed namespaces in header files. Registration # macros are typically OK, so we allow use of "namespace {" on lines # that end with backslashes. - if (file_extension == 'h' - and Search(r'\bnamespace\s*{', line) + if (IsHeaderExtension(file_extension) + and re.search(r'\bnamespace\s*{', line) and line[-1] != '\\'): - error(filename, linenum, 'build/namespaces', 4, + error(filename, linenum, 'build/namespaces_headers', 4, 'Do not use unnamed namespaces in header files. See ' - 'http://google-styleguide.googlecode.com/svn/trunk/cppguide.xml#Namespaces' + 'https://google-styleguide.googlecode.com/svn/trunk/cppguide.xml#Namespaces' ' for more information.') @@ -4930,14 +5497,18 @@ def CheckGlobalStatic(filename, clean_lines, linenum, error): line = clean_lines.elided[linenum] # Match two lines at a time to support multiline declarations - if linenum + 1 < clean_lines.NumLines() and not Search(r'[;({]', line): + if linenum + 1 < clean_lines.NumLines() and not re.search(r'[;({]', line): line += clean_lines.elided[linenum + 1].strip() # Check for people declaring static/global STL strings at the top level. # This is dangerous because the C++ language does not guarantee that - # globals with constructors are initialized before the first access. - match = Match( - r'((?:|static +)(?:|const +))string +([a-zA-Z0-9_:]+)\b(.*)', + # globals with constructors are initialized before the first access, and + # also because globals can be destroyed when some threads are still running. + # TODO(unknown): Generalize this to also find static unique_ptr instances. + # TODO(unknown): File bugs for clang-tidy to find these. + match = re.match( + r'((?:|static +)(?:|const +))(?::*std::)?string( +const)? +' + r'([a-zA-Z0-9_:]+)\b(.*)', line) # Remove false positives: @@ -4957,15 +5528,19 @@ def CheckGlobalStatic(filename, clean_lines, linenum, error): # matching identifiers. # string Class::operator*() if (match and - not Search(r'\bstring\b(\s+const)?\s*\*\s*(const\s+)?\w', line) and - not Search(r'\boperator\W', line) and - not Match(r'\s*(<.*>)?(::[a-zA-Z0-9_]+)*\s*\(([^"]|$)', match.group(3))): - error(filename, linenum, 'runtime/string', 4, - 'For a static/global string constant, use a C style string instead: ' - '"%schar %s[]".' % - (match.group(1), match.group(2))) - - if Search(r'\b([A-Za-z0-9_]*_)\(\1\)', line): + not re.search(r'\bstring\b(\s+const)?\s*[\*\&]\s*(const\s+)?\w', line) and + not re.search(r'\boperator\W', line) and + not re.match(r'\s*(<.*>)?(::[a-zA-Z0-9_]+)*\s*\(([^"]|$)', match.group(4))): + if re.search(r'\bconst\b', line): + error(filename, linenum, 'runtime/string', 4, + 'For a static/global string constant, use a C style string instead:' + f' "{match.group(1)}char{match.group(2) or ""} {match.group(3)}[]".') + else: + error(filename, linenum, 'runtime/string', 4, + 'Static/global string variables are not permitted.') + + if (re.search(r'\b([A-Za-z0-9_]*_)\(\1\)', line) or + re.search(r'\b([A-Za-z0-9_]*_)\(CHECK_NOTNULL\(\1\)\)', line)): error(filename, linenum, 'runtime/init', 4, 'You seem to be initializing a member variable with itself.') @@ -4982,21 +5557,21 @@ def CheckPrintf(filename, clean_lines, linenum, error): line = clean_lines.elided[linenum] # When snprintf is used, the second argument shouldn't be a literal. - match = Search(r'snprintf\s*\(([^,]*),\s*([0-9]*)\s*,', line) + match = re.search(r'snprintf\s*\(([^,]*),\s*([0-9]*)\s*,', line) if match and match.group(2) != '0': # If 2nd arg is zero, snprintf is used to calculate size. - error(filename, linenum, 'runtime/printf', 3, - 'If you can, use sizeof(%s) instead of %s as the 2nd arg ' - 'to snprintf.' % (match.group(1), match.group(2))) + error(filename, linenum, 'runtime/printf', 3, 'If you can, use' + f' sizeof({match.group(1)}) instead of {match.group(2)}' + ' as the 2nd arg to snprintf.') # Check if some verboten C functions are being used. - if Search(r'\bsprintf\s*\(', line): + if re.search(r'\bsprintf\s*\(', line): error(filename, linenum, 'runtime/printf', 5, 'Never use sprintf. Use snprintf instead.') - match = Search(r'\b(strcpy|strcat)\s*\(', line) + match = re.search(r'\b(strcpy|strcat)\s*\(', line) if match: error(filename, linenum, 'runtime/printf', 4, - 'Almost always, snprintf is better than %s' % match.group(1)) + f'Almost always, snprintf is better than {match.group(1)}') def IsDerivedFunction(clean_lines, linenum): @@ -5010,14 +5585,14 @@ def IsDerivedFunction(clean_lines, linenum): virt-specifier. """ # Scan back a few lines for start of current function - for i in xrange(linenum, max(-1, linenum - 10), -1): - match = Match(r'^([^()]*\w+)\(', clean_lines.elided[i]) + for i in range(linenum, max(-1, linenum - 10), -1): + match = re.match(r'^([^()]*\w+)\(', clean_lines.elided[i]) if match: # Look for "override" after the matching closing parenthesis line, _, closing_paren = CloseExpression( clean_lines, i, len(match.group(1))) return (closing_paren >= 0 and - Search(r'\boverride\b', line[closing_paren:])) + re.search(r'\boverride\b', line[closing_paren:])) return False @@ -5031,9 +5606,9 @@ def IsOutOfLineMethodDefinition(clean_lines, linenum): True if current line contains an out-of-line method definition. """ # Scan back a few lines for start of current function - for i in xrange(linenum, max(-1, linenum - 10), -1): - if Match(r'^([^()]*\w+)\(', clean_lines.elided[i]): - return Match(r'^[^()]*\w+::\w+\(', clean_lines.elided[i]) is not None + for i in range(linenum, max(-1, linenum - 10), -1): + if re.match(r'^([^()]*\w+)\(', clean_lines.elided[i]): + return re.match(r'^[^()]*\w+::\w+\(', clean_lines.elided[i]) is not None return False @@ -5047,24 +5622,24 @@ def IsInitializerList(clean_lines, linenum): True if current line appears to be inside constructor initializer list, False otherwise. """ - for i in xrange(linenum, 1, -1): + for i in range(linenum, 1, -1): line = clean_lines.elided[i] if i == linenum: - remove_function_body = Match(r'^(.*)\{\s*$', line) + remove_function_body = re.match(r'^(.*)\{\s*$', line) if remove_function_body: line = remove_function_body.group(1) - if Search(r'\s:\s*\w+[({]', line): + if re.search(r'\s:\s*\w+[({]', line): # A lone colon tend to indicate the start of a constructor # initializer list. It could also be a ternary operator, which # also tend to appear in constructor initializer lists as # opposed to parameter lists. return True - if Search(r'\}\s*,\s*$', line): + if re.search(r'\}\s*,\s*$', line): # A closing brace followed by a comma is probably the end of a # brace-initialized member in constructor initializer list. return True - if Search(r'[{};]\s*$', line): + if re.search(r'[{};]\s*$', line): # Found one of the following: # - A closing brace or semicolon, probably the end of the previous # function. @@ -5128,13 +5703,13 @@ def CheckForNonConstReference(filename, clean_lines, linenum, # that spans more than 2 lines, please use a typedef. if linenum > 1: previous = None - if Match(r'\s*::(?:[\w<>]|::)+\s*&\s*\S', line): + if re.match(r'\s*::(?:[\w<>]|::)+\s*&\s*\S', line): # previous_line\n + ::current_line - previous = Search(r'\b((?:const\s*)?(?:[\w<>]|::)+[\w<>])\s*$', + previous = re.search(r'\b((?:const\s*)?(?:[\w<>]|::)+[\w<>])\s*$', clean_lines.elided[linenum - 1]) - elif Match(r'\s*[a-zA-Z_]([\w<>]|::)+\s*&\s*\S', line): + elif re.match(r'\s*[a-zA-Z_]([\w<>]|::)+\s*&\s*\S', line): # previous_line::\n + current_line - previous = Search(r'\b((?:const\s*)?(?:[\w<>]|::)+::)\s*$', + previous = re.search(r'\b((?:const\s*)?(?:[\w<>]|::)+::)\s*$', clean_lines.elided[linenum - 1]) if previous: line = previous.group(1) + line.lstrip() @@ -5148,7 +5723,7 @@ def CheckForNonConstReference(filename, clean_lines, linenum, # Found the matching < on an earlier line, collect all # pieces up to current line. line = '' - for i in xrange(startline, linenum + 1): + for i in range(startline, linenum + 1): line += clean_lines.elided[i].strip() # Check for non-const references in function parameters. A single '&' may @@ -5172,15 +5747,15 @@ def CheckForNonConstReference(filename, clean_lines, linenum, # appear inside the second set of parentheses on the current line as # opposed to the first set. if linenum > 0: - for i in xrange(linenum - 1, max(0, linenum - 10), -1): + for i in range(linenum - 1, max(0, linenum - 10), -1): previous_line = clean_lines.elided[i] - if not Search(r'[),]\s*$', previous_line): + if not re.search(r'[),]\s*$', previous_line): break - if Match(r'^\s*:\s+\S', previous_line): + if re.match(r'^\s*:\s+\S', previous_line): return # Avoid preprocessors - if Search(r'\\\s*$', line): + if re.search(r'\\\s*$', line): return # Avoid constructor initializer lists @@ -5193,28 +5768,29 @@ def CheckForNonConstReference(filename, clean_lines, linenum, # # We also accept & in static_assert, which looks like a function but # it's actually a declaration expression. - whitelisted_functions = (r'(?:[sS]wap(?:<\w:+>)?|' + allowed_functions = (r'(?:[sS]wap(?:<\w:+>)?|' r'operator\s*[<>][<>]|' r'static_assert|COMPILE_ASSERT' r')\s*\(') - if Search(whitelisted_functions, line): + if re.search(allowed_functions, line): return - elif not Search(r'\S+\([^)]*$', line): - # Don't see a whitelisted function on this line. Actually we + elif not re.search(r'\S+\([^)]*$', line): + # Don't see an allowed function on this line. Actually we # didn't see any function name on this line, so this is likely a # multi-line parameter list. Try a bit harder to catch this case. - for i in xrange(2): + for i in range(2): if (linenum > i and - Search(whitelisted_functions, clean_lines.elided[linenum - i - 1])): + re.search(allowed_functions, clean_lines.elided[linenum - i - 1])): return - decls = ReplaceAll(r'{[^}]*}', ' ', line) # exclude function body + decls = re.sub(r'{[^}]*}', ' ', line) # exclude function body for parameter in re.findall(_RE_PATTERN_REF_PARAM, decls): - if not Match(_RE_PATTERN_CONST_REF_PARAM, parameter): + if (not re.match(_RE_PATTERN_CONST_REF_PARAM, parameter) and + not re.match(_RE_PATTERN_REF_STREAM_PARAM, parameter)): error(filename, linenum, 'runtime/references', 2, 'Is this a non-const reference? ' 'If so, make const or use a pointer: ' + - ReplaceAll(' *<', '<', parameter)) + re.sub(' *<', '<', parameter)) def CheckCasts(filename, clean_lines, linenum, error): @@ -5232,8 +5808,8 @@ def CheckCasts(filename, clean_lines, linenum, error): # I just try to capture the most common basic types, though there are more. # Parameterless conversion functions, such as bool(), are allowed as they are # probably a member operator declaration or default constructor. - match = Search( - r'(\bnew\s+|\S<\s*(?:const\s+)?)?\b' + match = re.search( + r'(\bnew\s+(?:const\s+)?|\S<\s*(?:const\s+)?)?\b' r'(int|float|double|bool|char|int32|uint32|int64|uint64)' r'(\([^)].*)', line) expecting_function = ExpectingFunctionArgs(clean_lines, linenum) @@ -5256,7 +5832,7 @@ def CheckCasts(filename, clean_lines, linenum, error): # Avoid arrays by looking for brackets that come after the closing # parenthesis. - if Match(r'\([^()]+\)\s*\[', match.group(3)): + if re.match(r'\([^()]+\)\s*\[', match.group(3)): return # Other things to ignore: @@ -5267,19 +5843,18 @@ def CheckCasts(filename, clean_lines, linenum, error): matched_funcptr = match.group(3) if (matched_new_or_template is None and not (matched_funcptr and - (Match(r'\((?:[^() ]+::\s*\*\s*)?[^() ]+\)\s*\(', + (re.match(r'\((?:[^() ]+::\s*\*\s*)?[^() ]+\)\s*\(', matched_funcptr) or matched_funcptr.startswith('(*)'))) and - not Match(r'\s*using\s+\S+\s*=\s*' + matched_type, line) and - not Search(r'new\(\S+\)\s*' + matched_type, line)): + not re.match(r'\s*using\s+\S+\s*=\s*' + matched_type, line) and + not re.search(r'new\(\S+\)\s*' + matched_type, line)): error(filename, linenum, 'readability/casting', 4, 'Using deprecated casting style. ' - 'Use static_cast<%s>(...) instead' % - matched_type) + f'Use static_cast<{matched_type}>(...) instead') if not expecting_function: CheckCStyleCast(filename, clean_lines, linenum, 'static_cast', - r'\((int|float|double|bool|char|u?int(16|32|64))\)', error) + r'\((int|float|double|bool|char|u?int(16|32|64)|size_t)\)', error) # This doesn't catch all cases. Consider (const char * const)"hello". # @@ -5304,7 +5879,7 @@ def CheckCasts(filename, clean_lines, linenum, error): # # This is not a cast: # reference_type&(int* function_param); - match = Search( + match = re.search( r'(?:[^\w]&\(([^)*][^)]*)\)[\w(])|' r'(?:[^\w]&(static|dynamic|down|reinterpret)_cast\b)', line) if match: @@ -5312,7 +5887,7 @@ def CheckCasts(filename, clean_lines, linenum, error): # dereferenced by the casted pointer, as opposed to the casted # pointer itself. parenthesis_error = False - match = Match(r'^(.*&(?:static|dynamic|down|reinterpret)_cast\b)<', line) + match = re.match(r'^(.*&(?:static|dynamic|down|reinterpret)_cast\b)<', line) if match: _, y1, x1 = CloseExpression(clean_lines, linenum, len(match.group(1))) if x1 >= 0 and clean_lines.elided[y1][x1] == '(': @@ -5321,7 +5896,7 @@ def CheckCasts(filename, clean_lines, linenum, error): extended_line = clean_lines.elided[y2][x2:] if y2 < clean_lines.NumLines() - 1: extended_line += clean_lines.elided[y2 + 1] - if Match(r'\s*(?:->|\[)', extended_line): + if re.match(r'\s*(?:->|\[)', extended_line): parenthesis_error = True if parenthesis_error: @@ -5353,89 +5928,38 @@ def CheckCStyleCast(filename, clean_lines, linenum, cast_type, pattern, error): False otherwise. """ line = clean_lines.elided[linenum] - match = Search(pattern, line) + match = re.search(pattern, line) if not match: return False # Exclude lines with keywords that tend to look like casts context = line[0:match.start(1) - 1] - if Match(r'.*\b(?:sizeof|alignof|alignas|[_A-Z][_A-Z0-9]*)\s*$', context): + if re.match(r'.*\b(?:sizeof|alignof|alignas|[_A-Z][_A-Z0-9]*)\s*$', context): return False # Try expanding current context to see if we one level of # parentheses inside a macro. if linenum > 0: - for i in xrange(linenum - 1, max(0, linenum - 5), -1): + for i in range(linenum - 1, max(0, linenum - 5), -1): context = clean_lines.elided[i] + context - if Match(r'.*\b[_A-Z][_A-Z0-9]*\s*\((?:\([^()]*\)|[^()])*$', context): + if re.match(r'.*\b[_A-Z][_A-Z0-9]*\s*\((?:\([^()]*\)|[^()])*$', context): return False # operator++(int) and operator--(int) - if context.endswith(' operator++') or context.endswith(' operator--'): + if (context.endswith(' operator++') or context.endswith(' operator--') or + context.endswith('::operator++') or context.endswith('::operator--')): return False - # A single unnamed argument for a function tends to look like old - # style cast. If we see those, don't issue warnings for deprecated - # casts, instead issue warnings for unnamed arguments where - # appropriate. - # - # These are things that we want warnings for, since the style guide - # explicitly require all parameters to be named: - # Function(int); - # Function(int) { - # ConstMember(int) const; - # ConstMember(int) const { - # ExceptionMember(int) throw (...); - # ExceptionMember(int) throw (...) { - # PureVirtual(int) = 0; - # [](int) -> bool { - # - # These are functions of some sort, where the compiler would be fine - # if they had named parameters, but people often omit those - # identifiers to reduce clutter: - # (FunctionPointer)(int); - # (FunctionPointer)(int) = value; - # Function((function_pointer_arg)(int)) - # Function((function_pointer_arg)(int), int param) - # ; - # <(FunctionPointerTemplateArgument)(int)>; + # A single unnamed argument for a function tends to look like old style cast. + # If we see those, don't issue warnings for deprecated casts. remainder = line[match.end(0):] - if Match(r'^\s*(?:;|const\b|throw\b|final\b|override\b|[=>{),]|->)', + if re.match(r'^\s*(?:;|const\b|throw\b|final\b|override\b|[=>{),]|->)', remainder): - # Looks like an unnamed parameter. - - # Don't warn on any kind of template arguments. - if Match(r'^\s*>', remainder): - return False - - # Don't warn on assignments to function pointers, but keep warnings for - # unnamed parameters to pure virtual functions. Note that this pattern - # will also pass on assignments of "0" to function pointers, but the - # preferred values for those would be "nullptr" or "NULL". - matched_zero = Match(r'^\s=\s*(\S+)\s*;', remainder) - if matched_zero and matched_zero.group(1) != '0': - return False - - # Don't warn on function pointer declarations. For this we need - # to check what came before the "(type)" string. - if Match(r'.*\)\s*$', line[0:match.start(0)]): - return False - - # Don't warn if the parameter is named with block comments, e.g.: - # Function(int /*unused_param*/); - raw_line = clean_lines.raw_lines[linenum] - if '/*' in raw_line: - return False - - # Passed all filters, issue warning here. - error(filename, linenum, 'readability/function', 3, - 'All parameters should be named in a function') - return True + return False # At this point, all that should be left is actual casts. error(filename, linenum, 'readability/casting', 4, - 'Using C-style cast. Use %s<%s>(...) instead' % - (cast_type, match.group(1))) + f'Using C-style cast. Use {cast_type}<{match.group(1)}>(...) instead') return True @@ -5452,13 +5976,13 @@ def ExpectingFunctionArgs(clean_lines, linenum): of function types. """ line = clean_lines.elided[linenum] - return (Match(r'^\s*MOCK_(CONST_)?METHOD\d+(_T)?\(', line) or + return (re.match(r'^\s*MOCK_(CONST_)?METHOD\d+(_T)?\(', line) or (linenum >= 2 and - (Match(r'^\s*MOCK_(?:CONST_)?METHOD\d+(?:_T)?\((?:\S+,)?\s*$', + (re.match(r'^\s*MOCK_(?:CONST_)?METHOD\d+(?:_T)?\((?:\S+,)?\s*$', clean_lines.elided[linenum - 1]) or - Match(r'^\s*MOCK_(?:CONST_)?METHOD\d+(?:_T)?\(\s*$', + re.match(r'^\s*MOCK_(?:CONST_)?METHOD\d+(?:_T)?\(\s*$', clean_lines.elided[linenum - 2]) or - Search(r'\bstd::m?function\s*\<\s*$', + re.search(r'\bstd::m?function\s*\<\s*$', clean_lines.elided[linenum - 1])))) @@ -5483,13 +6007,16 @@ def ExpectingFunctionArgs(clean_lines, linenum): )), ('', ('numeric_limits',)), ('', ('list',)), - ('', ('map', 'multimap',)), - ('', ('allocator',)), + ('', ('multimap',)), + ('', ('allocator', 'make_shared', 'make_unique', 'shared_ptr', + 'unique_ptr', 'weak_ptr')), ('', ('queue', 'priority_queue',)), ('', ('set', 'multiset',)), ('', ('stack',)), ('', ('char_traits', 'basic_string',)), ('', ('tuple',)), + ('', ('unordered_map', 'unordered_multimap')), + ('', ('unordered_set', 'unordered_multiset')), ('', ('pair',)), ('', ('vector',)), @@ -5500,26 +6027,77 @@ def ExpectingFunctionArgs(clean_lines, linenum): ('', ('slist',)), ) -_RE_PATTERN_STRING = re.compile(r'\bstring\b') - -_re_pattern_algorithm_header = [] -for _template in ('copy', 'max', 'min', 'min_element', 'sort', 'swap', - 'transform'): - # Match max(..., ...), max(..., ...), but not foo->max, foo.max or - # type::max(). - _re_pattern_algorithm_header.append( - (re.compile(r'[^>.]\b' + _template + r'(<.*?>)?\([^\)]'), - _template, - '')) +_HEADERS_MAYBE_TEMPLATES = ( + ('', ('copy', 'max', 'min', 'min_element', 'sort', + 'transform', + )), + ('', ('forward', 'make_pair', 'move', 'swap')), + ) +# Non templated types or global objects +_HEADERS_TYPES_OR_OBJS = ( + # String and others are special -- it is a non-templatized type in STL. + ('', ('string',)), + ('', ('cin', 'cout', 'cerr', 'clog', 'wcin', 'wcout', + 'wcerr', 'wclog')), + ('', ('FILE', 'fpos_t'))) + +# Non templated functions +_HEADERS_FUNCTIONS = ( + ('', ('fopen', 'freopen', + 'fclose', 'fflush', 'setbuf', 'setvbuf', 'fread', + 'fwrite', 'fgetc', 'getc', 'fgets', 'fputc', 'putc', + 'fputs', 'getchar', 'gets', 'putchar', 'puts', 'ungetc', + 'scanf', 'fscanf', 'sscanf', 'vscanf', 'vfscanf', + 'vsscanf', 'printf', 'fprintf', 'sprintf', 'snprintf', + 'vprintf', 'vfprintf', 'vsprintf', 'vsnprintf', + 'ftell', 'fgetpos', 'fseek', 'fsetpos', + 'clearerr', 'feof', 'ferror', 'perror', + 'tmpfile', 'tmpnam'),),) + +_re_pattern_headers_maybe_templates = [] +for _header, _templates in _HEADERS_MAYBE_TEMPLATES: + for _template in _templates: + # Match max(..., ...), max(..., ...), but not foo->max, foo.max or + # 'type::max()'. + _re_pattern_headers_maybe_templates.append( + (re.compile(r'((\bstd::)|[^>.:])\b' + _template + r'(<.*?>)?\([^\)]'), + _template, + _header)) + +# Map is often overloaded. Only check, if it is fully qualified. +# Match 'std::map(...)', but not 'map(...)'' +_re_pattern_headers_maybe_templates.append( + (re.compile(r'(std\b::\bmap\s*\<)|(^(std\b::\b)map\b\(\s*\<)'), + 'map<>', + '')) + +# Other scripts may reach in and modify this pattern. _re_pattern_templates = [] for _header, _templates in _HEADERS_CONTAINING_TEMPLATES: for _template in _templates: _re_pattern_templates.append( - (re.compile(r'(\<|\b)' + _template + r'\s*\<'), + (re.compile(r'((^|(^|\s|((^|\W)::))std::)|[^>.:]\b)' + _template + r'\s*\<'), _template + '<>', _header)) +_re_pattern_types_or_objs = [] +for _header, _types_or_objs in _HEADERS_TYPES_OR_OBJS: + for _type_or_obj in _types_or_objs: + _re_pattern_types_or_objs.append( + (re.compile(r'\b' + _type_or_obj + r'\b'), + _type_or_obj, + _header)) + +_re_pattern_functions = [] +for _header, _functions in _HEADERS_FUNCTIONS: + for _function in _functions: + # Match printf(..., ...), but not foo->printf, foo.printf or + # 'type::printf()'. + _re_pattern_functions.append( + (re.compile(r'([^>.]|^)\b' + _function + r'\([^\)]'), + _function, + _header)) def FilesBelongToSameModule(filename_cc, filename_h): """Check if these two filenames belong to the same module. @@ -5542,7 +6120,7 @@ def FilesBelongToSameModule(filename_cc, filename_h): some false positives. This should be sufficiently rare in practice. Args: - filename_cc: is the path for the .cc file + filename_cc: is the path for the source (e.g. .cc) file filename_h: is the path for the header path Returns: @@ -5550,20 +6128,23 @@ def FilesBelongToSameModule(filename_cc, filename_h): bool: True if filename_cc and filename_h belong to the same module. string: the additional prefix needed to open the header file. """ + fileinfo_cc = FileInfo(filename_cc) + if fileinfo_cc.Extension().lstrip('.') not in GetNonHeaderExtensions(): + return (False, '') - if not filename_cc.endswith('.cc'): + fileinfo_h = FileInfo(filename_h) + if not IsHeaderExtension(fileinfo_h.Extension().lstrip('.')): return (False, '') - filename_cc = filename_cc[:-len('.cc')] - if filename_cc.endswith('_unittest'): - filename_cc = filename_cc[:-len('_unittest')] - elif filename_cc.endswith('_test'): - filename_cc = filename_cc[:-len('_test')] + + filename_cc = filename_cc[:-(len(fileinfo_cc.Extension()))] + matched_test_suffix = re.search(_TEST_FILE_SUFFIX, fileinfo_cc.BaseName()) + if matched_test_suffix: + filename_cc = filename_cc[:-len(matched_test_suffix.group(1))] + filename_cc = filename_cc.replace('/public/', '/') filename_cc = filename_cc.replace('/internal/', '/') - if not filename_h.endswith('.h'): - return (False, '') - filename_h = filename_h[:-len('.h')] + filename_h = filename_h[:-(len(fileinfo_h.Extension()))] if filename_h.endswith('-inl'): filename_h = filename_h[:-len('-inl')] filename_h = filename_h.replace('/public/', '/') @@ -5576,33 +6157,6 @@ def FilesBelongToSameModule(filename_cc, filename_h): return files_belong_to_same_module, common_path -def UpdateIncludeState(filename, include_dict, io=codecs): - """Fill up the include_dict with new includes found from the file. - - Args: - filename: the name of the header to read. - include_dict: a dictionary in which the headers are inserted. - io: The io factory to use to read the file. Provided for testability. - - Returns: - True if a header was successfully added. False otherwise. - """ - headerfile = None - try: - headerfile = io.open(filename, 'r', 'utf8', 'replace') - except IOError: - return False - linenum = 0 - for line in headerfile: - linenum += 1 - clean_line = CleanseComments(line) - match = _RE_PATTERN_INCLUDE.search(clean_line) - if match: - include = match.group(2) - include_dict.setdefault(include, linenum) - return True - - def CheckForIncludeWhatYouUse(filename, clean_lines, include_state, error, io=codecs): """Reports for missing stl includes. @@ -5624,72 +6178,46 @@ def CheckForIncludeWhatYouUse(filename, clean_lines, include_state, error, required = {} # A map of header name to linenumber and the template entity. # Example of required: { '': (1219, 'less<>') } - for linenum in xrange(clean_lines.NumLines()): + for linenum in range(clean_lines.NumLines()): line = clean_lines.elided[linenum] if not line or line[0] == '#': continue - # String is special -- it is a non-templatized type in STL. - matched = _RE_PATTERN_STRING.search(line) - if matched: - # Don't warn about strings in non-STL namespaces: - # (We check only the first match per line; good enough.) - prefix = line[:matched.start()] - if prefix.endswith('std::') or not prefix.endswith('::'): - required[''] = (linenum, 'string') + _re_patterns = [] + _re_patterns.extend(_re_pattern_types_or_objs) + _re_patterns.extend(_re_pattern_functions) + for pattern, item, header in _re_patterns: + matched = pattern.search(line) + if matched: + # Don't warn about strings in non-STL namespaces: + # (We check only the first match per line; good enough.) + prefix = line[:matched.start()] + if prefix.endswith('std::') or not prefix.endswith('::'): + required[header] = (linenum, item) - for pattern, template, header in _re_pattern_algorithm_header: + for pattern, template, header in _re_pattern_headers_maybe_templates: if pattern.search(line): required[header] = (linenum, template) # The following function is just a speed up, no semantics are changed. - if not '<' in line: # Reduces the cpu time usage by skipping lines. + if '<' not in line: # Reduces the cpu time usage by skipping lines. continue for pattern, template, header in _re_pattern_templates: - if pattern.search(line): - required[header] = (linenum, template) + matched = pattern.search(line) + if matched: + # Don't warn about IWYU in non-STL namespaces: + # (We check only the first match per line; good enough.) + prefix = line[:matched.start()] + if prefix.endswith('std::') or not prefix.endswith('::'): + required[header] = (linenum, template) - # The policy is that if you #include something in foo.h you don't need to - # include it again in foo.cc. Here, we will look at possible includes. # Let's flatten the include_state include_list and copy it into a dictionary. include_dict = dict([item for sublist in include_state.include_list for item in sublist]) - # Did we find the header for this file (if any) and successfully load it? - header_found = False - - # Use the absolute path so that matching works properly. - abs_filename = FileInfo(filename).FullName() - - # For Emacs's flymake. - # If cpplint is invoked from Emacs's flymake, a temporary file is generated - # by flymake and that file name might end with '_flymake.cc'. In that case, - # restore original file name here so that the corresponding header file can be - # found. - # e.g. If the file name is 'foo_flymake.cc', we should search for 'foo.h' - # instead of 'foo_flymake.h' - abs_filename = re.sub(r'_flymake\.cc$', '.cc', abs_filename) - - # include_dict is modified during iteration, so we iterate over a copy of - # the keys. - header_keys = include_dict.keys() - for header in header_keys: - (same_module, common_path) = FilesBelongToSameModule(abs_filename, header) - fullpath = common_path + header - if same_module and UpdateIncludeState(fullpath, include_dict, io): - header_found = True - - # If we can't find the header file for a .cc, assume it's because we don't - # know where to look. In that case we'll give up as we're not sure they - # didn't include it in the .h file. - # TODO(unknown): Do a better job of finding .h files so we are confident that - # not having the .h file means there isn't one. - if filename.endswith('.cc') and not header_found: - return - # All the lines have been processed, report the errors found. - for required_header_unstripped in required: + for required_header_unstripped in sorted(required, key=required.__getitem__): template = required[required_header_unstripped][1] if required_header_unstripped.strip('<>"') not in include_dict: error(filename, required[required_header_unstripped][0], @@ -5721,31 +6249,6 @@ def CheckMakePairUsesDeduction(filename, clean_lines, linenum, error): ' OR use pair directly OR if appropriate, construct a pair directly') -def CheckDefaultLambdaCaptures(filename, clean_lines, linenum, error): - """Check that default lambda captures are not used. - - Args: - filename: The name of the current file. - clean_lines: A CleansedLines instance containing the file. - linenum: The number of the line to check. - error: The function to call with any errors found. - """ - line = clean_lines.elided[linenum] - - # A lambda introducer specifies a default capture if it starts with "[=" - # or if it starts with "[&" _not_ followed by an identifier. - match = Match(r'^(.*)\[\s*(?:=|&[^\w])', line) - if match: - # Found a potential error, check what comes after the lambda-introducer. - # If it's not open parenthesis (for lambda-declarator) or open brace - # (for compound-statement), it's not a lambda. - line, _, pos = CloseExpression(clean_lines, linenum, len(match.group(1))) - if pos >= 0 and Match(r'^\s*[{(]', line[pos:]): - error(filename, linenum, 'build/c++11', - 4, # 4 = high confidence - 'Default lambda captures are an unapproved C++ feature.') - - def CheckRedundantVirtual(filename, clean_lines, linenum, error): """Check if line contains a redundant "virtual" function-specifier. @@ -5757,20 +6260,20 @@ def CheckRedundantVirtual(filename, clean_lines, linenum, error): """ # Look for "virtual" on current line. line = clean_lines.elided[linenum] - virtual = Match(r'^(.*)(\bvirtual\b)(.*)$', line) + virtual = re.match(r'^(.*)(\bvirtual\b)(.*)$', line) if not virtual: return # Ignore "virtual" keywords that are near access-specifiers. These # are only used in class base-specifier and do not apply to member # functions. - if (Search(r'\b(public|protected|private)\s+$', virtual.group(1)) or - Match(r'^\s+(public|protected|private)\b', virtual.group(3))): + if (re.search(r'\b(public|protected|private)\s+$', virtual.group(1)) or + re.match(r'^\s+(public|protected|private)\b', virtual.group(3))): return # Ignore the "virtual" keyword from virtual base classes. Usually # there is a column on the same line in these cases (virtual base # classes are rare in google3 because multiple inheritance is rare). - if Match(r'^.*[^:]:[^:].*$', line): return + if re.match(r'^.*[^:]:[^:].*$', line): return # Look for the next opening parenthesis. This is the start of the # parameter list (possibly on the next line shortly after virtual). @@ -5780,9 +6283,9 @@ def CheckRedundantVirtual(filename, clean_lines, linenum, error): end_col = -1 end_line = -1 start_col = len(virtual.group(2)) - for start_line in xrange(linenum, min(linenum + 3, clean_lines.NumLines())): + for start_line in range(linenum, min(linenum + 3, clean_lines.NumLines())): line = clean_lines.elided[start_line][start_col:] - parameter_list = Match(r'^([^(]*)\(', line) + parameter_list = re.match(r'^([^(]*)\(', line) if parameter_list: # Match parentheses to find the end of the parameter list (_, end_line, end_col) = CloseExpression( @@ -5795,18 +6298,18 @@ def CheckRedundantVirtual(filename, clean_lines, linenum, error): # Look for "override" or "final" after the parameter list # (possibly on the next few lines). - for i in xrange(end_line, min(end_line + 3, clean_lines.NumLines())): + for i in range(end_line, min(end_line + 3, clean_lines.NumLines())): line = clean_lines.elided[i][end_col:] - match = Search(r'\b(override|final)\b', line) + match = re.search(r'\b(override|final)\b', line) if match: error(filename, linenum, 'readability/inheritance', 4, ('"virtual" is redundant since function is ' - 'already declared as "%s"' % match.group(1))) + f'already declared as "{match.group(1)}"')) # Set end_col to check whole lines after we are done with the # first line. end_col = 0 - if Search(r'[^\w]\s*$', line): + if re.search(r'[^\w]\s*$', line): break @@ -5833,7 +6336,7 @@ def CheckRedundantOverrideOrFinal(filename, clean_lines, linenum, error): return # Check that at most one of "override" or "final" is present, not both - if Search(r'\boverride\b', fragment) and Search(r'\bfinal\b', fragment): + if re.search(r'\boverride\b', fragment) and re.search(r'\bfinal\b', fragment): error(filename, linenum, 'readability/inheritance', 4, ('"override" is redundant since function is ' 'already declared as "final"')) @@ -5853,15 +6356,17 @@ def IsBlockInNameSpace(nesting_state, is_forward_declaration): Whether or not the new block is directly in a namespace. """ if is_forward_declaration: - if len(nesting_state.stack) >= 1 and ( - isinstance(nesting_state.stack[-1], _NamespaceInfo)): - return True - else: - return False + return len(nesting_state.stack) >= 1 and ( + isinstance(nesting_state.stack[-1], _NamespaceInfo)) - return (len(nesting_state.stack) > 1 and - nesting_state.stack[-1].check_namespace_indentation and - isinstance(nesting_state.stack[-2], _NamespaceInfo)) + if len(nesting_state.stack) >= 1: + if isinstance(nesting_state.stack[-1], _NamespaceInfo): + return True + elif (len(nesting_state.stack) > 1 and + isinstance(nesting_state.previous_stack_top, _NamespaceInfo) and + isinstance(nesting_state.stack[-2], _NamespaceInfo)): + return True + return False def ShouldCheckNamespaceIndentation(nesting_state, is_namespace_indent_item, @@ -5900,14 +6405,14 @@ def ShouldCheckNamespaceIndentation(nesting_state, is_namespace_indent_item, def CheckItemIndentationInNamespace(filename, raw_lines_no_comments, linenum, error): line = raw_lines_no_comments[linenum] - if Match(r'^\s+', line): - error(filename, linenum, 'runtime/indentation_namespace', 4, - 'Do not indent within a namespace') + if re.match(r'^\s+', line): + error(filename, linenum, 'whitespace/indent_namespace', 4, + 'Do not indent within a namespace.') def ProcessLine(filename, file_extension, clean_lines, line, include_state, function_state, nesting_state, error, - extra_check_functions=[]): + extra_check_functions=None): """Processes a single line in the file. Args: @@ -5944,14 +6449,15 @@ def ProcessLine(filename, file_extension, clean_lines, line, CheckPosixThreading(filename, clean_lines, line, error) CheckInvalidIncrement(filename, clean_lines, line, error) CheckMakePairUsesDeduction(filename, clean_lines, line, error) - CheckDefaultLambdaCaptures(filename, clean_lines, line, error) CheckRedundantVirtual(filename, clean_lines, line, error) CheckRedundantOverrideOrFinal(filename, clean_lines, line, error) - for check_fn in extra_check_functions: - check_fn(filename, clean_lines, line, error) + if extra_check_functions: + for check_fn in extra_check_functions: + check_fn(filename, clean_lines, line, error) -def FlagCxx11Features(filename, clean_lines, linenum, error): - """Flag those c++11 features that we only allow in certain places. + +def FlagCxxHeaders(filename, clean_lines, linenum, error): + """Flag C++ headers that the styleguide restricts. Args: filename: The name of the current file. @@ -5961,43 +6467,24 @@ def FlagCxx11Features(filename, clean_lines, linenum, error): """ line = clean_lines.elided[linenum] + include = re.match(r'\s*#\s*include\s+[<"]([^<"]+)[">]', line) + # Flag unapproved C++11 headers. - include = Match(r'\s*#\s*include\s+[<"]([^<"]+)[">]', line) if include and include.group(1) in ('cfenv', - 'condition_variable', 'fenv.h', - 'future', - 'mutex', - 'thread', - 'chrono', 'ratio', - 'regex', - 'system_error', ): error(filename, linenum, 'build/c++11', 5, - ('<%s> is an unapproved C++11 header.') % include.group(1)) - - # The only place where we need to worry about C++11 keywords and library - # features in preprocessor directives is in macro definitions. - if Match(r'\s*#', line) and not Match(r'\s*#\s*define\b', line): return - - # These are classes and free functions. The classes are always - # mentioned as std::*, but we only catch the free functions if - # they're not found by ADL. They're alphabetical by header. - for top_name in ( - # type_traits - 'alignment_of', - 'aligned_union', - ): - if Search(r'\bstd::%s\b' % top_name, line): - error(filename, linenum, 'build/c++11', 5, - ('std::%s is an unapproved C++11 class or function. Send c-style ' - 'an example of where it would make your code more readable, and ' - 'they may let you use it.') % top_name) + f"<{include.group(1)}> is an unapproved C++11 header.") + + # filesystem is the only unapproved C++17 header + if include and include.group(1) == 'filesystem': + error(filename, linenum, 'build/c++17', 5, + " is an unapproved C++17 header.") def ProcessFileData(filename, file_extension, lines, error, - extra_check_functions=[]): + extra_check_functions=None): """Performs lint checks and reports any errors to the given error function. Args: @@ -6021,24 +6508,26 @@ def ProcessFileData(filename, file_extension, lines, error, ResetNolintSuppressions() CheckForCopyright(filename, lines, error) - + ProcessGlobalSuppressions(lines) RemoveMultiLineComments(filename, lines, error) clean_lines = CleansedLines(lines) - if file_extension == 'h': + if IsHeaderExtension(file_extension): CheckForHeaderGuard(filename, clean_lines, error) - for line in xrange(clean_lines.NumLines()): + for line in range(clean_lines.NumLines()): ProcessLine(filename, file_extension, clean_lines, line, include_state, function_state, nesting_state, error, extra_check_functions) - FlagCxx11Features(filename, clean_lines, line, error) - nesting_state.CheckCompletedBlocks(filename, error) + FlagCxxHeaders(filename, clean_lines, line, error) + if _error_suppressions.HasOpenBlock(): + error(filename, _error_suppressions.GetOpenBlockStart(), 'readability/nolint', 5, + 'NONLINT block never ended') CheckForIncludeWhatYouUse(filename, clean_lines, include_state, error) - + # Check that the .cc file has included its header if it exists. - if file_extension == 'cc': + if _IsSourceExtension(file_extension): CheckHeaderFileIncluded(filename, include_state, error) # We check here rather than inside ProcessLine so that we see raw @@ -6065,13 +6554,13 @@ def ProcessConfigOverrides(filename): if not base_name: break # Reached the root directory. - cfg_file = os.path.join(abs_path, "CPPLINT.cfg") + cfg_file = os.path.join(abs_path, _config_filename) abs_filename = abs_path if not os.path.isfile(cfg_file): continue try: - with open(cfg_file) as file_handle: + with codecs.open(cfg_file, 'r', 'utf8', 'replace') as file_handle: for line in file_handle: line, _, _ = line.partition('#') # Remove comments. if not line.strip(): @@ -6094,36 +6583,48 @@ def ProcessConfigOverrides(filename): if base_name: pattern = re.compile(val) if pattern.match(base_name): - sys.stderr.write('Ignoring "%s": file excluded by "%s". ' + if _cpplint_state.quiet: + # Suppress "Ignoring file" warning when using --quiet. + return False + _cpplint_state.PrintInfo(f'Ignoring "{filename}": file excluded by "{cfg_file}". ' 'File path component "%s" matches ' 'pattern "%s"\n' % - (filename, cfg_file, base_name, val)) + (base_name, val)) return False elif name == 'linelength': global _line_length try: - _line_length = int(val) + _line_length = int(val) except ValueError: - sys.stderr.write('Line length must be numeric.') + _cpplint_state.PrintError('Line length must be numeric.') + elif name == 'extensions': + ProcessExtensionsOption(val) + elif name == 'root': + global _root + # root directories are specified relative to CPPLINT.cfg dir. + _root = os.path.join(os.path.dirname(cfg_file), val) + elif name == 'headers': + ProcessHppHeadersOption(val) + elif name == 'includeorder': + ProcessIncludeOrderOption(val) else: - sys.stderr.write( - 'Invalid configuration option (%s) in file %s\n' % - (name, cfg_file)) + _cpplint_state.PrintError( + f'Invalid configuration option ({name}) in file {cfg_file}\n') except IOError: - sys.stderr.write( - "Skipping config file '%s': Can't open for reading\n" % cfg_file) + _cpplint_state.PrintError( + f"Skipping config file '{cfg_file}': Can't open for reading\n") keep_looking = False # Apply all the accumulated filters in reverse order (top-level directory # config options having the least priority). - for filter in reversed(cfg_filters): - _AddFilters(filter) + for cfg_filter in reversed(cfg_filters): + _AddFilters(cfg_filter) return True -def ProcessFile(filename, vlevel, extra_check_functions=[]): +def ProcessFile(filename, vlevel, extra_check_functions=None): """Does google-lint on a single file. Args: @@ -6139,6 +6640,7 @@ def ProcessFile(filename, vlevel, extra_check_functions=[]): _SetVerboseLevel(vlevel) _BackupFilters() + old_errors = _cpplint_state.error_count if not ProcessConfigOverrides(filename): _RestoreFilters() @@ -6160,7 +6662,8 @@ def ProcessFile(filename, vlevel, extra_check_functions=[]): codecs.getwriter('utf8'), 'replace').read().split('\n') else: - lines = codecs.open(filename, 'r', 'utf8', 'replace').read().split('\n') + with codecs.open(filename, 'r', 'utf8', 'replace') as target_file: + lines = target_file.read().split('\n') # Remove trailing '\r'. # The -1 accounts for the extra trailing blank line we get from split() @@ -6172,8 +6675,9 @@ def ProcessFile(filename, vlevel, extra_check_functions=[]): lf_lines.append(linenum + 1) except IOError: - sys.stderr.write( - "Skipping input '%s': Can't open for reading\n" % filename) + # TODO: Maybe make this have an exit code of 2 after all is done + _cpplint_state.PrintError( + f"Skipping input '{filename}': Can't open for reading\n") _RestoreFilters() return @@ -6182,9 +6686,9 @@ def ProcessFile(filename, vlevel, extra_check_functions=[]): # When reading from stdin, the extension is unknown, so no cpplint tests # should rely on the extension. - if filename != '-' and file_extension not in _valid_extensions: - sys.stderr.write('Ignoring %s; not a valid file name ' - '(%s)\n' % (filename, ', '.join(_valid_extensions))) + if filename != '-' and file_extension not in GetAllExtensions(): + _cpplint_state.PrintError(f'Ignoring {filename}; not a valid file name' + f' ({(", ".join(GetAllExtensions()))})\n') else: ProcessFileData(filename, file_extension, lines, Error, extra_check_functions) @@ -6207,7 +6711,10 @@ def ProcessFile(filename, vlevel, extra_check_functions=[]): Error(filename, linenum, 'whitespace/newline', 1, 'Unexpected \\r (^M) found; better to use only \\n') - sys.stderr.write('Done processing %s\n' % filename) + # Suppress printing anything if --quiet was passed unless the error + # count has increased after processing this file. + if not _cpplint_state.quiet or old_errors != _cpplint_state.error_count: + _cpplint_state.PrintInfo(f'Done processing {filename}\n') _RestoreFilters() @@ -6217,19 +6724,28 @@ def PrintUsage(message): Args: message: The optional error message. """ - sys.stderr.write(_USAGE) + sys.stderr.write(_USAGE % (sorted(list(GetAllExtensions())), + ','.join(sorted(list(GetAllExtensions()))), + sorted(GetHeaderExtensions()), + ','.join(sorted(GetHeaderExtensions())))) + if message: sys.exit('\nFATAL ERROR: ' + message) else: - sys.exit(1) + sys.exit(0) +def PrintVersion(): + sys.stdout.write('Cpplint fork (https://github.com/cpplint/cpplint)\n') + sys.stdout.write('cpplint ' + __VERSION__ + '\n') + sys.stdout.write('Python ' + sys.version + '\n') + sys.exit(0) def PrintCategories(): """Prints a list of all the error-categories used by error messages. These are the categories used to filter messages via --filter. """ - sys.stderr.write(''.join(' %s\n' % cat for cat in _ERROR_CATEGORIES)) + sys.stderr.write(''.join(f' {cat}\n' for cat in _ERROR_CATEGORIES)) sys.exit(0) @@ -6246,27 +6762,43 @@ def ParseArguments(args): """ try: (opts, filenames) = getopt.getopt(args, '', ['help', 'output=', 'verbose=', + 'v=', + 'version', 'counting=', 'filter=', 'root=', + 'repository=', 'linelength=', - 'extensions=']) + 'extensions=', + 'exclude=', + 'recursive', + 'headers=', + 'includeorder=', + 'config=', + 'quiet']) except getopt.GetoptError: PrintUsage('Invalid arguments.') verbosity = _VerboseLevel() output_format = _OutputFormat() filters = '' + quiet = _Quiet() counting_style = '' + recursive = False for (opt, val) in opts: if opt == '--help': PrintUsage(None) + if opt == '--version': + PrintVersion() elif opt == '--output': - if val not in ('emacs', 'vs7', 'eclipse'): - PrintUsage('The only allowed output formats are emacs, vs7 and eclipse.') + if val not in ('emacs', 'vs7', 'eclipse', 'junit', 'sed', 'gsed'): + PrintUsage('The only allowed output formats are emacs, vs7, eclipse ' + 'sed, gsed and junit.') output_format = val - elif opt == '--verbose': + elif opt == '--quiet': + quiet = True + elif opt == '--verbose' or opt == '--v': verbosity = int(val) elif opt == '--filter': filters = val @@ -6279,44 +6811,157 @@ def ParseArguments(args): elif opt == '--root': global _root _root = val + elif opt == '--repository': + global _repository + _repository = val elif opt == '--linelength': global _line_length try: - _line_length = int(val) + _line_length = int(val) except ValueError: - PrintUsage('Line length must be digits.') + PrintUsage('Line length must be digits.') + elif opt == '--exclude': + global _excludes + if not _excludes: + _excludes = set() + _excludes.update(glob.glob(val)) elif opt == '--extensions': - global _valid_extensions - try: - _valid_extensions = set(val.split(',')) - except ValueError: - PrintUsage('Extensions must be comma seperated list.') + ProcessExtensionsOption(val) + elif opt == '--headers': + ProcessHppHeadersOption(val) + elif opt == '--recursive': + recursive = True + elif opt == '--includeorder': + ProcessIncludeOrderOption(val) + elif opt == '--config': + global _config_filename + _config_filename = val + if os.path.basename(_config_filename) != _config_filename: + PrintUsage('Config file name must not include directory components.') if not filenames: PrintUsage('No files were specified.') + if recursive: + filenames = _ExpandDirectories(filenames) + + if _excludes: + filenames = _FilterExcludedFiles(filenames) + _SetOutputFormat(output_format) + _SetQuiet(quiet) _SetVerboseLevel(verbosity) _SetFilters(filters) _SetCountingStyle(counting_style) + filenames.sort() return filenames +def _ParseFilterSelector(parameter): + """Parses the given command line parameter for file- and line-specific + exclusions. + readability/casting:file.cpp + readability/casting:file.cpp:43 -def main(): - filenames = ParseArguments(sys.argv[1:]) + Args: + parameter: The parameter value of --filter + + Returns: + [category, filename, line]. + Category is always given. + Filename is either a filename or empty if all files are meant. + Line is either a line in filename or -1 if all lines are meant. + """ + colon_pos = parameter.find(":") + if colon_pos == -1: + return parameter, "", -1 + category = parameter[:colon_pos] + second_colon_pos = parameter.find(":", colon_pos + 1) + if second_colon_pos == -1: + return category, parameter[colon_pos + 1:], -1 + else: + return category, parameter[colon_pos + 1: second_colon_pos], \ + int(parameter[second_colon_pos + 1:]) + +def _ExpandDirectories(filenames): + """Searches a list of filenames and replaces directories in the list with + all files descending from those directories. Files with extensions not in + the valid extensions list are excluded. - # Change stderr to write with replacement characters so we don't die - # if we try to print something containing non-ASCII characters. - sys.stderr = codecs.StreamReaderWriter(sys.stderr, - codecs.getreader('utf8'), - codecs.getwriter('utf8'), - 'replace') + Args: + filenames: A list of files or directories - _cpplint_state.ResetErrorCounts() + Returns: + A list of all files that are members of filenames or descended from a + directory in filenames + """ + expanded = set() for filename in filenames: - ProcessFile(filename, _cpplint_state.verbose_level) - _cpplint_state.PrintErrorCounts() + if not os.path.isdir(filename): + expanded.add(filename) + continue + + for root, _, files in os.walk(filename): + for loopfile in files: + fullname = os.path.join(root, loopfile) + if fullname.startswith('.' + os.path.sep): + fullname = fullname[len('.' + os.path.sep):] + expanded.add(fullname) + + filtered = [] + for filename in expanded: + if os.path.splitext(filename)[1][1:] in GetAllExtensions(): + filtered.append(filename) + return filtered + +def _FilterExcludedFiles(fnames): + """Filters out files listed in the --exclude command line switch. File paths + in the switch are evaluated relative to the current working directory + """ + exclude_paths = [os.path.abspath(f) for f in _excludes] + # because globbing does not work recursively, exclude all subpath of all excluded entries + return [f for f in fnames + if not any(e for e in exclude_paths + if _IsParentOrSame(e, os.path.abspath(f)))] + +def _IsParentOrSame(parent, child): + """Return true if child is subdirectory of parent. + Assumes both paths are absolute and don't contain symlinks. + """ + parent = os.path.normpath(parent) + child = os.path.normpath(child) + if parent == child: + return True + + prefix = os.path.commonprefix([parent, child]) + if prefix != parent: + return False + # Note: os.path.commonprefix operates on character basis, so + # take extra care of situations like '/foo/ba' and '/foo/bar/baz' + child_suffix = child[len(prefix):] + child_suffix = child_suffix.lstrip(os.sep) + return child == os.path.join(prefix, child_suffix) + +def main(): + filenames = ParseArguments(sys.argv[1:]) + backup_err = sys.stderr + try: + # Change stderr to write with replacement characters so we don't die + # if we try to print something containing non-ASCII characters. + sys.stderr = codecs.StreamReader(sys.stderr, 'replace') + + _cpplint_state.ResetErrorCounts() + for filename in filenames: + ProcessFile(filename, _cpplint_state.verbose_level) + # If --quiet is passed, suppress printing error count unless there are errors. + if not _cpplint_state.quiet or _cpplint_state.error_count > 0: + _cpplint_state.PrintErrorCounts() + + if _cpplint_state.output_format == 'junit': + sys.stderr.write(_cpplint_state.FormatJUnitXML()) + + finally: + sys.stderr = backup_err sys.exit(_cpplint_state.error_count > 0) diff --git a/eslint.config.js b/eslint.config.js new file mode 100644 index 00000000..89faf73e --- /dev/null +++ b/eslint.config.js @@ -0,0 +1,54 @@ +const js = require("@eslint/js"); +const jest = require('eslint-plugin-jest'); + +const ckjsSpecificSettings = { + languageOptions: { + globals: { + "require": "readonly", + "module": "writable", + "setImmediate": "readonly", + "setTimeout": "readonly", + "clearTimeout": "readonly", + "setInterval": "readonly", + "clearInterval": "readonly", + "console": "readonly" + } + }, + "rules": { + "eqeqeq": ["error", "always"], + "no-use-before-define": ["error", "nofunc"], + "no-caller": "error", + "no-new": "error", + "no-eq-null": "error", + "no-constant-condition": "off", + "semi": "error" + } +}; + +const ckjsSpecificJestSettings = { + "rules": { + "jest/no-disabled-tests": "off", + "jest/no-conditional-expect": "off", + } +}; + +module.exports = [ + { + ...js.configs.recommended, + files: ["lib/**/*.js", "test/promisified/**/*.js"], + ignores: ["lib/kafkajs/_heap.js"] + }, + { + ...ckjsSpecificSettings, + files: ["lib/**/*.js", "test/promisified/**/*.js"], + ignores: ["lib/kafkajs/_heap.js"] + }, + { + ...jest.configs['flat/recommended'], + files: ["test/promisified/**/*.js"] + }, + { + ...ckjsSpecificJestSettings, + files: ["test/promisified/**/*.js"] + } +]; diff --git a/lib/admin.js b/lib/admin.js index 6c7968ce..fe8750cc 100644 --- a/lib/admin.js +++ b/lib/admin.js @@ -48,7 +48,7 @@ var Client = require('./client'); var util = require('util'); var Kafka = require('../librdkafka'); var LibrdKafkaError = require('./error'); -var { shallowCopy, bindingVersion } = require('./util'); +var { shallowCopy } = require('./util'); util.inherits(AdminClient, Client); @@ -299,7 +299,7 @@ AdminClient.prototype.listGroups = function (options, cb) { cb(null, groups); } }); -} +}; /** * Describe consumer groups. @@ -340,7 +340,7 @@ AdminClient.prototype.describeGroups = function (groups, options, cb) { cb(null, descriptions); } }); -} +}; /** * Delete consumer groups. @@ -380,7 +380,7 @@ AdminClient.prototype.deleteGroups = function (groups, options, cb) { cb(null, reports); } }); -} +}; /** * List topics. @@ -425,7 +425,7 @@ AdminClient.prototype.listTopics = function (options, cb) { return; } - const topics = [] + const topics = []; if (metadata.topics) { for (const topic of metadata.topics) { topics.push(topic.name); @@ -436,4 +436,4 @@ AdminClient.prototype.listTopics = function (options, cb) { cb(null, topics); } }); -} +}; diff --git a/lib/client.js b/lib/client.js index 02f3702a..84f976cf 100644 --- a/lib/client.js +++ b/lib/client.js @@ -13,7 +13,6 @@ module.exports = Client; var Emitter = require('events').EventEmitter; var util = require('util'); var Kafka = require('../librdkafka.js'); -var assert = require('assert'); const { bindingVersion, dictToStringList } = require('./util'); @@ -74,12 +73,12 @@ function Client(globalConf, SubClientType, topicConf) { } } return obj2; - } + }; this._cb_configs = { global: extractFunctions(globalConf), topic: extractFunctions(topicConf), event: {}, - } + }; if (!no_event_cb) { this._cb_configs.event.event_cb = function(eventType, eventData) { @@ -146,7 +145,7 @@ function Client(globalConf, SubClientType, topicConf) { postProcessTokenRefresh(err); }); } - } + }; } this.metrics = {}; @@ -444,8 +443,6 @@ Client.prototype.queryWatermarkOffsets = function(topic, partition, timeout, cb) } } - var self = this; - if (typeof timeout === 'function') { cb = timeout; timeout = 1000; @@ -490,8 +487,6 @@ Client.prototype.offsetsForTimes = function(toppars, timeout, cb) { } } - var self = this; - if (typeof timeout === 'function') { cb = timeout; timeout = 1000; diff --git a/lib/error.js b/lib/error.js index b0fdb426..ccdc263e 100644 --- a/lib/error.js +++ b/lib/error.js @@ -446,9 +446,9 @@ function LibrdKafkaError(e) { } - if (e.hasOwnProperty('isFatal')) this.isFatal = e.isFatal; - if (e.hasOwnProperty('isRetriable')) this.isRetriable = e.isRetriable; - if (e.hasOwnProperty('isTxnRequiresAbort')) this.isTxnRequiresAbort = e.isTxnRequiresAbort; + if (Object.hasOwn(e, 'isFatal')) this.isFatal = e.isFatal; + if (Object.hasOwn(e, 'isRetriable')) this.isRetriable = e.isRetriable; + if (Object.hasOwn(e, 'isTxnRequiresAbort')) this.isTxnRequiresAbort = e.isTxnRequiresAbort; } diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 6efaa631..94d00d9e 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -227,7 +227,6 @@ KafkaConsumer.prototype.committed = function(toppars, timeout, cb) { toppars = toppars || this.assignments(); } - var self = this; this._client.committed(toppars, timeout, function(err, topicPartitions) { if (err) { cb(LibrdKafkaError.create(err)); @@ -264,7 +263,6 @@ KafkaConsumer.prototype.committed = function(toppars, timeout, cb) { * @return {Client} - Returns itself */ KafkaConsumer.prototype.seek = function(toppar, timeout, cb) { - var self = this; this._client.seek(TopicPartition.create(toppar), timeout, function(err) { if (err) { cb(LibrdKafkaError.create(err)); @@ -349,7 +347,7 @@ KafkaConsumer.prototype.assignments = function() { */ KafkaConsumer.prototype.rebalanceProtocol = function() { return this._client.rebalanceProtocol(); -} +}; /** * Subscribe to an array of topics (synchronously). @@ -451,7 +449,6 @@ KafkaConsumer.prototype.unsubscribe = function() { */ KafkaConsumer.prototype.consume = function(number, cb) { var timeoutMs = this._consumeTimeout !== undefined ? this._consumeTimeout : DEFAULT_CONSUME_TIME_OUT; - var self = this; if ((number && typeof number === 'number') || (number && cb)) { @@ -542,7 +539,7 @@ KafkaConsumer.prototype._consumeNum = function(timeoutMs, numMessages, cb) { function emitEofEventsFor(messageIndex) { while (currentEofEventsIndex < eofEvents.length && eofEvents[currentEofEventsIndex].messageIndex === messageIndex) { delete eofEvents[currentEofEventsIndex].messageIndex; - self.emit('partition.eof', eofEvents[currentEofEventsIndex]) + self.emit('partition.eof', eofEvents[currentEofEventsIndex]); ++currentEofEventsIndex; } } @@ -694,6 +691,26 @@ KafkaConsumer.prototype.offsetsStore = function(topicPartitions) { return this._errorWrap(this._client.offsetsStore(topicPartitions), true); }; +/** + * Store offset for a single topic partition. Do not use this method. + * This method is meant for internal use, and the API is not guaranteed to be stable. + * Use offsetsStore instead. + * + * @param {string} topic - Topic to store offset for. + * @param {number} partition - Partition of the provided topic to store offset for. + * @param {number} offset - Offset to store. + * @param {number} leaderEpoch - Leader epoch of the provided offset. + * @throws {LibrdKafkaError} - Throws when there is no offset stored + */ +KafkaConsumer.prototype._offsetsStoreSingle = function(topic, partition, offset, leaderEpoch) { + if (!this.isConnected()) { + throw new Error('Client is disconnected'); + } + + return this._errorWrap( + this._client.offsetsStoreSingle(topic, partition, offset, leaderEpoch), true); +}; + /** * Resume consumption for the provided list of partitions. * diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 1fe71467..bd8c15f8 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -157,7 +157,7 @@ class Admin { */ async disconnect() { /* Not yet connected - no error. */ - if (this.#state == AdminState.INIT) { + if (this.#state === AdminState.INIT) { return; } @@ -372,4 +372,4 @@ module.exports = { Admin, ConsumerGroupStates: RdKafka.AdminClient.ConsumerGroupStates, AclOperationTypes: RdKafka.AdminClient.AclOperationTypes -} +}; diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index e59513fc..4cc12bce 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -55,7 +55,7 @@ const kafkaJSProperties = { 'rebalanceListener', ], admin: [], -} +}; const logLevel = Object.freeze({ NOTHING: 0, @@ -149,7 +149,7 @@ function loggerTrampoline(msg, logger) { function createReplacementErrorMessage(cOrP, fnCall, property, propertyVal, replacementVal, isLK = false) { if (!isLK) { - replacementVal = `kafkaJS: { ${replacementVal}, ... }` + replacementVal = `kafkaJS: { ${replacementVal}, ... }`; } return `'${property}' is not supported as a property to '${fnCall}', but must be passed to the ${cOrP} during creation.\n` + `Before: \n` + @@ -319,7 +319,7 @@ function kafkaJSToRdKafkaConfig(config) { } throw err; }); - } + }; } /* It's a valid case (unlike in KafkaJS) for oauthBearerProvider to be * null, because librdkafka provides an unsecured token provider for @@ -633,7 +633,7 @@ class Lock { this[LockStates.TIMEOUT_ERROR_MESSAGE] = () => { const timeoutMessage = `Timeout while acquiring lock (${this[LockStates.WAITING].size} waiting locks)`; return description ? `${timeoutMessage}: "${description}"` : timeoutMessage; - } + }; } async acquire() { @@ -651,7 +651,7 @@ class Lock { this[LockStates.WAITING].delete(tryToAcquire); return resolve(); } - } + }; this[LockStates.WAITING].add(tryToAcquire); timeoutId = setTimeout(() => { @@ -660,7 +660,7 @@ class Lock { this[LockStates.WAITING].delete(tryToAcquire); reject(e); }, this[LockStates.TIMEOUT]); - }) + }); } async release() { diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index e6770e4a..2dbf2a77 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -599,7 +599,7 @@ class Consumer { let headers; if (message.headers) { - headers = {} + headers = {}; for (const header of message.headers) { for (const [key, value] of Object.entries(header)) { if (!Object.hasOwn(headers, key)) { @@ -662,12 +662,11 @@ class Consumer { try { if (!this.#userManagedStores) { - this.#internalClient.offsetsStore([{ + this.#internalClient._offsetsStoreSingle( topic, partition, - offset: offset + 1, - leaderEpoch: leaderEpoch, - }]); + offset + 1, + leaderEpoch); } this.#lastConsumedOffsets.set(key, offset + 1); } catch (e) { @@ -712,7 +711,7 @@ class Consumer { let headers; if (message.headers) { - headers = {} + headers = {}; for (const [key, value] of Object.entries(message.headers)) { if (!Object.hasOwn(headers, key)) { headers[key] = value; @@ -1073,9 +1072,7 @@ class Consumer { if (eachMessageProcessed) { try { if (!this.#userManagedStores) { - this.#internalClient.offsetsStore([{ - topic: m.topic, partition: m.partition, offset: Number(m.offset) + 1, leaderEpoch: m.leaderEpoch - }]); + this.#internalClient._offsetsStoreSingle(m.topic, m.partition, Number(m.offset) + 1, m.leaderEpoch); } this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); } catch (e) { @@ -1189,10 +1186,10 @@ class Consumer { while (!this.#workerTerminationScheduled) { /* Invalidate the message cache if needed */ const locallyStale = this.#messageCache.popLocallyStale(); - if (this.#messageCache.isStale()) { /* global staleness */ + if (this.#messageCache.isStale()) { this.#workerTerminationScheduled = true; break; - } else if (locallyStale.length !== 0) { /* local staleness */ + } else if (locallyStale.length !== 0) { // TODO: is it correct to await some concurrent promises for eachMessage here? // to be safe we can do it, but I don't think we really need to do that for // any correctness reason. @@ -1253,7 +1250,7 @@ class Consumer { })); /* Best we can do is log errors on worker issues - handled by the catch block above. */ - await Promise.allSettled(this.#workers) + await Promise.allSettled(this.#workers); /* One of the possible reasons for the workers to end is that the cache is globally stale. * We need to take care of expiring it. */ @@ -1504,7 +1501,7 @@ class Consumer { } this.#checkPendingSeeks = true; - const key = partitionKey(rdKafkaTopicPartitionOffset) + const key = partitionKey(rdKafkaTopicPartitionOffset); this.#pendingSeeks.set(key, rdKafkaTopicPartitionOffset.offset); /* Only for eachBatch: @@ -1672,7 +1669,7 @@ class Consumer { */ async disconnect() { /* Not yet connected - no error. */ - if (this.#state == ConsumerState.INIT) { + if (this.#state === ConsumerState.INIT) { return; } @@ -1704,10 +1701,10 @@ class Consumer { } this.#state = ConsumerState.DISCONNECTED; resolve(); - } + }; this.#internalClient.disconnect(cb); }); } } -module.exports = { Consumer, PartitionAssigners, } +module.exports = { Consumer, PartitionAssigners, }; diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 166487f4..06077250 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -263,7 +263,7 @@ class MessageCache { * Post-conditions: PPC is unstale, ppcList contains all caches with messages in them. */ #add(message) { - const key = partitionKey(message) + const key = partitionKey(message); const cache = this.tpToPpc.get(key); if (!cache) { this.logger.error("No cache found for message", message); @@ -299,7 +299,6 @@ class MessageCache { const key = partitionKey(tp); return this.tpToPpc.get(key).size() !== 0; })) { - logger.error('Locally stale caches should have been cleared before adding messages: ', this.locallyStaleCaches); throw new Error('Locally stale caches should have been cleared before adding messages.'); } @@ -314,7 +313,6 @@ class MessageCache { /* Reset the indices and pendingIndices because ppcList is being created newly. */ this.indices.clear(); - if (this.pendingIndices.size > 0) logger.error('addMessages: pendingIndices = ', this.pendingIndices, console.trace()); this.pendingIndices.clear(); this.maxIndicesIndex = Math.min(this.maxConcurrency, this.ppcList.length - 1); for (let i = 0; i <= this.maxIndicesIndex; i++) { @@ -393,7 +391,7 @@ class MessageCache { this.maxIndicesIndex = toAdd; index = toAdd; } else if (!this.indices.isEmpty()) { - index = this.indices.pop() + index = this.indices.pop(); } else { break; // nothing left. } @@ -452,7 +450,7 @@ class MessageCache { this.maxIndicesIndex = toAdd; index = toAdd; } else if (!this.indices.isEmpty()) { - index = this.indices.pop() + index = this.indices.pop(); } else { break; // nothing left. } @@ -475,7 +473,7 @@ class MessageCache { */ clear() { if (this.pendingSize() > 0) { - this.logger.error('clear: pendingIndices = ', this.pendingIndices, logger.trace()); + this.logger.error(`Cache cannot be cleared with ${this.pendingSize()} pending indices.`); throw new Error(`Cache cannot be cleared with ${this.pendingSize()} pending indices.`); } for (const cache of this.ppcList) { diff --git a/lib/kafkajs/_error.js b/lib/kafkajs/_error.js index 49a0f9b3..27584cbf 100644 --- a/lib/kafkajs/_error.js +++ b/lib/kafkajs/_error.js @@ -132,8 +132,8 @@ class KafkaJSTimeout extends KafkaJSError { class KafkaJSLockTimeout extends KafkaJSTimeout { constructor() { - super(...arguments) - this.name = 'KafkaJSLockTimeout' + super(...arguments); + this.name = 'KafkaJSLockTimeout'; } } diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 49cb0024..773a3186 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -200,8 +200,8 @@ class Producer { return topics.flatMap(topic => { return topic.partitions.map(partition => { return { partition: Number(partition.partition), offset: String(partition.offset), topic: String(topic.topic) }; - }) - }) + }); + }); } #readyTransactions(err) { @@ -319,7 +319,7 @@ class Producer { */ async disconnect() { /* Not yet connected - no error. */ - if (this.#state == ProducerState.INIT) { + if (this.#state === ProducerState.INIT) { return; } @@ -340,7 +340,7 @@ class Producer { } this.#state = ProducerState.DISCONNECTED; resolve(); - } + }; this.#internalClient.disconnect(5000 /* default timeout, 5000ms */, cb); }); } @@ -476,7 +476,7 @@ class Producer { reject(createKafkaJsErrorFromLibRdKafkaError(err)); else resolve(); - }) + }); }); } diff --git a/lib/producer-stream.js b/lib/producer-stream.js index b583eb7a..57051445 100644 --- a/lib/producer-stream.js +++ b/lib/producer-stream.js @@ -14,6 +14,7 @@ module.exports = ProducerStream; var Writable = require('stream').Writable; var util = require('util'); var ErrorCode = require('./error').codes; +var { Buffer } = require('buffer'); util.inherits(ProducerStream, Writable); @@ -78,8 +79,6 @@ function ProducerStream(producer, options) { this.connect(this.connectOptions); } - var self = this; - this.once('finish', function() { if (this.autoClose) { this.close(); @@ -89,7 +88,7 @@ function ProducerStream(producer, options) { } ProducerStream.prototype.connect = function(options) { - this.producer.connect(options, function(err, data) { + this.producer.connect(options, function(err) { if (err) { this.emit('error', err); return; @@ -258,13 +257,11 @@ ProducerStream.prototype._writev = function(data, cb) { var self = this; var len = data.length; var chunks = new Array(len); - var size = 0; for (var i = 0; i < len; i++) { var chunk = data[i].chunk; chunks[i] = chunk; - size += chunk.length; } writev(this.producer, this.topicName, chunks, function(err) { diff --git a/lib/producer.js b/lib/producer.js index 99b892ae..b89248fa 100644 --- a/lib/producer.js +++ b/lib/producer.js @@ -79,7 +79,7 @@ function Producer(conf, topicConf) { this.globalConfig = conf; this.topicConfig = topicConf; this.defaultTopic = gTopic || null; - this.defaultPartition = gPart == null ? -1 : gPart; + this.defaultPartition = gPart === null ? -1 : gPart; this.sentMessages = 0; @@ -134,7 +134,7 @@ Producer.prototype.produce = function(topic, partition, message, key, timestamp, this.sentMessages++; - partition = partition == null ? this.defaultPartition : partition; + partition = partition === null ? this.defaultPartition : partition; return this._errorWrap( this._client.produce(topic, partition, message, key, timestamp, opaque, headers)); @@ -214,7 +214,7 @@ Producer.prototype.setPollInterval = function(interval) { this.pollInterval = setInterval(function() { try { self.poll(); - } catch (e) { + } catch { // We can probably ignore errors here as far as broadcasting. // Disconnection issues will get handled below } @@ -245,7 +245,7 @@ Producer.prototype.setPollInBackground = function(set) { this.setPollInterval(0); // Clear poll interval from JS. } this._client.setPollInBackground(set); -} +}; /** * Flush the producer diff --git a/lib/producer/high-level-producer.js b/lib/producer/high-level-producer.js index 6d1ef959..673bb9e1 100644 --- a/lib/producer/high-level-producer.js +++ b/lib/producer/high-level-producer.js @@ -11,11 +11,9 @@ module.exports = HighLevelProducer; var util = require('util'); var Producer = require('../producer'); -var LibrdKafkaError = require('../error'); var EventEmitter = require('events').EventEmitter; var RefCounter = require('../tools/ref-counter'); var shallowCopy = require('../util').shallowCopy; -var isObject = require('../util').isObject; util.inherits(HighLevelProducer, Producer); @@ -109,7 +107,7 @@ function HighLevelProducer(conf, topicConf) { self._hl.pollingRefTimeout = setInterval(function() { try { self.poll(); - } catch (e) { + } catch { if (!self._isConnected) { // If we got disconnected for some reason there is no point // in polling anymore diff --git a/lib/rdkafka.js b/lib/rdkafka.js index 7a57e840..58d79b5d 100644 --- a/lib/rdkafka.js +++ b/lib/rdkafka.js @@ -31,4 +31,4 @@ module.exports = { Topic: Topic, features: features, librdkafkaVersion: lib.librdkafkaVersion, -} +}; diff --git a/lib/util.js b/lib/util.js index 3e3854c4..989f3f2f 100644 --- a/lib/util.js +++ b/lib/util.js @@ -17,7 +17,7 @@ util.shallowCopy = function (obj) { var copy = {}; for (var k in obj) { - if (obj.hasOwnProperty(k)) { + if (Object.hasOwn(obj, k)) { copy[k] = obj[k]; } } diff --git a/package-lock.json b/package-lock.json index 6970c390..f0173f89 100644 --- a/package-lock.json +++ b/package-lock.json @@ -15,13 +15,14 @@ "nan": "^2.17.0" }, "devDependencies": { + "@eslint/js": "^9.8.0", "@types/node": "^20.4.5", "bluebird": "^3.5.3", - "eslint": "8.54.0", + "eslint": "^8.57.0", + "eslint-plugin-jest": "^28.6.0", "jest": "^29.7.0", "jsdoc": "^4.0.2", - "jshint": "^2.13.6", - "mocha": "^10.2.0", + "mocha": "^10.7.0", "node-gyp": "^9.3.1", "typescript": "^5.1.6" }, @@ -29,15 +30,6 @@ "node": ">=18.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.3.0", "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.3.0.tgz", @@ -52,12 +44,12 @@ } }, "node_modules/@babel/code-frame": { - "version": "7.24.2", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.2.tgz", - "integrity": "sha512-y5+tLQyV8pg3fsiln67BVLD1P13Eg4lh5RW9mF0zUuvLrv9uIQ4MCL+CRT+FTsBlBjcIan6PGsLcBN0m3ClUyQ==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", + "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", "dev": true, "dependencies": { - "@babel/highlight": "^7.24.2", + "@babel/highlight": "^7.24.7", "picocolors": "^1.0.0" }, "engines": { @@ -65,30 +57,30 @@ } }, "node_modules/@babel/compat-data": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.24.1.tgz", - "integrity": "sha512-Pc65opHDliVpRHuKfzI+gSA4zcgr65O4cl64fFJIWEEh8JoHIHh0Oez1Eo8Arz8zq/JhgKodQaxEwUPRtZylVA==", + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.2.tgz", + "integrity": "sha512-bYcppcpKBvX4znYaPEeFau03bp89ShqNMLs+rmdptMw+heSZh9+z84d2YG+K7cYLbWwzdjtDoW/uqZmPjulClQ==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/core": { - "version": "7.24.3", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.24.3.tgz", - "integrity": "sha512-5FcvN1JHw2sHJChotgx8Ek0lyuh4kCKelgMTTqhYJJtloNvUfpAFMeNQUtdlIaktwrSV9LtCdqwk48wL2wBacQ==", + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.25.2.tgz", + "integrity": "sha512-BBt3opiCOxUr9euZ5/ro/Xv8/V7yJ5bjYMqG/C1YAo8MIKAnumZalCN+msbci3Pigy4lIQfPUpfMM27HMGaYEA==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.24.2", - "@babel/generator": "^7.24.1", - "@babel/helper-compilation-targets": "^7.23.6", - "@babel/helper-module-transforms": "^7.23.3", - "@babel/helpers": "^7.24.1", - "@babel/parser": "^7.24.1", - "@babel/template": "^7.24.0", - "@babel/traverse": "^7.24.1", - "@babel/types": "^7.24.0", + "@babel/code-frame": "^7.24.7", + "@babel/generator": "^7.25.0", + "@babel/helper-compilation-targets": "^7.25.2", + "@babel/helper-module-transforms": "^7.25.2", + "@babel/helpers": "^7.25.0", + "@babel/parser": "^7.25.0", + "@babel/template": "^7.25.0", + "@babel/traverse": "^7.25.2", + "@babel/types": "^7.25.2", "convert-source-map": "^2.0.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -113,12 +105,12 @@ } }, "node_modules/@babel/generator": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.24.1.tgz", - "integrity": "sha512-DfCRfZsBcrPEHUfuBMgbJ1Ut01Y/itOs+hY2nFLgqsqXd52/iSiVq5TITtUasIUgm+IIKdY2/1I7auiQOEeC9A==", + "version": "7.25.0", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.0.tgz", + "integrity": "sha512-3LEEcj3PVW8pW2R1SR1M89g/qrYk/m/mB/tLqn7dn4sbBUQyTqnlod+II2U4dqiGtUmkcnAmkMDralTFZttRiw==", "dev": true, "dependencies": { - "@babel/types": "^7.24.0", + "@babel/types": "^7.25.0", "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.25", "jsesc": "^2.5.1" @@ -128,14 +120,14 @@ } }, "node_modules/@babel/helper-compilation-targets": { - "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==", + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.25.2.tgz", + "integrity": "sha512-U2U5LsSaZ7TAt3cfaymQ8WHh0pxvdHoEk6HVpaexxixjyEquMh0L0YNJNM6CTGKMXV1iksi0iZkGw4AcFkPaaw==", "dev": true, "dependencies": { - "@babel/compat-data": "^7.23.5", - "@babel/helper-validator-option": "^7.23.5", - "browserslist": "^4.22.2", + "@babel/compat-data": "^7.25.2", + "@babel/helper-validator-option": "^7.24.8", + "browserslist": "^4.23.1", "lru-cache": "^5.1.1", "semver": "^6.3.1" }, @@ -152,63 +144,29 @@ "semver": "bin/semver.js" } }, - "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/@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": { - "@babel/template": "^7.22.15", - "@babel/types": "^7.23.0" - }, - "engines": { - "node": ">=6.9.0" - } - }, - "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": { - "@babel/types": "^7.22.5" - }, - "engines": { - "node": ">=6.9.0" - } - }, "node_modules/@babel/helper-module-imports": { - "version": "7.24.3", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.3.tgz", - "integrity": "sha512-viKb0F9f2s0BCS22QSF308z/+1YWKV/76mwt61NBzS5izMzDPwdq1pTrzf+Li3npBWX9KdQbkeCt1jSAM7lZqg==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", + "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", "dev": true, "dependencies": { - "@babel/types": "^7.24.0" + "@babel/traverse": "^7.24.7", + "@babel/types": "^7.24.7" }, "engines": { "node": ">=6.9.0" } }, "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==", + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.25.2.tgz", + "integrity": "sha512-BjyRAbix6j/wv83ftcVJmBt72QtHI56C7JXZoG2xATiLpmoC7dpd8WnkikExHDVPpi/3qCmO6WY1EaXOluiecQ==", "dev": true, "dependencies": { - "@babel/helper-environment-visitor": "^7.22.20", - "@babel/helper-module-imports": "^7.22.15", - "@babel/helper-simple-access": "^7.22.5", - "@babel/helper-split-export-declaration": "^7.22.6", - "@babel/helper-validator-identifier": "^7.22.20" + "@babel/helper-module-imports": "^7.24.7", + "@babel/helper-simple-access": "^7.24.7", + "@babel/helper-validator-identifier": "^7.24.7", + "@babel/traverse": "^7.25.2" }, "engines": { "node": ">=6.9.0" @@ -218,86 +176,74 @@ } }, "node_modules/@babel/helper-plugin-utils": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.0.tgz", - "integrity": "sha512-9cUznXMG0+FxRuJfvL82QlTqIzhVW9sL0KjMPHhAOOvpQGL8QtdxnBKILjBqxlHyliz0yCa1G903ZXI/FuHy2w==", + "version": "7.24.8", + "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.8.tgz", + "integrity": "sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==", "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==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz", + "integrity": "sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==", "dev": true, "dependencies": { - "@babel/types": "^7.22.5" - }, - "engines": { - "node": ">=6.9.0" - } - }, - "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": { - "@babel/types": "^7.22.5" + "@babel/traverse": "^7.24.7", + "@babel/types": "^7.24.7" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-string-parser": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.1.tgz", - "integrity": "sha512-2ofRCjnnA9y+wk8b9IAREroeUP02KHp431N2mhKniy2yKIDKpbrHv9eXwm8cBeWQYcJmzv5qKCu65P47eCF7CQ==", + "version": "7.24.8", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", + "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-identifier": { - "version": "7.22.20", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.20.tgz", - "integrity": "sha512-Y4OZ+ytlatR8AI+8KZfKuL5urKp7qey08ha31L8b3BwewJAoJamTzyvxPR/5D+KkdJCGPq/+8TukHBlY10FX9A==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", + "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", "dev": true, "engines": { "node": ">=6.9.0" } }, "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==", + "version": "7.24.8", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.24.8.tgz", + "integrity": "sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helpers": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.24.1.tgz", - "integrity": "sha512-BpU09QqEe6ZCHuIHFphEFgvNSrubve1FtyMton26ekZ85gRGi6LrTF7zArARp2YvyFxloeiRmtSCq5sjh1WqIg==", + "version": "7.25.0", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.0.tgz", + "integrity": "sha512-MjgLZ42aCm0oGjJj8CtSM3DB8NOOf8h2l7DCTePJs29u+v7yO/RBX9nShlKMgFnRks/Q4tBAe7Hxnov9VkGwLw==", "dev": true, "dependencies": { - "@babel/template": "^7.24.0", - "@babel/traverse": "^7.24.1", - "@babel/types": "^7.24.0" + "@babel/template": "^7.25.0", + "@babel/types": "^7.25.0" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/highlight": { - "version": "7.24.2", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.2.tgz", - "integrity": "sha512-Yac1ao4flkTxTteCDZLEvdxg2fZfz1v8M4QpaGypq/WPDqg3ijHYbDfs+LG5hvzSoqaSZ9/Z9lKSP3CjZjv+pA==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", + "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", "dev": true, "dependencies": { - "@babel/helper-validator-identifier": "^7.22.20", + "@babel/helper-validator-identifier": "^7.24.7", "chalk": "^2.4.2", "js-tokens": "^4.0.0", "picocolors": "^1.0.0" @@ -378,10 +324,13 @@ } }, "node_modules/@babel/parser": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.24.1.tgz", - "integrity": "sha512-Zo9c7N3xdOIQrNip7Lc9wvRPzlRtovHVE4lkz8WEDr7uYh/GMQhSiIgFxGIArRHYdJE5kxtZjAf8rT0xhdLCzg==", + "version": "7.25.3", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.3.tgz", + "integrity": "sha512-iLTJKDbJ4hMvFPgQwwsVoxtHyWpKKPBrxkANrSYewDPaPpT5py5yeVkgPIJ7XYXhndxJpaA3PyALSXQ7u8e/Dw==", "dev": true, + "dependencies": { + "@babel/types": "^7.25.2" + }, "bin": { "parser": "bin/babel-parser.js" }, @@ -450,12 +399,12 @@ } }, "node_modules/@babel/plugin-syntax-jsx": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.1.tgz", - "integrity": "sha512-2eCtxZXf+kbkMIsXS4poTvT4Yu5rXiRa+9xGVT56raghjmBTKMpFNc9R4IDiB4emao9eO22Ox7CxuJG7BgExqA==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz", + "integrity": "sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ==", "dev": true, "dependencies": { - "@babel/helper-plugin-utils": "^7.24.0" + "@babel/helper-plugin-utils": "^7.24.7" }, "engines": { "node": ">=6.9.0" @@ -552,12 +501,12 @@ } }, "node_modules/@babel/plugin-syntax-typescript": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.1.tgz", - "integrity": "sha512-Yhnmvy5HZEnHUty6i++gcfH1/l68AHnItFHnaCv6hn9dNh0hQvvQJsxpi4BMBFN5DLeHBuucT/0DgzXif/OyRw==", + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.7.tgz", + "integrity": "sha512-c/+fVeJBB0FeKsFvwytYiUD+LBvhHjGSI0g446PRGdSVGZLRNArBUno2PETbAly3tpiNAQR5XaZ+JslxkotsbA==", "dev": true, "dependencies": { - "@babel/helper-plugin-utils": "^7.24.0" + "@babel/helper-plugin-utils": "^7.24.7" }, "engines": { "node": ">=6.9.0" @@ -567,33 +516,30 @@ } }, "node_modules/@babel/template": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.24.0.tgz", - "integrity": "sha512-Bkf2q8lMB0AFpX0NFEqSbx1OkTHf0f+0j82mkw+ZpzBnkk7e9Ql0891vlfgi+kHwOk8tQjiQHpqh4LaSa0fKEA==", + "version": "7.25.0", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", + "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", "dev": true, "dependencies": { - "@babel/code-frame": "^7.23.5", - "@babel/parser": "^7.24.0", - "@babel/types": "^7.24.0" + "@babel/code-frame": "^7.24.7", + "@babel/parser": "^7.25.0", + "@babel/types": "^7.25.0" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/traverse": { - "version": "7.24.1", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.24.1.tgz", - "integrity": "sha512-xuU6o9m68KeqZbQuDt2TcKSxUw/mrsvavlEqQ1leZ/B+C9tk6E4sRWy97WaXgvq5E+nU3cXMxv3WKOCanVMCmQ==", - "dev": true, - "dependencies": { - "@babel/code-frame": "^7.24.1", - "@babel/generator": "^7.24.1", - "@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.24.1", - "@babel/types": "^7.24.0", + "version": "7.25.3", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.3.tgz", + "integrity": "sha512-HefgyP1x754oGCsKmV5reSmtV7IXj/kpaE1XYY+D9G5PvKKoFfSbiS4M77MdjuwlZKDIKFCffq9rPU+H/s3ZdQ==", + "dev": true, + "dependencies": { + "@babel/code-frame": "^7.24.7", + "@babel/generator": "^7.25.0", + "@babel/parser": "^7.25.3", + "@babel/template": "^7.25.0", + "@babel/types": "^7.25.2", "debug": "^4.3.1", "globals": "^11.1.0" }, @@ -611,13 +557,13 @@ } }, "node_modules/@babel/types": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.24.0.tgz", - "integrity": "sha512-+j7a5c253RfKh8iABBhywc8NSfP5LURe7Uh4qpsh6jc+aLJguvmIUBdjSdEMQv2bENrCR5MfRdjGo7vzS/ob7w==", + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.2.tgz", + "integrity": "sha512-YTnYtra7W9e6/oAZEHj0bJehPRUlLH9/fbpT5LfB0NhQXyALCRkRs3zH9v07IYhkgpqX6Z78FnuccZr/l4Fs4Q==", "dev": true, "dependencies": { - "@babel/helper-string-parser": "^7.23.4", - "@babel/helper-validator-identifier": "^7.22.20", + "@babel/helper-string-parser": "^7.24.8", + "@babel/helper-validator-identifier": "^7.24.7", "to-fast-properties": "^2.0.0" }, "engines": { @@ -646,9 +592,9 @@ } }, "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==", + "version": "4.11.0", + "resolved": "https://registry.npmjs.org/@eslint-community/regexpp/-/regexpp-4.11.0.tgz", + "integrity": "sha512-G/M/tIiMrTAxEWRfLfQJMmGNX28IxBg4PBz8XqQhqUHLFI6TL2htpIB1iQCj144V5ee/JaKyT9/WZ0MGZWfA7A==", "dev": true, "engines": { "node": "^12.0.0 || ^14.0.0 || >=16.0.0" @@ -678,12 +624,12 @@ } }, "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==", + "version": "9.8.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.8.0.tgz", + "integrity": "sha512-MfluB7EUfxXtv3i/++oh89uzAr4PDI4nn201hsp+qaXqsjAWzinlZEHEfPgAX4doIlKvPG/i0A9dpKxOLII8yA==", "dev": true, "engines": { - "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" } }, "node_modules/@gar/promisify": { @@ -696,6 +642,7 @@ "version": "0.11.14", "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.14.tgz", "integrity": "sha512-3T8LkOmg45BV5FICb15QQMsyUSWrQ8AygVfC7ZG32zOalnqrilm018ZVCw0eapXux8FtA33q8PSRSstjee3jSg==", + "deprecated": "Use @eslint/config-array instead", "dev": true, "dependencies": { "@humanwhocodes/object-schema": "^2.0.2", @@ -720,9 +667,10 @@ } }, "node_modules/@humanwhocodes/object-schema": { - "version": "2.0.2", - "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.2.tgz", - "integrity": "sha512-6EwiSjwWYP7pTckG6I5eyFANjPhmPjUX9JRLUSfNPC7FX7zK9gyZAfUEaECL6ALTpGX5AjnBq3C9XmVWPitNpw==", + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.3.tgz", + "integrity": "sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA==", + "deprecated": "Use @eslint/object-schema instead", "dev": true }, "node_modules/@istanbuljs/load-nyc-config": { @@ -1150,9 +1098,9 @@ } }, "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==", + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", "dev": true }, "node_modules/@jridgewell/trace-mapping": { @@ -1166,9 +1114,9 @@ } }, "node_modules/@jsdoc/salty": { - "version": "0.2.7", - "resolved": "https://registry.npmjs.org/@jsdoc/salty/-/salty-0.2.7.tgz", - "integrity": "sha512-mh8LbS9d4Jq84KLw8pzho7XC2q2/IJGiJss3xwRoLD1A+EE16SjN4PfaG4jRCzKegTFLlN0Zd8SdUPE6XdoPFg==", + "version": "0.2.8", + "resolved": "https://registry.npmjs.org/@jsdoc/salty/-/salty-0.2.8.tgz", + "integrity": "sha512-5e+SFVavj1ORKlKaKr2BmTOekmXbelU7dC0cDkQLqag7xfuTPuGMUFx7KWJuv4bYZrTsoL2Z18VVCOKYxzoHcg==", "dev": true, "dependencies": { "lodash": "^4.17.21" @@ -1324,9 +1272,9 @@ } }, "node_modules/@types/babel__traverse": { - "version": "7.20.5", - "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.5.tgz", - "integrity": "sha512-WXCyOcRtH37HAUkpXhUduaxdm82b4GSlyTqajXviN4EfiuPgNYR109xMCKvpl6zPIpua0DGlMEDCq+g8EdoheQ==", + "version": "7.20.6", + "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.6.tgz", + "integrity": "sha512-r1bzfrm0tomOI8g1SzvCaQHo6Lcv6zu0EA+W2kHrt8dyrHQxGzBBL4kdkzIS+jBMV+EYcMAEAqXqYaLJq5rOZg==", "dev": true, "dependencies": { "@babel/types": "^7.20.7" @@ -1366,31 +1314,31 @@ } }, "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==", + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/@types/linkify-it/-/linkify-it-5.0.0.tgz", + "integrity": "sha512-sVDA58zAw4eWAffKOaQH5/5j3XeayukzDk+ewSsnv3p4yJEZHCCzMDiZM8e0OUrRvmpGZ85jf4yDHkHsgBNr9Q==", "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==", + "version": "14.1.2", + "resolved": "https://registry.npmjs.org/@types/markdown-it/-/markdown-it-14.1.2.tgz", + "integrity": "sha512-promo4eFwuiW+TfGxhi+0x3czqTYJkG8qB17ZUJiVF10Xm7NLVRSLUsfRTU/6h1e24VvRnXCx+hG7li58lkzog==", "dev": true, "dependencies": { - "@types/linkify-it": "*", - "@types/mdurl": "*" + "@types/linkify-it": "^5", + "@types/mdurl": "^2" } }, "node_modules/@types/mdurl": { - "version": "1.0.5", - "resolved": "https://registry.npmjs.org/@types/mdurl/-/mdurl-1.0.5.tgz", - "integrity": "sha512-6L6VymKTzYSrEf4Nev4Xa1LCHKrlTlYCBMTlQKFuddo1CvQcE52I0mwfOJayueUC7MJuXOeHTcIU683lzd0cUA==", + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@types/mdurl/-/mdurl-2.0.0.tgz", + "integrity": "sha512-RGdgjQUZba5p6QEFAVx2OGb8rQDL/cPRG7GiedRzMcJ1tYnUANBncjbSB1NRGwbvjcPeikRABz2nshyPk1bhWg==", "dev": true }, "node_modules/@types/node": { - "version": "20.11.30", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.30.tgz", - "integrity": "sha512-dHM6ZxwlmuZaRmUPfv1p+KrdD1Dci04FbdEm/9wEMouFqxYoFl5aMkt0VMAUtYRQDyYvD41WJLukhq/ha3YuTw==", + "version": "20.14.14", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.14.14.tgz", + "integrity": "sha512-d64f00982fS9YoOgJkAMolK7MN8Iq3TDdVjchbYHdEmjth/DHowx82GnoA+tVUAN+7vxfYUgAzi+JXbKNd2SDQ==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -1417,6 +1365,127 @@ "integrity": "sha512-I4q9QU9MQv4oEOz4tAHJtNz1cwuLxn2F3xcc2iV5WdqLPpUnj30aUuxt1mAxYTG+oe8CZMV/+6rU4S4gRDzqtQ==", "dev": true }, + "node_modules/@typescript-eslint/scope-manager": { + "version": "7.18.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.18.0.tgz", + "integrity": "sha512-jjhdIE/FPF2B7Z1uzc6i3oWKbGcHb87Qw7AWj6jmEqNOfDFbJWtjt/XfwCpvNkpGWlcJaog5vTR+VV8+w9JflA==", + "dev": true, + "dependencies": { + "@typescript-eslint/types": "7.18.0", + "@typescript-eslint/visitor-keys": "7.18.0" + }, + "engines": { + "node": "^18.18.0 || >=20.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/types": { + "version": "7.18.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.18.0.tgz", + "integrity": "sha512-iZqi+Ds1y4EDYUtlOOC+aUmxnE9xS/yCigkjA7XpTKV6nCBd3Hp/PRGGmdwnfkV2ThMyYldP1wRpm/id99spTQ==", + "dev": true, + "engines": { + "node": "^18.18.0 || >=20.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/typescript-estree": { + "version": "7.18.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.18.0.tgz", + "integrity": "sha512-aP1v/BSPnnyhMHts8cf1qQ6Q1IFwwRvAQGRvBFkWlo3/lH29OXA3Pts+c10nxRxIBrDnoMqzhgdwVe5f2D6OzA==", + "dev": true, + "dependencies": { + "@typescript-eslint/types": "7.18.0", + "@typescript-eslint/visitor-keys": "7.18.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "^9.0.4", + "semver": "^7.6.0", + "ts-api-utils": "^1.3.0" + }, + "engines": { + "node": "^18.18.0 || >=20.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/typescript-estree/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/@typescript-eslint/typescript-estree/node_modules/minimatch": { + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", + "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", + "dev": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/@typescript-eslint/utils": { + "version": "7.18.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-7.18.0.tgz", + "integrity": "sha512-kK0/rNa2j74XuHVcoCZxdFBMF+aq/vH83CXAOHieC+2Gis4mF8jJXT5eAfyD3K0sAxtPuwxaIOIOvhwzVDt/kw==", + "dev": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@typescript-eslint/scope-manager": "7.18.0", + "@typescript-eslint/types": "7.18.0", + "@typescript-eslint/typescript-estree": "7.18.0" + }, + "engines": { + "node": "^18.18.0 || >=20.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + } + }, + "node_modules/@typescript-eslint/visitor-keys": { + "version": "7.18.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.18.0.tgz", + "integrity": "sha512-cDF0/Gf81QpY3xYyJKDV14Zwdmid5+uuENhjH2EqFaF0ni+yAyq/LzMaIJdhNJXZI7uLzwIlA+V7oWoyn6Curg==", + "dev": true, + "dependencies": { + "@typescript-eslint/types": "7.18.0", + "eslint-visitor-keys": "^3.4.3" + }, + "engines": { + "node": "^18.18.0 || >=20.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, "node_modules/@ungap/structured-clone": { "version": "1.2.0", "resolved": "https://registry.npmjs.org/@ungap/structured-clone/-/structured-clone-1.2.0.tgz", @@ -1429,9 +1498,9 @@ "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" }, "node_modules/acorn": { - "version": "8.11.3", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.11.3.tgz", - "integrity": "sha512-Y9rRfJG5jcKOE0CLisYbojUjIrIEE7AGMzA/Sm4BslANhbS+cDMpgBdcPT91oJ7OuJ9hYJBx59RjbhxVnrF8Xg==", + "version": "8.12.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", + "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true, "bin": { "acorn": "bin/acorn" @@ -1502,9 +1571,9 @@ } }, "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==", + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.3.tgz", + "integrity": "sha512-/6w/C21Pm1A7aZitlI5Ni/2J6FFQN8i1Cvz3kHABAAbw93v/NlvKdVOqz7CCWz/3iv/JplRSEEZ83XION15ovw==", "dev": true, "engines": { "node": ">=6" @@ -1582,6 +1651,7 @@ "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==", + "deprecated": "This package is no longer supported.", "dependencies": { "delegates": "^1.0.0", "readable-stream": "^3.6.0" @@ -1590,33 +1660,21 @@ "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": "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/array-union": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/array-union/-/array-union-2.1.0.tgz", + "integrity": "sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/babel-jest": { "version": "29.7.0", "resolved": "https://registry.npmjs.org/babel-jest/-/babel-jest-29.7.0.tgz", @@ -1774,12 +1832,12 @@ } }, "node_modules/braces": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", - "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", + "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", "dev": true, "dependencies": { - "fill-range": "^7.0.1" + "fill-range": "^7.1.1" }, "engines": { "node": ">=8" @@ -1792,9 +1850,9 @@ "dev": true }, "node_modules/browserslist": { - "version": "4.23.0", - "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.0.tgz", - "integrity": "sha512-QW8HiM1shhT2GuzkvklfjcKDiWFXHOeFCIA/huJPwHsslwcydgk7X+z2zXpEijP98UCY7HbubZt5J2Zgvf0CaQ==", + "version": "4.23.3", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.3.tgz", + "integrity": "sha512-btwCFJVjI4YWDNfau8RhZ+B1Q/VLoUITrm3RlP6y1tYGWIOa+InuYiRGXUBXo8nA1qKmHMyLB/iVQg5TT4eFoA==", "dev": true, "funding": [ { @@ -1811,10 +1869,10 @@ } ], "dependencies": { - "caniuse-lite": "^1.0.30001587", - "electron-to-chromium": "^1.4.668", - "node-releases": "^2.0.14", - "update-browserslist-db": "^1.0.13" + "caniuse-lite": "^1.0.30001646", + "electron-to-chromium": "^1.5.4", + "node-releases": "^2.0.18", + "update-browserslist-db": "^1.1.0" }, "bin": { "browserslist": "cli.js" @@ -1880,6 +1938,7 @@ "version": "8.1.0", "resolved": "https://registry.npmjs.org/glob/-/glob-8.1.0.tgz", "integrity": "sha512-r8hpEjiQEYlF2QU0df3dS+nxxSIreXQS1qRhMJM0Q5NDdR386C7jb7Hwwod8Fgiuex+k0GFjgft18yvxm5XoCQ==", + "deprecated": "Glob versions prior to v9 are no longer supported", "dev": true, "dependencies": { "fs.realpath": "^1.0.0", @@ -1935,9 +1994,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001600", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001600.tgz", - "integrity": "sha512-+2S9/2JFhYmYaDpZvo0lKkfvuKIglrx68MwOBqMGHhQsNkLjB5xtc/TGoEPs+MxjSyN/72qer2g97nzR641mOQ==", + "version": "1.0.30001646", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001646.tgz", + "integrity": "sha512-dRg00gudiBDDTmUhClSdv3hqRfpbOnU28IpI1T6PBTLWa+kOj0681C8uML3PifYfREuBrVjDGhL3adYpBT6spw==", "dev": true, "funding": [ { @@ -1992,16 +2051,10 @@ } }, "node_modules/chokidar": { - "version": "3.5.3", - "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", - "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.6.0.tgz", + "integrity": "sha512-7VT13fmjotKpGipCW9JEQAusEPE+Ei8nl6/g4FBAmIm0GOOLMua9NDDo/DWp0ZAxCr3cPq5ZpBqmPAQgDda2Pw==", "dev": true, - "funding": [ - { - "type": "individual", - "url": "https://paulmillr.com/funding/" - } - ], "dependencies": { "anymatch": "~3.1.2", "braces": "~3.0.2", @@ -2014,6 +2067,9 @@ "engines": { "node": ">= 8.10.0" }, + "funding": { + "url": "https://paulmillr.com/funding/" + }, "optionalDependencies": { "fsevents": "~2.3.2" } @@ -2054,9 +2110,9 @@ } }, "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==", + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.3.1.tgz", + "integrity": "sha512-a3KdPAANPbNE4ZUv9h6LckSl9zLsYOP4MBmhIPkRaeyybt+r4UghLvq+xw/YwUcC1gqylCkL4rdVs3Lwupjm4Q==", "dev": true }, "node_modules/clean-stack": { @@ -2068,19 +2124,6 @@ "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", @@ -2142,15 +2185,6 @@ "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", @@ -2162,12 +2196,6 @@ "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", @@ -2203,16 +2231,10 @@ "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==", + "version": "4.3.6", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.6.tgz", + "integrity": "sha512-O/09Bd4Z1fBrU4VzkhFqVgpPzaGbw6Sm9FEkBT1A/YBXQFGuuSxa1dN2nxgxS34JmKXqYx8CZAwEVoJFImUXIg==", "dependencies": { "ms": "2.1.2" }, @@ -2238,9 +2260,9 @@ } }, "node_modules/dedent": { - "version": "1.5.1", - "resolved": "https://registry.npmjs.org/dedent/-/dedent-1.5.1.tgz", - "integrity": "sha512-+LxW+KLWxu3HW3M2w2ympwtqPrqYRzU8fqi6Fhd18fBALe15blJPI/I4+UHveMVG6lJqB4JNd4UG0S5cnVHwIg==", + "version": "1.5.3", + "resolved": "https://registry.npmjs.org/dedent/-/dedent-1.5.3.tgz", + "integrity": "sha512-NHQtfOOW68WD8lgypbLA5oT+Bt0xXJhiYvoR6SmmNXZfpzOGXwdKWmcwG8N7PwVVWV3eF/68nmD9BaJSsTBhyQ==", "dev": true, "peerDependencies": { "babel-plugin-macros": "^3.1.0" @@ -2289,9 +2311,9 @@ } }, "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==", + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/diff/-/diff-5.2.0.tgz", + "integrity": "sha512-uIFDxqpRZGZ6ThOk84hEfqWoHx2devRFvpTZcTHur85vImfaxUbTW9Ryh4CpCuDnToOP1CEtXKIgytHBPVff5A==", "dev": true, "engines": { "node": ">=0.3.1" @@ -2306,6 +2328,18 @@ "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, + "node_modules/dir-glob": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-3.0.1.tgz", + "integrity": "sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA==", + "dev": true, + "dependencies": { + "path-type": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/doctrine": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", @@ -2318,66 +2352,10 @@ "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", - "integrity": "sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g==", - "dev": true, - "dependencies": { - "domelementtype": "^2.0.1", - "entities": "^2.0.0" - } - }, - "node_modules/dom-serializer/node_modules/domelementtype": { - "version": "2.3.0", - "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" - } - ] - }, - "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.716", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.716.tgz", - "integrity": "sha512-t/MXMzFKQC3UfMDpw7V5wdB/UAB8dWx4hEsy+fpPYJWW3gqh3u5T1uXp6vR+H6dGCPBxkRo+YBcapBLvbGQHRw==", + "version": "1.5.4", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.4.tgz", + "integrity": "sha512-orzA81VqLyIGUEA77YkVA1D+N+nNfl2isJVjjmOyrlxuooZ19ynb+dOlaDTqd/idKRS9lDCSBmtzM+kyCsMnkA==", "dev": true }, "node_modules/emittery": { @@ -2407,10 +2385,16 @@ } }, "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 + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/entities/-/entities-4.5.0.tgz", + "integrity": "sha512-V0hjH4dGPh9Ao5p0MoRY6BVqtwCjhz6vI5LT8AJ55H+4g9/4vbHx1I54fS0XuclLhDHArPQCiMjDxjaL8fPxhw==", + "dev": true, + "engines": { + "node": ">=0.12" + }, + "funding": { + "url": "https://github.com/fb55/entities?sponsor=1" + } }, "node_modules/env-paths": { "version": "2.2.1", @@ -2458,16 +2442,16 @@ } }, "node_modules/eslint": { - "version": "8.54.0", - "resolved": "https://registry.npmjs.org/eslint/-/eslint-8.54.0.tgz", - "integrity": "sha512-NY0DfAkM8BIZDVl6PgSa1ttZbx3xHgJzSNJKYcQglem6CppHyMhRIQkBVSSMaSRnLhig3jsDbEzOjwCVt4AmmA==", + "version": "8.57.0", + "resolved": "https://registry.npmjs.org/eslint/-/eslint-8.57.0.tgz", + "integrity": "sha512-dZ6+mexnaTIbSBZWgou51U6OmzIhYM2VcNdtiTtI7qPNZm35Akpr0f6vtw3w1Kmn5PYo+tZVfh13WrhpS6oLqQ==", "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", + "@eslint/eslintrc": "^2.1.4", + "@eslint/js": "8.57.0", + "@humanwhocodes/config-array": "^0.11.14", "@humanwhocodes/module-importer": "^1.0.1", "@nodelib/fs.walk": "^1.2.8", "@ungap/structured-clone": "^1.2.0", @@ -2512,6 +2496,31 @@ "url": "https://opencollective.com/eslint" } }, + "node_modules/eslint-plugin-jest": { + "version": "28.6.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-jest/-/eslint-plugin-jest-28.6.0.tgz", + "integrity": "sha512-YG28E1/MIKwnz+e2H7VwYPzHUYU4aMa19w0yGcwXnnmJH6EfgHahTJ2un3IyraUxNfnz/KUhJAFXNNwWPo12tg==", + "dev": true, + "dependencies": { + "@typescript-eslint/utils": "^6.0.0 || ^7.0.0" + }, + "engines": { + "node": "^16.10.0 || ^18.12.0 || >=20.0.0" + }, + "peerDependencies": { + "@typescript-eslint/eslint-plugin": "^6.0.0 || ^7.0.0", + "eslint": "^7.0.0 || ^8.0.0 || ^9.0.0", + "jest": "*" + }, + "peerDependenciesMeta": { + "@typescript-eslint/eslint-plugin": { + "optional": true + }, + "jest": { + "optional": true + } + } + }, "node_modules/eslint-scope": { "version": "7.2.2", "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-7.2.2.tgz", @@ -2540,6 +2549,15 @@ "url": "https://opencollective.com/eslint" } }, + "node_modules/eslint/node_modules/@eslint/js": { + "version": "8.57.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-8.57.0.tgz", + "integrity": "sha512-Ys+3g2TaW7gADOJzPt83SJtCDhMjndcDMFVQ/Tj9iA1BfJzFKD9mAUXT3OenpuPHbI6P/myECxRJrofUsDx/5g==", + "dev": true, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + } + }, "node_modules/espree": { "version": "9.6.1", "resolved": "https://registry.npmjs.org/espree/-/espree-9.6.1.tgz", @@ -2571,9 +2589,9 @@ } }, "node_modules/esquery": { - "version": "1.5.0", - "resolved": "https://registry.npmjs.org/esquery/-/esquery-1.5.0.tgz", - "integrity": "sha512-YQLXUplAwJgCydQ78IMJywZCceoqk1oH01OERdSAJc/7U2AylwjhSCLDEtqwg811idIS/9fIU5GjG73IgjKMVg==", + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/esquery/-/esquery-1.6.0.tgz", + "integrity": "sha512-ca9pw9fomFcKPvFLXhBKUK90ZvGibiGOvRJNbjljY7s7uq/5YO4BOzcYtJqExdx99rF6aAcnRxHmcUHcz6sQsg==", "dev": true, "dependencies": { "estraverse": "^5.1.0" @@ -2672,6 +2690,34 @@ "integrity": "sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q==", "dev": true }, + "node_modules/fast-glob": { + "version": "3.3.2", + "resolved": "https://registry.npmjs.org/fast-glob/-/fast-glob-3.3.2.tgz", + "integrity": "sha512-oX2ruAFQwf/Orj8m737Y5adxDQO0LAB7/S5MnxCdTNDd4p6BsyIVsv9JQsATbTSq8KHRpLwIHbVlUNatxd+1Ow==", + "dev": true, + "dependencies": { + "@nodelib/fs.stat": "^2.0.2", + "@nodelib/fs.walk": "^1.2.3", + "glob-parent": "^5.1.2", + "merge2": "^1.3.0", + "micromatch": "^4.0.4" + }, + "engines": { + "node": ">=8.6.0" + } + }, + "node_modules/fast-glob/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/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", @@ -2720,9 +2766,9 @@ "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==", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", + "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", "dev": true, "dependencies": { "to-regex-range": "^5.0.1" @@ -2792,6 +2838,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", @@ -2805,6 +2865,7 @@ "version": "3.0.2", "resolved": "https://registry.npmjs.org/gauge/-/gauge-3.0.2.tgz", "integrity": "sha512-+5J6MS/5XksCuXq++uFRsnUd7Ovu1XenbeuIuNRJxYWjgQbPuFhT14lAvsWfqfAmnwluf1OwMjz39HjfLPci0Q==", + "deprecated": "This package is no longer supported.", "dependencies": { "aproba": "^1.0.3 || ^2.0.0", "color-support": "^1.1.2", @@ -2863,6 +2924,7 @@ "version": "7.2.3", "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz", "integrity": "sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q==", + "deprecated": "Glob versions prior to v9 are no longer supported", "dependencies": { "fs.realpath": "^1.0.0", "inflight": "^1.0.4", @@ -2905,6 +2967,26 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/globby": { + "version": "11.1.0", + "resolved": "https://registry.npmjs.org/globby/-/globby-11.1.0.tgz", + "integrity": "sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g==", + "dev": true, + "dependencies": { + "array-union": "^2.1.0", + "dir-glob": "^3.0.1", + "fast-glob": "^3.2.9", + "ignore": "^5.2.0", + "merge2": "^1.4.1", + "slash": "^3.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/graceful-fs": { "version": "4.2.11", "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.11.tgz", @@ -2958,19 +3040,6 @@ "integrity": "sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg==", "dev": true }, - "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", - "domhandler": "2.3", - "domutils": "1.5", - "entities": "1.0", - "readable-stream": "1.1" - } - }, "node_modules/http-cache-semantics": { "version": "4.1.1", "resolved": "https://registry.npmjs.org/http-cache-semantics/-/http-cache-semantics-4.1.1.tgz", @@ -3059,9 +3128,9 @@ } }, "node_modules/import-local": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.1.0.tgz", - "integrity": "sha512-ASB07uLtnDs1o6EHjKpX34BKYDSqnFerfTOJL2HvMqF70LnxpjkzDB8J44oT9pu4AMPkQwf8jl6szgvNd2tRIg==", + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.2.0.tgz", + "integrity": "sha512-2SPlun1JUPWoM6t3F0dw0FkCF/jWY8kttcY4f599GLTSjh2OCuuhdTkJQsEcZzBqbXZGKMK2OqW1oZsjtf/gQA==", "dev": true, "dependencies": { "pkg-dir": "^4.2.0", @@ -3105,6 +3174,7 @@ "version": "1.0.6", "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", "integrity": "sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==", + "deprecated": "This module is not supported, and leaks memory. Do not use it. Check out lru-cache if you want a good and tested way to coalesce async requests by a key value, which is much more comprehensive and powerful.", "dependencies": { "once": "^1.3.0", "wrappy": "1" @@ -3147,12 +3217,15 @@ } }, "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==", + "version": "2.15.0", + "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.15.0.tgz", + "integrity": "sha512-Dd+Lb2/zvk9SKy1TGCt1wFJFo/MWBPMX5x7KcvLajWTGuomczdQX61PvY5yK6SVACwpoexWo81IfFyoKY2QnTA==", "dev": true, "dependencies": { - "hasown": "^2.0.0" + "hasown": "^2.0.2" + }, + "engines": { + "node": ">= 0.4" }, "funding": { "url": "https://github.com/sponsors/ljharb" @@ -3253,12 +3326,6 @@ "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/isexe": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", @@ -3275,9 +3342,9 @@ } }, "node_modules/istanbul-lib-instrument": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.2.tgz", - "integrity": "sha512-1WUsZ9R1lA0HtBSohTkm39WTPlNKSJ5iFk7UwqXkBLoHQT+hfqPsfsTDVuZdKGaBwn7din9bS7SsnoAr943hvw==", + "version": "6.0.3", + "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.3.tgz", + "integrity": "sha512-Vtgk7L/R2JHyyGW07spoFlB8/lpjiOLTjMdms6AFMraYt3BaJauod/NGrfnVG/y4Ix1JEuMRPDPEj2ua+zz1/Q==", "dev": true, "dependencies": { "@babel/core": "^7.23.9", @@ -3935,21 +4002,21 @@ "dev": true }, "node_modules/jsdoc": { - "version": "4.0.2", - "resolved": "https://registry.npmjs.org/jsdoc/-/jsdoc-4.0.2.tgz", - "integrity": "sha512-e8cIg2z62InH7azBBi3EsSEqrKx+nUtAS5bBcYTSpZFA+vhNPyhv8PTFZ0WsjOPDj04/dOLlm08EDcQJDqaGQg==", + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/jsdoc/-/jsdoc-4.0.3.tgz", + "integrity": "sha512-Nu7Sf35kXJ1MWDZIMAuATRQTg1iIPdzh7tqJ6jjvaU/GfDf+qi5UV8zJR3Mo+/pYFvm8mzay4+6O5EWigaQBQw==", "dev": true, "dependencies": { "@babel/parser": "^7.20.15", "@jsdoc/salty": "^0.2.1", - "@types/markdown-it": "^12.2.3", + "@types/markdown-it": "^14.1.1", "bluebird": "^3.7.2", "catharsis": "^0.9.0", "escape-string-regexp": "^2.0.0", "js2xmlparser": "^4.0.2", "klaw": "^3.0.0", - "markdown-it": "^12.3.2", - "markdown-it-anchor": "^8.4.1", + "markdown-it": "^14.1.0", + "markdown-it-anchor": "^8.6.7", "marked": "^4.0.10", "mkdirp": "^1.0.4", "requizzle": "^0.2.3", @@ -3984,48 +4051,6 @@ "node": ">=4" } }, - "node_modules/jshint": { - "version": "2.13.6", - "resolved": "https://registry.npmjs.org/jshint/-/jshint-2.13.6.tgz", - "integrity": "sha512-IVdB4G0NTTeQZrBoM8C5JFVLjV2KtZ9APgybDA1MK73xb09qFs0jCXyQLnCOp1cSZZZbvhq/6mfXHUTaDkffuQ==", - "dev": true, - "dependencies": { - "cli": "~1.0.0", - "console-browserify": "1.1.x", - "exit": "0.1.x", - "htmlparser2": "3.8.x", - "lodash": "~4.17.21", - "minimatch": "~3.0.2", - "strip-json-comments": "1.0.x" - }, - "bin": { - "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", - "integrity": "sha512-AOPG8EBc5wAikaG1/7uFCNFJwnKOuQwFTpYBdTW6OvWHeZBQBrAA/amefHGrEiOnCPcLFZK6FUPtWVKpQVIRgg==", - "dev": true, - "bin": { - "strip-json-comments": "cli.js" - }, - "engines": { - "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", @@ -4118,12 +4143,12 @@ "dev": true }, "node_modules/linkify-it": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/linkify-it/-/linkify-it-3.0.3.tgz", - "integrity": "sha512-ynTsyrFSdE5oZ/O9GEf00kPngmOfVwazR5GKDq6EYfhlpFug3J2zybX56a2PRRpc9P+FuSoGNAwjlbDs9jJBPQ==", + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/linkify-it/-/linkify-it-5.0.0.tgz", + "integrity": "sha512-5aHCbzQRADcdP+ATqnDuhhJ/MRIqDkZX5pyjFHRRysS8vZ5AbqGEoFIb6pYHPZ+L/OC2Lc+xT8uHVVR5CAK/wQ==", "dev": true, "dependencies": { - "uc.micro": "^1.0.1" + "uc.micro": "^2.0.0" } }, "node_modules/locate-path": { @@ -4246,19 +4271,20 @@ } }, "node_modules/markdown-it": { - "version": "12.3.2", - "resolved": "https://registry.npmjs.org/markdown-it/-/markdown-it-12.3.2.tgz", - "integrity": "sha512-TchMembfxfNVpHkbtriWltGWc+m3xszaRD0CZup7GFFhzIgQqxIfn3eGj1yZpfuflzPvfkt611B2Q/Bsk1YnGg==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/markdown-it/-/markdown-it-14.1.0.tgz", + "integrity": "sha512-a54IwgWPaeBCAAsv13YgmALOF1elABB08FxO9i+r4VFk5Vl4pKokRPeX8u5TCgSsPi6ec1otfLjdOpVcgbpshg==", "dev": true, "dependencies": { "argparse": "^2.0.1", - "entities": "~2.1.0", - "linkify-it": "^3.0.1", - "mdurl": "^1.0.1", - "uc.micro": "^1.0.5" + "entities": "^4.4.0", + "linkify-it": "^5.0.0", + "mdurl": "^2.0.0", + "punycode.js": "^2.3.1", + "uc.micro": "^2.1.0" }, "bin": { - "markdown-it": "bin/markdown-it.js" + "markdown-it": "bin/markdown-it.mjs" } }, "node_modules/markdown-it-anchor": { @@ -4271,15 +4297,6 @@ "markdown-it": "*" } }, - "node_modules/markdown-it/node_modules/entities": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/entities/-/entities-2.1.0.tgz", - "integrity": "sha512-hCx1oky9PFrJ611mf0ifBLBRW8lUUVRlFolb5gWRfIELabBlbp9xZvrqZLZAs+NxFnbfQoeGd8wDkygjg7U85w==", - "dev": true, - "funding": { - "url": "https://github.com/fb55/entities?sponsor=1" - } - }, "node_modules/marked": { "version": "4.3.0", "resolved": "https://registry.npmjs.org/marked/-/marked-4.3.0.tgz", @@ -4293,9 +4310,9 @@ } }, "node_modules/mdurl": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/mdurl/-/mdurl-1.0.1.tgz", - "integrity": "sha512-/sKlQJCBYVY9Ers9hqzKou4H6V5UWc/M59TH2dvkt+84itfnq7uFOMLpOiOS4ujvHP4etln18fmIxA5R5fll0g==", + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/mdurl/-/mdurl-2.0.0.tgz", + "integrity": "sha512-Lf+9+2r+Tdp5wXDXC4PcIBjTDtq4UKjCPMQhKIuzpJNW0b96kVqSwW0bT7FhRSfmAiFYgP+SCRvdrDozfh0U5w==", "dev": true }, "node_modules/merge-stream": { @@ -4304,13 +4321,22 @@ "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==", "dev": true }, + "node_modules/merge2": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/merge2/-/merge2-1.4.1.tgz", + "integrity": "sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg==", + "dev": true, + "engines": { + "node": ">= 8" + } + }, "node_modules/micromatch": { - "version": "4.0.5", - "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.5.tgz", - "integrity": "sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA==", + "version": "4.0.7", + "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.7.tgz", + "integrity": "sha512-LPP/3KorzCwBxfeUuZmaR6bG2kdeHSbe0P2tY3FLRU4vYrjYz5hI4QZwV0njUx3jeuKe67YukQ1LSPZBKDqO/Q==", "dev": true, "dependencies": { - "braces": "^3.0.2", + "braces": "^3.0.3", "picomatch": "^2.3.1" }, "engines": { @@ -4447,31 +4473,31 @@ } }, "node_modules/mocha": { - "version": "10.3.0", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.3.0.tgz", - "integrity": "sha512-uF2XJs+7xSLsrmIvn37i/wnc91nw7XjOQB8ccyx5aEgdnohr7n+rEiZP23WkCYHjilR6+EboEnbq/ZQDz4LSbg==", - "dev": true, - "dependencies": { - "ansi-colors": "4.1.1", - "browser-stdout": "1.3.1", - "chokidar": "3.5.3", - "debug": "4.3.4", - "diff": "5.0.0", - "escape-string-regexp": "4.0.0", - "find-up": "5.0.0", - "glob": "8.1.0", - "he": "1.2.0", - "js-yaml": "4.1.0", - "log-symbols": "4.1.0", - "minimatch": "5.0.1", - "ms": "2.1.3", - "serialize-javascript": "6.0.0", - "strip-json-comments": "3.1.1", - "supports-color": "8.1.1", - "workerpool": "6.2.1", - "yargs": "16.2.0", - "yargs-parser": "20.2.4", - "yargs-unparser": "2.0.0" + "version": "10.7.0", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.7.0.tgz", + "integrity": "sha512-v8/rBWr2VO5YkspYINnvu81inSz2y3ODJrhO175/Exzor1RcEZZkizgE2A+w/CAXXoESS8Kys5E62dOHGHzULA==", + "dev": true, + "dependencies": { + "ansi-colors": "^4.1.3", + "browser-stdout": "^1.3.1", + "chokidar": "^3.5.3", + "debug": "^4.3.5", + "diff": "^5.2.0", + "escape-string-regexp": "^4.0.0", + "find-up": "^5.0.0", + "glob": "^8.1.0", + "he": "^1.2.0", + "js-yaml": "^4.1.0", + "log-symbols": "^4.1.0", + "minimatch": "^5.1.6", + "ms": "^2.1.3", + "serialize-javascript": "^6.0.2", + "strip-json-comments": "^3.1.1", + "supports-color": "^8.1.1", + "workerpool": "^6.5.1", + "yargs": "^16.2.0", + "yargs-parser": "^20.2.9", + "yargs-unparser": "^2.0.0" }, "bin": { "_mocha": "bin/_mocha", @@ -4505,6 +4531,7 @@ "version": "8.1.0", "resolved": "https://registry.npmjs.org/glob/-/glob-8.1.0.tgz", "integrity": "sha512-r8hpEjiQEYlF2QU0df3dS+nxxSIreXQS1qRhMJM0Q5NDdR386C7jb7Hwwod8Fgiuex+k0GFjgft18yvxm5XoCQ==", + "deprecated": "Glob versions prior to v9 are no longer supported", "dev": true, "dependencies": { "fs.realpath": "^1.0.0", @@ -4521,9 +4548,9 @@ } }, "node_modules/mocha/node_modules/minimatch": { - "version": "5.0.1", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.0.1.tgz", - "integrity": "sha512-nLDxIFRyhDblz3qMuq+SoRZED4+miJ/G+tdDrjkkkRnjAsBexeGpgjLEQ0blJy7rHhR2b93rhQY4SvyWu9v03g==", + "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" @@ -4577,9 +4604,9 @@ "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" }, "node_modules/nan": { - "version": "2.19.0", - "resolved": "https://registry.npmjs.org/nan/-/nan-2.19.0.tgz", - "integrity": "sha512-nO1xXxfh/RWNxfd/XPfbIfFk5vgLsAxUR9y5O0cHMJu/AW9U95JLXqthYHjEp+8gQ5p96K9jUp8nbVOxCdRbtw==" + "version": "2.20.0", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.20.0.tgz", + "integrity": "sha512-bk3gXBZDGILuuo/6sKtr0DQmSThYHLtNCdSdXk9YkxD/jK6X2vmCyyXBBxyqZ4XcnzTyYEAThfX3DCEnLf6igw==" }, "node_modules/natural-compare": { "version": "1.4.0", @@ -4644,6 +4671,7 @@ "version": "3.0.1", "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-3.0.1.tgz", "integrity": "sha512-QZW4EDmGwlYur0Yyf/b2uGucHQMa8aFUP7eu9ddR73vvhFyt4V0Vl3QHPcTNJ8l6qYOBdxgXdnBXQrHilfRQBg==", + "deprecated": "This package is no longer supported.", "dev": true, "dependencies": { "delegates": "^1.0.0", @@ -4657,6 +4685,7 @@ "version": "4.0.4", "resolved": "https://registry.npmjs.org/gauge/-/gauge-4.0.4.tgz", "integrity": "sha512-f9m+BEN5jkg6a0fZjleidjN51VE1X+mPFQ2DJ0uv1V39oCLCbsGe6yjbBnp7eK7z/+GAon99a3nHuqbuuthyPg==", + "deprecated": "This package is no longer supported.", "dev": true, "dependencies": { "aproba": "^1.0.3 || ^2.0.0", @@ -4691,6 +4720,7 @@ "version": "6.0.2", "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-6.0.2.tgz", "integrity": "sha512-/vBvz5Jfr9dT/aFWd0FIRf+T/Q2WBsLENygUaFUqstqsycmZAP/t5BvFJTK0viFmSUxiUKTUplWy5vt+rvKIxg==", + "deprecated": "This package is no longer supported.", "dev": true, "dependencies": { "are-we-there-yet": "^3.0.0", @@ -4702,29 +4732,6 @@ "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, - "node_modules/node-gyp/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==", - "dev": true, - "dependencies": { - "inherits": "^2.0.3", - "string_decoder": "^1.1.1", - "util-deprecate": "^1.0.1" - }, - "engines": { - "node": ">= 6" - } - }, - "node_modules/node-gyp/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==", - "dev": true, - "dependencies": { - "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", @@ -4732,9 +4739,9 @@ "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==", + "version": "2.0.18", + "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.18.tgz", + "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==", "dev": true }, "node_modules/nopt": { @@ -4776,6 +4783,7 @@ "version": "5.0.1", "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-5.0.1.tgz", "integrity": "sha512-AqZtDUWOMKs1G/8lwylVjrdYgqA4d9nu8hc+0gzRxlDb1I10+FHBGMXs6aiQHFdCUUlqH99MUMuLfzWDNDtfxw==", + "deprecated": "This package is no longer supported.", "dependencies": { "are-we-there-yet": "^2.0.0", "console-control-strings": "^1.1.0", @@ -4815,17 +4823,17 @@ } }, "node_modules/optionator": { - "version": "0.9.3", - "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.3.tgz", - "integrity": "sha512-JjCoypp+jKn1ttEFExxhetCKeJt9zhAgAve5FXHixTvFDW/5aEktX9bufBKLRRMdU7bNtpLfcGu94B3cdEJgjg==", + "version": "0.9.4", + "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.4.tgz", + "integrity": "sha512-6IpQ7mKUxRcZNLIObR0hz7lxsapSSIYNZJwXPGeF0mTVqGKFIXj1DQcMoT22S3ROcLyY/rz0PWaWZ9ayWmad9g==", "dev": true, "dependencies": { - "@aashutoshrathi/word-wrap": "^1.2.3", "deep-is": "^0.1.3", "fast-levenshtein": "^2.0.6", "levn": "^0.4.1", "prelude-ls": "^1.2.1", - "type-check": "^0.4.0" + "type-check": "^0.4.0", + "word-wrap": "^1.2.5" }, "engines": { "node": ">= 0.8.0" @@ -4947,10 +4955,19 @@ "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==", "dev": true }, + "node_modules/path-type": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", + "integrity": "sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/picocolors": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.0.tgz", - "integrity": "sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ==", + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.1.tgz", + "integrity": "sha512-anP1Z8qwhkbmu7MFP5iTt+wQKXgwzf7zTyGlcdzabySa9vd0Xt392U0rVmz9poOaBj0uHJKyyo9/upk0HrEQew==", "dev": true }, "node_modules/picomatch": { @@ -5114,6 +5131,15 @@ "node": ">=6" } }, + "node_modules/punycode.js": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/punycode.js/-/punycode.js-2.3.1.tgz", + "integrity": "sha512-uxFIHU0YlHYhDQtV4R9J6a52SLx28BCjT+4ieh7IGbgwVJWO+km431c4yRlREUAsAmt/uMjQUyQHNEPf0M39CA==", + "dev": true, + "engines": { + "node": ">=6" + } + }, "node_modules/pure-rand": { "version": "6.1.0", "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.1.0.tgz", @@ -5160,21 +5186,22 @@ } }, "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==", + "version": "18.3.1", + "resolved": "https://registry.npmjs.org/react-is/-/react-is-18.3.1.tgz", + "integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==", "dev": true }, "node_modules/readable-stream": { - "version": "1.1.14", - "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-1.1.14.tgz", - "integrity": "sha512-+MeVjFf4L44XUkhM1eYbD8fyEsxcV81pqMSR5gblfcLCHfZvbrqy4/qYHE+/R5HoBUT11WV5O08Cr1n3YXkWVQ==", - "dev": true, + "version": "3.6.2", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-3.6.2.tgz", + "integrity": "sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA==", "dependencies": { - "core-util-is": "~1.0.0", - "inherits": "~2.0.1", - "isarray": "0.0.1", - "string_decoder": "~0.10.x" + "inherits": "^2.0.3", + "string_decoder": "^1.1.1", + "util-deprecate": "^1.0.1" + }, + "engines": { + "node": ">= 6" } }, "node_modules/readdirp": { @@ -5286,6 +5313,7 @@ "version": "3.0.2", "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-3.0.2.tgz", "integrity": "sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA==", + "deprecated": "Rimraf versions prior to v4 are no longer supported", "dependencies": { "glob": "^7.1.3" }, @@ -5345,12 +5373,9 @@ "optional": true }, "node_modules/semver": { - "version": "7.6.0", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.0.tgz", - "integrity": "sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==", - "dependencies": { - "lru-cache": "^6.0.0" - }, + "version": "7.6.3", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.3.tgz", + "integrity": "sha512-oVekP1cKtI+CTDvHWYFUcMtsK/00wmAEfyqKfNdARm8u1wNVhSgaX7A8d4UuIlUI5e84iEwOhs7ZPYRmzU9U6A==", "bin": { "semver": "bin/semver.js" }, @@ -5358,26 +5383,10 @@ "node": ">=10" } }, - "node_modules/semver/node_modules/lru-cache": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", - "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "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", - "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, "dependencies": { "randombytes": "^2.1.0" @@ -5440,9 +5449,9 @@ } }, "node_modules/socks": { - "version": "2.8.1", - "resolved": "https://registry.npmjs.org/socks/-/socks-2.8.1.tgz", - "integrity": "sha512-B6w7tkwNid7ToxjZ08rQMT8M9BJAf8DKx8Ft4NivzH0zBUfd6jldGcisJn/RLgxcX3FPNDdNQCUEMMT79b+oCQ==", + "version": "2.8.3", + "resolved": "https://registry.npmjs.org/socks/-/socks-2.8.3.tgz", + "integrity": "sha512-l5x7VUUWbjVFbafGLxPWkYsHIhEvmF85tbIeFZWc8ZPtoMyybuEhL7Jye/ooC4/d48FgOjSJXgsF/AJPYCW8Zw==", "dev": true, "dependencies": { "ip-address": "^9.0.5", @@ -5526,10 +5535,12 @@ } }, "node_modules/string_decoder": { - "version": "0.10.31", - "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-0.10.31.tgz", - "integrity": "sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==", - "dev": true + "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/string-length": { "version": "4.0.2", @@ -5703,6 +5714,18 @@ "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", "integrity": "sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw==" }, + "node_modules/ts-api-utils": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/ts-api-utils/-/ts-api-utils-1.3.0.tgz", + "integrity": "sha512-UQMIo7pb8WRomKR1/+MFVLTroIvDVtMX3K6OUir8ynLyzB8Jeriont2bTAtmNPa1ekAgN7YPDyf6V+ygrdU+eQ==", + "dev": true, + "engines": { + "node": ">=16" + }, + "peerDependencies": { + "typescript": ">=4.2.0" + } + }, "node_modules/type-check": { "version": "0.4.0", "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.4.0.tgz", @@ -5737,9 +5760,9 @@ } }, "node_modules/typescript": { - "version": "5.4.3", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.3.tgz", - "integrity": "sha512-KrPd3PKaCLr78MalgiwJnA25Nm8HAmdwN3mYUYZgG/wizIo9EainNVQI9/yDavtVFRN2h3k8uf3GLHuhDMgEHg==", + "version": "5.5.4", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.5.4.tgz", + "integrity": "sha512-Mtq29sKDAEYP7aljRgtPOpTvOfbwRWlS6dPRzwjdE+C0R4brX/GUyhHSecbHMFLNBLcJIPt9nl9yG5TZ1weH+Q==", "dev": true, "bin": { "tsc": "bin/tsc", @@ -5750,15 +5773,15 @@ } }, "node_modules/uc.micro": { - "version": "1.0.6", - "resolved": "https://registry.npmjs.org/uc.micro/-/uc.micro-1.0.6.tgz", - "integrity": "sha512-8Y75pvTYkLJW2hWQHXxoqRgV7qb9B+9vFEtidML+7koHUFapnVJAZ6cKs+Qjz5Aw3aZWHMC6u0wJE3At+nSGwA==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/uc.micro/-/uc.micro-2.1.0.tgz", + "integrity": "sha512-ARDJmphmdvUk6Glw7y9DQ2bFkKBHwQHLi2lsaH6PPmz/Ka9sFOBsBluozhDltWmnv9u/cF6Rt87znRTPV+yp/A==", "dev": true }, "node_modules/underscore": { - "version": "1.13.6", - "resolved": "https://registry.npmjs.org/underscore/-/underscore-1.13.6.tgz", - "integrity": "sha512-+A5Sja4HP1M08MaXya7p5LvjuM7K6q/2EaC0+iovj/wOcMsTzMvDFbasi/oSapiwOlt252IqsKqPjCl7huKS0A==", + "version": "1.13.7", + "resolved": "https://registry.npmjs.org/underscore/-/underscore-1.13.7.tgz", + "integrity": "sha512-GMXzWtsc57XAtguZgaQViUOzs0KTkk8ojr3/xAxXLITqf/3EMwxC0inyETfDFjH/Krbhuep0HNbbjI9i/q3F3g==", "dev": true }, "node_modules/undici-types": { @@ -5792,9 +5815,9 @@ } }, "node_modules/update-browserslist-db": { - "version": "1.0.13", - "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.13.tgz", - "integrity": "sha512-xebP81SNcPuNpPP3uzeW1NYXxI3rxyJzF3pD6sH4jE7o/IX+WtSpwnVU+qIsDPyk0d3hmFQ7mjqc6AtV604hbg==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.1.0.tgz", + "integrity": "sha512-EdRAaAyk2cUE1wOf2DkEhzxqOQvFOoRJFNS6NeyJ01Gp2beMRpBAINjM2iDXE3KCuKhwnvHIQCJm6ThL2Z+HzQ==", "dev": true, "funding": [ { @@ -5811,8 +5834,8 @@ } ], "dependencies": { - "escalade": "^3.1.1", - "picocolors": "^1.0.0" + "escalade": "^3.1.2", + "picocolors": "^1.0.1" }, "bin": { "update-browserslist-db": "cli.js" @@ -5836,9 +5859,9 @@ "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==", + "version": "9.3.0", + "resolved": "https://registry.npmjs.org/v8-to-istanbul/-/v8-to-istanbul-9.3.0.tgz", + "integrity": "sha512-kiGUalWN+rgBJ/1OHZsBtU4rXZOfj/7rKQxULKlIzwzQSvMJUUNgPwJEEh7gU6xEVxC0ahoOBvN2YI8GH6FNgA==", "dev": true, "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", @@ -5895,10 +5918,19 @@ "string-width": "^1.0.2 || 2 || 3 || 4" } }, + "node_modules/word-wrap": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/word-wrap/-/word-wrap-1.2.5.tgz", + "integrity": "sha512-BN22B5eaMMI9UMtjrGd5g5eCYPpCPDUy0FJXbYsaT5zYxjFOckS53SQDE3pWkVoWpHXVb3BrYcEN4Twa55B5cA==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/workerpool": { - "version": "6.2.1", - "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.2.1.tgz", - "integrity": "sha512-ILEIE97kDZvF9Wb9f6h5aXK4swSlKGUcOEGiIYb2OOu/IrDU9iwj0fD//SsA6E5ibwJxpEvhullJY4Sl4GcpAw==", + "version": "6.5.1", + "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.5.1.tgz", + "integrity": "sha512-Fs4dNYcsdpYSAfVxhnl1L5zTksjvOJxtC5hzMNl+1t9B8hTJTdKDyZ5ju7ztgPy+ft9tBFXoOlDNiOT9WUXZlA==", "dev": true }, "node_modules/wrap-ansi": { @@ -5976,9 +6008,9 @@ } }, "node_modules/yargs-parser": { - "version": "20.2.4", - "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-20.2.4.tgz", - "integrity": "sha512-WOkpgNhPTlE73h4VFAFsOnomJVaovO8VqLDzy5saChRBFQFBoMYirowyW+Q9HB4HFF4Z7VZTiG3iSzJJA29yRA==", + "version": "20.2.9", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-20.2.9.tgz", + "integrity": "sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w==", "dev": true, "engines": { "node": ">=10" diff --git a/package.json b/package.json index 28fd63ea..c38781e9 100644 --- a/package.json +++ b/package.json @@ -31,13 +31,14 @@ }, "license": "MIT", "devDependencies": { + "@eslint/js": "^9.8.0", "@types/node": "^20.4.5", "bluebird": "^3.5.3", - "eslint": "8.54.0", + "eslint": "^8.57.0", + "eslint-plugin-jest": "^28.6.0", "jest": "^29.7.0", "jsdoc": "^4.0.2", - "jshint": "^2.13.6", - "mocha": "^10.2.0", + "mocha": "^10.7.0", "node-gyp": "^9.3.1", "typescript": "^5.1.6" }, @@ -49,4 +50,4 @@ "engines": { "node": ">=18.0.0" } -} \ No newline at end of file +} diff --git a/src/admin.cc b/src/admin.cc index 39bfd76b..d485891c 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -8,12 +8,13 @@ * of the MIT license. See the LICENSE.txt file for details. */ +#include "src/admin.h" + +#include #include #include -#include #include "src/workers.h" -#include "src/admin.h" using Nan::FunctionCallbackInfo; diff --git a/src/admin.h b/src/admin.h index d931072c..3af30f21 100644 --- a/src/admin.h +++ b/src/admin.h @@ -17,7 +17,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "rdkafka.h" // NOLINT #include "src/common.h" diff --git a/src/binding.cc b/src/binding.cc index 04222635..7b3fe77c 100644 --- a/src/binding.cc +++ b/src/binding.cc @@ -2,11 +2,13 @@ * confluent-kafka-javascript - Node.js wrapper for RdKafka C/C++ library * * Copyright (c) 2016-2023 Blizzard Entertainment + * (c) 2024 Confluent, Inc. * * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. */ +#include #include #include "src/binding.h" diff --git a/src/binding.h b/src/binding.h index 45e08088..0d656b10 100644 --- a/src/binding.h +++ b/src/binding.h @@ -12,7 +12,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" #include "src/errors.h" #include "src/config.h" diff --git a/src/callbacks.cc b/src/callbacks.cc index a3a90148..d032a312 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -8,7 +8,11 @@ * of the MIT license. See the LICENSE.txt file for details. */ +#include "src/callbacks.h" + +#include #include +#include #include #include "src/kafka-consumer.h" diff --git a/src/callbacks.h b/src/callbacks.h index 315a1df6..1775c401 100644 --- a/src/callbacks.h +++ b/src/callbacks.h @@ -16,7 +16,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" typedef Nan::Persistent +#include #include #include -#include "src/common.h" - namespace NodeKafka { void Log(std::string str) { @@ -164,7 +165,9 @@ std::list v8ArrayToStringList(v8::Local parameter) { } template<> v8::Local GetParameter >( - v8::Local object, std::string field_name, v8::Local def) { + v8::Local object, + std::string field_name, + v8::Local def) { v8::Local field = Nan::New(field_name.c_str()).ToLocalChecked(); if (Nan::Has(object, field).FromMaybe(false)) { @@ -301,7 +304,8 @@ namespace TopicPartition { * @note This method returns a v8 array of a mix of topic partition * objects and errors. For a more uniform return type of * topic partitions (which have an internal error property), - * use `ToTopicPartitionV8Array(const rd_kafka_topic_partition_list_t*, bool)`. + * use `ToTopicPartitionV8Array(const rd_kafka_topic_partition_list_t*, + * bool)`. */ v8::Local ToV8Array( std::vector & topic_partition_list) { // NOLINT @@ -326,12 +330,14 @@ v8::Local ToV8Array( Nan::New(topic_partition->offset())); } - // If present, size >= 1, since it will include at least the null terminator. + // If present, size >= 1, since it will include at least the + // null terminator. if (topic_partition->get_metadata().size() > 0) { Nan::Set(obj, Nan::New("metadata").ToLocalChecked(), Nan::New( - reinterpret_cast(topic_partition->get_metadata().data()), - topic_partition->get_metadata().size() - 1) // null terminator is not required by the constructor. + reinterpret_cast(topic_partition->get_metadata().data()), // NOLINT + // null terminator is not required by the constructor. + topic_partition->get_metadata().size() - 1) .ToLocalChecked()); } @@ -444,14 +450,15 @@ RdKafka::TopicPartition * FromV8Object(v8::Local topic_partition) { int64_t offset = GetParameter(topic_partition, "offset", 0); if (partition == -1) { - return NULL; +return NULL; } if (topic.empty()) { return NULL; } - RdKafka::TopicPartition *toppar = RdKafka::TopicPartition::create(topic, partition, offset); + RdKafka::TopicPartition *toppar = + RdKafka::TopicPartition::create(topic, partition, offset); v8::Local metadataKey = Nan::New("metadata").ToLocalChecked(); if (Nan::Has(topic_partition, metadataKey).FromMaybe(false)) { diff --git a/src/common.h b/src/common.h index c8c979dc..d98508e3 100644 --- a/src/common.h +++ b/src/common.h @@ -13,11 +13,12 @@ #include +#include #include #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "rdkafka.h" // NOLINT #include "src/errors.h" diff --git a/src/config.cc b/src/config.cc index a781fdf7..5f66b2d8 100644 --- a/src/config.cc +++ b/src/config.cc @@ -6,13 +6,13 @@ * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. */ +#include "src/config.h" +#include #include #include #include -#include "src/config.h" - using Nan::MaybeLocal; using Nan::Maybe; using v8::Local; @@ -89,7 +89,10 @@ Conf * Conf::create(RdKafka::Conf::ConfType type, v8::Local object, return rdconf; } -void Conf::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add, std::string &errstr) { +void Conf::ConfigureCallback( + const std::string &string_key, + const v8::Local &cb, + bool add, std::string &errstr) { if (string_key.compare("rebalance_cb") == 0) { NodeKafka::Callbacks::Rebalance *rebalance = rebalance_cb(); if (add) { diff --git a/src/config.h b/src/config.h index f0ffdba4..d7a5a786 100644 --- a/src/config.h +++ b/src/config.h @@ -16,7 +16,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" #include "src/callbacks.h" @@ -32,7 +32,10 @@ class Conf : public RdKafka::Conf { void listen(); void stop(); - void ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add, std::string &errstr); + void ConfigureCallback( + const std::string &string_key, + const v8::Local &cb, + bool add, std::string &errstr); bool is_sasl_oauthbearer() const; diff --git a/src/connection.cc b/src/connection.cc index a7f26c29..833c34f2 100644 --- a/src/connection.cc +++ b/src/connection.cc @@ -7,11 +7,12 @@ * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. */ +#include "src/connection.h" +#include #include #include -#include "src/connection.h" #include "src/workers.h" using RdKafka::Conf; @@ -69,10 +70,9 @@ Connection::~Connection() { } Baton Connection::rdkafkaErrorToBaton(RdKafka::Error* error) { - if ( NULL == error) { + if (NULL == error) { return Baton(RdKafka::ERR_NO_ERROR); - } - else { + } else { Baton result(error->code(), error->str(), error->is_fatal(), error->is_retriable(), error->txn_requires_abort()); delete error; @@ -258,7 +258,7 @@ Baton Connection::GetMetadata( return Baton(metadata); } else { // metadata is not set here - // @see https://github.com/confluentinc/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L860 + // @see https://github.com/confluentinc/librdkafka/blob/master/src-cpp/rdkafkacpp.h#L860 // NOLINT return Baton(err); } } @@ -325,7 +325,8 @@ Baton Connection::SetOAuthBearerTokenFailure(const std::string& errstr) { return Baton(error_code); } -void Connection::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add) { +void Connection::ConfigureCallback( + const std::string &string_key, const v8::Local &cb, bool add) { if (string_key.compare("event_cb") == 0) { if (add) { this->m_event_cb.dispatcher.AddCallback(cb); @@ -494,14 +495,18 @@ NAN_METHOD(Connection::NodeConfigureCallbacks) { Connection* obj = ObjectWrap::Unwrap(info.This()); const bool add = Nan::To(info[0]).ToChecked(); - v8::Local configs_object = info[1]->ToObject(context).ToLocalChecked(); - v8::Local configs_property_names = configs_object->GetOwnPropertyNames(context).ToLocalChecked(); + v8::Local configs_object = + info[1]->ToObject(context).ToLocalChecked(); + v8::Local configs_property_names = + configs_object->GetOwnPropertyNames(context).ToLocalChecked(); for (unsigned int j = 0; j < configs_property_names->Length(); ++j) { std::string configs_string_key; - v8::Local configs_key = Nan::Get(configs_property_names, j).ToLocalChecked(); - v8::Local configs_value = Nan::Get(configs_object, configs_key).ToLocalChecked(); + v8::Local configs_key = + Nan::Get(configs_property_names, j).ToLocalChecked(); + v8::Local configs_value = + Nan::Get(configs_object, configs_key).ToLocalChecked(); int config_type = 0; if (configs_value->IsObject() && configs_key->IsString()) { @@ -520,8 +525,10 @@ NAN_METHOD(Connection::NodeConfigureCallbacks) { continue; } - v8::Local object = configs_value->ToObject(context).ToLocalChecked(); - v8::Local property_names = object->GetOwnPropertyNames(context).ToLocalChecked(); + v8::Local object = + configs_value->ToObject(context).ToLocalChecked(); + v8::Local property_names = + object->GetOwnPropertyNames(context).ToLocalChecked(); for (unsigned int i = 0; i < property_names->Length(); ++i) { std::string errstr; diff --git a/src/connection.h b/src/connection.h index 8d859a2a..c798814b 100644 --- a/src/connection.h +++ b/src/connection.h @@ -13,10 +13,11 @@ #include #include +#include #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" #include "src/errors.h" @@ -71,7 +72,8 @@ class Connection : public Nan::ObjectWrap { virtual void ActivateDispatchers() = 0; virtual void DeactivateDispatchers() = 0; - virtual void ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add); + virtual void ConfigureCallback( + const std::string &string_key, const v8::Local &cb, bool add); std::string Name() const; diff --git a/src/errors.cc b/src/errors.cc index f708f79c..9d1d9675 100644 --- a/src/errors.cc +++ b/src/errors.cc @@ -33,8 +33,9 @@ v8::Local RdKafkaError(const RdKafka::ErrorCode &err) { return RdKafkaError(err, errstr); } -v8::Local RdKafkaError(const RdKafka::ErrorCode &err, std::string errstr, - bool isFatal, bool isRetriable, bool isTxnRequiresAbort) { +v8::Local RdKafkaError( + const RdKafka::ErrorCode &err, std::string errstr, + bool isFatal, bool isRetriable, bool isTxnRequiresAbort) { v8::Local ret = RdKafkaError(err, errstr); Nan::Set(ret, Nan::New("isFatal").ToLocalChecked(), @@ -100,7 +101,7 @@ v8::Local Baton::ToObject() { } v8::Local Baton::ToTxnObject() { - return RdKafkaError(m_err, m_errstr, m_isFatal, m_isRetriable, m_isTxnRequiresAbort); + return RdKafkaError(m_err, m_errstr, m_isFatal, m_isRetriable, m_isTxnRequiresAbort); // NOLINT } RdKafka::ErrorCode Baton::err() { diff --git a/src/errors.h b/src/errors.h index 2b4f3464..248d26ad 100644 --- a/src/errors.h +++ b/src/errors.h @@ -15,7 +15,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 4338bdcf..58828e2b 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -8,6 +8,7 @@ * of the MIT license. See the LICENSE.txt file for details. */ +#include #include #include @@ -56,7 +57,8 @@ Baton KafkaConsumer::Connect() { std::string errstr; { scoped_shared_write_lock lock(m_connection_lock); - m_client = RdKafka::KafkaConsumer::create(m_gconfig, errstr); + m_consumer = RdKafka::KafkaConsumer::create(m_gconfig, errstr); + m_client = m_consumer; } if (!m_client || !errstr.empty()) { @@ -95,12 +97,11 @@ Baton KafkaConsumer::Disconnect() { { scoped_shared_write_lock lock(m_connection_lock); - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - err = consumer->close(); + err = m_consumer->close(); delete m_client; m_client = NULL; + m_consumer = nullptr; } } @@ -173,10 +174,8 @@ Baton KafkaConsumer::Assign(std::vector partitions) { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is disconnected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - RdKafka::ErrorCode errcode = consumer->assign(partitions); + RdKafka::ErrorCode errcode = m_consumer->assign(partitions); if (errcode == RdKafka::ERR_NO_ERROR) { m_partition_cnt = partitions.size(); @@ -196,10 +195,7 @@ Baton KafkaConsumer::Unassign() { return Baton(RdKafka::ERR__STATE); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode errcode = consumer->unassign(); + RdKafka::ErrorCode errcode = m_consumer->unassign(); if (errcode != RdKafka::ERR_NO_ERROR) { return Baton(errcode); @@ -213,20 +209,18 @@ Baton KafkaConsumer::Unassign() { return Baton(RdKafka::ERR_NO_ERROR); } -Baton KafkaConsumer::IncrementalAssign(std::vector partitions) { +Baton KafkaConsumer::IncrementalAssign( + std::vector partitions) { if (!IsConnected()) { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is disconnected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::Error* error = consumer->incremental_assign(partitions); + RdKafka::Error* error = m_consumer->incremental_assign(partitions); if (error == NULL) { m_partition_cnt += partitions.size(); // We assume here that there are no duplicate assigns and just transfer. - m_partitions.insert(m_partitions.end(), partitions.begin(), partitions.end()); + m_partitions.insert(m_partitions.end(), partitions.begin(), partitions.end()); // NOLINT } else { // If we're in error, destroy it, otherwise, don't (since we're using them). RdKafka::TopicPartition::destroy(partitions); @@ -235,20 +229,18 @@ Baton KafkaConsumer::IncrementalAssign(std::vector par return rdkafkaErrorToBaton(error); } -Baton KafkaConsumer::IncrementalUnassign(std::vector partitions) { +Baton KafkaConsumer::IncrementalUnassign( + std::vector partitions) { if (!IsClosing() && !IsConnected()) { return Baton(RdKafka::ERR__STATE); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::Error* error = consumer->incremental_unassign(partitions); + RdKafka::Error* error = m_consumer->incremental_unassign(partitions); std::vector delete_partitions; if (error == NULL) { - // For now, use two for loops. Make more efficient if needed at a later point. + // For now, use two for loops. Make more efficient if needed later. for (unsigned int i = 0; i < partitions.size(); i++) { for (unsigned int j = 0; j < m_partitions.size(); j++) { if (partitions[i]->partition() == m_partitions[j]->partition() && @@ -265,8 +257,8 @@ Baton KafkaConsumer::IncrementalUnassign(std::vector p // Destroy the old list of partitions since we are no longer using it RdKafka::TopicPartition::destroy(delete_partitions); - // Destroy the partition args since those are only used to lookup the partitions - // that needed to be deleted. + // Destroy the partition args since those are only used to lookup the + // partitions that needed to be deleted. RdKafka::TopicPartition::destroy(partitions); return rdkafkaErrorToBaton(error); @@ -277,10 +269,7 @@ Baton KafkaConsumer::Commit(std::vector toppars) { return Baton(RdKafka::ERR__STATE); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->commitAsync(toppars); + RdKafka::ErrorCode err = m_consumer->commitAsync(toppars); return Baton(err); } @@ -290,12 +279,9 @@ Baton KafkaConsumer::Commit(RdKafka::TopicPartition * toppar) { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - // Need to put topic in a vector for it to work std::vector offsets = {toppar}; - RdKafka::ErrorCode err = consumer->commitAsync(offsets); + RdKafka::ErrorCode err = m_consumer->commitAsync(offsets); return Baton(err); } @@ -306,10 +292,7 @@ Baton KafkaConsumer::Commit() { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->commitAsync(); + RdKafka::ErrorCode err = m_consumer->commitAsync(); return Baton(err); } @@ -320,10 +303,7 @@ Baton KafkaConsumer::CommitSync(std::vector toppars) { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->commitSync(toppars); + RdKafka::ErrorCode err = m_consumer->commitSync(toppars); // RdKafka::TopicPartition::destroy(toppars); return Baton(err); @@ -334,12 +314,9 @@ Baton KafkaConsumer::CommitSync(RdKafka::TopicPartition * toppar) { return Baton(RdKafka::ERR__STATE); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - // Need to put topic in a vector for it to work std::vector offsets = {toppar}; - RdKafka::ErrorCode err = consumer->commitSync(offsets); + RdKafka::ErrorCode err = m_consumer->commitSync(offsets); return Baton(err); } @@ -350,10 +327,7 @@ Baton KafkaConsumer::CommitSync() { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->commitSync(); + RdKafka::ErrorCode err = m_consumer->commitSync(); return Baton(err); } @@ -363,10 +337,7 @@ Baton KafkaConsumer::Seek(const RdKafka::TopicPartition &partition, int timeout_ return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->seek(partition, timeout_ms); + RdKafka::ErrorCode err = m_consumer->seek(partition, timeout_ms); return Baton(err); } @@ -377,10 +348,7 @@ Baton KafkaConsumer::Committed(std::vector &toppars, return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->committed(toppars, timeout_ms); + RdKafka::ErrorCode err = m_consumer->committed(toppars, timeout_ms); return Baton(err); } @@ -390,10 +358,7 @@ Baton KafkaConsumer::Position(std::vector &toppars) { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode err = consumer->position(toppars); + RdKafka::ErrorCode err = m_consumer->position(toppars); return Baton(err); } @@ -403,13 +368,10 @@ Baton KafkaConsumer::Subscription() { return Baton(RdKafka::ERR__STATE, "Consumer is not connected"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - // Needs to be a pointer since we're returning it through the baton std::vector * topics = new std::vector; - RdKafka::ErrorCode err = consumer->subscription(*topics); + RdKafka::ErrorCode err = m_consumer->subscription(*topics); if (err == RdKafka::ErrorCode::ERR_NO_ERROR) { // Good to go @@ -421,9 +383,7 @@ Baton KafkaConsumer::Subscription() { Baton KafkaConsumer::Unsubscribe() { if (IsConnected() && IsSubscribed()) { - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - consumer->unsubscribe(); + m_consumer->unsubscribe(); m_is_subscribed = false; } @@ -432,10 +392,7 @@ Baton KafkaConsumer::Unsubscribe() { Baton KafkaConsumer::Pause(std::vector & toppars) { if (IsConnected()) { - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - RdKafka::ErrorCode err = consumer->pause(toppars); - + RdKafka::ErrorCode err = m_consumer->pause(toppars); return Baton(err); } @@ -444,9 +401,7 @@ Baton KafkaConsumer::Pause(std::vector & toppars) { Baton KafkaConsumer::Resume(std::vector & toppars) { if (IsConnected()) { - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - RdKafka::ErrorCode err = consumer->resume(toppars); + RdKafka::ErrorCode err = m_consumer->resume(toppars); return Baton(err); } @@ -454,16 +409,15 @@ Baton KafkaConsumer::Resume(std::vector & toppars) { return Baton(RdKafka::ERR__STATE); } -Baton KafkaConsumer::OffsetsStore(std::vector & toppars) { // NOLINT - if (IsConnected() && IsSubscribed()) { - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - RdKafka::ErrorCode err = consumer->offsets_store(toppars); - - return Baton(err); +Baton KafkaConsumer::OffsetsStore( + std::vector& toppars) { // NOLINT + if (!IsSubscribed()) { /* IsSubscribed also checks IsConnected */ + return Baton(RdKafka::ERR__STATE); } - return Baton(RdKafka::ERR__STATE); + RdKafka::ErrorCode err = m_consumer->offsets_store(toppars); + + return Baton(err); } Baton KafkaConsumer::Subscribe(std::vector topics) { @@ -471,10 +425,7 @@ Baton KafkaConsumer::Subscribe(std::vector topics) { return Baton(RdKafka::ERR__STATE); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::ErrorCode errcode = consumer->subscribe(topics); + RdKafka::ErrorCode errcode = m_consumer->subscribe(topics); if (errcode != RdKafka::ERR_NO_ERROR) { return Baton(errcode); } @@ -490,10 +441,7 @@ Baton KafkaConsumer::Consume(int timeout_ms) { if (!IsConnected()) { return Baton(RdKafka::ERR__STATE, "KafkaConsumer is not connected"); } else { - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - RdKafka::Message * message = consumer->consume(timeout_ms); + RdKafka::Message * message = m_consumer->consume(timeout_ms); RdKafka::ErrorCode response_code = message->err(); // we want to handle these errors at the call site if (response_code != RdKafka::ERR_NO_ERROR && @@ -517,11 +465,8 @@ Baton KafkaConsumer::RefreshAssignments() { return Baton(RdKafka::ERR__STATE); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - std::vector partition_list; - RdKafka::ErrorCode err = consumer->assignment(partition_list); + RdKafka::ErrorCode err = m_consumer->assignment(partition_list); switch (err) { case RdKafka::ERR_NO_ERROR: @@ -545,10 +490,7 @@ std::string KafkaConsumer::RebalanceProtocol() { return std::string("NONE"); } - RdKafka::KafkaConsumer* consumer = - dynamic_cast(m_client); - - return consumer->rebalance_protocol(); + return m_consumer->rebalance_protocol(); } Nan::Persistent KafkaConsumer::constructor; @@ -615,6 +557,7 @@ void KafkaConsumer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "commit", NodeCommit); Nan::SetPrototypeMethod(tpl, "commitSync", NodeCommitSync); Nan::SetPrototypeMethod(tpl, "offsetsStore", NodeOffsetsStore); + Nan::SetPrototypeMethod(tpl, "offsetsStoreSingle", NodeOffsetsStoreSingle); constructor.Reset((tpl->GetFunction(Nan::GetCurrentContext())) .ToLocalChecked()); @@ -1179,6 +1122,39 @@ NAN_METHOD(KafkaConsumer::NodeOffsetsStore) { info.GetReturnValue().Set(Nan::New(error_code)); } +NAN_METHOD(KafkaConsumer::NodeOffsetsStoreSingle) { + Nan::HandleScope scope; + + // If number of parameters is less than 3 (need topic partition, partition, + // offset, and leader epoch), we can't call this. + if (info.Length() < 4) { + return Nan::ThrowError( + "Must provide topic, partition, offset and leaderEpoch"); + } + + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); + + // Get string pointer for the topic name + Nan::Utf8String topicUTF8(Nan::To(info[0]).ToLocalChecked()); + const std::string& topic_name(*topicUTF8); + + int64_t partition = Nan::To(info[1]).FromJust(); + int64_t offset = Nan::To(info[2]).FromJust(); + int64_t leader_epoch = Nan::To(info[3]).FromJust(); + + RdKafka::TopicPartition* toppar = + RdKafka::TopicPartition::create(topic_name, partition, offset); + toppar->set_leader_epoch(leader_epoch); + std::vector toppars = {toppar}; + + Baton b = consumer->OffsetsStore(toppars); + + delete toppar; + + int error_code = static_cast(b.err()); + info.GetReturnValue().Set(Nan::New(error_code)); +} + NAN_METHOD(KafkaConsumer::NodePause) { Nan::HandleScope scope; @@ -1306,7 +1282,8 @@ NAN_METHOD(KafkaConsumer::NodeConsumeLoop) { Nan::Callback *callback = new Nan::Callback(cb); - consumer->m_consume_loop = new Workers::KafkaConsumerConsumeLoop(callback, consumer, timeout_ms, timeout_sleep_delay_ms); + consumer->m_consume_loop = + new Workers::KafkaConsumerConsumeLoop(callback, consumer, timeout_ms, timeout_sleep_delay_ms); // NOLINT info.GetReturnValue().Set(Nan::Null()); } @@ -1348,14 +1325,15 @@ NAN_METHOD(KafkaConsumer::NodeConsume) { numMessages = numMessagesMaybe.FromJust(); } - v8::Local isTimeoutOnlyForFirstMessageBoolean = info[2].As(); - Nan::Maybe isTimeoutOnlyForFirstMessageMaybe = Nan::To(isTimeoutOnlyForFirstMessageBoolean); // NOLINT + v8::Local isTimeoutOnlyForFirstMessageBoolean = info[2].As(); // NOLINT + Nan::Maybe isTimeoutOnlyForFirstMessageMaybe = + Nan::To(isTimeoutOnlyForFirstMessageBoolean); bool isTimeoutOnlyForFirstMessage; if (isTimeoutOnlyForFirstMessageMaybe.IsNothing()) { return Nan::ThrowError("Parameter must be a boolean"); } else { - isTimeoutOnlyForFirstMessage = isTimeoutOnlyForFirstMessageMaybe.FromJust(); + isTimeoutOnlyForFirstMessage = isTimeoutOnlyForFirstMessageMaybe.FromJust(); // NOLINT } KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); @@ -1409,7 +1387,8 @@ NAN_METHOD(KafkaConsumer::NodeDisconnect) { Nan::Callback *callback = new Nan::Callback(cb); KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); - Workers::KafkaConsumerConsumeLoop* consumeLoop = (Workers::KafkaConsumerConsumeLoop*)consumer->m_consume_loop; + Workers::KafkaConsumerConsumeLoop* consumeLoop = + (Workers::KafkaConsumerConsumeLoop*)consumer->m_consume_loop; if (consumeLoop != nullptr) { // stop the consume loop consumeLoop->Close(); diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index 7a9b523a..ab8fcc18 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -17,7 +17,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" #include "src/connection.h" @@ -104,6 +104,10 @@ class KafkaConsumer : public Connection { void* m_consume_loop = nullptr; + /* This is the same client as stored in m_client. + * Prevents a dynamic_cast in every single method. */ + RdKafka::KafkaConsumer *m_consumer = nullptr; + // Node methods static NAN_METHOD(NodeConnect); static NAN_METHOD(NodeSubscribe); @@ -118,6 +122,7 @@ class KafkaConsumer : public Connection { static NAN_METHOD(NodeCommit); static NAN_METHOD(NodeCommitSync); static NAN_METHOD(NodeOffsetsStore); + static NAN_METHOD(NodeOffsetsStoreSingle); static NAN_METHOD(NodeCommitted); static NAN_METHOD(NodePosition); static NAN_METHOD(NodeSubscription); diff --git a/src/producer.cc b/src/producer.cc index 63b02a57..c5d1f23a 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -96,7 +96,7 @@ void Producer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "beginTransaction", NodeBeginTransaction); Nan::SetPrototypeMethod(tpl, "commitTransaction", NodeCommitTransaction); Nan::SetPrototypeMethod(tpl, "abortTransaction", NodeAbortTransaction); - Nan::SetPrototypeMethod(tpl, "sendOffsetsToTransaction", NodeSendOffsetsToTransaction); + Nan::SetPrototypeMethod(tpl, "sendOffsetsToTransaction", NodeSendOffsetsToTransaction); // NOLINT // connect. disconnect. resume. pause. get meta data constructor.Reset((tpl->GetFunction(Nan::GetCurrentContext())) @@ -132,8 +132,9 @@ void Producer::New(const Nan::FunctionCallbackInfo& info) { // 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); + 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 @@ -365,11 +366,12 @@ Baton Producer::SetPollInBackground(bool set) { return Baton(RdKafka::ERR_NO_ERROR); } -void Producer::ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add) { +void Producer::ConfigureCallback(const std::string& string_key, + const v8::Local& cb, bool add) { if (string_key.compare("delivery_cb") == 0) { if (add) { bool dr_msg_cb = false; - v8::Local dr_msg_cb_key = Nan::New("dr_msg_cb").ToLocalChecked(); + v8::Local dr_msg_cb_key = Nan::New("dr_msg_cb").ToLocalChecked(); // NOLINT if (Nan::Has(cb, dr_msg_cb_key).FromMaybe(false)) { v8::Local v = Nan::Get(cb, dr_msg_cb_key).ToLocalChecked(); if (v->IsBoolean()) { @@ -440,10 +442,12 @@ Baton Producer::SendOffsetsToTransaction( return Baton(RdKafka::ERR__STATE); } - RdKafka::ConsumerGroupMetadata* group_metadata = dynamic_cast(consumer->m_client)->groupMetadata(); + RdKafka::ConsumerGroupMetadata* group_metadata = + dynamic_cast(consumer->m_client)->groupMetadata(); // NOLINT RdKafka::Producer* producer = dynamic_cast(m_client); - RdKafka::Error* error = producer->send_offsets_to_transaction(offsets, group_metadata, timeout_ms); + RdKafka::Error* error = + producer->send_offsets_to_transaction(offsets, group_metadata, timeout_ms); delete group_metadata; return rdkafkaErrorToBaton( error); @@ -515,9 +519,11 @@ NAN_METHOD(Producer::NodeProduce) { message_buffer_data = node::Buffer::Data(message_buffer_object); if (message_buffer_data == NULL) { // empty string message buffer should not end up as null message - v8::Local message_buffer_object_emptystring = Nan::NewBuffer(new char[0], 0).ToLocalChecked(); - message_buffer_length = node::Buffer::Length(message_buffer_object_emptystring); - message_buffer_data = node::Buffer::Data(message_buffer_object_emptystring); + v8::Local message_buffer_object_emptystring = + Nan::NewBuffer(new char[0], 0).ToLocalChecked(); + message_buffer_length = + node::Buffer::Length(message_buffer_object_emptystring); + message_buffer_data = node::Buffer::Data(message_buffer_object_emptystring); // NOLINT } } @@ -545,9 +551,10 @@ NAN_METHOD(Producer::NodeProduce) { key_buffer_data = node::Buffer::Data(key_buffer_object); if (key_buffer_data == NULL) { // empty string key buffer should not end up as null key - v8::Local key_buffer_object_emptystring = Nan::NewBuffer(new char[0], 0).ToLocalChecked(); - key_buffer_length = node::Buffer::Length(key_buffer_object_emptystring); - key_buffer_data = node::Buffer::Data(key_buffer_object_emptystring); + v8::Local key_buffer_object_emptystring = + Nan::NewBuffer(new char[0], 0).ToLocalChecked(); + key_buffer_length = node::Buffer::Length(key_buffer_object_emptystring); + key_buffer_data = node::Buffer::Data(key_buffer_object_emptystring); } } else { // If it was a string just use the utf8 value. @@ -823,7 +830,8 @@ NAN_METHOD(Producer::NodeInitTransactions) { Nan::Callback *callback = new Nan::Callback(cb); Producer* producer = ObjectWrap::Unwrap(info.This()); - Nan::AsyncQueueWorker(new Workers::ProducerInitTransactions(callback, producer, timeout_ms)); + Nan::AsyncQueueWorker( + new Workers::ProducerInitTransactions(callback, producer, timeout_ms)); info.GetReturnValue().Set(Nan::Null()); } @@ -839,7 +847,7 @@ NAN_METHOD(Producer::NodeBeginTransaction) { Nan::Callback *callback = new Nan::Callback(cb); Producer* producer = ObjectWrap::Unwrap(info.This()); - Nan::AsyncQueueWorker(new Workers::ProducerBeginTransaction(callback, producer)); + Nan::AsyncQueueWorker(new Workers::ProducerBeginTransaction(callback, producer)); // NOLINT info.GetReturnValue().Set(Nan::Null()); } @@ -857,7 +865,8 @@ NAN_METHOD(Producer::NodeCommitTransaction) { Nan::Callback *callback = new Nan::Callback(cb); Producer* producer = ObjectWrap::Unwrap(info.This()); - Nan::AsyncQueueWorker(new Workers::ProducerCommitTransaction(callback, producer, timeout_ms)); + Nan::AsyncQueueWorker( + new Workers::ProducerCommitTransaction(callback, producer, timeout_ms)); info.GetReturnValue().Set(Nan::Null()); } @@ -875,7 +884,8 @@ NAN_METHOD(Producer::NodeAbortTransaction) { Nan::Callback *callback = new Nan::Callback(cb); Producer* producer = ObjectWrap::Unwrap(info.This()); - Nan::AsyncQueueWorker(new Workers::ProducerAbortTransaction(callback, producer, timeout_ms)); + Nan::AsyncQueueWorker( + new Workers::ProducerAbortTransaction(callback, producer, timeout_ms)); info.GetReturnValue().Set(Nan::Null()); } @@ -884,10 +894,12 @@ NAN_METHOD(Producer::NodeSendOffsetsToTransaction) { Nan::HandleScope scope; if (info.Length() < 4) { - return Nan::ThrowError("Need to specify offsets, consumer, timeout for 'send offsets to transaction', and callback"); + return Nan::ThrowError( + "Need to specify offsets, consumer, timeout for 'send offsets to transaction', and callback"); // NOLINT } if (!info[0]->IsArray()) { - return Nan::ThrowError("First argument to 'send offsets to transaction' has to be a consumer object"); + return Nan::ThrowError( + "First argument to 'send offsets to transaction' has to be a consumer object"); // NOLINT } if (!info[1]->IsObject()) { Nan::ThrowError("Kafka consumer must be provided"); @@ -913,8 +925,7 @@ NAN_METHOD(Producer::NodeSendOffsetsToTransaction) { producer, toppars, consumer, - timeout_ms - )); + timeout_ms)); info.GetReturnValue().Set(Nan::Null()); } diff --git a/src/producer.h b/src/producer.h index 974afe85..8df138e8 100644 --- a/src/producer.h +++ b/src/producer.h @@ -14,8 +14,9 @@ #include #include #include +#include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/common.h" #include "src/connection.h" @@ -79,7 +80,8 @@ class Producer : public Connection { void ActivateDispatchers(); void DeactivateDispatchers(); - void ConfigureCallback(const std::string &string_key, const v8::Local &cb, bool add) override; + void ConfigureCallback(const std::string& string_key, + const v8::Local& cb, bool add) override; Baton InitTransactions(int32_t timeout_ms); Baton BeginTransaction(); @@ -88,8 +90,7 @@ class Producer : public Connection { Baton SendOffsetsToTransaction( std::vector &offsets, NodeKafka::KafkaConsumer* consumer, - int timeout_ms - ); + int timeout_ms); protected: static Nan::Persistent constructor; diff --git a/src/topic.h b/src/topic.h index fea91314..d487d089 100644 --- a/src/topic.h +++ b/src/topic.h @@ -13,7 +13,7 @@ #include #include -#include "rdkafkacpp.h" +#include "rdkafkacpp.h" // NOLINT #include "src/config.h" diff --git a/src/workers.cc b/src/workers.cc index af924717..41658a36 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -7,12 +7,11 @@ * This software may be modified and distributed under the terms * of the MIT license. See the LICENSE.txt file for details. */ +#include "src/workers.h" #include #include -#include "src/workers.h" - #ifndef _WIN32 #include #else @@ -361,9 +360,9 @@ void ProducerInitTransactions::HandleErrorCallback() { * @sa NodeKafka::Producer::BeginTransaction */ -ProducerBeginTransaction::ProducerBeginTransaction(Nan::Callback *callback, Producer* producer): - ErrorAwareWorker(callback), - producer(producer) {} +ProducerBeginTransaction::ProducerBeginTransaction(Nan::Callback* callback, + Producer* producer) + : ErrorAwareWorker(callback), producer(producer) {} ProducerBeginTransaction::~ProducerBeginTransaction() {} @@ -513,11 +512,8 @@ ProducerSendOffsetsToTransaction::ProducerSendOffsetsToTransaction( ProducerSendOffsetsToTransaction::~ProducerSendOffsetsToTransaction() {} void ProducerSendOffsetsToTransaction::Execute() { - Baton b = producer->SendOffsetsToTransaction( - m_topic_partitions, - consumer, - m_timeout_ms - ); + Baton b = producer->SendOffsetsToTransaction(m_topic_partitions, consumer, + m_timeout_ms); if (b.err() != RdKafka::ERR_NO_ERROR) { SetErrorBaton(b); @@ -659,9 +655,9 @@ void KafkaConsumerDisconnect::HandleErrorCallback() { * consumer is flagged as disconnected or as unsubscribed. * * @todo thread-safe isConnected checking - * @note Chances are, when the connection is broken with the way librdkafka works, - * we are shutting down. But we want it to shut down properly so we probably - * need the consumer to have a thread lock that can be used when + * @note Chances are, when the connection is broken with the way librdkafka + * works, we are shutting down. But we want it to shut down properly so we + * probably need the consumer to have a thread lock that can be used when * we are dealing with manipulating the `client` * * @sa RdKafka::KafkaConsumer::Consume @@ -677,7 +673,8 @@ KafkaConsumerConsumeLoop::KafkaConsumerConsumeLoop(Nan::Callback *callback, m_looping(true), m_timeout_ms(timeout_ms), m_timeout_sleep_delay_ms(timeout_sleep_delay_ms) { - uv_thread_create(&thread_event_loop, KafkaConsumerConsumeLoop::ConsumeLoop, (void*)this); + uv_thread_create(&thread_event_loop, KafkaConsumerConsumeLoop::ConsumeLoop, + reinterpret_cast(this)); } KafkaConsumerConsumeLoop::~KafkaConsumerConsumeLoop() {} @@ -691,8 +688,9 @@ void KafkaConsumerConsumeLoop::Execute(const ExecutionMessageBus& bus) { // ConsumeLoop is used instead } -void KafkaConsumerConsumeLoop::ConsumeLoop(void *arg) { - KafkaConsumerConsumeLoop* consumerLoop = (KafkaConsumerConsumeLoop*)arg; +void KafkaConsumerConsumeLoop::ConsumeLoop(void* arg) { + KafkaConsumerConsumeLoop* consumerLoop = + reinterpret_cast(arg); ExecutionMessageBus bus(consumerLoop); KafkaConsumer* consumer = consumerLoop->consumer; @@ -730,7 +728,8 @@ void KafkaConsumerConsumeLoop::ConsumeLoop(void *arg) { consumerLoop->m_looping = false; break; } - } else if (ec == RdKafka::ERR_UNKNOWN_TOPIC_OR_PART || ec == RdKafka::ERR_TOPIC_AUTHORIZATION_FAILED) { + } else if (ec == RdKafka::ERR_UNKNOWN_TOPIC_OR_PART || + ec == RdKafka::ERR_TOPIC_AUTHORIZATION_FAILED) { bus.SendWarning(ec); } else { // Unknown error. We need to break out of this @@ -740,7 +739,8 @@ void KafkaConsumerConsumeLoop::ConsumeLoop(void *arg) { } } -void KafkaConsumerConsumeLoop::HandleMessageCallback(RdKafka::Message* msg, RdKafka::ErrorCode ec) { +void KafkaConsumerConsumeLoop::HandleMessageCallback(RdKafka::Message* msg, + RdKafka::ErrorCode ec) { Nan::HandleScope scope; const unsigned int argc = 4; @@ -838,8 +838,9 @@ void KafkaConsumerConsumeNum::Execute() { timeout_ms = 1; } - // We will only go into this code path when `enable.partition.eof` is set to true - // In this case, consumer is also interested in EOF messages, so we return an EOF message + // We will only go into this code path when `enable.partition.eof` + // is set to true. In this case, consumer is also interested in EOF + // messages, so we return an EOF message m_messages.push_back(message); eof_event_count += 1; break; @@ -854,7 +855,8 @@ void KafkaConsumerConsumeNum::Execute() { // This allows getting ready messages, while not waiting for new ones. // This is useful when we want to get the as many messages as possible - // within the timeout but not wait if we already have one or more messages. + // within the timeout but not wait if we already have one or more + // messages. if (m_timeout_only_for_first_message) { timeout_ms = 1; } @@ -897,7 +899,8 @@ void KafkaConsumerConsumeNum::HandleOKCallback() { switch (message->err()) { case RdKafka::ERR_NO_ERROR: ++returnArrayIndex; - Nan::Set(returnArray, returnArrayIndex, Conversion::Message::ToV8Object(message)); + Nan::Set(returnArray, returnArrayIndex, + Conversion::Message::ToV8Object(message)); break; case RdKafka::ERR__PARTITION_EOF: ++eofEventsArrayIndex; @@ -912,10 +915,12 @@ void KafkaConsumerConsumeNum::HandleOKCallback() { Nan::Set(eofEvent, Nan::New("partition").ToLocalChecked(), Nan::New(message->partition())); - // also store index at which position in the message array this event was emitted - // this way, we can later emit it at the right point in time - Nan::Set(eofEvent, Nan::New("messageIndex").ToLocalChecked(), - Nan::New(returnArrayIndex)); + // also store index at which position in the message array this event + // was emitted this way, we can later emit it at the right point in + // time + Nan::Set(eofEvent, + Nan::New("messageIndex").ToLocalChecked(), + Nan::New(returnArrayIndex)); Nan::Set(eofEventsArray, eofEventsArrayIndex, eofEvent); } @@ -1397,7 +1402,8 @@ AdminClientDeleteGroups::~AdminClientDeleteGroups() { } void AdminClientDeleteGroups::Execute() { - Baton b = m_client->DeleteGroups(m_group_list, m_group_cnt, m_timeout_ms, &m_event_response); + Baton b = m_client->DeleteGroups(m_group_list, m_group_cnt, m_timeout_ms, + &m_event_response); if (b.err() != RdKafka::ERR_NO_ERROR) { SetErrorBaton(b); } @@ -1410,7 +1416,8 @@ void AdminClientDeleteGroups::HandleOKCallback() { v8::Local argv[argc]; argv[0] = Nan::Null(); - argv[1] = Conversion::Admin::FromDeleteGroupsResult(rd_kafka_event_DeleteGroups_result(m_event_response)); + argv[1] = Conversion::Admin::FromDeleteGroupsResult( + rd_kafka_event_DeleteGroups_result(m_event_response)); callback->Call(argc, argv); } diff --git a/test/promisified/admin/delete_groups.spec.js b/test/promisified/admin/delete_groups.spec.js index 6c308d91..07fddf9b 100644 --- a/test/promisified/admin/delete_groups.spec.js +++ b/test/promisified/admin/delete_groups.spec.js @@ -7,7 +7,7 @@ const { waitFor, createAdmin, } = require('../testhelpers'); -const { ConsumerGroupStates, ErrorCodes } = require('../../../lib').KafkaJS; +const { ErrorCodes } = require('../../../lib').KafkaJS; describe('Admin > deleteGroups', () => { let topicName, groupId, consumer, admin; diff --git a/test/promisified/admin/list_topics.spec.js b/test/promisified/admin/list_topics.spec.js index 1e0dd638..77a3447b 100644 --- a/test/promisified/admin/list_topics.spec.js +++ b/test/promisified/admin/list_topics.spec.js @@ -3,10 +3,9 @@ jest.setTimeout(30000); const { secureRandom, createTopic, - waitFor, createAdmin, } = require('../testhelpers'); -const { ConsumerGroupStates, ErrorCodes } = require('../../../lib').KafkaJS; +const { ErrorCodes } = require('../../../lib').KafkaJS; describe('Admin > listTopics', () => { let topicNames, admin; diff --git a/test/promisified/consumer/commit.spec.js b/test/promisified/consumer/commit.spec.js index 2f28deaf..2bc3957e 100644 --- a/test/promisified/consumer/commit.spec.js +++ b/test/promisified/consumer/commit.spec.js @@ -1,4 +1,4 @@ -jest.setTimeout(30000) +jest.setTimeout(30000); const { secureRandom, @@ -13,10 +13,10 @@ describe('Consumer commit', () => { let topicName, groupId, producer, consumer; beforeEach(async () => { - topicName = `test-topic-${secureRandom()}` - groupId = `consumer-group-id-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; - await createTopic({ topic: topicName, partitions: 3 }) + await createTopic({ topic: topicName, partitions: 3 }); producer = createProducer({}); @@ -30,8 +30,8 @@ describe('Consumer commit', () => { }); afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); }); it('should commit offsets', async () => { @@ -40,17 +40,17 @@ describe('Consumer commit', () => { const messages = Array(3 * 10) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); await producer.connect(); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); let msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachMessage: async ({ topic, partition, message }) => { msgCount++; @@ -64,7 +64,7 @@ describe('Consumer commit', () => { await consumer.disconnect(); /* Send 30 more messages */ - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); @@ -76,12 +76,12 @@ describe('Consumer commit', () => { msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ - eachMessage: async ({ message }) => { + eachMessage: async () => { msgCount++; } - }) + }); /* Only the extra 30 messages should come to us */ await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); await sleep(1000); @@ -94,18 +94,18 @@ describe('Consumer commit', () => { const messages = Array(3 * 10) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); await producer.connect(); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); let msgCount = 0; const metadata = 'unicode-metadata-😊'; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachMessage: async ({ topic, partition, message }) => { msgCount++; @@ -134,7 +134,7 @@ describe('Consumer commit', () => { msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: 'not-a-real-topic-name' }) + await consumer.subscribe({ topic: 'not-a-real-topic-name' }); /* At this point, we're not actually assigned anything, but we should be able to fetch * the stored offsets and metadata anyway since we're of the same consumer group. */ @@ -157,12 +157,12 @@ describe('Consumer commit', () => { const messages = Array(3 * numMsgs) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); await producer.connect(); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); consumer = createConsumer({ @@ -175,7 +175,7 @@ describe('Consumer commit', () => { let msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachBatchAutoResolve: false, eachBatch: async ({ batch, resolveOffset, commitOffsetsIfNecessary }) => { @@ -222,6 +222,6 @@ describe('Consumer commit', () => { offset: halfOffset, }) ]) - ) + ); }); }); diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index d03d6bce..4e0ad4ee 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -1,4 +1,4 @@ -jest.setTimeout(30000) +jest.setTimeout(30000); const { CompressionTypes } = require('../../../lib').KafkaJS; const { @@ -8,9 +8,9 @@ const { createProducer, createConsumer, waitForMessages, - waitForConsumerToJoinGroup, sleep, } = require('../testhelpers'); +const { Buffer } = require('buffer'); /* All variations of partitionsConsumedConcurrently */ const cases = Array(3).fill().map((_, i) => [(i % 3) + 1]); @@ -21,10 +21,10 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { beforeEach(async () => { console.log("Starting:", expect.getState().currentTestName, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); - topicName = `test-topic-${secureRandom()}` - groupId = `consumer-group-id-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; - await createTopic({ topic: topicName, partitions }) + await createTopic({ topic: topicName, partitions }); producer = createProducer({}); consumer = createConsumer({ @@ -36,15 +36,15 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { }); afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); console.log("Ending:", expect.getState().currentTestName, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); }); it('consume messages', async () => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; consumer.run({ @@ -55,12 +55,12 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const messages = Array(10) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: 0 } - }) + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; + }); - await producer.send({ topic: topicName, messages }) - await waitForMessages(messagesConsumed, { number: messages.length }) + await producer.send({ topic: topicName, messages }); + await waitForMessages(messagesConsumed, { number: messages.length }); expect(messagesConsumed[0]).toEqual( expect.objectContaining({ @@ -72,7 +72,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { offset: '0', }), }) - ) + ); expect(messagesConsumed[messagesConsumed.length - 1]).toEqual( expect.objectContaining({ @@ -84,16 +84,16 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { offset: '' + (messagesConsumed.length - 1), }), }) - ) + ); // check if all offsets are present - expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) + expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)); }); it('consume messages with headers', async () => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; consumer.run({ @@ -110,10 +110,10 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { 'header-4': Buffer.from([1, 0, 1, 0, 1]), }, partition: 0, - }] + }]; - await producer.send({ topic: topicName, messages }) - await waitForMessages(messagesConsumed, { number: messages.length }) + await producer.send({ topic: topicName, messages }); + await waitForMessages(messagesConsumed, { number: messages.length }); expect(messagesConsumed[0]).toEqual( expect.objectContaining({ @@ -131,13 +131,13 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { } }), }) - ) + ); }); it.each([[true], [false]])('consumes messages using eachBatch', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; consumer.run({ @@ -161,12 +161,12 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const messages = Array(100 * partitions) .fill() .map((_, i) => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions } - }) + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions }; + }); - await producer.send({ topic: topicName, messages }) - await waitForMessages(messagesConsumed, { number: messages.length }) + await producer.send({ topic: topicName, messages }); + await waitForMessages(messagesConsumed, { number: messages.length }); for (let p = 0; p < partitions; p++) { const specificPartitionMessages = messagesConsumed.filter(m => m.partition === p); @@ -196,7 +196,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { ); // check if all offsets are present - expect(specificPartitionMessages.map(m => m.message.offset)).toEqual(specificExpectedMessages.map((_, i) => `${i}`)) + expect(specificPartitionMessages.map(m => m.message.offset)).toEqual(specificExpectedMessages.map((_, i) => `${i}`)); } }); @@ -235,9 +235,9 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const messages = Array(messagesPerPartition * partitions) .fill() .map((_, i) => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions } - }) + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions }; + }); await producer.send({ topic: topicName, messages }); @@ -245,12 +245,13 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { * cache growth characteristics, which may change in the future. So just check if there * is at least 1 message more than we sent. */ await waitForMessages(messagesConsumed, { number: messages.length + 1 }); + expect(messagesConsumed.length).toBeGreaterThan(messages.length); }); it('is able to reconsume messages after not resolving it', async () => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); let messageSeen = false; const messagesConsumed = []; @@ -277,9 +278,9 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const messages = Array(1) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: 0 } - }) + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; + }); await producer.send({ topic: topicName, messages }); await waitFor(() => consumer.assignment().length > 0, () => { }, 100); @@ -289,7 +290,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { it.each([[true], [false]])('is able to reconsume messages when an error is thrown', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); let messageSeen = false; const messagesConsumed = []; @@ -313,9 +314,9 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const messages = Array(1) .fill() .map(() => { - const value = secureRandom() + const value = secureRandom(); return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; - }) + }); await producer.send({ topic: topicName, messages }); await waitForMessages(messagesConsumed, { number: messages.length }); @@ -324,7 +325,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { it.each([[true], [false]])('does not reconsume resolved messages even on error', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; consumer.run({ @@ -341,11 +342,11 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const messages = Array(2) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: 0 } - }) + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; + }); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await waitForMessages(messagesConsumed, { number: messages.length }); expect(messagesConsumed[0].key.toString()).toBe(messages[0].key); @@ -356,14 +357,14 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { const partitions = 3; /* We want partitionsConsumedConcurrently to be 2, 3, and 4 rather than 1, 2, and 3 that is tested by the test. */ const partitionsConsumedConcurrentlyDiff = partitionsConsumedConcurrently + 1; - topicName = `test-topic-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; await createTopic({ topic: topicName, partitions: partitions, - }) - await consumer.connect() - await producer.connect() - await consumer.subscribe({ topic: topicName }) + }); + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); let inProgress = 0; let inProgressMaxValue = 0; @@ -374,18 +375,18 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { inProgress++; await sleep(1); messagesConsumed.push(event); - inProgressMaxValue = Math.max(inProgress, inProgressMaxValue) + inProgressMaxValue = Math.max(inProgress, inProgressMaxValue); inProgress--; }, - }) + }); await waitFor(() => consumer.assignment().length > 0, () => { }, 100); const messages = Array(1024*9) .fill() .map((_, i) => { - const value = secureRandom(512) - return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions } + const value = secureRandom(512); + return { key: `key-${value}`, value: `value-${value}`, partition: i % partitions }; }); await producer.send({ topic: topicName, messages }); @@ -435,68 +436,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { offset: '1', }), }), - ]) - }); - - /* Skip as it uses consumer events. */ - 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 () => { @@ -507,7 +447,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { let calls = 0; consumer.run({ - eachMessage: async event => { + eachMessage: async () => { calls++; await sleep(100); }, @@ -525,54 +465,6 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { expect(calls).toEqual(1); }); - /* Skip as it uses consumer events */ - 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) - }); - it('does not disconnect in the middle of message processing', async () => { await producer.connect(); await consumer.connect(); @@ -586,7 +478,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { await sleep(7000); try { consumer.seek({ topic: topicName, partition: 0, offset: message.offset }); - } catch (e) { + } catch { failedSeek = true; } calls++; diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index f40b775d..0e15a8d6 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -1,4 +1,4 @@ -jest.setTimeout(30000) +jest.setTimeout(30000); const { secureRandom, @@ -23,10 +23,10 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed beforeEach(async () => { console.log("Starting:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); - topicName = `test-topic-${secureRandom()}` - groupId = `consumer-group-id-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; - await createTopic({ topic: topicName, partitions: 3 }) + await createTopic({ topic: topicName, partitions: 3 }); producer = createProducer({}); @@ -39,15 +39,15 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed }); afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); console.log("Ending:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); }); it('is cleared on pause', async () => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const msgs = 1024; const messagesConsumed = []; @@ -66,11 +66,11 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed const messages = Array(msgs * 9) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: ((i++) % 3) } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: ((i++) % 3) }; + }); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); // Wait for the messages. // We consume msgs*1 messages from partition 0, and msgs*3 from partition 1 and 2. @@ -92,7 +92,7 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed it('is cleared on seek', async () => { await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; let hasBeenSeeked = false; @@ -112,11 +112,11 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed const messages = Array(1024 * 9) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: ((i++) % 3) } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: ((i++) % 3) }; + }); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); // Wait for the messages. // We consume 1024*4 messages from partition 0, and 1024*3 from partition 1 and 2. @@ -159,7 +159,7 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed await consumer.connect(); await producer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; const messagesConsumedConsumer1 = []; @@ -190,11 +190,11 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed const messages = Array(1024 * multiplier) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); // Wait for the messages - some of them, before starting the // second consumer. @@ -247,7 +247,7 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed await producer.connect(); await impatientConsumer.connect(); - await impatientConsumer.subscribe({ topic: topicName }) + await impatientConsumer.subscribe({ topic: topicName }); const messagesConsumed = []; let impatientConsumerMessages = []; @@ -280,11 +280,11 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed const messages = Array(1024 * 10) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); /* Wait for the messages - some of them, before starting the * second consumer. diff --git a/test/promisified/consumer/consumerTransactions.spec.js b/test/promisified/consumer/consumerTransactions.spec.js index 4afd8bbf..980a7514 100644 --- a/test/promisified/consumer/consumerTransactions.spec.js +++ b/test/promisified/consumer/consumerTransactions.spec.js @@ -14,10 +14,10 @@ describe('Consumer transactions', () => { let topicName, groupId, producer, consumer; beforeEach(async () => { - topicName = `test-topic-${secureRandom()}` - groupId = `consumer-group-id-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; - await createTopic({ topic: topicName }) + await createTopic({ topic: topicName }); producer = createProducer({ idempotent: true, maxInFlightRequests: 1, @@ -32,8 +32,8 @@ describe('Consumer transactions', () => { }); afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); }); it('accepts messages from an idempotent producer', async () => { @@ -41,25 +41,25 @@ describe('Consumer transactions', () => { await producer.connect(); await consumer.subscribe({ topic: topicName }); - const messagesConsumed = [] - const idempotentMessages = generateMessages({ prefix: 'idempotent', partition: 0 }) + const messagesConsumed = []; + const idempotentMessages = generateMessages({ prefix: 'idempotent', partition: 0 }); 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/) + 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 () => { @@ -106,10 +106,10 @@ describe('Consumer transactions', () => { 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/) + 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 () => { @@ -122,7 +122,7 @@ describe('Consumer transactions', () => { await producer.connect(); await consumer.subscribe({ topic: topicName }); - const messagesConsumed = [] + const messagesConsumed = []; const abortedMessages1 = generateMessages({ prefix: 'aborted-txn-1', partition: 0 }); const abortedMessages2 = generateMessages({ prefix: 'aborted-txn-2', partition: 0 }); @@ -150,7 +150,7 @@ describe('Consumer transactions', () => { }); await committedTxn.commit(); - const number = committedMessages.length + const number = committedMessages.length; await waitForMessages(messagesConsumed, { number, }); @@ -166,7 +166,7 @@ describe('Consumer transactions', () => { producer = createProducer({ transactionalId: `transactional-id-${secureRandom()}`, maxInFlightRequests: 1, - }) + }); consumer = createConsumer({ groupId, @@ -174,7 +174,7 @@ describe('Consumer transactions', () => { readUncommitted: true, fromBeginning: true, autoCommit: true, - }) + }); await consumer.connect(); await producer.connect(); @@ -221,14 +221,14 @@ describe('Consumer transactions', () => { await producer.send({ topic: topicName, messages, - }) + }); await producer.disconnect(); producer = createProducer({ transactionalId: `transactional-id-${secureRandom()}`, maxInFlightRequests: 1, - }) + }); consumer = createConsumer({ groupId, @@ -248,8 +248,8 @@ describe('Consumer transactions', () => { let uncommittedOffsetsPerMessage = []; let latestOffsetsPerPartition = {}; - const eachMessage = async ({ topic, partition, message }) => { - messagesConsumed.push(message) + const eachMessage = async ({ 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; @@ -258,18 +258,18 @@ describe('Consumer transactions', () => { consumer.run({ eachMessage, - }) + }); // 2. 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) + 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(); @@ -301,12 +301,12 @@ describe('Consumer transactions', () => { messagesConsumed = []; uncommittedOffsetsPerMessage = []; - consumer.run({ eachMessage }) + 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/); @@ -329,14 +329,14 @@ describe('Consumer transactions', () => { await producer.send({ topic: topicName, messages, - }) + }); await producer.disconnect(); producer = createProducer({ transactionalId: `transactional-id-${secureRandom()}`, maxInFlightRequests: 1, - }) + }); consumer = createConsumer({ groupId, @@ -356,8 +356,8 @@ describe('Consumer transactions', () => { let uncommittedOffsetsPerMessage = []; let latestOffsetsPerPartition = {}; - const eachMessage = async ({ topic, partition, message }) => { - messagesConsumed.push(message) + const eachMessage = async ({ 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; @@ -366,7 +366,7 @@ describe('Consumer transactions', () => { consumer.run({ eachMessage, - }) + }); // Consume produced messages. await waitForMessages(messagesConsumed, { number: messages.length }); @@ -387,11 +387,11 @@ describe('Consumer transactions', () => { consumer, topics: [topicPartitionOffsets], }); - await txnToAbort.abort() + await txnToAbort.abort(); /* Restart consumer - we cannot stop it, so we recreate it. */ - messagesConsumed = [] - uncommittedOffsetsPerMessage = [] + messagesConsumed = []; + uncommittedOffsetsPerMessage = []; await consumer.disconnect(); @@ -410,9 +410,9 @@ describe('Consumer transactions', () => { }); await waitForMessages(messagesConsumed, { number: 1 }); - expect(messagesConsumed[0].value.toString()).toMatch(/value-0/) + expect(messagesConsumed[0].value.toString()).toMatch(/value-0/); await waitForMessages(messagesConsumed, { number: messages.length }); - expect(messagesConsumed[messagesConsumed.length - 1].value.toString()).toMatch(/value-99/) + expect(messagesConsumed[messagesConsumed.length - 1].value.toString()).toMatch(/value-99/); } ); }); \ No newline at end of file diff --git a/test/promisified/consumer/groupInstanceId.spec.js b/test/promisified/consumer/groupInstanceId.spec.js index b3477888..bf814000 100644 --- a/test/promisified/consumer/groupInstanceId.spec.js +++ b/test/promisified/consumer/groupInstanceId.spec.js @@ -15,7 +15,7 @@ describe('Consumer with static membership', () => { beforeEach(async () => { topicName = `test-topic1-${secureRandom()}`; - groupId = `consumer-group-id-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}`; consumerConfig = { groupId, }; @@ -24,13 +24,13 @@ describe('Consumer with static membership', () => { }); afterEach(async () => { - consumer && (await consumer.disconnect()) + consumer && (await consumer.disconnect()); }); it('does not rebalance after disconnect', async () => { let assigns = 0; let revokes = 0; - const rebalanceCallback = function (err, assignment) { + const rebalanceCallback = function (err) { if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { assigns++; } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { @@ -39,7 +39,7 @@ describe('Consumer with static membership', () => { // It's either assign or revoke and nothing else. jest.fail('Unexpected error code'); } - } + }; // Create and start two consumers. consumer = createConsumer(consumerConfig, { @@ -84,7 +84,7 @@ describe('Consumer with static membership', () => { it('does rebalance after session timeout', async () => { let assigns = 0; let revokes = 0; - const rebalanceCallback = function (err, assignment) { + const rebalanceCallback = function (err) { if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { assigns++; } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { @@ -93,7 +93,7 @@ describe('Consumer with static membership', () => { // It's either assign or revoke and nothing else. jest.fail('Unexpected error code'); } - } + }; // Create and start two consumers. consumer = createConsumer(consumerConfig, { diff --git a/test/promisified/consumer/incrementalRebalance.spec.js b/test/promisified/consumer/incrementalRebalance.spec.js index f62e7483..37463739 100644 --- a/test/promisified/consumer/incrementalRebalance.spec.js +++ b/test/promisified/consumer/incrementalRebalance.spec.js @@ -3,7 +3,6 @@ jest.setTimeout(30000); const { waitFor, secureRandom, createTopic, - createProducer, createConsumer, } = require("../testhelpers"); const { PartitionAssigners, ErrorCodes } = require('../../../lib').KafkaJS; @@ -18,13 +17,13 @@ describe('Consumer > incremental rebalance', () => { beforeEach(async () => { topicName = `test-topic1-${secureRandom()}`; - groupId = `consumer-group-id-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}`; consumer = null; await createTopic({ topic: topicName, partitions: 2 }); }); afterEach(async () => { - consumer && (await consumer.disconnect()) + consumer && (await consumer.disconnect()); }); it('returns protocol name', async () => { @@ -52,7 +51,7 @@ describe('Consumer > incremental rebalance', () => { // It's either assign or revoke and nothing else. jest.fail('Unexpected error code'); } - } + }; consumer = createConsumer(consumerConfig, { @@ -84,7 +83,7 @@ describe('Consumer > incremental rebalance', () => { // It's either assign or revoke and nothing else. expect(err.code).toBe(ErrorCodes.ERR__REVOKE_PARTITIONS); } - } + }; consumer = createConsumer(consumerConfig, { @@ -113,7 +112,7 @@ describe('Consumer > incremental rebalance', () => { // It's either assign or revoke and nothing else. jest.fail('Unexpected error code'); } - } + }; /* First consumer joins and gets all partitions. */ expectedAssignmentCount = 2; diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index 18d8dd71..af720834 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -16,11 +16,11 @@ describe('Consumer', () => { beforeEach(async () => { console.log("Starting:", expect.getState().currentTestName); - topics = [`test-topic1-${secureRandom()}`, `test-topic2-${secureRandom()}`] - groupId = `consumer-group-id-${secureRandom()}` + topics = [`test-topic1-${secureRandom()}`, `test-topic2-${secureRandom()}`]; + groupId = `consumer-group-id-${secureRandom()}`; for (const topic of topics) { - await createTopic({ topic, partitions: 2 }) + await createTopic({ topic, partitions: 2 }); } producer = createProducer({ @@ -32,13 +32,13 @@ describe('Consumer', () => { maxBytesPerPartition: 180, fromBeginning: true, }); - }) + }); afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); console.log("Ending:", expect.getState().currentTestName); - }) + }); describe('when pausing', () => { it('throws an error if the topic is invalid', async () => { @@ -56,9 +56,9 @@ describe('Consumer', () => { /* 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 } - }) + const key = secureRandom(); + return { key: `key-${key}`, value: `value-${key}`, partition }; + }); /* Send the first 2 messages to each topic. */ for (const topic of topics) { @@ -74,8 +74,8 @@ describe('Consumer', () => { eachMessage: async event => { const { topic, message, pause } = event; - const whichTopic = topics.indexOf(topic) - const whichMessage = messages.findIndex(m => String(m.key) === String(message.key)) + 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. */ @@ -98,113 +98,113 @@ describe('Consumer', () => { * 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] }]) + expect(consumer.paused()).toEqual([{ topic: pausedTopic, partitions: [0] }]); for (const topic of topics) { - await producer.send({ topic, messages: messages.slice(2) }) + await producer.send({ topic, messages: messages.slice(2) }); } - await waitForMessages(messagesConsumed, { number: 6, delay: 10 }) + 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).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 + 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 }) + 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 + 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() + 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 } - }) + 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 producer.send({ topic, messages: messages.slice(0, 2) }); } await consumer.subscribe({ topics, replace: true }); - let shouldPause = true - const messagesConsumed = [] + 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)) + 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] }]) + 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] }]) + 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 producer.send({ topic, messages: messages.slice(2) }); } - await waitForMessages(messagesConsumed, { number: 6, delay: 10 }) + 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).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 + 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()) + shouldPause = false; + consumer.resume(consumer.paused()); - await waitForMessages(messagesConsumed, { number: 8 }) + 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 + expect(messagesConsumed).toHaveLength(8); + expect(messagesConsumed).toContainEqual({ topic: 0, message: 1 }); // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 0, message: 3 }); // partition 0 }, 15000); it('pauses when pausing via the eachBatch callback', async () => { - await consumer.connect() - await producer.connect() + 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 } - }) + 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 }) + await producer.send({ topic, messages: originalMessages }); + await consumer.subscribe({ topic }); } - let shouldPause = true - const messagesConsumed = [] - const resumeCallbacks = [] + let shouldPause = true; + const messagesConsumed = []; + const resumeCallbacks = []; consumer.run({ eachBatch: async event => { const { @@ -212,42 +212,42 @@ describe('Consumer', () => { pause, resolveOffset, commitOffsetsIfNecessary, - } = event + } = event; messages.every(message => { - const whichTopic = topics.indexOf(topic) + 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 + resumeCallbacks.push(pause()); + return false; } else if (shouldPause && whichTopic === 1 && whichMessage === 3) { - resumeCallbacks.push(pause()) - return false + resumeCallbacks.push(pause()); + return false; } messagesConsumed.push({ topic: whichTopic, message: whichMessage, - }) - resolveOffset(message.offset) - return true - }) - await commitOffsetsIfNecessary() + }); + resolveOffset(message.offset); + return true; + }); + await commitOffsetsIfNecessary(); }, eachBatchAutoResolve: false, - }) - await waitForConsumerToJoinGroup(consumer) - await waitForMessages(messagesConsumed, { number: 5 }) + }); + await waitForConsumerToJoinGroup(consumer); + await waitForMessages(messagesConsumed, { number: 5 }); expect(messagesConsumed.length).toEqual(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 }) + 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 }); }, 10000); it('does not fetch messages for the paused topic', async () => { @@ -286,7 +286,7 @@ describe('Consumer', () => { }), ]); - const byPartition = (a, b) => a.partition - b.partition + const byPartition = (a, b) => a.partition - b.partition; expect( consumedMessages.filter(({ topic }) => topic === activeTopic).sort(byPartition) ).toEqual([ @@ -320,8 +320,8 @@ describe('Consumer', () => { const messages = Array(1) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}` }; }); const forPartition = partition => message => ({ ...message, partition }); @@ -330,7 +330,7 @@ describe('Consumer', () => { } await consumer.subscribe({ topic }); - const messagesConsumed = [] + const messagesConsumed = []; consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); await waitForMessages(messagesConsumed, { number: messages.length * partitions.length }); @@ -392,10 +392,10 @@ describe('Consumer', () => { consumer.run({ eachMessage: async event => { - messagesConsumed.push(event) + messagesConsumed.push(event); if (shouldThrow) { - consumer.pause([{ topic }]) - throw new Error('Should fail') + consumer.pause([{ topic }]); + throw new Error('Should fail'); } }, }); @@ -407,17 +407,17 @@ describe('Consumer', () => { shouldThrow = false; consumer.resume([{ topic }]); - const consumedMessages = await waitForMessages(messagesConsumed, { number: 2 }) + const consumedMessages = await waitForMessages(messagesConsumed, { number: 2 }); - expect(consumedMessagesTillError).toHaveLength(1) + 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).toHaveLength(2); expect(consumedMessages).toEqual([ expect.objectContaining({ topic, @@ -429,7 +429,7 @@ describe('Consumer', () => { partition: expect.any(Number), message: expect.objectContaining({ offset: '0' }), }), - ]) + ]); }, 10000); it('does not process messages when consumption from topic-partition is paused', async () => { @@ -445,15 +445,15 @@ describe('Consumer', () => { await consumer.connect(); await producer.connect(); - await producer.send({ topic, messages: [message1, message2] }) - await consumer.subscribe({ topic }) + await producer.send({ topic, messages: [message1, message2] }); + await consumer.subscribe({ topic }); consumer.run({ eachMessage: async event => { - messagesConsumed.push(event) + messagesConsumed.push(event); if (shouldThrow && event.partition === pausedPartition) { - consumer.pause([{ topic, partitions: [pausedPartition] }]) - throw new Error('Should fail') + consumer.pause([{ topic, partitions: [pausedPartition] }]); + throw new Error('Should fail'); } }, }); @@ -532,7 +532,7 @@ describe('Consumer', () => { await producer.send({ topic: topic2, messages: [message1] }); expect(eachMessage).not.toHaveBeenCalled(); - }) + }); }); describe('when resuming', () => { @@ -600,8 +600,8 @@ describe('Consumer', () => { const messages = Array(1) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}` }; }); const forPartition = partition => message => ({ ...message, partition }); @@ -640,7 +640,7 @@ describe('Consumer', () => { message: expect.objectContaining({ offset: `${i}` }), }) ) - ) + ); expect(consumedMessages.filter(({ partition }) => partition !== pausedPartition)).toEqual( messages.concat(messages).map((message, i) => @@ -650,9 +650,9 @@ describe('Consumer', () => { message: expect.objectContaining({ offset: `${i}` }), }) ) - ) + ); - expect(consumer.paused()).toEqual([]) + expect(consumer.paused()).toEqual([]); }, 10000); }); -}) +}); diff --git a/test/promisified/consumer/seek.spec.js b/test/promisified/consumer/seek.spec.js index c400ed3b..17e1c749 100644 --- a/test/promisified/consumer/seek.spec.js +++ b/test/promisified/consumer/seek.spec.js @@ -8,7 +8,7 @@ const { waitForMessages, waitFor, sleep, -} = require('../testhelpers') +} = require('../testhelpers'); describe('Consumer seek >', () => { let topicName, groupId, producer, consumer; @@ -35,7 +35,7 @@ describe('Consumer seek >', () => { describe('when seek offset', () => { describe('with one partition', () => { beforeEach(async () => { - await createTopic({ topic: topicName, partitions: 1 }) + await createTopic({ topic: topicName, partitions: 1 }); }); it('throws an error if the topic is invalid', async () => { @@ -105,8 +105,8 @@ describe('Consumer seek >', () => { }); await consumer.subscribe({ topic: topicName }); - let messagesConsumed = [] - consumer.seek({ topic: topicName, partition: 0, offset: 2 }) + let messagesConsumed = []; + consumer.seek({ topic: topicName, partition: 0, offset: 2 }); consumer.run({ eachMessage: async event => messagesConsumed.push(event), }); @@ -159,7 +159,7 @@ describe('Consumer seek >', () => { describe('with two partitions', () => { beforeEach(async () => { - await createTopic({ topic: topicName, partitions: 2 }) + await createTopic({ topic: topicName, partitions: 2 }); }); it('updates the partition offset to the given offset', async () => { @@ -182,7 +182,7 @@ describe('Consumer seek >', () => { await consumer.subscribe({ topic: topicName }); - const messagesConsumed = [] + const messagesConsumed = []; consumer.seek({ topic: topicName, partition: 1, offset: 1 }); consumer.run({ eachMessage: async event => { @@ -190,9 +190,9 @@ describe('Consumer seek >', () => { } }); - let check = await expect(waitForMessages(messagesConsumed, { number: 3 })).resolves; + let check = await waitForMessages(messagesConsumed, { number: 3 }); - await check.toEqual( + expect(check).toEqual( expect.arrayContaining([ expect.objectContaining({ topic: topicName, @@ -212,7 +212,7 @@ describe('Consumer seek >', () => { ]) ); - await check.toEqual( + expect(check).toEqual( expect.not.arrayContaining([ expect.objectContaining({ topic: topicName, @@ -243,7 +243,7 @@ describe('Consumer seek >', () => { topic: topicName, messages: [message1, message2, message3, message4, message5], }); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); const messagesConsumed = []; consumer.seek({ topic: topicName, partition: 0, offset: 2 }); @@ -254,9 +254,9 @@ describe('Consumer seek >', () => { } }); - let check = await expect(waitForMessages(messagesConsumed, { number: 2 })).resolves; + let check = await waitForMessages(messagesConsumed, { number: 2 }); - await check.toEqual( + expect(check).toEqual( expect.arrayContaining([ expect.objectContaining({ topic: topicName, @@ -271,7 +271,7 @@ describe('Consumer seek >', () => { ]) ); - await check.toEqual( + expect(check).toEqual( expect.not.arrayContaining([ expect.objectContaining({ topic: topicName, @@ -294,20 +294,20 @@ describe('Consumer seek >', () => { }); it('uses the last seek for a given topic/partition', async () => { - await consumer.connect() - await producer.connect() + await consumer.connect(); + await producer.connect(); - const value1 = secureRandom() - const message1 = { key: `key-0`, value: `value-${value1}`, partition: 0 } - const value2 = secureRandom() - const message2 = { key: `key-0`, value: `value-${value2}`, partition: 0 } - const value3 = secureRandom() - const message3 = { key: `key-0`, value: `value-${value3}`, partition: 0 } + const value1 = secureRandom(); + const message1 = { key: `key-0`, value: `value-${value1}`, partition: 0 }; + const value2 = secureRandom(); + const message2 = { key: `key-0`, value: `value-${value2}`, partition: 0 }; + const value3 = secureRandom(); + const message3 = { key: `key-0`, value: `value-${value3}`, partition: 0 }; - await producer.send({ topic: topicName, messages: [message1, message2, message3] }) - await consumer.subscribe({ topic: topicName, }) + await producer.send({ topic: topicName, messages: [message1, message2, message3] }); + await consumer.subscribe({ topic: topicName, }); - const messagesConsumed = [] + 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 }); @@ -317,9 +317,9 @@ describe('Consumer seek >', () => { } }); - let check = await expect(waitForMessages(messagesConsumed, { number: 1 })).resolves; + let check = await waitForMessages(messagesConsumed, { number: 1 }); - await check.toEqual( + expect(check).toEqual( expect.arrayContaining([ expect.objectContaining({ topic: topicName, @@ -329,7 +329,7 @@ describe('Consumer seek >', () => { ]) ); - await check.toEqual( + expect(check).toEqual( expect.not.arrayContaining([ expect.objectContaining({ topic: topicName, @@ -358,8 +358,8 @@ describe('Consumer seek >', () => { const messages = Array(10) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: 0 } + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; }); await consumer.connect(); @@ -371,17 +371,17 @@ describe('Consumer seek >', () => { consumer.run({ eachMessage: async ({ message }) => { - offsetsConsumed.push(message.offset) + offsetsConsumed.push(message.offset); if (offsetsConsumed.length === 1) { consumer.seek({ topic: topicName, partition: 0, offset: message.offset }); } }, - }) + }); - await waitFor(() => offsetsConsumed.length >= 2, () => { }, { delay: 50 }) + await waitFor(() => offsetsConsumed.length >= 2, () => { }, { delay: 50 }); - expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]); }); it('resolves a batch as stale when seek was called while processing it', async () => { @@ -392,13 +392,13 @@ describe('Consumer seek >', () => { maxWaitTimeInMs: 500, fromBeginning: true, autoCommit: true, - }) + }); const messages = Array(10) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}` }; }); await consumer.connect(); @@ -413,20 +413,20 @@ describe('Consumer seek >', () => { for (const message of batch.messages) { if (isStale()) break; - offsetsConsumed.push(message.offset) + offsetsConsumed.push(message.offset); if (offsetsConsumed.length === 1) { - consumer.seek({ topic: topicName, partition: 0, offset: +message.offset }) + consumer.seek({ topic: topicName, partition: 0, offset: +message.offset }); } - resolveOffset(message.offset) + resolveOffset(message.offset); } }, - }) + }); - await waitFor(() => offsetsConsumed.length >= 2, () => null, { delay: 50 }) + await waitFor(() => offsetsConsumed.length >= 2, () => null, { delay: 50 }); - expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]); }); it('resolves a batch as stale when seek is called from outside eachBatch', async () => { @@ -435,13 +435,13 @@ describe('Consumer seek >', () => { maxWaitTimeInMs: 500, fromBeginning: true, autoCommit: true, - }) + }); const messages = Array(10) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}`, partition: 0 } + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}`, partition: 0 }; }); await consumer.connect(); @@ -456,22 +456,22 @@ describe('Consumer seek >', () => { for (const message of batch.messages) { if (isStale()) break; - offsetsConsumed.push(message.offset) + offsetsConsumed.push(message.offset); /* Slow things down so we can call seek predictably. */ await sleep(1000); - resolveOffset(message.offset) + resolveOffset(message.offset); } }, - }) + }); - await waitFor(() => offsetsConsumed.length === 1, () => null, { delay: 50 }) + await waitFor(() => offsetsConsumed.length === 1, () => null, { delay: 50 }); consumer.seek({ topic: topicName, partition: 0, offset: offsetsConsumed[0] }); - await waitFor(() => offsetsConsumed.length >= 2, () => null, { delay: 50 }) + await waitFor(() => offsetsConsumed.length >= 2, () => null, { delay: 50 }); - expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]); }); it('resolves a batch as stale when pause was called while processing it', async () => { @@ -480,14 +480,14 @@ describe('Consumer seek >', () => { maxWaitTimeInMs: 500, fromBeginning: true, autoCommit: true, - }) + }); const numMessages = 100; const messages = Array(numMessages) .fill() .map(() => { - const value = secureRandom() - return { key: `key-${value}`, value: `value-${value}` } + const value = secureRandom(); + return { key: `key-${value}`, value: `value-${value}` }; }); await consumer.connect(); @@ -504,7 +504,7 @@ describe('Consumer seek >', () => { for (const message of batch.messages) { if (isStale()) break; - offsetsConsumed.push(message.offset) + offsetsConsumed.push(message.offset); if (offsetsConsumed.length === Math.floor(numMessages/2)) { resume = pause(); @@ -513,7 +513,7 @@ describe('Consumer seek >', () => { resolveOffset(message.offset); } }, - }) + }); /* Despite eachBatchAutoResolve being true, it shouldn't resolve offsets on its own. * However, manual resolution of offsets should still count. */ @@ -523,60 +523,10 @@ describe('Consumer seek >', () => { /* Since we've properly resolved all offsets before pause, including the offset that we paused at, * there is no repeat. */ - await waitFor(() => offsetsConsumed.length >= numMessages, () => null, { delay: 50 }) + await waitFor(() => offsetsConsumed.length >= numMessages, () => null, { delay: 50 }); expect(offsetsConsumed.length).toBe(numMessages); - expect(+offsetsConsumed[Math.floor(numMessages/2)]).toEqual(+offsetsConsumed[Math.floor(numMessages/2) + 1] - 1) - }); - - /* Skip as it uses consumer events */ - 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) + expect(+offsetsConsumed[Math.floor(numMessages/2)]).toEqual(+offsetsConsumed[Math.floor(numMessages/2) + 1] - 1); }); }); -}) +}); diff --git a/test/promisified/consumer/store.spec.js b/test/promisified/consumer/store.spec.js index f69b9fc6..64d67ffd 100644 --- a/test/promisified/consumer/store.spec.js +++ b/test/promisified/consumer/store.spec.js @@ -1,4 +1,4 @@ -jest.setTimeout(30000) +jest.setTimeout(30000); const { secureRandom, @@ -14,10 +14,10 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { let topicName, groupId, producer, consumer; beforeEach(async () => { - topicName = `test-topic-${secureRandom()}` - groupId = `consumer-group-id-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; - await createTopic({ topic: topicName, partitions: 3 }) + await createTopic({ topic: topicName, partitions: 3 }); producer = createProducer({}); @@ -33,8 +33,8 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { }); afterEach(async () => { - consumer && (await consumer.disconnect()) - producer && (await producer.disconnect()) + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); }); it('should not work if enable.auto.offset.store = true', async () => { @@ -54,7 +54,7 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { }); await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachMessage: async () => { } @@ -81,7 +81,7 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { }); await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachMessage: async () => { } @@ -98,17 +98,17 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { const messages = Array(3 * 10) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); await producer.connect(); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); let msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachMessage: async ({ topic, partition, message }) => { msgCount++; @@ -127,7 +127,7 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { await consumer.disconnect(); /* Send 30 more messages */ - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); @@ -139,12 +139,12 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ - eachMessage: async ({ message }) => { + eachMessage: async () => { msgCount++; } - }) + }); /* Only the extra 30 messages should come to us */ await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); await sleep(1000); @@ -157,18 +157,18 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { const messages = Array(3 * 10) .fill() .map(() => { - const value = secureRandom() - return { value: `value-${value}`, partition: (i++) % 3 } - }) + const value = secureRandom(); + return { value: `value-${value}`, partition: (i++) % 3 }; + }); await producer.connect(); - await producer.send({ topic: topicName, messages }) + await producer.send({ topic: topicName, messages }); await producer.flush(); let msgCount = 0; const metadata = 'unicode-metadata-😊'; await consumer.connect(); - await consumer.subscribe({ topic: topicName }) + await consumer.subscribe({ topic: topicName }); await consumer.run({ eachMessage: async ({ topic, partition, message }) => { msgCount++; @@ -202,7 +202,7 @@ describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { msgCount = 0; await consumer.connect(); - await consumer.subscribe({ topic: 'not-a-real-topic-name' }) + await consumer.subscribe({ topic: 'not-a-real-topic-name' }); /* At this point, we're not actually assigned anything, but we should be able to fetch * the stored offsets and metadata anyway since we're of the same consumer group. */ diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index 34727e0d..8619fe76 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -1,6 +1,6 @@ jest.setTimeout(30000); -const { Kafka, ErrorCodes } = require('../../../lib').KafkaJS; +const { ErrorCodes } = require('../../../lib').KafkaJS; const { secureRandom, createTopic, waitFor, @@ -22,12 +22,12 @@ describe('Consumer', () => { }); 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 () => { @@ -68,7 +68,7 @@ describe('Consumer', () => { expect.arrayContaining(['drink', 'your', 'tea']) ); }); - }) + }); describe('Deprecated "topic" interface', () => { describe('when subscribing', () => { @@ -90,7 +90,7 @@ describe('Consumer', () => { describe('with a string', () => { it('subscribes to the topic', async () => { - const topic = `topic-${secureRandom()}` + const topic = `topic-${secureRandom()}`; await createTopic({ topic }); diff --git a/test/promisified/oauthbearer_cb.spec.js b/test/promisified/oauthbearer_cb.spec.js index d33f4eb5..d736646c 100644 --- a/test/promisified/oauthbearer_cb.spec.js +++ b/test/promisified/oauthbearer_cb.spec.js @@ -19,7 +19,7 @@ describe('Client > oauthbearer callback', () => { beforeEach(async () => { oauthbearer_cb_called = 0; - }) + }); it('works for producer', async () => { @@ -77,4 +77,4 @@ describe('Client > oauthbearer callback', () => { } ); -}) +}); diff --git a/test/promisified/producer/concurrentSend.spec.js b/test/promisified/producer/concurrentSend.spec.js index 1fc29b30..e793e0f0 100644 --- a/test/promisified/producer/concurrentSend.spec.js +++ b/test/promisified/producer/concurrentSend.spec.js @@ -19,11 +19,11 @@ describe('Producer', () => { topicName = `test-topic-${secureRandom()}`; await createTopic({ topic: topicName, partitions: 3 }); - }) + }); afterEach(async () => { producer && (await producer.disconnect()); - }) + }); it('can send messages concurrently', @@ -42,4 +42,4 @@ describe('Producer', () => { } } ); -}) +}); diff --git a/test/promisified/producer/concurrentTransaction.spec.js b/test/promisified/producer/concurrentTransaction.spec.js index d530c1a7..3cfbc6a4 100644 --- a/test/promisified/producer/concurrentTransaction.spec.js +++ b/test/promisified/producer/concurrentTransaction.spec.js @@ -2,7 +2,7 @@ const { secureRandom, createProducer, createTopic, -} = require('../testhelpers') +} = require('../testhelpers'); describe('Producer > Transactional producer', () => { let producer1, producer2, topicName, transactionalId, message; @@ -20,12 +20,12 @@ describe('Producer > Transactional producer', () => { 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', @@ -45,6 +45,6 @@ describe('Producer > Transactional producer', () => { await transaction2.send({ topic: topicName, messages: [message] }); await transaction2.commit(); } - ) - }) -}) + ); + }); +}); diff --git a/test/promisified/producer/flush.spec.js b/test/promisified/producer/flush.spec.js index debee0fe..c4f7daf9 100644 --- a/test/promisified/producer/flush.spec.js +++ b/test/promisified/producer/flush.spec.js @@ -22,11 +22,11 @@ describe('Producer > Flush', () => { message = { key: `key-${secureRandom()}`, value: `value-${secureRandom()}` }; await createTopic({ topic: topicName }); - }) + }); afterEach(async () => { producer && (await producer.disconnect()); - }) + }); it('does not wait for linger.ms', @@ -85,4 +85,4 @@ describe('Producer > Flush', () => { } ); -}) +}); diff --git a/test/promisified/producer/idempotentProduceMessage.spec.js b/test/promisified/producer/idempotentProduceMessage.spec.js index 00d0de18..d882f977 100644 --- a/test/promisified/producer/idempotentProduceMessage.spec.js +++ b/test/promisified/producer/idempotentProduceMessage.spec.js @@ -1,4 +1,4 @@ -jest.setTimeout(10000) +jest.setTimeout(10000); const { secureRandom, @@ -16,21 +16,21 @@ describe('Producer > Idempotent producer', () => { messages = Array(4) .fill() .map((_, i) => { - const value = secureRandom() - return { key: `key-${value}`, value: `${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, fromBeginning: true, - }) + }); await createTopic({ topic: topicName, partitions: 1 }); await Promise.all([producer.connect(), consumer.connect()]); await consumer.subscribe({ topic: topicName }); @@ -48,7 +48,7 @@ describe('Producer > Idempotent producer', () => { const messagesConsumed = []; for (const m of messages) { - await producer.send({ topic: topicName, messages: [m] }) + await producer.send({ topic: topicName, messages: [m] }); } await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }); @@ -62,146 +62,68 @@ describe('Producer > Idempotent producer', () => { /* 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 broker = await cluster.findBroker({ nodeId }); - const brokerProduce = jest.spyOn(broker, 'produce') + const brokerProduce = jest.spyOn(broker, 'produce'); brokerProduce.mockImplementationOnce(() => { - throw new KafkaJSError('retriable error') - }) + throw new KafkaJSError('retriable error'); + }); } - const messagesConsumed = [] + const messagesConsumed = []; for (const m of messages) { - await producer.send({ acks: -1, topic: topicName, messages: [m] }) + await producer.send({ acks: -1, topic: topicName, messages: [m] }); } - await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }); - await waitForMessages(messagesConsumed, { number: messages.length }) + 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() + 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') - }) + await originalCall(...args); + throw new KafkaJSError('retriable error'); + }); } - const messagesConsumed = [] + const messagesConsumed = []; for (const m of messages) { - await producer.send({ acks: -1, topic: topicName, messages: [m] }) + await producer.send({ acks: -1, topic: topicName, messages: [m] }); } - await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }); - await waitForMessages(messagesConsumed, { number: messages.length }) + 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 = [] + const messagesConsumed = []; await Promise.all( messages.map(m => producer.send({ topic: topicName, messages: [m] })) - ) + ); - await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }); - await waitForMessages(messagesConsumed, { number: messages.length }) - expect(messagesConsumed).toHaveLength(messages.length) + 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 index ad624661..2c713d22 100644 --- a/test/promisified/producer/producingToInvalidTopic.spec.js +++ b/test/promisified/producer/producingToInvalidTopic.spec.js @@ -5,19 +5,19 @@ describe('Producer > Producing to invalid topics', () => { let producer, topicName; beforeEach(async () => { - topicName = `test-topic-${secureRandom()}` + topicName = `test-topic-${secureRandom()}`; producer = createProducer({ - }) + }); await producer.connect(); await createTopic({ topic: topicName }); - }) + }); afterEach(async () => { - producer && (await producer.disconnect()) - }) + 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 () => { + 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( @@ -27,4 +27,4 @@ describe('Producer > Producing to invalid topics', () => { await expect(producer.send({ topic: topicName, messages: [message] })).resolves.toBeTruthy(); }); -}) +}); diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index d7e2f212..bb7cb062 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -81,13 +81,13 @@ async function sleep(ms) { } const generateMessages = options => { - const { prefix, number = 100, partition } = options || {} - const prefixOrEmpty = prefix ? `-${prefix}` : '' + const { prefix, number = 100, partition } = options || {}; + const prefixOrEmpty = prefix ? `-${prefix}` : ''; return Array(number) .fill() .map((v, i) => { - const value = secureRandom() + const value = secureRandom(); const message = { key: `key${prefixOrEmpty}-${i}-${value}`, value: `value${prefixOrEmpty}-${i}-${value}`, @@ -96,8 +96,8 @@ const generateMessages = options => { message.partition = partition; } return message; - }) -} + }); +}; module.exports = { createConsumer, @@ -111,4 +111,4 @@ module.exports = { sleep, generateMessages, clusterInformation, -} +}; diff --git a/test/promisified/unit/cache.spec.js b/test/promisified/unit/cache.spec.js index daf03c60..5beb0ab6 100644 --- a/test/promisified/unit/cache.spec.js +++ b/test/promisified/unit/cache.spec.js @@ -255,7 +255,7 @@ describe('MessageCache', () => { const receivedMessages = []; let nextIdx = -1; - const slowMsg = cache.next(nextIdx); + cache.next(nextIdx); for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ const next = cache.next(nextIdx); expect(next).not.toBeNull(); @@ -280,7 +280,7 @@ describe('MessageCache', () => { const receivedMessages = []; let nextIdx = -1; - const slowMsg = cache.nextN(nextIdx, 11); + cache.nextN(nextIdx, 11); for (let i = 0; i < 60/11; i++) { /* 60 - for non-partition 0 msgs */ const next = cache.nextN(nextIdx, 11); expect(next).not.toBeNull(); @@ -305,7 +305,7 @@ describe('MessageCache', () => { const receivedMessages = []; let nextIdx = -1; - const slowMsg = cache.next(nextIdx); + cache.next(nextIdx); for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ const next = cache.next(nextIdx); expect(next).not.toBeNull(); @@ -329,7 +329,7 @@ describe('MessageCache', () => { const receivedMessages = []; let nextIdx = -1; - const slowMsg = cache.next(nextIdx); + cache.next(nextIdx); for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ const next = cache.next(nextIdx); expect(next).not.toBeNull(); From 72c7e207dbddcecc70c0c6ba03a881855b0fd42c Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Thu, 8 Aug 2024 12:08:11 -0400 Subject: [PATCH 167/224] Add SchemaRegistryClient, RestService, and testing (#1) * Add SchemaRegistryClient, RestService, and testing * Add new Makefile for schema registry * Merging * Revert to throwing exceptions --- .eslintrc.js | 37 + Makefile | 2 +- Makefile.schemaregistry | 26 + .../schemaregistry-client.spec.ts | 224 ++++++ jest.config.js | 8 + package-lock.json | 2 +- package.json | 3 +- schemaregistry/rest-service.ts | 72 ++ schemaregistry/schemaregistry-client.ts | 543 ++++++++++++++ .../schemaregistry-client.spec.ts | 670 ++++++++++++++++++ tsconfig.json | 7 +- 11 files changed, 1589 insertions(+), 5 deletions(-) create mode 100644 .eslintrc.js create mode 100644 Makefile.schemaregistry create mode 100644 e2e/schemaregistry/schemaregistry-client.spec.ts create mode 100644 jest.config.js create mode 100644 schemaregistry/rest-service.ts create mode 100644 schemaregistry/schemaregistry-client.ts create mode 100644 test/schemaregistry/schemaregistry-client.spec.ts diff --git a/.eslintrc.js b/.eslintrc.js new file mode 100644 index 00000000..d9b79160 --- /dev/null +++ b/.eslintrc.js @@ -0,0 +1,37 @@ +module.exports = { + "env": { + "browser": true, + "commonjs": true, + "es2021": true + }, + "extends": "eslint:recommended", + "overrides": [ + { + "env": { + "node": true + }, + "files": [ + ".eslintrc.{js,cjs}" + ], + "parserOptions": { + "sourceType": "script" + } + }, + { + "files": ["*.ts"], + "parser": "@typescript-eslint/parser", + "parserOptions": { + "ecmaVersion": 2020, + "sourceType": "module" + }, + "extends": [ + "plugin:@typescript-eslint/recommended", + ] + } + ], + "parserOptions": { + "ecmaVersion": "latest" + }, + "rules": { + } +} diff --git a/Makefile b/Makefile index 75b85441..e8ab3f5e 100644 --- a/Makefile +++ b/Makefile @@ -88,4 +88,4 @@ release-patch: clean: node_modules/.dirstamp @rm -f deps/librdkafka/config.h - @$(NODE-GYP) clean + @$(NODE-GYP) clean \ No newline at end of file diff --git a/Makefile.schemaregistry b/Makefile.schemaregistry new file mode 100644 index 00000000..55b828ab --- /dev/null +++ b/Makefile.schemaregistry @@ -0,0 +1,26 @@ +# Makefile.schemaregistry + +# Variables +NODE ?= node +ESLINT ?= ./node_modules/.bin/eslint +JEST ?= ./node_modules/.bin/jest +TS_NODE ?= ./node_modules/.bin/ts-node + +# Paths +SRC_DIR = schemaregistry +TEST_DIR = test/schemaregistry +INTEG_DIR = e2e/schemaregistry + +# Tasks +.PHONY: all lint test integtest + +all: lint test + +lint: + $(ESLINT) $(SRC_DIR) $(TEST_DIR) $(INTEG_DIR) + +test: + $(JEST) $(TEST_DIR) + +integtest: + $(JEST) $(INTEG_DIR) \ No newline at end of file diff --git a/e2e/schemaregistry/schemaregistry-client.spec.ts b/e2e/schemaregistry/schemaregistry-client.spec.ts new file mode 100644 index 00000000..227fca8f --- /dev/null +++ b/e2e/schemaregistry/schemaregistry-client.spec.ts @@ -0,0 +1,224 @@ +import { RestService } from '../../schemaregistry/rest-service'; +import { + Compatibility, + SchemaRegistryClient, + ServerConfig, + SchemaInfo, + SchemaMetadata, + Metadata +} from '../../schemaregistry/schemaregistry-client'; +import { beforeEach, describe, expect, it } from '@jest/globals'; + +/* eslint-disable @typescript-eslint/no-non-null-asserted-optional-chain */ + +const baseUrls = ['http://localhost:8081']; +const headers = { 'Content-Type': 'application/vnd.schemaregistry.v1+json' }; +const restService = new RestService(baseUrls, false); +restService.setHeaders(headers); + +const basicAuth = Buffer.from('RBACAllowedUser-lsrc1:nohash').toString('base64'); +restService.setAuth(basicAuth); + +restService.setTimeout(10000); + +let schemaRegistryClient: SchemaRegistryClient; +const testSubject = 'integ-test-subject'; +const testServerConfigSubject = 'integ-test-server-config-subject'; + +const schemaString: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + ], +}); + +const metadata: Metadata = { + properties: { + owner: 'Bob Jones', + email: 'bob@acme.com', + }, +}; + +const schemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata, +}; + +const backwardCompatibleSchemaString: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + { name: 'email', type: 'string', default: "" }, + ], +}); + +const backwardCompatibleMetadata: Metadata = { + properties: { + owner: 'Bob Jones2', + email: 'bob@acme.com', + }, +}; + +const backwardCompatibleSchemaInfo: SchemaInfo = { + schema: backwardCompatibleSchemaString, + schemaType: 'AVRO', + metadata: backwardCompatibleMetadata, +}; + +describe('SchemaRegistryClient Integration Test', () => { + + beforeEach(async () => { + schemaRegistryClient = new SchemaRegistryClient(restService); + const subjects: string[] = await schemaRegistryClient.getAllSubjects(); + + if (subjects && subjects.includes(testSubject)) { + await schemaRegistryClient.deleteSubject(testSubject); + await schemaRegistryClient.deleteSubject(testSubject, true); + } + + if (subjects && subjects.includes(testServerConfigSubject)) { + await schemaRegistryClient.deleteSubject(testServerConfigSubject); + await schemaRegistryClient.deleteSubject(testServerConfigSubject, true); + } + }); + + it('should register, retrieve, and delete a schema', async () => { + // Register a schema + const registerResponse: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testSubject, schemaInfo); + expect(registerResponse).toBeDefined(); + + const schemaId = registerResponse?.id!; + const version = registerResponse?.version!; + + const getSchemaResponse: SchemaInfo = await schemaRegistryClient.getBySubjectAndId(testSubject, schemaId); + expect(getSchemaResponse).toEqual(schemaInfo); + + const getIdResponse: number = await schemaRegistryClient.getId(testSubject, schemaInfo); + expect(getIdResponse).toEqual(schemaId); + + // Delete the schema + const deleteSubjectResponse: number = await schemaRegistryClient.deleteSubjectVersion(testSubject, version); + expect(deleteSubjectResponse).toEqual(version); + + const permanentDeleteSubjectResponse: number = await schemaRegistryClient.deleteSubjectVersion(testSubject, version, true); + expect(permanentDeleteSubjectResponse).toEqual(version); + }); + + it('Should get all versions and a specific version of a schema', async () => { + // Register a schema + const registerResponse: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testSubject, schemaInfo); + expect(registerResponse).toBeDefined(); + + const version = registerResponse?.version!; + + const getVersionResponse: number = await schemaRegistryClient.getVersion(testSubject, schemaInfo); + expect(getVersionResponse).toEqual(version); + + const allVersionsResponse: number[] = await schemaRegistryClient.getAllVersions(testSubject); + expect(allVersionsResponse).toEqual([version]); + }); + + it('Should get schema metadata', async () => { + // Register a schema + const registerResponse: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testSubject, schemaInfo); + expect(registerResponse).toBeDefined(); + + const schemaVersion: number = registerResponse?.version!; + + const registerResponse2: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testSubject, backwardCompatibleSchemaInfo); + expect(registerResponse2).toBeDefined(); + + const schemaMetadata: SchemaMetadata = { + id: registerResponse?.id!, + version: schemaVersion, + schema: schemaInfo.schema, + subject: testSubject, + metadata: metadata, + }; + + const schemaMetadata2: SchemaMetadata = { + id: registerResponse2?.id!, + version: registerResponse2?.version!, + schema: backwardCompatibleSchemaInfo.schema, + subject: testSubject, + metadata: backwardCompatibleMetadata, + }; + + const getLatestMetadataResponse: SchemaMetadata = await schemaRegistryClient.getLatestSchemaMetadata(testSubject); + expect(schemaMetadata2).toEqual(getLatestMetadataResponse); + + const getMetadataResponse: SchemaMetadata = await schemaRegistryClient.getSchemaMetadata(testSubject, schemaVersion); + expect(schemaMetadata).toEqual(getMetadataResponse); + }); + + it('Should test compatibility for a version and subject, getting and updating', async () => { + const registerResponse: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testSubject, schemaInfo); + expect(registerResponse).toBeDefined(); + + const version = registerResponse?.version!; + + const updateCompatibilityResponse: Compatibility = await schemaRegistryClient.updateCompatibility(testSubject, Compatibility.BackwardTransitive); + expect(updateCompatibilityResponse).toEqual(Compatibility.BackwardTransitive); + + const getCompatibilityResponse: Compatibility = await schemaRegistryClient.getCompatibility(testSubject); + expect(getCompatibilityResponse).toEqual(Compatibility.BackwardTransitive); + + const testSubjectCompatibilityResponse: boolean = await schemaRegistryClient.testSubjectCompatibility(testSubject, backwardCompatibleSchemaInfo); + expect(testSubjectCompatibilityResponse).toEqual(true); + + const testCompatibilityResponse: boolean = await schemaRegistryClient.testCompatibility(testSubject, version, backwardCompatibleSchemaInfo); + expect(testCompatibilityResponse).toEqual(true); + }); + + it('Should update and get default compatibility', async () => { + const updateDefaultCompatibilityResponse: Compatibility = await schemaRegistryClient.updateDefaultCompatibility(Compatibility.Full); + expect(updateDefaultCompatibilityResponse).toEqual(Compatibility.Full); + + const getDefaultCompatibilityResponse: Compatibility = await schemaRegistryClient.getDefaultCompatibility(); + expect(getDefaultCompatibilityResponse).toEqual(Compatibility.Full); + }); + + it('Should update and get subject Config', async () => { + const subjectConfigRequest: ServerConfig = { + compatibility: Compatibility.Full, + normalize: true + }; + + const subjectConfigResponse: ServerConfig = { + compatibilityLevel: Compatibility.Full, + normalize: true + }; + + const registerResponse: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testServerConfigSubject, schemaInfo); + expect(registerResponse).toBeDefined(); + + const updateConfigResponse: ServerConfig = await schemaRegistryClient.updateConfig(testServerConfigSubject, subjectConfigRequest); + expect(updateConfigResponse).toBeDefined(); + + const getConfigResponse: ServerConfig = await schemaRegistryClient.getConfig(testServerConfigSubject); + expect(getConfigResponse).toEqual(subjectConfigResponse); + }); + + it('Should get and set default Config', async () => { + const serverConfigRequest: ServerConfig = { + compatibility: Compatibility.Full, + normalize: false + }; + + const serverConfigResponse: ServerConfig = { + compatibilityLevel: Compatibility.Full, + normalize: false + }; + + const updateDefaultConfigResponse: ServerConfig = await schemaRegistryClient.updateDefaultConfig(serverConfigRequest); + expect(updateDefaultConfigResponse).toBeDefined(); + + const getDefaultConfigResponse: ServerConfig = await schemaRegistryClient.getDefaultConfig(); + expect(getDefaultConfigResponse).toEqual(serverConfigResponse); + }); + +}); diff --git a/jest.config.js b/jest.config.js new file mode 100644 index 00000000..8d146e52 --- /dev/null +++ b/jest.config.js @@ -0,0 +1,8 @@ +module.exports = { + preset: 'ts-jest', + testEnvironment: 'node', + testMatch: ['**/test/**/*.ts', '**/e2e/**/*.ts'], + transform: { + '^.+\\.tsx?$': 'ts-jest', + }, + }; \ No newline at end of file diff --git a/package-lock.json b/package-lock.json index f0173f89..be9ad9ed 100644 --- a/package-lock.json +++ b/package-lock.json @@ -6065,4 +6065,4 @@ } } } -} +} \ No newline at end of file diff --git a/package.json b/package.json index c38781e9..ae15f155 100644 --- a/package.json +++ b/package.json @@ -12,7 +12,8 @@ "test": "make test", "install": "node-pre-gyp install --fallback-to-build", "prepack": "node ./ci/prepublish.js", - "test:types": "tsc -p ." + "test:types": "tsc -p .", + "test:schemaregistry": "make -f Makefile.schemaregistry test" }, "binary": { "module_name": "confluent-kafka-javascript", diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts new file mode 100644 index 00000000..e93d7746 --- /dev/null +++ b/schemaregistry/rest-service.ts @@ -0,0 +1,72 @@ +import axios, { AxiosInstance, AxiosRequestConfig, AxiosResponse } from 'axios'; + +/* + * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry + * + * Copyright (c) 2024 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +export class RestService { + private client: AxiosInstance + + constructor(baseUrls: string[], isForward = false) { + this.client = axios.create({ + baseURL: baseUrls[0], // Use the first base URL as the default + timeout: 5000, // Default timeout + headers: { 'Content-Type': 'application/vnd.schemaregistry.v1+json' }, + }) + + if (isForward) { + this.client.defaults.headers.common['X-Forward'] = 'true' + } + } + + public async sendHttpRequest( + url: string, + method: 'GET' | 'POST' | 'PUT' | 'DELETE', + data?: any, // eslint-disable-line @typescript-eslint/no-explicit-any + config?: AxiosRequestConfig, + ): Promise> { + try { + const response = await this.client.request({ + url, + method, + data, + ...config, + }) + return response + } catch (error) { + if (axios.isAxiosError(error) && error.response) { + throw new Error(`HTTP error: ${error.response.status} - ${error.response.data}`) + } else { + const err = error as Error; + throw new Error(`Unknown error: ${err.message}`) + } + } + } + + public setHeaders(headers: Record): void { + this.client.defaults.headers.common = { ...this.client.defaults.headers.common, ...headers } + } + + public setAuth(basicAuth?: string, bearerToken?: string): void { + if (basicAuth) { + this.client.defaults.headers.common['Authorization'] = `Basic ${basicAuth}` + } + + if (bearerToken) { + this.client.defaults.headers.common['Authorization'] = `Bearer ${bearerToken}` + } + } + + public setTimeout(timeout: number): void { + this.client.defaults.timeout = timeout + } + + public setBaseURL(baseUrl: string): void { + this.client.defaults.baseURL = baseUrl + } +} diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts new file mode 100644 index 00000000..9f719974 --- /dev/null +++ b/schemaregistry/schemaregistry-client.ts @@ -0,0 +1,543 @@ +import { RestService } from './rest-service'; +import { AxiosResponse } from 'axios'; +import stringify from "json-stringify-deterministic"; +import { LRUCache } from 'lru-cache'; +import { Mutex } from 'async-mutex'; + +/* + * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry + * + * Copyright (c) 2024 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + +enum Compatibility { + None = "NONE", + Backward = "BACKWARD", + Forward = "FORWARD", + Full = "FULL", + BackwardTransitive = "BACKWARD_TRANSITIVE", + ForwardTransitive = "FORWARD_TRANSITIVE", + FullTransitive = "FULL_TRANSITIVE" +} + +interface CompatibilityLevel { + compatibility?: Compatibility; + compatibilityLevel?: Compatibility; +} + +interface Result { + data?: T; + error?: Error; +} + +interface Rule { + name: string; + subject: string; + version: number; +} + +interface SchemaInfo { + schema?: string; + schemaType?: string; + references?: Reference[]; + metadata?: Metadata; + ruleSet?: RuleSet; +} + +interface SchemaMetadata extends SchemaInfo { + id: number; + subject?: string; + version?: number; +} + +interface Reference { + Name: string; + Subject: string; + Version: number; +} + +interface Metadata { + tags?: { [key: string]: string[] }; + properties?: { [key: string]: string }; + sensitive?: string[]; +} + +interface RuleSet { + migrationRules: Rule[]; + compatibilityRules: Rule[]; +} + +interface ServerConfig { + alias?: string; + normalize?: boolean; + compatibility?: Compatibility; + compatibilityLevel?: Compatibility; + compatibilityGroup?: string; + defaultMetadata?: Metadata; + overrideMetadata?: Metadata; + defaultRuleSet?: RuleSet; + overrideRuleSet?: RuleSet; +} + +interface isCompatibleResponse { + is_compatible: boolean; +} + +class SchemaRegistryClient { + private restService: RestService; + + private schemaToIdCache: LRUCache; + private idToSchemaInfoCache: LRUCache; + private infoToSchemaCache: LRUCache; + private latestToSchemaCache: LRUCache; + private schemaToVersionCache: LRUCache; + private versionToSchemaCache: LRUCache; + private metadataToSchemaCache: LRUCache; + + private schemaToIdMutex: Mutex; + private idToSchemaInfoMutex: Mutex; + private infoToSchemaMutex: Mutex; + private latestToSchemaMutex: Mutex; + private schemaToVersionMutex: Mutex; + private versionToSchemaMutex: Mutex; + private metadataToSchemaMutex: Mutex; + + constructor(restService: RestService, cacheSize: number = 512, cacheTTL?: number) { + const cacheOptions = { + max: cacheSize, + ...(cacheTTL !== undefined && { maxAge: cacheTTL }) + }; + + this.restService = restService; + this.schemaToIdCache = new LRUCache(cacheOptions); + this.idToSchemaInfoCache = new LRUCache(cacheOptions); + this.infoToSchemaCache = new LRUCache(cacheOptions); + this.latestToSchemaCache = new LRUCache(cacheOptions); + this.schemaToVersionCache = new LRUCache(cacheOptions); + this.versionToSchemaCache = new LRUCache(cacheOptions); + this.metadataToSchemaCache = new LRUCache(cacheOptions); + this.schemaToIdMutex = new Mutex(); + this.idToSchemaInfoMutex = new Mutex(); + this.infoToSchemaMutex = new Mutex(); + this.latestToSchemaMutex = new Mutex(); + this.schemaToVersionMutex = new Mutex(); + this.versionToSchemaMutex = new Mutex(); + this.metadataToSchemaMutex = new Mutex(); + } + + public async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const metadataResult = await this.registerFullResponse(subject, schema, normalize); + + return metadataResult.id; + } + + public async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const cacheKey = stringify({ subject, schema }); + + return await this.infoToSchemaMutex.runExclusive(async () => { + const cachedSchemaMetadata: SchemaMetadata | undefined = this.infoToSchemaCache.get(cacheKey); + if (cachedSchemaMetadata) { + return cachedSchemaMetadata; + } + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}/versions?normalize=${normalize}`, + 'POST', + schema + ); + this.infoToSchemaCache.set(cacheKey, response.data); + return response.data; + }); + } + + public async getBySubjectAndId(subject: string, id: number): Promise { + const cacheKey = stringify({ subject, id }); + return await this.idToSchemaInfoMutex.runExclusive(async () => { + const cachedSchema: SchemaInfo | undefined = this.idToSchemaInfoCache.get(cacheKey); + if (cachedSchema) { + return cachedSchema; + } + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/schemas/ids/${id}?subject=${subject}`, + 'GET' + ); + this.idToSchemaInfoCache.set(cacheKey, response.data); + return response.data; + }); + } + + public async getId(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const cacheKey = stringify({ subject, schema }); + + return await this.schemaToIdMutex.runExclusive(async () => { + const cachedId: number | undefined = this.schemaToIdCache.get(cacheKey); + if (cachedId) { + return cachedId; + } + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}?normalize=${normalize}`, + 'POST', + schema + ); + this.schemaToIdCache.set(cacheKey, response.data.id); + return response.data.id; + }); + } + + public async getLatestSchemaMetadata(subject: string): Promise { + return await this.latestToSchemaMutex.runExclusive(async () => { + const cachedSchema: SchemaMetadata | undefined = this.latestToSchemaCache.get(subject); + if (cachedSchema) { + return cachedSchema; + } + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}/versions/latest`, + 'GET' + ); + this.latestToSchemaCache.set(subject, response.data); + return response.data; + }); + } + + public async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { + const cacheKey = stringify({ subject, version, deleted }); + + return await this.versionToSchemaMutex.runExclusive(async () => { + const cachedSchemaMetadata: SchemaMetadata | undefined = this.versionToSchemaCache.get(cacheKey); + if (cachedSchemaMetadata) { + return cachedSchemaMetadata; + } + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}/versions/${version}?deleted=${deleted}`, + 'GET' + ); + this.versionToSchemaCache.set(cacheKey, response.data); + return response.data; + }); + } + + private convertToQueryParams(metadata: Metadata): string { + const params = new URLSearchParams(); + + if (metadata.tags) { + for (const [key, values] of Object.entries(metadata.tags)) { + values.forEach((value, index) => { + params.append(`tags.${key}[${index}]`, value); + }); + } + } + + if (metadata.properties) { + for (const [key, value] of Object.entries(metadata.properties)) { + params.append(`properties.${key}`, value); + } + } + + if (metadata.sensitive) { + metadata.sensitive.forEach((value, index) => { + params.append(`sensitive[${index}]`, value); + }); + } + + return params.toString(); + } + + //TODO: Get clarification with getLatestWithMetadata + public async getLatestWithMetadata(subject: string, metadata: Metadata, deleted: boolean = false): Promise { + const cacheKey = stringify({ subject, metadata, deleted }); + + return await this.metadataToSchemaMutex.runExclusive(async () => { + const cachedSchemaMetadata: SchemaMetadata | undefined = this.metadataToSchemaCache.get(cacheKey); + if (cachedSchemaMetadata) { + return cachedSchemaMetadata; + } + + const queryParams = this.convertToQueryParams(metadata); + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}/metadata?deleted=${deleted}&${queryParams}`, + 'GET' + ); + this.metadataToSchemaCache.set(cacheKey, response.data); + return response.data; + }); + } + + + public async getAllVersions(subject: string): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}/versions`, + 'GET' + ); + return response.data; + } + + public async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const cacheKey = stringify({ subject, schema }); + + return await this.schemaToVersionMutex.runExclusive(async () => { + const cachedVersion: number | undefined = this.schemaToVersionCache.get(cacheKey); + if (cachedVersion) { + return cachedVersion; + } + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}?normalize=${normalize}`, + 'POST', + schema + ); + this.schemaToVersionCache.set(cacheKey, response.data.version); + return response.data.version!; + }); + } + + public async getAllSubjects(): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects`, + 'GET' + ); + return response.data; + } + + public async deleteSubject(subject: string, permanent: boolean = false): Promise { + await this.infoToSchemaMutex.runExclusive(async () => { + this.infoToSchemaCache.forEach((_, key) => { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject) { + this.infoToSchemaCache.delete(key); + } + }); + }); + + await this.schemaToVersionMutex.runExclusive(async () => { + this.schemaToVersionCache.forEach((_, key) => { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject) { + this.schemaToVersionCache.delete(key); + } + }); + }); + + await this.versionToSchemaMutex.runExclusive(async () => { + this.versionToSchemaCache.forEach((_, key) => { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject) { + this.versionToSchemaCache.delete(key); + } + }); + }); + + await this.idToSchemaInfoMutex.runExclusive(async () => { + this.idToSchemaInfoCache.forEach((_, key) => { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject) { + this.idToSchemaInfoCache.delete(key); + } + }); + }); + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}?permanent=${permanent}`, + 'DELETE' + ); + return response.data; + } + + public async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { + return await this.schemaToVersionMutex.runExclusive(async () => { + let metadataValue: SchemaMetadata | undefined; + + this.schemaToVersionCache.forEach((value, key) => { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && value === version) { + this.schemaToVersionCache.delete(key); + const infoToSchemaCacheKey = stringify({ subject: subject, schema: parsedKey.schema }); + + this.infoToSchemaMutex.runExclusive(async () => { + metadataValue = this.infoToSchemaCache.get(infoToSchemaCacheKey); + if (metadataValue) { + this.infoToSchemaCache.delete(infoToSchemaCacheKey); + const cacheKeyID = stringify({ subject: subject, id: metadataValue.id }); + + this.idToSchemaInfoMutex.runExclusive(async () => { + this.idToSchemaInfoCache.delete(cacheKeyID); + }); + } + }); + } + }); + + const cacheKey = stringify({ subject: subject, version: version }); + this.versionToSchemaMutex.runExclusive(async () => { + this.versionToSchemaCache.delete(cacheKey); + }); + + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/subjects/${subject}/versions/${version}?permanent=${permanent}`, + 'DELETE' + ); + return response.data; + }); + } + + public async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/compatibility/subjects/${subject}/versions/latest`, + 'POST', + schema + ); + return response.data.is_compatible; + } + + public async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/compatibility/subjects/${subject}/versions/${version}`, + 'POST', + schema + ); + return response.data.is_compatible; + } + + public async getCompatibility(subject: string): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config/${subject}`, + 'GET' + ); + return response.data.compatibilityLevel!; + } + + public async updateCompatibility(subject: string, update: Compatibility): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config/${subject}`, + 'PUT', + { compatibility: update } + ); + return response.data.compatibility!; + } + + public async getDefaultCompatibility(): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config`, + 'GET' + ); + return response.data.compatibilityLevel!; + } + + public async updateDefaultCompatibility(update: Compatibility): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config`, + 'PUT', + { compatibility: update } + ); + return response.data.compatibility!; + } + + public async getConfig(subject: string): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config/${subject}`, + 'GET' + ); + return response.data; + } + + public async updateConfig(subject: string, update: ServerConfig): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config/${subject}`, + 'PUT', + update + ); + return response.data; + } + + public async getDefaultConfig(): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config`, + 'GET' + ); + return response.data; + } + + public async updateDefaultConfig(update: ServerConfig): Promise { + const response: AxiosResponse = await this.restService.sendHttpRequest( + `/config`, + 'PUT', + update + ); + return response.data; + } + + public close(): void { + this.infoToSchemaCache.clear(); + this.schemaToVersionCache.clear(); + this.versionToSchemaCache.clear(); + this.idToSchemaInfoCache.clear(); + + return; + } + + // Cache methods for testing + public async addToInfoToSchemaCache(subject: string, schema: SchemaInfo, metadata: SchemaMetadata): Promise { + const cacheKey = stringify({ subject, schema }); + await this.infoToSchemaMutex.runExclusive(async () => { + this.infoToSchemaCache.set(cacheKey, metadata); + }); + } + + public async addToSchemaToVersionCache(subject: string, schema: SchemaInfo, version: number): Promise { + const cacheKey = stringify({ subject, schema }); + await this.schemaToVersionMutex.runExclusive(async () => { + this.schemaToVersionCache.set(cacheKey, version); + }); + } + + public async addToVersionToSchemaCache(subject: string, version: number, metadata: SchemaMetadata): Promise { + const cacheKey = stringify({ subject, version }); + await this.versionToSchemaMutex.runExclusive(async () => { + this.versionToSchemaCache.set(cacheKey, metadata); + }); + } + + public async addToIdToSchemaInfoCache(subject: string, id: number, schema: SchemaInfo): Promise { + const cacheKey = stringify({ subject, id }); + await this.idToSchemaInfoMutex.runExclusive(async () => { + this.idToSchemaInfoCache.set(cacheKey, schema); + }); + } + + public async getInfoToSchemaCacheSize(): Promise { + return await this.infoToSchemaMutex.runExclusive(async () => { + return this.infoToSchemaCache.size; + }); + } + + public async getSchemaToVersionCacheSize(): Promise { + return await this.schemaToVersionMutex.runExclusive(async () => { + return this.schemaToVersionCache.size; + }); + } + + public async getVersionToSchemaCacheSize(): Promise { + return await this.versionToSchemaMutex.runExclusive(async () => { + return this.versionToSchemaCache.size; + }); + } + + public async getIdToSchemaInfoCacheSize(): Promise { + return await this.idToSchemaInfoMutex.runExclusive(async () => { + return this.idToSchemaInfoCache.size; + }); + } + +} + +export { + SchemaRegistryClient, SchemaInfo, Metadata, Compatibility, + CompatibilityLevel, ServerConfig, RuleSet, Rule, Reference, SchemaMetadata, Result +}; diff --git a/test/schemaregistry/schemaregistry-client.spec.ts b/test/schemaregistry/schemaregistry-client.spec.ts new file mode 100644 index 00000000..77d13508 --- /dev/null +++ b/test/schemaregistry/schemaregistry-client.spec.ts @@ -0,0 +1,670 @@ +import { + SchemaRegistryClient, + Metadata, + Compatibility, + SchemaInfo, + SchemaMetadata, + ServerConfig +} from '../../schemaregistry/schemaregistry-client'; +import { RestService } from '../../schemaregistry/rest-service'; +import { AxiosResponse } from 'axios'; +import stringify from "json-stringify-deterministic"; +import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; + +jest.mock('../../schemaregistry/rest-service'); + +const baseUrls = ['http://mocked-url']; + +let client: SchemaRegistryClient; +let restService: jest.Mocked; +const mockSubject = 'mock-subject'; +const mockSubject2 = 'mock-subject2'; +const schemaString = stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' } + ] +}); +const schemaString2 = stringify({ + type: 'record', + name: 'User2', + fields: [ + { name: 'name2', type: 'string' }, + { name: 'age2', type: 'int' } + ] +}); +const metadata: Metadata = { + properties: { + owner: 'Alice Bob', + email: 'Alice@bob.com', + } +}; +const metadata2: Metadata = { + properties: { + owner: 'Alice Bob2', + email: 'Alice@bob2.com', + } +}; +const schemaInfo = { + schema: schemaString, + schemaType: 'AVRO', +}; +const schemaInfo2 = { + schema: schemaString, + schemaType: 'AVRO', +}; +const schemaInfoMetadata = { + schema: schemaString, + schemaType: 'AVRO', + metadata: metadata, +}; +const schemaInfoMetadata2 = { + schema: schemaString, + schemaType: 'AVRO', + metadata: metadata2, +}; +const subjects: string[] = [mockSubject, mockSubject2]; +const versions: number[] = [1, 2, 3]; + +describe('SchemaRegistryClient-Register', () => { + + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new SchemaRegistryClient(restService); + }); + + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return id when Register is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + + const response: number = await client.register(mockSubject, schemaInfo); + + expect(response).toEqual(1); + + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return from cache when Register is called twice', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + + const response: number = await client.register(mockSubject, schemaInfo); + expect(response).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: { id: 2 } } as AxiosResponse); + + const response2: number = await client.register(mockSubject2, schemaInfo2); + expect(response2).toEqual(2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + //Try to create same objects again + + const cachedResponse: number = await client.register(mockSubject, schemaInfo); + expect(cachedResponse).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: number = await client.register(mockSubject2, schemaInfo2); + expect(cachedResponse2).toEqual(2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should return id, version, metadata, and schema when RegisterFullResponse is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.registerFullResponse(mockSubject, schemaInfoMetadata); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return id, version, metadata, and schema from cache when RegisterFullResponse is called twice', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + const expectedResponse2 = { + id: 2, + version: 1, + schema: schemaString2, + metadata: metadata2, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.registerFullResponse(mockSubject, schemaInfoMetadata); + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + + const response2: SchemaMetadata = await client.registerFullResponse(mockSubject2, schemaInfoMetadata2); + expect(response2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: SchemaMetadata = await client.registerFullResponse(mockSubject, schemaInfoMetadata); + expect(cachedResponse).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: SchemaMetadata = await client.registerFullResponse(mockSubject2, schemaInfoMetadata2); + expect(cachedResponse2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); +}); + +describe('SchemaRegistryClient-Get-ID', () => { + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new SchemaRegistryClient(restService); + }); + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return id when GetId is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + + const response: number = await client.getId(mockSubject, schemaInfo); + + expect(response).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return id from cache when GetId is called twice', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + + const response: number = await client.getId(mockSubject, schemaInfo); + expect(response).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: { id: 2 } } as AxiosResponse); + + const response2: number = await client.getId(mockSubject2, schemaInfo2); + expect(response2).toEqual(2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: number = await client.getId(mockSubject, schemaInfo); + expect(cachedResponse).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: number = await client.getId(mockSubject2, schemaInfo2); + expect(cachedResponse2).toEqual(2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should return SchemaInfo when GetBySubjectAndId is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaInfo = await client.getBySubjectAndId(mockSubject, 1); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return SchemaInfo from cache when GetBySubjectAndId is called twice', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + const expectedResponse2 = { + id: 2, + version: 1, + schema: schemaString2, + metadata: metadata2, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaInfo = await client.getBySubjectAndId(mockSubject, 1); + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + + const response2: SchemaInfo = await client.getBySubjectAndId(mockSubject2, 2); + expect(response2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: SchemaInfo = await client.getBySubjectAndId(mockSubject, 1); + expect(cachedResponse).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: SchemaInfo = await client.getBySubjectAndId(mockSubject2, 2); + expect(cachedResponse2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); +}); + +describe('SchemaRegistryClient-Get-Schema-Metadata', () => { + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new SchemaRegistryClient(restService); + }); + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return latest schema with metadata when GetLatestWithMetadata is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return latest schema with metadata from cache when GetLatestWithMetadata is called twice', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + const expectedResponse2 = { + id: 2, + version: 1, + schema: schemaString2, + metadata: metadata2, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + + const response2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); + expect(response2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + expect(cachedResponse).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); + expect(cachedResponse2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should return SchemaMetadata when GetSchemaMetadata is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.getSchemaMetadata(mockSubject, 1, true); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return SchemaMetadata from cache when GetSchemaMetadata is called twice', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + const expectedResponse2 = { + id: 2, + version: 1, + schema: schemaString2, + metadata: metadata2, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.getSchemaMetadata(mockSubject, 1, true); + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + + const response2: SchemaMetadata = await client.getSchemaMetadata(mockSubject2, 2, false); + expect(response2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: SchemaMetadata = await client.getSchemaMetadata(mockSubject, 1, true); + expect(cachedResponse).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: SchemaMetadata = await client.getSchemaMetadata(mockSubject2, 2, false); + expect(cachedResponse2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should get latest schema with metadata when GetLatestWithMetadata is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should get latest schema with metadata from cache when GetLatestWithMetadata is called twice', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + const expectedResponse2 = { + id: 2, + version: 1, + schema: schemaString2, + metadata: metadata2, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + + const response2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); + expect(response2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + expect(cachedResponse).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); + expect(cachedResponse2).toMatchObject(expectedResponse2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); +}); + +describe('SchemaRegistryClient-Subjects', () => { + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new SchemaRegistryClient(restService); + }); + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return all subjects when GetAllSubjects is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: subjects } as AxiosResponse); + + const response: string[] = await client.getAllSubjects(); + + expect(response).toEqual(subjects); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return all versions when GetAllVersions is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: versions } as AxiosResponse); + + const response: number[] = await client.getAllVersions(mockSubject); + + expect(response).toEqual(versions); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return version when GetVersion is called', async () => { + const schemaInfo = { + schema: schemaString, + schemaType: 'AVRO', + }; + restService.sendHttpRequest.mockResolvedValue({ data: { version: 1 } } as AxiosResponse); + + const response: number = await client.getVersion(mockSubject, schemaInfo, true); + + expect(response).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return version from cache when GetVersion is called twice', async () => { + const schemaInfo = { + schema: schemaString, + schemaType: 'AVRO', + }; + const schemaInfo2 = { + schema: schemaString2, + schemaType: 'AVRO', + }; + + restService.sendHttpRequest.mockResolvedValue({ data: { version: 1 } } as AxiosResponse); + + const response: number = await client.getVersion(mockSubject, schemaInfo, true); + expect(response).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + + restService.sendHttpRequest.mockResolvedValue({ data: { version: 2 } } as AxiosResponse); + + const response2: number = await client.getVersion(mockSubject2, schemaInfo2, false); + expect(response2).toEqual(2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse: number = await client.getVersion(mockSubject, schemaInfo, true); + expect(cachedResponse).toEqual(1); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + + const cachedResponse2: number = await client.getVersion(mockSubject2, schemaInfo2, false); + expect(cachedResponse2).toEqual(2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should delete subject from all caches and registry when deleteSubject is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + await client.addToInfoToSchemaCache(mockSubject, schemaInfo, expectedResponse); + await client.addToSchemaToVersionCache(mockSubject, schemaInfo, 1); + await client.addToVersionToSchemaCache(mockSubject, 1, expectedResponse); + await client.addToIdToSchemaInfoCache(mockSubject, 1, schemaInfo); + + restService.sendHttpRequest.mockResolvedValue({ data: [1] } as AxiosResponse); + + const response: number[] = await client.deleteSubject(mockSubject); + + expect(await client.getInfoToSchemaCacheSize()).toEqual(0); + expect(await client.getSchemaToVersionCacheSize()).toEqual(0); + expect(await client.getVersionToSchemaCacheSize()).toEqual(0); + expect(await client.getIdToSchemaInfoCacheSize()).toEqual(0); + + expect(response).toEqual([1]); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should delete subject version from all caches and registry when deleteSubjectVersion is called', async () => { + const expectedResponse = { + id: 1, + version: 1, + schema: schemaString, + metadata: metadata, + }; + await client.addToInfoToSchemaCache(mockSubject, schemaInfo, expectedResponse); + await client.addToSchemaToVersionCache(mockSubject, schemaInfo, 1); + await client.addToVersionToSchemaCache(mockSubject, 1, expectedResponse); + await client.addToIdToSchemaInfoCache(mockSubject, 1, schemaInfo); + + restService.sendHttpRequest.mockResolvedValue({ data: [1] } as AxiosResponse); + + const response: number = await client.deleteSubjectVersion(mockSubject, 1); + + expect(await client.getVersionToSchemaCacheSize()).toEqual(0); + expect(await client.getInfoToSchemaCacheSize()).toEqual(0); + expect(await client.getSchemaToVersionCacheSize()).toEqual(0); + expect(await client.getIdToSchemaInfoCacheSize()).toEqual(0); + + expect(response).toEqual([1]); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); +}); + +describe('SchemaRegistryClient-Compatibility', () => { + + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new SchemaRegistryClient(restService); + }); + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return compatibility level when GetCompatibility is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { compatibilityLevel: "BACKWARD" } } as AxiosResponse); + + const response: Compatibility = await client.getCompatibility(mockSubject); + + expect(response).toEqual('BACKWARD'); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should update compatibility level when updateCompatibility is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); + + const response: Compatibility = await client.updateCompatibility(mockSubject, Compatibility.Backward); + + expect(response).toEqual(Compatibility.Backward); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return Compatibility when getDefaultCompatibility is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { compatibilityLevel: 'BACKWARD' } } as AxiosResponse); + + const response: Compatibility = await client.getDefaultCompatibility(); + + expect(response).toEqual(Compatibility.Backward); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should update default compatibility level when updateDefaultCompatibility is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); + + const response: Compatibility = await client.updateDefaultCompatibility(Compatibility.Backward); + + expect(response).toEqual(Compatibility.Backward); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); +}); + +describe('SchemaRegistryClient-Config', () => { + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new SchemaRegistryClient(restService); + }); + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return config when getConfig is called', async () => { + const expectedResponse = { + compatibilityLevel: 'BACKWARD', + alias: 'test-config', + normalize: true, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: ServerConfig = await client.getConfig(mockSubject); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should update config when updateConfig is called', async () => { + const request = { + compatibility: Compatibility.Backward, + alias: 'test-config', + normalize: true, + }; + const expectedResponse = { + compatibilityLevel: 'BACKWARD', + alias: 'test-config', + normalize: true, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: ServerConfig = await client.updateConfig(mockSubject, request); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return config when getDefaultConfig is called', async () => { + const expectedResponse = { + compatibilityLevel: 'BACKWARD', + alias: 'test-config', + normalize: true, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: ServerConfig = await client.getDefaultConfig(); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should update default config when updateDefaultConfig is called', async () => { + const request = { + compatibility: Compatibility.Backward, + alias: 'test-config', + normalize: true, + }; + const expectedResponse = { + compatibilityLevel: 'BACKWARD', + alias: 'test-config', + normalize: true, + }; + + restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + + const response: ServerConfig = await client.updateDefaultConfig(request); + + expect(response).toMatchObject(expectedResponse); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); +}); \ No newline at end of file diff --git a/tsconfig.json b/tsconfig.json index d33b7a90..717448a9 100644 --- a/tsconfig.json +++ b/tsconfig.json @@ -6,13 +6,16 @@ "noImplicitThis": true, "strictNullChecks": true, "baseUrl": ".", - "types": ["node_modules/@types/node"], + "types": ["node_modules/@types/node", "jest"], "typeRoots": ["."], "noEmit": true, + "esModuleInterop": true, "forceConsistentCasingInFileNames": true, "strictFunctionTypes": true }, "files": [ - "index.d.ts" + "index.d.ts", + "schemaregistry/**/*", + "test/**/*" ] } From 2887c2c05ce9777f1675eb8f2d693cad921d39a7 Mon Sep 17 00:00:00 2001 From: claimundefine Date: Fri, 9 Aug 2024 14:48:14 -0400 Subject: [PATCH 168/224] Add mock client for testing --- Makefile.schemaregistry | 2 +- .../schemaregistry-client.spec.ts | 8 + package-lock.json | 627 +++++++++--------- package.json | 7 +- schemaregistry/mock-schemaregistry-client.ts | 418 ++++++++++++ schemaregistry/schemaregistry-client.ts | 71 +- .../mock-schemaregistery-client.spec.ts | 221 ++++++ .../schemaregistry-client.spec.ts | 70 +- tsconfig.json | 2 +- 9 files changed, 1035 insertions(+), 391 deletions(-) create mode 100644 schemaregistry/mock-schemaregistry-client.ts create mode 100644 test/schemaregistry/mock-schemaregistery-client.spec.ts diff --git a/Makefile.schemaregistry b/Makefile.schemaregistry index 55b828ab..f7d1c4f3 100644 --- a/Makefile.schemaregistry +++ b/Makefile.schemaregistry @@ -20,7 +20,7 @@ lint: $(ESLINT) $(SRC_DIR) $(TEST_DIR) $(INTEG_DIR) test: - $(JEST) $(TEST_DIR) + $(JEST) $(TEST_DIR) --verbose integtest: $(JEST) $(INTEG_DIR) \ No newline at end of file diff --git a/e2e/schemaregistry/schemaregistry-client.spec.ts b/e2e/schemaregistry/schemaregistry-client.spec.ts index 227fca8f..3854e2f2 100644 --- a/e2e/schemaregistry/schemaregistry-client.spec.ts +++ b/e2e/schemaregistry/schemaregistry-client.spec.ts @@ -153,6 +153,14 @@ describe('SchemaRegistryClient Integration Test', () => { const getMetadataResponse: SchemaMetadata = await schemaRegistryClient.getSchemaMetadata(testSubject, schemaVersion); expect(schemaMetadata).toEqual(getMetadataResponse); + + const keyValueMetadata: { [key: string]: string } = { + 'owner': 'Bob Jones', + 'email': 'bob@acme.com' + } + + const getLatestWithMetadataResponse: SchemaMetadata = await schemaRegistryClient.getLatestWithMetadata(testSubject, keyValueMetadata); + expect(schemaMetadata).toEqual(getLatestWithMetadataResponse); }); it('Should test compatibility for a version and subject, getting and updating', async () => { diff --git a/package-lock.json b/package-lock.json index be9ad9ed..dcd65699 100644 --- a/package-lock.json +++ b/package-lock.json @@ -11,8 +11,13 @@ "license": "MIT", "dependencies": { "@mapbox/node-pre-gyp": "^1.0.11", + "async-mutex": "^0.5.0", + "axios": "^1.7.3", "bindings": "^1.3.1", - "nan": "^2.17.0" + "json-stringify-deterministic": "^1.0.12", + "lru-cache": "^11.0.0", + "nan": "^2.17.0", + "ts-jest": "^29.2.4" }, "devDependencies": { "@eslint/js": "^9.8.0", @@ -34,7 +39,6 @@ "version": "2.3.0", "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.3.0.tgz", "integrity": "sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw==", - "dev": true, "dependencies": { "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.24" @@ -47,7 +51,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", - "dev": true, "dependencies": { "@babel/highlight": "^7.24.7", "picocolors": "^1.0.0" @@ -60,7 +63,6 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.2.tgz", "integrity": "sha512-bYcppcpKBvX4znYaPEeFau03bp89ShqNMLs+rmdptMw+heSZh9+z84d2YG+K7cYLbWwzdjtDoW/uqZmPjulClQ==", - "dev": true, "engines": { "node": ">=6.9.0" } @@ -69,7 +71,6 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.25.2.tgz", "integrity": "sha512-BBt3opiCOxUr9euZ5/ro/Xv8/V7yJ5bjYMqG/C1YAo8MIKAnumZalCN+msbci3Pigy4lIQfPUpfMM27HMGaYEA==", - "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", "@babel/code-frame": "^7.24.7", @@ -99,7 +100,6 @@ "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" } @@ -108,7 +108,6 @@ "version": "7.25.0", "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.0.tgz", "integrity": "sha512-3LEEcj3PVW8pW2R1SR1M89g/qrYk/m/mB/tLqn7dn4sbBUQyTqnlod+II2U4dqiGtUmkcnAmkMDralTFZttRiw==", - "dev": true, "dependencies": { "@babel/types": "^7.25.0", "@jridgewell/gen-mapping": "^0.3.5", @@ -123,7 +122,6 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.25.2.tgz", "integrity": "sha512-U2U5LsSaZ7TAt3cfaymQ8WHh0pxvdHoEk6HVpaexxixjyEquMh0L0YNJNM6CTGKMXV1iksi0iZkGw4AcFkPaaw==", - "dev": true, "dependencies": { "@babel/compat-data": "^7.25.2", "@babel/helper-validator-option": "^7.24.8", @@ -135,11 +133,19 @@ "node": ">=6.9.0" } }, + "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-5.1.1.tgz", + "integrity": "sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w==", + "license": "ISC", + "dependencies": { + "yallist": "^3.0.2" + } + }, "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, "bin": { "semver": "bin/semver.js" } @@ -148,7 +154,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", - "dev": true, "dependencies": { "@babel/traverse": "^7.24.7", "@babel/types": "^7.24.7" @@ -161,7 +166,6 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.25.2.tgz", "integrity": "sha512-BjyRAbix6j/wv83ftcVJmBt72QtHI56C7JXZoG2xATiLpmoC7dpd8WnkikExHDVPpi/3qCmO6WY1EaXOluiecQ==", - "dev": true, "dependencies": { "@babel/helper-module-imports": "^7.24.7", "@babel/helper-simple-access": "^7.24.7", @@ -179,7 +183,6 @@ "version": "7.24.8", "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.8.tgz", "integrity": "sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==", - "dev": true, "engines": { "node": ">=6.9.0" } @@ -188,7 +191,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz", "integrity": "sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==", - "dev": true, "dependencies": { "@babel/traverse": "^7.24.7", "@babel/types": "^7.24.7" @@ -201,7 +203,6 @@ "version": "7.24.8", "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", - "dev": true, "engines": { "node": ">=6.9.0" } @@ -210,7 +211,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", - "dev": true, "engines": { "node": ">=6.9.0" } @@ -219,7 +219,6 @@ "version": "7.24.8", "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.24.8.tgz", "integrity": "sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==", - "dev": true, "engines": { "node": ">=6.9.0" } @@ -228,7 +227,6 @@ "version": "7.25.0", "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.0.tgz", "integrity": "sha512-MjgLZ42aCm0oGjJj8CtSM3DB8NOOf8h2l7DCTePJs29u+v7yO/RBX9nShlKMgFnRks/Q4tBAe7Hxnov9VkGwLw==", - "dev": true, "dependencies": { "@babel/template": "^7.25.0", "@babel/types": "^7.25.0" @@ -241,7 +239,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", - "dev": true, "dependencies": { "@babel/helper-validator-identifier": "^7.24.7", "chalk": "^2.4.2", @@ -256,7 +253,6 @@ "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": { "color-convert": "^1.9.0" }, @@ -268,7 +264,6 @@ "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": { "ansi-styles": "^3.2.1", "escape-string-regexp": "^1.0.5", @@ -282,7 +277,6 @@ "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" } @@ -290,14 +284,12 @@ "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 + "integrity": "sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==" }, "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": ">=0.8.0" } @@ -306,7 +298,6 @@ "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" } @@ -315,7 +306,6 @@ "version": "5.5.0", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==", - "dev": true, "dependencies": { "has-flag": "^3.0.0" }, @@ -327,7 +317,6 @@ "version": "7.25.3", "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.3.tgz", "integrity": "sha512-iLTJKDbJ4hMvFPgQwwsVoxtHyWpKKPBrxkANrSYewDPaPpT5py5yeVkgPIJ7XYXhndxJpaA3PyALSXQ7u8e/Dw==", - "dev": true, "dependencies": { "@babel/types": "^7.25.2" }, @@ -342,7 +331,6 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -354,7 +342,6 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -366,7 +353,6 @@ "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": { "@babel/helper-plugin-utils": "^7.12.13" }, @@ -378,7 +364,6 @@ "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" }, @@ -390,7 +375,6 @@ "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" }, @@ -402,7 +386,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz", "integrity": "sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ==", - "dev": true, "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" }, @@ -417,7 +400,6 @@ "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, "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -429,7 +411,6 @@ "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, "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -441,7 +422,6 @@ "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": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -453,7 +433,6 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -465,7 +444,6 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -477,7 +455,6 @@ "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, "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -489,7 +466,6 @@ "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" }, @@ -504,7 +480,6 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.7.tgz", "integrity": "sha512-c/+fVeJBB0FeKsFvwytYiUD+LBvhHjGSI0g446PRGdSVGZLRNArBUno2PETbAly3tpiNAQR5XaZ+JslxkotsbA==", - "dev": true, "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" }, @@ -519,7 +494,6 @@ "version": "7.25.0", "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", - "dev": true, "dependencies": { "@babel/code-frame": "^7.24.7", "@babel/parser": "^7.25.0", @@ -533,7 +507,6 @@ "version": "7.25.3", "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.3.tgz", "integrity": "sha512-HefgyP1x754oGCsKmV5reSmtV7IXj/kpaE1XYY+D9G5PvKKoFfSbiS4M77MdjuwlZKDIKFCffq9rPU+H/s3ZdQ==", - "dev": true, "dependencies": { "@babel/code-frame": "^7.24.7", "@babel/generator": "^7.25.0", @@ -551,7 +524,6 @@ "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" } @@ -560,7 +532,6 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.2.tgz", "integrity": "sha512-YTnYtra7W9e6/oAZEHj0bJehPRUlLH9/fbpT5LfB0NhQXyALCRkRs3zH9v07IYhkgpqX6Z78FnuccZr/l4Fs4Q==", - "dev": true, "dependencies": { "@babel/helper-string-parser": "^7.24.8", "@babel/helper-validator-identifier": "^7.24.7", @@ -573,8 +544,7 @@ "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 + "integrity": "sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw==" }, "node_modules/@eslint-community/eslint-utils": { "version": "4.4.0", @@ -677,7 +647,6 @@ "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", "integrity": "sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ==", - "dev": true, "dependencies": { "camelcase": "^5.3.1", "find-up": "^4.1.0", @@ -693,7 +662,6 @@ "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" } @@ -702,7 +670,6 @@ "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" @@ -715,7 +682,6 @@ "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" @@ -728,7 +694,6 @@ "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" }, @@ -740,7 +705,6 @@ "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" }, @@ -755,7 +719,6 @@ "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" }, @@ -767,7 +730,6 @@ "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" } @@ -775,14 +737,12 @@ "node_modules/@istanbuljs/load-nyc-config/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 + "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==" }, "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, "engines": { "node": ">=8" } @@ -791,7 +751,6 @@ "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": "*", @@ -808,7 +767,6 @@ "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", @@ -855,7 +813,6 @@ "version": "29.7.0", "resolved": "https://registry.npmjs.org/@jest/environment/-/environment-29.7.0.tgz", "integrity": "sha512-aQIfHDq33ExsN4jP1NWGXhxgQ/wixs60gDiKO+XVMd8Mn0NWPWgc34ZQDTb2jKaUWQ7MuwoitXAsN2XVXNMpAw==", - "dev": true, "dependencies": { "@jest/fake-timers": "^29.7.0", "@jest/types": "^29.6.3", @@ -870,7 +827,6 @@ "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" @@ -883,7 +839,6 @@ "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" }, @@ -895,7 +850,6 @@ "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": { "@jest/types": "^29.6.3", "@sinonjs/fake-timers": "^10.0.2", @@ -912,7 +866,6 @@ "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", @@ -927,7 +880,6 @@ "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", @@ -970,7 +922,6 @@ "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" }, @@ -982,7 +933,6 @@ "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", @@ -996,7 +946,6 @@ "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", @@ -1011,7 +960,6 @@ "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", @@ -1026,7 +974,6 @@ "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", @@ -1052,7 +999,6 @@ "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", @@ -1069,7 +1015,6 @@ "version": "0.3.5", "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", - "dev": true, "dependencies": { "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", @@ -1083,7 +1028,6 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", - "dev": true, "engines": { "node": ">=6.0.0" } @@ -1092,7 +1036,6 @@ "version": "1.2.1", "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", - "dev": true, "engines": { "node": ">=6.0.0" } @@ -1100,14 +1043,12 @@ "node_modules/@jridgewell/sourcemap-codec": { "version": "1.5.0", "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", - "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", - "dev": true + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==" }, "node_modules/@jridgewell/trace-mapping": { "version": "0.3.25", "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", - "dev": true, "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", "@jridgewell/sourcemap-codec": "^1.4.14" @@ -1209,14 +1150,12 @@ "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 + "integrity": "sha512-+Fj43pSMwJs4KRrH/938Uf+uAELIgVBmQzg/q1YG10djyfA3TnrU8N8XzqCh/okZdszqBQTZf96idMfE5lnwTA==" }, "node_modules/@sinonjs/commons": { "version": "3.0.1", "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.1.tgz", "integrity": "sha512-K3mCHKQ9sVh8o1C9cxkwxaOmXoAMlDxC1mYyHrjqOWEcBjYr76t96zL2zlj5dUGZ3HSw240X1qgH3Mjf1yJWpQ==", - "dev": true, "dependencies": { "type-detect": "4.0.8" } @@ -1225,7 +1164,6 @@ "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" } @@ -1243,7 +1181,6 @@ "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", @@ -1256,7 +1193,6 @@ "version": "7.6.8", "resolved": "https://registry.npmjs.org/@types/babel__generator/-/babel__generator-7.6.8.tgz", "integrity": "sha512-ASsj+tpEDsEiFr1arWrlN6V3mdfjRMZt6LtK/Vp/kreFLnr5QH5+DhvD5nINYZXzwJvXeGq+05iUXcAzVrqWtw==", - "dev": true, "dependencies": { "@babel/types": "^7.0.0" } @@ -1265,7 +1201,6 @@ "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" @@ -1275,7 +1210,6 @@ "version": "7.20.6", "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.6.tgz", "integrity": "sha512-r1bzfrm0tomOI8g1SzvCaQHo6Lcv6zu0EA+W2kHrt8dyrHQxGzBBL4kdkzIS+jBMV+EYcMAEAqXqYaLJq5rOZg==", - "dev": true, "dependencies": { "@babel/types": "^7.20.7" } @@ -1284,7 +1218,6 @@ "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": "*" } @@ -1292,14 +1225,12 @@ "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 + "integrity": "sha512-2QF/t/auWm0lsy8XtKVPG19v3sSOQlJe/YHZgfjb/KBBHOGSV+J2q/S671rcq9uTBrLAXmZpqJiaQbMT+zNU1w==" }, "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": "*" } @@ -1308,7 +1239,6 @@ "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": "*" } @@ -1339,7 +1269,6 @@ "version": "20.14.14", "resolved": "https://registry.npmjs.org/@types/node/-/node-20.14.14.tgz", "integrity": "sha512-d64f00982fS9YoOgJkAMolK7MN8Iq3TDdVjchbYHdEmjth/DHowx82GnoA+tVUAN+7vxfYUgAzi+JXbKNd2SDQ==", - "dev": true, "dependencies": { "undici-types": "~5.26.4" } @@ -1347,14 +1276,12 @@ "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 + "integrity": "sha512-9aEbYZ3TbYMznPdcdr3SmIrLXwC/AKZXQeCf9Pgao5CKb8CyHuEX5jzWPTkvregvhRJHcpRO6BFoGW9ycaOkYw==" }, "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": "*" } @@ -1362,8 +1289,7 @@ "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 + "integrity": "sha512-I4q9QU9MQv4oEOz4tAHJtNz1cwuLxn2F3xcc2iV5WdqLPpUnj30aUuxt1mAxYTG+oe8CZMV/+6rU4S4gRDzqtQ==" }, "node_modules/@typescript-eslint/scope-manager": { "version": "7.18.0", @@ -1583,7 +1509,6 @@ "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" }, @@ -1598,7 +1523,6 @@ "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" }, @@ -1618,7 +1542,6 @@ "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" }, @@ -1633,7 +1556,6 @@ "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" @@ -1675,11 +1597,42 @@ "node": ">=8" } }, + "node_modules/async": { + "version": "3.2.5", + "resolved": "https://registry.npmjs.org/async/-/async-3.2.5.tgz", + "integrity": "sha512-baNZyqaaLhyLVKm/DlvdW051MSgO6b8eVfIezl9E5PqWxFgzLm/wQntEW4zOytVburDEr0JlALEpdOFwvErLsg==", + "license": "MIT" + }, + "node_modules/async-mutex": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/async-mutex/-/async-mutex-0.5.0.tgz", + "integrity": "sha512-1A94B18jkJ3DYq284ohPxoXbfTA5HsQ7/Mf4DEhcyLx3Bz27Rh59iScbB6EPiP+B+joue6YCxcMXSbFC1tZKwA==", + "license": "MIT", + "dependencies": { + "tslib": "^2.4.0" + } + }, + "node_modules/asynckit": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz", + "integrity": "sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==", + "license": "MIT" + }, + "node_modules/axios": { + "version": "1.7.3", + "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.3.tgz", + "integrity": "sha512-Ar7ND9pU99eJ9GpoGQKhKf58GpUOgnzuaB7ueNQ5BMi0p+LZ5oaEnfF999fAArcTIBwXTCHAmGcHOZJaWPq9Nw==", + "license": "MIT", + "dependencies": { + "follow-redirects": "^1.15.6", + "form-data": "^4.0.0", + "proxy-from-env": "^1.1.0" + } + }, "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", @@ -1700,7 +1653,6 @@ "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", @@ -1716,7 +1668,6 @@ "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", @@ -1732,7 +1683,6 @@ "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" } @@ -1741,7 +1691,6 @@ "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", @@ -1756,7 +1705,6 @@ "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", @@ -1779,7 +1727,6 @@ "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" @@ -1835,7 +1782,6 @@ "version": "3.0.3", "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", - "dev": true, "dependencies": { "fill-range": "^7.1.1" }, @@ -1853,7 +1799,6 @@ "version": "4.23.3", "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.3.tgz", "integrity": "sha512-btwCFJVjI4YWDNfau8RhZ+B1Q/VLoUITrm3RlP6y1tYGWIOa+InuYiRGXUBXo8nA1qKmHMyLB/iVQg5TT4eFoA==", - "dev": true, "funding": [ { "type": "opencollective", @@ -1881,11 +1826,22 @@ "node": "^6 || ^7 || ^8 || ^9 || ^10 || ^11 || ^12 || >=13.7" } }, + "node_modules/bs-logger": { + "version": "0.2.6", + "resolved": "https://registry.npmjs.org/bs-logger/-/bs-logger-0.2.6.tgz", + "integrity": "sha512-pd8DCoxmbgc7hyPKOvxtqNcjYoOsABPQdcCUjGp3d42VR2CX1ORhk2A87oqqu5R1kk+76nsxZupkmyd+MVtCog==", + "license": "MIT", + "dependencies": { + "fast-json-stable-stringify": "2.x" + }, + "engines": { + "node": ">= 6" + } + }, "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" } @@ -1893,8 +1849,7 @@ "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 + "integrity": "sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ==" }, "node_modules/cacache": { "version": "16.1.3", @@ -1979,7 +1934,6 @@ "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" } @@ -1988,7 +1942,6 @@ "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" } @@ -1997,7 +1950,6 @@ "version": "1.0.30001646", "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001646.tgz", "integrity": "sha512-dRg00gudiBDDTmUhClSdv3hqRfpbOnU28IpI1T6PBTLWa+kOj0681C8uML3PifYfREuBrVjDGhL3adYpBT6spw==", - "dev": true, "funding": [ { "type": "opencollective", @@ -2029,7 +1981,6 @@ "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" @@ -2045,7 +1996,6 @@ "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" } @@ -2098,7 +2048,6 @@ "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", @@ -2112,8 +2061,7 @@ "node_modules/cjs-module-lexer": { "version": "1.3.1", "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.3.1.tgz", - "integrity": "sha512-a3KdPAANPbNE4ZUv9h6LckSl9zLsYOP4MBmhIPkRaeyybt+r4UghLvq+xw/YwUcC1gqylCkL4rdVs3Lwupjm4Q==", - "dev": true + "integrity": "sha512-a3KdPAANPbNE4ZUv9h6LckSl9zLsYOP4MBmhIPkRaeyybt+r4UghLvq+xw/YwUcC1gqylCkL4rdVs3Lwupjm4Q==" }, "node_modules/clean-stack": { "version": "2.2.0", @@ -2128,7 +2076,6 @@ "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", @@ -2142,7 +2089,6 @@ "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" @@ -2151,14 +2097,12 @@ "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 + "integrity": "sha512-lHl4d5/ONEbLlJvaJNtsF/Lz+WvB07u2ycqTYbdrq7UypDXailES4valYb2eWiJFxZlVmpGekfqoxQhzyFdT4Q==" }, "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" }, @@ -2169,8 +2113,7 @@ "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 + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==" }, "node_modules/color-support": { "version": "1.1.3", @@ -2180,6 +2123,18 @@ "color-support": "bin.js" } }, + "node_modules/combined-stream": { + "version": "1.0.8", + "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.8.tgz", + "integrity": "sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg==", + "license": "MIT", + "dependencies": { + "delayed-stream": "~1.0.0" + }, + "engines": { + "node": ">= 0.8" + } + }, "node_modules/concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", @@ -2193,14 +2148,12 @@ "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 + "integrity": "sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg==" }, "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", @@ -2221,7 +2174,6 @@ "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", @@ -2263,7 +2215,6 @@ "version": "1.5.3", "resolved": "https://registry.npmjs.org/dedent/-/dedent-1.5.3.tgz", "integrity": "sha512-NHQtfOOW68WD8lgypbLA5oT+Bt0xXJhiYvoR6SmmNXZfpzOGXwdKWmcwG8N7PwVVWV3eF/68nmD9BaJSsTBhyQ==", - "dev": true, "peerDependencies": { "babel-plugin-macros": "^3.1.0" }, @@ -2283,11 +2234,19 @@ "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/delayed-stream": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz", + "integrity": "sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ==", + "license": "MIT", + "engines": { + "node": ">=0.4.0" + } + }, "node_modules/delegates": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz", @@ -2305,7 +2264,6 @@ "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" } @@ -2323,7 +2281,6 @@ "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" } @@ -2352,17 +2309,30 @@ "node": ">=6.0.0" } }, + "node_modules/ejs": { + "version": "3.1.10", + "resolved": "https://registry.npmjs.org/ejs/-/ejs-3.1.10.tgz", + "integrity": "sha512-UeJmFfOrAQS8OJWPZ4qtgHyWExa088/MtK5UEyoJGFH67cDEXkZSviOiKRCZ4Xij0zxI3JECgYs3oKx+AizQBA==", + "license": "Apache-2.0", + "dependencies": { + "jake": "^10.8.5" + }, + "bin": { + "ejs": "bin/cli.js" + }, + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/electron-to-chromium": { "version": "1.5.4", "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.4.tgz", - "integrity": "sha512-orzA81VqLyIGUEA77YkVA1D+N+nNfl2isJVjjmOyrlxuooZ19ynb+dOlaDTqd/idKRS9lDCSBmtzM+kyCsMnkA==", - "dev": true + "integrity": "sha512-orzA81VqLyIGUEA77YkVA1D+N+nNfl2isJVjjmOyrlxuooZ19ynb+dOlaDTqd/idKRS9lDCSBmtzM+kyCsMnkA==" }, "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" }, @@ -2415,7 +2385,6 @@ "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" } @@ -2424,7 +2393,6 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.2.tgz", "integrity": "sha512-ErCHMCae19vR8vQGe50xIsVomy19rg6gFu3+r3jkEO46suLMWBksvVyoGgQV+jOfl84ZSOSlmv6Gxa89PmTGmA==", - "dev": true, "engines": { "node": ">=6" } @@ -2579,7 +2547,6 @@ "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" @@ -2634,7 +2601,6 @@ "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", @@ -2657,7 +2623,6 @@ "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" } @@ -2666,7 +2631,6 @@ "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", @@ -2721,8 +2685,7 @@ "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 + "integrity": "sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw==" }, "node_modules/fast-levenshtein": { "version": "2.0.6", @@ -2743,7 +2706,6 @@ "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" } @@ -2765,11 +2727,40 @@ "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/filelist": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/filelist/-/filelist-1.0.4.tgz", + "integrity": "sha512-w1cEuf3S+DrLCQL7ET6kz+gmlJdbq9J7yXCSjK/OZCPA+qEN1WyF4ZAf0YYJa4/shHJra2t/d/r8SV4Ji+x+8Q==", + "license": "Apache-2.0", + "dependencies": { + "minimatch": "^5.0.1" + } + }, + "node_modules/filelist/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==", + "license": "MIT", + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/filelist/node_modules/minimatch": { + "version": "5.1.6", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.1.6.tgz", + "integrity": "sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g==", + "license": "ISC", + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=10" + } + }, "node_modules/fill-range": { "version": "7.1.1", "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", - "dev": true, "dependencies": { "to-regex-range": "^5.0.1" }, @@ -2822,6 +2813,40 @@ "integrity": "sha512-X8cqMLLie7KsNUDSdzeN8FYK9rEt4Dt67OsG/DNGnYTSDBG4uFAJFBnUeiV+zCVAvwFy56IjM9sH51jVaEhNxw==", "dev": true }, + "node_modules/follow-redirects": { + "version": "1.15.6", + "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.6.tgz", + "integrity": "sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA==", + "funding": [ + { + "type": "individual", + "url": "https://github.com/sponsors/RubenVerborgh" + } + ], + "license": "MIT", + "engines": { + "node": ">=4.0" + }, + "peerDependenciesMeta": { + "debug": { + "optional": true + } + } + }, + "node_modules/form-data": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/form-data/-/form-data-4.0.0.tgz", + "integrity": "sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww==", + "license": "MIT", + "dependencies": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.8", + "mime-types": "^2.1.12" + }, + "engines": { + "node": ">= 6" + } + }, "node_modules/fs-minipass": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/fs-minipass/-/fs-minipass-2.1.0.tgz", @@ -2842,7 +2867,6 @@ "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": [ @@ -2856,7 +2880,6 @@ "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" } @@ -2885,7 +2908,6 @@ "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" } @@ -2894,7 +2916,6 @@ "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.*" } @@ -2903,7 +2924,6 @@ "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" } @@ -2912,7 +2932,6 @@ "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" }, @@ -2990,8 +3009,7 @@ "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 + "integrity": "sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ==" }, "node_modules/graphemer": { "version": "1.4.0", @@ -3003,7 +3021,6 @@ "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" } @@ -3017,7 +3034,6 @@ "version": "2.0.2", "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.2.tgz", "integrity": "sha512-0hJU9SCPvmMzIBdZFqNPXWa6dqh7WdH0cII9y+CyS8rG3nL48Bclra9HmKhVVUHyPWNH5Y7xDwAB7bfgSjkUMQ==", - "dev": true, "dependencies": { "function-bind": "^1.1.2" }, @@ -3037,8 +3053,7 @@ "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 + "integrity": "sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg==" }, "node_modules/http-cache-semantics": { "version": "4.1.1", @@ -3076,7 +3091,6 @@ "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" } @@ -3131,7 +3145,6 @@ "version": "3.2.0", "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.2.0.tgz", "integrity": "sha512-2SPlun1JUPWoM6t3F0dw0FkCF/jWY8kttcY4f599GLTSjh2OCuuhdTkJQsEcZzBqbXZGKMK2OqW1oZsjtf/gQA==", - "dev": true, "dependencies": { "pkg-dir": "^4.2.0", "resolve-cwd": "^3.0.0" @@ -3150,7 +3163,6 @@ "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" } @@ -3201,8 +3213,7 @@ "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 + "integrity": "sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg==" }, "node_modules/is-binary-path": { "version": "2.1.0", @@ -3220,7 +3231,6 @@ "version": "2.15.0", "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.15.0.tgz", "integrity": "sha512-Dd+Lb2/zvk9SKy1TGCt1wFJFo/MWBPMX5x7KcvLajWTGuomczdQX61PvY5yK6SVACwpoexWo81IfFyoKY2QnTA==", - "dev": true, "dependencies": { "hasown": "^2.0.2" }, @@ -3252,7 +3262,6 @@ "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" } @@ -3279,7 +3288,6 @@ "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" } @@ -3306,7 +3314,6 @@ "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" }, @@ -3329,14 +3336,12 @@ "node_modules/isexe": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", - "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==", - "dev": true + "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==" }, "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, "engines": { "node": ">=8" } @@ -3345,7 +3350,6 @@ "version": "6.0.3", "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.3.tgz", "integrity": "sha512-Vtgk7L/R2JHyyGW07spoFlB8/lpjiOLTjMdms6AFMraYt3BaJauod/NGrfnVG/y4Ix1JEuMRPDPEj2ua+zz1/Q==", - "dev": true, "dependencies": { "@babel/core": "^7.23.9", "@babel/parser": "^7.23.9", @@ -3361,7 +3365,6 @@ "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": { "istanbul-lib-coverage": "^3.0.0", "make-dir": "^4.0.0", @@ -3375,7 +3378,6 @@ "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" }, @@ -3390,7 +3392,6 @@ "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": { "debug": "^4.1.1", "istanbul-lib-coverage": "^3.0.0", @@ -3404,7 +3405,6 @@ "version": "3.1.7", "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-3.1.7.tgz", "integrity": "sha512-BewmUXImeuRk2YY0PVbxgKAysvhRPUQE0h5QRM++nVWyubKGV0l8qQ5op8+B2DOmwSe63Jivj0BjkPQVf8fP5g==", - "dev": true, "dependencies": { "html-escaper": "^2.0.0", "istanbul-lib-report": "^3.0.0" @@ -3413,11 +3413,28 @@ "node": ">=8" } }, + "node_modules/jake": { + "version": "10.9.2", + "resolved": "https://registry.npmjs.org/jake/-/jake-10.9.2.tgz", + "integrity": "sha512-2P4SQ0HrLQ+fw6llpLnOaGAvN2Zu6778SJMrCUwns4fOoG9ayrTiZk3VV8sCPkVZF8ab0zksVpS8FDY5pRCNBA==", + "license": "Apache-2.0", + "dependencies": { + "async": "^3.2.3", + "chalk": "^4.0.2", + "filelist": "^1.0.4", + "minimatch": "^3.1.2" + }, + "bin": { + "jake": "bin/cli.js" + }, + "engines": { + "node": ">=10" + } + }, "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, "dependencies": { "@jest/core": "^29.7.0", "@jest/types": "^29.6.3", @@ -3443,7 +3460,6 @@ "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, "dependencies": { "execa": "^5.0.0", "jest-util": "^29.7.0", @@ -3457,7 +3473,6 @@ "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": { "@jest/environment": "^29.7.0", "@jest/expect": "^29.7.0", @@ -3488,7 +3503,6 @@ "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": { "@jest/core": "^29.7.0", "@jest/test-result": "^29.7.0", @@ -3521,7 +3535,6 @@ "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", @@ -3566,7 +3579,6 @@ "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", @@ -3581,7 +3593,6 @@ "version": "29.7.0", "resolved": "https://registry.npmjs.org/jest-docblock/-/jest-docblock-29.7.0.tgz", "integrity": "sha512-q617Auw3A612guyaFgsbFeYpNP5t2aoUNLwBUbc/0kD1R4t9ixDbyFTHd1nok4epoVFpr7PmeWHrhvuV3XaJ4g==", - "dev": true, "dependencies": { "detect-newline": "^3.0.0" }, @@ -3593,7 +3604,6 @@ "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": { "@jest/types": "^29.6.3", "chalk": "^4.0.0", @@ -3609,7 +3619,6 @@ "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": { "@jest/environment": "^29.7.0", "@jest/fake-timers": "^29.7.0", @@ -3626,7 +3635,6 @@ "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": "^14.15.0 || ^16.10.0 || >=18.0.0" } @@ -3635,7 +3643,6 @@ "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": { "@jest/types": "^29.6.3", "@types/graceful-fs": "^4.1.3", @@ -3660,7 +3667,6 @@ "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": { "jest-get-type": "^29.6.3", "pretty-format": "^29.7.0" @@ -3673,7 +3679,6 @@ "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", @@ -3688,7 +3693,6 @@ "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", @@ -3708,7 +3712,6 @@ "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": { "@jest/types": "^29.6.3", "@types/node": "*", @@ -3722,7 +3725,6 @@ "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" }, @@ -3739,7 +3741,6 @@ "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" } @@ -3748,7 +3749,6 @@ "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": { "chalk": "^4.0.0", "graceful-fs": "^4.2.9", @@ -3768,7 +3768,6 @@ "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" @@ -3781,7 +3780,6 @@ "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", @@ -3813,7 +3811,6 @@ "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": { "@jest/environment": "^29.7.0", "@jest/fake-timers": "^29.7.0", @@ -3846,7 +3843,6 @@ "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", @@ -3877,7 +3873,6 @@ "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": "*", @@ -3894,7 +3889,6 @@ "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", @@ -3911,7 +3905,6 @@ "version": "6.3.0", "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", - "dev": true, "engines": { "node": ">=10" }, @@ -3923,7 +3916,6 @@ "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", @@ -3942,7 +3934,6 @@ "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", @@ -3957,7 +3948,6 @@ "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" }, @@ -3971,8 +3961,7 @@ "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 + "integrity": "sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==" }, "node_modules/js-yaml": { "version": "4.1.0", @@ -4043,7 +4032,6 @@ "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" }, @@ -4060,8 +4048,7 @@ "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 + "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==" }, "node_modules/json-schema-traverse": { "version": "0.4.1", @@ -4075,11 +4062,19 @@ "integrity": "sha512-Bdboy+l7tA3OGW6FjyFHWkP5LuByj1Tk33Ljyq0axyzdk9//JSi2u3fP1QSmd1KNwq6VOKYGlAu87CisVir6Pw==", "dev": true }, + "node_modules/json-stringify-deterministic": { + "version": "1.0.12", + "resolved": "https://registry.npmjs.org/json-stringify-deterministic/-/json-stringify-deterministic-1.0.12.tgz", + "integrity": "sha512-q3PN0lbUdv0pmurkBNdJH3pfFvOTL/Zp0lquqpvcjfKzt6Y0j49EPHAmVHCAS4Ceq/Y+PejWTzyiVpoY71+D6g==", + "license": "MIT", + "engines": { + "node": ">= 4" + } + }, "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" }, @@ -4109,7 +4104,6 @@ "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" } @@ -4118,7 +4112,6 @@ "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" } @@ -4139,8 +4132,7 @@ "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 + "integrity": "sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg==" }, "node_modules/linkify-it": { "version": "5.0.0", @@ -4172,6 +4164,12 @@ "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==", "dev": true }, + "node_modules/lodash.memoize": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", + "integrity": "sha512-t7j+NzmgnQzTAYXcsHYLgimltOV1MXHtlOWf6GjL9Kj8GK5FInw5JotxvbOs+IvV1/Dzo04/fCGfLVs7aXb4Ag==", + "license": "MIT" + }, "node_modules/lodash.merge": { "version": "4.6.2", "resolved": "https://registry.npmjs.org/lodash.merge/-/lodash.merge-4.6.2.tgz", @@ -4195,12 +4193,12 @@ } }, "node_modules/lru-cache": { - "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, - "dependencies": { - "yallist": "^3.0.2" + "version": "11.0.0", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-11.0.0.tgz", + "integrity": "sha512-Qv32eSV1RSCfhY3fpPE2GNZ8jgM9X7rdAfemLWqTUxwiyIC4jJ6Sy0fZ8H+oLWevO6i4/bizg7c8d8i6bxrzbA==", + "license": "ISC", + "engines": { + "node": "20 || >=22" } }, "node_modules/make-dir": { @@ -4225,6 +4223,12 @@ "semver": "bin/semver.js" } }, + "node_modules/make-error": { + "version": "1.3.6", + "resolved": "https://registry.npmjs.org/make-error/-/make-error-1.3.6.tgz", + "integrity": "sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw==", + "license": "ISC" + }, "node_modules/make-fetch-happen": { "version": "10.2.1", "resolved": "https://registry.npmjs.org/make-fetch-happen/-/make-fetch-happen-10.2.1.tgz", @@ -4265,7 +4269,6 @@ "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" } @@ -4318,8 +4321,7 @@ "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 + "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==" }, "node_modules/merge2": { "version": "1.4.1", @@ -4334,7 +4336,6 @@ "version": "4.0.7", "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.7.tgz", "integrity": "sha512-LPP/3KorzCwBxfeUuZmaR6bG2kdeHSbe0P2tY3FLRU4vYrjYz5hI4QZwV0njUx3jeuKe67YukQ1LSPZBKDqO/Q==", - "dev": true, "dependencies": { "braces": "^3.0.3", "picomatch": "^2.3.1" @@ -4343,11 +4344,31 @@ "node": ">=8.6" } }, + "node_modules/mime-db": { + "version": "1.52.0", + "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.52.0.tgz", + "integrity": "sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==", + "license": "MIT", + "engines": { + "node": ">= 0.6" + } + }, + "node_modules/mime-types": { + "version": "2.1.35", + "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.35.tgz", + "integrity": "sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==", + "license": "MIT", + "dependencies": { + "mime-db": "1.52.0" + }, + "engines": { + "node": ">= 0.6" + } + }, "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": ">=6" } @@ -4611,8 +4632,7 @@ "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 + "integrity": "sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw==" }, "node_modules/negotiator": { "version": "0.6.3", @@ -4735,14 +4755,12 @@ "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 + "integrity": "sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw==" }, "node_modules/node-releases": { "version": "2.0.18", "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.18.tgz", - "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==", - "dev": true + "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==" }, "node_modules/nopt": { "version": "5.0.0", @@ -4762,7 +4780,6 @@ "version": "3.0.0", "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", - "dev": true, "engines": { "node": ">=0.10.0" } @@ -4771,7 +4788,6 @@ "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" }, @@ -4811,7 +4827,6 @@ "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" }, @@ -4843,7 +4858,6 @@ "version": "3.1.0", "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", - "dev": true, "dependencies": { "yocto-queue": "^0.1.0" }, @@ -4888,7 +4902,6 @@ "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" } @@ -4909,7 +4922,6 @@ "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", @@ -4927,7 +4939,6 @@ "version": "4.0.0", "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==", - "dev": true, "engines": { "node": ">=8" } @@ -4944,7 +4955,6 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", - "dev": true, "engines": { "node": ">=8" } @@ -4952,8 +4962,7 @@ "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 + "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==" }, "node_modules/path-type": { "version": "4.0.0", @@ -4967,14 +4976,12 @@ "node_modules/picocolors": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.1.tgz", - "integrity": "sha512-anP1Z8qwhkbmu7MFP5iTt+wQKXgwzf7zTyGlcdzabySa9vd0Xt392U0rVmz9poOaBj0uHJKyyo9/upk0HrEQew==", - "dev": true + "integrity": "sha512-anP1Z8qwhkbmu7MFP5iTt+wQKXgwzf7zTyGlcdzabySa9vd0Xt392U0rVmz9poOaBj0uHJKyyo9/upk0HrEQew==" }, "node_modules/picomatch": { "version": "2.3.1", "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", - "dev": true, "engines": { "node": ">=8.6" }, @@ -4986,7 +4993,6 @@ "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": ">= 6" } @@ -4995,7 +5001,6 @@ "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" }, @@ -5007,7 +5012,6 @@ "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" @@ -5020,7 +5024,6 @@ "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" }, @@ -5032,7 +5035,6 @@ "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" }, @@ -5047,7 +5049,6 @@ "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" }, @@ -5068,7 +5069,6 @@ "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", @@ -5082,7 +5082,6 @@ "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": ">=10" }, @@ -5113,7 +5112,6 @@ "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" @@ -5122,6 +5120,12 @@ "node": ">= 6" } }, + "node_modules/proxy-from-env": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/proxy-from-env/-/proxy-from-env-1.1.0.tgz", + "integrity": "sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg==", + "license": "MIT" + }, "node_modules/punycode": { "version": "2.3.1", "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.3.1.tgz", @@ -5144,7 +5148,6 @@ "version": "6.1.0", "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.1.0.tgz", "integrity": "sha512-bVWawvoZoBYpp6yIoQtQXHZjmz35RSVHnUOTefl8Vcjr8snTPY1wnpSPMWekcFwbxI6gtmT7rSYPFvz71ldiOA==", - "dev": true, "funding": [ { "type": "individual", @@ -5188,8 +5191,7 @@ "node_modules/react-is": { "version": "18.3.1", "resolved": "https://registry.npmjs.org/react-is/-/react-is-18.3.1.tgz", - "integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==", - "dev": true + "integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==" }, "node_modules/readable-stream": { "version": "3.6.2", @@ -5220,7 +5222,6 @@ "version": "2.1.1", "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", "integrity": "sha512-fGxEI7+wsG9xrvdjsrlmL22OMTTiHRwAMroiEeMgq8gzoLC/PQr7RsRDSTLUg/bZAZtF+TVIkHc6/4RIKrui+Q==", - "dev": true, "engines": { "node": ">=0.10.0" } @@ -5238,7 +5239,6 @@ "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", @@ -5255,7 +5255,6 @@ "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" }, @@ -5267,7 +5266,6 @@ "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" } @@ -5285,7 +5283,6 @@ "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" } @@ -5401,7 +5398,6 @@ "version": "2.0.0", "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", - "dev": true, "dependencies": { "shebang-regex": "^3.0.0" }, @@ -5413,7 +5409,6 @@ "version": "3.0.0", "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", - "dev": true, "engines": { "node": ">=8" } @@ -5426,14 +5421,12 @@ "node_modules/sisteransi": { "version": "1.0.5", "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", - "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==", - "dev": true + "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==" }, "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" } @@ -5480,7 +5473,6 @@ "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" } @@ -5489,7 +5481,6 @@ "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" @@ -5517,7 +5508,6 @@ "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" }, @@ -5529,7 +5519,6 @@ "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" } @@ -5546,7 +5535,6 @@ "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": { "char-regex": "^1.0.2", "strip-ansi": "^6.0.0" @@ -5583,7 +5571,6 @@ "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, "engines": { "node": ">=8" } @@ -5592,7 +5579,6 @@ "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" } @@ -5601,7 +5587,6 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==", - "dev": true, "engines": { "node": ">=8" }, @@ -5613,7 +5598,6 @@ "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" }, @@ -5625,7 +5609,6 @@ "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" }, @@ -5666,7 +5649,6 @@ "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": { "@istanbuljs/schema": "^0.1.2", "glob": "^7.1.4", @@ -5685,14 +5667,12 @@ "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 + "integrity": "sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw==" }, "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" } @@ -5701,7 +5681,6 @@ "version": "5.0.1", "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", - "dev": true, "dependencies": { "is-number": "^7.0.0" }, @@ -5726,6 +5705,69 @@ "typescript": ">=4.2.0" } }, + "node_modules/ts-jest": { + "version": "29.2.4", + "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.4.tgz", + "integrity": "sha512-3d6tgDyhCI29HlpwIq87sNuI+3Q6GLTTCeYRHCs7vDz+/3GCMwEtV9jezLyl4ZtnBgx00I7hm8PCP8cTksMGrw==", + "license": "MIT", + "dependencies": { + "bs-logger": "0.x", + "ejs": "^3.1.10", + "fast-json-stable-stringify": "2.x", + "jest-util": "^29.0.0", + "json5": "^2.2.3", + "lodash.memoize": "4.x", + "make-error": "1.x", + "semver": "^7.5.3", + "yargs-parser": "^21.0.1" + }, + "bin": { + "ts-jest": "cli.js" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || ^18.0.0 || >=20.0.0" + }, + "peerDependencies": { + "@babel/core": ">=7.0.0-beta.0 <8", + "@jest/transform": "^29.0.0", + "@jest/types": "^29.0.0", + "babel-jest": "^29.0.0", + "jest": "^29.0.0", + "typescript": ">=4.3 <6" + }, + "peerDependenciesMeta": { + "@babel/core": { + "optional": true + }, + "@jest/transform": { + "optional": true + }, + "@jest/types": { + "optional": true + }, + "babel-jest": { + "optional": true + }, + "esbuild": { + "optional": true + } + } + }, + "node_modules/ts-jest/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==", + "license": "ISC", + "engines": { + "node": ">=12" + } + }, + "node_modules/tslib": { + "version": "2.6.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.6.3.tgz", + "integrity": "sha512-xNvxJEOUiWPGhUuUdQgAJPKOOJfGnIyKySOc09XkKsgdUV/3E2zvwZYdejjmRgPCgcym1juLH3226yA7sEFJKQ==", + "license": "0BSD" + }, "node_modules/type-check": { "version": "0.4.0", "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.4.0.tgz", @@ -5742,7 +5784,6 @@ "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" } @@ -5763,7 +5804,6 @@ "version": "5.5.4", "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.5.4.tgz", "integrity": "sha512-Mtq29sKDAEYP7aljRgtPOpTvOfbwRWlS6dPRzwjdE+C0R4brX/GUyhHSecbHMFLNBLcJIPt9nl9yG5TZ1weH+Q==", - "dev": true, "bin": { "tsc": "bin/tsc", "tsserver": "bin/tsserver" @@ -5787,8 +5827,7 @@ "node_modules/undici-types": { "version": "5.26.5", "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-5.26.5.tgz", - "integrity": "sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==", - "dev": true + "integrity": "sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==" }, "node_modules/unique-filename": { "version": "2.0.1", @@ -5818,7 +5857,6 @@ "version": "1.1.0", "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.1.0.tgz", "integrity": "sha512-EdRAaAyk2cUE1wOf2DkEhzxqOQvFOoRJFNS6NeyJ01Gp2beMRpBAINjM2iDXE3KCuKhwnvHIQCJm6ThL2Z+HzQ==", - "dev": true, "funding": [ { "type": "opencollective", @@ -5862,7 +5900,6 @@ "version": "9.3.0", "resolved": "https://registry.npmjs.org/v8-to-istanbul/-/v8-to-istanbul-9.3.0.tgz", "integrity": "sha512-kiGUalWN+rgBJ/1OHZsBtU4rXZOfj/7rKQxULKlIzwzQSvMJUUNgPwJEEh7gU6xEVxC0ahoOBvN2YI8GH6FNgA==", - "dev": true, "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", "@types/istanbul-lib-coverage": "^2.0.1", @@ -5876,7 +5913,6 @@ "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" } @@ -5899,7 +5935,6 @@ "version": "2.0.2", "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", - "dev": true, "dependencies": { "isexe": "^2.0.0" }, @@ -5937,7 +5972,6 @@ "version": "7.0.0", "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", - "dev": true, "dependencies": { "ansi-styles": "^4.0.0", "string-width": "^4.1.0", @@ -5959,7 +5993,6 @@ "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": { "imurmurhash": "^0.1.4", "signal-exit": "^3.0.7" @@ -5978,7 +6011,6 @@ "version": "5.0.8", "resolved": "https://registry.npmjs.org/y18n/-/y18n-5.0.8.tgz", "integrity": "sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA==", - "dev": true, "engines": { "node": ">=10" } @@ -5987,13 +6019,12 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/yallist/-/yallist-3.1.1.tgz", "integrity": "sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g==", - "dev": true + "license": "ISC" }, "node_modules/yargs": { "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": "^8.0.1", "escalade": "^3.1.1", @@ -6047,7 +6078,6 @@ "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" } @@ -6056,7 +6086,6 @@ "version": "0.1.0", "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==", - "dev": true, "engines": { "node": ">=10" }, @@ -6065,4 +6094,4 @@ } } } -} \ No newline at end of file +} diff --git a/package.json b/package.json index ae15f155..dffa1280 100644 --- a/package.json +++ b/package.json @@ -45,8 +45,13 @@ }, "dependencies": { "@mapbox/node-pre-gyp": "^1.0.11", + "async-mutex": "^0.5.0", + "axios": "^1.7.3", "bindings": "^1.3.1", - "nan": "^2.17.0" + "json-stringify-deterministic": "^1.0.12", + "lru-cache": "^11.0.0", + "nan": "^2.17.0", + "ts-jest": "^29.2.4" }, "engines": { "node": ">=18.0.0" diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts new file mode 100644 index 00000000..477b1b7b --- /dev/null +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -0,0 +1,418 @@ + +import { Client, Compatibility, Metadata, SchemaInfo, SchemaMetadata, ServerConfig } from './schemaregistry-client'; +import stringify from "json-stringify-deterministic"; + +interface VersionCacheEntry { + version: number; + softDeleted: boolean; +} + +interface InfoCacheEntry { + info: SchemaInfo; + softDeleted: boolean; +} + +interface MetadataCacheEntry { + metadata: SchemaMetadata; + softDeleted: boolean; +} + +class Counter { + private count: number = 0; + + currentValue(): number { + return this.count; + } + + increment(): number { + this.count++; + return this.count; + } +} + +const noSubject = ""; + +class MockClient implements Client { + private infoToSchemaCache: Map; + private idToSchemaCache: Map; + private schemaToVersionCache: Map; + private configCache: Map; + private counter: Counter; + + constructor() { + this.infoToSchemaCache = new Map(); + this.idToSchemaCache = new Map(); + this.schemaToVersionCache = new Map(); + this.configCache = new Map(); + this.counter = new Counter(); + } + + public async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const metadata = await this.registerFullResponse(subject, schema, normalize); + if (!metadata) { + throw new Error("Failed to register schema"); + } + return metadata.id; + } + + public async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const cacheKey = stringify({ subject, schema }); + + const cacheEntry = this.infoToSchemaCache.get(cacheKey); + if (cacheEntry && !cacheEntry.softDeleted) { + return cacheEntry.metadata; + } + + const id = await this.getIDFromRegistry(subject, schema); + if (id === -1) { + throw new Error("Failed to retrieve schema ID from registry"); + } + + const metadata: SchemaMetadata = { ...schema, id }; + this.infoToSchemaCache.set(cacheKey, { metadata, softDeleted: false }); + + return metadata; + } + + private async getIDFromRegistry(subject: string, schema: SchemaInfo): Promise { + let id = -1; + + for (const [key, value] of this.idToSchemaCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && this.schemasEqual(value.info, schema)) { + id = parsedKey.id; + break; + } + } + + await this.generateVersion(subject, schema); + if (id < 0) { + id = this.counter.increment(); + const idCacheKey = stringify({ subject, id }); + this.idToSchemaCache.set(idCacheKey, { info: schema, softDeleted: false }); + } + + return id; + } + + private async generateVersion(subject: string, schema: SchemaInfo): Promise { + const versions = await this.allVersions(subject); + let newVersion: number; + + if (versions.length === 0) { + newVersion = 1; + } else { + newVersion = versions[versions.length - 1] + 1; + } + + const cacheKey = stringify({ subject, schema: schema }); + this.schemaToVersionCache.set(cacheKey, { version: newVersion, softDeleted: false }); + } + + public async getBySubjectAndId(subject: string, id: number): Promise { + const cacheKey = stringify({ subject, id }); + const cacheEntry = this.idToSchemaCache.get(cacheKey); + + if (!cacheEntry || cacheEntry.softDeleted) { + throw new Error("Schema not found"); + } + return cacheEntry.info; + } + + public async getId(subject: string, schema: SchemaInfo): Promise { + const cacheKey = stringify({ subject, schema }); + const cacheEntry = this.infoToSchemaCache.get(cacheKey); + if (!cacheEntry || cacheEntry.softDeleted) { + throw new Error("Schema not found"); + } + return cacheEntry.metadata.id; + } + + public async getLatestSchemaMetadata(subject: string): Promise { + const version = await this.latestVersion(subject); + if (version === -1) { + throw new Error("No versions found for subject"); + } + + return this.getSchemaMetadata(subject, version); + } + + public async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { + let json; + for (const [key, value] of this.schemaToVersionCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && value.version === version && value.softDeleted === deleted) { + json = parsedKey; + } + } + + if (!json) { + throw new Error("Schema not found"); + } + + let id: number = -1; + for (const [key, value] of this.idToSchemaCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && value.info.schema === json.schema.schema) { + id = parsedKey.id; + } + } + if (id === -1) { + throw new Error("Schema not found"); + } + + + return { + id, + version, + subject, + schema: json.schema.schema + }; + } + + public async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { + let metadataStr = ''; + + for (const key in metadata) { + const encodedKey = encodeURIComponent(key); + const encodedValue = encodeURIComponent(metadata[key]); + metadataStr += `&key=${encodedKey}&value=${encodedValue}`; + } + + let results: SchemaMetadata[] = []; + + for (const [key, value] of this.schemaToVersionCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && (!value.softDeleted || deleted)) { + if (parsedKey.schema.metadata && this.isSubset(metadata, parsedKey.schema.metadata.properties)) { + results.push({ id: parsedKey.schema.id, version: value.version, subject, schema: parsedKey.schema.schema }); + } + } + } + + if (results.length === 0) { + throw new Error("Schema not found"); + } + + let latest: SchemaMetadata = results[0]; + + results.forEach((result) => { + if (result.version! > latest.version!) { + latest = result; + } + }); + + return latest; + } + + private isSubset(containee: { [key: string]: string }, container: { [key: string]: string }){ + for (const key in containee) { + if (containee[key] !== container[key]) { + return false; + } + } + return true; + } + + public async getAllVersions(subject: string): Promise { + const results = await this.allVersions(subject); + + if (results.length === 0) { + throw new Error("No versions found for subject"); + } + return results; + } + + private async allVersions(subject: string): Promise { + const versions: number[] = []; + + for (const [key, value] of this.schemaToVersionCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && !value.softDeleted) { + versions.push(value.version); + } + } + return versions; + } + + private async latestVersion(subject: string): Promise { + const versions = await this.allVersions(subject); + if (versions.length === 0) { + return -1; + } + return versions[versions.length - 1]; + } + + private async deleteVersion(cacheKey: string, version: number, permanent: boolean): Promise { + if (permanent) { + this.schemaToVersionCache.delete(cacheKey); + } else { + this.schemaToVersionCache.set(cacheKey, { version, softDeleted: true }); + } + } + + private async deleteInfo(cacheKey: string, info: SchemaInfo, permanent: boolean): Promise { + if (permanent) { + this.idToSchemaCache.delete(cacheKey); + } else { + this.idToSchemaCache.set(cacheKey, { info, softDeleted: true }); + } + } + + private async deleteMetadata(cacheKey: string, metadata: SchemaMetadata, permanent: boolean): Promise { + if (permanent) { + this.infoToSchemaCache.delete(cacheKey); + } else { + this.infoToSchemaCache.set(cacheKey, { metadata, softDeleted: true }); + } + } + + public async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + const cacheKey = stringify({ subject, schema }); + const cacheEntry = this.schemaToVersionCache.get(cacheKey); + + if (!cacheEntry || cacheEntry.softDeleted) { + throw new Error("Schema not found"); + } + + return cacheEntry.version; + } + + public async getAllSubjects(): Promise { + const subjects: string[] = []; + for (const [key, value] of this.schemaToVersionCache.entries()) { + const parsedKey = JSON.parse(key); + if (!value.softDeleted && !subjects.includes(parsedKey.subject)) { + subjects.push(parsedKey.subject); + } + } + console.log(subjects.sort()); + return subjects.sort(); + } + + public async deleteSubject(subject: string, permanent: boolean = false): Promise { + const deletedVersions: number[] = []; + for (const [key, value] of this.infoToSchemaCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && (permanent || !value.softDeleted)) { + await this.deleteMetadata(key, value.metadata, permanent); + } + } + + for (const [key, value] of this.schemaToVersionCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && (permanent || !value.softDeleted)) { + await this.deleteVersion(key, value.version, permanent); + deletedVersions.push(value.version); + } + } + + this.configCache.delete(subject); + + if (permanent) { + for (const [key, value] of this.idToSchemaCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && (!value.softDeleted)) { + await this.deleteInfo(key, value.info, permanent); + } + } + } + + return deletedVersions; + } + + public async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { + for (const [key, value] of this.schemaToVersionCache.entries()) { + const parsedKey = JSON.parse(key); + if (parsedKey.subject === subject && value.version === version) { + await this.deleteVersion(key, version, permanent); + + const cacheKeySchema = stringify({ subject, schema: parsedKey.schema }); + const cacheEntry = this.infoToSchemaCache.get(cacheKeySchema); + if (cacheEntry) { + await this.deleteMetadata(cacheKeySchema, cacheEntry.metadata, permanent); + } + + if (permanent && cacheEntry) { + const cacheKeyInfo = stringify({ subject, id: cacheEntry.metadata.id }); + const cacheSchemaEntry = this.idToSchemaCache.get(cacheKeyInfo); + if (cacheSchemaEntry) { + await this.deleteInfo(cacheKeyInfo, cacheSchemaEntry.info, permanent); + } + } + } + } + + return version; + } + + public async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { + throw new Error("Unsupported operation"); + } + + public async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { + throw new Error("Unsupported operation"); + } + + public async getCompatibility(subject: string): Promise { + const cacheEntry = this.configCache.get(subject); + if (!cacheEntry) { + throw new Error("Subject not found"); + } + return cacheEntry.compatibilityLevel as Compatibility; + } + + public async updateCompatibility(subject: string, compatibility: Compatibility): Promise { + this.configCache.set(subject, { compatibilityLevel: compatibility }); + return compatibility; + } + + public async getDefaultCompatibility(): Promise { + const cacheEntry = this.configCache.get(noSubject); + if (!cacheEntry) { + throw new Error("Default compatibility not found"); + } + return cacheEntry.compatibilityLevel as Compatibility; + } + + public async updateDefaultCompatibility(compatibility: Compatibility): Promise { + this.configCache.set(noSubject, { compatibilityLevel: compatibility }); + return compatibility; + } + + public async getConfig(subject: string): Promise { + const cacheEntry = this.configCache.get(subject); + if (!cacheEntry) { + throw new Error("Subject not found"); + } + return cacheEntry; + } + + public async updateConfig(subject: string, config: ServerConfig): Promise { + this.configCache.set(subject, config); + return config; + } + + public async getDefaultConfig(): Promise { + const cacheEntry = this.configCache.get(noSubject); + if (!cacheEntry) { + throw new Error("Default config not found"); + } + return cacheEntry; + } + + public async updateDefaultConfig(config: ServerConfig): Promise { + this.configCache.set(noSubject, config); + return config; + } + + public async close(): Promise { + return; + } + + private schemasEqual(schema1: SchemaInfo, schema2: SchemaInfo): boolean { + return stringify(schema1) === stringify(schema2); + } +} + +export { MockClient }; \ No newline at end of file diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index 9f719974..4bd62459 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -86,7 +86,33 @@ interface isCompatibleResponse { is_compatible: boolean; } -class SchemaRegistryClient { +interface Client { + register(subject: string, schema: SchemaInfo, normalize: boolean): Promise; + registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean): Promise; + getBySubjectAndId(subject: string, id: number): Promise; + getId(subject: string, schema: SchemaInfo, normalize: boolean): Promise; + getLatestSchemaMetadata(subject: string): Promise; + getSchemaMetadata(subject: string, version: number, deleted: boolean): Promise; + getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean): Promise; + getAllVersions(subject: string): Promise; + getVersion(subject: string, schema: SchemaInfo, normalize: boolean): Promise; + getAllSubjects(): Promise; + deleteSubject(subject: string, permanent: boolean): Promise; + deleteSubjectVersion(subject: string, version: number, permanent: boolean): Promise; + testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise; + testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise; + getCompatibility(subject: string): Promise; + updateCompatibility(subject: string, update: Compatibility): Promise; + getDefaultCompatibility(): Promise; + updateDefaultCompatibility(update: Compatibility): Promise; + getConfig(subject: string): Promise; + updateConfig(subject: string, update: ServerConfig): Promise; + getDefaultConfig(): Promise; + updateDefaultConfig(update: ServerConfig): Promise; + close(): void; +} + +class SchemaRegistryClient implements Client { private restService: RestService; private schemaToIdCache: LRUCache; @@ -223,46 +249,25 @@ class SchemaRegistryClient { }); } - private convertToQueryParams(metadata: Metadata): string { - const params = new URLSearchParams(); - - if (metadata.tags) { - for (const [key, values] of Object.entries(metadata.tags)) { - values.forEach((value, index) => { - params.append(`tags.${key}[${index}]`, value); - }); - } - } - - if (metadata.properties) { - for (const [key, value] of Object.entries(metadata.properties)) { - params.append(`properties.${key}`, value); - } - } - - if (metadata.sensitive) { - metadata.sensitive.forEach((value, index) => { - params.append(`sensitive[${index}]`, value); - }); - } - - return params.toString(); - } - - //TODO: Get clarification with getLatestWithMetadata - public async getLatestWithMetadata(subject: string, metadata: Metadata, deleted: boolean = false): Promise { + public async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { const cacheKey = stringify({ subject, metadata, deleted }); - + return await this.metadataToSchemaMutex.runExclusive(async () => { const cachedSchemaMetadata: SchemaMetadata | undefined = this.metadataToSchemaCache.get(cacheKey); if (cachedSchemaMetadata) { return cachedSchemaMetadata; } - const queryParams = this.convertToQueryParams(metadata); + let metadataStr = ''; + + for (const key in metadata) { + const encodedKey = encodeURIComponent(key); + const encodedValue = encodeURIComponent(metadata[key]); + metadataStr += `&key=${encodedKey}&value=${encodedValue}`; + } const response: AxiosResponse = await this.restService.sendHttpRequest( - `/subjects/${subject}/metadata?deleted=${deleted}&${queryParams}`, + `/subjects/${subject}/metadata?deleted=${deleted}&${metadataStr}`, 'GET' ); this.metadataToSchemaCache.set(cacheKey, response.data); @@ -538,6 +543,6 @@ class SchemaRegistryClient { } export { - SchemaRegistryClient, SchemaInfo, Metadata, Compatibility, + Client, SchemaRegistryClient, SchemaInfo, Metadata, Compatibility, CompatibilityLevel, ServerConfig, RuleSet, Rule, Reference, SchemaMetadata, Result }; diff --git a/test/schemaregistry/mock-schemaregistery-client.spec.ts b/test/schemaregistry/mock-schemaregistery-client.spec.ts new file mode 100644 index 00000000..86bc2c4a --- /dev/null +++ b/test/schemaregistry/mock-schemaregistery-client.spec.ts @@ -0,0 +1,221 @@ +import { MockClient } from '../../schemaregistry/mock-schemaregistry-client'; +import { Compatibility, Metadata, SchemaInfo, SchemaMetadata } from '../../schemaregistry/schemaregistry-client'; +import { RestService } from '../../schemaregistry/rest-service'; +import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; + +const schemaString: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + ], +}); + +const schemaString2: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + { name: 'email', type: 'string' }, + ], +}); + +const metadata: Metadata = { + properties: { + owner: 'Alice Bob', + email: 'alice@bob.com', + } +}; + +const metadata2: Metadata = { + properties: { + owner: 'Alice Bob2', + email: 'alice@bob2.com' + } +}; + +const metadataKeyValue: { [key: string]: string } = { + owner: 'Alice Bob', + email: 'alice@bob.com' +}; + +const metadataKeyValue2: { [key: string]: string } = { + owner: 'Alice Bob2', + email: 'alice@bob2.com' +}; + +const schemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata +}; + +const schemaInfo2: SchemaInfo = { + schema: schemaString2, + metadata: metadata2 +}; + +const testSubject = 'test-subject'; +const testSubject2 = 'test-subject2'; + + +describe('MockClient-tests', () => { + let mockClient: MockClient; + let restService: RestService; + + beforeEach(() => { + mockClient = new MockClient(); + }); + + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return schemaId when calling register', async () => { + const response: number = await mockClient.register(testSubject, schemaInfo); + expect(response).toBe(1); + }); + + it('Should return SchemaMetadata when calling registerFullResponse', async () => { + const response: SchemaMetadata = await mockClient.registerFullResponse(testSubject, schemaInfo); + expect(response.id).toBe(1); + }); + + it('Should return SchemaInfo when getting with subject and Id', async () => { + await mockClient.register(testSubject, schemaInfo); + const response: SchemaInfo = await mockClient.getBySubjectAndId(testSubject, 1); + expect(response.schema).toBe(schemaString); + }); + + it('Should throw error when getBySubjectAndId is called with non-existing schemaId', async () => { + await mockClient.register(testSubject, schemaInfo); + await expect(mockClient.getBySubjectAndId(testSubject, 2)).rejects.toThrowError(); + }); + + it('Should return schemaId when calling getId', async () => { + await mockClient.register(testSubject, schemaInfo); + const response: number = await mockClient.getId(testSubject, schemaInfo); + expect(response).toBe(1); + }); + + it('Should throw error when getId is called with non-existing schema', async () => { + await expect(mockClient.getId(testSubject, schemaInfo)).rejects.toThrowError(); + }); + + it('Should return latest schema metadata when calling getLatestSchemaMetadata', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject, schemaInfo2); + const response: SchemaMetadata = await mockClient.getLatestSchemaMetadata(testSubject); + expect(response.id).toBe(2); + expect(response.schema).toBe(schemaString2); + }); + + it('Should return latest Schema with metadata when calling getLatestWithMetadata', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject, schemaInfo2); + const response = await mockClient.getLatestWithMetadata(testSubject, metadataKeyValue); + expect(response.schema).toBe(schemaString); + expect(response.version).toBe(1); + const response2 = await mockClient.getLatestWithMetadata(testSubject, metadataKeyValue2); + expect(response2.schema).toBe(schemaString2); + expect(response2.version).toBe(2); + }); + + it('Should return specific schemaMetadata version when calling getSchemaMetadata', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject, schemaInfo2); + const response: SchemaMetadata = await mockClient.getSchemaMetadata(testSubject, 1); + expect(response.id).toBe(1); + expect(response.schema).toBe(schemaString); + const response2: SchemaMetadata = await mockClient.getSchemaMetadata(testSubject, 2); + expect(response2.id).toBe(2); + expect(response2.schema).toBe(schemaString2); + }); + + it('Should return the correct version when calling getVersion', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject, schemaInfo2); + const response: number = await mockClient.getVersion(testSubject, schemaInfo2); + expect(response).toBe(2); + }); + + it('Should throw error when getVersion is called with non-existing schema', async () => { + await expect(mockClient.getVersion(testSubject, schemaInfo)).rejects.toThrowError(); + }); + + it('Should return all versions when calling getAllVersions', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject, schemaInfo2); + const response: number[] = await mockClient.getAllVersions(testSubject); + expect(response).toEqual([1, 2]); + }); + + it('Should update compatibility when calling updateCompatibility', async () => { + const response: Compatibility = await mockClient.updateCompatibility(testSubject, Compatibility.BackwardTransitive); + expect(response).toBe(Compatibility.BackwardTransitive); + }); + + it('Should return compatibility when calling getCompatibility', async () => { + await mockClient.updateCompatibility(testSubject, Compatibility.BackwardTransitive); + const response: Compatibility = await mockClient.getCompatibility(testSubject); + expect(response).toBe(Compatibility.BackwardTransitive); + }); + + it('Should throw error when getCompatibility is called with non-existing subject', async () => { + await expect(mockClient.getCompatibility(testSubject)).rejects.toThrowError(); + }); + + it('Should update default compatibility when calling updateDefaultCompatibility', async () => { + const response: Compatibility = await mockClient.updateDefaultCompatibility(Compatibility.BackwardTransitive); + expect(response).toBe(Compatibility.BackwardTransitive); + }); + + it('Should return default compatibility when calling getDefaultCompatibility', async () => { + await mockClient.updateDefaultCompatibility(Compatibility.BackwardTransitive); + const response: Compatibility = await mockClient.getDefaultCompatibility(); + expect(response).toBe(Compatibility.BackwardTransitive); + }); + + it('Should throw error when getDefaultCompatibility is called with non-existing default compatibility', async () => { + await expect(mockClient.getDefaultCompatibility()).rejects.toThrowError(); + }); + + it('Should get all subjects when calling getAllSubjects', async () => { + expect(await mockClient.getAllSubjects()).toEqual([]); + + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject2, schemaInfo); + const response: string[] = await mockClient.getAllSubjects(); + expect(response).toEqual([testSubject, testSubject2]); + }); + + it('Should soft delete subject when calling deleteSubject', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.deleteSubject(testSubject); + await expect(mockClient.getId(testSubject, schemaInfo)).rejects.toThrowError(); + await expect(mockClient.getVersion(testSubject, schemaInfo)).rejects.toThrowError(); + const response: SchemaInfo = await mockClient.getBySubjectAndId(testSubject, 1); + await expect(response.schema).toBe(schemaString); + }); + + it('Should permanent delete subject when calling deleteSubject with permanent flag', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.deleteSubject(testSubject, true); + await expect(mockClient.getId(testSubject, schemaInfo)).rejects.toThrowError(); + await expect(mockClient.getVersion(testSubject, schemaInfo)).rejects.toThrowError(); + await expect(mockClient.getBySubjectAndId(testSubject, 1)).rejects.toThrowError(); + }); + + it('Should soft delete subject version when calling deleteSubjectVersion', async () => { + await mockClient.register(testSubject, schemaInfo); + await mockClient.register(testSubject, schemaInfo2); + await mockClient.deleteSubjectVersion(testSubject, 1); + await expect(mockClient.getId(testSubject, schemaInfo)).rejects.toThrowError(); + await expect(mockClient.getVersion(testSubject, schemaInfo)).rejects.toThrowError(); + const response: SchemaInfo = await mockClient.getBySubjectAndId(testSubject, 1); + await expect(response.schema).toBe(schemaString); + const response2: SchemaInfo = await mockClient.getBySubjectAndId(testSubject, 2); + await expect(response2.schema).toBe(schemaString2); + }); +}); \ No newline at end of file diff --git a/test/schemaregistry/schemaregistry-client.spec.ts b/test/schemaregistry/schemaregistry-client.spec.ts index 77d13508..a146a2de 100644 --- a/test/schemaregistry/schemaregistry-client.spec.ts +++ b/test/schemaregistry/schemaregistry-client.spec.ts @@ -47,6 +47,15 @@ const metadata2: Metadata = { email: 'Alice@bob2.com', } }; +const metadataKeyValue = { + 'owner': 'Alice Bob', + 'email': 'Alice@bob.com', +} + +const metadataKeyValue2 = { + 'owner': 'Alice Bob2', + 'email': 'Alice@bob2.com' +}; const schemaInfo = { schema: schemaString, schemaType: 'AVRO', @@ -276,7 +285,7 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); - const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadataKeyValue); expect(response).toMatchObject(expectedResponse); expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); @@ -298,21 +307,21 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); - const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadataKeyValue); expect(response).toMatchObject(expectedResponse); expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); - const response2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); + const response2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadataKeyValue2); expect(response2).toMatchObject(expectedResponse2); expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); - const cachedResponse: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); + const cachedResponse: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadataKeyValue); expect(cachedResponse).toMatchObject(expectedResponse); expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); - const cachedResponse2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); + const cachedResponse2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadataKeyValue2); expect(cachedResponse2).toMatchObject(expectedResponse2); expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); }); @@ -367,57 +376,6 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { expect(cachedResponse2).toMatchObject(expectedResponse2); expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); }); - - it('Should get latest schema with metadata when GetLatestWithMetadata is called', async () => { - const expectedResponse = { - id: 1, - version: 1, - schema: schemaString, - metadata: metadata, - }; - - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); - - const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); - - expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); - }); - - it('Should get latest schema with metadata from cache when GetLatestWithMetadata is called twice', async () => { - const expectedResponse = { - id: 1, - version: 1, - schema: schemaString, - metadata: metadata, - }; - const expectedResponse2 = { - id: 2, - version: 1, - schema: schemaString2, - metadata: metadata2, - }; - - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); - - const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); - expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); - - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); - - const response2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); - expect(response2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); - - const cachedResponse: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadata); - expect(cachedResponse).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); - - const cachedResponse2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadata2); - expect(cachedResponse2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); - }); }); describe('SchemaRegistryClient-Subjects', () => { diff --git a/tsconfig.json b/tsconfig.json index 717448a9..2af9470f 100644 --- a/tsconfig.json +++ b/tsconfig.json @@ -18,4 +18,4 @@ "schemaregistry/**/*", "test/**/*" ] -} +} \ No newline at end of file From 285895419570db5ca15109a8488e9f87478126bf Mon Sep 17 00:00:00 2001 From: claimundefine Date: Fri, 9 Aug 2024 14:51:34 -0400 Subject: [PATCH 169/224] Remove testing artifacts --- Makefile.schemaregistry | 2 +- schemaregistry/mock-schemaregistry-client.ts | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/Makefile.schemaregistry b/Makefile.schemaregistry index f7d1c4f3..55b828ab 100644 --- a/Makefile.schemaregistry +++ b/Makefile.schemaregistry @@ -20,7 +20,7 @@ lint: $(ESLINT) $(SRC_DIR) $(TEST_DIR) $(INTEG_DIR) test: - $(JEST) $(TEST_DIR) --verbose + $(JEST) $(TEST_DIR) integtest: $(JEST) $(INTEG_DIR) \ No newline at end of file diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts index 477b1b7b..bde07ffb 100644 --- a/schemaregistry/mock-schemaregistry-client.ts +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -286,7 +286,6 @@ class MockClient implements Client { subjects.push(parsedKey.subject); } } - console.log(subjects.sort()); return subjects.sort(); } From 267d103c9dacde8eb711b4e4b8d730e38ad97df7 Mon Sep 17 00:00:00 2001 From: PratRanj07 <156985928+PratRanj07@users.noreply.github.com> Date: Mon, 12 Aug 2024 10:55:49 +0530 Subject: [PATCH 170/224] Fix flaky e2e tests (#54) * fixing empty message and key * indentation * 1st commit * changes * changes * alignment --- e2e/both.spec.js | 100 ++++++++++++++++++++++++++++++++--------------- 1 file changed, 68 insertions(+), 32 deletions(-) diff --git a/e2e/both.spec.js b/e2e/both.spec.js index 62983179..1b53840e 100644 --- a/e2e/both.spec.js +++ b/e2e/both.spec.js @@ -12,14 +12,16 @@ var t = require('assert'); var Kafka = require('../'); var kafkaBrokerList = process.env.KAFKA_HOST || 'localhost:9092'; +const { createTopics, deleteTopics } = require('./topicUtils'); var eventListener = require('./listener'); -var topic = 'test'; -var topic2 = 'test2'; +var topic; describe('Consumer/Producer', function() { - var producer; var consumer; + var grp; + + let createdTopics = []; beforeEach(function(done) { var finished = 0; @@ -36,12 +38,24 @@ describe('Consumer/Producer', function() { return done(err); } - if (finished === 2) { + if (finished === 3) { done(); } } - var grp = 'kafka-mocha-grp-' + crypto.randomBytes(20).toString('hex'); + grp = 'kafka-mocha-grp-' + crypto.randomBytes(20).toString('hex'); + topic = 'test' + crypto.randomBytes(20).toString('hex'); + + createTopics( + [{ topic, num_partitions: 1, replication_factor: 1 }], + kafkaBrokerList, + function(err) { + t.ifError(err); + maybeDone(err); + } + ); + + createdTopics.push(topic); consumer = new Kafka.KafkaConsumer({ 'metadata.broker.list': kafkaBrokerList, @@ -53,7 +67,7 @@ describe('Consumer/Producer', function() { 'debug': 'all' // paused: true, }, { - 'auto.offset.reset': 'largest' + 'auto.offset.reset': 'smallest' }); consumer.connect({}, function(err, d) { @@ -99,16 +113,21 @@ describe('Consumer/Producer', function() { return done(err); } - if (finished === 2) { + if (finished === 3) { done(); } } - consumer.disconnect(function(err) { + producer.disconnect(function(err) { maybeDone(err); }); - producer.disconnect(function(err) { + deleteTopics(createdTopics, kafkaBrokerList, function(err) { + createdTopics.length = 0; + maybeDone(err); + }); + + consumer.disconnect(function(err) { maybeDone(err); }); }); @@ -154,6 +173,7 @@ describe('Consumer/Producer', function() { t.equal(position.length, 1); t.deepStrictEqual(position[0].partition, 0); t.ok(position[0].offset >= 0); + consumer.unsubscribe(); done(); }); }; @@ -180,6 +200,7 @@ describe('Consumer/Producer', function() { consumer.consume(100000, function(err, messages) { t.ifError(err); t.equal(messages.length, 1); + consumer.unsubscribe(); done(); }); }; @@ -228,12 +249,13 @@ describe('Consumer/Producer', function() { setTimeout(function() { producer.produce(topic, null, buffer, null); - }, 500) - consumer.setDefaultConsumeTimeout(2000); + }, 500); + consumer.setDefaultConsumeTimeout(20000); consumer.consume(1000, function(err, messages) { t.ifError(err); t.equal(messages.length, 1); t.deepStrictEqual(events, ["data", "partition.eof"]); + consumer.unsubscribe(); done(); }); }); @@ -261,12 +283,13 @@ describe('Consumer/Producer', function() { setTimeout(function() { producer.produce(topic, null, buffer, null); - }, 2000) - consumer.setDefaultConsumeTimeout(3000); + }, 4000); + consumer.setDefaultConsumeTimeout(20000); consumer.consume(1000, function(err, messages) { t.ifError(err); t.equal(messages.length, 1); t.deepStrictEqual(events, ["partition.eof", "data", "partition.eof"]); + consumer.unsubscribe(); done(); }); }); @@ -276,7 +299,6 @@ describe('Consumer/Producer', function() { var key = 'key'; crypto.randomBytes(4096, function(ex, buffer) { - producer.setPollInterval(10); producer.once('delivery-report', function(err, report) { @@ -292,6 +314,7 @@ describe('Consumer/Producer', function() { t.equal(key, message.key, 'invalid message key'); t.equal(topic, message.topic, 'invalid message topic'); t.ok(message.offset >= 0, 'invalid message offset'); + consumer.unsubscribe(); done(); }); @@ -306,7 +329,6 @@ describe('Consumer/Producer', function() { }); it('should emit \'partition.eof\' events in consumeLoop', function(done) { - crypto.randomBytes(4096, function(ex, buffer) { producer.setPollInterval(10); @@ -314,7 +336,6 @@ describe('Consumer/Producer', function() { t.ifError(err); }); - var events = []; var offsets = []; @@ -337,11 +358,11 @@ describe('Consumer/Producer', function() { setTimeout(function() { producer.produce(topic, null, buffer); - }, 2000); + }, 4000); setTimeout(function() { producer.produce(topic, null, buffer); - }, 4000); + }, 6000); setTimeout(function() { t.deepStrictEqual(events, ['partition.eof', 'data', 'partition.eof', 'data', 'partition.eof']); @@ -352,8 +373,9 @@ describe('Consumer/Producer', function() { startOffset + 1, startOffset + 1, startOffset + 2 ]); + consumer.unsubscribe(); done(); - }, 6000); + }, 8000); }); }); @@ -386,16 +408,26 @@ describe('Consumer/Producer', function() { run_headers_test(done, headers); }); - it('should be able to produce and consume messages with one header value as int: consumeLoop', function(done) { + it('should be able to produce and consume messages with one header value as string with unicode: consumeLoop', function(done) { var headers = [ - { key: 10 } + { key: '10👍' }, + { key: 'こんにちは' }, + { key: '🌍🌎🌏' } ]; run_headers_test(done, headers); }); - it('should be able to produce and consume messages with one header value as float: consumeLoop', function(done) { + it('should be able to produce and consume messages with one header value as string with emojis: consumeLoop', function(done) { var headers = [ - { key: 1.11 } + { key: '😀😃😄😁' } + ]; + run_headers_test(done, headers); + }); + + it('should be able to produce and consume messages with one header value as string in other languages: consumeLoop', function(done) { + var headers = [ + { key: '你好' }, + { key: 'Привет' } ]; run_headers_test(done, headers); }); @@ -422,8 +454,8 @@ describe('Consumer/Producer', function() { it('should be able to produce and consume messages with multiple headers with mixed values: consumeLoop', function(done) { var headers = [ - { key1: 'value1' }, - { key2: Buffer.from('value2') }, + { key1: 'value1' }, + { key2: Buffer.from('value2') } ]; run_headers_test(done, headers); }); @@ -440,7 +472,7 @@ describe('Consumer/Producer', function() { const buffer = Buffer.from('value'); const key = 'key'; t.throws( - () => producer.produce(topic, null, buffer, key, null, "", headerCase), + () => producer.produce(topic, null, buffer, key, null, '', headerCase), 'must be string or buffer' ); } @@ -451,7 +483,7 @@ describe('Consumer/Producer', function() { it('should be able to produce and consume messages: empty buffer key and empty value', function(done) { var emptyString = ''; var key = Buffer.from(emptyString); - var value = Buffer.from(''); + var value = Buffer.from(emptyString); producer.setPollInterval(10); @@ -459,7 +491,8 @@ describe('Consumer/Producer', function() { t.notEqual(message.value, null, 'message should not be null'); t.equal(value.toString(), message.value.toString(), 'invalid message value'); t.equal(emptyString, message.key, 'invalid message key'); - done(); + consumer.unsubscribe(); + done(); }); consumer.subscribe([topic]); @@ -480,6 +513,7 @@ describe('Consumer/Producer', function() { t.notEqual(message.value, null, 'message should not be null'); t.equal(value.toString(), message.value.toString(), 'invalid message value'); t.equal(key, message.key, 'invalid message key'); + consumer.unsubscribe(); done(); }); @@ -500,6 +534,7 @@ describe('Consumer/Producer', function() { consumer.once('data', function(message) { t.equal(value, message.value, 'invalid message value'); t.equal(key, message.key, 'invalid message key'); + consumer.unsubscribe(); done(); }); @@ -525,7 +560,7 @@ describe('Consumer/Producer', function() { beforeEach(function(done) { consumer = new Kafka.KafkaConsumer(consumerOpts, { - 'auto.offset.reset': 'largest', + 'auto.offset.reset': 'smallest', }); consumer.connect({}, function(err, d) { @@ -569,6 +604,7 @@ describe('Consumer/Producer', function() { }); consumer.subscribe([topic]); + consumer.setDefaultConsumeTimeout(4000); consumer.consume(); setTimeout(function() { @@ -612,7 +648,7 @@ describe('Consumer/Producer', function() { } }; consumer = new Kafka.KafkaConsumer(consumerOpts, { - 'auto.offset.reset': 'largest', + 'auto.offset.reset': 'smallest', }); eventListener(consumer); @@ -620,6 +656,7 @@ describe('Consumer/Producer', function() { t.ifError(err); t.equal(typeof d, 'object', 'metadata should be returned'); consumer.subscribe([topic]); + consumer.setDefaultConsumeTimeout(4000); consumer.consume(); setTimeout(function() { producer.produce(topic, null, Buffer.from(''), ''); @@ -668,6 +705,7 @@ describe('Consumer/Producer', function() { t.equal(topic, message.topic, 'invalid message topic'); t.ok(message.offset >= 0, 'invalid message offset'); assert_headers_match(headers, message.headers); + consumer.unsubscribe(); done(); }); @@ -678,8 +716,6 @@ describe('Consumer/Producer', function() { var timestamp = new Date().getTime(); producer.produce(topic, null, buffer, key, timestamp, "", headers); }, 2000); - }); } - }); From a903b6e7a0e5234c72abfeaa369956264f79db41 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Tue, 13 Aug 2024 10:35:02 -0400 Subject: [PATCH 171/224] Preset fix (#6) * Fix jest to unblock tests * Add git history --- jest.config.js | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/jest.config.js b/jest.config.js index 8d146e52..a6110529 100644 --- a/jest.config.js +++ b/jest.config.js @@ -1,8 +1,6 @@ module.exports = { - preset: 'ts-jest', - testEnvironment: 'node', - testMatch: ['**/test/**/*.ts', '**/e2e/**/*.ts'], transform: { '^.+\\.tsx?$': 'ts-jest', }, - }; \ No newline at end of file + }; + \ No newline at end of file From 1be847f0c420bf60a5dfebeaf33c309ff8b34fbe Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 13 Aug 2024 12:01:12 +0530 Subject: [PATCH 172/224] Do not modify RegExps which don't start with a ^ 1. Adds test. Also fixes some flakiness in the subscribe test. 2. Fixes typing issue for assignment() --- MIGRATION.md | 2 +- lib/kafkajs/_common.js | 4 ++- lib/kafkajs/_consumer.js | 6 ++--- test/promisified/consumer/subscribe.spec.js | 30 +++++++++++++++++++-- types/kafkajs.d.ts | 2 +- 5 files changed, 36 insertions(+), 8 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 0225819e..d1c7e0e8 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -236,7 +236,7 @@ producerRun().then(consumerRun).catch(console.error); * `subscribe`: - - Regex flags are ignored while passing a topic subscription (like 'i' or 'g'). + - Regex flags are ignored while passing a topic subscription (like 'i' or 'g'). Regexes must start with '^', otherwise, an error is thrown. - Subscribe must be called only 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, for example, diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 4cc12bce..074f1f6a 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -235,7 +235,9 @@ const CompatibilityErrorMessages = Object.freeze({ subscribeOptionsMandatoryMissing: () => "The argument passed to subscribe must be an object, and must contain the 'topics' or the 'topic' property: {topics: string[]} or {topic: string}\n", subscribeOptionsRegexFlag: () => - "If subscribing to topic by RegExp, no flags are allowed. /abcd/ is okay, but /abcd/i is not.\n", + "If subscribing to topic by RegExp, no flags are allowed. /^abcd/ is okay, but /^abcd/i is not.\n", + subscribeOptionsRegexStart: () => + "If subscribing to topic by RegExp, the pattern must start with a '^'. If you want to use something like /abcd/, /^.*abcd/ must be used.\n", runOptionsAutoCommit: () => createReplacementErrorMessage('consumer', 'run', 'autoCommit', 'autoCommit: ', 'autoCommit: ', false), runOptionsAutoCommitInterval: () => diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 2dbf2a77..222f14e9 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -970,9 +970,9 @@ class Consumer { } const regexSource = topic.source; if (regexSource.charAt(0) !== '^') - return '^' + regexSource; - else - return regexSource; + throw new error.KafkaJSError(CompatibilityErrorMessages.subscribeOptionsRegexStart(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + + 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 }); } diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index 8619fe76..dc131c61 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -7,7 +7,8 @@ const { secureRandom, waitForMessages, waitForConsumerToJoinGroup, createProducer, - createConsumer } = require('../testhelpers'); + createConsumer, + sleep } = require('../testhelpers'); describe('Consumer', () => { let groupId, consumer, producer; @@ -44,6 +45,9 @@ describe('Consumer', () => { const topics = [`topic-${secureRandom()}`, `topic-${secureRandom()}`, regexMatchingTopic]; await Promise.all(topics.map(topic => createTopic({ topic }))); + /* It takes some time for the topics to be propagated in the metadata. We could check + * by listing topics in a loop, but this serves as well to get rid of flakiness. */ + await sleep(1000); const messagesConsumed = []; await consumer.connect(); @@ -123,10 +127,14 @@ describe('Consumer', () => { [topicUS, topicSE, topicUK, topicBR].map(topic => createTopic({ topic })) ); + /* It takes some time for the topics to be propagated in the metadata. We could check + * by listing topics in a loop, but this serves as well to get rid of flakiness. */ + await sleep(1000); + const messagesConsumed = []; await consumer.connect(); await consumer.subscribe({ - topic: new RegExp(`pattern-${testScope}-(se|br)-.*`), + topic: new RegExp(`^pattern-${testScope}-(se|br)-.*`), }); consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); @@ -151,5 +159,23 @@ describe('Consumer', () => { }); }); }); + + describe('throws if subscribing with a RegExp incorrectly', () => { + it('when RegExp contains a flag', async () => { + await consumer.connect(); + await expect(consumer.subscribe({ topics: [new RegExp('^test', 'g')] })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__INVALID_ARG + ); + }); + + it('when RegExp does not start with a ^', async () => { + await consumer.connect(); + await expect(consumer.subscribe({ topics: [new RegExp('test')] })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__INVALID_ARG + ); + }); + }); }); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 9ba640b4..b3387e84 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -321,7 +321,7 @@ export type Consumer = Client & { pause(topics: Array<{ topic: string; partitions?: number[] }>): void paused(): TopicPartitions[] resume(topics: Array<{ topic: string; partitions?: number[] }>): void - assignment(): TopicPartitions[] + assignment(): TopicPartition[] } export interface AdminConfig { From c4f3a43ece3d3544beb95634707ae3bdcfd150bf Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 14 Aug 2024 15:52:13 +0530 Subject: [PATCH 173/224] Fix argument mutation in run, pause and resume --- lib/kafkajs/_consumer.js | 51 ++++++++++++++++--------- test/promisified/consumer/pause.spec.js | 27 +++++++++++++ 2 files changed, 61 insertions(+), 17 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 222f14e9..e878bd79 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1012,21 +1012,24 @@ class Consumer { } this.#running = true; + /* We're going to add keys to the configuration, so make a copy */ + const configCopy = Object.assign({}, config); + /* Batches are auto resolved by default. */ if (!Object.hasOwn(config, 'eachBatchAutoResolve')) { - config.eachBatchAutoResolve = true; + configCopy.eachBatchAutoResolve = true; } if (!Object.hasOwn(config, 'partitionsConsumedConcurrently')) { - config.partitionsConsumedConcurrently = 1; + configCopy.partitionsConsumedConcurrently = 1; } const rdKafkaConfig = this.#config(); const maxPollInterval = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; - this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), config.partitionsConsumedConcurrently, this.#logger); + this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), configCopy.partitionsConsumedConcurrently, this.#logger); /* We deliberately don't await this because we want to return from this method immediately. */ - this.#runInternal(config); + this.#runInternal(configCopy); } /** @@ -1565,34 +1568,42 @@ class Consumer { throw new error.KafkaJSError('Pause can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } + const toppars = []; 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 }); } + const toppar = { topic: topic.topic }; + if (!topic.partitions) { - topic.partitions = this.#getAllAssignedPartition(topic.topic); + toppar.partitions = this.#getAllAssignedPartition(topic.topic); + } else { + toppar.partitions = [...topic.partitions]; } + + toppars.push(toppar); } - topics = this.#flattenTopicPartitions(topics); - if (topics.length === 0) { + const flattenedToppars = this.#flattenTopicPartitions(toppars); + if (flattenedToppars.length === 0) { return; } - this.#internalClient.pause(topics); + this.#internalClient.pause(flattenedToppars); /* Mark the messages in the cache as stale, runInternal* will deal with * making it unusable. */ - this.#messageCache.markStale(topics); + this.#messageCache.markStale(flattenedToppars); /* If anyone's using eachBatch, mark the batch as stale. */ - topics.map(partitionKey) + flattenedToppars.map(partitionKey) .filter(key => this.#topicPartitionToBatchPayload.has(key)) .forEach(key => this.#topicPartitionToBatchPayload.get(key)._stale = true); - topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); + flattenedToppars.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); - return () => this.resume(topics); + /* Note: we don't use flattenedToppars here because resume flattens them again. */ + return () => this.resume(toppars); } /** @@ -1627,23 +1638,29 @@ class Consumer { throw new error.KafkaJSError('Resume can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } + const toppars = []; 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 }); } + const toppar = { topic: topic.topic }; if (!topic.partitions) { - topic.partitions = this.#getAllAssignedPartition(topic.topic); + toppar.partitions = this.#getAllAssignedPartition(topic.topic); + } else { + toppar.partitions = [...topic.partitions]; } + + toppars.push(toppar); } - topics = this.#flattenTopicPartitions(topics); - if (topics.length === 0) { + const flattenedToppars = this.#flattenTopicPartitions(toppars); + if (flattenedToppars.length === 0) { return; } - this.#internalClient.resume(topics); + this.#internalClient.resume(flattenedToppars); - topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.delete(topicPartition)); + flattenedToppars.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 index af720834..c16bad39 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -654,5 +654,32 @@ describe('Consumer', () => { expect(consumer.paused()).toEqual([]); }, 10000); + + it('resumes via the function returned by pause', async () => { + await consumer.connect(); + consumer.subscribe({ topic: topics[0] }); + consumer.run({ + eachMessage: async () => {} + }); + + await waitFor(() => consumer.assignment().length > 0, () => { }, { delay: 10 }); + + const tp0 = { topic: topics[0], partitions: [0] }; + const tp1 = { topic: topics[0], partitions: [1] }; + + const resumeTopic0Partition0 = consumer.pause([ tp0 ]); + const resumeTopic0Partition1 = consumer.pause([ tp1 ]); + + let paused = consumer.paused(); + expect(paused).toEqual([{ topic: topics[0], partitions: [0, 1] }]); + + resumeTopic0Partition0(); + paused = consumer.paused(); + expect(paused).toEqual([{ topic: topics[0], partitions: [1] }]); + + resumeTopic0Partition1(); + paused = consumer.paused(); + expect(paused).toEqual([]); + }); }); }); From 4aee89fc187cbc55d8c9ec336a4c53170dfcc525 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Mon, 19 Aug 2024 12:09:55 -0400 Subject: [PATCH 174/224] Dekregistry client (#67) * Add Mock Schema Registry Client (#9) (#66) * Add mock client for testing * Remove testing artifacts * Dekregistry client (#10) * Add mock client for testing * Remove testing artifacts * Add dekregistry client, mock dekregistry client --------- Co-authored-by: Robert Yokota --------- Co-authored-by: Robert Yokota --- Makefile.schemaregistry | 7 +- dekregistry/constants.ts | 5 + dekregistry/dekregistry-client.ts | 218 ++++++++++++++++++ dekregistry/mock-dekregistry-client.ts | 97 ++++++++ test/dekregistry/dekregistry-client.spec.ts | 105 +++++++++ .../mock-dekregistry-client.spec.ts | 54 +++++ test/dekregistry/test-constants.ts | 74 ++++++ 7 files changed, 557 insertions(+), 3 deletions(-) create mode 100644 dekregistry/constants.ts create mode 100644 dekregistry/dekregistry-client.ts create mode 100644 dekregistry/mock-dekregistry-client.ts create mode 100644 test/dekregistry/dekregistry-client.spec.ts create mode 100644 test/dekregistry/mock-dekregistry-client.spec.ts create mode 100644 test/dekregistry/test-constants.ts diff --git a/Makefile.schemaregistry b/Makefile.schemaregistry index 55b828ab..096b22ab 100644 --- a/Makefile.schemaregistry +++ b/Makefile.schemaregistry @@ -8,7 +8,8 @@ TS_NODE ?= ./node_modules/.bin/ts-node # Paths SRC_DIR = schemaregistry -TEST_DIR = test/schemaregistry +SR_TEST_DIR = test/schemaregistry +DEK_TEST_DIR = test/dekregistry INTEG_DIR = e2e/schemaregistry # Tasks @@ -17,10 +18,10 @@ INTEG_DIR = e2e/schemaregistry all: lint test lint: - $(ESLINT) $(SRC_DIR) $(TEST_DIR) $(INTEG_DIR) + $(ESLINT) $(SRC_DIR) $(TEST_DIR) $(INTEG_DIR) $(DEK_TEST_DIR) test: - $(JEST) $(TEST_DIR) + $(JEST) $(SR_TEST_DIR) $(DEK_TEST_DIR) integtest: $(JEST) $(INTEG_DIR) \ No newline at end of file diff --git a/dekregistry/constants.ts b/dekregistry/constants.ts new file mode 100644 index 00000000..ce8dfce0 --- /dev/null +++ b/dekregistry/constants.ts @@ -0,0 +1,5 @@ +const MOCK_TS = 11112223334; + +export { + MOCK_TS +}; \ No newline at end of file diff --git a/dekregistry/dekregistry-client.ts b/dekregistry/dekregistry-client.ts new file mode 100644 index 00000000..886b62a9 --- /dev/null +++ b/dekregistry/dekregistry-client.ts @@ -0,0 +1,218 @@ +import { LRUCache } from 'lru-cache'; +import { Mutex } from 'async-mutex'; +import { RestService } from '../schemaregistry/rest-service'; +import stringify from 'json-stringify-deterministic'; + +interface Kek { + name?: string; + kmsType?: string; + kmsKeyId?: string; + kmsProps?: { [key: string]: string }; + doc?: string; + shared?: boolean; + ts?: number; + deleted?: boolean; +} + +interface CreateKekRequest { + name?: string; + kmsType?: string; + kmsKeyId?: string; + kmsProps?: { [key: string]: string }; + doc?: string; + shared?: boolean; +} + +interface Dek { + kekName?: string; + subject?: string; + version?: number; + algorithm?: string; + encryptedKeyMaterial?: string; + encryptedKeyMaterialBytes?: Buffer; + keyMaterial?: string; + keyMaterialBytes?: Buffer; + ts?: number; + deleted?: boolean; +} + +interface Client { + registerKek(name: string, kmsType: string, kmsKeyId: string, kmsProps: { [key: string]: string }, doc: string, shared: boolean): Promise; + getKek(name: string, deleted: boolean): Promise; + registerDek(kekName: string, subject: string, algorithm: string, encryptedKeyMaterial: string, version: number): Promise; + getDek(kekName: string, subject: string, algorithm: string, version: number, deleted: boolean): Promise; + close(): Promise; +} + +class DekRegistryClient implements Client { + private restService: RestService; + private kekCache: LRUCache; + private dekCache: LRUCache; + private kekMutex: Mutex; + private dekMutex: Mutex; + + constructor(restService: RestService, cacheSize: number = 512, cacheTTL?: number) { + const cacheOptions = { + max: cacheSize, + ...(cacheTTL !== undefined && { maxAge: cacheTTL }) + }; + + this.restService = restService; + this.kekCache = new LRUCache(cacheOptions); + this.dekCache = new LRUCache(cacheOptions); + this.kekMutex = new Mutex(); + this.dekMutex = new Mutex(); + } + + public static getEncryptedKeyMaterialBytes(dek: Dek): Buffer | null { + if (!dek.encryptedKeyMaterial) { + return null; + } + + if (!dek.encryptedKeyMaterialBytes) { + try { + const bytes = Buffer.from(dek.encryptedKeyMaterial, 'base64'); + dek.encryptedKeyMaterialBytes = bytes; + } catch (err) { + throw new Error(`Failed to decode base64 string: ${err.message}`); + } + } + + return dek.encryptedKeyMaterialBytes; + } + + public static getKeyMaterialBytes(dek: Dek): Buffer | null { + if (!dek.keyMaterial) { + return null; + } + + if (!dek.keyMaterialBytes) { + try { + const bytes = Buffer.from(dek.keyMaterial, 'base64'); + dek.keyMaterialBytes = bytes; + } catch (err) { + throw new Error(`Failed to decode base64 string: ${err.message}`); + } + } + + return dek.keyMaterialBytes; + } + + public static setKeyMaterial(dek: Dek, keyMaterialBytes: Buffer): void { + if (keyMaterialBytes) { + const str = keyMaterialBytes.toString('base64'); + dek.keyMaterial = str; + } + } + + public async registerKek(name: string, kmsType: string, kmsKeyId: string, + kmsProps: { [key: string]: string }, doc: string, shared: boolean): Promise { + const cacheKey = stringify({ name, deleted: false }); + + return await this.kekMutex.runExclusive(async () => { + const kek = this.kekCache.get(cacheKey); + if (kek) { + return kek; + } + + const request: CreateKekRequest = { + name, + kmsType, + kmsKeyId, + kmsProps, + doc, + shared, + }; + + const response = await this.restService.sendHttpRequest( + '/dek-registry/v1/keks', + 'POST', + request); + this.kekCache.set(cacheKey, response.data); + return response.data; + }); + } + + public async getKek(name: string, deleted: boolean = false): Promise { + const cacheKey = stringify({ name, deleted }); + + return await this.kekMutex.runExclusive(async () => { + const kek = this.kekCache.get(cacheKey); + if (kek) { + return kek; + } + name = encodeURIComponent(name); + + const response = await this.restService.sendHttpRequest( + `/dek-registry/v1/keks/${name}?deleted=${deleted}`, + 'GET'); + this.kekCache.set(cacheKey, response.data); + return response.data; + }); + } + + public async registerDek(kekName: string, subject: string, + algorithm: string, encryptedKeyMaterial: string, version: number = 1): Promise { + const cacheKey = stringify({ kekName, subject, version, algorithm, deleted: false }); + + return await this.dekMutex.runExclusive(async () => { + const dek = this.dekCache.get(cacheKey); + if (dek) { + return dek; + } + + const request: Dek = { + subject, + version, + algorithm, + encryptedKeyMaterial, + }; + kekName = encodeURIComponent(kekName); + + const response = await this.restService.sendHttpRequest( + `/dek-registry/v1/keks/${kekName}/deks`, + 'POST', + request); + this.dekCache.set(cacheKey, response.data); + + this.dekCache.delete(stringify({ kekName, subject, version: -1, algorithm, deleted: false })); + this.dekCache.delete(stringify({ kekName, subject, version: -1, algorithm, deleted: true })); + + return response.data; + }); + } + + public async getDek(kekName: string, subject: string, + algorithm: string, version: number = 1, deleted: boolean = false): Promise { + const cacheKey = stringify({ kekName, subject, version, algorithm, deleted }); + + return await this.dekMutex.runExclusive(async () => { + const dek = this.dekCache.get(cacheKey); + if (dek) { + return dek; + } + kekName = encodeURIComponent(kekName); + subject = encodeURIComponent(subject); + + const response = await this.restService.sendHttpRequest( + `/dek-registry/v1/keks/${kekName}/deks/${subject}/versions/${version}?deleted=${deleted}`, + 'GET'); + this.dekCache.set(cacheKey, response.data); + return response.data; + }); + } + + public async close(): Promise { + return; + } + + //Cache methods for testing + public async checkLatestDekInCache(kekName: string, subject: string, algorithm: string): Promise { + const cacheKey = stringify({ kekName, subject, version: -1, algorithm, deleted: false }); + const cachedDek = this.dekCache.get(cacheKey); + return cachedDek !== undefined; + } +} + +export { DekRegistryClient, Client, Kek, Dek }; + diff --git a/dekregistry/mock-dekregistry-client.ts b/dekregistry/mock-dekregistry-client.ts new file mode 100644 index 00000000..08d7e0ec --- /dev/null +++ b/dekregistry/mock-dekregistry-client.ts @@ -0,0 +1,97 @@ +import { Client, Dek, Kek } from "./dekregistry-client"; +import { MOCK_TS } from "./constants"; +import stringify from "json-stringify-deterministic"; + +class MockDekRegistryClient implements Client { + private kekCache: Map; + private dekCache: Map; + + constructor() { + this.kekCache = new Map(); + this.dekCache = new Map(); + } + + public async registerKek(name: string, kmsType: string, kmsKeyId: string, + kmsProps: { [key: string]: string }, doc: string, shared: boolean): Promise { + const cacheKey = stringify({ name, deleted: false }); + const cachedKek = this.kekCache.get(cacheKey); + if (cachedKek) { + return cachedKek; + } + + const kek: Kek = { + name, + kmsType, + kmsKeyId, + kmsProps, + doc, + shared + }; + + this.kekCache.set(cacheKey, kek); + return kek; + } + + public async getKek(name: string, deleted: boolean = false): Promise { + const cacheKey = stringify({ name, deleted }); + const cachedKek = this.kekCache.get(cacheKey); + if (cachedKek && (!cachedKek.deleted || deleted)) { + return cachedKek; + } + + throw new Error(`Kek not found: ${name}`); + } + + public async registerDek(kekName: string, subject: string, + algorithm: string, encryptedKeyMaterial: string, version: number): Promise { + const cacheKey = stringify({ kekName, subject, version, algorithm, deleted: false }); + const cachedDek = this.dekCache.get(cacheKey); + if (cachedDek) { + return cachedDek; + } + + const dek: Dek = { + kekName, + subject, + algorithm, + encryptedKeyMaterial, + version, + ts: MOCK_TS + }; + + this.dekCache.set(cacheKey, dek); + return dek; + } + + public async getDek(kekName: string, subject: string, + algorithm: string, version: number = 1, deleted: boolean = false): Promise { + if (version === -1) { + let latestVersion = 0; + for (let key of this.dekCache.keys()) { + const parsedKey = JSON.parse(key); + if (parsedKey.kekName === kekName && parsedKey.subject === subject + && parsedKey.algorithm === algorithm && parsedKey.deleted === deleted) { + latestVersion = Math.max(latestVersion, parsedKey.version); + } + } + if (latestVersion === 0) { + throw new Error(`Dek not found: ${subject}`); + } + version = latestVersion; + } + + const cacheKey = stringify({ kekName, subject, version, algorithm, deleted }); + const cachedDek = this.dekCache.get(cacheKey); + if (cachedDek) { + return cachedDek; + } + + throw new Error(`Dek not found: ${subject}`); + } + + public async close() { + return; + } +} + +export { MockDekRegistryClient }; \ No newline at end of file diff --git a/test/dekregistry/dekregistry-client.spec.ts b/test/dekregistry/dekregistry-client.spec.ts new file mode 100644 index 00000000..bd0823b7 --- /dev/null +++ b/test/dekregistry/dekregistry-client.spec.ts @@ -0,0 +1,105 @@ +import { DekRegistryClient, Dek, Kek } from "../../dekregistry/dekregistry-client"; +import { RestService } from "../../schemaregistry/rest-service"; +import { AxiosResponse } from 'axios'; +import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; +import { TEST_KEK, TEST_KEK_2, TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, + TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_2, TEST_ALGORITHM, + TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION, + TEST_DEK_LATEST} from "./test-constants"; + +jest.mock('../../schemaregistry/rest-service'); + +const baseUrls = ['http://mocked-url']; + +let client: DekRegistryClient; +let restService: jest.Mocked; + +describe('DekRegistryClient', () => { + + beforeEach(() => { + restService = new RestService(baseUrls) as jest.Mocked; + client = new DekRegistryClient(restService); + }); + + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should register kek when registerKek is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); + const response: Kek = await client.registerKek( + TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + + expect(response).toEqual(TEST_KEK); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return kek from cache when registerKek is called with same kek name', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); + await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK_2 } as AxiosResponse); + await client.registerKek(TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + + const response: Kek = await client.registerKek( + TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + const response2: Kek = await client.registerKek( + TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + + expect(response).toEqual(TEST_KEK); + expect(response2).toEqual(TEST_KEK_2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should return kek from cache when getKek is called with same kek name', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); + await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + const response: Kek = await client.getKek(TEST_KEK_NAME); + + expect(response).toEqual(TEST_KEK); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should register dek when registerDek is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + expect(response).toEqual(TEST_DEK); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should return dek from cache when registerDek is called with same kek name, subject, algorithm, and version', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK_2 } as AxiosResponse); + await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + + const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + const response2: Dek = await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + + expect(response).toEqual(TEST_DEK); + expect(response2).toEqual(TEST_DEK_2); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + }); + + it('Should return dek from cache when getDek is called with same kek name, subject, algorithm, and version', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + const response: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION); + + expect(response).toEqual(TEST_DEK); + expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + }); + + it('Should delete dek with version -1 when registerDek is called', async () => { + restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK_LATEST } as AxiosResponse); + const getDekResponse: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, -1); + expect(getDekResponse).toEqual(TEST_DEK_LATEST); + expect(await client.checkLatestDekInCache(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM)).toBe(true); + + restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + const getDekResponse2: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM); + + expect(getDekResponse2).toEqual(TEST_DEK); + expect(await client.checkLatestDekInCache(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM)).toBe(false); + }); +}); \ No newline at end of file diff --git a/test/dekregistry/mock-dekregistry-client.spec.ts b/test/dekregistry/mock-dekregistry-client.spec.ts new file mode 100644 index 00000000..2a2a1b3b --- /dev/null +++ b/test/dekregistry/mock-dekregistry-client.spec.ts @@ -0,0 +1,54 @@ +import { Dek, Kek } from "../../dekregistry/dekregistry-client"; +import { MockDekRegistryClient } from "../../dekregistry/mock-dekregistry-client"; +import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; +import { TEST_KEK, TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, + TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_V2, TEST_DEK_LATEST, TEST_ALGORITHM, + TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION } from "./test-constants"; + +describe('MockClient-tests', () => { + let mockClient: MockDekRegistryClient; + + beforeEach(() => { + mockClient = new MockDekRegistryClient(); + }); + + afterEach(() => { + jest.clearAllMocks(); + }); + + it('Should return kek when registering Kek', async () => { + const registerKekResponse: Kek = await mockClient.registerKek( + TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + + expect(registerKekResponse).toEqual(TEST_KEK); + }); + + it('Should return kek when getting Kek', async () => { + await mockClient.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + const getKekResponse: Kek = await mockClient.getKek(TEST_KEK_NAME); + + expect(getKekResponse).toEqual(TEST_KEK); + }); + + it('Should return dek when registering Dek', async () => { + const registerDekResponse: Dek = await mockClient.registerDek( + TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + + expect(registerDekResponse).toEqual(TEST_DEK); + }); + + it('Should return dek when getting Dek', async () => { + await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + const getDekResponse: Dek = await mockClient.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION); + + expect(getDekResponse).toEqual(TEST_DEK); + }); + + it('Should return latest dek when getting Dek with version -1', async () => { + await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, 2); + await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + const getDekResponse: Dek = await mockClient.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, -1); + + expect(getDekResponse).toEqual(TEST_DEK_V2); + }); + }); \ No newline at end of file diff --git a/test/dekregistry/test-constants.ts b/test/dekregistry/test-constants.ts new file mode 100644 index 00000000..e88387cb --- /dev/null +++ b/test/dekregistry/test-constants.ts @@ -0,0 +1,74 @@ +import { MOCK_TS } from "../../dekregistry/constants"; +import { Kek, Dek } from "../../dekregistry/dekregistry-client"; + +const TEST_KEK_NAME: string = 'test-kek-name'; +const TEST_KEK_NAME_2: string = 'test-kek-name2'; +const TEST_KMS_TYPE: string = 'test-kms-type'; +const TEST_KMS_KEY_ID: string = 'test-kms-key-id'; +const TEST_KMS_PROPS = { testKey: 'testValue' }; +const TEST_DOC: string = 'test-doc'; + +const TEST_SUBJECT: string = 'test-subject'; +const TEST_ALGORITHM: string = 'test-algorithm'; +const TEST_ENCRYPTED_KEY_MATERIAL: string = 'test-encrypted-key-material'; +const TEST_VERSION: number = 1; + +const TEST_KEK: Kek = { + name: TEST_KEK_NAME, + kmsType: TEST_KMS_TYPE, + kmsKeyId: TEST_KMS_KEY_ID, + kmsProps: TEST_KMS_PROPS, + doc: TEST_DOC, + shared: true +}; + +const TEST_KEK_2: Kek = { + name: TEST_KEK_NAME_2, + kmsType: TEST_KMS_TYPE, + kmsKeyId: TEST_KMS_KEY_ID, + kmsProps: TEST_KMS_PROPS, + doc: TEST_DOC, + shared: true +}; + +const TEST_DEK: Dek = { + kekName: TEST_KEK_NAME, + subject: TEST_SUBJECT, + algorithm: TEST_ALGORITHM, + encryptedKeyMaterial: TEST_ENCRYPTED_KEY_MATERIAL, + version: TEST_VERSION, + ts: MOCK_TS +}; + +const TEST_DEK_V2: Dek = { + kekName: TEST_KEK_NAME, + subject: TEST_SUBJECT, + algorithm: TEST_ALGORITHM, + encryptedKeyMaterial: TEST_ENCRYPTED_KEY_MATERIAL, + version: 2, + ts: MOCK_TS +}; + +const TEST_DEK_2: Dek = { + kekName: TEST_KEK_NAME_2, + subject: TEST_SUBJECT, + algorithm: TEST_ALGORITHM, + encryptedKeyMaterial: TEST_ENCRYPTED_KEY_MATERIAL, + version: TEST_VERSION, + ts: MOCK_TS +}; + +const TEST_DEK_LATEST: Dek = { + kekName: TEST_KEK_NAME, + subject: TEST_SUBJECT, + algorithm: TEST_ALGORITHM, + encryptedKeyMaterial: TEST_ENCRYPTED_KEY_MATERIAL, + version: -1, + ts: MOCK_TS +}; + +export { + TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, + TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION, + TEST_KEK, TEST_KEK_2, TEST_DEK, TEST_DEK_V2, TEST_DEK_2, TEST_DEK_LATEST +}; \ No newline at end of file From d73496fd51e5eb98dcdc8006f4b516dc21d33656 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Tue, 20 Aug 2024 22:18:52 -0400 Subject: [PATCH 175/224] Add clientConfig, baseUrl retry, RestError, encodeURIComponent (#12) (#68) * Add clientConfig, baseUrl retry, RestError * refactor such that RestService takes in necessary dependencies --- dekregistry/dekregistry-client.ts | 28 ++- .../schemaregistry-client.spec.ts | 20 +- schemaregistry/rest-error.ts | 10 + schemaregistry/rest-service.ts | 64 +++--- schemaregistry/schemaregistry-client.ts | 144 +++++++------ test/dekregistry/dekregistry-client.spec.ts | 39 ++-- .../schemaregistry-client.spec.ts | 194 +++++++++--------- test/schemaregistry/test-constants.ts | 35 ++++ 8 files changed, 315 insertions(+), 219 deletions(-) create mode 100644 schemaregistry/rest-error.ts create mode 100644 test/schemaregistry/test-constants.ts diff --git a/dekregistry/dekregistry-client.ts b/dekregistry/dekregistry-client.ts index 886b62a9..ff59fa5a 100644 --- a/dekregistry/dekregistry-client.ts +++ b/dekregistry/dekregistry-client.ts @@ -1,8 +1,17 @@ import { LRUCache } from 'lru-cache'; import { Mutex } from 'async-mutex'; -import { RestService } from '../schemaregistry/rest-service'; +import { ClientConfig, RestService } from '../schemaregistry/rest-service'; import stringify from 'json-stringify-deterministic'; +/* + * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry + * + * Copyright (c) 2024 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ + interface Kek { name?: string; kmsType?: string; @@ -51,13 +60,14 @@ class DekRegistryClient implements Client { private kekMutex: Mutex; private dekMutex: Mutex; - constructor(restService: RestService, cacheSize: number = 512, cacheTTL?: number) { + constructor(config: ClientConfig) { const cacheOptions = { - max: cacheSize, - ...(cacheTTL !== undefined && { maxAge: cacheTTL }) + max: config.cacheCapacity, + ...(config.cacheLatestTtlSecs !== undefined && { maxAge: config.cacheLatestTtlSecs * 1000 }), }; - this.restService = restService; + + this.restService = new RestService(config.createAxiosDefaults, config.baseURLs, config.isForward); this.kekCache = new LRUCache(cacheOptions); this.dekCache = new LRUCache(cacheOptions); this.kekMutex = new Mutex(); @@ -124,7 +134,7 @@ class DekRegistryClient implements Client { shared, }; - const response = await this.restService.sendHttpRequest( + const response = await this.restService.handleRequest( '/dek-registry/v1/keks', 'POST', request); @@ -143,7 +153,7 @@ class DekRegistryClient implements Client { } name = encodeURIComponent(name); - const response = await this.restService.sendHttpRequest( + const response = await this.restService.handleRequest( `/dek-registry/v1/keks/${name}?deleted=${deleted}`, 'GET'); this.kekCache.set(cacheKey, response.data); @@ -169,7 +179,7 @@ class DekRegistryClient implements Client { }; kekName = encodeURIComponent(kekName); - const response = await this.restService.sendHttpRequest( + const response = await this.restService.handleRequest( `/dek-registry/v1/keks/${kekName}/deks`, 'POST', request); @@ -194,7 +204,7 @@ class DekRegistryClient implements Client { kekName = encodeURIComponent(kekName); subject = encodeURIComponent(subject); - const response = await this.restService.sendHttpRequest( + const response = await this.restService.handleRequest( `/dek-registry/v1/keks/${kekName}/deks/${subject}/versions/${version}?deleted=${deleted}`, 'GET'); this.dekCache.set(cacheKey, response.data); diff --git a/e2e/schemaregistry/schemaregistry-client.spec.ts b/e2e/schemaregistry/schemaregistry-client.spec.ts index 3854e2f2..2b0684fa 100644 --- a/e2e/schemaregistry/schemaregistry-client.spec.ts +++ b/e2e/schemaregistry/schemaregistry-client.spec.ts @@ -1,4 +1,3 @@ -import { RestService } from '../../schemaregistry/rest-service'; import { Compatibility, SchemaRegistryClient, @@ -8,19 +7,10 @@ import { Metadata } from '../../schemaregistry/schemaregistry-client'; import { beforeEach, describe, expect, it } from '@jest/globals'; +import { clientConfig } from '../../test/schemaregistry/test-constants'; /* eslint-disable @typescript-eslint/no-non-null-asserted-optional-chain */ -const baseUrls = ['http://localhost:8081']; -const headers = { 'Content-Type': 'application/vnd.schemaregistry.v1+json' }; -const restService = new RestService(baseUrls, false); -restService.setHeaders(headers); - -const basicAuth = Buffer.from('RBACAllowedUser-lsrc1:nohash').toString('base64'); -restService.setAuth(basicAuth); - -restService.setTimeout(10000); - let schemaRegistryClient: SchemaRegistryClient; const testSubject = 'integ-test-subject'; const testServerConfigSubject = 'integ-test-server-config-subject'; @@ -72,7 +62,7 @@ const backwardCompatibleSchemaInfo: SchemaInfo = { describe('SchemaRegistryClient Integration Test', () => { beforeEach(async () => { - schemaRegistryClient = new SchemaRegistryClient(restService); + schemaRegistryClient = new SchemaRegistryClient(clientConfig); const subjects: string[] = await schemaRegistryClient.getAllSubjects(); if (subjects && subjects.includes(testSubject)) { @@ -86,7 +76,11 @@ describe('SchemaRegistryClient Integration Test', () => { } }); - it('should register, retrieve, and delete a schema', async () => { + it("Should return RestError when retrieving non-existent schema", async () => { + await expect(schemaRegistryClient.getBySubjectAndId(testSubject, 1)).rejects.toThrow(); + }); + + it('Should register, retrieve, and delete a schema', async () => { // Register a schema const registerResponse: SchemaMetadata = await schemaRegistryClient.registerFullResponse(testSubject, schemaInfo); expect(registerResponse).toBeDefined(); diff --git a/schemaregistry/rest-error.ts b/schemaregistry/rest-error.ts new file mode 100644 index 00000000..19fd086a --- /dev/null +++ b/schemaregistry/rest-error.ts @@ -0,0 +1,10 @@ +export class RestError extends Error { + status: number; + errorCode: number; + + constructor(message: string, status: number, errorCode: number) { + super(message + "; Error code: " + errorCode); + this.status = status; + this.errorCode = errorCode; + } +} \ No newline at end of file diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index e93d7746..333fc46a 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -1,4 +1,5 @@ -import axios, { AxiosInstance, AxiosRequestConfig, AxiosResponse } from 'axios'; +import axios, { AxiosInstance, AxiosRequestConfig, AxiosResponse, CreateAxiosDefaults } from 'axios'; +import { RestError } from './rest-error'; /* * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry @@ -9,43 +10,60 @@ import axios, { AxiosInstance, AxiosRequestConfig, AxiosResponse } from 'axios'; * of the MIT license. See the LICENSE.txt file for details. */ +export type ClientConfig = { + createAxiosDefaults: CreateAxiosDefaults, + baseURLs: string[], + cacheCapacity: number, + cacheLatestTtlSecs?: number, + isForward?: boolean +} + export class RestService { - private client: AxiosInstance + private client: AxiosInstance; + private baseURLs: string[]; - constructor(baseUrls: string[], isForward = false) { - this.client = axios.create({ - baseURL: baseUrls[0], // Use the first base URL as the default - timeout: 5000, // Default timeout - headers: { 'Content-Type': 'application/vnd.schemaregistry.v1+json' }, - }) + constructor(axiosDefaults: CreateAxiosDefaults, baseURLs: string[], isForward?: boolean) { + this.client = axios.create(axiosDefaults); + this.baseURLs = baseURLs; if (isForward) { this.client.defaults.headers.common['X-Forward'] = 'true' } } - public async sendHttpRequest( + public async handleRequest( url: string, method: 'GET' | 'POST' | 'PUT' | 'DELETE', data?: any, // eslint-disable-line @typescript-eslint/no-explicit-any config?: AxiosRequestConfig, ): Promise> { - try { - const response = await this.client.request({ - url, - method, - data, - ...config, - }) - return response - } catch (error) { - if (axios.isAxiosError(error) && error.response) { - throw new Error(`HTTP error: ${error.response.status} - ${error.response.data}`) - } else { - const err = error as Error; - throw new Error(`Unknown error: ${err.message}`) + + for (let i = 0; i < this.baseURLs.length; i++) { + try { + this.setBaseURL(this.baseURLs[i]); + const response = await this.client.request({ + url, + method, + data, + ...config, + }) + return response; + } catch (error) { + if (axios.isAxiosError(error) && error.response && (error.response.status < 200 || error.response.status > 299)) { + const data = error.response.data; + if (data.error_code && data.message) { + error = new RestError(data.message, error.response.status, data.error_code); + } else { + error = new Error(`Unknown error: ${error.message}`) + } + } + if (i === this.baseURLs.length - 1) { + throw error; + } } } + + throw new Error('Internal HTTP retry error'); // Should never reach here } public setHeaders(headers: Record): void { diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index 4bd62459..bb0a25e3 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -1,4 +1,4 @@ -import { RestService } from './rest-service'; +import { RestService, ClientConfig } from './rest-service'; import { AxiosResponse } from 'axios'; import stringify from "json-stringify-deterministic"; import { LRUCache } from 'lru-cache'; @@ -28,11 +28,6 @@ interface CompatibilityLevel { compatibilityLevel?: Compatibility; } -interface Result { - data?: T; - error?: Error; -} - interface Rule { name: string; subject: string; @@ -131,13 +126,14 @@ class SchemaRegistryClient implements Client { private versionToSchemaMutex: Mutex; private metadataToSchemaMutex: Mutex; - constructor(restService: RestService, cacheSize: number = 512, cacheTTL?: number) { + constructor(config: ClientConfig) { const cacheOptions = { - max: cacheSize, - ...(cacheTTL !== undefined && { maxAge: cacheTTL }) + max: config.cacheCapacity, + ...(config.cacheLatestTtlSecs !== undefined && { maxAge: config.cacheLatestTtlSecs * 1000 }) }; - this.restService = restService; + this.restService = new RestService(config.createAxiosDefaults, config.baseURLs, config.isForward); + this.schemaToIdCache = new LRUCache(cacheOptions); this.idToSchemaInfoCache = new LRUCache(cacheOptions); this.infoToSchemaCache = new LRUCache(cacheOptions); @@ -169,7 +165,9 @@ class SchemaRegistryClient implements Client { return cachedSchemaMetadata; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions?normalize=${normalize}`, 'POST', schema @@ -187,7 +185,9 @@ class SchemaRegistryClient implements Client { return cachedSchema; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/schemas/ids/${id}?subject=${subject}`, 'GET' ); @@ -205,7 +205,9 @@ class SchemaRegistryClient implements Client { return cachedId; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}?normalize=${normalize}`, 'POST', schema @@ -222,7 +224,9 @@ class SchemaRegistryClient implements Client { return cachedSchema; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions/latest`, 'GET' ); @@ -240,7 +244,9 @@ class SchemaRegistryClient implements Client { return cachedSchemaMetadata; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions/${version}?deleted=${deleted}`, 'GET' ); @@ -251,13 +257,15 @@ class SchemaRegistryClient implements Client { public async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { const cacheKey = stringify({ subject, metadata, deleted }); - + return await this.metadataToSchemaMutex.runExclusive(async () => { const cachedSchemaMetadata: SchemaMetadata | undefined = this.metadataToSchemaCache.get(cacheKey); if (cachedSchemaMetadata) { return cachedSchemaMetadata; } + subject = encodeURIComponent(subject); + let metadataStr = ''; for (const key in metadata) { @@ -266,7 +274,7 @@ class SchemaRegistryClient implements Client { metadataStr += `&key=${encodedKey}&value=${encodedValue}`; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/metadata?deleted=${deleted}&${metadataStr}`, 'GET' ); @@ -277,7 +285,7 @@ class SchemaRegistryClient implements Client { public async getAllVersions(subject: string): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions`, 'GET' ); @@ -293,7 +301,9 @@ class SchemaRegistryClient implements Client { return cachedVersion; } - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}?normalize=${normalize}`, 'POST', schema @@ -304,7 +314,7 @@ class SchemaRegistryClient implements Client { } public async getAllSubjects(): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( + const response: AxiosResponse = await this.restService.handleRequest( `/subjects`, 'GET' ); @@ -348,7 +358,9 @@ class SchemaRegistryClient implements Client { }); }); - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}?permanent=${permanent}`, 'DELETE' ); @@ -384,7 +396,9 @@ class SchemaRegistryClient implements Client { this.versionToSchemaCache.delete(cacheKey); }); - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions/${version}?permanent=${permanent}`, 'DELETE' ); @@ -393,7 +407,9 @@ class SchemaRegistryClient implements Client { } public async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/compatibility/subjects/${subject}/versions/latest`, 'POST', schema @@ -402,7 +418,9 @@ class SchemaRegistryClient implements Client { } public async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/compatibility/subjects/${subject}/versions/${version}`, 'POST', schema @@ -411,7 +429,9 @@ class SchemaRegistryClient implements Client { } public async getCompatibility(subject: string): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/config/${subject}`, 'GET' ); @@ -419,7 +439,9 @@ class SchemaRegistryClient implements Client { } public async updateCompatibility(subject: string, update: Compatibility): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( `/config/${subject}`, 'PUT', { compatibility: update } @@ -428,54 +450,56 @@ class SchemaRegistryClient implements Client { } public async getDefaultCompatibility(): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( - `/config`, - 'GET' - ); - return response.data.compatibilityLevel!; + const response: AxiosResponse = await this.restService.handleRequest( + `/config`, + 'GET' + ); + return response.data.compatibilityLevel!; } public async updateDefaultCompatibility(update: Compatibility): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( - `/config`, - 'PUT', - { compatibility: update } - ); - return response.data.compatibility!; + const response: AxiosResponse = await this.restService.handleRequest( + `/config`, + 'PUT', + { compatibility: update } + ); + return response.data.compatibility!; } public async getConfig(subject: string): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( - `/config/${subject}`, - 'GET' - ); - return response.data; + subject = encodeURIComponent(subject); + + const response: AxiosResponse = await this.restService.handleRequest( + `/config/${subject}`, + 'GET' + ); + return response.data; } public async updateConfig(subject: string, update: ServerConfig): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( - `/config/${subject}`, - 'PUT', - update - ); - return response.data; + const response: AxiosResponse = await this.restService.handleRequest( + `/config/${subject}`, + 'PUT', + update + ); + return response.data; } public async getDefaultConfig(): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( - `/config`, - 'GET' - ); - return response.data; + const response: AxiosResponse = await this.restService.handleRequest( + `/config`, + 'GET' + ); + return response.data; } public async updateDefaultConfig(update: ServerConfig): Promise { - const response: AxiosResponse = await this.restService.sendHttpRequest( - `/config`, - 'PUT', - update - ); - return response.data; + const response: AxiosResponse = await this.restService.handleRequest( + `/config`, + 'PUT', + update + ); + return response.data; } public close(): void { @@ -544,5 +568,5 @@ class SchemaRegistryClient implements Client { export { Client, SchemaRegistryClient, SchemaInfo, Metadata, Compatibility, - CompatibilityLevel, ServerConfig, RuleSet, Rule, Reference, SchemaMetadata, Result + CompatibilityLevel, ServerConfig, RuleSet, Rule, Reference, SchemaMetadata }; diff --git a/test/dekregistry/dekregistry-client.spec.ts b/test/dekregistry/dekregistry-client.spec.ts index bd0823b7..f6de5fcd 100644 --- a/test/dekregistry/dekregistry-client.spec.ts +++ b/test/dekregistry/dekregistry-client.spec.ts @@ -6,10 +6,10 @@ import { TEST_KEK, TEST_KEK_2, TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TE TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_2, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION, TEST_DEK_LATEST} from "./test-constants"; +import { mockClientConfig } from "../schemaregistry/test-constants"; jest.mock('../../schemaregistry/rest-service'); -const baseUrls = ['http://mocked-url']; let client: DekRegistryClient; let restService: jest.Mocked; @@ -17,8 +17,9 @@ let restService: jest.Mocked; describe('DekRegistryClient', () => { beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new DekRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new DekRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { @@ -26,18 +27,18 @@ describe('DekRegistryClient', () => { }); it('Should register kek when registerKek is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); const response: Kek = await client.registerKek( TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); expect(response).toEqual(TEST_KEK); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return kek from cache when registerKek is called with same kek name', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); - restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK_2 } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_KEK_2 } as AxiosResponse); await client.registerKek(TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); const response: Kek = await client.registerKek( @@ -47,29 +48,29 @@ describe('DekRegistryClient', () => { expect(response).toEqual(TEST_KEK); expect(response2).toEqual(TEST_KEK_2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); it('Should return kek from cache when getKek is called with same kek name', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); const response: Kek = await client.getKek(TEST_KEK_NAME); expect(response).toEqual(TEST_KEK); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should register dek when registerDek is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); expect(response).toEqual(TEST_DEK); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return dek from cache when registerDek is called with same kek name, subject, algorithm, and version', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); - restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK_2 } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_DEK_2 } as AxiosResponse); await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); @@ -77,25 +78,25 @@ describe('DekRegistryClient', () => { expect(response).toEqual(TEST_DEK); expect(response2).toEqual(TEST_DEK_2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); it('Should return dek from cache when getDek is called with same kek name, subject, algorithm, and version', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); const response: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION); expect(response).toEqual(TEST_DEK); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should delete dek with version -1 when registerDek is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK_LATEST } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_DEK_LATEST } as AxiosResponse); const getDekResponse: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, -1); expect(getDekResponse).toEqual(TEST_DEK_LATEST); expect(await client.checkLatestDekInCache(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM)).toBe(true); - restService.sendHttpRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); const getDekResponse2: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM); diff --git a/test/schemaregistry/schemaregistry-client.spec.ts b/test/schemaregistry/schemaregistry-client.spec.ts index a146a2de..6c43ba6e 100644 --- a/test/schemaregistry/schemaregistry-client.spec.ts +++ b/test/schemaregistry/schemaregistry-client.spec.ts @@ -10,11 +10,10 @@ import { RestService } from '../../schemaregistry/rest-service'; import { AxiosResponse } from 'axios'; import stringify from "json-stringify-deterministic"; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; +import { mockClientConfig } from '../../test/schemaregistry/test-constants'; jest.mock('../../schemaregistry/rest-service'); -const baseUrls = ['http://mocked-url']; - let client: SchemaRegistryClient; let restService: jest.Mocked; const mockSubject = 'mock-subject'; @@ -80,8 +79,9 @@ const versions: number[] = [1, 2, 3]; describe('SchemaRegistryClient-Register', () => { beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new SchemaRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new SchemaRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { @@ -89,37 +89,37 @@ describe('SchemaRegistryClient-Register', () => { }); it('Should return id when Register is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); const response: number = await client.register(mockSubject, schemaInfo); expect(response).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return from cache when Register is called twice', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); const response: number = await client.register(mockSubject, schemaInfo); expect(response).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: { id: 2 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { id: 2 } } as AxiosResponse); const response2: number = await client.register(mockSubject2, schemaInfo2); expect(response2).toEqual(2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); //Try to create same objects again const cachedResponse: number = await client.register(mockSubject, schemaInfo); expect(cachedResponse).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: number = await client.register(mockSubject2, schemaInfo2); expect(cachedResponse2).toEqual(2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); it('Should return id, version, metadata, and schema when RegisterFullResponse is called', async () => { @@ -130,12 +130,12 @@ describe('SchemaRegistryClient-Register', () => { metadata: metadata, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaMetadata = await client.registerFullResponse(mockSubject, schemaInfoMetadata); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return id, version, metadata, and schema from cache when RegisterFullResponse is called twice', async () => { @@ -152,66 +152,67 @@ describe('SchemaRegistryClient-Register', () => { metadata: metadata2, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaMetadata = await client.registerFullResponse(mockSubject, schemaInfoMetadata); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); const response2: SchemaMetadata = await client.registerFullResponse(mockSubject2, schemaInfoMetadata2); expect(response2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse: SchemaMetadata = await client.registerFullResponse(mockSubject, schemaInfoMetadata); expect(cachedResponse).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: SchemaMetadata = await client.registerFullResponse(mockSubject2, schemaInfoMetadata2); expect(cachedResponse2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); }); describe('SchemaRegistryClient-Get-ID', () => { beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new SchemaRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new SchemaRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { jest.clearAllMocks(); }); it('Should return id when GetId is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); const response: number = await client.getId(mockSubject, schemaInfo); expect(response).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return id from cache when GetId is called twice', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { id: 1 } } as AxiosResponse); const response: number = await client.getId(mockSubject, schemaInfo); expect(response).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: { id: 2 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { id: 2 } } as AxiosResponse); const response2: number = await client.getId(mockSubject2, schemaInfo2); expect(response2).toEqual(2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse: number = await client.getId(mockSubject, schemaInfo); expect(cachedResponse).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: number = await client.getId(mockSubject2, schemaInfo2); expect(cachedResponse2).toEqual(2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); it('Should return SchemaInfo when GetBySubjectAndId is called', async () => { @@ -222,12 +223,12 @@ describe('SchemaRegistryClient-Get-ID', () => { metadata: metadata, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaInfo = await client.getBySubjectAndId(mockSubject, 1); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return SchemaInfo from cache when GetBySubjectAndId is called twice', async () => { @@ -244,32 +245,33 @@ describe('SchemaRegistryClient-Get-ID', () => { metadata: metadata2, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaInfo = await client.getBySubjectAndId(mockSubject, 1); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); const response2: SchemaInfo = await client.getBySubjectAndId(mockSubject2, 2); expect(response2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse: SchemaInfo = await client.getBySubjectAndId(mockSubject, 1); expect(cachedResponse).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: SchemaInfo = await client.getBySubjectAndId(mockSubject2, 2); expect(cachedResponse2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); }); describe('SchemaRegistryClient-Get-Schema-Metadata', () => { beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new SchemaRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new SchemaRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { jest.clearAllMocks(); @@ -283,12 +285,12 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { metadata: metadata, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadataKeyValue); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return latest schema with metadata from cache when GetLatestWithMetadata is called twice', async () => { @@ -305,25 +307,25 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { metadata: metadata2, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadataKeyValue); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); const response2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadataKeyValue2); expect(response2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse: SchemaMetadata = await client.getLatestWithMetadata(mockSubject, metadataKeyValue); expect(cachedResponse).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: SchemaMetadata = await client.getLatestWithMetadata(mockSubject2, metadataKeyValue2); expect(cachedResponse2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); it('Should return SchemaMetadata when GetSchemaMetadata is called', async () => { @@ -334,12 +336,12 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { metadata: metadata, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaMetadata = await client.getSchemaMetadata(mockSubject, 1, true); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return SchemaMetadata from cache when GetSchemaMetadata is called twice', async () => { @@ -356,53 +358,54 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { metadata: metadata2, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: SchemaMetadata = await client.getSchemaMetadata(mockSubject, 1, true); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse2 } as AxiosResponse); const response2: SchemaMetadata = await client.getSchemaMetadata(mockSubject2, 2, false); expect(response2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse: SchemaMetadata = await client.getSchemaMetadata(mockSubject, 1, true); expect(cachedResponse).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: SchemaMetadata = await client.getSchemaMetadata(mockSubject2, 2, false); expect(cachedResponse2).toMatchObject(expectedResponse2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); }); describe('SchemaRegistryClient-Subjects', () => { beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new SchemaRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new SchemaRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { jest.clearAllMocks(); }); it('Should return all subjects when GetAllSubjects is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: subjects } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: subjects } as AxiosResponse); const response: string[] = await client.getAllSubjects(); expect(response).toEqual(subjects); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return all versions when GetAllVersions is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: versions } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: versions } as AxiosResponse); const response: number[] = await client.getAllVersions(mockSubject); expect(response).toEqual(versions); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return version when GetVersion is called', async () => { @@ -410,12 +413,12 @@ describe('SchemaRegistryClient-Subjects', () => { schema: schemaString, schemaType: 'AVRO', }; - restService.sendHttpRequest.mockResolvedValue({ data: { version: 1 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { version: 1 } } as AxiosResponse); const response: number = await client.getVersion(mockSubject, schemaInfo, true); expect(response).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return version from cache when GetVersion is called twice', async () => { @@ -428,25 +431,25 @@ describe('SchemaRegistryClient-Subjects', () => { schemaType: 'AVRO', }; - restService.sendHttpRequest.mockResolvedValue({ data: { version: 1 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { version: 1 } } as AxiosResponse); const response: number = await client.getVersion(mockSubject, schemaInfo, true); expect(response).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); - restService.sendHttpRequest.mockResolvedValue({ data: { version: 2 } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { version: 2 } } as AxiosResponse); const response2: number = await client.getVersion(mockSubject2, schemaInfo2, false); expect(response2).toEqual(2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse: number = await client.getVersion(mockSubject, schemaInfo, true); expect(cachedResponse).toEqual(1); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); const cachedResponse2: number = await client.getVersion(mockSubject2, schemaInfo2, false); expect(cachedResponse2).toEqual(2); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(2); + expect(restService.handleRequest).toHaveBeenCalledTimes(2); }); it('Should delete subject from all caches and registry when deleteSubject is called', async () => { @@ -461,7 +464,7 @@ describe('SchemaRegistryClient-Subjects', () => { await client.addToVersionToSchemaCache(mockSubject, 1, expectedResponse); await client.addToIdToSchemaInfoCache(mockSubject, 1, schemaInfo); - restService.sendHttpRequest.mockResolvedValue({ data: [1] } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: [1] } as AxiosResponse); const response: number[] = await client.deleteSubject(mockSubject); @@ -471,7 +474,7 @@ describe('SchemaRegistryClient-Subjects', () => { expect(await client.getIdToSchemaInfoCacheSize()).toEqual(0); expect(response).toEqual([1]); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should delete subject version from all caches and registry when deleteSubjectVersion is called', async () => { @@ -486,7 +489,7 @@ describe('SchemaRegistryClient-Subjects', () => { await client.addToVersionToSchemaCache(mockSubject, 1, expectedResponse); await client.addToIdToSchemaInfoCache(mockSubject, 1, schemaInfo); - restService.sendHttpRequest.mockResolvedValue({ data: [1] } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: [1] } as AxiosResponse); const response: number = await client.deleteSubjectVersion(mockSubject, 1); @@ -496,61 +499,62 @@ describe('SchemaRegistryClient-Subjects', () => { expect(await client.getIdToSchemaInfoCacheSize()).toEqual(0); expect(response).toEqual([1]); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); }); describe('SchemaRegistryClient-Compatibility', () => { - beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new SchemaRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new SchemaRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { jest.clearAllMocks(); }); it('Should return compatibility level when GetCompatibility is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { compatibilityLevel: "BACKWARD" } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { compatibilityLevel: "BACKWARD" } } as AxiosResponse); const response: Compatibility = await client.getCompatibility(mockSubject); expect(response).toEqual('BACKWARD'); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should update compatibility level when updateCompatibility is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); const response: Compatibility = await client.updateCompatibility(mockSubject, Compatibility.Backward); expect(response).toEqual(Compatibility.Backward); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return Compatibility when getDefaultCompatibility is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { compatibilityLevel: 'BACKWARD' } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { compatibilityLevel: 'BACKWARD' } } as AxiosResponse); const response: Compatibility = await client.getDefaultCompatibility(); expect(response).toEqual(Compatibility.Backward); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should update default compatibility level when updateDefaultCompatibility is called', async () => { - restService.sendHttpRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); const response: Compatibility = await client.updateDefaultCompatibility(Compatibility.Backward); expect(response).toEqual(Compatibility.Backward); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); }); describe('SchemaRegistryClient-Config', () => { beforeEach(() => { - restService = new RestService(baseUrls) as jest.Mocked; - client = new SchemaRegistryClient(restService); + restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + client = new SchemaRegistryClient(mockClientConfig); + (client as any).restService = restService; }); afterEach(() => { jest.clearAllMocks(); @@ -563,12 +567,12 @@ describe('SchemaRegistryClient-Config', () => { normalize: true, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: ServerConfig = await client.getConfig(mockSubject); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should update config when updateConfig is called', async () => { @@ -583,12 +587,12 @@ describe('SchemaRegistryClient-Config', () => { normalize: true, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: ServerConfig = await client.updateConfig(mockSubject, request); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return config when getDefaultConfig is called', async () => { @@ -598,12 +602,12 @@ describe('SchemaRegistryClient-Config', () => { normalize: true, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: ServerConfig = await client.getDefaultConfig(); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should update default config when updateDefaultConfig is called', async () => { @@ -618,11 +622,11 @@ describe('SchemaRegistryClient-Config', () => { normalize: true, }; - restService.sendHttpRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); + restService.handleRequest.mockResolvedValue({ data: expectedResponse } as AxiosResponse); const response: ServerConfig = await client.updateDefaultConfig(request); expect(response).toMatchObject(expectedResponse); - expect(restService.sendHttpRequest).toHaveBeenCalledTimes(1); + expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); }); \ No newline at end of file diff --git a/test/schemaregistry/test-constants.ts b/test/schemaregistry/test-constants.ts new file mode 100644 index 00000000..7a478765 --- /dev/null +++ b/test/schemaregistry/test-constants.ts @@ -0,0 +1,35 @@ +import { CreateAxiosDefaults } from 'axios'; +import { ClientConfig } from '../../schemaregistry/rest-service'; + +const baseUrls = ['http://localhost:8081']; + +const mockBaseUrls = ['http://mocked-url']; + +const createAxiosDefaults: CreateAxiosDefaults = { + headers: { + 'Content-Type': 'application/vnd.schemaregistry.v1+json', + }, + auth: { + username: 'RBACAllowedUser-lsrc1', + password: 'nohash', + }, + timeout: 10000 +}; + +const clientConfig: ClientConfig = { + baseURLs: baseUrls, + createAxiosDefaults: createAxiosDefaults, + isForward: false, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, +}; + +const mockClientConfig: ClientConfig = { + baseURLs: mockBaseUrls, + createAxiosDefaults: createAxiosDefaults, + isForward: false, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, +}; + +export { clientConfig, mockClientConfig }; From 3333a2f08c79433464a540a0ae56f4c637b873f6 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Wed, 21 Aug 2024 14:44:28 -0700 Subject: [PATCH 176/224] Update tsconfig.json (#69) --- .gitignore | 1 + tsconfig.json | 30 ++++++++++++++++++++---------- 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/.gitignore b/.gitignore index 8ac250f5..7624e00c 100644 --- a/.gitignore +++ b/.gitignore @@ -13,4 +13,5 @@ deps/* .DS_Store +.idea .vscode diff --git a/tsconfig.json b/tsconfig.json index 2af9470f..05732ed7 100644 --- a/tsconfig.json +++ b/tsconfig.json @@ -1,21 +1,31 @@ { "compilerOptions": { - "module": "commonjs", - "lib": ["es6"], - "noImplicitAny": true, - "noImplicitThis": true, - "strictNullChecks": true, "baseUrl": ".", + "lib": [ + "es2021" + ], + "module": "commonjs", + "target": "es2021", + "strict": true, + "esModuleInterop": true, + "forceConsistentCasingInFileNames": true, + "moduleResolution": "node16", + "allowUnusedLabels": false, + "allowUnreachableCode": false, + "noFallthroughCasesInSwitch": true, + "noImplicitOverride": true, + "noImplicitReturns": true, + "noPropertyAccessFromIndexSignature": true, + "noUnusedLocals": true, + "useUnknownInCatchVariables": true, "types": ["node_modules/@types/node", "jest"], "typeRoots": ["."], "noEmit": true, - "esModuleInterop": true, - "forceConsistentCasingInFileNames": true, - "strictFunctionTypes": true + "resolveJsonModule": true }, - "files": [ + "include": [ "index.d.ts", "schemaregistry/**/*", "test/**/*" ] -} \ No newline at end of file +} From 4748d8abdb6f31f2834b759ee239c78013630ac1 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Wed, 21 Aug 2024 19:18:24 -0400 Subject: [PATCH 177/224] Fix broken tests (#70) Co-authored-by: Robert Yokota --- .../dekregistry}/constants.ts | 0 .../dekregistry}/dekregistry-client.ts | 12 +++++++++--- .../dekregistry}/mock-dekregistry-client.ts | 0 schemaregistry/mock-schemaregistry-client.ts | 2 +- .../dekregistry/dekregistry-client.spec.ts | 8 ++++---- .../dekregistry/mock-dekregistry-client.spec.ts | 6 +++--- .../dekregistry/test-constants.ts | 4 ++-- .../mock-schemaregistery-client.spec.ts | 2 -- 8 files changed, 19 insertions(+), 15 deletions(-) rename {dekregistry => schemaregistry/dekregistry}/constants.ts (100%) rename {dekregistry => schemaregistry/dekregistry}/dekregistry-client.ts (94%) rename {dekregistry => schemaregistry/dekregistry}/mock-dekregistry-client.ts (100%) rename test/{ => schemaregistry}/dekregistry/dekregistry-client.spec.ts (95%) rename test/{ => schemaregistry}/dekregistry/mock-dekregistry-client.spec.ts (90%) rename test/{ => schemaregistry}/dekregistry/test-constants.ts (92%) diff --git a/dekregistry/constants.ts b/schemaregistry/dekregistry/constants.ts similarity index 100% rename from dekregistry/constants.ts rename to schemaregistry/dekregistry/constants.ts diff --git a/dekregistry/dekregistry-client.ts b/schemaregistry/dekregistry/dekregistry-client.ts similarity index 94% rename from dekregistry/dekregistry-client.ts rename to schemaregistry/dekregistry/dekregistry-client.ts index ff59fa5a..c40796f5 100644 --- a/dekregistry/dekregistry-client.ts +++ b/schemaregistry/dekregistry/dekregistry-client.ts @@ -1,6 +1,6 @@ import { LRUCache } from 'lru-cache'; import { Mutex } from 'async-mutex'; -import { ClientConfig, RestService } from '../schemaregistry/rest-service'; +import { ClientConfig, RestService } from '../rest-service'; import stringify from 'json-stringify-deterministic'; /* @@ -84,7 +84,10 @@ class DekRegistryClient implements Client { const bytes = Buffer.from(dek.encryptedKeyMaterial, 'base64'); dek.encryptedKeyMaterialBytes = bytes; } catch (err) { - throw new Error(`Failed to decode base64 string: ${err.message}`); + if (err instanceof Error) { + throw new Error(`Failed to decode base64 string: ${err.message}`); + } + throw new Error(`Unknown error: ${err}`); } } @@ -101,7 +104,10 @@ class DekRegistryClient implements Client { const bytes = Buffer.from(dek.keyMaterial, 'base64'); dek.keyMaterialBytes = bytes; } catch (err) { - throw new Error(`Failed to decode base64 string: ${err.message}`); + if (err instanceof Error) { + throw new Error(`Failed to decode base64 string: ${err.message}`); + } + throw new Error(`Unknown error: ${err}`); } } diff --git a/dekregistry/mock-dekregistry-client.ts b/schemaregistry/dekregistry/mock-dekregistry-client.ts similarity index 100% rename from dekregistry/mock-dekregistry-client.ts rename to schemaregistry/dekregistry/mock-dekregistry-client.ts diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts index bde07ffb..c7550170 100644 --- a/schemaregistry/mock-schemaregistry-client.ts +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -1,5 +1,5 @@ -import { Client, Compatibility, Metadata, SchemaInfo, SchemaMetadata, ServerConfig } from './schemaregistry-client'; +import { Client, Compatibility, SchemaInfo, SchemaMetadata, ServerConfig } from './schemaregistry-client'; import stringify from "json-stringify-deterministic"; interface VersionCacheEntry { diff --git a/test/dekregistry/dekregistry-client.spec.ts b/test/schemaregistry/dekregistry/dekregistry-client.spec.ts similarity index 95% rename from test/dekregistry/dekregistry-client.spec.ts rename to test/schemaregistry/dekregistry/dekregistry-client.spec.ts index f6de5fcd..2efbfb5e 100644 --- a/test/dekregistry/dekregistry-client.spec.ts +++ b/test/schemaregistry/dekregistry/dekregistry-client.spec.ts @@ -1,14 +1,14 @@ -import { DekRegistryClient, Dek, Kek } from "../../dekregistry/dekregistry-client"; -import { RestService } from "../../schemaregistry/rest-service"; +import { DekRegistryClient, Dek, Kek } from "../../../schemaregistry/dekregistry/dekregistry-client"; +import { RestService } from "../../../schemaregistry/rest-service"; import { AxiosResponse } from 'axios'; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; import { TEST_KEK, TEST_KEK_2, TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_2, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION, TEST_DEK_LATEST} from "./test-constants"; -import { mockClientConfig } from "../schemaregistry/test-constants"; +import { mockClientConfig } from "../test-constants"; -jest.mock('../../schemaregistry/rest-service'); +jest.mock('../../../schemaregistry/rest-service'); let client: DekRegistryClient; diff --git a/test/dekregistry/mock-dekregistry-client.spec.ts b/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts similarity index 90% rename from test/dekregistry/mock-dekregistry-client.spec.ts rename to test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts index 2a2a1b3b..d611e532 100644 --- a/test/dekregistry/mock-dekregistry-client.spec.ts +++ b/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts @@ -1,8 +1,8 @@ -import { Dek, Kek } from "../../dekregistry/dekregistry-client"; -import { MockDekRegistryClient } from "../../dekregistry/mock-dekregistry-client"; +import { Dek, Kek } from "../../../schemaregistry/dekregistry/dekregistry-client"; +import { MockDekRegistryClient } from "../../../schemaregistry/dekregistry/mock-dekregistry-client"; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; import { TEST_KEK, TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, - TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_V2, TEST_DEK_LATEST, TEST_ALGORITHM, + TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_V2, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION } from "./test-constants"; describe('MockClient-tests', () => { diff --git a/test/dekregistry/test-constants.ts b/test/schemaregistry/dekregistry/test-constants.ts similarity index 92% rename from test/dekregistry/test-constants.ts rename to test/schemaregistry/dekregistry/test-constants.ts index e88387cb..4719fdce 100644 --- a/test/dekregistry/test-constants.ts +++ b/test/schemaregistry/dekregistry/test-constants.ts @@ -1,5 +1,5 @@ -import { MOCK_TS } from "../../dekregistry/constants"; -import { Kek, Dek } from "../../dekregistry/dekregistry-client"; +import { MOCK_TS } from "../../../schemaregistry/dekregistry/constants"; +import { Kek, Dek } from "../../../schemaregistry/dekregistry/dekregistry-client"; const TEST_KEK_NAME: string = 'test-kek-name'; const TEST_KEK_NAME_2: string = 'test-kek-name2'; diff --git a/test/schemaregistry/mock-schemaregistery-client.spec.ts b/test/schemaregistry/mock-schemaregistery-client.spec.ts index 86bc2c4a..ae830cb7 100644 --- a/test/schemaregistry/mock-schemaregistery-client.spec.ts +++ b/test/schemaregistry/mock-schemaregistery-client.spec.ts @@ -1,6 +1,5 @@ import { MockClient } from '../../schemaregistry/mock-schemaregistry-client'; import { Compatibility, Metadata, SchemaInfo, SchemaMetadata } from '../../schemaregistry/schemaregistry-client'; -import { RestService } from '../../schemaregistry/rest-service'; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; const schemaString: string = JSON.stringify({ @@ -62,7 +61,6 @@ const testSubject2 = 'test-subject2'; describe('MockClient-tests', () => { let mockClient: MockClient; - let restService: RestService; beforeEach(() => { mockClient = new MockClient(); From 40fe41ae16132c732ecf4b4417dd9befb933e606 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 22 Aug 2024 05:13:53 +0200 Subject: [PATCH 178/224] Add commitCb method (#59) to avoid blocking while committing and return a Promise without having to call consume() --- lib/kafka-consumer.js | 20 ++++++++++++++++ lib/kafkajs/_consumer.js | 32 +++++++++++++++---------- src/kafka-consumer.cc | 40 +++++++++++++++++++++++++++++++ src/kafka-consumer.h | 1 + src/workers.cc | 52 ++++++++++++++++++++++++++++++++++++++++ src/workers.h | 16 +++++++++++++ 6 files changed, 149 insertions(+), 12 deletions(-) diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 94d00d9e..03b1f20a 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -646,6 +646,26 @@ KafkaConsumer.prototype.commitMessageSync = function(msg) { return this; }; +/** + * Commits a list of offsets per topic partition, using provided callback. + * + * @param {TopicPartition[]} toppars - Topic partition list to commit + * offsets for. Defaults to the current assignment + * @param {Function} cb - Callback method to execute when finished + * @return {Client} - Returns itself + */ +KafkaConsumer.prototype.commitCb = function(toppars, cb) { + this._client.commitCb(toppars, function(err) { + if (err) { + cb(LibrdKafkaError.create(err)); + return; + } + + cb(null); + }); + return this; +}; + /** * Get last known offsets from the client. * diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index e878bd79..770eaf50 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1342,19 +1342,27 @@ class Consumer { throw new error.KafkaJSError('Commit can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } - try { - if (topicPartitions === null) { - this.#internalClient.commitSync(); - } else { - const topicPartitionsRdKafka = topicPartitions.map( - topicPartitionOffsetMetadataToRdKafka); - this.#internalClient.commitSync(topicPartitionsRdKafka); - } - } catch (e) { - if (!e.code || e.code !== error.ErrorCodes.ERR__NO_OFFSET) { - throw createKafkaJsErrorFromLibRdKafkaError(e); + return new Promise((resolve, reject) => { + try { + let cb = (e) => { + if (e) + reject(createKafkaJsErrorFromLibRdKafkaError(e)); + else + resolve(); + }; + + if (topicPartitions) + topicPartitions = topicPartitions.map(topicPartitionOffsetMetadataToRdKafka); + else + topicPartitions = null; + this.#internalClient.commitCb(topicPartitions, cb); + } catch (e) { + if (!e.code || e.code !== error.ErrorCodes.ERR__NO_OFFSET) + reject(createKafkaJsErrorFromLibRdKafkaError(e)); + else + resolve(); } - } + }); } /** diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 58828e2b..32c963f4 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -556,6 +556,7 @@ void KafkaConsumer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "commit", NodeCommit); Nan::SetPrototypeMethod(tpl, "commitSync", NodeCommitSync); + Nan::SetPrototypeMethod(tpl, "commitCb", NodeCommitCb); Nan::SetPrototypeMethod(tpl, "offsetsStore", NodeOffsetsStore); Nan::SetPrototypeMethod(tpl, "offsetsStoreSingle", NodeOffsetsStoreSingle); @@ -1025,6 +1026,45 @@ NAN_METHOD(KafkaConsumer::NodeCommitSync) { info.GetReturnValue().Set(Nan::New(error_code)); } +NAN_METHOD(KafkaConsumer::NodeCommitCb) { + Nan::HandleScope scope; + int error_code; + std::optional> toppars = std::nullopt; + Nan::Callback *callback; + + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); + + if (!consumer->IsConnected()) { + Nan::ThrowError("KafkaConsumer is disconnected"); + return; + } + + if (info.Length() != 2) { + Nan::ThrowError("Two arguments are required"); + return; + } + + if (!( + (info[0]->IsArray() || info[0]->IsNull()) && + info[1]->IsFunction())) { + Nan::ThrowError( + "First argument should be an array or null and second one a callback"); + return; + } + + if (info[0]->IsArray()) { + toppars = + Conversion::TopicPartition::FromV8Array(info[0].As()); + } + callback = new Nan::Callback(info[1].As()); + + Nan::AsyncQueueWorker( + new Workers::KafkaConsumerCommitCb(callback, consumer, + toppars)); + + info.GetReturnValue().Set(Nan::Null()); +} + NAN_METHOD(KafkaConsumer::NodeSubscribe) { Nan::HandleScope scope; diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index ab8fcc18..e4b04fee 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -121,6 +121,7 @@ class KafkaConsumer : public Connection { static NAN_METHOD(NodeUnsubscribe); static NAN_METHOD(NodeCommit); static NAN_METHOD(NodeCommitSync); + static NAN_METHOD(NodeCommitCb); static NAN_METHOD(NodeOffsetsStore); static NAN_METHOD(NodeOffsetsStoreSingle); static NAN_METHOD(NodeCommitted); diff --git a/src/workers.cc b/src/workers.cc index 41658a36..571cc1e7 100644 --- a/src/workers.cc +++ b/src/workers.cc @@ -1062,6 +1062,58 @@ void KafkaConsumerCommitted::HandleErrorCallback() { callback->Call(argc, argv); } +/** + * @brief KafkaConsumer commit offsets with a callback function. + * + * The first callback argument is the commit error, or null on success. + * + * @see RdKafka::KafkaConsumer::commitSync + */ +KafkaConsumerCommitCb::KafkaConsumerCommitCb(Nan::Callback *callback, + KafkaConsumer* consumer, + std::optional> & t) : + ErrorAwareWorker(callback), + m_consumer(consumer), + m_topic_partitions(t) {} + +KafkaConsumerCommitCb::~KafkaConsumerCommitCb() { + // Delete the underlying topic partitions as they are ephemeral or cloned + if (m_topic_partitions.has_value()) + RdKafka::TopicPartition::destroy(m_topic_partitions.value()); +} + +void KafkaConsumerCommitCb::Execute() { + Baton b = Baton(NULL); + if (m_topic_partitions.has_value()) { + b = m_consumer->Commit(m_topic_partitions.value()); + } else { + b = m_consumer->Commit(); + } + if (b.err() != RdKafka::ERR_NO_ERROR) { + SetErrorBaton(b); + } +} + +void KafkaConsumerCommitCb::HandleOKCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 1; + v8::Local argv[argc]; + + argv[0] = Nan::Null(); + + callback->Call(argc, argv); +} + +void KafkaConsumerCommitCb::HandleErrorCallback() { + Nan::HandleScope scope; + + const unsigned int argc = 1; + v8::Local argv[argc] = { GetErrorObject() }; + + callback->Call(argc, argv); +} + /** * @brief KafkaConsumer seek * diff --git a/src/workers.h b/src/workers.h index 7da14599..e103163f 100644 --- a/src/workers.h +++ b/src/workers.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include "src/common.h" @@ -423,6 +424,21 @@ class KafkaConsumerCommitted : public ErrorAwareWorker { const int m_timeout_ms; }; +class KafkaConsumerCommitCb : public ErrorAwareWorker { + public: + KafkaConsumerCommitCb(Nan::Callback*, + NodeKafka::KafkaConsumer*, + std::optional> &); + ~KafkaConsumerCommitCb(); + + void Execute(); + void HandleOKCallback(); + void HandleErrorCallback(); + private: + NodeKafka::KafkaConsumer * m_consumer; + std::optional> m_topic_partitions; +}; + class KafkaConsumerSeek : public ErrorAwareWorker { public: KafkaConsumerSeek(Nan::Callback*, NodeKafka::KafkaConsumer*, From 579d326cf638ab950e35e991aca459baf8f19799 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 22 Aug 2024 11:49:29 -0700 Subject: [PATCH 179/224] Fix eslint config (#71) --- .eslintrc.js | 37 --- eslint.config.js | 11 +- package-lock.json | 763 +++++++++++++++++++++++++++++++++++++++++++++- package.json | 6 +- 4 files changed, 770 insertions(+), 47 deletions(-) delete mode 100644 .eslintrc.js diff --git a/.eslintrc.js b/.eslintrc.js deleted file mode 100644 index d9b79160..00000000 --- a/.eslintrc.js +++ /dev/null @@ -1,37 +0,0 @@ -module.exports = { - "env": { - "browser": true, - "commonjs": true, - "es2021": true - }, - "extends": "eslint:recommended", - "overrides": [ - { - "env": { - "node": true - }, - "files": [ - ".eslintrc.{js,cjs}" - ], - "parserOptions": { - "sourceType": "script" - } - }, - { - "files": ["*.ts"], - "parser": "@typescript-eslint/parser", - "parserOptions": { - "ecmaVersion": 2020, - "sourceType": "module" - }, - "extends": [ - "plugin:@typescript-eslint/recommended", - ] - } - ], - "parserOptions": { - "ecmaVersion": "latest" - }, - "rules": { - } -} diff --git a/eslint.config.js b/eslint.config.js index 89faf73e..8f489ec6 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -1,5 +1,6 @@ const js = require("@eslint/js"); const jest = require('eslint-plugin-jest'); +const ts = require('typescript-eslint'); const ckjsSpecificSettings = { languageOptions: { @@ -32,7 +33,7 @@ const ckjsSpecificJestSettings = { } }; -module.exports = [ +module.exports = ts.config( { ...js.configs.recommended, files: ["lib/**/*.js", "test/promisified/**/*.js"], @@ -50,5 +51,9 @@ module.exports = [ { ...ckjsSpecificJestSettings, files: ["test/promisified/**/*.js"] - } -]; + }, + ...ts.configs.recommended.map((config) => ({ + ...config, + ignores: ["**/*.js"], + })), +); diff --git a/package-lock.json b/package-lock.json index dcd65699..1d946b41 100644 --- a/package-lock.json +++ b/package-lock.json @@ -20,7 +20,8 @@ "ts-jest": "^29.2.4" }, "devDependencies": { - "@eslint/js": "^9.8.0", + "@eslint/js": "^9.9.0", + "@types/eslint__js": "^8.42.3", "@types/node": "^20.4.5", "bluebird": "^3.5.3", "eslint": "^8.57.0", @@ -29,7 +30,8 @@ "jsdoc": "^4.0.2", "mocha": "^10.7.0", "node-gyp": "^9.3.1", - "typescript": "^5.1.6" + "typescript": "^5.5.4", + "typescript-eslint": "^8.2.0" }, "engines": { "node": ">=18.0.0" @@ -594,9 +596,9 @@ } }, "node_modules/@eslint/js": { - "version": "9.8.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.8.0.tgz", - "integrity": "sha512-MfluB7EUfxXtv3i/++oh89uzAr4PDI4nn201hsp+qaXqsjAWzinlZEHEfPgAX4doIlKvPG/i0A9dpKxOLII8yA==", + "version": "9.9.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.9.0.tgz", + "integrity": "sha512-hhetes6ZHP3BlXLxmd8K2SNgkhNSi+UcecbnwWKwpP7kyi/uC75DJ1lOOBO3xrC4jyojtGE3YxKZPHfk4yrgug==", "dev": true, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -1214,6 +1216,31 @@ "@babel/types": "^7.20.7" } }, + "node_modules/@types/eslint": { + "version": "9.6.0", + "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-9.6.0.tgz", + "integrity": "sha512-gi6WQJ7cHRgZxtkQEoyHMppPjq9Kxo5Tjn2prSKDSmZrCz8TZ3jSRCeTJm+WoM+oB0WG37bRqLzaaU3q7JypGg==", + "dev": true, + "dependencies": { + "@types/estree": "*", + "@types/json-schema": "*" + } + }, + "node_modules/@types/eslint__js": { + "version": "8.42.3", + "resolved": "https://registry.npmjs.org/@types/eslint__js/-/eslint__js-8.42.3.tgz", + "integrity": "sha512-alfG737uhmPdnvkrLdZLcEKJ/B8s9Y4hrZ+YAdzUeoArBlSUERA2E87ROfOaS4jd/C45fzOoZzidLc1IPwLqOw==", + "dev": true, + "dependencies": { + "@types/eslint": "*" + } + }, + "node_modules/@types/estree": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", + "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", + "dev": true + }, "node_modules/@types/graceful-fs": { "version": "4.1.9", "resolved": "https://registry.npmjs.org/@types/graceful-fs/-/graceful-fs-4.1.9.tgz", @@ -1243,6 +1270,12 @@ "@types/istanbul-lib-report": "*" } }, + "node_modules/@types/json-schema": { + "version": "7.0.15", + "resolved": "https://registry.npmjs.org/@types/json-schema/-/json-schema-7.0.15.tgz", + "integrity": "sha512-5+fP8P8MFNC+AyZCDxrB2pkZFPGzqQWUzpSeuuVLvm8VMcorNYavBqoFcxK8bQz4Qsbn4oUEEem4wDLfcysGHA==", + "dev": true + }, "node_modules/@types/linkify-it": { "version": "5.0.0", "resolved": "https://registry.npmjs.org/@types/linkify-it/-/linkify-it-5.0.0.tgz", @@ -1273,6 +1306,14 @@ "undici-types": "~5.26.4" } }, + "node_modules/@types/semver": { + "version": "7.5.8", + "resolved": "https://registry.npmjs.org/@types/semver/-/semver-7.5.8.tgz", + "integrity": "sha512-I8EUhyrgfLrcTkzV3TSsGyl1tSuPrEDzr0yd5m90UgNxQkyDXULk3b6MlQqTCpZpNtWe1K0hzclnZkTcLBe2UQ==", + "dev": true, + "optional": true, + "peer": true + }, "node_modules/@types/stack-utils": { "version": "2.0.3", "resolved": "https://registry.npmjs.org/@types/stack-utils/-/stack-utils-2.0.3.tgz", @@ -1291,6 +1332,322 @@ "resolved": "https://registry.npmjs.org/@types/yargs-parser/-/yargs-parser-21.0.3.tgz", "integrity": "sha512-I4q9QU9MQv4oEOz4tAHJtNz1cwuLxn2F3xcc2iV5WdqLPpUnj30aUuxt1mAxYTG+oe8CZMV/+6rU4S4gRDzqtQ==" }, + "node_modules/@typescript-eslint/eslint-plugin": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-7.2.0.tgz", + "integrity": "sha512-mdekAHOqS9UjlmyF/LSs6AIEvfceV749GFxoBAjwAv0nkevfKHWQFDMcBZWUiIC5ft6ePWivXoS36aKQ0Cy3sw==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@eslint-community/regexpp": "^4.5.1", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/type-utils": "7.2.0", + "@typescript-eslint/utils": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "graphemer": "^1.4.0", + "ignore": "^5.2.4", + "natural-compare": "^1.4.0", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "@typescript-eslint/parser": "^7.0.0", + "eslint": "^8.56.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/scope-manager": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", + "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/types": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", + "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", + "dev": true, + "optional": true, + "peer": true, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/typescript-estree": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", + "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "9.0.3", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/utils": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-7.2.0.tgz", + "integrity": "sha512-YfHpnMAGb1Eekpm3XRK8hcMwGLGsnT6L+7b2XyRv6ouDuJU1tZir1GS2i0+VXRatMwSI1/UfcyPe53ADkU+IuA==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@types/json-schema": "^7.0.12", + "@types/semver": "^7.5.0", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "semver": "^7.5.4" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/visitor-keys": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", + "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/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, + "optional": true, + "peer": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/minimatch": { + "version": "9.0.3", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", + "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/@typescript-eslint/parser": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-7.2.0.tgz", + "integrity": "sha512-5FKsVcHTk6TafQKQbuIVkXq58Fnbkd2wDL4LB7AURN7RUOu1utVP+G8+6u3ZhEroW3DF6hyo3ZEXxgKgp4KeCg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/scope-manager": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", + "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/types": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", + "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", + "dev": true, + "optional": true, + "peer": true, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/typescript-estree": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", + "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "9.0.3", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/visitor-keys": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", + "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/parser/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, + "optional": true, + "peer": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/minimatch": { + "version": "9.0.3", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", + "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/@typescript-eslint/scope-manager": { "version": "7.18.0", "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.18.0.tgz", @@ -1308,6 +1665,173 @@ "url": "https://opencollective.com/typescript-eslint" } }, + "node_modules/@typescript-eslint/type-utils": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-7.2.0.tgz", + "integrity": "sha512-xHi51adBHo9O9330J8GQYQwrKBqbIPJGZZVQTHHmy200hvkLZFWJIFtAG/7IYTWUyun6DE6w5InDReePJYJlJA==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/typescript-estree": "7.2.0", + "@typescript-eslint/utils": "7.2.0", + "debug": "^4.3.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/scope-manager": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", + "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/types": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", + "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", + "dev": true, + "optional": true, + "peer": true, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/typescript-estree": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", + "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "9.0.3", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/utils": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-7.2.0.tgz", + "integrity": "sha512-YfHpnMAGb1Eekpm3XRK8hcMwGLGsnT6L+7b2XyRv6ouDuJU1tZir1GS2i0+VXRatMwSI1/UfcyPe53ADkU+IuA==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@types/json-schema": "^7.0.12", + "@types/semver": "^7.5.0", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "semver": "^7.5.4" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/visitor-keys": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", + "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/type-utils/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, + "optional": true, + "peer": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/minimatch": { + "version": "9.0.3", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", + "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", + "dev": true, + "optional": true, + "peer": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/@typescript-eslint/types": { "version": "7.18.0", "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.18.0.tgz", @@ -5812,6 +6336,235 @@ "node": ">=14.17" } }, + "node_modules/typescript-eslint": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/typescript-eslint/-/typescript-eslint-8.2.0.tgz", + "integrity": "sha512-DmnqaPcML0xYwUzgNbM1XaKXpEb7BShYf2P1tkUmmcl8hyeG7Pj08Er7R9bNy6AufabywzJcOybQAtnD/c9DGw==", + "dev": true, + "dependencies": { + "@typescript-eslint/eslint-plugin": "8.2.0", + "@typescript-eslint/parser": "8.2.0", + "@typescript-eslint/utils": "8.2.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/eslint-plugin": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.2.0.tgz", + "integrity": "sha512-02tJIs655em7fvt9gps/+4k4OsKULYGtLBPJfOsmOq1+3cdClYiF0+d6mHu6qDnTcg88wJBkcPLpQhq7FyDz0A==", + "dev": true, + "dependencies": { + "@eslint-community/regexpp": "^4.10.0", + "@typescript-eslint/scope-manager": "8.2.0", + "@typescript-eslint/type-utils": "8.2.0", + "@typescript-eslint/utils": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0", + "graphemer": "^1.4.0", + "ignore": "^5.3.1", + "natural-compare": "^1.4.0", + "ts-api-utils": "^1.3.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "@typescript-eslint/parser": "^8.0.0 || ^8.0.0-alpha.0", + "eslint": "^8.57.0 || ^9.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/parser": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.2.0.tgz", + "integrity": "sha512-j3Di+o0lHgPrb7FxL3fdEy6LJ/j2NE8u+AP/5cQ9SKb+JLH6V6UHDqJ+e0hXBkHP1wn1YDFjYCS9LBQsZDlDEg==", + "dev": true, + "dependencies": { + "@typescript-eslint/scope-manager": "8.2.0", + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/typescript-estree": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0", + "debug": "^4.3.4" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/scope-manager": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.2.0.tgz", + "integrity": "sha512-OFn80B38yD6WwpoHU2Tz/fTz7CgFqInllBoC3WP+/jLbTb4gGPTy9HBSTsbDWkMdN55XlVU0mMDYAtgvlUspGw==", + "dev": true, + "dependencies": { + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/type-utils": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.2.0.tgz", + "integrity": "sha512-g1CfXGFMQdT5S+0PSO0fvGXUaiSkl73U1n9LTK5aRAFnPlJ8dLKkXr4AaLFvPedW8lVDoMgLLE3JN98ZZfsj0w==", + "dev": true, + "dependencies": { + "@typescript-eslint/typescript-estree": "8.2.0", + "@typescript-eslint/utils": "8.2.0", + "debug": "^4.3.4", + "ts-api-utils": "^1.3.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/types": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.2.0.tgz", + "integrity": "sha512-6a9QSK396YqmiBKPkJtxsgZZZVjYQ6wQ/TlI0C65z7vInaETuC6HAHD98AGLC8DyIPqHytvNuS8bBVvNLKyqvQ==", + "dev": true, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/typescript-estree": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.2.0.tgz", + "integrity": "sha512-kiG4EDUT4dImplOsbh47B1QnNmXSoUqOjWDvCJw/o8LgfD0yr7k2uy54D5Wm0j4t71Ge1NkynGhpWdS0dEIAUA==", + "dev": true, + "dependencies": { + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "^9.0.4", + "semver": "^7.6.0", + "ts-api-utils": "^1.3.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/utils": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.2.0.tgz", + "integrity": "sha512-O46eaYKDlV3TvAVDNcoDzd5N550ckSe8G4phko++OCSC1dYIb9LTc3HDGYdWqWIAT5qDUKphO6sd9RrpIJJPfg==", + "dev": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@typescript-eslint/scope-manager": "8.2.0", + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/typescript-estree": "8.2.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0" + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/visitor-keys": { + "version": "8.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.2.0.tgz", + "integrity": "sha512-sbgsPMW9yLvS7IhCi8IpuK1oBmtbWUNP+hBdwl/I9nzqVsszGnNGti5r9dUtF5RLivHUFFIdRvLiTsPhzSyJ3Q==", + "dev": true, + "dependencies": { + "@typescript-eslint/types": "8.2.0", + "eslint-visitor-keys": "^3.4.3" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/typescript-eslint/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/typescript-eslint/node_modules/minimatch": { + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", + "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", + "dev": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/uc.micro": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/uc.micro/-/uc.micro-2.1.0.tgz", diff --git a/package.json b/package.json index dffa1280..2c3f4015 100644 --- a/package.json +++ b/package.json @@ -32,7 +32,8 @@ }, "license": "MIT", "devDependencies": { - "@eslint/js": "^9.8.0", + "@eslint/js": "^9.9.0", + "@types/eslint__js": "^8.42.3", "@types/node": "^20.4.5", "bluebird": "^3.5.3", "eslint": "^8.57.0", @@ -41,7 +42,8 @@ "jsdoc": "^4.0.2", "mocha": "^10.7.0", "node-gyp": "^9.3.1", - "typescript": "^5.1.6" + "typescript": "^5.5.4", + "typescript-eslint": "^8.2.0" }, "dependencies": { "@mapbox/node-pre-gyp": "^1.0.11", From 9563d4a567b12405e61de026431d93fbbdf33949 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 22 Aug 2024 15:56:09 -0700 Subject: [PATCH 180/224] Add eslint rules (#72) * Add some ts eslint rules * Fix makefile --- Makefile.schemaregistry | 4 ++-- eslint.config.js | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/Makefile.schemaregistry b/Makefile.schemaregistry index 096b22ab..248ff096 100644 --- a/Makefile.schemaregistry +++ b/Makefile.schemaregistry @@ -9,7 +9,7 @@ TS_NODE ?= ./node_modules/.bin/ts-node # Paths SRC_DIR = schemaregistry SR_TEST_DIR = test/schemaregistry -DEK_TEST_DIR = test/dekregistry +DEK_TEST_DIR = test/schemaregistry/dekregistry INTEG_DIR = e2e/schemaregistry # Tasks @@ -24,4 +24,4 @@ test: $(JEST) $(SR_TEST_DIR) $(DEK_TEST_DIR) integtest: - $(JEST) $(INTEG_DIR) \ No newline at end of file + $(JEST) $(INTEG_DIR) diff --git a/eslint.config.js b/eslint.config.js index 8f489ec6..8ccf17d0 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -55,5 +55,11 @@ module.exports = ts.config( ...ts.configs.recommended.map((config) => ({ ...config, ignores: ["**/*.js"], + rules: { + ...config.rules, + "prefer-const": 0, + "@typescript-eslint/no-explicit-any": "warn", + "@typescript-eslint/no-unused-vars": "warn", + } })), ); From 56759d83148f10d24695c127d5172c88578b33fc Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 23 Aug 2024 14:29:28 -0700 Subject: [PATCH 181/224] First cut at JavaScript serdes (#73) * First cut at serdes * Checkpoint - no errs * Minor cleanup * Add siv * Fix eslint errs * Minor cleanup * Minor cleanup * Fix configs * Fix execute calls * Clean up public, compat levels * Fix test * Incorporate review feedback --- buf.gen.yaml | 8 + .../schemaregistry-client.spec.ts | 22 +- eslint.config.js | 2 +- package-lock.json | 416 +++++++++- package.json | 13 +- proto/confluent/meta.proto | 28 + proto/confluent/types/decimal.proto | 17 + .../rules/encryption/tink/proto/aes_gcm.proto | 67 ++ .../rules/encryption/tink/proto/aes_siv.proto | 36 + schemaregistry/confluent/meta_pb.ts | 73 ++ schemaregistry/confluent/types/decimal_pb.ts | 47 ++ .../dekregistry/dekregistry-client.ts | 36 +- .../dekregistry/mock-dekregistry-client.ts | 24 +- schemaregistry/mock-schemaregistry-client.ts | 59 +- schemaregistry/rest-service.ts | 16 +- .../rules/encryption/encrypt-executor.ts | 529 +++++++++++++ .../rules/encryption/kms-registry.ts | 44 ++ schemaregistry/rules/encryption/tink/aead.ts | 52 ++ .../rules/encryption/tink/aes_gcm.ts | 107 +++ .../rules/encryption/tink/aes_siv.ts | 41 + schemaregistry/rules/encryption/tink/bytes.ts | 186 +++++ .../exception/invalid_arguments_exception.ts | 17 + .../tink/exception/security_exception.ts | 17 + .../rules/encryption/tink/proto/aes_gcm_pb.ts | 74 ++ .../rules/encryption/tink/proto/aes_siv_pb.ts | 80 ++ .../rules/encryption/tink/random.ts | 27 + .../rules/encryption/tink/validators.ts | 83 ++ schemaregistry/schemaregistry-client.ts | 154 ++-- schemaregistry/serde/avro.ts | 380 +++++++++ schemaregistry/serde/buffer-wrapper.ts | 63 ++ schemaregistry/serde/json.ts | 411 ++++++++++ schemaregistry/serde/json_util.ts | 122 +++ schemaregistry/serde/protobuf.ts | 465 +++++++++++ schemaregistry/serde/rule-registry.ts | 42 + schemaregistry/serde/serde.ts | 746 ++++++++++++++++++ schemaregistry/serde/wildcard-matcher.ts | 90 +++ .../dekregistry/dekregistry-client.spec.ts | 42 +- .../mock-dekregistry-client.spec.ts | 24 +- .../mock-schemaregistery-client.spec.ts | 20 +- .../schemaregistry-client.spec.ts | 28 +- test/schemaregistry/serde/avro.spec.ts | 32 + .../serde/buffer-wrapper.spec.ts | 26 + .../serde/wildcard-matcher.spec.ts | 86 ++ tsconfig.json | 2 +- 44 files changed, 4636 insertions(+), 218 deletions(-) create mode 100644 buf.gen.yaml create mode 100644 proto/confluent/meta.proto create mode 100644 proto/confluent/types/decimal.proto create mode 100644 proto/rules/encryption/tink/proto/aes_gcm.proto create mode 100644 proto/rules/encryption/tink/proto/aes_siv.proto create mode 100644 schemaregistry/confluent/meta_pb.ts create mode 100644 schemaregistry/confluent/types/decimal_pb.ts create mode 100644 schemaregistry/rules/encryption/encrypt-executor.ts create mode 100644 schemaregistry/rules/encryption/kms-registry.ts create mode 100644 schemaregistry/rules/encryption/tink/aead.ts create mode 100644 schemaregistry/rules/encryption/tink/aes_gcm.ts create mode 100644 schemaregistry/rules/encryption/tink/aes_siv.ts create mode 100644 schemaregistry/rules/encryption/tink/bytes.ts create mode 100644 schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts create mode 100644 schemaregistry/rules/encryption/tink/exception/security_exception.ts create mode 100644 schemaregistry/rules/encryption/tink/proto/aes_gcm_pb.ts create mode 100644 schemaregistry/rules/encryption/tink/proto/aes_siv_pb.ts create mode 100644 schemaregistry/rules/encryption/tink/random.ts create mode 100644 schemaregistry/rules/encryption/tink/validators.ts create mode 100644 schemaregistry/serde/avro.ts create mode 100644 schemaregistry/serde/buffer-wrapper.ts create mode 100644 schemaregistry/serde/json.ts create mode 100644 schemaregistry/serde/json_util.ts create mode 100644 schemaregistry/serde/protobuf.ts create mode 100644 schemaregistry/serde/rule-registry.ts create mode 100644 schemaregistry/serde/serde.ts create mode 100644 schemaregistry/serde/wildcard-matcher.ts create mode 100644 test/schemaregistry/serde/avro.spec.ts create mode 100644 test/schemaregistry/serde/buffer-wrapper.spec.ts create mode 100644 test/schemaregistry/serde/wildcard-matcher.spec.ts diff --git a/buf.gen.yaml b/buf.gen.yaml new file mode 100644 index 00000000..7fe5b593 --- /dev/null +++ b/buf.gen.yaml @@ -0,0 +1,8 @@ +# Learn more: https://buf.build/docs/configuration/v2/buf-gen-yaml +version: v2 +inputs: + - directory: proto +plugins: + - local: protoc-gen-es + opt: target=ts + out: schemaregistry diff --git a/e2e/schemaregistry/schemaregistry-client.spec.ts b/e2e/schemaregistry/schemaregistry-client.spec.ts index 2b0684fa..79380a22 100644 --- a/e2e/schemaregistry/schemaregistry-client.spec.ts +++ b/e2e/schemaregistry/schemaregistry-client.spec.ts @@ -148,7 +148,7 @@ describe('SchemaRegistryClient Integration Test', () => { const getMetadataResponse: SchemaMetadata = await schemaRegistryClient.getSchemaMetadata(testSubject, schemaVersion); expect(schemaMetadata).toEqual(getMetadataResponse); - const keyValueMetadata: { [key: string]: string } = { + const keyValueMetadata: { [key: string]: string } = { 'owner': 'Bob Jones', 'email': 'bob@acme.com' } @@ -163,11 +163,11 @@ describe('SchemaRegistryClient Integration Test', () => { const version = registerResponse?.version!; - const updateCompatibilityResponse: Compatibility = await schemaRegistryClient.updateCompatibility(testSubject, Compatibility.BackwardTransitive); - expect(updateCompatibilityResponse).toEqual(Compatibility.BackwardTransitive); + const updateCompatibilityResponse: Compatibility = await schemaRegistryClient.updateCompatibility(testSubject, Compatibility.BACKWARD_TRANSITIVE); + expect(updateCompatibilityResponse).toEqual(Compatibility.BACKWARD_TRANSITIVE); const getCompatibilityResponse: Compatibility = await schemaRegistryClient.getCompatibility(testSubject); - expect(getCompatibilityResponse).toEqual(Compatibility.BackwardTransitive); + expect(getCompatibilityResponse).toEqual(Compatibility.BACKWARD_TRANSITIVE); const testSubjectCompatibilityResponse: boolean = await schemaRegistryClient.testSubjectCompatibility(testSubject, backwardCompatibleSchemaInfo); expect(testSubjectCompatibilityResponse).toEqual(true); @@ -177,21 +177,21 @@ describe('SchemaRegistryClient Integration Test', () => { }); it('Should update and get default compatibility', async () => { - const updateDefaultCompatibilityResponse: Compatibility = await schemaRegistryClient.updateDefaultCompatibility(Compatibility.Full); - expect(updateDefaultCompatibilityResponse).toEqual(Compatibility.Full); + const updateDefaultCompatibilityResponse: Compatibility = await schemaRegistryClient.updateDefaultCompatibility(Compatibility.FULL); + expect(updateDefaultCompatibilityResponse).toEqual(Compatibility.FULL); const getDefaultCompatibilityResponse: Compatibility = await schemaRegistryClient.getDefaultCompatibility(); - expect(getDefaultCompatibilityResponse).toEqual(Compatibility.Full); + expect(getDefaultCompatibilityResponse).toEqual(Compatibility.FULL); }); it('Should update and get subject Config', async () => { const subjectConfigRequest: ServerConfig = { - compatibility: Compatibility.Full, + compatibility: Compatibility.FULL, normalize: true }; const subjectConfigResponse: ServerConfig = { - compatibilityLevel: Compatibility.Full, + compatibilityLevel: Compatibility.FULL, normalize: true }; @@ -207,12 +207,12 @@ describe('SchemaRegistryClient Integration Test', () => { it('Should get and set default Config', async () => { const serverConfigRequest: ServerConfig = { - compatibility: Compatibility.Full, + compatibility: Compatibility.FULL, normalize: false }; const serverConfigResponse: ServerConfig = { - compatibilityLevel: Compatibility.Full, + compatibilityLevel: Compatibility.FULL, normalize: false }; diff --git a/eslint.config.js b/eslint.config.js index 8ccf17d0..823b769b 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -57,7 +57,7 @@ module.exports = ts.config( ignores: ["**/*.js"], rules: { ...config.rules, - "prefer-const": 0, + "prefer-const": "warn", "@typescript-eslint/no-explicit-any": "warn", "@typescript-eslint/no-unused-vars": "warn", } diff --git a/package-lock.json b/package-lock.json index 1d946b41..cf540dae 100644 --- a/package-lock.json +++ b/package-lock.json @@ -10,14 +10,25 @@ "hasInstallScript": true, "license": "MIT", "dependencies": { + "@bufbuild/buf": "^1.37.0", + "@bufbuild/protobuf": "^2.0.0", + "@bufbuild/protoc-gen-es": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", "async-mutex": "^0.5.0", + "avsc": "^5.7.7", "axios": "^1.7.3", "bindings": "^1.3.1", "json-stringify-deterministic": "^1.0.12", "lru-cache": "^11.0.0", + "miscreant": "^0.3.2", "nan": "^2.17.0", - "ts-jest": "^29.2.4" + "ts-jest": "^29.2.4", + "validator": "^13.12.0" }, "devDependencies": { "@eslint/js": "^9.9.0", @@ -548,6 +559,204 @@ "resolved": "https://registry.npmjs.org/@bcoe/v8-coverage/-/v8-coverage-0.2.3.tgz", "integrity": "sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw==" }, + "node_modules/@bufbuild/buf": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf/-/buf-1.37.0.tgz", + "integrity": "sha512-33Rp90nRa7ebTwAjI8Yc7f8WOQMOEfRM0mg/o1bkNvE6cGQhYiq82Ag27pcegvYIyXl413XCeLMAUegsqjuFqw==", + "hasInstallScript": true, + "bin": { + "buf": "bin/buf", + "protoc-gen-buf-breaking": "bin/protoc-gen-buf-breaking", + "protoc-gen-buf-lint": "bin/protoc-gen-buf-lint" + }, + "engines": { + "node": ">=12" + }, + "optionalDependencies": { + "@bufbuild/buf-darwin-arm64": "1.37.0", + "@bufbuild/buf-darwin-x64": "1.37.0", + "@bufbuild/buf-linux-aarch64": "1.37.0", + "@bufbuild/buf-linux-x64": "1.37.0", + "@bufbuild/buf-win32-arm64": "1.37.0", + "@bufbuild/buf-win32-x64": "1.37.0" + } + }, + "node_modules/@bufbuild/buf-darwin-arm64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-arm64/-/buf-darwin-arm64-1.37.0.tgz", + "integrity": "sha512-YrT0HqLf7qJJ9VjXmnG900oGOjskerqfY4UTbpLb67b76uRxZm7pfSfqpM4bCaSru4j6GPBDwH9dkwiuNOenEA==", + "cpu": [ + "arm64" + ], + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-darwin-x64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-x64/-/buf-darwin-x64-1.37.0.tgz", + "integrity": "sha512-c1iDr1LaYcdzAK7v4ZBTiYVRPltPa1UNFQ3JJ6BI/RUaBk/eYFsykuNtsJQb5lXh/usjMz76juzOxAdYikcw7w==", + "cpu": [ + "x64" + ], + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-linux-aarch64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-aarch64/-/buf-linux-aarch64-1.37.0.tgz", + "integrity": "sha512-ENaCLU4XZxQFc6+pWjNGeKITlzY0jYeEmz6R2cEeR9kVl+HyNNcTMaKxdffG5HNU5Ci1dC3pSY/ZIF3ck8UHVw==", + "cpu": [ + "arm64" + ], + "optional": true, + "os": [ + "linux" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-linux-x64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-x64/-/buf-linux-x64-1.37.0.tgz", + "integrity": "sha512-/buQuUaBVU4rYqdoS/xmbyohJFaE4uy7boHMabCnI2aLsVBNPHYa/k0JMc+ZDRqgq0Wnth9VbIHn6DSGHG+cmw==", + "cpu": [ + "x64" + ], + "optional": true, + "os": [ + "linux" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-win32-arm64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-arm64/-/buf-win32-arm64-1.37.0.tgz", + "integrity": "sha512-5Y1aaHge9iQP+YNyxUayMd+rSVbjNXLBXniJDQhim56B+6eufPiKzmTgAKjDE9hGTT+mcBdLznrCMI0fzDLHJw==", + "cpu": [ + "arm64" + ], + "optional": true, + "os": [ + "win32" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-win32-x64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-x64/-/buf-win32-x64-1.37.0.tgz", + "integrity": "sha512-2BioEPdC6EM5zEKmTmM14ZJuuPjKuYIvoAwQ4hoCSJBllIhGvshk61NDYBorr1huEEq6baa4ITL/gWHJ86B0bw==", + "cpu": [ + "x64" + ], + "optional": true, + "os": [ + "win32" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/protobuf": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@bufbuild/protobuf/-/protobuf-2.0.0.tgz", + "integrity": "sha512-sw2JhwJyvyL0zlhG61aDzOVryEfJg2PDZFSV7i7IdC7nAE41WuXCru3QWLGiP87At0BMzKOoKO/FqEGoKygGZQ==" + }, + "node_modules/@bufbuild/protoc-gen-es": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@bufbuild/protoc-gen-es/-/protoc-gen-es-2.0.0.tgz", + "integrity": "sha512-xXDrDFjSD97jGu9RMA8O4Myj66DdCLOFeYG7Z0mX1aCE9hayymfor+BKgKCQouUfty6CYv8709FPYpEtDurLFQ==", + "dependencies": { + "@bufbuild/protobuf": "^2.0.0", + "@bufbuild/protoplugin": "2.0.0" + }, + "bin": { + "protoc-gen-es": "bin/protoc-gen-es" + }, + "engines": { + "node": ">=14" + }, + "peerDependencies": { + "@bufbuild/protobuf": "2.0.0" + }, + "peerDependenciesMeta": { + "@bufbuild/protobuf": { + "optional": true + } + } + }, + "node_modules/@bufbuild/protoplugin": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@bufbuild/protoplugin/-/protoplugin-2.0.0.tgz", + "integrity": "sha512-y77IarGNbJfr5YZHQ+biO0O9B8rXicKLLOFRJu9mKr9LYE5/CLIKqonIBfkc/lggCK2YOtinjZJEYb39HymIog==", + "dependencies": { + "@bufbuild/protobuf": "2.0.0", + "@typescript/vfs": "^1.5.2", + "typescript": "5.4.5" + } + }, + "node_modules/@bufbuild/protoplugin/node_modules/typescript": { + "version": "5.4.5", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.5.tgz", + "integrity": "sha512-vcI4UpRgg81oIRUFwR0WSIHKt11nJ7SAVlYNIu+QpqeyXP+gpQJy/Z4+F0aGxSE4MqwjyXvW/TzgkLAx2AGHwQ==", + "bin": { + "tsc": "bin/tsc", + "tsserver": "bin/tsserver" + }, + "engines": { + "node": ">=14.17" + } + }, + "node_modules/@criteria/json-pointer": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/@criteria/json-pointer/-/json-pointer-0.2.1.tgz", + "integrity": "sha512-+UcEXcbZLkmhB4vuuuLAfnWD7cGmUGrZxzldFFdLwFKglkGuLiE4YTBX0gvEQuVHTwMN/f1R/0TB3YRzf5/BPw==", + "engines": { + "node": ">=18.12.1" + } + }, + "node_modules/@criteria/json-schema": { + "version": "0.10.0", + "resolved": "https://registry.npmjs.org/@criteria/json-schema/-/json-schema-0.10.0.tgz", + "integrity": "sha512-Y+gLIVPdgflQ3zeayUcfYjHOf+KJOp8dx5LWpLfy3XJ0P61JxU5srk2cGpVZYPJNTnixm8tBwb8T26DCMjw+hw==", + "dependencies": { + "@criteria/json-pointer": "^0.2.1", + "toad-uri-js": "^5.0.1" + }, + "engines": { + "node": ">=18.12.1" + } + }, + "node_modules/@criteria/json-schema-validation": { + "version": "0.10.0", + "resolved": "https://registry.npmjs.org/@criteria/json-schema-validation/-/json-schema-validation-0.10.0.tgz", + "integrity": "sha512-QqkO0uEAjMIEdybZPSGZEOa2HExAr83lE5ya3bRTVFMByb9sXq7JK7ZyHHAhSOWQyUN1Tgkk/sozfLygG5VOuw==", + "dependencies": { + "@criteria/json-pointer": "^0.2.1", + "@criteria/json-schema": "^0.10.0", + "fast-deep-equal": "^3.1.3", + "punycode": "^2.3.1", + "smtp-address-parser": "^1.0.10", + "toad-uri-js": "^5.0.1" + }, + "engines": { + "node": ">=18.12.1" + } + }, "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", @@ -595,6 +804,28 @@ "url": "https://opencollective.com/eslint" } }, + "node_modules/@eslint/eslintrc/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/@eslint/eslintrc/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/@eslint/js": { "version": "9.9.0", "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.9.0.tgz", @@ -610,6 +841,11 @@ "integrity": "sha512-k2Ty1JcVojjJFwrg/ThKi2ujJ7XNLYaFGNB/bWT9wGR+oSMJHMa5w+CUq6p/pVrKeNNgA7pCqEcjSnHVoqJQFw==", "dev": true }, + "node_modules/@hackbg/miscreant-esm": { + "version": "0.3.2-patch.3", + "resolved": "https://registry.npmjs.org/@hackbg/miscreant-esm/-/miscreant-esm-0.3.2-patch.3.tgz", + "integrity": "sha512-CxWupG9uSc0e0toliMBc+ZSC2cXuHqcSgvwCgCiWXkpF6adzcttNFr6KyumI3utO/WQKQrdgq93LcQEueBjTHQ==" + }, "node_modules/@humanwhocodes/config-array": { "version": "0.11.14", "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.14.tgz", @@ -1319,6 +1555,11 @@ "resolved": "https://registry.npmjs.org/@types/stack-utils/-/stack-utils-2.0.3.tgz", "integrity": "sha512-9aEbYZ3TbYMznPdcdr3SmIrLXwC/AKZXQeCf9Pgao5CKb8CyHuEX5jzWPTkvregvhRJHcpRO6BFoGW9ycaOkYw==" }, + "node_modules/@types/validator": { + "version": "13.12.0", + "resolved": "https://registry.npmjs.org/@types/validator/-/validator-13.12.0.tgz", + "integrity": "sha512-nH45Lk7oPIJ1RVOF6JgFI6Dy0QpHEzq4QecZhvguxYPDwT8c93prCMqAtiIttm39voZ+DDR+qkNnMpJmMBRqag==" + }, "node_modules/@types/yargs": { "version": "17.0.32", "resolved": "https://registry.npmjs.org/@types/yargs/-/yargs-17.0.32.tgz", @@ -1936,6 +2177,17 @@ "url": "https://opencollective.com/typescript-eslint" } }, + "node_modules/@typescript/vfs": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/@typescript/vfs/-/vfs-1.6.0.tgz", + "integrity": "sha512-hvJUjNVeBMp77qPINuUvYXj4FyWeeMMKZkxEATEU3hqBAQ7qdTBCUFT7Sp0Zu0faeEtFf+ldXxMEDr/bk73ISg==", + "dependencies": { + "debug": "^4.1.1" + }, + "peerDependencies": { + "typescript": "*" + } + }, "node_modules/@ungap/structured-clone": { "version": "1.2.0", "resolved": "https://registry.npmjs.org/@ungap/structured-clone/-/structured-clone-1.2.0.tgz", @@ -2005,15 +2257,14 @@ } }, "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, + "version": "8.17.1", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.17.1.tgz", + "integrity": "sha512-B/gBuNg5SiMTrPkC+A2+cW0RszwxYmn6VYxB/inlBStS5nx6xHIt/ehKRhIMhqusl7a8LjQoZnjCs5vhwxOQ1g==", "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" + "fast-deep-equal": "^3.1.3", + "fast-uri": "^3.0.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2" }, "funding": { "type": "github", @@ -2142,6 +2393,14 @@ "integrity": "sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==", "license": "MIT" }, + "node_modules/avsc": { + "version": "5.7.7", + "resolved": "https://registry.npmjs.org/avsc/-/avsc-5.7.7.tgz", + "integrity": "sha512-9cYNccliXZDByFsFliVwk5GvTq058Fj513CiR4E60ndDwmuXzTJEp/Bp8FyuRmGyYupLjHLs+JA9/CBoVS4/NQ==", + "engines": { + "node": ">=0.11" + } + }, "node_modules/axios": { "version": "1.7.3", "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.3.tgz", @@ -2659,6 +2918,11 @@ "node": ">= 0.8" } }, + "node_modules/commander": { + "version": "2.20.3", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.3.tgz", + "integrity": "sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==" + }, "node_modules/concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", @@ -2821,6 +3085,11 @@ "node": ">=8" } }, + "node_modules/discontinuous-range": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/discontinuous-range/-/discontinuous-range-1.0.0.tgz", + "integrity": "sha512-c68LpLbO+7kP/b1Hr1qs8/BJ09F5khZGTxqxZuhzxpmwJKOgRFHJWIb9/KmqnqHhLdO55aOxFH/EGBvUQbL/RQ==" + }, "node_modules/doctrine": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", @@ -3050,6 +3319,28 @@ "node": "^12.22.0 || ^14.17.0 || >=16.0.0" } }, + "node_modules/eslint/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/eslint/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/espree": { "version": "9.6.1", "resolved": "https://registry.npmjs.org/espree/-/espree-9.6.1.tgz", @@ -3175,8 +3466,7 @@ "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 + "integrity": "sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q==" }, "node_modules/fast-glob": { "version": "3.3.2", @@ -3217,6 +3507,11 @@ "integrity": "sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw==", "dev": true }, + "node_modules/fast-uri": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/fast-uri/-/fast-uri-3.0.1.tgz", + "integrity": "sha512-MWipKbbYiYI0UC7cl8m/i/IWTqfC8YXsqjzybjddLsFjStroQzsHXkc73JutMvBiXmOvapk+axIl79ig5t55Bw==" + }, "node_modules/fastq": { "version": "1.17.1", "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.17.1.tgz", @@ -4575,10 +4870,9 @@ "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==" }, "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 + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz", + "integrity": "sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug==" }, "node_modules/json-stable-stringify-without-jsonify": { "version": "1.0.1", @@ -5006,6 +5300,11 @@ "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" }, + "node_modules/miscreant": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/miscreant/-/miscreant-0.3.2.tgz", + "integrity": "sha512-fL9KxsQz9BJB2KGPMHFrReioywkiomBiuaLk6EuChijK0BsJsIKJXdVomR+/bPj5mvbFD6wM0CM3bZio9g7OHA==" + }, "node_modules/mkdirp": { "version": "1.0.4", "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-1.0.4.tgz", @@ -5143,6 +5442,11 @@ "node": ">=10" } }, + "node_modules/moo": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/moo/-/moo-0.5.2.tgz", + "integrity": "sha512-iSAJLHYKnX41mKcJKjqvnAN9sf0LMDTXDEvFv+ffuRR9a1MIuXLjMNL6EsnDHSkKLTWNqQQ5uo61P4EbU4NU+Q==" + }, "node_modules/ms": { "version": "2.1.2", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", @@ -5158,6 +5462,27 @@ "resolved": "https://registry.npmjs.org/natural-compare/-/natural-compare-1.4.0.tgz", "integrity": "sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw==" }, + "node_modules/nearley": { + "version": "2.20.1", + "resolved": "https://registry.npmjs.org/nearley/-/nearley-2.20.1.tgz", + "integrity": "sha512-+Mc8UaAebFzgV+KpI5n7DasuuQCHA89dmwm7JXw3TV43ukfNQ9DnBH3Mdb2g/I4Fdxc26pwimBWvjIw0UAILSQ==", + "dependencies": { + "commander": "^2.19.0", + "moo": "^0.5.0", + "railroad-diagrams": "^1.0.0", + "randexp": "0.4.6" + }, + "bin": { + "nearley-railroad": "bin/nearley-railroad.js", + "nearley-test": "bin/nearley-test.js", + "nearley-unparse": "bin/nearley-unparse.js", + "nearleyc": "bin/nearleyc.js" + }, + "funding": { + "type": "individual", + "url": "https://nearley.js.org/#give-to-nearley" + } + }, "node_modules/negotiator": { "version": "0.6.3", "resolved": "https://registry.npmjs.org/negotiator/-/negotiator-0.6.3.tgz", @@ -5654,7 +5979,6 @@ "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" } @@ -5703,6 +6027,23 @@ } ] }, + "node_modules/railroad-diagrams": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/railroad-diagrams/-/railroad-diagrams-1.0.0.tgz", + "integrity": "sha512-cz93DjNeLY0idrCNOH6PviZGRN9GJhsdm9hpn1YCS879fj4W+x5IFJhhkRZcwVgMmFF7R82UA/7Oh+R8lLZg6A==" + }, + "node_modules/randexp": { + "version": "0.4.6", + "resolved": "https://registry.npmjs.org/randexp/-/randexp-0.4.6.tgz", + "integrity": "sha512-80WNmd9DA0tmZrw9qQa62GPPWfuXJknrmVmLcxvq4uZBdYqb1wYoKTmnlGUchvVWe0XiLupYkBoXVOxz3C8DYQ==", + "dependencies": { + "discontinuous-range": "1.0.0", + "ret": "~0.1.10" + }, + "engines": { + "node": ">=0.12" + } + }, "node_modules/randombytes": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", @@ -5750,6 +6091,14 @@ "node": ">=0.10.0" } }, + "node_modules/require-from-string": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/require-from-string/-/require-from-string-2.0.2.tgz", + "integrity": "sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw==", + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/requizzle": { "version": "0.2.4", "resolved": "https://registry.npmjs.org/requizzle/-/requizzle-0.2.4.tgz", @@ -5811,6 +6160,14 @@ "node": ">=10" } }, + "node_modules/ret": { + "version": "0.1.15", + "resolved": "https://registry.npmjs.org/ret/-/ret-0.1.15.tgz", + "integrity": "sha512-TTlYpa+OL+vMMNG24xSlQGEJ3B/RzEfUlLct7b5G/ytav+wPrplCpVMFuwzXbkecJrb6IYo1iFb0S9v37754mg==", + "engines": { + "node": ">=0.12" + } + }, "node_modules/retry": { "version": "0.12.0", "resolved": "https://registry.npmjs.org/retry/-/retry-0.12.0.tgz", @@ -5965,6 +6322,17 @@ "npm": ">= 3.0.0" } }, + "node_modules/smtp-address-parser": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/smtp-address-parser/-/smtp-address-parser-1.1.0.tgz", + "integrity": "sha512-Gz11jbNU0plrReU9Sj7fmshSBxxJ9ShdD2q4ktHIHo/rpTH6lFyQoYHYKINPJtPe8aHFnsbtW46Ls0tCCBsIZg==", + "dependencies": { + "nearley": "^2.20.1" + }, + "engines": { + "node": ">=0.10" + } + }, "node_modules/socks": { "version": "2.8.3", "resolved": "https://registry.npmjs.org/socks/-/socks-2.8.3.tgz", @@ -6212,6 +6580,14 @@ "node": ">=8.0" } }, + "node_modules/toad-uri-js": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/toad-uri-js/-/toad-uri-js-5.0.1.tgz", + "integrity": "sha512-r2c5hs10O0tcRvjUpgJdJf5CalaOZhY7oS9kvYBDu/rPg+02PWa1QAOb7+tvKtpmNCkW6w6F5WZt9BDWLCNHkQ==", + "dependencies": { + "punycode": "^2.3.1" + } + }, "node_modules/tr46": { "version": "0.0.3", "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", @@ -6662,6 +7038,14 @@ "node": ">=10.12.0" } }, + "node_modules/validator": { + "version": "13.12.0", + "resolved": "https://registry.npmjs.org/validator/-/validator-13.12.0.tgz", + "integrity": "sha512-c1Q0mCiPlgdTVVVIJIrBuxNicYE+t/7oKeI9MWLj3fh/uq2Pxh/3eeWbVZ4OcGW1TUf53At0njHw5SMdA3tmMg==", + "engines": { + "node": ">= 0.10" + } + }, "node_modules/walker": { "version": "1.0.8", "resolved": "https://registry.npmjs.org/walker/-/walker-1.0.8.tgz", diff --git a/package.json b/package.json index 2c3f4015..085efcfa 100644 --- a/package.json +++ b/package.json @@ -46,14 +46,25 @@ "typescript-eslint": "^8.2.0" }, "dependencies": { + "@bufbuild/buf": "^1.37.0", + "@bufbuild/protobuf": "^2.0.0", + "@bufbuild/protoc-gen-es": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", "async-mutex": "^0.5.0", + "avsc": "^5.7.7", "axios": "^1.7.3", "bindings": "^1.3.1", "json-stringify-deterministic": "^1.0.12", "lru-cache": "^11.0.0", + "miscreant": "^0.3.2", "nan": "^2.17.0", - "ts-jest": "^29.2.4" + "ts-jest": "^29.2.4", + "validator": "^13.12.0" }, "engines": { "node": ">=18.0.0" diff --git a/proto/confluent/meta.proto b/proto/confluent/meta.proto new file mode 100644 index 00000000..6016459b --- /dev/null +++ b/proto/confluent/meta.proto @@ -0,0 +1,28 @@ +syntax = "proto3"; +package confluent; + +import "google/protobuf/descriptor.proto"; + +option go_package="../confluent"; + +message Meta { + string doc = 1; + map params = 2; + repeated string tags = 3; +} + +extend google.protobuf.FileOptions { + Meta file_meta = 1088; +} +extend google.protobuf.MessageOptions { + Meta message_meta = 1088; +} +extend google.protobuf.FieldOptions { + Meta field_meta = 1088; +} +extend google.protobuf.EnumOptions { + Meta enum_meta = 1088; +} +extend google.protobuf.EnumValueOptions { + Meta enum_value_meta = 1088; +} diff --git a/proto/confluent/types/decimal.proto b/proto/confluent/types/decimal.proto new file mode 100644 index 00000000..75d8b9b4 --- /dev/null +++ b/proto/confluent/types/decimal.proto @@ -0,0 +1,17 @@ +syntax = "proto3"; + +package confluent.type; + +option go_package="../types"; + +message Decimal { + + // The two's-complement representation of the unscaled integer value in big-endian byte order + bytes value = 1; + + // The precision + uint32 precision = 2; + + // The scale + int32 scale = 3; +} \ No newline at end of file diff --git a/proto/rules/encryption/tink/proto/aes_gcm.proto b/proto/rules/encryption/tink/proto/aes_gcm.proto new file mode 100644 index 00000000..fba7a89e --- /dev/null +++ b/proto/rules/encryption/tink/proto/aes_gcm.proto @@ -0,0 +1,67 @@ +// Copyright 2017 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +//////////////////////////////////////////////////////////////////////////////// + +syntax = "proto3"; + +package google.crypto.tink; + +option java_package = "com.google.crypto.tink.proto"; +option java_multiple_files = true; +option go_package = "github.com/google/tink/proto/aes_gcm_go_proto"; +option objc_class_prefix = "TINKPB"; + +message AesGcmKeyFormat { + uint32 key_size = 2; + uint32 version = 3; +} + +// key_type: type.googleapis.com/google.crypto.tink.AesGcmKey +// +// A AesGcmKey is an AEAD key. Mathematically, it represents the functions +// Encrypt and Decrypt which we define in the following. +// +// First, Tink computes a "output prefix" OP by considering the +// "OutputPrefixType" message in Keyset.Key and the ID of the key using the +// Tink function "AEAD-OutputPrefix": (AesGcmKeys must always be stored in a +// keyset). +// +// AEAD-OutputPrefix(output_prefix_type, id): +// if output_prefix_type == RAW: +// return ""; +// if output_prefix_type == TINK: +// return 0x01 + BigEndian(id) +// if output_prefix_type == CRUNCHY: +// return 0x00 + BigEndian(id) +// +// Then, the function defined by this is defined as: +// [GCM], Section 5.2.1: +// * "Encrypt" maps a plaintext P and associated data A to a ciphertext given +// by the concatenation OP || IV || C || T. In addition to [GCM], Tink +// has the following restriction: IV is a uniformly random initialization +// vector of length 12 bytes and T is restricted to 16 bytes. +// +// * If OP matches the result of AEAD-OutputPrefix, then "Decrypt" maps the +// input OP || IV || C || T and A to the the output P in the manner as +// described in [GCM], Section 5.2.2. If OP does not match, then "Decrypt" +// returns an error. +// [GCM]: NIST Special Publication 800-38D: Recommendation for Block Cipher +// Modes of Operation: Galois/Counter Mode (GCM) and GMAC. +// http://csrc.nist.gov/publications/nistpubs/800-38D/SP-800-38D.pdf. + +message AesGcmKey { + uint32 version = 1; + bytes key_value = 3; +} diff --git a/proto/rules/encryption/tink/proto/aes_siv.proto b/proto/rules/encryption/tink/proto/aes_siv.proto new file mode 100644 index 00000000..00230278 --- /dev/null +++ b/proto/rules/encryption/tink/proto/aes_siv.proto @@ -0,0 +1,36 @@ +// Copyright 2017 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +//////////////////////////////////////////////////////////////////////////////// + +syntax = "proto3"; + +package google.crypto.tink; + +option java_package = "com.google.crypto.tink.proto"; +option java_multiple_files = true; +option go_package = "github.com/google/tink/proto/aes_siv_go_proto"; + +message AesSivKeyFormat { + // Only valid value is: 64. + uint32 key_size = 1; + uint32 version = 2; +} + +// key_type: type.googleapis.com/google.crypto.tink.AesSivKey +message AesSivKey { + uint32 version = 1; + // First half is AES-CTR key, second is AES-SIV. + bytes key_value = 2; +} diff --git a/schemaregistry/confluent/meta_pb.ts b/schemaregistry/confluent/meta_pb.ts new file mode 100644 index 00000000..0f38f3c3 --- /dev/null +++ b/schemaregistry/confluent/meta_pb.ts @@ -0,0 +1,73 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file confluent/meta.proto (package confluent, syntax proto3) +/* eslint-disable */ + +import type { GenExtension, GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { extDesc, fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { EnumOptions, EnumValueOptions, FieldOptions, FileOptions, MessageOptions } from "@bufbuild/protobuf/wkt"; +import { file_google_protobuf_descriptor } from "@bufbuild/protobuf/wkt"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file confluent/meta.proto. + */ +export const file_confluent_meta: GenFile = /*@__PURE__*/ + fileDesc("ChRjb25mbHVlbnQvbWV0YS5wcm90bxIJY29uZmx1ZW50In0KBE1ldGESCwoDZG9jGAEgASgJEisKBnBhcmFtcxgCIAMoCzIbLmNvbmZsdWVudC5NZXRhLlBhcmFtc0VudHJ5EgwKBHRhZ3MYAyADKAkaLQoLUGFyYW1zRW50cnkSCwoDa2V5GAEgASgJEg0KBXZhbHVlGAIgASgJOgI4ATpLCglmaWxlX21ldGESHC5nb29nbGUucHJvdG9idWYuRmlsZU9wdGlvbnMYwAggASgLMg8uY29uZmx1ZW50Lk1ldGFSCGZpbGVNZXRhOlQKDG1lc3NhZ2VfbWV0YRIfLmdvb2dsZS5wcm90b2J1Zi5NZXNzYWdlT3B0aW9ucxjACCABKAsyDy5jb25mbHVlbnQuTWV0YVILbWVzc2FnZU1ldGE6TgoKZmllbGRfbWV0YRIdLmdvb2dsZS5wcm90b2J1Zi5GaWVsZE9wdGlvbnMYwAggASgLMg8uY29uZmx1ZW50Lk1ldGFSCWZpZWxkTWV0YTpLCgllbnVtX21ldGESHC5nb29nbGUucHJvdG9idWYuRW51bU9wdGlvbnMYwAggASgLMg8uY29uZmx1ZW50Lk1ldGFSCGVudW1NZXRhOlsKD2VudW1fdmFsdWVfbWV0YRIhLmdvb2dsZS5wcm90b2J1Zi5FbnVtVmFsdWVPcHRpb25zGMAIIAEoCzIPLmNvbmZsdWVudC5NZXRhUg1lbnVtVmFsdWVNZXRhQg5aDC4uL2NvbmZsdWVudGIGcHJvdG8z", [file_google_protobuf_descriptor]); + +/** + * @generated from message confluent.Meta + */ +export type Meta = Message<"confluent.Meta"> & { + /** + * @generated from field: string doc = 1; + */ + doc: string; + + /** + * @generated from field: map params = 2; + */ + params: { [key: string]: string }; + + /** + * @generated from field: repeated string tags = 3; + */ + tags: string[]; +}; + +/** + * Describes the message confluent.Meta. + * Use `create(MetaSchema)` to create a new message. + */ +export const MetaSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_confluent_meta, 0); + +/** + * @generated from extension: confluent.Meta file_meta = 1088; + */ +export const file_meta: GenExtension = /*@__PURE__*/ + extDesc(file_confluent_meta, 0); + +/** + * @generated from extension: confluent.Meta message_meta = 1088; + */ +export const message_meta: GenExtension = /*@__PURE__*/ + extDesc(file_confluent_meta, 1); + +/** + * @generated from extension: confluent.Meta field_meta = 1088; + */ +export const field_meta: GenExtension = /*@__PURE__*/ + extDesc(file_confluent_meta, 2); + +/** + * @generated from extension: confluent.Meta enum_meta = 1088; + */ +export const enum_meta: GenExtension = /*@__PURE__*/ + extDesc(file_confluent_meta, 3); + +/** + * @generated from extension: confluent.Meta enum_value_meta = 1088; + */ +export const enum_value_meta: GenExtension = /*@__PURE__*/ + extDesc(file_confluent_meta, 4); + diff --git a/schemaregistry/confluent/types/decimal_pb.ts b/schemaregistry/confluent/types/decimal_pb.ts new file mode 100644 index 00000000..67160fc5 --- /dev/null +++ b/schemaregistry/confluent/types/decimal_pb.ts @@ -0,0 +1,47 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file confluent/types/decimal.proto (package confluent.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file confluent/types/decimal.proto. + */ +export const file_confluent_types_decimal: GenFile = /*@__PURE__*/ + fileDesc("Ch1jb25mbHVlbnQvdHlwZXMvZGVjaW1hbC5wcm90bxIOY29uZmx1ZW50LnR5cGUiOgoHRGVjaW1hbBINCgV2YWx1ZRgBIAEoDBIRCglwcmVjaXNpb24YAiABKA0SDQoFc2NhbGUYAyABKAVCCloILi4vdHlwZXNiBnByb3RvMw"); + +/** + * @generated from message confluent.type.Decimal + */ +export type Decimal = Message<"confluent.type.Decimal"> & { + /** + * The two's-complement representation of the unscaled integer value in big-endian byte order + * + * @generated from field: bytes value = 1; + */ + value: Uint8Array; + + /** + * The precision + * + * @generated from field: uint32 precision = 2; + */ + precision: number; + + /** + * The scale + * + * @generated from field: int32 scale = 3; + */ + scale: number; +}; + +/** + * Describes the message confluent.type.Decimal. + * Use `create(DecimalSchema)` to create a new message. + */ +export const DecimalSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_confluent_types_decimal, 0); + diff --git a/schemaregistry/dekregistry/dekregistry-client.ts b/schemaregistry/dekregistry/dekregistry-client.ts index c40796f5..84216a62 100644 --- a/schemaregistry/dekregistry/dekregistry-client.ts +++ b/schemaregistry/dekregistry/dekregistry-client.ts @@ -46,9 +46,11 @@ interface Dek { } interface Client { - registerKek(name: string, kmsType: string, kmsKeyId: string, kmsProps: { [key: string]: string }, doc: string, shared: boolean): Promise; + registerKek(name: string, kmsType: string, kmsKeyId: string, shared: boolean, + kmsProps?: { [key: string]: string }, doc?: string): Promise; getKek(name: string, deleted: boolean): Promise; - registerDek(kekName: string, subject: string, algorithm: string, encryptedKeyMaterial: string, version: number): Promise; + registerDek(kekName: string, subject: string, algorithm: string, version: number, + encryptedKeyMaterial?: string): Promise; getDek(kekName: string, subject: string, algorithm: string, version: number, deleted: boolean): Promise; close(): Promise; } @@ -67,14 +69,14 @@ class DekRegistryClient implements Client { }; - this.restService = new RestService(config.createAxiosDefaults, config.baseURLs, config.isForward); + this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults); this.kekCache = new LRUCache(cacheOptions); this.dekCache = new LRUCache(cacheOptions); this.kekMutex = new Mutex(); this.dekMutex = new Mutex(); } - public static getEncryptedKeyMaterialBytes(dek: Dek): Buffer | null { + static getEncryptedKeyMaterialBytes(dek: Dek): Buffer | null { if (!dek.encryptedKeyMaterial) { return null; } @@ -94,7 +96,7 @@ class DekRegistryClient implements Client { return dek.encryptedKeyMaterialBytes; } - public static getKeyMaterialBytes(dek: Dek): Buffer | null { + static getKeyMaterialBytes(dek: Dek): Buffer | null { if (!dek.keyMaterial) { return null; } @@ -114,15 +116,15 @@ class DekRegistryClient implements Client { return dek.keyMaterialBytes; } - public static setKeyMaterial(dek: Dek, keyMaterialBytes: Buffer): void { + static setKeyMaterial(dek: Dek, keyMaterialBytes: Buffer): void { if (keyMaterialBytes) { const str = keyMaterialBytes.toString('base64'); dek.keyMaterial = str; } } - public async registerKek(name: string, kmsType: string, kmsKeyId: string, - kmsProps: { [key: string]: string }, doc: string, shared: boolean): Promise { + async registerKek(name: string, kmsType: string, kmsKeyId: string, shared: boolean, + kmsProps?: { [key: string]: string }, doc?: string): Promise { const cacheKey = stringify({ name, deleted: false }); return await this.kekMutex.runExclusive(async () => { @@ -135,8 +137,8 @@ class DekRegistryClient implements Client { name, kmsType, kmsKeyId, - kmsProps, - doc, + ...kmsProps && { kmsProps }, + ...doc && { doc }, shared, }; @@ -149,7 +151,7 @@ class DekRegistryClient implements Client { }); } - public async getKek(name: string, deleted: boolean = false): Promise { + async getKek(name: string, deleted: boolean = false): Promise { const cacheKey = stringify({ name, deleted }); return await this.kekMutex.runExclusive(async () => { @@ -167,8 +169,8 @@ class DekRegistryClient implements Client { }); } - public async registerDek(kekName: string, subject: string, - algorithm: string, encryptedKeyMaterial: string, version: number = 1): Promise { + async registerDek(kekName: string, subject: string, algorithm: string, + version: number = 1, encryptedKeyMaterial?: string): Promise { const cacheKey = stringify({ kekName, subject, version, algorithm, deleted: false }); return await this.dekMutex.runExclusive(async () => { @@ -181,7 +183,7 @@ class DekRegistryClient implements Client { subject, version, algorithm, - encryptedKeyMaterial, + ...encryptedKeyMaterial && { encryptedKeyMaterial }, }; kekName = encodeURIComponent(kekName); @@ -198,7 +200,7 @@ class DekRegistryClient implements Client { }); } - public async getDek(kekName: string, subject: string, + async getDek(kekName: string, subject: string, algorithm: string, version: number = 1, deleted: boolean = false): Promise { const cacheKey = stringify({ kekName, subject, version, algorithm, deleted }); @@ -218,12 +220,12 @@ class DekRegistryClient implements Client { }); } - public async close(): Promise { + async close(): Promise { return; } //Cache methods for testing - public async checkLatestDekInCache(kekName: string, subject: string, algorithm: string): Promise { + async checkLatestDekInCache(kekName: string, subject: string, algorithm: string): Promise { const cacheKey = stringify({ kekName, subject, version: -1, algorithm, deleted: false }); const cachedDek = this.dekCache.get(cacheKey); return cachedDek !== undefined; diff --git a/schemaregistry/dekregistry/mock-dekregistry-client.ts b/schemaregistry/dekregistry/mock-dekregistry-client.ts index 08d7e0ec..f0145275 100644 --- a/schemaregistry/dekregistry/mock-dekregistry-client.ts +++ b/schemaregistry/dekregistry/mock-dekregistry-client.ts @@ -11,8 +11,8 @@ class MockDekRegistryClient implements Client { this.dekCache = new Map(); } - public async registerKek(name: string, kmsType: string, kmsKeyId: string, - kmsProps: { [key: string]: string }, doc: string, shared: boolean): Promise { + async registerKek(name: string, kmsType: string, kmsKeyId: string, shared: boolean, + kmsProps?: { [key: string]: string }, doc?: string): Promise { const cacheKey = stringify({ name, deleted: false }); const cachedKek = this.kekCache.get(cacheKey); if (cachedKek) { @@ -23,8 +23,8 @@ class MockDekRegistryClient implements Client { name, kmsType, kmsKeyId, - kmsProps, - doc, + ...kmsProps && { kmsProps }, + ...doc && { doc }, shared }; @@ -32,7 +32,7 @@ class MockDekRegistryClient implements Client { return kek; } - public async getKek(name: string, deleted: boolean = false): Promise { + async getKek(name: string, deleted: boolean = false): Promise { const cacheKey = stringify({ name, deleted }); const cachedKek = this.kekCache.get(cacheKey); if (cachedKek && (!cachedKek.deleted || deleted)) { @@ -42,8 +42,8 @@ class MockDekRegistryClient implements Client { throw new Error(`Kek not found: ${name}`); } - public async registerDek(kekName: string, subject: string, - algorithm: string, encryptedKeyMaterial: string, version: number): Promise { + async registerDek(kekName: string, subject: string, algorithm: string, + version: number = 1, encryptedKeyMaterial?: string): Promise { const cacheKey = stringify({ kekName, subject, version, algorithm, deleted: false }); const cachedDek = this.dekCache.get(cacheKey); if (cachedDek) { @@ -54,7 +54,7 @@ class MockDekRegistryClient implements Client { kekName, subject, algorithm, - encryptedKeyMaterial, + ...encryptedKeyMaterial && { encryptedKeyMaterial }, version, ts: MOCK_TS }; @@ -63,13 +63,13 @@ class MockDekRegistryClient implements Client { return dek; } - public async getDek(kekName: string, subject: string, + async getDek(kekName: string, subject: string, algorithm: string, version: number = 1, deleted: boolean = false): Promise { if (version === -1) { let latestVersion = 0; for (let key of this.dekCache.keys()) { const parsedKey = JSON.parse(key); - if (parsedKey.kekName === kekName && parsedKey.subject === subject + if (parsedKey.kekName === kekName && parsedKey.subject === subject && parsedKey.algorithm === algorithm && parsedKey.deleted === deleted) { latestVersion = Math.max(latestVersion, parsedKey.version); } @@ -89,9 +89,9 @@ class MockDekRegistryClient implements Client { throw new Error(`Dek not found: ${subject}`); } - public async close() { + async close() { return; } } -export { MockDekRegistryClient }; \ No newline at end of file +export { MockDekRegistryClient }; diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts index c7550170..7e018983 100644 --- a/schemaregistry/mock-schemaregistry-client.ts +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -1,6 +1,7 @@ import { Client, Compatibility, SchemaInfo, SchemaMetadata, ServerConfig } from './schemaregistry-client'; import stringify from "json-stringify-deterministic"; +import {ClientConfig} from "./rest-service"; interface VersionCacheEntry { version: number; @@ -33,13 +34,15 @@ class Counter { const noSubject = ""; class MockClient implements Client { + private clientConfig?: ClientConfig; private infoToSchemaCache: Map; private idToSchemaCache: Map; private schemaToVersionCache: Map; private configCache: Map; private counter: Counter; - constructor() { + constructor(config?: ClientConfig) { + this.clientConfig = config this.infoToSchemaCache = new Map(); this.idToSchemaCache = new Map(); this.schemaToVersionCache = new Map(); @@ -47,7 +50,11 @@ class MockClient implements Client { this.counter = new Counter(); } - public async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + config(): ClientConfig { + return this.clientConfig! + } + + async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const metadata = await this.registerFullResponse(subject, schema, normalize); if (!metadata) { throw new Error("Failed to register schema"); @@ -55,7 +62,7 @@ class MockClient implements Client { return metadata.id; } - public async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema }); const cacheEntry = this.infoToSchemaCache.get(cacheKey); @@ -109,7 +116,7 @@ class MockClient implements Client { this.schemaToVersionCache.set(cacheKey, { version: newVersion, softDeleted: false }); } - public async getBySubjectAndId(subject: string, id: number): Promise { + async getBySubjectAndId(subject: string, id: number): Promise { const cacheKey = stringify({ subject, id }); const cacheEntry = this.idToSchemaCache.get(cacheKey); @@ -119,7 +126,7 @@ class MockClient implements Client { return cacheEntry.info; } - public async getId(subject: string, schema: SchemaInfo): Promise { + async getId(subject: string, schema: SchemaInfo): Promise { const cacheKey = stringify({ subject, schema }); const cacheEntry = this.infoToSchemaCache.get(cacheKey); if (!cacheEntry || cacheEntry.softDeleted) { @@ -128,7 +135,7 @@ class MockClient implements Client { return cacheEntry.metadata.id; } - public async getLatestSchemaMetadata(subject: string): Promise { + async getLatestSchemaMetadata(subject: string): Promise { const version = await this.latestVersion(subject); if (version === -1) { throw new Error("No versions found for subject"); @@ -137,7 +144,7 @@ class MockClient implements Client { return this.getSchemaMetadata(subject, version); } - public async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { + async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { let json; for (const [key, value] of this.schemaToVersionCache.entries()) { const parsedKey = JSON.parse(key); @@ -170,7 +177,7 @@ class MockClient implements Client { }; } - public async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { + async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { let metadataStr = ''; for (const key in metadata) { @@ -214,7 +221,7 @@ class MockClient implements Client { return true; } - public async getAllVersions(subject: string): Promise { + async getAllVersions(subject: string): Promise { const results = await this.allVersions(subject); if (results.length === 0) { @@ -267,7 +274,7 @@ class MockClient implements Client { } } - public async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema }); const cacheEntry = this.schemaToVersionCache.get(cacheKey); @@ -278,7 +285,7 @@ class MockClient implements Client { return cacheEntry.version; } - public async getAllSubjects(): Promise { + async getAllSubjects(): Promise { const subjects: string[] = []; for (const [key, value] of this.schemaToVersionCache.entries()) { const parsedKey = JSON.parse(key); @@ -289,7 +296,7 @@ class MockClient implements Client { return subjects.sort(); } - public async deleteSubject(subject: string, permanent: boolean = false): Promise { + async deleteSubject(subject: string, permanent: boolean = false): Promise { const deletedVersions: number[] = []; for (const [key, value] of this.infoToSchemaCache.entries()) { const parsedKey = JSON.parse(key); @@ -320,12 +327,12 @@ class MockClient implements Client { return deletedVersions; } - public async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { + async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { for (const [key, value] of this.schemaToVersionCache.entries()) { const parsedKey = JSON.parse(key); if (parsedKey.subject === subject && value.version === version) { await this.deleteVersion(key, version, permanent); - + const cacheKeySchema = stringify({ subject, schema: parsedKey.schema }); const cacheEntry = this.infoToSchemaCache.get(cacheKeySchema); if (cacheEntry) { @@ -345,15 +352,15 @@ class MockClient implements Client { return version; } - public async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { + async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { throw new Error("Unsupported operation"); } - public async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { + async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { throw new Error("Unsupported operation"); } - public async getCompatibility(subject: string): Promise { + async getCompatibility(subject: string): Promise { const cacheEntry = this.configCache.get(subject); if (!cacheEntry) { throw new Error("Subject not found"); @@ -361,12 +368,12 @@ class MockClient implements Client { return cacheEntry.compatibilityLevel as Compatibility; } - public async updateCompatibility(subject: string, compatibility: Compatibility): Promise { + async updateCompatibility(subject: string, compatibility: Compatibility): Promise { this.configCache.set(subject, { compatibilityLevel: compatibility }); return compatibility; } - public async getDefaultCompatibility(): Promise { + async getDefaultCompatibility(): Promise { const cacheEntry = this.configCache.get(noSubject); if (!cacheEntry) { throw new Error("Default compatibility not found"); @@ -374,12 +381,12 @@ class MockClient implements Client { return cacheEntry.compatibilityLevel as Compatibility; } - public async updateDefaultCompatibility(compatibility: Compatibility): Promise { + async updateDefaultCompatibility(compatibility: Compatibility): Promise { this.configCache.set(noSubject, { compatibilityLevel: compatibility }); return compatibility; } - public async getConfig(subject: string): Promise { + async getConfig(subject: string): Promise { const cacheEntry = this.configCache.get(subject); if (!cacheEntry) { throw new Error("Subject not found"); @@ -387,12 +394,12 @@ class MockClient implements Client { return cacheEntry; } - public async updateConfig(subject: string, config: ServerConfig): Promise { + async updateConfig(subject: string, config: ServerConfig): Promise { this.configCache.set(subject, config); return config; } - public async getDefaultConfig(): Promise { + async getDefaultConfig(): Promise { const cacheEntry = this.configCache.get(noSubject); if (!cacheEntry) { throw new Error("Default config not found"); @@ -400,12 +407,12 @@ class MockClient implements Client { return cacheEntry; } - public async updateDefaultConfig(config: ServerConfig): Promise { + async updateDefaultConfig(config: ServerConfig): Promise { this.configCache.set(noSubject, config); return config; } - public async close(): Promise { + async close(): Promise { return; } @@ -414,4 +421,4 @@ class MockClient implements Client { } } -export { MockClient }; \ No newline at end of file +export { MockClient }; diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index 333fc46a..cf992c9e 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -10,19 +10,19 @@ import { RestError } from './rest-error'; * of the MIT license. See the LICENSE.txt file for details. */ -export type ClientConfig = { - createAxiosDefaults: CreateAxiosDefaults, +export interface ClientConfig { baseURLs: string[], cacheCapacity: number, cacheLatestTtlSecs?: number, isForward?: boolean + createAxiosDefaults?: CreateAxiosDefaults, } export class RestService { private client: AxiosInstance; private baseURLs: string[]; - constructor(axiosDefaults: CreateAxiosDefaults, baseURLs: string[], isForward?: boolean) { + constructor(baseURLs: string[], isForward?: boolean, axiosDefaults?: CreateAxiosDefaults) { this.client = axios.create(axiosDefaults); this.baseURLs = baseURLs; @@ -31,7 +31,7 @@ export class RestService { } } - public async handleRequest( + async handleRequest( url: string, method: 'GET' | 'POST' | 'PUT' | 'DELETE', data?: any, // eslint-disable-line @typescript-eslint/no-explicit-any @@ -66,11 +66,11 @@ export class RestService { throw new Error('Internal HTTP retry error'); // Should never reach here } - public setHeaders(headers: Record): void { + setHeaders(headers: Record): void { this.client.defaults.headers.common = { ...this.client.defaults.headers.common, ...headers } } - public setAuth(basicAuth?: string, bearerToken?: string): void { + setAuth(basicAuth?: string, bearerToken?: string): void { if (basicAuth) { this.client.defaults.headers.common['Authorization'] = `Basic ${basicAuth}` } @@ -80,11 +80,11 @@ export class RestService { } } - public setTimeout(timeout: number): void { + setTimeout(timeout: number): void { this.client.defaults.timeout = timeout } - public setBaseURL(baseUrl: string): void { + setBaseURL(baseUrl: string): void { this.client.defaults.baseURL = baseUrl } } diff --git a/schemaregistry/rules/encryption/encrypt-executor.ts b/schemaregistry/rules/encryption/encrypt-executor.ts new file mode 100644 index 00000000..6ba09148 --- /dev/null +++ b/schemaregistry/rules/encryption/encrypt-executor.ts @@ -0,0 +1,529 @@ +import { + FieldContext, + FieldRuleExecutor, + FieldTransform, + FieldType, + MAGIC_BYTE, + RuleContext, + RuleError +} from "../../serde/serde"; +import {RuleMode,} from "../../schemaregistry-client"; +import {Client, Dek, DekRegistryClient, Kek} from "../../dekregistry/dekregistry-client"; +import {registerRuleExecutor} from "../../serde/rule-registry"; +import {ClientConfig} from "../../rest-service"; +import {RestError} from "../../rest-error"; +import * as Random from './tink/random'; +import * as Registry from './kms-registry' +import {KmsClient} from "./kms-registry"; +import {AesGcmKey, AesGcmKeySchema} from "./tink/proto/aes_gcm_pb"; +import {AesSivKey, AesSivKeySchema} from "./tink/proto/aes_siv_pb"; +import {create, fromBinary, toBinary} from "@bufbuild/protobuf"; +import {fromRawKey as aesGcmFromRawKey} from "./tink/aes_gcm"; +import {fromRawKey as aesSivFromRawKey} from "./tink/aes_siv"; + +// EncryptKekName represents a kek name +const ENCRYPT_KEK_NAME = 'encrypt.kek.name' +// EncryptKmsKeyId represents a kms key ID +const ENCRYPT_KMS_KEY_ID = 'encrypt.kms.key.id' +// EncryptKmsType represents a kms type +const ENCRYPT_KMS_TYPE = 'encrypt.kms.type' +// EncryptDekAlgorithm represents a dek algorithm +const ENCRYPT_DEK_ALGORITHM = 'encrypt.dek.algorithm' +// EncryptDekExpiryDays represents dek expiry days +const ENCRYPT_DEK_EXPIRY_DAYS = 'encrypt.dek.expiry.days' + +// MillisInDay represents number of milliseconds in a day +const MILLIS_IN_DAY = 24 * 60 * 60 * 1000 + +enum DekFormat { + AES128_GCM = 'AES128_GCM', + AES256_GCM = 'AES256_GCM', + AES256_SIV = 'AES256_SIV', +} + +interface KekId { + name: string + deleted: boolean +} + +interface DekId { + kekName: string + subject: string + version: number | null + algorithm: string + deleted: boolean +} + +export class FieldEncryptionExecutor extends FieldRuleExecutor { + client: Client | null = null + + static register() { + registerRuleExecutor(new FieldEncryptionExecutor()) + } + + override configure(clientConfig: ClientConfig, config: Map) { + // TODO use mock + this.client = new DekRegistryClient(clientConfig) + this.config = config + } + + type(): string { + return 'ENCRYPT' + } + + newTransform(ctx: RuleContext): FieldTransform { + const cryptor = this.getCryptor(ctx) + const kekName = this.getKekName(ctx) + const dekExpiryDays = this.getDekExpiryDays(ctx) + const transform = + new FieldEncryptionExecutorTransform(this, cryptor, kekName, dekExpiryDays) + return transform + } + + override close() { + if (this.client != null) { + this.client.close() + } + } + + private getCryptor(ctx: RuleContext): Cryptor { + let dekAlgorithm = DekFormat.AES256_GCM + const dekAlgorithmStr = ctx.getParameter(ENCRYPT_DEK_ALGORITHM) + if (dekAlgorithmStr != null) { + dekAlgorithm = DekFormat[dekAlgorithmStr as keyof typeof DekFormat] + } + const cryptor = new Cryptor(dekAlgorithm) + return cryptor + } + + private getKekName(ctx: RuleContext): string { + const kekName = ctx.getParameter(ENCRYPT_KEK_NAME) + if (kekName == null) { + throw new RuleError('no kek name found') + } + if (kekName === '') { + throw new RuleError('empty kek name') + } + return kekName + } + + private getDekExpiryDays(ctx: RuleContext): number { + const expiryDaysStr = ctx.getParameter(ENCRYPT_DEK_EXPIRY_DAYS) + if (expiryDaysStr == null) { + return 0 + } + const expiryDays = Number(expiryDaysStr) + if (isNaN(expiryDays)) { + throw new RuleError('invalid expiry days') + } + if (expiryDays < 0) { + throw new RuleError('negative expiry days') + } + return expiryDays + } +} + +export class Cryptor { + static readonly EMPTY_AAD = Buffer.from([]) + + dekFormat: DekFormat + isDeterministic: boolean + + constructor(dekFormat: DekFormat) { + this.dekFormat = dekFormat + this.isDeterministic = dekFormat === DekFormat.AES256_SIV + } + + private keySize(): number { + switch (this.dekFormat) { + case DekFormat.AES256_SIV: + // Generate 2 256-bit keys + return 64 + case DekFormat.AES128_GCM: + // Generate 128-bit key + return 16 + case DekFormat.AES256_GCM: + // Generate 256-bit key + return 32 + default: + throw new RuleError('unsupported dek format') + } + } + + generateKey(): Buffer { + let rawKey = Random.randBytes(this.keySize()) + switch (this.dekFormat) { + case DekFormat.AES256_SIV: + const aesSivKey: AesSivKey = create(AesSivKeySchema, { + version: 0, + keyValue: rawKey + }); + return Buffer.from(toBinary(AesSivKeySchema, aesSivKey)) + case DekFormat.AES128_GCM: + case DekFormat.AES256_GCM: + const aesGcmKey: AesGcmKey = create(AesGcmKeySchema, { + version: 0, + keyValue: rawKey + }); + return Buffer.from(toBinary(AesGcmKeySchema, aesGcmKey)) + default: + throw new RuleError('unsupported dek format') + } + } + + async encrypt(dek: Buffer, plaintext: Buffer): Promise { + let rawKey + switch (this.dekFormat) { + case DekFormat.AES256_SIV: + const aesSivKey = fromBinary(AesSivKeySchema, dek) + rawKey = aesSivKey.keyValue + return Buffer.from(await this.encryptWithAesSiv(rawKey, plaintext)) + case DekFormat.AES128_GCM: + case DekFormat.AES256_GCM: + const aesGcmKey = fromBinary(AesGcmKeySchema, dek) + rawKey = aesGcmKey.keyValue + return Buffer.from(await this.encryptWithAesGcm(rawKey, plaintext)) + default: + throw new RuleError('unsupported dek format') + } + } + + async decrypt(dek: Buffer, ciphertext: Buffer): Promise { + let rawKey + switch (this.dekFormat) { + case DekFormat.AES256_SIV: + const aesSivKey = fromBinary(AesSivKeySchema, dek) + rawKey = aesSivKey.keyValue + return Buffer.from(await this.decryptWithAesSiv(rawKey, ciphertext)) + case DekFormat.AES128_GCM: + case DekFormat.AES256_GCM: + const aesGcmKey = fromBinary(AesGcmKeySchema, dek) + rawKey = aesGcmKey.keyValue + return Buffer.from(await this.decryptWithAesGcm(rawKey, ciphertext)) + default: + throw new RuleError('unsupported dek format') + } + } + + async encryptWithAesSiv(key: Uint8Array, plaintext: Uint8Array): Promise { + const aead = await aesSivFromRawKey(key) + return aead.encrypt(plaintext, Cryptor.EMPTY_AAD) + } + + async decryptWithAesSiv(key: Uint8Array, ciphertext: Uint8Array): Promise { + const aead = await aesSivFromRawKey(key) + return aead.decrypt(ciphertext, Cryptor.EMPTY_AAD) + } + + async encryptWithAesGcm(key: Uint8Array, plaintext: Uint8Array): Promise { + const aead = await aesGcmFromRawKey(key) + return aead.encrypt(plaintext, Cryptor.EMPTY_AAD) + } + + async decryptWithAesGcm(key: Uint8Array, ciphertext: Uint8Array): Promise { + const aead = await aesGcmFromRawKey(key) + return aead.decrypt(ciphertext, Cryptor.EMPTY_AAD) + } +} + +export class FieldEncryptionExecutorTransform implements FieldTransform { + private executor: FieldEncryptionExecutor + private cryptor: Cryptor + private kekName: string + private kek: Kek | null = null + private dekExpiryDays: number + + constructor( + executor: FieldEncryptionExecutor, + cryptor: Cryptor, + kekName: string, + dekExpiryDays: number, + ) { + this.executor = executor + this.cryptor = cryptor + this.kekName = kekName + this.dekExpiryDays = dekExpiryDays + } + + isDekRotated() { + return this.dekExpiryDays > 0 + } + + async getKek(ctx: RuleContext) { + if (this.kek == null) { + this.kek = await this.getOrCreateKek(ctx) + } + return this.kek + } + + async getOrCreateKek(ctx: RuleContext): Promise { + const isRead = ctx.ruleMode === RuleMode.READ + const kmsType = ctx.getParameter(ENCRYPT_KMS_TYPE) + const kmsKeyId = ctx.getParameter(ENCRYPT_KMS_KEY_ID) + const kekId: KekId = { + name: this.kekName, + deleted: false, + } + let kek = await this.retrieveKekFromRegistry(kekId) + if (kek == null) { + if (isRead) { + throw new RuleError(`no kek found for ${this.kekName} during consume`) + } + if (kmsType == null || kmsType.length === 0) { + throw new RuleError(`no kms type found for ${this.kekName} during produce`) + } + if (kmsKeyId == null || kmsKeyId.length === 0) { + throw new RuleError(`no kms key id found for ${this.kekName} during produce`) + } + kek = await this.storeKekToRegistry(kekId, kmsType, kmsKeyId, false) + if (kek == null) { + // handle conflicts (409) + kek = await this.retrieveKekFromRegistry(kekId) + } + if (kek == null) { + throw new RuleError(`no kek found for ${this.kekName} during produce`) + } + } + if (kmsType != null && kmsType.length !== 0 && kmsType !== kek.kmsType) { + throw new RuleError( + `found ${this.kekName} with kms type ${kek.kmsType} which differs from rule kms type ${kmsType}`, + ) + } + if (kmsKeyId != null && kmsKeyId.length !== 0 && kmsKeyId !== kek.kmsKeyId) { + throw new RuleError( + `found ${this.kekName} with kms key id ${kek.kmsKeyId} which differs from rule kms keyId ${kmsKeyId}`, + ) + } + return kek + } + + async retrieveKekFromRegistry(key: KekId): Promise { + try { + return await this.executor.client!.getKek(key.name, key.deleted) + } catch (err) { + if (err instanceof RestError && err.status === 404) { + return null + } + throw new RuleError(`could not get kek ${key.name}: ${err}`) + } + } + + async storeKekToRegistry(key: KekId, kmsType: string, kmsKeyId: string, shared: boolean): Promise { + try { + return await this.executor.client!.registerKek(key.name, kmsType, kmsKeyId, shared) + } catch (err) { + if (err instanceof RestError && err.status === 409) { + return null + } + throw new RuleError(`could not register kek ${key.name}: ${err}`) + } + } + + async getOrCreateDek(ctx: RuleContext, version: number | null): Promise { + const kek = await this.getKek(ctx) + const isRead = ctx.ruleMode === RuleMode.READ + if (version == null || version === 0) { + version = 1 + } + const dekId: DekId = { + kekName: this.kekName, + subject: ctx.subject, + version, + algorithm: this.cryptor.dekFormat, + deleted: isRead + } + let dek = await this.retrieveDekFromRegistry(dekId) + const isExpired = this.isExpired(ctx, dek) + let kmsClient: KmsClient | null = null + if (dek == null || isExpired) { + if (isRead) { + throw new RuleError(`no dek found for ${this.kekName} during consume`) + } + let encryptedDek: Buffer | null = null + if (!kek.shared) { + kmsClient = getKmsClient(this.executor.config!, kek) + // Generate new dek + const rawDek = this.cryptor.generateKey() + encryptedDek = await kmsClient.encrypt(rawDek) + } + const newVersion = isExpired ? dek!.version! + 1 : null + const newDekId: DekId = { + kekName: this.kekName, + subject: ctx.subject, + version: newVersion, + algorithm: this.cryptor.dekFormat, + deleted: isRead, + } + // encryptedDek may be passed as null if kek is shared + dek = await this.storeDekToRegistry(newDekId, encryptedDek) + if (dek == null) { + // handle conflicts (409) + dek = await this.retrieveDekFromRegistry(dekId) + } + if (dek == null) { + throw new RuleError(`no dek found for ${this.kekName} during produce`) + } + } + + if (DekRegistryClient.getKeyMaterialBytes(dek) == null) { + if (kmsClient == null) { + kmsClient = getKmsClient(this.executor.config!, kek) + } + const rawDek = await kmsClient.decrypt(DekRegistryClient.getEncryptedKeyMaterialBytes(dek)!) + DekRegistryClient.setKeyMaterial(dek, rawDek) + } + + return dek + } + + async retrieveDekFromRegistry(key: DekId): Promise { + try { + let dek: Dek + let version = key.version + if (version == null || version === 0) { + version = 1 + } + dek = await this.executor.client!.getDek(key.kekName, key.subject, key.algorithm, version, key.deleted) + return dek != null && dek.encryptedKeyMaterial != null ? dek : null + } catch (err) { + if (err instanceof RestError && err.status === 404) { + return null + } + throw new RuleError(`could not get dek for kek ${key.kekName}, subject ${key.subject}: ${err}`) + } + } + + async storeDekToRegistry(key: DekId, encryptedDek: Buffer | null): Promise { + try { + let dek: Dek + let encryptedDekStr: string | undefined = undefined + if (encryptedDek != null) { + encryptedDekStr = encryptedDek.toString('base64') + } + let version = key.version + if (version == null || version === 0) { + version = 1 + } + dek = await this.executor.client!.registerDek(key.kekName, key.subject, key.algorithm, version, encryptedDekStr) + return dek + } catch (err) { + if (err instanceof RestError && err.status === 409) { + return null + } + throw new RuleError(`could not register dek for kek ${key.kekName}, subject ${key.subject}: ${err}`) + } + } + + isExpired(ctx: RuleContext, dek: Dek | null): boolean { + const now = Date.now() + return ctx.ruleMode !== RuleMode.READ && + this.dekExpiryDays > 0 && + dek != null && + (now - dek.ts!) / MILLIS_IN_DAY >= this.dekExpiryDays + } + + async transform(ctx: RuleContext, fieldCtx: FieldContext, fieldValue: any): Promise { + if (fieldValue == null) { + return null + } + switch (ctx.ruleMode) { + case RuleMode.WRITE: { + let plaintext = this.toBytes(fieldCtx.type, fieldValue) + if (plaintext == null) { + throw new RuleError(`type ${fieldCtx.type} not supported for encryption`) + } + let version: number | null = null + if (this.isDekRotated()) { + version = -1 + } + let dek = await this.getOrCreateDek(ctx, version) + let keyMaterialBytes = DekRegistryClient.getKeyMaterialBytes(dek)! + let ciphertext = await this.cryptor.encrypt(keyMaterialBytes, plaintext) + if (this.isDekRotated()) { + ciphertext = this.prefixVersion(dek.version!, ciphertext) + } + if (fieldCtx.type === FieldType.STRING) { + return ciphertext.toString('base64') + } else { + return this.toObject(fieldCtx.type, ciphertext) + } + } + case RuleMode.READ: { + let ciphertext + if (fieldCtx.type === FieldType.STRING) { + ciphertext = Buffer.from(fieldValue, 'base64') + } else { + ciphertext = this.toBytes(fieldCtx.type, fieldValue) + } + if (ciphertext == null) { + return fieldValue + } + let version: number | null = null + if (this.isDekRotated()) { + version = this.extractVersion(ciphertext) + if (version == null) { + throw new RuleError('no version found in ciphertext') + } + ciphertext = ciphertext.subarray(5) + } + let dek = await this.getOrCreateDek(ctx, version) + let keyMaterialBytes = DekRegistryClient.getKeyMaterialBytes(dek)! + let plaintext = await this.cryptor.decrypt(keyMaterialBytes, ciphertext) + return this.toObject(fieldCtx.type, plaintext) + } + default: + throw new RuleError(`unsupported rule mode ${ctx.ruleMode}`) + } + } + + prefixVersion(version: number, ciphertext: Buffer): Buffer { + const versionBuf = Buffer.alloc(4) + versionBuf.writeInt32BE(version) + return Buffer.concat([MAGIC_BYTE, versionBuf, ciphertext]) + } + + extractVersion(ciphertext: Buffer): number | null { + let magicByte = ciphertext.subarray(0, 1) + if (!magicByte.equals(MAGIC_BYTE)) { + throw new RuleError( + `Message encoded with magic byte ${JSON.stringify(magicByte)}, expected ${JSON.stringify( + MAGIC_BYTE, + )}`, + ) + } + return ciphertext.subarray(1, 5).readInt32BE(0) + } + + toBytes(type: FieldType, value: any): Buffer | null { + switch (type) { + case FieldType.BYTES: + return value as Buffer + case FieldType.STRING: + return Buffer.from(value as string) + default: + return null + } + } + + toObject(type: FieldType, value: Buffer): any { + switch (type) { + case FieldType.BYTES: + return value + case FieldType.STRING: + return value.toString() + default: + return null + } + } +} + +function getKmsClient(config: Map, kek: Kek): KmsClient { + let keyUrl = kek.kmsType + '://' + kek.kmsKeyId + let kmsClient = Registry.getKmsClient(keyUrl) + if (kmsClient == null) { + let kmsDriver = Registry.getKmsDriver(keyUrl) + kmsClient = kmsDriver.newKmsClient(config, keyUrl) + Registry.registerKmsClient(kmsClient) + } + return kmsClient +} diff --git a/schemaregistry/rules/encryption/kms-registry.ts b/schemaregistry/rules/encryption/kms-registry.ts new file mode 100644 index 00000000..e37b6f69 --- /dev/null +++ b/schemaregistry/rules/encryption/kms-registry.ts @@ -0,0 +1,44 @@ +import {SecurityException} from "./tink/exception/security_exception"; + +export interface KmsDriver { + getKeyUrlPrefix(): string + newKmsClient(config: Map, keyUrl: string): KmsClient +} + +export interface KmsClient { + supported(keyUri: string): boolean + encrypt(plaintext: Buffer): Promise + decrypt(ciphertext: Buffer): Promise +} + +const kmsDrivers: KmsDriver[] = [] + +const kmsClients: KmsClient[] = [] + + +export function registerKmsDriver(kmsDriver: KmsDriver): void { + kmsDrivers.push(kmsDriver) +} + +export function getKmsDriver(keyUrl: string): KmsDriver { + for (let driver of kmsDrivers) { + if (keyUrl.startsWith(driver.getKeyUrlPrefix())) { + return driver + } + } + throw new SecurityException('no KMS driver found for key URL: ' + keyUrl) +} + +export function registerKmsClient(kmsClient: KmsClient): void { + kmsClients.push(kmsClient) +} + +export function getKmsClient(keyUrl: string): KmsClient | null { + for (let client of kmsClients) { + if (client.supported(keyUrl)) { + return client + } + } + return null +} + diff --git a/schemaregistry/rules/encryption/tink/aead.ts b/schemaregistry/rules/encryption/tink/aead.ts new file mode 100644 index 00000000..75754a7e --- /dev/null +++ b/schemaregistry/rules/encryption/tink/aead.ts @@ -0,0 +1,52 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + + +/** + * Interface for Authenticated Encryption with Associated Data (AEAD). + * + * Security guarantees: Implementations of this interface are secure against + * adaptive chosen ciphertext attacks. Encryption with associated data ensures + * authenticity (who the sender is) and integrity (the data has not been + * tampered with) of that data, but not its secrecy. + * + * @see https://tools.ietf.org/html/rfc5116 + */ +export abstract class Aead { + /** + * Encrypts `plaintext` with `opt_associatedData` as associated authenticated + * data. The resulting ciphertext allows for checking authenticity and + * integrity of associated data, but does not guarantee its secrecy. + * + * @param plaintext the plaintext to be encrypted. It must be + * non-null, but can also be an empty (zero-length) byte array. + * @param opt_associatedData optional associated data to be + * authenticated, but not encrypted. A null value is equivalent to an + * empty (zero-length) byte array. For successful decryption the same + * associated data must be provided along with the ciphertext. + * @return resulting ciphertext + * + */ + abstract encrypt(plaintext: Uint8Array, opt_associatedData?: Uint8Array|null): + Promise; + + /** + * Decrypts ciphertext with associated authenticated data. + * The decryption verifies the authenticity and integrity of the associated + * data, but there are no guarantees wrt. secrecy of that data. + * + * @param ciphertext the ciphertext to be decrypted, must be + * non-null. + * @param opt_associatedData optional associated data to be + * authenticated. A null value is equivalent to an empty (zero-length) + * byte array. For successful decryption the same associated data must be + * provided along with the ciphertext. + * @return resulting plaintext + */ + abstract decrypt( + ciphertext: Uint8Array, + opt_associatedData?: Uint8Array|null): Promise; +} diff --git a/schemaregistry/rules/encryption/tink/aes_gcm.ts b/schemaregistry/rules/encryption/tink/aes_gcm.ts new file mode 100644 index 00000000..d9723abe --- /dev/null +++ b/schemaregistry/rules/encryption/tink/aes_gcm.ts @@ -0,0 +1,107 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + +import {Aead} from './aead'; +import {SecurityException} from './exception/security_exception'; + +import * as Bytes from './bytes'; +import * as Random from './random'; +import * as Validators from './validators'; + +/** + * The only supported IV size. + * + */ +const IV_SIZE_IN_BYTES: number = 12; + +/** + * The only supported tag size. + * + */ +const TAG_SIZE_IN_BITS: number = 128; + +/** + * Implementation of AES-GCM. + * + * @final + */ +export class AesGcm extends Aead { + constructor(private readonly key: CryptoKey) { + super(); + } + + /** + */ + async encrypt(plaintext: Uint8Array, associatedData?: Uint8Array): + Promise { + Validators.requireUint8Array(plaintext); + if (associatedData != null) { + Validators.requireUint8Array(associatedData); + } + const iv = Random.randBytes(IV_SIZE_IN_BYTES); + const alg: AesGcmParams = { + 'name': 'AES-GCM', + 'iv': iv, + 'tagLength': TAG_SIZE_IN_BITS + }; + if (associatedData) { + alg['additionalData'] = associatedData; + } + const ciphertext = + await self.crypto.subtle.encrypt(alg, this.key, plaintext); + return Bytes.concat(iv, new Uint8Array(ciphertext)); + } + + /** + */ + async decrypt(ciphertext: Uint8Array, associatedData?: Uint8Array): + Promise { + Validators.requireUint8Array(ciphertext); + if (ciphertext.length < IV_SIZE_IN_BYTES + TAG_SIZE_IN_BITS / 8) { + throw new SecurityException('ciphertext too short'); + } + if (associatedData != null) { + Validators.requireUint8Array(associatedData); + } + const iv = new Uint8Array(IV_SIZE_IN_BYTES); + iv.set(ciphertext.subarray(0, IV_SIZE_IN_BYTES)); + const alg: AesGcmParams = { + 'name': 'AES-GCM', + 'iv': iv, + 'tagLength': TAG_SIZE_IN_BITS + }; + if (associatedData) { + alg['additionalData'] = associatedData; + } + try { + return new Uint8Array(await self.crypto.subtle.decrypt( + alg, this.key, + new Uint8Array(ciphertext.subarray(IV_SIZE_IN_BYTES)))); + // Preserving old behavior when moving to + // https://www.typescriptlang.org/tsconfig#useUnknownInCatchVariables + // tslint:disable-next-line:no-any + } catch (e: any) { + throw new SecurityException(e.toString()); + } + } +} + +export async function fromRawKey(key: Uint8Array): Promise { + Validators.requireUint8Array(key); + Validators.validateAesKeySize(key.length); + const webCryptoKey = await self.crypto.subtle.importKey( + /* format */ + 'raw', key, + /* keyData */ + {'name': 'AES-GCM', 'length': key.length}, + /* algo */ + false, + /* extractable*/ + ['encrypt', 'decrypt']); + + /* usage */ + return new AesGcm(webCryptoKey); +} diff --git a/schemaregistry/rules/encryption/tink/aes_siv.ts b/schemaregistry/rules/encryption/tink/aes_siv.ts new file mode 100644 index 00000000..5f9521b6 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/aes_siv.ts @@ -0,0 +1,41 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + +import {Aead} from './aead'; + +// @ts-expect-error miscreant does not have types +import {AEAD} from "@hackbg/miscreant-esm"; + +/** + * Implementation of AES-SIV. + * + * @final + */ +export class AesSiv extends Aead { + constructor(private readonly key: Uint8Array) { + super(); + } + + /** + */ + async encrypt(plaintext: Uint8Array, associatedData?: Uint8Array): + Promise { + let key = await AEAD.importKey(this.key, "AES-CMAC-SIV"); + return key.seal(plaintext, null, associatedData); + } + + /** + */ + async decrypt(ciphertext: Uint8Array, associatedData?: Uint8Array): + Promise { + let key = await AEAD.importKey(this.key, "AES-CMAC-SIV"); + return key.open(ciphertext, null, associatedData); + } +} + +export async function fromRawKey(key: Uint8Array): Promise { + return new AesSiv(key); +} diff --git a/schemaregistry/rules/encryption/tink/bytes.ts b/schemaregistry/rules/encryption/tink/bytes.ts new file mode 100644 index 00000000..49c568c5 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/bytes.ts @@ -0,0 +1,186 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + +import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; + +/** + * Does near constant time byte array comparison. + * @param ba1 The first bytearray to check. + * @param ba2 The second bytearray to check. + * @return If the array are equal. + */ +export function isEqual(ba1: Uint8Array, ba2: Uint8Array): boolean { + if (ba1.length !== ba2.length) { + return false; + } + let result = 0; + for (let i = 0; i < ba1.length; i++) { + result |= ba1[i] ^ ba2[i]; + } + return result == 0; +} + +/** + * Returns a new array that is the result of joining the arguments. + */ +export function concat(...var_args: Uint8Array[]): Uint8Array { + let length = 0; + for (let i = 0; i < arguments.length; i++) { + // eslint-disable-next-line prefer-rest-params + length += arguments[i].length; + } + const result = new Uint8Array(length); + let curOffset = 0; + for (let i = 0; i < arguments.length; i++) { + // eslint-disable-next-line prefer-rest-params + result.set(arguments[i], curOffset); + // eslint-disable-next-line prefer-rest-params + curOffset += arguments[i].length; + } + return result; +} + +/** + * Converts a non-negative integer number to a 64-bit big-endian byte array. + * @param value The number to convert. + * @return The number as a big-endian byte array. + * @throws {InvalidArgumentsException} + * @static + */ +export function fromNumber(value: number): Uint8Array { + if (Number.isNaN(value) || value % 1 !== 0) { + throw new InvalidArgumentsException('cannot convert non-integer value'); + } + if (value < 0) { + throw new InvalidArgumentsException('cannot convert negative number'); + } + if (value > Number.MAX_SAFE_INTEGER) { + throw new InvalidArgumentsException( + 'cannot convert number larger than ' + Number.MAX_SAFE_INTEGER); + } + const twoPower32 = 2 ** 32; + let low = value % twoPower32; + let high = value / twoPower32; + const result = new Uint8Array(8); + for (let i = 7; i >= 4; i--) { + result[i] = low & 255; + low >>>= 8; + } + for (let i = 3; i >= 0; i--) { + result[i] = high & 255; + high >>>= 8; + } + return result; +} + +/** + * Converts the hex string to a byte array. + * + * @param hex the input + * @return the byte array output + * @throws {!InvalidArgumentsException} + * @static + */ +export function fromHex(hex: string): Uint8Array { + if (hex.length % 2 != 0) { + throw new InvalidArgumentsException( + 'Hex string length must be multiple of 2'); + } + const arr = new Uint8Array(hex.length / 2); + for (let i = 0; i < hex.length; i += 2) { + arr[i / 2] = parseInt(hex.substring(i, i + 2), 16); + } + return arr; +} + +/** + * Converts a byte array to hex. + * + * @param bytes the byte array input + * @return hex the output + * @static + */ +export function toHex(bytes: Uint8Array): string { + let result = ''; + for (let i = 0; i < bytes.length; i++) { + const hexByte = bytes[i].toString(16); + result += hexByte.length > 1 ? hexByte : '0' + hexByte; + } + return result; +} + +/** + * Converts the Base64 string to a byte array. + * + * @param encoded the base64 string + * @param opt_webSafe True indicates we should use the alternative + * alphabet, which does not require escaping for use in URLs. + * @return the byte array output + * @static + */ +export function fromBase64(encoded: string, opt_webSafe?: boolean): Uint8Array { + if (opt_webSafe) { + const normalBase64 = encoded.replace(/-/g, '+').replace(/_/g, '/'); + return fromByteString(window.atob(normalBase64)); + } + return fromByteString(window.atob(encoded)); +} + +/** + * Base64 encode a byte array. + * + * @param bytes the byte array input + * @param opt_webSafe True indicates we should use the alternative + * alphabet, which does not require escaping for use in URLs. + * @return base64 output + * @static + */ +export function toBase64(bytes: Uint8Array, opt_webSafe?: boolean): string { + const encoded = window + .btoa( + /* padding */ + toByteString(bytes)) + .replace(/=/g, ''); + if (opt_webSafe) { + return encoded.replace(/\+/g, '-').replace(/\//g, '_'); + } + return encoded; +} + +/** + * Converts a byte string to a byte array. Only support ASCII and Latin-1 + * strings, does not support multi-byte characters. + * + * @param str the input + * @return the byte array output + * @static + */ +export function fromByteString(str: string): Uint8Array { + const output = []; + let p = 0; + for (let i = 0; i < str.length; i++) { + const c = str.charCodeAt(i); + output[p++] = c; + } + return new Uint8Array(output); +} + +/** + * Turns a byte array into the string given by the concatenation of the + * characters to which the numbers correspond. Each byte is corresponding to a + * character. Does not support multi-byte characters. + * + * @param bytes Array of numbers representing + * characters. + * @return Stringification of the array. + */ +export function toByteString(bytes: Uint8Array): string { + let str = ''; + for (let i = 0; i < bytes.length; i += 1) { + str += String.fromCharCode(bytes[i]); + } + return str; +} diff --git a/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts b/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts new file mode 100644 index 00000000..83edc8e0 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts @@ -0,0 +1,17 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + + +/** + * Exception used when a function receives an invalid argument. + */ +export class InvalidArgumentsException extends Error { + constructor(message?: string) { + super(message); + Object.setPrototypeOf(this, InvalidArgumentsException.prototype); + } +} +InvalidArgumentsException.prototype.name = 'InvalidArgumentsException'; diff --git a/schemaregistry/rules/encryption/tink/exception/security_exception.ts b/schemaregistry/rules/encryption/tink/exception/security_exception.ts new file mode 100644 index 00000000..e4c7c5c3 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/exception/security_exception.ts @@ -0,0 +1,17 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + + +/** + * The base class for all security exceptions. + */ +export class SecurityException extends Error { + constructor(message?: string) { + super(message); + Object.setPrototypeOf(this, SecurityException.prototype); + } +} +SecurityException.prototype.name = 'SecurityException'; diff --git a/schemaregistry/rules/encryption/tink/proto/aes_gcm_pb.ts b/schemaregistry/rules/encryption/tink/proto/aes_gcm_pb.ts new file mode 100644 index 00000000..f774cf14 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/proto/aes_gcm_pb.ts @@ -0,0 +1,74 @@ +// Copyright 2017 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +//////////////////////////////////////////////////////////////////////////////// + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file rules/encryption/tink/proto/aes_gcm.proto (package google.crypto.tink, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file rules/encryption/tink/proto/aes_gcm.proto. + */ +export const file_rules_encryption_tink_proto_aes_gcm: GenFile = /*@__PURE__*/ + fileDesc("CilydWxlcy9lbmNyeXB0aW9uL3RpbmsvcHJvdG8vYWVzX2djbS5wcm90bxISZ29vZ2xlLmNyeXB0by50aW5rIjQKD0Flc0djbUtleUZvcm1hdBIQCghrZXlfc2l6ZRgCIAEoDRIPCgd2ZXJzaW9uGAMgASgNIi8KCUFlc0djbUtleRIPCgd2ZXJzaW9uGAEgASgNEhEKCWtleV92YWx1ZRgDIAEoDEJYChxjb20uZ29vZ2xlLmNyeXB0by50aW5rLnByb3RvUAFaLWdpdGh1Yi5jb20vZ29vZ2xlL3RpbmsvcHJvdG8vYWVzX2djbV9nb19wcm90b6ICBlRJTktQQmIGcHJvdG8z"); + +/** + * @generated from message google.crypto.tink.AesGcmKeyFormat + */ +export type AesGcmKeyFormat = Message<"google.crypto.tink.AesGcmKeyFormat"> & { + /** + * @generated from field: uint32 key_size = 2; + */ + keySize: number; + + /** + * @generated from field: uint32 version = 3; + */ + version: number; +}; + +/** + * Describes the message google.crypto.tink.AesGcmKeyFormat. + * Use `create(AesGcmKeyFormatSchema)` to create a new message. + */ +export const AesGcmKeyFormatSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_rules_encryption_tink_proto_aes_gcm, 0); + +/** + * @generated from message google.crypto.tink.AesGcmKey + */ +export type AesGcmKey = Message<"google.crypto.tink.AesGcmKey"> & { + /** + * @generated from field: uint32 version = 1; + */ + version: number; + + /** + * @generated from field: bytes key_value = 3; + */ + keyValue: Uint8Array; +}; + +/** + * Describes the message google.crypto.tink.AesGcmKey. + * Use `create(AesGcmKeySchema)` to create a new message. + */ +export const AesGcmKeySchema: GenMessage = /*@__PURE__*/ + messageDesc(file_rules_encryption_tink_proto_aes_gcm, 1); + diff --git a/schemaregistry/rules/encryption/tink/proto/aes_siv_pb.ts b/schemaregistry/rules/encryption/tink/proto/aes_siv_pb.ts new file mode 100644 index 00000000..95d871bb --- /dev/null +++ b/schemaregistry/rules/encryption/tink/proto/aes_siv_pb.ts @@ -0,0 +1,80 @@ +// Copyright 2017 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +//////////////////////////////////////////////////////////////////////////////// + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file rules/encryption/tink/proto/aes_siv.proto (package google.crypto.tink, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file rules/encryption/tink/proto/aes_siv.proto. + */ +export const file_rules_encryption_tink_proto_aes_siv: GenFile = /*@__PURE__*/ + fileDesc("CilydWxlcy9lbmNyeXB0aW9uL3RpbmsvcHJvdG8vYWVzX3Npdi5wcm90bxISZ29vZ2xlLmNyeXB0by50aW5rIjQKD0Flc1NpdktleUZvcm1hdBIQCghrZXlfc2l6ZRgBIAEoDRIPCgd2ZXJzaW9uGAIgASgNIi8KCUFlc1NpdktleRIPCgd2ZXJzaW9uGAEgASgNEhEKCWtleV92YWx1ZRgCIAEoDEJPChxjb20uZ29vZ2xlLmNyeXB0by50aW5rLnByb3RvUAFaLWdpdGh1Yi5jb20vZ29vZ2xlL3RpbmsvcHJvdG8vYWVzX3Npdl9nb19wcm90b2IGcHJvdG8z"); + +/** + * @generated from message google.crypto.tink.AesSivKeyFormat + */ +export type AesSivKeyFormat = Message<"google.crypto.tink.AesSivKeyFormat"> & { + /** + * Only valid value is: 64. + * + * @generated from field: uint32 key_size = 1; + */ + keySize: number; + + /** + * @generated from field: uint32 version = 2; + */ + version: number; +}; + +/** + * Describes the message google.crypto.tink.AesSivKeyFormat. + * Use `create(AesSivKeyFormatSchema)` to create a new message. + */ +export const AesSivKeyFormatSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_rules_encryption_tink_proto_aes_siv, 0); + +/** + * key_type: type.googleapis.com/google.crypto.tink.AesSivKey + * + * @generated from message google.crypto.tink.AesSivKey + */ +export type AesSivKey = Message<"google.crypto.tink.AesSivKey"> & { + /** + * @generated from field: uint32 version = 1; + */ + version: number; + + /** + * First half is AES-CTR key, second is AES-SIV. + * + * @generated from field: bytes key_value = 2; + */ + keyValue: Uint8Array; +}; + +/** + * Describes the message google.crypto.tink.AesSivKey. + * Use `create(AesSivKeySchema)` to create a new message. + */ +export const AesSivKeySchema: GenMessage = /*@__PURE__*/ + messageDesc(file_rules_encryption_tink_proto_aes_siv, 1); + diff --git a/schemaregistry/rules/encryption/tink/random.ts b/schemaregistry/rules/encryption/tink/random.ts new file mode 100644 index 00000000..89315cf2 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/random.ts @@ -0,0 +1,27 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + + +/** + * @fileoverview Several simple wrappers of crypto.getRandomValues. + */ +import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; + +/** + * Randomly generates `n` bytes. + * + * @param n number of bytes to generate + * @return the random bytes + * @static + */ +export function randBytes(n: number): Uint8Array { + if (!Number.isInteger(n) || n < 0) { + throw new InvalidArgumentsException('n must be a nonnegative integer'); + } + const result = new Uint8Array(n); + crypto.getRandomValues(result); + return result; +} diff --git a/schemaregistry/rules/encryption/tink/validators.ts b/schemaregistry/rules/encryption/tink/validators.ts new file mode 100644 index 00000000..3f952fc0 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/validators.ts @@ -0,0 +1,83 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + +import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; +import {SecurityException} from './exception/security_exception'; +const SUPPORTED_AES_KEY_SIZES: number[] = [16, 32]; + +/** + * Validates AES key sizes, at the moment only 128-bit and 256-bit keys are + * supported. + * + * @param n the key size in bytes + * @throws {!InvalidArgumentsException} + * @static + */ +export function validateAesKeySize(n: number) { + if (!SUPPORTED_AES_KEY_SIZES.includes(n)) { + throw new InvalidArgumentsException('unsupported AES key size: ' + n); + } +} + +/** + * Validates that the input is a non null Uint8Array. + * + * @throws {!InvalidArgumentsException} + * @static + */ +export function requireUint8Array(input: Uint8Array) { + if (input == null || !(input instanceof Uint8Array)) { + throw new InvalidArgumentsException('input must be a non null Uint8Array'); + } +} + +/** + * Validates version, throws exception if candidate version is negative or + * bigger than expected. + * + * @param candidate - version to be validated + * @param maxVersion - upper bound on version + * @throws {!SecurityException} + * @static + */ +export function validateVersion(candidate: number, maxVersion: number) { + if (candidate < 0 || candidate > maxVersion) { + throw new SecurityException( + 'Version is out of bound, must be ' + + 'between 0 and ' + maxVersion + '.'); + } +} + +/** + * Validates ECDSA parameters. + * + * @throws {!SecurityException} + */ +export function validateEcdsaParams(curve: string, hash: string) { + switch (curve) { + case 'P-256': + if (hash != 'SHA-256') { + throw new SecurityException( + 'expected SHA-256 (because curve is P-256) but got ' + hash); + } + break; + case 'P-384': + if (hash != 'SHA-384' && hash != 'SHA-512') { + throw new SecurityException( + 'expected SHA-384 or SHA-512 (because curve is P-384) but got ' + + hash); + } + break; + case 'P-521': + if (hash != 'SHA-512') { + throw new SecurityException( + 'expected SHA-512 (because curve is P-521) but got ' + hash); + } + break; + default: + throw new SecurityException('unsupported curve: ' + curve); + } +} diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index bb0a25e3..2e0145a5 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -13,59 +13,76 @@ import { Mutex } from 'async-mutex'; * of the MIT license. See the LICENSE.txt file for details. */ -enum Compatibility { - None = "NONE", - Backward = "BACKWARD", - Forward = "FORWARD", - Full = "FULL", - BackwardTransitive = "BACKWARD_TRANSITIVE", - ForwardTransitive = "FORWARD_TRANSITIVE", - FullTransitive = "FULL_TRANSITIVE" +export enum Compatibility { + NONE = "NONE", + BACKWARD = "BACKWARD", + FORWARD = "FORWARD", + FULL = "FULL", + BACKWARD_TRANSITIVE = "BACKWARD_TRANSITIVE", + FORWARD_TRANSITIVE = "FORWARD_TRANSITIVE", + FULL_TRANSITIVE = "FULL_TRANSITIVE" } -interface CompatibilityLevel { +export interface CompatibilityLevel { compatibility?: Compatibility; compatibilityLevel?: Compatibility; } -interface Rule { - name: string; - subject: string; - version: number; +export interface Rule { + name: string + doc?: string + kind?: string + mode?: RuleMode + type: string + tags?: Set + params?: { [key: string]: string } + expr?: string + onSuccess?: string + onFailure?: string + disabled?: boolean } -interface SchemaInfo { - schema?: string; +export enum RuleMode { + UPGRADE = 'UPGRADE', + DOWNGRADE = 'DOWNGRADE', + UPDOWN = 'UPDOWN', + WRITE = 'WRITE', + READ = 'READ', + WRITEREAD = 'WRITEREAD', +} + +export interface SchemaInfo { + schema: string; schemaType?: string; references?: Reference[]; metadata?: Metadata; ruleSet?: RuleSet; } -interface SchemaMetadata extends SchemaInfo { +export interface SchemaMetadata extends SchemaInfo { id: number; subject?: string; version?: number; } -interface Reference { - Name: string; - Subject: string; - Version: number; +export interface Reference { + name: string; + subject: string; + version: number; } -interface Metadata { - tags?: { [key: string]: string[] }; +export interface Metadata { + tags?: { [key: string]: Set }; properties?: { [key: string]: string }; - sensitive?: string[]; + sensitive?: Set; } -interface RuleSet { - migrationRules: Rule[]; - compatibilityRules: Rule[]; +export interface RuleSet { + migrationRules?: Rule[]; + domainRules?: Rule[]; } -interface ServerConfig { +export interface ServerConfig { alias?: string; normalize?: boolean; compatibility?: Compatibility; @@ -77,11 +94,12 @@ interface ServerConfig { overrideRuleSet?: RuleSet; } -interface isCompatibleResponse { +export interface isCompatibleResponse { is_compatible: boolean; } -interface Client { +export interface Client { + config(): ClientConfig; register(subject: string, schema: SchemaInfo, normalize: boolean): Promise; registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean): Promise; getBySubjectAndId(subject: string, id: number): Promise; @@ -107,7 +125,8 @@ interface Client { close(): void; } -class SchemaRegistryClient implements Client { +export class SchemaRegistryClient implements Client { + private clientConfig: ClientConfig; private restService: RestService; private schemaToIdCache: LRUCache; @@ -127,12 +146,13 @@ class SchemaRegistryClient implements Client { private metadataToSchemaMutex: Mutex; constructor(config: ClientConfig) { + this.clientConfig = config const cacheOptions = { max: config.cacheCapacity, ...(config.cacheLatestTtlSecs !== undefined && { maxAge: config.cacheLatestTtlSecs * 1000 }) }; - this.restService = new RestService(config.createAxiosDefaults, config.baseURLs, config.isForward); + this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults); this.schemaToIdCache = new LRUCache(cacheOptions); this.idToSchemaInfoCache = new LRUCache(cacheOptions); @@ -150,13 +170,17 @@ class SchemaRegistryClient implements Client { this.metadataToSchemaMutex = new Mutex(); } - public async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + config(): ClientConfig { + return this.clientConfig + } + + async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const metadataResult = await this.registerFullResponse(subject, schema, normalize); return metadataResult.id; } - public async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema }); return await this.infoToSchemaMutex.runExclusive(async () => { @@ -177,7 +201,7 @@ class SchemaRegistryClient implements Client { }); } - public async getBySubjectAndId(subject: string, id: number): Promise { + async getBySubjectAndId(subject: string, id: number): Promise { const cacheKey = stringify({ subject, id }); return await this.idToSchemaInfoMutex.runExclusive(async () => { const cachedSchema: SchemaInfo | undefined = this.idToSchemaInfoCache.get(cacheKey); @@ -196,7 +220,7 @@ class SchemaRegistryClient implements Client { }); } - public async getId(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + async getId(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema }); return await this.schemaToIdMutex.runExclusive(async () => { @@ -217,7 +241,7 @@ class SchemaRegistryClient implements Client { }); } - public async getLatestSchemaMetadata(subject: string): Promise { + async getLatestSchemaMetadata(subject: string): Promise { return await this.latestToSchemaMutex.runExclusive(async () => { const cachedSchema: SchemaMetadata | undefined = this.latestToSchemaCache.get(subject); if (cachedSchema) { @@ -235,7 +259,7 @@ class SchemaRegistryClient implements Client { }); } - public async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { + async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { const cacheKey = stringify({ subject, version, deleted }); return await this.versionToSchemaMutex.runExclusive(async () => { @@ -255,7 +279,7 @@ class SchemaRegistryClient implements Client { }); } - public async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { + async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { const cacheKey = stringify({ subject, metadata, deleted }); return await this.metadataToSchemaMutex.runExclusive(async () => { @@ -284,7 +308,7 @@ class SchemaRegistryClient implements Client { } - public async getAllVersions(subject: string): Promise { + async getAllVersions(subject: string): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions`, 'GET' @@ -292,7 +316,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { + async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema }); return await this.schemaToVersionMutex.runExclusive(async () => { @@ -313,7 +337,7 @@ class SchemaRegistryClient implements Client { }); } - public async getAllSubjects(): Promise { + async getAllSubjects(): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/subjects`, 'GET' @@ -321,7 +345,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public async deleteSubject(subject: string, permanent: boolean = false): Promise { + async deleteSubject(subject: string, permanent: boolean = false): Promise { await this.infoToSchemaMutex.runExclusive(async () => { this.infoToSchemaCache.forEach((_, key) => { const parsedKey = JSON.parse(key); @@ -367,7 +391,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { + async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { return await this.schemaToVersionMutex.runExclusive(async () => { let metadataValue: SchemaMetadata | undefined; @@ -406,7 +430,7 @@ class SchemaRegistryClient implements Client { }); } - public async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { + async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { subject = encodeURIComponent(subject); const response: AxiosResponse = await this.restService.handleRequest( @@ -417,7 +441,7 @@ class SchemaRegistryClient implements Client { return response.data.is_compatible; } - public async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { + async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { subject = encodeURIComponent(subject); const response: AxiosResponse = await this.restService.handleRequest( @@ -428,7 +452,7 @@ class SchemaRegistryClient implements Client { return response.data.is_compatible; } - public async getCompatibility(subject: string): Promise { + async getCompatibility(subject: string): Promise { subject = encodeURIComponent(subject); const response: AxiosResponse = await this.restService.handleRequest( @@ -438,7 +462,7 @@ class SchemaRegistryClient implements Client { return response.data.compatibilityLevel!; } - public async updateCompatibility(subject: string, update: Compatibility): Promise { + async updateCompatibility(subject: string, update: Compatibility): Promise { subject = encodeURIComponent(subject); const response: AxiosResponse = await this.restService.handleRequest( @@ -449,7 +473,7 @@ class SchemaRegistryClient implements Client { return response.data.compatibility!; } - public async getDefaultCompatibility(): Promise { + async getDefaultCompatibility(): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, 'GET' @@ -457,7 +481,7 @@ class SchemaRegistryClient implements Client { return response.data.compatibilityLevel!; } - public async updateDefaultCompatibility(update: Compatibility): Promise { + async updateDefaultCompatibility(update: Compatibility): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, 'PUT', @@ -466,7 +490,7 @@ class SchemaRegistryClient implements Client { return response.data.compatibility!; } - public async getConfig(subject: string): Promise { + async getConfig(subject: string): Promise { subject = encodeURIComponent(subject); const response: AxiosResponse = await this.restService.handleRequest( @@ -476,7 +500,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public async updateConfig(subject: string, update: ServerConfig): Promise { + async updateConfig(subject: string, update: ServerConfig): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config/${subject}`, 'PUT', @@ -485,7 +509,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public async getDefaultConfig(): Promise { + async getDefaultConfig(): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, 'GET' @@ -493,7 +517,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public async updateDefaultConfig(update: ServerConfig): Promise { + async updateDefaultConfig(update: ServerConfig): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, 'PUT', @@ -502,7 +526,7 @@ class SchemaRegistryClient implements Client { return response.data; } - public close(): void { + close(): void { this.infoToSchemaCache.clear(); this.schemaToVersionCache.clear(); this.versionToSchemaCache.clear(); @@ -512,61 +536,55 @@ class SchemaRegistryClient implements Client { } // Cache methods for testing - public async addToInfoToSchemaCache(subject: string, schema: SchemaInfo, metadata: SchemaMetadata): Promise { + async addToInfoToSchemaCache(subject: string, schema: SchemaInfo, metadata: SchemaMetadata): Promise { const cacheKey = stringify({ subject, schema }); await this.infoToSchemaMutex.runExclusive(async () => { this.infoToSchemaCache.set(cacheKey, metadata); }); } - public async addToSchemaToVersionCache(subject: string, schema: SchemaInfo, version: number): Promise { + async addToSchemaToVersionCache(subject: string, schema: SchemaInfo, version: number): Promise { const cacheKey = stringify({ subject, schema }); await this.schemaToVersionMutex.runExclusive(async () => { this.schemaToVersionCache.set(cacheKey, version); }); } - public async addToVersionToSchemaCache(subject: string, version: number, metadata: SchemaMetadata): Promise { + async addToVersionToSchemaCache(subject: string, version: number, metadata: SchemaMetadata): Promise { const cacheKey = stringify({ subject, version }); await this.versionToSchemaMutex.runExclusive(async () => { this.versionToSchemaCache.set(cacheKey, metadata); }); } - public async addToIdToSchemaInfoCache(subject: string, id: number, schema: SchemaInfo): Promise { + async addToIdToSchemaInfoCache(subject: string, id: number, schema: SchemaInfo): Promise { const cacheKey = stringify({ subject, id }); await this.idToSchemaInfoMutex.runExclusive(async () => { this.idToSchemaInfoCache.set(cacheKey, schema); }); } - public async getInfoToSchemaCacheSize(): Promise { + async getInfoToSchemaCacheSize(): Promise { return await this.infoToSchemaMutex.runExclusive(async () => { return this.infoToSchemaCache.size; }); } - public async getSchemaToVersionCacheSize(): Promise { + async getSchemaToVersionCacheSize(): Promise { return await this.schemaToVersionMutex.runExclusive(async () => { return this.schemaToVersionCache.size; }); } - public async getVersionToSchemaCacheSize(): Promise { + async getVersionToSchemaCacheSize(): Promise { return await this.versionToSchemaMutex.runExclusive(async () => { return this.versionToSchemaCache.size; }); } - public async getIdToSchemaInfoCacheSize(): Promise { + async getIdToSchemaInfoCacheSize(): Promise { return await this.idToSchemaInfoMutex.runExclusive(async () => { return this.idToSchemaInfoCache.size; }); } - } - -export { - Client, SchemaRegistryClient, SchemaInfo, Metadata, Compatibility, - CompatibilityLevel, ServerConfig, RuleSet, Rule, Reference, SchemaMetadata -}; diff --git a/schemaregistry/serde/avro.ts b/schemaregistry/serde/avro.ts new file mode 100644 index 00000000..d1590607 --- /dev/null +++ b/schemaregistry/serde/avro.ts @@ -0,0 +1,380 @@ +import { + Deserializer, DeserializerConfig, + FieldTransform, + FieldType, Migration, RefResolver, + RuleConditionError, + RuleContext, SerdeType, + Serializer, SerializerConfig +} from "./serde"; +import { + Client, RuleMode, + SchemaInfo +} from "../schemaregistry-client"; +import avro, { ForSchemaOptions, Type, types } from "avsc"; +import UnwrappedUnionType = types.UnwrappedUnionType +import WrappedUnionType = types.WrappedUnionType +import ArrayType = types.ArrayType +import MapType = types.MapType +import RecordType = types.RecordType +import Field = types.Field +import { LRUCache } from 'lru-cache' +import {getRuleExecutors} from "./rule-registry"; +import stringify from "json-stringify-deterministic"; + +type TypeHook = (schema: avro.Schema, opts: ForSchemaOptions) => Type | undefined + +export type AvroSerdeConfig = Partial + +export interface AvroSerde { + schemaToTypeCache: LRUCache +} + +export type AvroSerializerConfig = SerializerConfig & AvroSerdeConfig + +export class AvroSerializer extends Serializer implements AvroSerde { + schemaToTypeCache: LRUCache + + constructor(client: Client, serdeType: SerdeType, conf: AvroSerializerConfig) { + super(client, serdeType, conf) + this.schemaToTypeCache = new LRUCache({ max: this.conf.cacheCapacity ?? 1000 }) + this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { + return await this.fieldTransform(ctx, fieldTransform, msg) + } + for (const rule of getRuleExecutors()) { + rule.configure(client.config(), conf.ruleConfig ?? new Map) + } + } + + async serialize(topic: string, msg: any): Promise { + if (this.client == null) { + throw new Error('client is not initialized') + } + if (msg == null) { + throw new Error('message is empty') + } + + let avroSchema = Type.forValue(msg) + const schema: SchemaInfo = { + schemaType: 'AVRO', + schema: JSON.stringify(avroSchema), + } + const [id, info] = await this.getId(topic, msg, schema) + avroSchema = await this.toType(info) + const subject = this.subjectName(topic, info) + msg = await this.executeRules(subject, topic, RuleMode.WRITE, null, info, msg, getInlineTags(avroSchema)) + const msgBytes = avroSchema.toBuffer(msg) + return this.writeBytes(id, msgBytes) + } + + async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { + const schema = await this.toType(ctx.target) + return await transform(ctx, schema, msg, fieldTransform) + } + + async toType(info: SchemaInfo): Promise { + return toType(this.client, this.conf as AvroDeserializerConfig, this, info, async (client, info) => { + const deps = new Map() + await this.resolveReferences(client, info, deps) + return deps + }) + } +} + +export type AvroDeserializerConfig = DeserializerConfig & AvroSerdeConfig + +export class AvroDeserializer extends Deserializer implements AvroSerde { + schemaToTypeCache: LRUCache + + constructor(client: Client, serdeType: SerdeType, conf: AvroDeserializerConfig) { + super(client, serdeType, conf) + this.schemaToTypeCache = new LRUCache({ max: this.conf.cacheCapacity ?? 1000 }) + this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { + return await this.fieldTransform(ctx, fieldTransform, msg) + } + for (const rule of getRuleExecutors()) { + rule.configure(client.config(), conf.ruleConfig ?? new Map) + } + } + + async deserialize(topic: string, payload: Buffer): Promise { + if (!Buffer.isBuffer(payload)) { + throw new Error('Invalid buffer') + } + if (payload.length === 0) { + return null + } + + const info = await this.getSchema(topic, payload) + const subject = this.subjectName(topic, info) + const readerMeta = await this.getReaderSchema(subject) + let migrations: Migration[] = [] + if (readerMeta != null) { + migrations = await this.getMigrations(subject, info, readerMeta) + } + const writer = await this.toType(info) + + let msg: any + const msgBytes = payload.subarray(5) + if (migrations.length > 0) { + msg = writer.fromBuffer(msgBytes) + msg = await this.executeMigrations(migrations, subject, topic, msg) + } else { + if (readerMeta != null) { + const reader = await this.toType(readerMeta) + if (reader.equals(writer)) { + msg = reader.fromBuffer(msgBytes) + } else { + msg = reader.fromBuffer(msgBytes, reader.createResolver(writer)) + } + } else { + msg = writer.fromBuffer(msgBytes) + } + } + let target: SchemaInfo + if (readerMeta != null) { + target = readerMeta + } else { + target = info + } + msg = await this.executeRules(subject, topic, RuleMode.READ, null, target, msg, getInlineTags(writer)) + return msg + } + + async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { + const schema = await this.toType(ctx.target) + return await transform(ctx, schema, msg, fieldTransform) + } + + async toType(info: SchemaInfo): Promise { + return toType(this.client, this.conf as AvroDeserializerConfig, this, info, async (client, info) => { + const deps = new Map() + await this.resolveReferences(client, info, deps) + return deps + }) + } +} + +async function toType( + client: Client, + conf: AvroSerdeConfig, + serde: AvroSerde, + info: SchemaInfo, + refResolver: RefResolver, +): Promise { + let type = serde.schemaToTypeCache.get(stringify(info.schema)) + if (type != null) { + return type + } + + const deps = await refResolver(client, info) + + const addReferencedSchemas = (userHook?: TypeHook): TypeHook | undefined => ( + schema: avro.Schema, + opts: ForSchemaOptions, + ) => { + deps.forEach((_name, schema) => { + avro.Type.forSchema(JSON.parse(schema), opts) + }) + if (userHook) { + return userHook(schema, opts) + } + return + } + + const avroOpts = conf + type = avro.Type.forSchema(JSON.parse(info.schema), { + ...avroOpts, + typeHook: addReferencedSchemas(avroOpts?.typeHook), + }) + serde.schemaToTypeCache.set(stringify(info.schema), type) + return type +} + +async function transform(ctx: RuleContext, schema: Type, msg: any, fieldTransform: FieldTransform): Promise { + if (msg == null || schema == null) { + return msg + } + const fieldCtx = ctx.currentField() + if (fieldCtx != null) { + fieldCtx.type = getType(schema) + } + switch (schema.typeName) { + case 'union:unwrapped': + case 'union:wrapped': + const subschema = resolveUnion(schema, msg) + if (subschema == null) { + return null + } + return await transform(ctx, subschema, msg, fieldTransform) + case 'array': + const arraySchema = schema as ArrayType + const array = msg as any[] + return await Promise.all(array.map(item => transform(ctx, arraySchema.itemsType, item, fieldTransform))) + case 'map': + const mapSchema = schema as MapType + const map = msg as Map + for (const key of Object.keys(map)) { + map.set(key, await transform(ctx, mapSchema.valuesType, map.get(key), fieldTransform)) + } + return map + case 'record': + const recordSchema = schema as RecordType + const record = msg as Record + for (const field of recordSchema.fields) { + await transformField(ctx, recordSchema, field, record, record[field.name], fieldTransform) + } + return record + default: + if (fieldCtx != null) { + const ruleTags = ctx.rule.tags + if (ruleTags == null || ruleTags.size === 0 || !disjoint(ruleTags, fieldCtx.tags)) { + return await fieldTransform.transform(ctx, fieldCtx, msg) + } + } + return msg + } +} + +async function transformField( + ctx: RuleContext, + recordSchema: RecordType, + field: Field, + record: Record, + val: any, + fieldTransform: FieldTransform, +): Promise { + const fullName = recordSchema.name + '.' + field.name + try { + ctx.enterField( + val.Interface(), + fullName, + field.name, + getType(field.type), + ctx.getInlineTags(fullName), + ) + const newVal = await transform(ctx, field.type, record[field.name], fieldTransform) + if (ctx.rule.kind === 'CONDITION') { + if (!newVal) { + throw new RuleConditionError(ctx.rule) + } + } else { + record[field.name] = newVal + } + } finally { + ctx.leaveField() + } +} + +function getType(schema: Type): FieldType { + switch (schema.typeName) { + case 'record': + return FieldType.RECORD + case 'enum': + return FieldType.ENUM + case 'array': + return FieldType.ARRAY + case 'map': + return FieldType.MAP + case 'union:unwrapped': + case 'union:wrapped': + return FieldType.COMBINED + case 'fixed': + return FieldType.FIXED + case 'string': + return FieldType.STRING + case 'bytes': + return FieldType.BYTES + case 'int': + return FieldType.INT + case 'abstract:long': + case 'long': + return FieldType.LONG + case 'float': + return FieldType.FLOAT + case 'double': + return FieldType.DOUBLE + case 'boolean': + return FieldType.BOOLEAN + case 'null': + return FieldType.NULL + default: + return FieldType.NULL + } +} + +function disjoint(slice1: Set, map1: Set): boolean { + for (const v of slice1) { + if (map1.has(v)) { + return false + } + } + return true +} + +function resolveUnion(schema: Type, msg: any): Type | null { + let unionTypes = null + if (schema.typeName === 'union:unwrapped') { + const union = schema as UnwrappedUnionType + unionTypes = union.types.slice() + } else if (schema.typeName === 'union:wrapped') { + const union = schema as WrappedUnionType + unionTypes = union.types.slice() + } + if (unionTypes != null) { + for (let i = 0; i < unionTypes.length; i++) { + if (unionTypes[i].isValid(msg)) { + return unionTypes[i] + } + } + } + return null +} + +function getInlineTags(schema: object): Map> { + const inlineTags = new Map>() + getInlineTagsRecursively('', '', schema, inlineTags) + return inlineTags +} + +// iterate over the object and get all properties named 'confluent:tags' +function getInlineTagsRecursively(ns: string, name: string, schema: any, tags: Map>): void { + if (schema == null || typeof schema === 'string') { + return + } else if (Array.isArray(schema)) { + for (let i = 0; i < schema.length; i++) { + getInlineTagsRecursively(ns, name, schema[i], tags) + } + } else if (typeof schema === 'object') { + const type = schema['type'] + if (type === 'record') { + let recordNs = schema['namespace'] + let recordName = schema['name'] + if (recordNs === undefined) { + recordNs = impliedNamespace(name) + } + if (recordNs == null) { + recordNs = ns + } + if (recordNs !== '' && !recordName.startsWith(recordNs)) { + recordName = recordNs + '.' + recordName + } + const fields = schema['fields'] + for (const field of fields) { + const fieldTags = field['confluent:tags'] + const fieldName = field['name'] + if (fieldTags !== undefined && fieldName !== undefined) { + tags.set(recordName + '.' + fieldName, new Set(fieldTags)) + } + const fieldType = field['type'] + if (fieldType !== undefined) { + getInlineTagsRecursively(recordNs, recordName, fieldType, tags) + } + } + } + } +} + +function impliedNamespace(name: string): string | null { + const match = /^(.*)\.[^.]+$/.exec(name) + return match ? match[1] : null +} diff --git a/schemaregistry/serde/buffer-wrapper.ts b/schemaregistry/serde/buffer-wrapper.ts new file mode 100644 index 00000000..98f2c1a8 --- /dev/null +++ b/schemaregistry/serde/buffer-wrapper.ts @@ -0,0 +1,63 @@ +export const MAX_VARINT_LEN_16 = 3 +export const MAX_VARINT_LEN_32 = 5 +export const MAX_VARINT_LEN_64 = 10 + +export class BufferWrapper { + buf: Buffer + pos: number + + constructor(buf: Buffer) { + this.buf = buf + this.pos = 0 + } + + // Adapted from avro-js + writeVarInt(n: number): void { + let f, m + + if (n >= -1073741824 && n < 1073741824) { + // Won't overflow, we can use integer arithmetic. + m = n >= 0 ? n << 1 : (~n << 1) | 1 + do { + this.buf[this.pos] = m & 0x7f + m >>= 7 + } while (m && (this.buf[this.pos++] |= 0x80)) + } else { + // We have to use slower floating arithmetic. + f = n >= 0 ? n * 2 : -n * 2 - 1 + do { + this.buf[this.pos] = f & 0x7f + f /= 128 + } while (f >= 1 && (this.buf[this.pos++] |= 0x80)) + } + this.pos++ + } + + // Adapted from avro-js + readVarInt(): number { + let n = 0 + let k = 0 + let b, h, f, fk + + do { + b = this.buf[this.pos++] + h = b & 0x80 + n |= (b & 0x7f) << k + k += 7 + } while (h && k < 28) + + if (h) { + // Switch to float arithmetic, otherwise we might overflow. + f = n + fk = 268435456 // 2 ** 28. + do { + b = this.buf[this.pos++] + f += (b & 0x7f) * fk + fk *= 128 + } while (b & 0x80) + return (f % 2 ? -(f + 1) : f) / 2 + } + + return (n >> 1) ^ -(n & 1) + } +} diff --git a/schemaregistry/serde/json.ts b/schemaregistry/serde/json.ts new file mode 100644 index 00000000..7c53d1db --- /dev/null +++ b/schemaregistry/serde/json.ts @@ -0,0 +1,411 @@ +import { + Deserializer, DeserializerConfig, + FieldTransform, + FieldType, Migration, RefResolver, RuleConditionError, + RuleContext, + SerdeType, SerializationError, + Serializer, SerializerConfig +} from "./serde"; +import { + Client, RuleMode, + SchemaInfo +} from "../schemaregistry-client"; +import Ajv, {ErrorObject} from "ajv"; +import Ajv2019 from "ajv/dist/2019"; +import Ajv2020 from "ajv/dist/2020"; +import * as draft6MetaSchema from 'ajv/dist/refs/json-schema-draft-06.json' +import * as draft7MetaSchema from 'ajv/dist/refs/json-schema-draft-07.json' +import { + DereferencedJSONSchemaDraft07, + DereferencedJSONSchemaDraft2020_12, +} from '@criteria/json-schema' +import { + dereferenceJSONSchema as dereferenceJSONSchemaDraft2020_12, +} from '@criteria/json-schema/draft-2020-12' +import { + dereferenceJSONSchema as dereferenceJSONSchemaDraft07, +} from '@criteria/json-schema/draft-07' +import { validateJSON } from '@criteria/json-schema-validation' +import { LRUCache } from "lru-cache"; +import { generateSchema } from "./json_util"; +import {getRuleExecutors} from "./rule-registry"; +import stringify from "json-stringify-deterministic"; + +export interface ValidateFunction { + (this: any, data: any): boolean + errors?: null | ErrorObject[] +} + +export type DereferencedJSONSchema = DereferencedJSONSchemaDraft07 | DereferencedJSONSchemaDraft2020_12 + +export type JsonSerdeConfig = ConstructorParameters[0] & { + validate?: boolean +} + +export interface JsonSerde { + schemaToTypeCache: LRUCache + schemaToValidateCache: LRUCache +} + +export type JsonSerializerConfig = SerializerConfig & JsonSerdeConfig + +export class JsonSerializer extends Serializer implements JsonSerde { + schemaToTypeCache: LRUCache + schemaToValidateCache: LRUCache + + constructor(client: Client, serdeType: SerdeType, conf: JsonSerializerConfig) { + super(client, serdeType, conf) + this.schemaToTypeCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) + this.schemaToValidateCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) + this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { + return await this.fieldTransform(ctx, fieldTransform, msg) + } + for (const rule of getRuleExecutors()) { + rule.configure(client.config(), conf.ruleConfig ?? new Map) + } + } + + async serialize(topic: string, msg: any): Promise { + if (this.client == null) { + throw new Error('client is not initialized') + } + if (msg == null) { + throw new Error('message is empty') + } + + const jsonSchema = generateSchema(msg) + const schema: SchemaInfo = { + schemaType: 'JSON', + schema: JSON.stringify(jsonSchema), + } + const [id, info] = await this.getId(topic, msg, schema) + const subject = this.subjectName(topic, info) + msg = await this.executeRules(subject, topic, RuleMode.WRITE, null, info, msg, null) + const msgBytes = Buffer.from(JSON.stringify(msg)) + if ((this.conf as JsonSerdeConfig).validate) { + const validate = await this.toValidateFunction(info) + if (validate != null && !validate(msg)) { + throw new SerializationError('Invalid message') + } + } + return this.writeBytes(id, msgBytes) + } + + async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { + const schema = this.toType(ctx.target) + if (typeof schema === 'boolean') { + return msg + } + return await transform(ctx, schema, '$', msg, fieldTransform) + } + + toType(info: SchemaInfo): DereferencedJSONSchema { + return toType(this.client, this.conf as JsonDeserializerConfig, this, info, async (client, info) => { + const deps = new Map() + await this.resolveReferences(client, info, deps) + return deps + }) + } + + async toValidateFunction(info: SchemaInfo): Promise { + return await toValidateFunction(this.client, this.conf as JsonDeserializerConfig, this, info, async (client, info) => { + const deps = new Map() + await this.resolveReferences(client, info, deps) + return deps + }, + ) + } +} + +export type JsonDeserializerConfig = DeserializerConfig & JsonSerdeConfig + +export class JsonDeserializer extends Deserializer implements JsonSerde { + schemaToTypeCache: LRUCache + schemaToValidateCache: LRUCache + + constructor(client: Client, serdeType: SerdeType, conf: JsonDeserializerConfig) { + super(client, serdeType, conf) + this.schemaToTypeCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) + this.schemaToValidateCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) + this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { + return await this.fieldTransform(ctx, fieldTransform, msg) + } + for (const rule of getRuleExecutors()) { + rule.configure(client.config(), conf.ruleConfig ?? new Map) + } + } + + async deserialize(topic: string, payload: Buffer): Promise { + if (!Buffer.isBuffer(payload)) { + throw new Error('Invalid buffer') + } + if (payload.length === 0) { + return null + } + + const info = await this.getSchema(topic, payload) + if ((this.conf as JsonSerdeConfig).validate) { + const validate = await this.toValidateFunction(info) + if (validate != null && !validate(JSON.parse(payload.subarray(5).toString()))) { + throw new SerializationError('Invalid message') + } + + } + const subject = this.subjectName(topic, info) + const readerMeta = await this.getReaderSchema(subject) + let migrations: Migration[] = [] + if (readerMeta != null) { + migrations = await this.getMigrations(subject, info, readerMeta) + } + const msgBytes = payload.subarray(5) + let msg = JSON.parse(msgBytes.toString()) + if (migrations.length > 0) { + msg = await this.executeMigrations(migrations, subject, topic, msg) + } + let target: SchemaInfo + if (readerMeta != null) { + target = readerMeta + } else { + target = info + } + msg = this.executeRules(subject, topic, RuleMode.READ, null, target, msg, null) + return msg + } + + async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { + const schema = this.toType(ctx.target) + return await transform(ctx, schema, '$', msg, fieldTransform) + } + + toType(info: SchemaInfo): DereferencedJSONSchema { + return toType(this.client, this.conf as JsonDeserializerConfig, this, info, async (client, info) => { + const deps = new Map() + await this.resolveReferences(client, info, deps) + return deps + }) + } + + async toValidateFunction(info: SchemaInfo): Promise { + return await toValidateFunction(this.client, this.conf as JsonDeserializerConfig, this, info, async (client, info) => { + const deps = new Map() + await this.resolveReferences(client, info, deps) + return deps + }, + ) + } +} + +async function toValidateFunction( + client: Client, + conf: JsonSerdeConfig, + serde: JsonSerde, + info: SchemaInfo, + refResolver: RefResolver, +): Promise { + let fn = serde.schemaToValidateCache.get(stringify(info.schema)) + if (fn != null) { + return fn + } + + const deps = await refResolver(client, info) + + const json = JSON.parse(info.schema) + const spec = json.$schema + if (spec === 'http://json-schema.org/draft/2020-12/schema') { + const ajv2020 = new Ajv2020(conf as JsonSerdeConfig) + deps.forEach((name, schema) => { + ajv2020.addSchema(JSON.parse(schema), name) + }) + fn = ajv2020.compile(json) + } else { + const ajv = new Ajv2019(conf as JsonSerdeConfig) + ajv.addMetaSchema(draft6MetaSchema) + ajv.addMetaSchema(draft7MetaSchema) + deps.forEach((name, schema) => { + ajv.addSchema(JSON.parse(schema), name) + }) + fn = ajv.compile(json) + } + serde.schemaToValidateCache.set(stringify(info.schema), fn) + return fn +} + +async function toType( + client: Client, + conf: JsonSerdeConfig, + serde: JsonSerde, + info: SchemaInfo, + refResolver: RefResolver, +): Promise { + let type = serde.schemaToTypeCache.get(stringify(info.schema)) + if (type != null) { + return type + } + + const deps = await refResolver(client, info) + + const retrieve = (uri: string) => { + const data = deps.get(uri) + if (data == null) { + throw new SerializationError(`Schema not found: ${uri}`) + } + return JSON.parse(data) + } + + const json = JSON.parse(info.schema) + const spec = json.$schema + let schema + if (spec === 'http://json-schema.org/draft/2020-12/schema') { + schema = await dereferenceJSONSchemaDraft2020_12(json, { retrieve }) + } else { + schema = await dereferenceJSONSchemaDraft07(json, { retrieve }) + } + serde.schemaToTypeCache.set(stringify(info.schema), schema) + return schema +} + +async function transform(ctx: RuleContext, schema: DereferencedJSONSchema, path:string, msg: any, fieldTransform: FieldTransform): Promise { + if (msg == null || schema == null || typeof schema === 'boolean') { + return msg + } + let fieldCtx = ctx.currentField() + if (fieldCtx != null) { + fieldCtx.type = getType(schema) + } + if (schema.allOf != null && schema.allOf.length > 0) { + let subschema = validateSubschemas(schema.allOf, msg) + if (subschema != null) { + return await transform(ctx, subschema, path, msg, fieldTransform) + } + } + if (schema.anyOf != null && schema.anyOf.length > 0) { + let subschema = validateSubschemas(schema.anyOf, msg) + if (subschema != null) { + return await transform(ctx, subschema, path, msg, fieldTransform) + } + } + if (schema.oneOf != null && schema.oneOf.length > 0) { + let subschema = validateSubschemas(schema.oneOf, msg) + if (subschema != null) { + return await transform(ctx, subschema, path, msg, fieldTransform) + } + } + if (schema.items != null) { + if (Array.isArray(msg)) { + for (let i = 0; i < msg.length; i++) { + msg[i] = await transform(ctx, schema.items, path, msg[i], fieldTransform) + } + } + } + if (schema.$ref != null) { + return await transform(ctx, schema.$ref, path, msg, fieldTransform) + } + let type = getType(schema) + switch (type) { + case FieldType.RECORD: + if (schema.properties != null) { + for (let [propName, propSchema] of Object.entries(schema.properties)) { + let value = msg[propName] + await transformField(ctx, path, propName, msg, value, propSchema, fieldTransform) + } + } + return msg + case FieldType.ENUM: + case FieldType.STRING: + case FieldType.INT: + case FieldType.DOUBLE: + case FieldType.BOOLEAN: + if (fieldCtx != null) { + const ruleTags = ctx.rule.tags + if (ruleTags == null || ruleTags.size === 0 || !disjoint(ruleTags, fieldCtx.tags)) { + return await fieldTransform.transform(ctx, fieldCtx, msg) + } + } + } + + return msg +} + +async function transformField(ctx: RuleContext, path: string, propName: string, msg: any, value: any, + propSchema: DereferencedJSONSchema, + fieldTransform: FieldTransform): Promise { + const fullName = path + '.' + propName + try { + ctx.enterField(msg, fullName, propName, getType(propSchema), getInlineTags(propSchema)) + const newVal = await transform(ctx, propSchema, fullName, value, fieldTransform) + if (ctx.rule.kind === 'CONDITION') { + if (newVal === false) { + throw new RuleConditionError(ctx.rule) + } + } else { + msg[propName] = newVal + } + } finally { + ctx.leaveField() + } +} + +function validateSubschemas(subschemas: DereferencedJSONSchema[], msg: any): DereferencedJSONSchema | null { + for (let subschema of subschemas) { + try { + validateJSON(subschema, msg) + return subschema + } catch (error) { + // ignore + } + } + return null +} + +function getType(schema: DereferencedJSONSchema): FieldType { + if (typeof schema === 'boolean') { + return FieldType.NULL + } + if (schema.type == null) { + return FieldType.NULL + } + if (Array.isArray(schema.type)) { + return FieldType.COMBINED + } + if (schema.const != null || schema.enum != null) { + return FieldType.ENUM + } + switch (schema.type) { + case 'object': + if (schema.properties == null || Object.keys(schema.properties).length === 0) { + return FieldType.MAP + } + return FieldType.RECORD + case 'array': + return FieldType.ARRAY + case 'string': + return FieldType.STRING + case 'integer': + return FieldType.INT + case 'number': + return FieldType.DOUBLE + case 'boolean': + return FieldType.BOOLEAN + case 'null': + return FieldType.NULL + default: + return FieldType.NULL + } +} + +function getInlineTags(schema: DereferencedJSONSchema): Set { + let tagsKey = 'confluent:tags' as keyof DereferencedJSONSchema + return new Set(schema[tagsKey]) +} + +function disjoint(tags1: Set, tags2: Set): boolean { + for (let tag of tags1) { + if (tags2.has(tag)) { + return false + } + } + return true +} + + + diff --git a/schemaregistry/serde/json_util.ts b/schemaregistry/serde/json_util.ts new file mode 100644 index 00000000..15ec584f --- /dev/null +++ b/schemaregistry/serde/json_util.ts @@ -0,0 +1,122 @@ +/* + * Copyright (c) 2023 Menglin "Mark" Xu + * (c) 2024 Confluent, Inc. + * + * This software may be modified and distributed under the terms + * of the MIT license. See the LICENSE.txt file for details. + */ +import validator from 'validator'; +import { deepStrictEqual } from 'assert'; + +/** + * Generate JSON schema from value. + * + * @param value - Value. + * @returns - JSON schema. + */ +export function generateSchema(value: any): any { + switch (true) { + case value === undefined: + case typeof value === 'undefined': + case typeof value === 'function': + case typeof value === 'symbol': + case value instanceof Date: + throw new TypeError(`Invalid JSON value: ${String(value)}`); + + /** + * @see https://json-schema.org/understanding-json-schema/reference/null.html + */ + case value === null: + return { type: 'null' }; + + /** + * @see https://json-schema.org/understanding-json-schema/reference/numeric.html + */ + case typeof value === 'number': + return { type: Number.isInteger(value) ? 'integer' : 'number' }; + + /** + * @see https://json-schema.org/understanding-json-schema/reference/boolean.html + */ + case typeof value === 'boolean': + return { type: 'boolean' }; + + /** + * @see https://json-schema.org/understanding-json-schema/reference/string.html + */ + case typeof value === 'string': + if (validator.isISO8601(value)) { + return { + type: 'string', + format: value.includes('T') ? 'date-time' : 'date', + }; + } + + if (validator.isTime(value.split('+')[0], { mode: 'withSeconds' })) { + return { type: 'string', format: 'time' }; + } + + if (validator.isEmail(value)) { + return { type: 'string', format: 'email' }; + } + + return { type: 'string' }; + + /** + * @see https://json-schema.org/understanding-json-schema/reference/array.html + */ + case Array.isArray(value): + if (value.length === 1) { + return { type: 'array', items: generateSchema(value[0]) }; + } + + if (value.length > 1) { + const items = value.map(generateSchema); + if (deepEqual(...items)) { + return { type: 'array', items: items[0] }; + } + } + + return { type: 'array' }; + + /** + * @see https://json-schema.org/understanding-json-schema/reference/object.html + */ + case value instanceof Object: + if (!Object.keys(value).length) { + return { type: 'object' }; + } + + return { + type: 'object', + properties: Object.entries(value).reduce( + (accumulator, [key, value]) => { + accumulator[key] = generateSchema(value); + return accumulator; + }, + {} as Record, + ), + }; + + /* istanbul ignore next */ + default: + throw new TypeError(`Invalid JSON value: ${value}`); + } +} + +/** + * Tests for deep equality between the `actual` and `expected` parameters. + */ +export function deepEqual(...args: unknown[]): boolean { + try { + for (let index = 0, count = args.length; index < count; index++) { + if (index + 1 === count) { + continue; + } + deepStrictEqual(args[index], args[index + 1]); + } + return true; + } catch (error) { + return false; + } +} diff --git a/schemaregistry/serde/protobuf.ts b/schemaregistry/serde/protobuf.ts new file mode 100644 index 00000000..363a322c --- /dev/null +++ b/schemaregistry/serde/protobuf.ts @@ -0,0 +1,465 @@ +import { + Deserializer, + DeserializerConfig, + FieldTransform, + FieldType, RuleConditionError, + RuleContext, + SerdeType, SerializationError, + Serializer, + SerializerConfig +} from "./serde"; +import { + Client, Reference, RuleMode, + SchemaInfo, + SchemaMetadata +} from "../schemaregistry-client"; +import { + createFileRegistry, + DescField, + DescFile, + DescMessage, + FileRegistry, + fromBinary, getExtension, hasExtension, + Registry, + ScalarType, + toBinary +} from "@bufbuild/protobuf"; +import { FileDescriptorProtoSchema } from "@bufbuild/protobuf/wkt"; +import { BufferWrapper, MAX_VARINT_LEN_64 } from "./buffer-wrapper"; +import { LRUCache } from "lru-cache"; +import {field_meta, Meta} from "../confluent/meta_pb"; +import {getRuleExecutors} from "./rule-registry"; +import stringify from "json-stringify-deterministic"; + +export interface ProtobufSerde { + schemaToDescCache: LRUCache +} + +export type ProtobufSerializerConfig = SerializerConfig & { + registry: Registry + descToSchemaCache: LRUCache +} + +export class ProtobufSerializer extends Serializer implements ProtobufSerde { + registry: Registry + schemaToDescCache: LRUCache + descToSchemaCache: LRUCache + + constructor(client: Client, serdeType: SerdeType, conf: ProtobufSerializerConfig) { + super(client, serdeType, conf) + this.registry = conf.registry + this.schemaToDescCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 } ) + this.descToSchemaCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 } ) + this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { + return await this.fieldTransform(ctx, fieldTransform, msg) + } + for (const rule of getRuleExecutors()) { + rule.configure(client.config(), conf.ruleConfig ?? new Map) + } + } + + async serialize(topic: string, msg: any): Promise { + if (this.client == null) { + throw new Error('client is not initialized') + } + if (msg == null) { + throw new Error('message is empty') + } + + const typeName = msg.$typeName + if (typeName == null) { + throw new SerializationError('message type name is empty') + } + const messageDesc = this.registry.getMessage(typeName) + if (messageDesc == null) { + throw new SerializationError('message descriptor not in registry') + } + const fileDesc = messageDesc.file + const schema = await this.getSchemaInfo(fileDesc) + const [id, info] = await this.getId(topic, msg, schema) + const subject = this.subjectName(topic, info) + msg = await this.executeRules(subject, topic, RuleMode.WRITE, null, info, msg, null) + const msgIndexBytes = this.toMessageIndexBytes(messageDesc) + const msgBytes = Buffer.from(toBinary(messageDesc, msg)) + return this.writeBytes(id, Buffer.concat([msgIndexBytes, msgBytes])) + } + + async getSchemaInfo(fileDesc: DescFile): Promise { + const value = this.descToSchemaCache.get(fileDesc.name) + if (value != null) { + return value + } + const deps = this.toProtobufSchema(fileDesc) + const autoRegister = this.config().autoRegisterSchemas + const normalize = this.config().normalizeSchemas + const metadata = await this.resolveDependencies( + fileDesc, deps, "", Boolean(autoRegister), Boolean(normalize)) + const info = { + schema: metadata.schema, + schemaType: metadata.schemaType, + references: metadata.references, + metadata: metadata.metadata, + ruleSet: metadata.ruleSet, + } + this.descToSchemaCache.set(fileDesc.name, info) + return info + } + + toProtobufSchema(fileDesc: DescFile): Map { + const deps = new Map() + this.toDependencies(fileDesc, deps) + return deps + } + + toDependencies(fileDesc: DescFile, deps: Map) { + deps.set(fileDesc.name, Buffer.from(toBinary(FileDescriptorProtoSchema, fileDesc.proto)).toString('base64')) + fileDesc.dependencies.forEach((dep) => { + if (!this.ignoreFile(dep.name)) { + this.toDependencies(dep, deps) + } + }) + } + + async resolveDependencies(fileDesc: DescFile, deps: Map, subject: string, + autoRegister: boolean, normalize: boolean): Promise { + const refs: Reference[] = [] + refs.length = fileDesc.dependencies.length + for (let i = 0; i < fileDesc.dependencies.length; i++) { + const dep = fileDesc.dependencies[i] + if (this.ignoreFile(dep.name)) { + continue + } + const ref = await this.resolveDependencies(dep, deps, dep.name, autoRegister, normalize) + if (ref == null) { + throw new SerializationError('dependency not found') + } + refs.push({name: dep.name, subject: ref.subject!, version: ref.version!}) + } + const info: SchemaInfo = { + schema: deps.get(fileDesc.name)!, + schemaType: 'PROTOBUF', + references: refs + } + let id = -1 + let version = 0 + if (subject !== '') { + if (autoRegister) { + id = await this.client.register(subject, info, normalize) + } else { + id = await this.client.getId(subject, info, normalize) + + } + version = await this.client.getVersion(subject, info, normalize) + } + return { + id: id, + subject: subject, + version: version, + schema: info.schema, + schemaType: info.schemaType, + references: info.references, + metadata: info.metadata, + ruleSet: info.ruleSet, + } + } + + toMessageIndexBytes(messageDesc: DescMessage): Buffer { + const msgIndexes: number[] = this.toMessageIndexes(messageDesc, 0) + const buffer = Buffer.alloc((1 + msgIndexes.length) * MAX_VARINT_LEN_64) + const bw = new BufferWrapper(buffer) + bw.writeVarInt(msgIndexes.length) + for (let i = 0; i < msgIndexes.length; i++) { + bw.writeVarInt(msgIndexes[i]) + } + return buffer.subarray(0, bw.pos) + } + + toMessageIndexes(messageDesc: DescMessage, count: number): number[] { + const index = this.toIndex(messageDesc) + const parent = messageDesc.parent + if (parent == null) { + // parent is FileDescriptor, we reached the top of the stack, so we are + // done. Allocate an array large enough to hold count+1 entries and + // populate first value with index + const msgIndexes: number[] = [] + msgIndexes.length = count + 1 + msgIndexes[0] = index + return msgIndexes + } else { + const msgIndexes = this.toMessageIndexes(parent, count + 1) + msgIndexes.push(index) + return msgIndexes + } + } + + toIndex(messageDesc: DescMessage) { + const parent = messageDesc.parent + if (parent == null) { + const fileDesc = messageDesc.file + for (let i = 0; i < fileDesc.messages.length; i++) { + if (fileDesc.messages[i] === messageDesc) { + return i + } + } + } else { + for (let i = 0; i < parent.nestedMessages.length; i++) { + if (parent.nestedMessages[i] === messageDesc) { + return i + } + } + } + throw new SerializationError('message descriptor not found in file descriptor'); + } + + ignoreFile(name: string): boolean { + return name.startsWith('confluent/') || + name.startsWith('google/protobuf/') || + name.startsWith('google/type/') + } + + async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { + const typeName = msg.$typeName + if (typeName == null) { + throw new SerializationError('message type name is empty') + } + const messageDesc = this.registry.getMessage(typeName) + if (messageDesc == null) { + throw new SerializationError('message descriptor not in registry') + } + return await transform(ctx, messageDesc, msg, fieldTransform) + } +} + +export type ProtobufDeserializerConfig = DeserializerConfig + +export class ProtobufDeserializer extends Deserializer implements ProtobufSerde { + registry: FileRegistry + schemaToDescCache: LRUCache + + constructor(client: Client, serdeType: SerdeType, conf: ProtobufDeserializerConfig) { + super(client, serdeType, conf) + this.registry = createFileRegistry() + this.schemaToDescCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 } ) + this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { + return await this.fieldTransform(ctx, fieldTransform, msg) + } + for (const rule of getRuleExecutors()) { + rule.configure(client.config(), conf.ruleConfig ?? new Map) + } + } + + async deserialize(topic: string, payload: Buffer): Promise { + if (!Buffer.isBuffer(payload)) { + throw new Error('Invalid buffer') + } + if (payload.length === 0) { + return null + } + + const info = await this.getSchema(topic, payload) + const fd = await this.toFileDesc(this.client, info) + const [bytesRead, msgIndexes] = this.readMessageIndexes(payload.subarray(5)) + const messageDesc = this.toMessageDesc(fd, msgIndexes) + + const subject = this.subjectName(topic, info) + const readerMeta = await this.getReaderSchema(subject) + + const msgBytes = payload.subarray(5 + bytesRead) + let msg = fromBinary(messageDesc, msgBytes) + + // Currently JavaScript does not support migration rules + // because of lack of support for DynamicMessage + let target: SchemaInfo + if (readerMeta != null) { + target = readerMeta + } else { + target = info + } + msg = await this.executeRules(subject, topic, RuleMode.READ, null, target, msg, null) + return msg + } + + async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { + const typeName = msg.$typeName + if (typeName == null) { + throw new SerializationError('message type name is empty') + } + const messageDesc = this.registry.getMessage(typeName) + if (messageDesc == null) { + throw new SerializationError('message descriptor not in registry') + } + return await transform(ctx, messageDesc, msg, fieldTransform) + } + + async toFileDesc(client: Client, info: SchemaInfo): Promise { + const value = this.schemaToDescCache.get(stringify(info.schema)) + if (value != null) { + return value + } + const fileDesc = await this.parseFileDesc(client, info) + if (fileDesc == null) { + throw new SerializationError('file descriptor not found') + } + this.schemaToDescCache.set(stringify(info.schema), fileDesc) + return fileDesc + } + + async parseFileDesc(client: Client, info: SchemaInfo): Promise { + const deps = new Map() + await this.resolveReferences(client, info, deps) + const fileDesc = fromBinary(FileDescriptorProtoSchema, Buffer.from(info.schema, 'base64')) + const resolve = (depName: string) => { + const dep = deps.get(depName) + if (dep == null) { + throw new SerializationError('dependency not found') + } + return fromBinary(FileDescriptorProtoSchema, Buffer.from(dep, 'base64')) + } + // TODO check google protos already in registry + const fileRegistry = createFileRegistry(fileDesc, resolve) + this.registry = createFileRegistry(this.registry, fileRegistry) + return this.registry.getFile(fileDesc.name) + } + + readMessageIndexes(payload: Buffer): [number, number[]] { + const bw = new BufferWrapper(payload) + const count = bw.readVarInt() + const msgIndexes = [] + msgIndexes.length = count + for (let i = 0; i < count; i++) { + msgIndexes[i] = bw.readVarInt() + } + return [bw.pos, msgIndexes] + } + + toMessageDesc(fd: DescFile, msgIndexes: number[]): DescMessage { + let index = msgIndexes[0] + if (msgIndexes.length === 1) { + return fd.messages[index] + } + return this.toNestedMessageDesc(fd.messages[index], msgIndexes.slice(1)) + } + + toNestedMessageDesc(parent: DescMessage, msgIndexes: number[]): DescMessage { + let index = msgIndexes[0] + if (msgIndexes.length === 1) { + return parent.nestedMessages[index] + } + return this.toNestedMessageDesc(parent.nestedMessages[index], msgIndexes.slice(1)) + } +} + +async function transform(ctx: RuleContext, descriptor: DescMessage, msg: any, fieldTransform: FieldTransform): Promise { + if (msg == null || descriptor == null) { + return msg + } + if (Array.isArray(msg)) { + for (let i = 0; i < msg.length; i++) { + msg[i] = await transform(ctx, descriptor, msg[i], fieldTransform) + } + } + if (msg instanceof Map) { + return msg + } + const typeName = msg.$typeName + if (typeName != null) { + const fields = descriptor.fields + for (let i = 0; i < fields.length; i++) { + const fd = fields[i] + await transformField(ctx, fd, descriptor, msg, fieldTransform) + } + return msg + } + const fieldCtx = ctx.currentField() + if (fieldCtx != null) { + const ruleTags = ctx.rule.tags + if (ruleTags == null || ruleTags.size === 0 || !disjoint(ruleTags, fieldCtx.tags)) { + return await fieldTransform.transform(ctx, fieldCtx, msg) + } + } + return msg +} + +async function transformField(ctx: RuleContext, fd: DescField, desc: DescMessage, + msg: any, fieldTransform: FieldTransform) { + try { + ctx.enterField( + msg, + desc.name + '.' + fd.name, + fd.name, + getType(fd), + getInlineTags(fd) + ) + const value = msg[fd.name] + const newValue = await transform(ctx, desc, value, fieldTransform) + if (ctx.rule.kind === 'CONDITION') { + if (newValue === false) { + throw new RuleConditionError(ctx.rule) + } + } else { + msg[fd.name] = newValue + } + } finally { + ctx.leaveField() + } +} + +function getType(fd: DescField): FieldType { + switch (fd.fieldKind) { + case 'map': + return FieldType.MAP + case 'list': + return FieldType.ARRAY + case 'message': + return FieldType.RECORD + case 'enum': + return FieldType.ENUM + case 'scalar': + switch (fd.scalar) { + case ScalarType.STRING: + return FieldType.STRING + case ScalarType.BYTES: + return FieldType.BYTES + case ScalarType.INT32: + case ScalarType.SINT32: + case ScalarType.UINT32: + case ScalarType.FIXED32: + case ScalarType.SFIXED32: + return FieldType.INT + case ScalarType.INT64: + case ScalarType.SINT64: + case ScalarType.UINT64: + case ScalarType.FIXED64: + case ScalarType.SFIXED64: + return FieldType.LONG + case ScalarType.FLOAT: + case ScalarType.DOUBLE: + return FieldType.DOUBLE + case ScalarType.BOOL: + return FieldType.BOOLEAN + } + default: + return FieldType.NULL + } +} + +function getInlineTags(fd: DescField): Set { + const options = fd.proto.options + if (options != null && hasExtension(options, field_meta)) { + const option: Meta = getExtension(options, field_meta) + return new Set(option.tags) + } + return new Set() +} + +function disjoint(tags1: Set, tags2: Set): boolean { + for (let tag of tags1) { + if (tags2.has(tag)) { + return false + } + } + return true +} + + + diff --git a/schemaregistry/serde/rule-registry.ts b/schemaregistry/serde/rule-registry.ts new file mode 100644 index 00000000..e1fb1514 --- /dev/null +++ b/schemaregistry/serde/rule-registry.ts @@ -0,0 +1,42 @@ +import {RuleAction, RuleExecutor} from "./serde"; + +const ruleExecutors = new Map + +const ruleActions = new Map + + +// registerRuleExecutor is used to register a new rule executor. +export function registerRuleExecutor(ruleExecutor: RuleExecutor): void { + ruleExecutors.set(ruleExecutor.type(), ruleExecutor) +} + +// getRuleExecutor fetches a rule executor by a given name. +export function getRuleExecutor(name: string): RuleExecutor | undefined { + return ruleExecutors.get(name) +} + +// getRuleExecutors fetches all rule executors +export function getRuleExecutors(): RuleExecutor[] { + return Array.from(ruleExecutors.values()) +} + +// registerRuleAction is used to register a new rule action. +export function registerRuleAction(ruleAction: RuleAction): void { + ruleActions.set(ruleAction.type(), ruleAction) +} + +// getRuleAction fetches a rule action by a given name. +export function getRuleAction(name: string): RuleAction | undefined { + return ruleActions.get(name) +} + +// getRuleActions fetches all rule actions +export function getRuleActions(): RuleAction[] { + return Array.from(ruleActions.values()) +} + +// clearRules clears all registered rules +export function clearRules(): void { + ruleExecutors.clear() + ruleActions.clear() +} diff --git a/schemaregistry/serde/serde.ts b/schemaregistry/serde/serde.ts new file mode 100644 index 00000000..b550c941 --- /dev/null +++ b/schemaregistry/serde/serde.ts @@ -0,0 +1,746 @@ +import {match} from './wildcard-matcher'; +import { + Client, + Rule, + RuleMode, + RuleSet, + SchemaInfo, + SchemaMetadata, SchemaRegistryClient +} from "../schemaregistry-client"; +import {getRuleAction, getRuleExecutor} from "./rule-registry"; +import {ClientConfig} from "../rest-service"; +import {MockClient} from "../mock-schemaregistry-client"; + +export enum SerdeType { + KEY = 'KEY', + VALUE = 'VALUE' +} + +export const MAGIC_BYTE = Buffer.alloc(1) + +export class SerializationError extends Error { + + constructor(message?: string) { + super(message) + } +} + +export interface SerdeConfig { + // useLatestVersion specifies whether to use the latest schema version + useLatestVersion?: boolean + // useLatestWithMetadata specifies whether to use the latest schema with metadata + useLatestWithMetadata?: Map + // cacheCapacity specifies the cache capacity + cacheCapacity?: number, + // cacheLatestTtlSecs specifies the cache latest TTL in seconds + cacheLatestTtlSecs?: number + // ruleConfig specifies configuration options to the rules + ruleConfig?: Map + // subjectNameStrategy specifies a function to generate a subject name + subjectNameStrategy?: SubjectNameStrategyFunc +} + +export type RefResolver = (client: Client, info: SchemaInfo) => Promise> + +export abstract class Serde { + client: Client + serdeType: SerdeType + conf: SerdeConfig + fieldTransformer: FieldTransformer | null = null + + protected constructor(client: Client, serdeType: SerdeType, conf: SerdeConfig) { + this.client = client + this.serdeType = serdeType + this.conf = conf + } + + abstract config(): SerdeConfig + + close(): void { + return + } + + subjectName(topic: string, info?: SchemaInfo): string { + const strategy = this.conf.subjectNameStrategy ?? TopicNameStrategy + return strategy(topic, this.serdeType, info) + } + + async resolveReferences(client: Client, schema: SchemaInfo, deps: Map): Promise { + let references = schema.references + if (references == null) { + return + } + for (let ref of references) { + let metadata = await client.getSchemaMetadata(ref.subject, ref.version, true) + let info = { + schema: schema.schema, + schemaType: schema.schemaType, + references: schema.references, + metadata: schema.metadata, + ruleSet: schema.ruleSet, + } + deps.set(ref.name, metadata.schema) + await this.resolveReferences(client, info, deps) + } + } + + async executeRules(subject: string, topic: string, ruleMode: RuleMode, + source: SchemaInfo | null, target: SchemaInfo | null, msg: any, + inlineTags: Map> | null): Promise { + if (msg == null || target == null) { + return msg + } + let rules: Rule[] | undefined + switch (ruleMode) { + case RuleMode.UPGRADE: + rules = target.ruleSet?.migrationRules + break + case RuleMode.DOWNGRADE: + rules = source?.ruleSet?.migrationRules?.reverse() + break + default: + rules = target.ruleSet?.domainRules + if (ruleMode === RuleMode.READ) { + // Execute read rules in reverse order for symmetry + rules = rules?.reverse() + } + break + } + if (rules == null) { + return msg + } + for (let i = 0; i < rules.length; i++ ) { + let rule = rules[i] + if (rule.disabled) { + continue + } + let mode = rule.mode + switch (mode) { + case RuleMode.WRITEREAD: + if (ruleMode !== RuleMode.WRITE && ruleMode !== RuleMode.READ) { + continue + } + break + case RuleMode.UPDOWN: + if (ruleMode !== RuleMode.UPGRADE && ruleMode !== RuleMode.DOWNGRADE) { + continue + } + break + default: + if (mode !== ruleMode) { + continue + } + break + } + let ctx = new RuleContext(source, target, subject, topic, + this.serdeType === SerdeType.KEY, ruleMode, rule, i, rules, inlineTags, this.fieldTransformer!) + let ruleExecutor = getRuleExecutor(rule.type) + if (ruleExecutor == null) { + await this.runAction(ctx, ruleMode, rule, rule.onFailure, msg, + new Error(`could not find rule executor of type ${rule.type}`), 'ERROR') + return msg + } + try { + let result = await ruleExecutor.transform(ctx, msg) + switch (rule.kind) { + case 'CONDITION': + if (result === false) { + throw new RuleConditionError(rule) + } + break + case 'TRANSFORM': + msg = result + break + } + await this.runAction(ctx, ruleMode, rule, msg != null ? rule.onSuccess : rule.onFailure, + msg, null, msg != null ? 'NONE' : 'ERROR') + } catch (error) { + if (error instanceof SerializationError) { + throw error + } + await this.runAction(ctx, ruleMode, rule, rule.onFailure, msg, error as Error, 'ERROR') + } + } + return msg + } + + async runAction(ctx: RuleContext, ruleMode: RuleMode, rule: Rule, action: string | undefined, + msg: any, err: Error | null, defaultAction: string): Promise { + let actionName = this.getRuleActionName(rule, ruleMode, action) + if (actionName == null) { + actionName = defaultAction + } + let ruleAction = this.getRuleAction(ctx, actionName) + if (ruleAction == null) { + throw new RuleError(`Could not find rule action of type ${actionName}`) + } + try { + await ruleAction.run(ctx, msg, err) + } catch (error) { + if (error instanceof SerializationError) { + throw error + } + console.warn("could not run post-rule action %s: %s", actionName, error) + } + } + + getRuleActionName(rule: Rule, ruleMode: RuleMode, actionName: string | undefined): string | null { + if (actionName == null || actionName === '') { + return null + } + if ((rule.mode === RuleMode.WRITEREAD || rule.mode === RuleMode.UPDOWN) && actionName.includes(',')) { + let parts = actionName.split(',') + switch (ruleMode) { + case RuleMode.WRITE: + case RuleMode.UPGRADE: + return parts[0] + case RuleMode.READ: + case RuleMode.DOWNGRADE: + return parts[1] + } + } + return actionName + } + + getRuleAction(ctx: RuleContext, actionName: string): RuleAction | undefined { + if (actionName === 'ERROR') { + return new ErrorAction(); + } else if (actionName === 'NONE') { + return new NoneAction() + } + return getRuleAction(actionName) + } +} + +export interface SerializerConfig extends SerdeConfig { + // autoRegisterSchemas determines whether to automatically register schemas + autoRegisterSchemas?: boolean + // useSchemaID specifies a schema ID to use + useSchemaId?: number + // normalizeSchemas determines whether to normalize schemas + normalizeSchemas?: boolean +} + +export abstract class Serializer extends Serde { + protected constructor(client: Client, serdeType: SerdeType, conf: SerializerConfig) { + super(client, serdeType, conf) + } + + override config(): SerializerConfig { + return this.conf as SerializerConfig + } + + // serialize will serialize the given message + abstract serialize(topic: string, msg: any): Promise + + // GetID returns a schema ID for the given schema + async getId(topic: string, msg: any, info: SchemaInfo): Promise<[number, SchemaInfo]> { + let autoRegister = this.config().autoRegisterSchemas + let useSchemaId = this.config().useSchemaId + let useLatestWithMetadata = this.conf.useLatestWithMetadata + let useLatest = this.config().useLatestVersion + let normalizeSchema = this.config().normalizeSchemas + + let id = -1 + let subject = this.subjectName(topic, info) + if (autoRegister) { + id = await this.client.register(subject, info, Boolean(normalizeSchema)) + } else if (useSchemaId != null && useSchemaId >= 0) { + info = await this.client.getBySubjectAndId(subject, useSchemaId) + id = await this.client.getId(subject, info, false) + if (id !== useSchemaId) { + throw new SerializationError(`failed to match schema ID (${id} != ${useSchemaId})`) + } + } else if (useLatestWithMetadata != null && useLatestWithMetadata.size !== 0) { + info = await this.client.getLatestWithMetadata( + subject, Object.fromEntries(useLatestWithMetadata), true) + id = await this.client.getId(subject, info, false) + } else if (useLatest) { + info = await this.client.getLatestSchemaMetadata(subject) + id = await this.client.getId(subject, info, false) + } else { + id = await this.client.getId(subject, info, Boolean(normalizeSchema)) + } + return [id, info] + } + + writeBytes(id: number, msgBytes: Buffer): Buffer { + const idBuffer = Buffer.alloc(4) + idBuffer.writeInt32BE(id, 0) + return Buffer.concat([MAGIC_BYTE, idBuffer, msgBytes]) + } +} + +export type DeserializerConfig = SerdeConfig + +// Migration represents a migration +export interface Migration { + ruleMode: RuleMode + source: SchemaMetadata | null + target: SchemaMetadata | null +} + +export abstract class Deserializer extends Serde { + protected constructor(client: Client, serdeType: SerdeType, conf: DeserializerConfig) { + super(client, serdeType, conf) + } + + override config(): DeserializerConfig { + return this.conf as DeserializerConfig + } + + async getSchema(topic: string, payload: Buffer): Promise { + const magicByte = payload.subarray(0, 1) + if (!magicByte.equals(MAGIC_BYTE)) { + throw new SerializationError( + `Message encoded with magic byte ${JSON.stringify(magicByte)}, expected ${JSON.stringify( + MAGIC_BYTE, + )}`, + ) + } + const id = payload.subarray(1, 5).readInt32BE(0) + let subject = this.subjectName(topic) + return await this.client.getBySubjectAndId(subject, id) + } + + async getReaderSchema(subject: string): Promise { + let useLatestWithMetadata = this.config().useLatestWithMetadata + let useLatest = this.config().useLatestVersion + if (useLatestWithMetadata != null && useLatestWithMetadata.size !== 0) { + return await this.client.getLatestWithMetadata( + subject, Object.fromEntries(useLatestWithMetadata), true) + } + if (useLatest) { + return await this.client.getLatestSchemaMetadata(subject) + } + return null + } + + hasRules(ruleSet: RuleSet, mode: RuleMode): boolean { + switch (mode) { + case RuleMode.UPGRADE: + case RuleMode.DOWNGRADE: + return this.checkRules(ruleSet?.migrationRules, (ruleMode: RuleMode): boolean => + ruleMode === mode || ruleMode === RuleMode.UPDOWN) + case RuleMode.UPDOWN: + return this.checkRules(ruleSet?.migrationRules, (ruleMode: RuleMode): boolean => + ruleMode === mode) + case RuleMode.WRITE: + case RuleMode.READ: + return this.checkRules(ruleSet?.domainRules, (ruleMode: RuleMode): boolean => + ruleMode === mode || ruleMode === RuleMode.WRITEREAD) + case RuleMode.WRITEREAD: + return this.checkRules(ruleSet?.domainRules, (ruleMode: RuleMode): boolean => + ruleMode === mode) + } + } + + checkRules(rules: Rule[] | undefined, filter: (ruleMode: RuleMode) => boolean): boolean { + if (rules == null) { + return false + } + for (let rule of rules) { + let ruleMode = rule.mode + if (ruleMode && filter(ruleMode)) { + return true + } + } + return false + } + + async getMigrations(subject: string, sourceInfo: SchemaInfo, + target: SchemaMetadata): Promise { + let version = await this.client.getVersion(subject, sourceInfo, false) + let source: SchemaMetadata = { + id: 0, + version: version, + schema: sourceInfo.schema, + references: sourceInfo.references, + metadata: sourceInfo.metadata, + ruleSet: sourceInfo.ruleSet, + } + let migrationMode: RuleMode + let migrations: Migration[] = [] + let first: SchemaMetadata + let last: SchemaMetadata + if (source.version! < target.version!) { + migrationMode = RuleMode.UPGRADE + first = source + last = target + } else if (source.version! > target.version!) { + migrationMode = RuleMode.DOWNGRADE + first = target + last = source + } else { + return migrations + } + let previous: SchemaMetadata | null = null + let versions = await this.getSchemasBetween(subject, first, last) + for (let i = 0; i < versions.length; i++) { + let version = versions[i] + if (i === 0) { + previous = version + continue + } + if (version.ruleSet != null && this.hasRules(version.ruleSet, migrationMode)) { + let m: Migration + if (migrationMode === RuleMode.UPGRADE) { + m = { + ruleMode: migrationMode, + source: previous, + target: version, + } + } else { + m = { + ruleMode: migrationMode, + source: version, + target: previous, + } + } + migrations.push(m) + } + previous = version + } + if (migrationMode === RuleMode.DOWNGRADE) { + migrations = migrations.reverse() + } + return migrations + } + + async getSchemasBetween(subject: string, first: SchemaMetadata, + last: SchemaMetadata): Promise { + if (last.version!-first.version! <= 1) { + return [first, last] + } + let version1 = first.version! + let version2 = last.version! + let result = [first] + for (let i = version1 + 1; i < version2; i++) { + let meta = await this.client.getSchemaMetadata(subject, i, true) + result.push(meta) + } + result.push(last) + return result + } + + async executeMigrations(migrations: Migration[], subject: string, topic: string, msg: any): Promise { + for (let migration of migrations) { + // TODO fix source, target? + msg = await this.executeRules(subject, topic, migration.ruleMode, migration.source, migration.target, msg, null) + } + return msg + } +} + +export type SubjectNameStrategyFunc = ( + topic: string, + serdeType: SerdeType, + schema?: SchemaInfo, +) => string + +// TopicNameStrategy creates a subject name by appending -[key|value] to the topic name. +export const TopicNameStrategy: SubjectNameStrategyFunc = (topic: string, serdeType: SerdeType) => { + let suffix = '-value' + if (serdeType === SerdeType.KEY) { + suffix = '-key' + } + return topic + suffix +} + +// RuleContext represents a rule context +export class RuleContext { + source: SchemaInfo | null + target: SchemaInfo + subject: string + topic: string + isKey: boolean + ruleMode: RuleMode + rule: Rule + index: number + rules: Rule[] + inlineTags: Map> | null + fieldTransformer: FieldTransformer + private fieldContexts: FieldContext[] + + constructor(source: SchemaInfo | null, target: SchemaInfo, subject: string, topic: string, + isKey: boolean, ruleMode: RuleMode, rule: Rule, index: number, rules: Rule[], + inlineTags: Map> | null, fieldTransformer: FieldTransformer) { + this.source = source + this.target = target + this.subject = subject + this.topic = topic + this.isKey = isKey + this.ruleMode = ruleMode + this.rule = rule + this.index = index + this.rules = rules + this.inlineTags = inlineTags + this.fieldTransformer = fieldTransformer + this.fieldContexts = [] + } + + getParameter(name: string): string | null { + const params = this.rule.params + if (params == null) { + return null + } + let value = params[name] + if (value != null) { + return value + } + let metadata = this.target.metadata + if (metadata != null && metadata.properties != null) { + value = metadata.properties[name] + if (value != null) { + return value + } + } + return null + } + + getInlineTags(name: string): Set { + let tags = this.inlineTags?.get(name) + if (tags != null) { + return tags + } + return new Set() + } + + currentField(): FieldContext | null { + let size = this.fieldContexts.length + if (size === 0) { + return null + } + return this.fieldContexts[size - 1] + } + + enterField(containingMessage: any, fullName: string, name: string, fieldType: FieldType, tags: Set): FieldContext { + let allTags = new Set(tags) + for (let v of this.getTags(fullName)) { + allTags.add(v) + } + let fieldContext = new FieldContext( + containingMessage, + fullName, + name, + fieldType, + allTags + ) + this.fieldContexts.push(fieldContext) + return fieldContext + } + + getTags(fullName: string): Set { + let tags = new Set() + let metadata = this.target.metadata + if (metadata?.tags != null) { + for (let [k, v] of Object.entries(metadata.tags)) { + if (match(fullName, k)) { + for (let tag of v) { + tags.add(tag) + } + } + } + } + return tags + } + + leaveField(): void { + let size = this.fieldContexts.length - 1 + this.fieldContexts = this.fieldContexts.slice(0, size) + } +} + +// RuleBase represents a rule base +export interface RuleBase { + configure(clientConfig: ClientConfig, config: Map): void + + type(): string; + + close(): void +} + +// RuleExecutor represents a rule executor +export interface RuleExecutor extends RuleBase { + transform(ctx: RuleContext, msg: any): Promise +} + +// FieldTransformer represents a field transformer +export type FieldTransformer = (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => any; + +// FieldTransform represents a field transform +export interface FieldTransform { + transform(ctx: RuleContext, fieldCtx: FieldContext, fieldValue: any): Promise; +} + +// FieldRuleExecutor represents a field rule executor +export abstract class FieldRuleExecutor implements RuleExecutor { + config: Map | null = null + + abstract configure(clientConfig: ClientConfig, config: Map): void + + abstract type(): string; + + abstract newTransform(ctx: RuleContext): FieldTransform; + + async transform(ctx: RuleContext, msg: any): Promise { + // TODO preserve source + switch (ctx.ruleMode) { + case RuleMode.WRITE: + case RuleMode.UPGRADE: + for (let i = 0; i < ctx.index; i++) { + let otherRule = ctx.rules[i] + if (areTransformsWithSameTag(ctx.rule, otherRule)) { + // ignore this transform if an earlier one has the same tag + return msg + } + } + break + case RuleMode.READ: + case RuleMode.DOWNGRADE: + for (let i = ctx.index + 1; i < ctx.rules.length; i++) { + let otherRule = ctx.rules[i] + if (areTransformsWithSameTag(ctx.rule, otherRule)) { + // ignore this transform if a later one has the same tag + return msg + } + } + break + } + let fieldTransform = this.newTransform(ctx) + return ctx.fieldTransformer(ctx, fieldTransform, msg) + } + + abstract close(): void +} + +function areTransformsWithSameTag(rule1: Rule, rule2: Rule): boolean { + return rule1.tags != null && rule1.tags.size > 0 + && rule1.kind === 'TRANSFORM' + && rule1.kind === rule2.kind + && rule1.mode === rule2.mode + && rule1.type === rule2.type + && rule1.tags === rule2.tags +} + +// FieldContext represents a field context +export class FieldContext { + containingMessage: any + fullName: string + name: string + type: FieldType + tags: Set + + constructor(containingMessage: any, fullName: string, name: string, fieldType: FieldType, tags: Set) { + this.containingMessage = containingMessage + this.fullName = fullName + this.name = name + this.type = fieldType + this.tags = new Set(tags) + } + + isPrimitive(): boolean { + let t = this.type + return t === FieldType.STRING || t === FieldType.BYTES || t === FieldType.INT + || t === FieldType.LONG || t === FieldType.FLOAT || t === FieldType.DOUBLE + || t === FieldType.BOOLEAN || t === FieldType.NULL + } + + typeName(): string { + return this.type.toString() + } +} + +// FieldType represents the field type +export enum FieldType { + RECORD = 'RECORD', + ENUM = 'ENUM', + ARRAY = 'ARRAY', + MAP = 'MAP', + COMBINED = 'COMBINED', + FIXED = 'FIXED', + STRING = 'STRING', + BYTES = 'BYTES', + INT = 'INT', + LONG = 'LONG', + FLOAT = 'FLOAT', + DOUBLE = 'DOUBLE', + BOOLEAN = 'BOOLEAN', + NULL = 'NULL', +} + +// RuleAction represents a rule action +export interface RuleAction extends RuleBase { + run(ctx: RuleContext, msg: any, err: Error | null): Promise +} + +// ErrorAction represents an error action +export class ErrorAction implements RuleAction { + configure(clientConfig: ClientConfig, config: Map): void { + } + + type(): string { + return 'ERROR' + } + + async run(ctx: RuleContext, msg: any, err: Error): Promise { + throw new SerializationError(err.message) + } + + close(): void { + } +} + +// NoneAction represents a no-op action +export class NoneAction implements RuleAction { + configure(clientConfig: ClientConfig, config: Map): void { + } + + type(): string { + return 'NONE' + } + + async run(ctx: RuleContext, msg: any, err: Error): Promise { + return + } + + close(): void { + } +} + +// RuleError represents a rule condition error +export class RuleError extends Error { + + constructor(message?: string) { + super(message) + } +} + +// RuleConditionError represents a rule condition error +export class RuleConditionError extends RuleError { + rule: Rule + + constructor(rule: Rule) { + super(RuleConditionError.error(rule)) + this.rule = rule + } + + static error(rule: Rule): string { + let errMsg = rule.doc + if (!errMsg) { + if (rule.expr !== '') { + return `Expr failed: '${rule.expr}'` + } + return `Condition failed: '${rule.name}'` + } + return errMsg + } +} + +export function newClient(config: ClientConfig): Client { + let url = config.baseURLs[0] + if (url.startsWith("mock://")) { + return new MockClient(config) + } + return new SchemaRegistryClient(config) +} diff --git a/schemaregistry/serde/wildcard-matcher.ts b/schemaregistry/serde/wildcard-matcher.ts new file mode 100644 index 00000000..1635e95d --- /dev/null +++ b/schemaregistry/serde/wildcard-matcher.ts @@ -0,0 +1,90 @@ +/** + * Matches fully-qualified names that use dot (.) as the name boundary. + * + *

A '?' matches a single character. + * A '*' matches one or more characters within a name boundary. + * A '**' matches one or more characters across name boundaries. + * + *

Examples: + *

+ * wildcardMatch("eve", "eve*")                  --> true
+ * wildcardMatch("alice.bob.eve", "a*.bob.eve")  --> true
+ * wildcardMatch("alice.bob.eve", "a*.bob.e*")   --> true
+ * wildcardMatch("alice.bob.eve", "a*")          --> false
+ * wildcardMatch("alice.bob.eve", "a**")         --> true
+ * wildcardMatch("alice.bob.eve", "alice.bob*")  --> false
+ * wildcardMatch("alice.bob.eve", "alice.bob**") --> true
+ * 
+ * + * @param str the string to match on + * @param wildcardMatcher the wildcard string to match against + * @return true if the string matches the wildcard string + */ +export function match(str: string, wildcardMatcher: string): boolean { + let re = wildcardToRegexp(wildcardMatcher, '.') + let pattern: RegExp + try { + pattern = new RegExp(re) + } catch (error) { + return false + } + let match = str.match(pattern) + return match != null && match[0] === str +} + +function wildcardToRegexp(globExp: string, separator: string): string { + let dst = '' + let src = globExp.replaceAll('**'+separator+'*', '**') + let i = 0; + let size = src.length; + while (i < size) { + let c = src[i] + i++ + switch (c) { + case '*': + // One char lookahead for ** + if (i < src.length && src[i] == '*') { + dst += '.*' + i++ + } else { + dst += '[^' + separator + ']*' + } + break + case '?': + dst += '[^' + separator + ']' + break + case '.': + case '+': + case '{': + case '}': + case '(': + case ')': + case '|': + case '^': + case '$': + // These need to be escaped in regular expressions + dst += '\\' + c + break + case '\\': + [dst, i] = doubleSlashes(dst, src, i) + break + default: + dst += c + break + } + } + return dst +} + +function doubleSlashes(dst: string, src: string, i: number): [string, number] { + // Emit the next character without special interpretation + dst += '\\' + if (i+1 < src.length) { + dst += '\\' + src[i] + i++ + } else { + // A backslash at the very end is treated like an escaped backslash + dst += '\\' + } + return [dst, i] +} diff --git a/test/schemaregistry/dekregistry/dekregistry-client.spec.ts b/test/schemaregistry/dekregistry/dekregistry-client.spec.ts index 2efbfb5e..4a7eb47c 100644 --- a/test/schemaregistry/dekregistry/dekregistry-client.spec.ts +++ b/test/schemaregistry/dekregistry/dekregistry-client.spec.ts @@ -2,9 +2,9 @@ import { DekRegistryClient, Dek, Kek } from "../../../schemaregistry/dekregistry import { RestService } from "../../../schemaregistry/rest-service"; import { AxiosResponse } from 'axios'; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; -import { TEST_KEK, TEST_KEK_2, TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, - TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_2, TEST_ALGORITHM, - TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION, +import { TEST_KEK, TEST_KEK_2, TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, + TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_2, TEST_ALGORITHM, + TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION, TEST_DEK_LATEST} from "./test-constants"; import { mockClientConfig } from "../test-constants"; @@ -17,7 +17,7 @@ let restService: jest.Mocked; describe('DekRegistryClient', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new DekRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -29,7 +29,7 @@ describe('DekRegistryClient', () => { it('Should register kek when registerKek is called', async () => { restService.handleRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); const response: Kek = await client.registerKek( - TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); expect(response).toEqual(TEST_KEK); expect(restService.handleRequest).toHaveBeenCalledTimes(1); @@ -37,14 +37,14 @@ describe('DekRegistryClient', () => { it('Should return kek from cache when registerKek is called with same kek name', async () => { restService.handleRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); - await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); restService.handleRequest.mockResolvedValue({ data: TEST_KEK_2 } as AxiosResponse); - await client.registerKek(TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); - + await client.registerKek(TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); + const response: Kek = await client.registerKek( - TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); const response2: Kek = await client.registerKek( - TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); expect(response).toEqual(TEST_KEK); expect(response2).toEqual(TEST_KEK_2); @@ -53,7 +53,7 @@ describe('DekRegistryClient', () => { it('Should return kek from cache when getKek is called with same kek name', async () => { restService.handleRequest.mockResolvedValue({ data: TEST_KEK } as AxiosResponse); - await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + await client.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); const response: Kek = await client.getKek(TEST_KEK_NAME); expect(response).toEqual(TEST_KEK); @@ -62,19 +62,19 @@ describe('DekRegistryClient', () => { it('Should register dek when registerDek is called', async () => { restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); - const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); expect(response).toEqual(TEST_DEK); expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should return dek from cache when registerDek is called with same kek name, subject, algorithm, and version', async () => { restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); - await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); restService.handleRequest.mockResolvedValue({ data: TEST_DEK_2 } as AxiosResponse); - await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); - - const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); - const response2: Dek = await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); + + const response: Dek = await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); + const response2: Dek = await client.registerDek(TEST_KEK_NAME_2, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); expect(response).toEqual(TEST_DEK); expect(response2).toEqual(TEST_DEK_2); @@ -83,7 +83,7 @@ describe('DekRegistryClient', () => { it('Should return dek from cache when getDek is called with same kek name, subject, algorithm, and version', async () => { restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); - await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); const response: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION); expect(response).toEqual(TEST_DEK); @@ -97,10 +97,10 @@ describe('DekRegistryClient', () => { expect(await client.checkLatestDekInCache(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM)).toBe(true); restService.handleRequest.mockResolvedValue({ data: TEST_DEK } as AxiosResponse); - await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + await client.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); const getDekResponse2: Dek = await client.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM); - + expect(getDekResponse2).toEqual(TEST_DEK); expect(await client.checkLatestDekInCache(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM)).toBe(false); }); -}); \ No newline at end of file +}); diff --git a/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts b/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts index d611e532..71176814 100644 --- a/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts +++ b/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts @@ -1,30 +1,30 @@ import { Dek, Kek } from "../../../schemaregistry/dekregistry/dekregistry-client"; import { MockDekRegistryClient } from "../../../schemaregistry/dekregistry/mock-dekregistry-client"; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; -import { TEST_KEK, TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, - TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_V2, TEST_ALGORITHM, +import { TEST_KEK, TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, + TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_V2, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION } from "./test-constants"; describe('MockClient-tests', () => { let mockClient: MockDekRegistryClient; - + beforeEach(() => { mockClient = new MockDekRegistryClient(); }); - + afterEach(() => { jest.clearAllMocks(); }); - + it('Should return kek when registering Kek', async () => { const registerKekResponse: Kek = await mockClient.registerKek( - TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); expect(registerKekResponse).toEqual(TEST_KEK); }); it('Should return kek when getting Kek', async () => { - await mockClient.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, true); + await mockClient.registerKek(TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, true, TEST_KMS_PROPS, TEST_DOC); const getKekResponse: Kek = await mockClient.getKek(TEST_KEK_NAME); expect(getKekResponse).toEqual(TEST_KEK); @@ -32,23 +32,23 @@ describe('MockClient-tests', () => { it('Should return dek when registering Dek', async () => { const registerDekResponse: Dek = await mockClient.registerDek( - TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); expect(registerDekResponse).toEqual(TEST_DEK); }); it('Should return dek when getting Dek', async () => { - await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); const getDekResponse: Dek = await mockClient.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION); expect(getDekResponse).toEqual(TEST_DEK); }); it('Should return latest dek when getting Dek with version -1', async () => { - await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, 2); - await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION); + await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, 2, TEST_ENCRYPTED_KEY_MATERIAL); + await mockClient.registerDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, TEST_VERSION, TEST_ENCRYPTED_KEY_MATERIAL); const getDekResponse: Dek = await mockClient.getDek(TEST_KEK_NAME, TEST_SUBJECT, TEST_ALGORITHM, -1); expect(getDekResponse).toEqual(TEST_DEK_V2); }); - }); \ No newline at end of file + }); diff --git a/test/schemaregistry/mock-schemaregistery-client.spec.ts b/test/schemaregistry/mock-schemaregistery-client.spec.ts index ae830cb7..c878ee8a 100644 --- a/test/schemaregistry/mock-schemaregistery-client.spec.ts +++ b/test/schemaregistry/mock-schemaregistery-client.spec.ts @@ -1,6 +1,6 @@ +import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; import { MockClient } from '../../schemaregistry/mock-schemaregistry-client'; import { Compatibility, Metadata, SchemaInfo, SchemaMetadata } from '../../schemaregistry/schemaregistry-client'; -import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; const schemaString: string = JSON.stringify({ type: 'record', @@ -150,14 +150,14 @@ describe('MockClient-tests', () => { }); it('Should update compatibility when calling updateCompatibility', async () => { - const response: Compatibility = await mockClient.updateCompatibility(testSubject, Compatibility.BackwardTransitive); - expect(response).toBe(Compatibility.BackwardTransitive); + const response: Compatibility = await mockClient.updateCompatibility(testSubject, Compatibility.BACKWARD_TRANSITIVE); + expect(response).toBe(Compatibility.BACKWARD_TRANSITIVE); }); it('Should return compatibility when calling getCompatibility', async () => { - await mockClient.updateCompatibility(testSubject, Compatibility.BackwardTransitive); + await mockClient.updateCompatibility(testSubject, Compatibility.BACKWARD_TRANSITIVE); const response: Compatibility = await mockClient.getCompatibility(testSubject); - expect(response).toBe(Compatibility.BackwardTransitive); + expect(response).toBe(Compatibility.BACKWARD_TRANSITIVE); }); it('Should throw error when getCompatibility is called with non-existing subject', async () => { @@ -165,14 +165,14 @@ describe('MockClient-tests', () => { }); it('Should update default compatibility when calling updateDefaultCompatibility', async () => { - const response: Compatibility = await mockClient.updateDefaultCompatibility(Compatibility.BackwardTransitive); - expect(response).toBe(Compatibility.BackwardTransitive); + const response: Compatibility = await mockClient.updateDefaultCompatibility(Compatibility.BACKWARD_TRANSITIVE); + expect(response).toBe(Compatibility.BACKWARD_TRANSITIVE); }); it('Should return default compatibility when calling getDefaultCompatibility', async () => { - await mockClient.updateDefaultCompatibility(Compatibility.BackwardTransitive); + await mockClient.updateDefaultCompatibility(Compatibility.BACKWARD_TRANSITIVE); const response: Compatibility = await mockClient.getDefaultCompatibility(); - expect(response).toBe(Compatibility.BackwardTransitive); + expect(response).toBe(Compatibility.BACKWARD_TRANSITIVE); }); it('Should throw error when getDefaultCompatibility is called with non-existing default compatibility', async () => { @@ -216,4 +216,4 @@ describe('MockClient-tests', () => { const response2: SchemaInfo = await mockClient.getBySubjectAndId(testSubject, 2); await expect(response2.schema).toBe(schemaString2); }); -}); \ No newline at end of file +}); diff --git a/test/schemaregistry/schemaregistry-client.spec.ts b/test/schemaregistry/schemaregistry-client.spec.ts index 6c43ba6e..14c7e060 100644 --- a/test/schemaregistry/schemaregistry-client.spec.ts +++ b/test/schemaregistry/schemaregistry-client.spec.ts @@ -79,7 +79,7 @@ const versions: number[] = [1, 2, 3]; describe('SchemaRegistryClient-Register', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new SchemaRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -176,7 +176,7 @@ describe('SchemaRegistryClient-Register', () => { describe('SchemaRegistryClient-Get-ID', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new SchemaRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -269,7 +269,7 @@ describe('SchemaRegistryClient-Get-ID', () => { describe('SchemaRegistryClient-Get-Schema-Metadata', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new SchemaRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -382,7 +382,7 @@ describe('SchemaRegistryClient-Get-Schema-Metadata', () => { describe('SchemaRegistryClient-Subjects', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new SchemaRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -505,7 +505,7 @@ describe('SchemaRegistryClient-Subjects', () => { describe('SchemaRegistryClient-Compatibility', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new SchemaRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -525,9 +525,9 @@ describe('SchemaRegistryClient-Compatibility', () => { it('Should update compatibility level when updateCompatibility is called', async () => { restService.handleRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); - const response: Compatibility = await client.updateCompatibility(mockSubject, Compatibility.Backward); + const response: Compatibility = await client.updateCompatibility(mockSubject, Compatibility.BACKWARD); - expect(response).toEqual(Compatibility.Backward); + expect(response).toEqual(Compatibility.BACKWARD); expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); @@ -536,23 +536,23 @@ describe('SchemaRegistryClient-Compatibility', () => { const response: Compatibility = await client.getDefaultCompatibility(); - expect(response).toEqual(Compatibility.Backward); + expect(response).toEqual(Compatibility.BACKWARD); expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); it('Should update default compatibility level when updateDefaultCompatibility is called', async () => { restService.handleRequest.mockResolvedValue({ data: { compatibility: 'BACKWARD' } } as AxiosResponse); - const response: Compatibility = await client.updateDefaultCompatibility(Compatibility.Backward); + const response: Compatibility = await client.updateDefaultCompatibility(Compatibility.BACKWARD); - expect(response).toEqual(Compatibility.Backward); + expect(response).toEqual(Compatibility.BACKWARD); expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); }); describe('SchemaRegistryClient-Config', () => { beforeEach(() => { - restService = new RestService(mockClientConfig.createAxiosDefaults, mockClientConfig.baseURLs) as jest.Mocked; + restService = new RestService(mockClientConfig.baseURLs) as jest.Mocked; client = new SchemaRegistryClient(mockClientConfig); (client as any).restService = restService; }); @@ -577,7 +577,7 @@ describe('SchemaRegistryClient-Config', () => { it('Should update config when updateConfig is called', async () => { const request = { - compatibility: Compatibility.Backward, + compatibility: Compatibility.BACKWARD, alias: 'test-config', normalize: true, }; @@ -612,7 +612,7 @@ describe('SchemaRegistryClient-Config', () => { it('Should update default config when updateDefaultConfig is called', async () => { const request = { - compatibility: Compatibility.Backward, + compatibility: Compatibility.BACKWARD, alias: 'test-config', normalize: true, }; @@ -629,4 +629,4 @@ describe('SchemaRegistryClient-Config', () => { expect(response).toMatchObject(expectedResponse); expect(restService.handleRequest).toHaveBeenCalledTimes(1); }); -}); \ No newline at end of file +}); diff --git a/test/schemaregistry/serde/avro.spec.ts b/test/schemaregistry/serde/avro.spec.ts new file mode 100644 index 00000000..068d99e9 --- /dev/null +++ b/test/schemaregistry/serde/avro.spec.ts @@ -0,0 +1,32 @@ +import {describe, expect, it} from '@jest/globals'; +import {ClientConfig} from "../../../schemaregistry/rest-service"; +import {AvroDeserializer, AvroSerializer} from "../../../schemaregistry/serde/avro"; +import {newClient, SerdeType} from "../../../schemaregistry/serde/serde"; + +describe('AvroSerializer', () => { + it('basic serialization', async () => { + let conf: ClientConfig = { + baseURLs: ['mock://'], + cacheCapacity: 1000, + createAxiosDefaults: {} + } + let client = newClient(conf) + let ser = new AvroSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: new Buffer([1, 2]), + } + let bytes = await ser.serialize("topic1", obj) + + let deser = new AvroDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize("topic1", bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + }) +}) diff --git a/test/schemaregistry/serde/buffer-wrapper.spec.ts b/test/schemaregistry/serde/buffer-wrapper.spec.ts new file mode 100644 index 00000000..5c4ed3af --- /dev/null +++ b/test/schemaregistry/serde/buffer-wrapper.spec.ts @@ -0,0 +1,26 @@ +import { describe, expect, it } from '@jest/globals'; +import { BufferWrapper, MAX_VARINT_LEN_32 } from "../../../schemaregistry/serde/buffer-wrapper"; + +describe('BufferWrapper', () => { + it('write and read 100', () => { + const buf = new Buffer(MAX_VARINT_LEN_32) + const bw = new BufferWrapper(buf) + bw.writeVarInt(100) + const bw2 = new BufferWrapper(bw.buf.subarray(0, bw.pos)) + expect(bw2.readVarInt()).toBe(100) + }) + it('write and read max pos int', () => { + const buf = new Buffer(MAX_VARINT_LEN_32) + const bw = new BufferWrapper(buf) + bw.writeVarInt(2147483647) + const bw2 = new BufferWrapper(bw.buf.subarray(0, bw.pos)) + expect(bw2.readVarInt()).toBe(2147483647) + }) + it('write and read max neg int', () => { + const buf = new Buffer(MAX_VARINT_LEN_32) + const bw = new BufferWrapper(buf) + bw.writeVarInt(-2147483648) + const bw2 = new BufferWrapper(bw.buf.subarray(0, bw.pos)) + expect(bw2.readVarInt()).toBe(-2147483648) + }) +}) diff --git a/test/schemaregistry/serde/wildcard-matcher.spec.ts b/test/schemaregistry/serde/wildcard-matcher.spec.ts new file mode 100644 index 00000000..4e76f79e --- /dev/null +++ b/test/schemaregistry/serde/wildcard-matcher.spec.ts @@ -0,0 +1,86 @@ +import { match } from '../../../schemaregistry/serde/wildcard-matcher'; +import { describe, expect, it } from '@jest/globals'; + +describe('WildcardMatcher', () => { + it('when match', () => { + expect(match('', 'Foo')).toBe(false) + }) + it('when match', () => { + expect(match('Foo', '')).toBe(false) + }) + it('when match', () => { + expect(match('', '')).toBe(true) + }) + it('when match', () => { + expect(match('Foo', 'Foo')).toBe(true) + }) + it('when match', () => { + expect(match('', '*')).toBe(true) + }) + it('when match', () => { + expect(match('', '?')).toBe(false) + }) + it('when match', () => { + expect(match('Foo', 'Fo*')).toBe(true) + }) + it('when match', () => { + expect(match('Foo', 'Fo?')).toBe(true) + }) + it('when match', () => { + expect(match('Foo Bar and Catflag', 'Fo*')).toBe(true) + }) + it('when match', () => { + expect(match('New Bookmarks', 'N?w ?o?k??r?s')).toBe(true) + }) + it('when match', () => { + expect(match('Foo', 'Bar')).toBe(false) + }) + it('when match', () => { + expect(match('Foo Bar Foo', 'F*o Bar*')).toBe(true) + }) + it('when match', () => { + expect(match('Adobe Acrobat Installer', 'Ad*er')).toBe(true) + }) + it('when match', () => { + expect(match('Foo', '*Foo')).toBe(true) + }) + it('when match', () => { + expect(match('BarFoo', '*Foo')).toBe(true) + }) + it('when match', () => { + expect(match('Foo', 'Foo*')).toBe(true) + }) + it('when match', () => { + expect(match('FOO', '*Foo')).toBe(false) + }) + it('when match', () => { + expect(match('BARFOO', '*Foo')).toBe(false) + }) + it('when match', () => { + expect(match('FOO', 'Foo*')).toBe(false) + }) + it('when match', () => { + expect(match('FOOBAR', 'Foo*')).toBe(false) + }) + it('when match', () => { + expect(match('eve', 'eve*')).toBe(true) + }) + it('when match', () => { + expect(match('alice.bob.eve', 'a*.bob.eve')).toBe(true) + }) + it('when match', () => { + expect(match('alice.bob.eve', 'a*.bob.e*')).toBe(true) + }) + it('when match', () => { + expect(match('alice.bob.eve', 'a*')).toBe(false) + }) + it('when match', () => { + expect(match('alice.bob.eve', 'a**')).toBe(true) + }) + it('when match', () => { + expect(match('alice.bob.eve', 'alice.bob*')).toBe(false) + }) + it('when match', () => { + expect(match('alice.bob.eve', 'alice.bob**')).toBe(true) + }) +}) diff --git a/tsconfig.json b/tsconfig.json index 05732ed7..52031e99 100644 --- a/tsconfig.json +++ b/tsconfig.json @@ -2,7 +2,7 @@ "compilerOptions": { "baseUrl": ".", "lib": [ - "es2021" + "es2021", "dom" ], "module": "commonjs", "target": "es2021", From 2adc75eda6518a77ac005ad0c6115b8562c83053 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 27 Aug 2024 18:55:26 +0530 Subject: [PATCH 182/224] Add assign/unassign within rebalance callbacks Also remove onPartitionsAssigned and onPartitionsRevoked. --- examples/kafkajs/consumer.js | 25 ++- lib/kafkajs/_common.js | 1 - lib/kafkajs/_consumer.js | 111 +++++++------- .../consumer/incrementalRebalance.spec.js | 7 +- .../consumer/rebalanceCallback.spec.js | 143 ++++++++++++++++++ 5 files changed, 214 insertions(+), 73 deletions(-) create mode 100644 test/promisified/consumer/rebalanceCallback.spec.js diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 85e5c072..fd4579b8 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,5 +1,5 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; +const { Kafka, ErrorCodes } = require('@confluentinc/kafka-javascript').KafkaJS; async function consumerStart() { let consumer; @@ -22,22 +22,17 @@ async function consumerStart() { kafkaJS: { groupId: 'test-group', autoCommit: false, - rebalanceListener: { - onPartitionsAssigned: async (assignment) => { - console.log(`Assigned partitions ${JSON.stringify(assignment)}`); - }, - onPartitionsRevoked: async (assignment) => { - console.log(`Revoked partitions ${JSON.stringify(assignment)}`); - if (!stopped) { - await consumer.commitOffsets().catch((e) => { - console.error(`Failed to commit ${e}`); - }) - } - } - }, }, - /* Properties from librdkafka can also be used */ + rebalance_cb: (err, assignment) => { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + console.log(`Assigned partitions ${JSON.stringify(assignment)}`); + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + console.log(`Revoked partitions ${JSON.stringify(assignment)}`); + } else { + console.error(`Rebalance error ${err}`); + } + }, 'auto.commit.interval.ms': 6000, }); diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 074f1f6a..6e33a886 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -52,7 +52,6 @@ const kafkaJSProperties = { 'autoCommit', 'autoCommitInterval', 'autoCommitThreshold', - 'rebalanceListener', ], admin: [], }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 770eaf50..165c1a13 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -248,45 +248,67 @@ class Consumer { err = LibrdKafkaError.create(err); const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; - let call; + let assignmentFnCalled = false; + function assignmentFn(userAssignment) { + if (assignmentFnCalled) + return; + assignmentFnCalled = true; + + if (this.#internalClient.rebalanceProtocol() === "EAGER") { + this.#internalClient.assign(userAssignment); + this.#partitionCount = userAssignment.length; + } else { + this.#internalClient.incrementalAssign(userAssignment); + this.#partitionCount += userAssignment.length; + } + } - /* Since we don't expose assign() or incremental_assign() methods, we allow the user - * to modify the assignment by returning it. If a truthy value is returned, we use that - * and do not apply any pending seeks to it either. */ + function unassignmentFn(userAssignment) { + if (assignmentFnCalled) + return; + + assignmentFnCalled = true; + if (this.#internalClient.rebalanceProtocol() === "EAGER") { + this.#internalClient.unassign(); + this.#messageCache.removeTopicPartitions(); + this.#partitionCount = 0; + } else { + this.#internalClient.incrementalUnassign(userAssignment); + this.#messageCache.removeTopicPartitions(userAssignment); + this.#partitionCount -= userAssignment.length; + } + } + + let call = Promise.resolve(); + + /* We allow the user to modify the assignment by returning it. If a truthy + * value is returned, we use that and do not apply any pending seeks to it either. + * The user can alternatively use the assignmentFns argument. + * Precedence is given to the calling of functions within assignmentFns. */ let assignmentModified = false; if (typeof userSpecifiedRebalanceCb === 'function') { call = new Promise((resolve, reject) => { - try { - const alternateAssignment = userSpecifiedRebalanceCb(err, assignment); + const assignmentFns = { + assign: assignmentFn.bind(this), + unassign: unassignmentFn.bind(this), + }; + + /* The user specified callback may be async, or sync. Wrapping it in a + * Promise.resolve ensures that we always get a promise back. */ + return Promise.resolve( + userSpecifiedRebalanceCb(err, assignment, assignmentFns) + ).then(alternateAssignment => { if (alternateAssignment) { assignment = alternateAssignment; assignmentModified = true; } resolve(); - } catch (e) { - reject(e); - } + }).catch(reject); }); - } else { - 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.#userConfig.rebalanceListener.onPartitionsAssigned ? - this.#userConfig.rebalanceListener.onPartitionsAssigned(assignment) : - Promise.resolve()).catch(e => this.#logger.error(e)); - break; - case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: - call = (this.#userConfig.rebalanceListener.onPartitionsRevoked ? - this.#userConfig.rebalanceListener.onPartitionsRevoked(assignment) : - Promise.resolve()).catch(e => this.#logger.error(e)); - break; - default: - call = Promise.reject(`Unexpected rebalanceListener error code ${err.code}`).catch((e) => { - this.#logger.error(e); - }); - break; - } + } else if (err.code !== LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS && err.code !== LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS) { + call = Promise.reject(`Unexpected rebalance_cb error code ${err.code}`).catch((e) => { + this.#logger.error(e); + }); } call @@ -311,16 +333,10 @@ class Consumer { if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { const checkPendingSeeks = this.#pendingSeeks.size !== 0; - if (checkPendingSeeks && !assignmentModified) + if (checkPendingSeeks && !assignmentModified && !assignmentFnCalled) assignment = this.#assignAsPerSeekedOffsets(assignment); - if (this.#internalClient.rebalanceProtocol() === "EAGER") { - this.#internalClient.assign(assignment); - this.#partitionCount = assignment.length; - } else { - this.#internalClient.incrementalAssign(assignment); - this.#partitionCount += assignment.length; - } + assignmentFn.call(this, assignment); if (checkPendingSeeks) { const offsetsToCommit = assignment @@ -342,15 +358,7 @@ class Consumer { this.#messageCache.addTopicPartitions(assignment); } else { - if (this.#internalClient.rebalanceProtocol() === "EAGER") { - this.#internalClient.unassign(); - this.#messageCache.removeTopicPartitions(); - this.#partitionCount = 0; - } else { - this.#internalClient.incrementalUnassign(assignment); - this.#messageCache.removeTopicPartitions(assignment); - this.#partitionCount -= assignment.length; - } + unassignmentFn.call(this, assignment); } } catch (e) { // Ignore exceptions if we are not connected @@ -522,16 +530,10 @@ class Consumer { /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ delete rdKafkaConfig.kafkaJS; - delete rdKafkaConfig.rebalanceListener; /* Certain properties that the user has set are overridden. We use trampolines to accommodate the user's callbacks. * TODO: add trampoline method for offset commit callback. */ rdKafkaConfig['offset_commit_cb'] = true; - - if (!Object.hasOwn(this.#userConfig, '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.#userConfig.rebalanceListener = {}; - } rdKafkaConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); /* Offset management is different from case to case. @@ -1587,6 +1589,7 @@ class Consumer { if (!topic.partitions) { toppar.partitions = this.#getAllAssignedPartition(topic.topic); } else { + /* TODO: add a check here to make sure we own each partition */ toppar.partitions = [...topic.partitions]; } @@ -1597,6 +1600,8 @@ class Consumer { if (flattenedToppars.length === 0) { return; } + + /* TODO: error handling is lacking for pause, including partition level errors. */ this.#internalClient.pause(flattenedToppars); /* Mark the messages in the cache as stale, runInternal* will deal with @@ -1608,7 +1613,7 @@ class Consumer { .filter(key => this.#topicPartitionToBatchPayload.has(key)) .forEach(key => this.#topicPartitionToBatchPayload.get(key)._stale = true); - flattenedToppars.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); + flattenedToppars.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); /* Note: we don't use flattenedToppars here because resume flattens them again. */ return () => this.resume(toppars); diff --git a/test/promisified/consumer/incrementalRebalance.spec.js b/test/promisified/consumer/incrementalRebalance.spec.js index 37463739..61647ad4 100644 --- a/test/promisified/consumer/incrementalRebalance.spec.js +++ b/test/promisified/consumer/incrementalRebalance.spec.js @@ -148,12 +148,11 @@ describe('Consumer > incremental rebalance', () => { let revokes = 0; consumer = createConsumer(consumerConfig, { - rebalanceListener: { - onPartitionsAssigned: async (assignment) => { + rebalance_cb: async (err, assignment) => { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { assigns++; expect(assignment.length).toBe(2); - }, - onPartitionsRevoked: async (assignment) => { + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { revokes++; expect(assignment.length).toBe(2); } diff --git a/test/promisified/consumer/rebalanceCallback.spec.js b/test/promisified/consumer/rebalanceCallback.spec.js new file mode 100644 index 00000000..9269e19b --- /dev/null +++ b/test/promisified/consumer/rebalanceCallback.spec.js @@ -0,0 +1,143 @@ +jest.setTimeout(30000); + +const { waitFor, + secureRandom, + createTopic, + createConsumer, + createProducer, + sleep, } = require("../testhelpers"); +const { ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Consumer', () => { + let consumer; + let groupId, topicName; + let consumerConfig; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; + consumerConfig = { + groupId, + }; + consumer = null; + await createTopic({ topic: topicName, partitions: 3 }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()); + }); + + it('calls rebalance callback', async () => { + let calls = 0; + consumer = createConsumer(consumerConfig, { + rebalance_cb: function () { + calls++; + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(calls).toBe(1); /* assign */ + await consumer.disconnect(); + expect(calls).toBe(2); /* assign + unassign */ + consumer = null; + }); + + it('allows modifying the assignment via returns', async () => { + consumer = createConsumer(consumerConfig, { + rebalance_cb: function (err, assignment) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + expect(assignment.length).toBe(3); + return assignment.filter(a => a.partition !== 0); + } + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(consumer.assignment().length).toBe(2); + expect(consumer.assignment()).toEqual( + expect.arrayContaining([ + { topic: topicName, partition: 1 }, + { topic: topicName, partition: 2 }])); + }); + + it('allows modifying the assigment via assignment functions', async () => { + let calls = 0; + consumer = createConsumer(consumerConfig, { + rebalance_cb: function (err, assignment, assignmentFns) { + calls++; + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + expect(assignment.length).toBe(3); + assignmentFns.assign(assignment.filter(a => a.partition !== 0)); + } else { + assignmentFns.unassign(assignment); + } + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async () => { } }); + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + expect(consumer.assignment().length).toBe(2); + expect(consumer.assignment()).toEqual( + expect.arrayContaining([ + { topic: topicName, partition: 1 }, + { topic: topicName, partition: 2 }])); + await consumer.disconnect(); + expect(calls).toBe(2); + consumer = null; + }); + + it('pauses correctly from the rebalance callback after assign', async () => { + consumer = createConsumer(consumerConfig, { + rebalance_cb: function (err, assignment, assignmentFns) { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + expect(assignment.length).toBe(3); + + /* Assign first so we can pause. */ + assignmentFns.assign(assignment); + + /* Convert the assignment into format suitable for pause argument. */ + const pausablePartitions = [{ topic: topicName, partitions: [0, 1, 2] }]; + consumer.pause(pausablePartitions); + } else { + assignmentFns.unassign(assignment); + } + } + }); + + let messagesConsumed = []; + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + consumer.run({ eachMessage: async (e) => { messagesConsumed.push(e); } }); + await waitFor(() => consumer.assignment().length > 0, () => null, 1000); + + const producer = createProducer({}); + await producer.connect(); + const key1 = secureRandom(); + for (const partition of [0, 1, 2]) { + const message = { key: `key-${key1}`, value: `value-${key1}`, partition }; + await producer.send({ + topic: topicName, + messages: [message], + }); + } + await producer.disconnect(); + + expect(consumer.paused()).toEqual([{ topic: topicName, partitions: [0, 1, 2] }]); + + /* Give it some extra time just in case - should be enough to get the messages if a partition isn't paused. */ + await sleep(1000); + expect(messagesConsumed.length).toBe(0); + + consumer.resume([ { topic: topicName } ]); + await waitFor(() => messagesConsumed.length === 3, () => null, 1000); + expect(messagesConsumed.length).toBe(3); + }); +}); \ No newline at end of file From 4f7b7daacb1dbd74e81e1733914e996bda50c104 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 11 Sep 2024 08:53:01 +0530 Subject: [PATCH 183/224] Add performance benchmarking script modes and README * Add performance benchmarking script modes and README * Fix createTopics return * Add topic creation to benchmarks * Remove needless batch size (msgs) 1 * Add performance example to semaphore * Clean up the perf runner script --- .semaphore/semaphore.yml | 32 +++ ci/tests/run_perf_test.sh | 63 +++++ examples/performance/README.md | 39 +++ .../performance/performance-consolidated.js | 24 +- .../performance-primitives-kafkajs.js | 245 ++++++++++++++++++ .../performance/performance-primitives.js | 54 +++- lib/kafkajs/_admin.js | 11 +- 7 files changed, 458 insertions(+), 10 deletions(-) create mode 100755 ci/tests/run_perf_test.sh create mode 100644 examples/performance/README.md create mode 100644 examples/performance/performance-primitives-kafkajs.js diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index e7bb13ad..981c54b7 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -108,6 +108,38 @@ blocks: commands: - npx eslint lib/kafkajs + - name: "Linux amd64: Performance" + dependencies: [ ] + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-4 + env_vars: + - name: TARGET_PRODUCE_PERFORMANCE + value: "35" + - name: TARGET_CONSUME_PERFORMANCE + value: "18" + - name: TARGET_CTP_PERFORMANCE + value: "0.02" + prologue: + commands: + - sudo apt-get install -y librdkafka-dev bc + - export CKJS_LINKING=dynamic + - export BUILD_LIBRDKAFKA=0 + - npm install + - npx node-pre-gyp --build-from-source clean + - npx node-pre-gyp --build-from-source configure + - npx node-pre-gyp --build-from-source build + jobs: + - name: "Performance Test" + commands: + - '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY' + - docker compose up -d && sleep 30 + - export NODE_OPTIONS='--max-old-space-size=1536' + - cd examples/performance + - npm install + - ../../ci/tests/run_perf_tests.sh + - name: "Linux amd64: Release" dependencies: [ ] run: diff --git a/ci/tests/run_perf_test.sh b/ci/tests/run_perf_test.sh new file mode 100755 index 00000000..9dd5176a --- /dev/null +++ b/ci/tests/run_perf_test.sh @@ -0,0 +1,63 @@ +#!/bin/bash + +testresultConfluentProducerConsumer=$(mktemp) +testresultConfluentCtp=$(mktemp) +testresultKjsProducerConsumer=$(mktemp) +testresultKjsCtp=$(mktemp) + +MODE=confluent MESSAGE_COUNT=500000 node performance-consolidated.js --create-topics --consumer --producer 2>&1 | tee "$testresultConfluentProducerConsumer" +MODE=kafkajs MESSAGE_COUNT=500000 node performance-consolidated.js --create-topics --consumer --producer 2>&1 | tee "$testresultKjsProducerConsumer" +MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp 2>&1 | tee "$testresultConfluentCtp" +MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp 2>&1 | tee "$testresultKjsCtp" + +producerConfluent=$(grep "=== Producer Rate:" "$testresultConfluentProducerConsumer" | cut -d':' -f2 | tr -d ' ') +consumerConfluent=$(grep "=== Consumer Rate:" "$testresultConfluentProducerConsumer" | cut -d':' -f2 | tr -d ' ') +ctpConfluent=$(grep "=== Consume-Transform-Produce Rate:" "$testresultConfluentCtp" | cut -d':' -f2 | tr -d ' ') +producerKjs=$(grep "=== Producer Rate:" "$testresultKjsProducerConsumer" | cut -d':' -f2 | tr -d ' ') +consumerKjs=$(grep "=== Consumer Rate:" "$testresultKjsProducerConsumer" | cut -d':' -f2 | tr -d ' ') +ctpKjs=$(grep "=== Consume-Transform-Produce Rate:" "$testresultKjsCtp" | cut -d':' -f2 | tr -d ' ') + +echo "Producer rates: confluent $producerConfluent, kafkajs $producerKjs" +echo "Consumer rates: confluent $consumerConfluent, kafkajs $consumerKjs" +echo "CTP rates: confluent $ctpConfluent, kafkajs $ctpKjs" + +errcode=0 + +# Compare against KJS +if [[ $(echo "$producerConfluent < $producerKjs * 70 / 100" | bc -l) -eq 1 ]]; then + echo "Producer rates differ by more than 30%: confluent $producerConfluent, kafkajs $producerKjs" + errcode=1 +fi + +if [[ $(echo "$consumerConfluent < $consumerKjs * 70 / 100" | bc -l) -eq 1 ]]; then + echo "Consumer rates differ by more than 30%: confluent $consumerConfluent, kafkajs $consumerKjs" + errcode=1 +fi + +if [[ $(echo "$ctpConfluent < $ctpKjs * 70 / 100" | bc -l) -eq 1 ]]; then + echo "CTP rates differ by more than 30%: confluent $ctpConfluent, kafkajs $ctpKjs" + errcode=1 +fi + +# Compare against numbers set within semaphore config +TARGET_PRODUCE="${TARGET_PRODUCE_PERFORMANCE:-35}" +TARGET_CONSUME="${TARGET_CONSUME_PERFORMANCE:-18}" +TARGET_CTP="${TARGET_CTP_PERFORMANCE:-0.02}" + +if [[ $(echo "$producerConfluent < $TARGET_PRODUCE" | bc -l) -eq 1 ]]; then + echo "Confluent producer rate is below target: $producerConfluent" + errcode=1 +fi + +if [[ $(echo "$consumerConfluent < $TARGET_CONSUME" | bc -l) -eq 1 ]]; then + echo "Confluent consumer rate is below target: $consumerConfluent" + errcode=1 +fi + +if [[ $(echo "$ctpConfluent < $TARGET_CTP" | bc -l) -eq 1 ]]; then + echo "Confluent CTP rate is below target: $ctpConfluent" + errcode=1 +fi + +exit $errcode + diff --git a/examples/performance/README.md b/examples/performance/README.md new file mode 100644 index 00000000..5a34d49a --- /dev/null +++ b/examples/performance/README.md @@ -0,0 +1,39 @@ +# Performance Benchmarking + +The library can be benchmarked by running the following command: + +```bash +node performance-consolidated.js [--producer] [--consumer] [--ctp] [--all] +``` + +The `--producer` flag will run the producer benchmark, the `--consumer` flag +will run the consumer benchmark, and the `--ctp` flag will run the +consume-transform-produce benchmark. + +The `--create-topics` flag will create the topics before running the benchmarks +(and delete any existing topics of the same name). It's recommended to use this +unless the number of partitions or replication factor needs to be changed. + +If no flags are provided, no benchmarks will be run. If the `--all` flag is +provided, all benchmarks will be run ignoring any other flags. + +The benchmarks assume topics are already created (unless usig `--create-topics`). +The consumer benchmark assumes that the topic already has at least `MESSAGE_COUNT` messages within, +which can generally be done by running the producer benchmark along with it. + +The following environment variables can be set to configure the benchmark, with +default values given in parentheses. + +| Variable | Description | Default | +|----------|-------------|---------| +| KAFKA_BROKERS | Kafka brokers to connect to | localhost:9092 | +| KAFKA_TOPIC | Kafka topic to produce to/consume from | test-topic | +| KAFKA_TOPIC2 | Kafka topic to produce to after consumption in consume-transform-produce | test-topic2 | +| MESSAGE_COUNT | Number of messages to produce/consume | 1000000 | +| MESSAGE_SIZE | Size of each message in bytes | 256 | +| BATCH_SIZE | Number of messages to produce in a single batch | 100 | +| COMPRESSION | Compression codec to use (None, GZIP, Snappy, LZ4, ZSTD) | None | +| WARMUP_MESSAGES | Number of messages to produce before starting the produce benchmark | BATCH_SIZE * 10 | +| MESSAGE_PROCESS_TIME_MS | Time to sleep after consuming each message in the consume-transform-produce benchmark. Simulates "transform". May be 0. | 5 | +| CONSUME_TRANSFORM_PRODUCE_CONCURRENCY | partitionsConsumedConcurrently for the consume-transform-produce benchmark | 1 | +| MODE | Mode to run the benchmarks in (confluent, kafkajs). Can be used for comparison with KafkaJS | confluent | diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 3ebd4b0c..2d483308 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -1,6 +1,11 @@ -const { runProducer, runConsumer, runConsumeTransformProduce } = require('./performance-primitives'); +const mode = process.env.MODE ? process.env.MODE : 'confluent'; -const { CompressionTypes } = require('../../').KafkaJS; +let runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics; +if (mode === 'confluent') { + ({ runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics } = require('./performance-primitives')); +} else { + ({ runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics } = require('./performance-primitives-kafkajs')); +} const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; const topic = process.env.KAFKA_TOPIC || 'test-topic'; @@ -8,14 +13,25 @@ const topic2 = process.env.KAFKA_TOPIC2 || 'test-topic2'; const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; -const compression = process.env.COMPRESSION || CompressionTypes.NONE; +const compression = process.env.COMPRESSION || 'None'; const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); +const messageProcessTimeMs = process.env.MESSAGE_PROCESS_TIME_MS ? +process.env.MESSAGE_PROCESS_TIME_MS : 5; +const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY : 1; (async function () { const producer = process.argv.includes('--producer'); const consumer = process.argv.includes('--consumer'); const ctp = process.argv.includes('--ctp'); const all = process.argv.includes('--all'); + const createTopics = process.argv.includes('--create-topics'); + + if (createTopics || all) { + console.log("=== Creating Topics (deleting if they exist already):"); + console.log(` Brokers: ${brokers}`); + console.log(` Topic: ${topic}`); + console.log(` Topic2: ${topic2}`); + await runCreateTopics(brokers, topic, topic2); + } if (producer || all) { console.log("=== Running Basic Producer Performance Test:") @@ -48,7 +64,7 @@ const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGE console.log(` Message Count: ${messageCount}`); // Seed the topic with messages await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); - const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, messageCount); + const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, messageCount, messageProcessTimeMs, ctpConcurrency); console.log("=== Consume-Transform-Produce Rate: ", ctpRate); } diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js new file mode 100644 index 00000000..1dba3588 --- /dev/null +++ b/examples/performance/performance-primitives-kafkajs.js @@ -0,0 +1,245 @@ +const { Kafka, CompressionTypes } = require('kafkajs'); +const { randomBytes } = require('crypto'); +const { hrtime } = require('process'); + +module.exports = { + runProducer, + runConsumer, + runConsumeTransformProduce, + runCreateTopics, +}; + +async function runCreateTopics(brokers, topic, topic2) { + const kafka = new Kafka({ + clientId: 'kafka-test-performance', + brokers: brokers.split(','), + }); + + const admin = kafka.admin(); + await admin.connect(); + + for (let t of [topic, topic2]) { + let topicCreated = await admin.createTopics({ + topics: [{ topic: t, numPartitions: 3 }], + }).catch(console.error); + if (topicCreated) { + console.log(`Created topic ${t}`); + continue; + } + + console.log(`Topic ${t} already exists, deleting and recreating.`); + await admin.deleteTopics({ topics: [t] }).catch(console.error); + await new Promise(resolve => setTimeout(resolve, 1000)); /* Propagate. */ + await admin.createTopics({ + topics: [ + { topic: t, numPartitions: 3 }, + ], + }).catch(console.error); + console.log(`Created topic ${t}`); + } + + await admin.disconnect(); +} + +async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { + let totalMessagesSent = 0; + let totalBytesSent = 0; + + const message = { + value: randomBytes(msgSize), + } + + const messages = Array(batchSize).fill(message); + + const kafka = new Kafka({ + clientId: 'kafka-test-performance', + brokers: brokers.split(','), + }); + + const producer = kafka.producer(); + await producer.connect(); + + console.log('Sending ' + warmupMessages + ' warmup messages.'); + while (warmupMessages > 0) { + await producer.send({ + topic, + messages, + compression: CompressionTypes[compression], + }); + warmupMessages -= batchSize; + } + console.log('Sent warmup messages'); + + // Now that warmup is done, start measuring... + let startTime; + let promises = []; + startTime = hrtime(); + let messagesDispatched = 0; + + // The double while-loop allows us to send a bunch of messages and then + // await them all at once. We need the second while loop to keep sending + // in case of queue full errors, which surface only on awaiting. + while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + promises.push(producer.send({ + topic, + messages, + compression: CompressionTypes[compression], + }).then(() => { + totalMessagesSent += batchSize; + totalBytesSent += batchSize * msgSize; + }).catch((err) => { + console.error(err); + throw err; + })); + messagesDispatched += batchSize; + } + await Promise.all(promises); + } + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + let rate = (totalBytesSent / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Sent ${totalMessagesSent} messages, ${totalBytesSent} bytes; rate is ${rate} MB/s`); + + await producer.disconnect(); + return rate; +} + +async function runConsumer(brokers, topic, totalMessageCnt) { + const kafka = new Kafka({ + clientId: 'kafka-test-performance', + brokers: brokers.split(','), + }); + + const consumer = kafka.consumer({ + groupId: 'test-group' + Math.random(), + }); + await consumer.connect(); + await consumer.subscribe({ topic, fromBeginning: true }); + + let messagesReceived = 0; + let totalMessageSize = 0; + let startTime; + let rate; + consumer.run({ + autoCommit: false, + eachMessage: async ({ topic, partition, message }) => { + messagesReceived++; + totalMessageSize += message.value.length; + if (messagesReceived === 1) { + consumer.pause([{ topic }]); + } else if (messagesReceived === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + // } else if (messagesReceived % 100 == 0) { + // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); + } + } + }); + + // Wait until the first message is received + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived > 0) { + clearInterval(interval); + resolve(); + } + }, 100); + }); + + console.log("Starting consumer.") + + totalMessageSize = 0; + startTime = hrtime(); + consumer.resume([{ topic }]); + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived >= totalMessageCnt) { + clearInterval(interval); + resolve(); + } + }, 1000); + }); + + await consumer.disconnect(); + return rate; +} + +async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { + const kafka = new Kafka({ + clientId: 'kafka-test-performance', + brokers: brokers.split(','), + }); + + const producer = kafka.producer({}); + await producer.connect(); + + const consumer = kafka.consumer({ + groupId: 'test-group' + Math.random(), + }); + await consumer.connect(); + await consumer.subscribe({ topic: consumeTopic, fromBeginning: true }); + + let messagesReceived = 0; + let totalMessageSize = 0; + let startTime; + let rate; + consumer.run({ + autoCommit: false, + partitionsConsumedConcurrently: ctpConcurrency, + eachMessage: async ({ topic, partition, message }) => { + /* Simulate message processing for messageProcessTimeMs */ + if (messageProcessTimeMs > 0) { + await new Promise((resolve) => setTimeout(resolve, messageProcessTimeMs)); + } + await producer.send({ + topic: produceTopic, + messages: [{ value: message.value }], + }) + messagesReceived++; + totalMessageSize += message.value.length; + if (messagesReceived === 1) { + consumer.pause([{ topic }]); + } else if (messagesReceived === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd, transformed and sent ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + // } else if (messagesReceived % 1 == 0) { + // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); + } + } + }); + + // Wait until the first message is received + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived > 0) { + clearInterval(interval); + resolve(); + } + }, 100); + }); + + console.log("Starting consume-transform-produce.") + + totalMessageSize = 0; + startTime = hrtime(); + consumer.resume([{ topic: consumeTopic }]); + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesReceived >= totalMessageCnt) { + clearInterval(interval); + resolve(); + } + }, 1000); + }); + + await consumer.disconnect(); + await producer.disconnect(); + return rate; +} diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index af5a0e64..4de54ac4 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -1,4 +1,4 @@ -const { Kafka, ErrorCodes } = require('../../').KafkaJS; +const { Kafka, ErrorCodes, CompressionTypes } = require('../../').KafkaJS; const { randomBytes } = require('crypto'); const { hrtime } = require('process'); @@ -6,8 +6,42 @@ module.exports = { runProducer, runConsumer, runConsumeTransformProduce, + runCreateTopics, }; +async function runCreateTopics(brokers, topic, topic2) { + const kafka = new Kafka({ + 'client.id': 'kafka-test-performance', + "metadata.broker.list": brokers, + }); + + const admin = kafka.admin(); + await admin.connect(); + + for (let t of [topic, topic2]) { + let topicCreated = await admin.createTopics({ + topics: [{ topic: t, numPartitions: 3 }], + }).catch(console.error); + if (topicCreated) { + console.log(`Created topic ${t}`); + continue; + } + + console.log(`Topic ${t} already exists, deleting and recreating.`); + await admin.deleteTopics({ topics: [t] }).catch(console.error); + await new Promise(resolve => setTimeout(resolve, 1000)); /* Propagate. */ + await admin.createTopics({ + topics: [ + { topic: t, numPartitions: 3 }, + ], + }).catch(console.error); + console.log(`Created topic ${t}`); + await new Promise(resolve => setTimeout(resolve, 1000)); /* Propagate. */ + } + + await admin.disconnect(); +} + async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { let totalMessagesSent = 0; let totalBytesSent = 0; @@ -21,7 +55,7 @@ async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessa const kafka = new Kafka({ 'client.id': 'kafka-test-performance', 'metadata.broker.list': brokers, - 'compression.codec': compression, + 'compression.codec': CompressionTypes[compression], }); const producer = kafka.producer(); @@ -140,7 +174,7 @@ async function runConsumer(brokers, topic, totalMessageCnt) { return rate; } -async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, totalMessageCnt) { +async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { const kafka = new Kafka({ 'client.id': 'kafka-test-performance', 'metadata.broker.list': brokers, @@ -156,6 +190,15 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t 'group.id': 'test-group' + Math.random(), 'enable.auto.commit': false, 'auto.offset.reset': 'earliest', + + /* These fields are more-or-less required for cases where eachMessage includes + * any async operatiosn, else `partitionsConsumedConcurrently` does not have + * much effect. Reason for this is that, internally, librdkafka fetches + * a large number of messages from one topic partition and that fills the + * cache up, and we end up underutilizing concurrency. + * TODO: remove or change these, discuss this issue and make changes in the code. */ + 'message.max.bytes': 1000, + 'fetch.max.bytes': 1000, }); await consumer.connect(); await consumer.subscribe({ topic: consumeTopic }); @@ -165,7 +208,12 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t let startTime; let rate; consumer.run({ + partitionsConsumedConcurrently: ctpConcurrency, eachMessage: async ({ topic, partition, message }) => { + /* Simulate message processing for messageProcessTimeMs */ + if (messageProcessTimeMs > 0) { + await new Promise((resolve) => setTimeout(resolve, messageProcessTimeMs)); + } await producer.send({ topic: produceTopic, messages: [{ value: message.value }], diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index bd8c15f8..fe642828 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -4,6 +4,7 @@ const { kafkaJSToRdKafkaConfig, DefaultLogger, CompatibilityErrorMessages, logLevel, + loggerTrampoline, checkAllowedKeys } = require('./_common'); const error = require('./_error'); @@ -144,6 +145,7 @@ class Admin { this.#internalClient = RdKafka.AdminClient.create(config, { 'error': this.#errorCb.bind(this), 'ready': this.#readyCb.bind(this), + 'event.log': (msg) => loggerTrampoline(msg, this.#logger) }); } catch (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); @@ -208,6 +210,7 @@ class Admin { * Create topics with the given configuration. * @param {{ validateOnly?: boolean, waitForLeaders?: boolean, timeout?: number, topics: import("../../types/kafkajs").ITopicConfig[] }} options * @returns {Promise} Resolves true when the topics are created, false if topic exists already, rejects on error. + * In case even one topic already exists, this will return false. */ async createTopics(options) { if (this.#state !== AdminState.CONNECTED) { @@ -223,6 +226,7 @@ class Admin { } /* Convert each topic to a format suitable for node-rdkafka, and dispatch the call. */ + let allTopicsCreated = true; const ret = options.topics .map(this.#topicConfigToRdKafka) @@ -230,17 +234,18 @@ class Admin { this.#internalClient.createTopic(topicConfig, options.timeout ?? 5000, (err) => { if (err) { if (err.code === error.ErrorCodes.ERR_TOPIC_ALREADY_EXISTS) { - resolve(false); + allTopicsCreated = false; + resolve(); return; } reject(createKafkaJsErrorFromLibRdKafkaError(err)); } else { - resolve(true); + resolve(); } }); })); - return Promise.all(ret); + return Promise.all(ret).then(() => allTopicsCreated); } /** From de76f03e3beb8895eaefa7d3d355f5cbe31631e3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 11 Sep 2024 13:48:41 +0530 Subject: [PATCH 184/224] Add confluent debian repo for performance benchmark --- .semaphore/semaphore.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 981c54b7..9de6e380 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -123,6 +123,10 @@ blocks: value: "0.02" prologue: commands: + - wget -qO - https://packages.confluent.io/deb/7.7/archive.key | sudo apt-key add - + - sudo add-apt-repository "deb https://packages.confluent.io/clients/deb $(lsb_release -cs) main" + - sudo apt-get update + - sudo apt-get install -y build-essential gcc g++ make python3 - sudo apt-get install -y librdkafka-dev bc - export CKJS_LINKING=dynamic - export BUILD_LIBRDKAFKA=0 From 52a6ccb8a1c3e1dc4311de84fe5bbdf939ac94c3 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 12 Aug 2024 12:00:22 +0530 Subject: [PATCH 185/224] Remove store from promisified API --- deps/librdkafka | 2 +- lib/kafkajs/_consumer.js | 65 ++----- test/promisified/consumer/store.spec.js | 221 ------------------------ 3 files changed, 14 insertions(+), 274 deletions(-) delete mode 100644 test/promisified/consumer/store.spec.js diff --git a/deps/librdkafka b/deps/librdkafka index 2587cac7..6eaf89fb 160000 --- a/deps/librdkafka +++ b/deps/librdkafka @@ -1 +1 @@ -Subproject commit 2587cac70f83fced42c51f921bed325a434f5bc7 +Subproject commit 6eaf89fb124c421b66b43b195879d458a3a31f86 diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 165c1a13..778d4179 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -125,11 +125,6 @@ class Consumer { */ #messageCache = null; - /** - * Whether the user has enabled manual offset management (stores). - */ - #userManagedStores = false; - /** * Whether the user has enabled manual offset management (commits). */ @@ -536,20 +531,14 @@ class Consumer { rdKafkaConfig['offset_commit_cb'] = true; rdKafkaConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); - /* Offset management is different from case to case. - * Case 1: User has changed value of enable.auto.offset.store. In this case, we respect that. - * Case 2: automatic committing is on. In this case, we turn off auto.offset.store and store offsets manually. - * this is necessary for cache invalidation and management, as we want to put things into the store - * after eachMessage is called, and not on consume itself. - * Case 3: automatic committing is off. In this case, we turn off auto.offset.store too. Since the user might - * call an empty commit() and expect things to work properly (ie. the right offsets be stored). - * All this works out a singular, simple condition. - */ - if (!Object.hasOwn(this.#userConfig, 'enable.auto.offset.store')) { - rdKafkaConfig['enable.auto.offset.store'] = false; - } else { - this.#userManagedStores = !rdKafkaConfig['enable.auto.offset.store']; + /* We handle offset storage within the promisified API by ourselves. Thus we don't allow the user to change this + * setting and set it to false. */ + if (Object.hasOwn(this.#userConfig, 'enable.auto.offset.store')) { + throw new error.KafkaJSError( + "Changing 'enable.auto.offset.store' is unsupported while using the promisified API.", + { code: error.ErrorCodes.ERR__INVALID_ARG }); } + rdKafkaConfig['enable.auto.offset.store'] = false; if (!Object.hasOwn(rdKafkaConfig, 'enable.auto.commit')) { this.#autoCommit = true; /* librdkafka default. */ @@ -663,13 +652,11 @@ class Consumer { payload._lastResolvedOffset = { offset, leaderEpoch }; try { - if (!this.#userManagedStores) { - this.#internalClient._offsetsStoreSingle( - topic, - partition, - offset + 1, - leaderEpoch); - } + this.#internalClient._offsetsStoreSingle( + topic, + partition, + offset + 1, + leaderEpoch); this.#lastConsumedOffsets.set(key, offset + 1); } catch (e) { /* Not much we can do, except log the error. */ @@ -1076,9 +1063,7 @@ class Consumer { /* Store the offsets we need to store, or at least record them for cache invalidation reasons. */ if (eachMessageProcessed) { try { - if (!this.#userManagedStores) { - this.#internalClient._offsetsStoreSingle(m.topic, m.partition, Number(m.offset) + 1, m.leaderEpoch); - } + this.#internalClient._offsetsStoreSingle(m.topic, m.partition, Number(m.offset) + 1, m.leaderEpoch); this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); } catch (e) { /* Not much we can do, except log the error. */ @@ -1299,30 +1284,6 @@ class Consumer { // return m ?? null; } - /** - * Store offsets for the given topic partitions. - * - * Stored offsets will be commited automatically at a later point if autoCommit is enabled. - * Otherwise, they will be committed when commitOffsets is called without arguments. - * - * enable.auto.offset.store must be set to false to use this API. - * @param {import("../../types/kafkajs").TopicPartitionOffset[]?} topicPartitions - */ - storeOffsets(topicPartitions) { - if (this.#state !== ConsumerState.CONNECTED) { - throw new error.KafkaJSError('Store can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); - } - - if (!this.#userManagedStores) { - throw new error.KafkaJSError( - 'Store can only be called when enable.auto.offset.store is explicitly set to false.', { code: error.ErrorCodes.ERR__INVALID_ARG }); - } - - const topicPartitionsRdKafka = topicPartitions.map( - topicPartitionOffsetMetadataToRdKafka); - this.#internalClient.offsetsStore(topicPartitionsRdKafka); - } - async #commitOffsetsUntilNoStateErr(offsetsToCommit) { let err = { code: error.ErrorCodes.ERR_NO_ERROR }; do { diff --git a/test/promisified/consumer/store.spec.js b/test/promisified/consumer/store.spec.js deleted file mode 100644 index 64d67ffd..00000000 --- a/test/promisified/consumer/store.spec.js +++ /dev/null @@ -1,221 +0,0 @@ -jest.setTimeout(30000); - -const { - secureRandom, - createTopic, - waitFor, - createProducer, - createConsumer, - sleep, -} = require('../testhelpers'); -const { ErrorCodes } = require('../../../lib').KafkaJS; - -describe.each([[false], [true]])('Consumer store', (isAutoCommit) => { - let topicName, groupId, producer, consumer; - - beforeEach(async () => { - topicName = `test-topic-${secureRandom()}`; - groupId = `consumer-group-id-${secureRandom()}`; - - await createTopic({ topic: topicName, partitions: 3 }); - - producer = createProducer({}); - - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - autoCommit: isAutoCommit, - autoCommitInterval: 500, - }, { - 'enable.auto.offset.store': false, - }); - }); - - afterEach(async () => { - consumer && (await consumer.disconnect()); - producer && (await producer.disconnect()); - }); - - it('should not work if enable.auto.offset.store = true', async () => { - let assignment = []; - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - }, { - /* Set to true manually - the default value with kafkaJS block is false. */ - 'enable.auto.offset.store': true, - 'rebalance_cb': function (err, asg) { - if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { - assignment = asg; - } - } - }); - - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - await consumer.run({ - eachMessage: async () => { - } - }); - await waitFor(() => assignment.length > 0, () => null, 1000); - expect( - () => consumer.storeOffsets([{ topic: topicName, partition: 0, offset: '10' }]) - ).toThrow(/Store can only be called when enable.auto.offset.store is explicitly set to false/); - }); - - it('should not work if enable.auto.offset.store is unset', async () => { - let assignment = []; - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - }, { - /* Set to true manually - the default value with kafkaJS block is false. */ - 'rebalance_cb': function (err, asg) { - if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { - assignment = asg; - } - } - }); - - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - await consumer.run({ - eachMessage: async () => { - } - }); - await waitFor(() => assignment.length > 0, () => null, 1000); - expect( - () => consumer.storeOffsets([{ topic: topicName, partition: 0, offset: '10' }]) - ).toThrow(/Store can only be called when enable.auto.offset.store is explicitly set to false/); - }); - - it('should commit stored offsets', async () => { - /* Evenly distribute 30 messages across 3 partitions */ - let i = 0; - const messages = Array(3 * 10) - .fill() - .map(() => { - const value = secureRandom(); - return { value: `value-${value}`, partition: (i++) % 3 }; - }); - - await producer.connect(); - await producer.send({ topic: topicName, messages }); - await producer.flush(); - - let msgCount = 0; - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - await consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - msgCount++; - const offset = (Number(message.offset) + 1).toString(); - expect(() => consumer.storeOffsets([{ topic, partition, offset }])).not.toThrow(); - } - }); - await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); - expect(msgCount).toEqual(30); - - if (!isAutoCommit) - await expect(consumer.commitOffsets()).resolves.toBeUndefined(); - else - await sleep(1000); /* Wait for auto-commit */ - - await consumer.disconnect(); - - /* Send 30 more messages */ - await producer.send({ topic: topicName, messages }); - await producer.flush(); - - - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - }); - - msgCount = 0; - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - await consumer.run({ - eachMessage: async () => { - msgCount++; - } - }); - /* Only the extra 30 messages should come to us */ - await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); - await sleep(1000); - expect(msgCount).toEqual(30); - }); - - it('should commit stored offsets with metadata', async () => { - /* Evenly distribute 30 messages across 3 partitions */ - let i = 0; - const messages = Array(3 * 10) - .fill() - .map(() => { - const value = secureRandom(); - return { value: `value-${value}`, partition: (i++) % 3 }; - }); - - await producer.connect(); - await producer.send({ topic: topicName, messages }); - await producer.flush(); - - let msgCount = 0; - const metadata = 'unicode-metadata-😊'; - await consumer.connect(); - await consumer.subscribe({ topic: topicName }); - await consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - msgCount++; - const offset = (Number(message.offset) + 1).toString(); - const leaderEpoch = message.leaderEpoch; - expect(() => consumer.storeOffsets([{ topic, partition, offset, metadata, leaderEpoch }])).not.toThrow(); - } - }); - await waitFor(() => msgCount >= 30, () => null, { delay: 100 }); - expect(msgCount).toEqual(30); - - if (!isAutoCommit) - await expect(consumer.commitOffsets()).resolves.toBeUndefined(); - else - await sleep(1000); /* Wait for auto-commit */ - - let committed = await consumer.committed(null, 5000); - expect(committed).toEqual([ - { topic: topicName, partition: 0, offset: '10', metadata, leaderEpoch: expect.any(Number) }, - { topic: topicName, partition: 1, offset: '10', metadata, leaderEpoch: expect.any(Number) }, - { topic: topicName, partition: 2, offset: '10', metadata, leaderEpoch: expect.any(Number) } - ]); - - await consumer.disconnect(); - - consumer = createConsumer({ - groupId, - maxWaitTimeInMs: 100, - fromBeginning: true, - }); - - msgCount = 0; - await consumer.connect(); - await consumer.subscribe({ topic: 'not-a-real-topic-name' }); - - /* At this point, we're not actually assigned anything, but we should be able to fetch - * the stored offsets and metadata anyway since we're of the same consumer group. */ - committed = await consumer.committed([ - { topic: topicName, partition: 0 }, - { topic: topicName, partition: 1 }, - { topic: topicName, partition: 2 } - ]); - expect(committed).toEqual([ - { topic: topicName, partition: 0, offset: '10', metadata, leaderEpoch: expect.any(Number) }, - { topic: topicName, partition: 1, offset: '10', metadata, leaderEpoch: expect.any(Number) }, - { topic: topicName, partition: 2, offset: '10', metadata, leaderEpoch: expect.any(Number) } - ]); - }); - -}); From 9bef0d81c1401634234d4b7b68d6fa35806ada84 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 11 Sep 2024 08:54:39 +0530 Subject: [PATCH 186/224] Add binding level debug logging and client name to logs * Add binding level debug logging and client name to logs * Add C++ changes for binding-level logging --- lib/kafkajs/_admin.js | 38 +++++++++++++++++++-- lib/kafkajs/_common.js | 23 ++++++++++--- lib/kafkajs/_consumer.js | 74 +++++++++++++++++++++++++++++----------- lib/kafkajs/_producer.js | 29 ++++++++++++++-- src/admin.cc | 3 ++ src/callbacks.cc | 8 ++++- src/callbacks.h | 2 ++ src/kafka-consumer.cc | 3 ++ src/producer.cc | 3 ++ 9 files changed, 152 insertions(+), 31 deletions(-) diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index fe642828..83a177ff 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -3,9 +3,12 @@ const { kafkaJSToRdKafkaConfig, createKafkaJsErrorFromLibRdKafkaError, DefaultLogger, CompatibilityErrorMessages, + createBindingMessageMetadata, logLevel, + checkAllowedKeys, loggerTrampoline, - checkAllowedKeys } = require('./_common'); + severityToLogLevel, +} = require('./_common'); const error = require('./_error'); /** @@ -57,6 +60,20 @@ class Admin { */ #connectPromiseFunc = null; + /** + * The client name used by the admin client for logging - determined by librdkafka + * using a combination of clientId and an integer. + * @type {string|undefined} + */ + #clientName = undefined; + + /** + * Convenience function to create the metadata object needed for logging. + */ + #createAdminBindingMessageMetadata() { + return createBindingMessageMetadata(this.#clientName); + } + /** * @constructor * @param {import("../../types/kafkajs").AdminConstructorConfig} config @@ -82,6 +99,17 @@ class Admin { } const rdKafkaConfig = kafkaJSToRdKafkaConfig(kjsConfig); + + /* Set the logger */ + if (Object.hasOwn(kjsConfig, 'logger')) { + this.#logger = kjsConfig.logger; + } + + /* Set the log level - INFO for compatibility with kafkaJS, or DEBUG if that is turned + * on using the logLevel property. rdKafkaConfig.log_level is guaranteed to be set if we're + * here, and containing the correct value. */ + this.#logger.setLogLevel(severityToLogLevel[rdKafkaConfig.log_level]); + return rdKafkaConfig; } @@ -121,7 +149,7 @@ class Admin { if (this.#state === AdminState.CONNECTING) { this.#connectPromiseFunc['reject'](err); } else { - this.#logger.error(err); + this.#logger.error(`Error: ${err.message}`, this.#createAdminBindingMessageMetadata()); } } @@ -145,8 +173,11 @@ class Admin { this.#internalClient = RdKafka.AdminClient.create(config, { 'error': this.#errorCb.bind(this), 'ready': this.#readyCb.bind(this), - 'event.log': (msg) => loggerTrampoline(msg, this.#logger) + 'event.log': (msg) => loggerTrampoline(msg, this.#logger), }); + + this.#clientName = this.#internalClient.name; + this.#logger.info("Admin client connected", this.#createAdminBindingMessageMetadata()); } catch (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); } @@ -174,6 +205,7 @@ class Admin { /* AdminClient disconnect for node-rdkakfa is synchronous. */ this.#internalClient.disconnect(); this.#state = AdminState.DISCONNECTED; + this.#logger.info("Admin client disconnected", this.#createAdminBindingMessageMetadata()); resolve(); } catch (err) { reject(createKafkaJsErrorFromLibRdKafkaError(err)); diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 6e33a886..39cfcc39 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -113,6 +113,20 @@ class DefaultLogger { } } +/** + * Convenience function to create a new object to be used as metadata for log messages. + * Returned object is intended to be used immediately and not stored. + * + * @param {string|undefined} clientName + */ +function createBindingMessageMetadata(clientName) { + return { + name: clientName, + fac: 'BINDING', + timestamp: Date.now(), + }; +} + /** * Trampoline for user defined logger, if any. * @param {{severity: number, fac: string, message: string}} msg @@ -128,16 +142,16 @@ function loggerTrampoline(msg, logger) { case logLevel.NOTHING: break; case logLevel.ERROR: - logger.error(msg.message, { fac: msg.fac, timestamp: Date.now() }); + logger.error(msg.message, { fac: msg.fac, timestamp: Date.now(), name: msg.name }); break; case logLevel.WARN: - logger.warn(msg.message, { fac: msg.fac, timestamp: Date.now() }); + logger.warn(msg.message, { fac: msg.fac, timestamp: Date.now(), name: msg.name }); break; case logLevel.INFO: - logger.info(msg.message, { fac: msg.fac, timestamp: Date.now() }); + logger.info(msg.message, { fac: msg.fac, timestamp: Date.now(), name: msg.name }); break; case logLevel.DEBUG: - logger.debug(msg.message, { fac: msg.fac, timestamp: Date.now() }); + logger.debug(msg.message, { fac: msg.fac, timestamp: Date.now(), name: msg.name }); break; default: throw new error.KafkaJSError("Invalid logLevel", { @@ -706,6 +720,7 @@ module.exports = { topicPartitionOffsetMetadataToKafkaJS, createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders, + createBindingMessageMetadata, notImplemented, logLevel, loggerTrampoline, diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 778d4179..fca14570 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -6,6 +6,7 @@ const { topicPartitionOffsetToRdKafka, topicPartitionOffsetMetadataToRdKafka, topicPartitionOffsetMetadataToKafkaJS, + createBindingMessageMetadata, createKafkaJsErrorFromLibRdKafkaError, notImplemented, loggerTrampoline, @@ -171,6 +172,20 @@ class Consumer { */ #topicPartitionToBatchPayload = new Map(); + /** + * The client name used by the consumer for logging - determined by librdkafka + * using a combination of clientId and an integer. + * @type {string|undefined} + */ + #clientName = undefined; + + /** + * Convenience function to create the metadata object needed for logging. + */ + #createConsumerBindingMessageMetadata() { + return createBindingMessageMetadata(this.#clientName); + } + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -222,8 +237,8 @@ class Consumer { } /* TODO: we should cry more about this and render the consumer unusable. */ - await Promise.all(seekPromises).catch(err => this.#logger.error("Seek error. This is effectively a fatal error:" + err)); - + await Promise.all(seekPromises) + .catch(err => this.#logger.error(`Seek error. This is effectively a fatal error: ${err}`), this.#createConsumerBindingMessageMetadata()); /* Clear the cache and stored offsets. * We need to do this only if topicPartitions = null (global cache expiry). @@ -242,6 +257,9 @@ class Consumer { #rebalanceCallback(err, assignment) { err = LibrdKafkaError.create(err); const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; + this.#logger.info( + `Received rebalance event with message: '${err.message}' and ${assignment.length} partition(s)`, + this.#createConsumerBindingMessageMetadata()); let assignmentFnCalled = false; function assignmentFn(userAssignment) { @@ -559,6 +577,9 @@ class Consumer { /* Slight optimization for cases where the size of messages in our subscription is less than the cache size. */ this.#internalClient.setDefaultIsTimeoutOnlyForFirstMessage(true); + this.#clientName = this.#internalClient.name; + this.#logger.info('Consumer connected', this.#createConsumerBindingMessageMetadata()); + // Resolve the promise. this.#connectPromiseFunc['resolve'](); } @@ -571,7 +592,7 @@ class Consumer { if (this.#state === ConsumerState.CONNECTING) { this.#connectPromiseFunc['reject'](err); } else { - this.#logger.error(err); + this.#logger.error(err, this.#createConsumerBindingMessageMetadata()); } } @@ -660,8 +681,7 @@ class Consumer { this.#lastConsumedOffsets.set(key, offset + 1); } catch (e) { /* Not much we can do, except log the error. */ - if (this.#logger) - this.#logger.error(`Consumer encountered error while storing offset. Error details: ${e}:${e.stack}`); + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${e}:${e.stack}`, this.#createConsumerBindingMessageMetadata()); } } @@ -782,6 +802,7 @@ class Consumer { } this.#fetchInProgress = true; + this.#logger.debug(`Attempting to fetch ${this.#messageCache.maxSize} messages to the message cache`, this.#createConsumerBindingMessageMetadata()); return new Promise((resolve, reject) => { this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { this.#fetchInProgress = false; @@ -828,6 +849,7 @@ class Consumer { } this.#fetchInProgress = true; + this.#logger.debug(`Attempting to fetch ${this.#messageCache.maxSize} messages to the message cache`, this.#createConsumerBindingMessageMetadata()); return new Promise((resolve, reject) => { this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { this.#fetchInProgress = false; @@ -968,6 +990,7 @@ class Consumer { }); this.#storedSubscriptions = subscription.replace ? topics : this.#storedSubscriptions.concat(topics); + this.#logger.debug(`${subscription.replace ? 'Replacing' : 'Adding'} topics [${topics.join(', ')}] to subscription`, this.#createConsumerBindingMessageMetadata()); this.#internalClient.subscribe(this.#storedSubscriptions); } @@ -1043,12 +1066,12 @@ class Consumer { * However, we don't do this inside the catch, but just outside it. This is because throwing an * error is not the only case where we might want to seek back. * - * So - do nothing but a debug log, but at this point eachMessageProcessed is false. + * So - do nothing but a log, but at this point eachMessageProcessed is false. + * TODO: log error only if error type is not KafkaJSError and if no pause() has been called, else log debug. */ - this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); - - /* TODO: log error if error type is not KafkaJSError and if no pause() has been called */ - this.#logger.error(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); + this.#logger.error( + `Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`, + this.#createConsumerBindingMessageMetadata()); } /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ @@ -1067,8 +1090,7 @@ class Consumer { this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); } catch (e) { /* Not much we can do, except log the error. */ - if (this.#logger) - this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`); + this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`, this.#createConsumerBindingMessageMetadata()); } } @@ -1123,13 +1145,13 @@ class Consumer { * if the user has not called `resolveOffset` manually in case of using eachBatch without * eachBatchAutoResolve being set. * - * So - do nothing but a debug log, but at this point eachMessageProcessed needs to be false unless + * So - do nothing but a log, but at this point eachMessageProcessed needs to be false unless * the user has explicitly marked it as true. + * TODO: log error only if error type is not KafkaJSError and if no pause() has been called, else log debug. */ - this.#logger.debug(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); - - /* TODO: log error if error type is not KafkaJSError and if no pause() has been called */ - this.#logger.error(`Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`); + this.#logger.error( + `Consumer encountered error while processing message. Error details: ${e}: ${e.stack}. The same message may be reprocessed.`, + this.#createConsumerBindingMessageMetadata()); /* The value of eachBatchAutoResolve is not important. The only place where a message is marked processed * despite an error is if the user says so, and the user can use resolveOffset for both the possible @@ -1177,6 +1199,7 @@ class Consumer { /* Invalidate the message cache if needed */ const locallyStale = this.#messageCache.popLocallyStale(); if (this.#messageCache.isStale()) { + this.#logger.debug("Scheduling worker termination", this.#createConsumerBindingMessageMetadata()); this.#workerTerminationScheduled = true; break; } else if (locallyStale.length !== 0) { @@ -1191,7 +1214,7 @@ class Consumer { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ if (this.#logger) - this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${e} : ${e.stack}`); + this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${e} : ${e.stack}`, this.#createConsumerBindingMessageMetadata()); }); nextIdx = -1; @@ -1229,6 +1252,7 @@ class Consumer { while (!this.#disconnectStarted) { this.#workerTerminationScheduled = false; const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); + this.#logger.debug(`Spawning ${workersToSpawn} workers`, this.#createConsumerBindingMessageMetadata()); this.#workers = Array(workersToSpawn) .fill() @@ -1236,7 +1260,7 @@ class Consumer { this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this)) .catch(e => { if (this.#logger) - this.#logger.error(`Worker ${i} encountered an error: ${e}:${e.stack}`); + this.#logger.error(`Worker ${i} encountered an error: ${e}:${e.stack}`, this.#createConsumerBindingMessageMetadata()); })); /* Best we can do is log errors on worker issues - handled by the catch block above. */ @@ -1427,7 +1451,10 @@ class Consumer { /* 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. */ /* TODO: is it worth awaiting seeks to finish? */ - this.#internalClient.seek(topicPartitionOffset, 0, err => err ? this.#logger.error(err) : null); + this.#internalClient.seek(topicPartitionOffset, 0, err => { + if (err) + this.#logger.error(`Error while calling seek from within seekInternal: ${err}`, this.#createConsumerBindingMessageMetadata()); + }); offsetsToCommit.push({ topic: topicPartition.topic, partition: topicPartition.partition, @@ -1539,6 +1566,8 @@ class Consumer { throw new error.KafkaJSError('Pause can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } + this.#logger.debug(`Pausing ${topics.length} topics`, this.#createConsumerBindingMessageMetadata()); + const toppars = []; for (let topic of topics) { if (typeof topic.topic !== 'string') { @@ -1612,6 +1641,8 @@ class Consumer { throw new error.KafkaJSError('Resume can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } + this.#logger.debug(`Resuming ${topics.length} topics`, this.#createConsumerBindingMessageMetadata()); + const toppars = []; for (let topic of topics) { if (typeof topic.topic !== 'string') { @@ -1677,6 +1708,8 @@ class Consumer { this.#disconnectStarted = true; this.#workerTerminationScheduled = true; + + this.#logger.debug("Signalling disconnection attempt to workers", this.#createConsumerBindingMessageMetadata()); while (!(await acquireOrLog(this.#lock, this.#logger))); /* Just retry... */ this.#state = ConsumerState.DISCONNECTING; @@ -1691,6 +1724,7 @@ class Consumer { return; } this.#state = ConsumerState.DISCONNECTED; + this.#logger.info("Consumer disconnected", this.#createConsumerBindingMessageMetadata()); resolve(); }; this.#internalClient.disconnect(cb); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 773a3186..9c59ab3f 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -3,6 +3,7 @@ const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders, + createBindingMessageMetadata, DefaultLogger, loggerTrampoline, severityToLogLevel, @@ -83,6 +84,20 @@ class Producer { this.#userConfig = kJSConfig; } + /** + * The client name used by the producer for logging - determined by librdkafka + * using a combination of clientId and an integer. + * @type {string|undefined} + */ + #clientName = undefined; + + /** + * Convenience function to create the metadata object needed for logging. + */ + #createProducerBindingMessageMetadata() { + return createBindingMessageMetadata(this.#clientName); + } + #config() { if (!this.#internalConfig) this.#internalConfig = this.#finalizedConfig(); @@ -258,16 +273,19 @@ class Producer { } const rdKafkaConfig = this.#config(); + this.#clientName = this.#internalClient.name; + if (Object.hasOwn(rdKafkaConfig, 'transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { this.#state = ProducerState.INITIALIZING_TRANSACTIONS; + this.#logger.debug("Attempting to initialize transactions", this.#createProducerBindingMessageMetadata()); this.#internalClient.initTransactions(5000 /* default: 5s */, this.#readyTransactions.bind(this)); return; } this.#state = ProducerState.CONNECTED; - this.#internalClient.setPollInBackground(true); this.#internalClient.on('delivery-report', this.#deliveryCallback.bind(this)); + this.#logger.info("Producer connected", this.#createProducerBindingMessageMetadata()); // Resolve the promise. this.#connectPromiseFunc["resolve"](); @@ -281,7 +299,7 @@ class Producer { if (this.#state === ProducerState.CONNECTING) { this.#connectPromiseFunc["reject"](err); } else { - this.#logger.error(err); + this.#logger.error(err, this.#createProducerBindingMessageMetadata()); } } @@ -339,6 +357,7 @@ class Producer { return; } this.#state = ProducerState.DISCONNECTED; + this.#logger.info("Producr disconnected", this.#createProducerBindingMessageMetadata()); resolve(); }; this.#internalClient.disconnect(5000 /* default timeout, 5000ms */, cb); @@ -358,6 +377,7 @@ class Producer { throw new error.KafkaJSError("Can only start one transaction at a time.", { code: error.ErrorCodes.ERR__STATE }); } + this.#logger.debug("Attempting to begin transaction", this.#createProducerBindingMessageMetadata()); return new Promise((resolve, reject) => { this.#internalClient.beginTransaction((err) => { if (err) { @@ -387,6 +407,7 @@ class Producer { throw new error.KafkaJSError("Cannot commit, no transaction ongoing.", { code: error.ErrorCodes.ERR__STATE }); } + this.#logger.debug("Attempting to commit transaction", this.#createProducerBindingMessageMetadata()); return new Promise((resolve, reject) => { this.#internalClient.commitTransaction(5000 /* default: 5000ms */, err => { if (err) { @@ -413,6 +434,7 @@ class Producer { throw new error.KafkaJSError("Cannot abort, no transaction ongoing.", { code: error.ErrorCodes.ERR__STATE }); } + this.#logger.debug("Attempting to abort transaction", this.#createProducerBindingMessageMetadata()); return new Promise((resolve, reject) => { this.#internalClient.abortTransaction(5000 /* default: 5000ms */, err => { if (err) { @@ -660,7 +682,7 @@ class Producer { return; } - + this.#logger.info("Setting SASL credentials", this.#createProducerBindingMessageMetadata()); this.#internalClient.setSaslCredentials(args.username, args.password); } @@ -698,6 +720,7 @@ class Producer { throw new error.KafkaJSError("timeout must be set for flushing", { code: error.ErrorCodes.ERR__INVALID_ARG }); } + this.#logger.debug(`Attempting to flush messages for ${args.timeout}ms`, this.#createProducerBindingMessageMetadata()); return new Promise((resolve, reject) => { this.#internalClient.flush(args.timeout, (err) => { if (err) { diff --git a/src/admin.cc b/src/admin.cc index d485891c..608cd756 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -65,6 +65,9 @@ Baton AdminClient::Connect() { return Baton(RdKafka::ERR__STATE, errstr); } + /* Set the client name at the first possible opportunity for logging. */ + m_event_cb.dispatcher.SetClientName(m_client->name()); + if (rkqu == NULL) { rkqu = rd_kafka_queue_new(m_client->c_ptr()); } diff --git a/src/callbacks.cc b/src/callbacks.cc index d032a312..faebcec4 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -181,7 +181,7 @@ void Event::event_cb(RdKafka::Event &event) { dispatcher.Execute(); } -EventDispatcher::EventDispatcher() {} +EventDispatcher::EventDispatcher() : client_name("") {} EventDispatcher::~EventDispatcher() {} void EventDispatcher::Add(const event_t &e) { @@ -232,6 +232,8 @@ void EventDispatcher::Flush() { Nan::New(_events[i].fac.c_str()).ToLocalChecked()); Nan::Set(jsobj, Nan::New("message").ToLocalChecked(), Nan::New(_events[i].message.c_str()).ToLocalChecked()); + Nan::Set(jsobj, Nan::New("name").ToLocalChecked(), + Nan::New(this->client_name.c_str()).ToLocalChecked()); break; case RdKafka::Event::EVENT_THROTTLE: @@ -266,6 +268,10 @@ void EventDispatcher::Flush() { } } +void EventDispatcher::SetClientName(const std::string& client_name) { + this->client_name = client_name; +} + DeliveryReportDispatcher::DeliveryReportDispatcher() {} DeliveryReportDispatcher::~DeliveryReportDispatcher() {} diff --git a/src/callbacks.h b/src/callbacks.h index 1775c401..f69c0685 100644 --- a/src/callbacks.h +++ b/src/callbacks.h @@ -79,8 +79,10 @@ class EventDispatcher : public Dispatcher { ~EventDispatcher(); void Add(const event_t &); void Flush(); + void SetClientName(const std::string &); protected: std::vector events; + std::string client_name; }; class Event : public RdKafka::EventCb { diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index 32c963f4..d93a49c4 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -65,6 +65,9 @@ Baton KafkaConsumer::Connect() { return Baton(RdKafka::ERR__STATE, errstr); } + /* Set the client name at the first possible opportunity for logging. */ + m_event_cb.dispatcher.SetClientName(m_client->name()); + baton = setupSaslOAuthBearerBackgroundQueue(); if (baton.err() != RdKafka::ERR_NO_ERROR) { return baton; diff --git a/src/producer.cc b/src/producer.cc index c5d1f23a..aeab634e 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -189,6 +189,9 @@ Baton Producer::Connect() { return Baton(RdKafka::ERR__STATE, errstr); } + /* Set the client name at the first possible opportunity for logging. */ + m_event_cb.dispatcher.SetClientName(m_client->name()); + baton = setupSaslOAuthBearerBackgroundQueue(); return baton; } From 99d0c5eed03d248b549a58ee2e3d5cdc18aca535 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 11 Sep 2024 13:58:14 +0530 Subject: [PATCH 187/224] Fix typo in script name --- .semaphore/semaphore.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 9de6e380..c977e28f 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -142,7 +142,7 @@ blocks: - export NODE_OPTIONS='--max-old-space-size=1536' - cd examples/performance - npm install - - ../../ci/tests/run_perf_tests.sh + - bash -c '../../ci/tests/run_perf_test.sh' - name: "Linux amd64: Release" dependencies: [ ] From a744a82a2bc4112d65060940c4d7bf3cea4c856f Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Wed, 11 Sep 2024 12:30:16 -0700 Subject: [PATCH 188/224] First cut at Data Contract rules (#77) * First cut at encryption rules (#74) * First cut at encryption rules * Add tests * Clean up package.json * Clean up package.json * Add kms clients * Minor fix * First cut at additional serde tests (#75) * First cut at additional serde tests * Add JSON test * Add protobuf ref test * Add format params * Checkpoint * Add coverage to gitignore * Remove coverage * Minor fix * Minor fix * Avro ref test * Add json ref test * Add nested tests * First cut at Data Contract rules * Remove CEL executors for now * Minor refactor to use RuleRegistry * Move DEK registry under encryption * Clean up package.json * Add CSFLE test with logical type * Minor fix * Add CEL executors * Revert "Add CEL executors" This reverts commit 850c3de715c54fb7d0ec0010aef936b6c7e0f2d1. * Minor fixes * Minor fixes --- .gitignore | 2 + Makefile.schemaregistry | 2 +- jest.config.js | 1 - package-lock.json | 4408 +++++++++++++---- package.json | 14 +- proto/test/schemaregistry/serde/cycle.proto | 9 + proto/test/schemaregistry/serde/dep.proto | 11 + proto/test/schemaregistry/serde/example.proto | 22 + proto/test/schemaregistry/serde/nested.proto | 59 + .../schemaregistry/serde/newerwidget.proto | 10 + .../test/schemaregistry/serde/newwidget.proto | 10 + proto/test/schemaregistry/serde/test.proto | 24 + proto/test/schemaregistry/serde/widget.proto | 10 + .../google/type/calendar_period_pb.ts | 102 + schemaregistry/google/type/color_pb.ts | 204 + schemaregistry/google/type/date_pb.ts | 79 + schemaregistry/google/type/datetime_pb.ts | 180 + schemaregistry/google/type/dayofweek_pb.ts | 96 + schemaregistry/google/type/decimal_pb.ts | 114 + schemaregistry/google/type/expr_pb.ts | 105 + schemaregistry/google/type/fraction_pb.ts | 57 + schemaregistry/google/type/interval_pb.ts | 69 + schemaregistry/google/type/latlng_pb.ts | 60 + .../google/type/localized_text_pb.ts | 59 + schemaregistry/google/type/money_pb.ts | 69 + schemaregistry/google/type/month_pb.ts | 131 + schemaregistry/google/type/phone_number_pb.ts | 165 + .../google/type/postal_address_pb.ts | 193 + schemaregistry/google/type/quaternion_pb.ts | 125 + schemaregistry/google/type/timeofday_pb.ts | 75 + schemaregistry/mock-schemaregistry-client.ts | 61 +- .../rules/encryption/awskms/aws-client.ts | 46 + .../rules/encryption/awskms/aws-driver.ts | 29 + .../rules/encryption/azurekms/azure-client.ts | 33 + .../rules/encryption/azurekms/azure-driver.ts | 33 + .../encryption}/dekregistry/constants.ts | 0 .../dekregistry/dekregistry-client.ts | 11 +- .../dekregistry/mock-dekregistry-client.ts | 9 +- .../rules/encryption/encrypt-executor.ts | 21 +- .../rules/encryption/gcpkms/gcp-client.ts | 43 + .../rules/encryption/gcpkms/gcp-driver.ts | 51 + .../encryption/hcvault/hcvault-client.ts | 47 + .../encryption/hcvault/hcvault-driver.ts | 24 + .../rules/encryption/localkms/local-client.ts | 44 + .../rules/encryption/localkms/local-driver.ts | 21 + .../rules/encryption/tink/aes_gcm.ts | 7 +- .../rules/encryption/tink/aes_siv.ts | 11 +- schemaregistry/rules/encryption/tink/hkdf.ts | 99 + schemaregistry/rules/encryption/tink/hmac.ts | 98 + schemaregistry/rules/encryption/tink/mac.ts | 34 + .../rules/encryption/tink/random.ts | 1 + .../rules/jsonata/jsonata-executor.ts | 40 + schemaregistry/schemaregistry-client.ts | 72 +- schemaregistry/serde/avro.ts | 112 +- .../serde/{json_util.ts => json-util.ts} | 0 schemaregistry/serde/json.ts | 26 +- schemaregistry/serde/protobuf.ts | 150 +- schemaregistry/serde/rule-registry.ts | 92 +- schemaregistry/serde/serde.ts | 80 +- .../dekregistry/dekregistry-client.spec.ts | 8 +- .../mock-dekregistry-client.spec.ts | 4 +- .../encryption}/dekregistry/test-constants.ts | 6 +- test/schemaregistry/serde/avro.spec.ts | 504 +- .../serde/buffer-wrapper.spec.ts | 6 +- test/schemaregistry/serde/json.spec.ts | 209 + test/schemaregistry/serde/protobuf.spec.ts | 205 + test/schemaregistry/serde/test/cycle_pb.ts | 36 + test/schemaregistry/serde/test/dep_pb.ts | 38 + test/schemaregistry/serde/test/example_pb.ts | 69 + test/schemaregistry/serde/test/nested_pb.ts | 221 + .../serde/test/newerwidget_pb.ts | 41 + .../schemaregistry/serde/test/newwidget_pb.ts | 41 + test/schemaregistry/serde/test/test_pb.ts | 102 + test/schemaregistry/serde/test/widget_pb.ts | 41 + tsconfig.json | 9 +- 75 files changed, 8067 insertions(+), 1233 deletions(-) create mode 100644 proto/test/schemaregistry/serde/cycle.proto create mode 100644 proto/test/schemaregistry/serde/dep.proto create mode 100644 proto/test/schemaregistry/serde/example.proto create mode 100644 proto/test/schemaregistry/serde/nested.proto create mode 100644 proto/test/schemaregistry/serde/newerwidget.proto create mode 100644 proto/test/schemaregistry/serde/newwidget.proto create mode 100644 proto/test/schemaregistry/serde/test.proto create mode 100644 proto/test/schemaregistry/serde/widget.proto create mode 100644 schemaregistry/google/type/calendar_period_pb.ts create mode 100644 schemaregistry/google/type/color_pb.ts create mode 100644 schemaregistry/google/type/date_pb.ts create mode 100644 schemaregistry/google/type/datetime_pb.ts create mode 100644 schemaregistry/google/type/dayofweek_pb.ts create mode 100644 schemaregistry/google/type/decimal_pb.ts create mode 100644 schemaregistry/google/type/expr_pb.ts create mode 100644 schemaregistry/google/type/fraction_pb.ts create mode 100644 schemaregistry/google/type/interval_pb.ts create mode 100644 schemaregistry/google/type/latlng_pb.ts create mode 100644 schemaregistry/google/type/localized_text_pb.ts create mode 100644 schemaregistry/google/type/money_pb.ts create mode 100644 schemaregistry/google/type/month_pb.ts create mode 100644 schemaregistry/google/type/phone_number_pb.ts create mode 100644 schemaregistry/google/type/postal_address_pb.ts create mode 100644 schemaregistry/google/type/quaternion_pb.ts create mode 100644 schemaregistry/google/type/timeofday_pb.ts create mode 100644 schemaregistry/rules/encryption/awskms/aws-client.ts create mode 100644 schemaregistry/rules/encryption/awskms/aws-driver.ts create mode 100644 schemaregistry/rules/encryption/azurekms/azure-client.ts create mode 100644 schemaregistry/rules/encryption/azurekms/azure-driver.ts rename schemaregistry/{ => rules/encryption}/dekregistry/constants.ts (100%) rename schemaregistry/{ => rules/encryption}/dekregistry/dekregistry-client.ts (95%) rename schemaregistry/{ => rules/encryption}/dekregistry/mock-dekregistry-client.ts (90%) create mode 100644 schemaregistry/rules/encryption/gcpkms/gcp-client.ts create mode 100644 schemaregistry/rules/encryption/gcpkms/gcp-driver.ts create mode 100644 schemaregistry/rules/encryption/hcvault/hcvault-client.ts create mode 100644 schemaregistry/rules/encryption/hcvault/hcvault-driver.ts create mode 100644 schemaregistry/rules/encryption/localkms/local-client.ts create mode 100644 schemaregistry/rules/encryption/localkms/local-driver.ts create mode 100644 schemaregistry/rules/encryption/tink/hkdf.ts create mode 100644 schemaregistry/rules/encryption/tink/hmac.ts create mode 100644 schemaregistry/rules/encryption/tink/mac.ts create mode 100644 schemaregistry/rules/jsonata/jsonata-executor.ts rename schemaregistry/serde/{json_util.ts => json-util.ts} (100%) rename test/schemaregistry/{ => rules/encryption}/dekregistry/dekregistry-client.spec.ts (94%) rename test/schemaregistry/{ => rules/encryption}/dekregistry/mock-dekregistry-client.spec.ts (90%) rename test/schemaregistry/{ => rules/encryption}/dekregistry/test-constants.ts (90%) create mode 100644 test/schemaregistry/serde/json.spec.ts create mode 100644 test/schemaregistry/serde/protobuf.spec.ts create mode 100644 test/schemaregistry/serde/test/cycle_pb.ts create mode 100644 test/schemaregistry/serde/test/dep_pb.ts create mode 100644 test/schemaregistry/serde/test/example_pb.ts create mode 100644 test/schemaregistry/serde/test/nested_pb.ts create mode 100644 test/schemaregistry/serde/test/newerwidget_pb.ts create mode 100644 test/schemaregistry/serde/test/newwidget_pb.ts create mode 100644 test/schemaregistry/serde/test/test_pb.ts create mode 100644 test/schemaregistry/serde/test/widget_pb.ts diff --git a/.gitignore b/.gitignore index 7624e00c..603212d2 100644 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,5 @@ build/ +dist/ node_modules/ deps/librdkafka npm-debug.log @@ -15,3 +16,4 @@ deps/* .idea .vscode +coverage diff --git a/Makefile.schemaregistry b/Makefile.schemaregistry index 248ff096..37f002bc 100644 --- a/Makefile.schemaregistry +++ b/Makefile.schemaregistry @@ -9,7 +9,7 @@ TS_NODE ?= ./node_modules/.bin/ts-node # Paths SRC_DIR = schemaregistry SR_TEST_DIR = test/schemaregistry -DEK_TEST_DIR = test/schemaregistry/dekregistry +DEK_TEST_DIR = test/schemaregistry/rules/encryption/dekregistry INTEG_DIR = e2e/schemaregistry # Tasks diff --git a/jest.config.js b/jest.config.js index a6110529..e3471a95 100644 --- a/jest.config.js +++ b/jest.config.js @@ -3,4 +3,3 @@ module.exports = { '^.+\\.tsx?$': 'ts-jest', }, }; - \ No newline at end of file diff --git a/package-lock.json b/package-lock.json index cf540dae..aacda59a 100644 --- a/package-lock.json +++ b/package-lock.json @@ -10,13 +10,16 @@ "hasInstallScript": true, "license": "MIT", "dependencies": { - "@bufbuild/buf": "^1.37.0", + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", "@bufbuild/protobuf": "^2.0.0", - "@bufbuild/protoc-gen-es": "^2.0.0", "@criteria/json-schema": "^0.10.0", "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", + "@smithy/types": "^3.3.0", "@types/validator": "^13.12.0", "ajv": "^8.17.1", "async-mutex": "^0.5.0", @@ -24,16 +27,19 @@ "axios": "^1.7.3", "bindings": "^1.3.1", "json-stringify-deterministic": "^1.0.12", + "jsonata": "^2.0.5", "lru-cache": "^11.0.0", - "miscreant": "^0.3.2", "nan": "^2.17.0", + "node-vault": "^0.10.2", "ts-jest": "^29.2.4", "validator": "^13.12.0" }, "devDependencies": { + "@bufbuild/buf": "^1.37.0", + "@bufbuild/protoc-gen-es": "^2.0.0", "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", - "@types/node": "^20.4.5", + "@types/node": "^20.16.1", "bluebird": "^3.5.3", "eslint": "^8.57.0", "eslint-plugin-jest": "^28.6.0", @@ -60,1357 +66,2962 @@ "node": ">=6.0.0" } }, - "node_modules/@babel/code-frame": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", - "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", + "node_modules/@aws-crypto/sha256-browser": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/@aws-crypto/sha256-browser/-/sha256-browser-5.2.0.tgz", + "integrity": "sha512-AXfN/lGotSQwu6HNcEsIASo7kWXZ5HYWvfOmSNKDsEqC4OashTp8alTmaz+F7TC2L083SFv5RdB+qU3Vs1kZqw==", "dependencies": { - "@babel/highlight": "^7.24.7", - "picocolors": "^1.0.0" - }, - "engines": { - "node": ">=6.9.0" + "@aws-crypto/sha256-js": "^5.2.0", + "@aws-crypto/supports-web-crypto": "^5.2.0", + "@aws-crypto/util": "^5.2.0", + "@aws-sdk/types": "^3.222.0", + "@aws-sdk/util-locate-window": "^3.0.0", + "@smithy/util-utf8": "^2.0.0", + "tslib": "^2.6.2" } }, - "node_modules/@babel/compat-data": { - "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.2.tgz", - "integrity": "sha512-bYcppcpKBvX4znYaPEeFau03bp89ShqNMLs+rmdptMw+heSZh9+z84d2YG+K7cYLbWwzdjtDoW/uqZmPjulClQ==", + "node_modules/@aws-crypto/sha256-browser/node_modules/@smithy/is-array-buffer": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@smithy/is-array-buffer/-/is-array-buffer-2.2.0.tgz", + "integrity": "sha512-GGP3O9QFD24uGeAXYUjwSTXARoqpZykHadOmA8G5vfJPK0/DC67qa//0qvqrJzL1xc8WQWX7/yc7fwudjPHPhA==", + "dependencies": { + "tslib": "^2.6.2" + }, "engines": { - "node": ">=6.9.0" + "node": ">=14.0.0" } }, - "node_modules/@babel/core": { - "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.25.2.tgz", - "integrity": "sha512-BBt3opiCOxUr9euZ5/ro/Xv8/V7yJ5bjYMqG/C1YAo8MIKAnumZalCN+msbci3Pigy4lIQfPUpfMM27HMGaYEA==", + "node_modules/@aws-crypto/sha256-browser/node_modules/@smithy/util-buffer-from": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@smithy/util-buffer-from/-/util-buffer-from-2.2.0.tgz", + "integrity": "sha512-IJdWBbTcMQ6DA0gdNhh/BwrLkDR+ADW5Kr1aZmd4k3DIF6ezMV4R2NIAmT08wQJ3yUK82thHWmC/TnK/wpMMIA==", "dependencies": { - "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.24.7", - "@babel/generator": "^7.25.0", - "@babel/helper-compilation-targets": "^7.25.2", - "@babel/helper-module-transforms": "^7.25.2", - "@babel/helpers": "^7.25.0", - "@babel/parser": "^7.25.0", - "@babel/template": "^7.25.0", - "@babel/traverse": "^7.25.2", - "@babel/types": "^7.25.2", - "convert-source-map": "^2.0.0", - "debug": "^4.1.0", - "gensync": "^1.0.0-beta.2", - "json5": "^2.2.3", - "semver": "^6.3.1" + "@smithy/is-array-buffer": "^2.2.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/babel" - } - }, - "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==", - "bin": { - "semver": "bin/semver.js" + "node": ">=14.0.0" } }, - "node_modules/@babel/generator": { - "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.0.tgz", - "integrity": "sha512-3LEEcj3PVW8pW2R1SR1M89g/qrYk/m/mB/tLqn7dn4sbBUQyTqnlod+II2U4dqiGtUmkcnAmkMDralTFZttRiw==", + "node_modules/@aws-crypto/sha256-browser/node_modules/@smithy/util-utf8": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/@smithy/util-utf8/-/util-utf8-2.3.0.tgz", + "integrity": "sha512-R8Rdn8Hy72KKcebgLiv8jQcQkXoLMOGGv5uI1/k0l+snqkOzQ1R0ChUBCxWMlBsFMekWjq0wRudIweFs7sKT5A==", "dependencies": { - "@babel/types": "^7.25.0", - "@jridgewell/gen-mapping": "^0.3.5", - "@jridgewell/trace-mapping": "^0.3.25", - "jsesc": "^2.5.1" + "@smithy/util-buffer-from": "^2.2.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=14.0.0" } }, - "node_modules/@babel/helper-compilation-targets": { - "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.25.2.tgz", - "integrity": "sha512-U2U5LsSaZ7TAt3cfaymQ8WHh0pxvdHoEk6HVpaexxixjyEquMh0L0YNJNM6CTGKMXV1iksi0iZkGw4AcFkPaaw==", + "node_modules/@aws-crypto/sha256-js": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/@aws-crypto/sha256-js/-/sha256-js-5.2.0.tgz", + "integrity": "sha512-FFQQyu7edu4ufvIZ+OadFpHHOt+eSTBaYaki44c+akjg7qZg9oOQeLlk77F6tSYqjDAFClrHJk9tMf0HdVyOvA==", "dependencies": { - "@babel/compat-data": "^7.25.2", - "@babel/helper-validator-option": "^7.24.8", - "browserslist": "^4.23.1", - "lru-cache": "^5.1.1", - "semver": "^6.3.1" + "@aws-crypto/util": "^5.2.0", + "@aws-sdk/types": "^3.222.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { - "version": "5.1.1", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-5.1.1.tgz", - "integrity": "sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w==", - "license": "ISC", + "node_modules/@aws-crypto/supports-web-crypto": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/@aws-crypto/supports-web-crypto/-/supports-web-crypto-5.2.0.tgz", + "integrity": "sha512-iAvUotm021kM33eCdNfwIN//F77/IADDSs58i+MDaOqFrVjZo9bAal0NK7HurRuWLLpF1iLX7gbWrjHjeo+YFg==", "dependencies": { - "yallist": "^3.0.2" + "tslib": "^2.6.2" } }, - "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==", - "bin": { - "semver": "bin/semver.js" + "node_modules/@aws-crypto/util": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/@aws-crypto/util/-/util-5.2.0.tgz", + "integrity": "sha512-4RkU9EsI6ZpBve5fseQlGNUWKMa1RLPQ1dnjnQoe07ldfIzcsGb5hC5W0Dm7u423KWzawlrpbjXBrXCEv9zazQ==", + "dependencies": { + "@aws-sdk/types": "^3.222.0", + "@smithy/util-utf8": "^2.0.0", + "tslib": "^2.6.2" } }, - "node_modules/@babel/helper-module-imports": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", - "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", + "node_modules/@aws-crypto/util/node_modules/@smithy/is-array-buffer": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@smithy/is-array-buffer/-/is-array-buffer-2.2.0.tgz", + "integrity": "sha512-GGP3O9QFD24uGeAXYUjwSTXARoqpZykHadOmA8G5vfJPK0/DC67qa//0qvqrJzL1xc8WQWX7/yc7fwudjPHPhA==", "dependencies": { - "@babel/traverse": "^7.24.7", - "@babel/types": "^7.24.7" + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=14.0.0" } }, - "node_modules/@babel/helper-module-transforms": { - "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.25.2.tgz", - "integrity": "sha512-BjyRAbix6j/wv83ftcVJmBt72QtHI56C7JXZoG2xATiLpmoC7dpd8WnkikExHDVPpi/3qCmO6WY1EaXOluiecQ==", + "node_modules/@aws-crypto/util/node_modules/@smithy/util-buffer-from": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/@smithy/util-buffer-from/-/util-buffer-from-2.2.0.tgz", + "integrity": "sha512-IJdWBbTcMQ6DA0gdNhh/BwrLkDR+ADW5Kr1aZmd4k3DIF6ezMV4R2NIAmT08wQJ3yUK82thHWmC/TnK/wpMMIA==", "dependencies": { - "@babel/helper-module-imports": "^7.24.7", - "@babel/helper-simple-access": "^7.24.7", - "@babel/helper-validator-identifier": "^7.24.7", - "@babel/traverse": "^7.25.2" + "@smithy/is-array-buffer": "^2.2.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=14.0.0" + } + }, + "node_modules/@aws-crypto/util/node_modules/@smithy/util-utf8": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/@smithy/util-utf8/-/util-utf8-2.3.0.tgz", + "integrity": "sha512-R8Rdn8Hy72KKcebgLiv8jQcQkXoLMOGGv5uI1/k0l+snqkOzQ1R0ChUBCxWMlBsFMekWjq0wRudIweFs7sKT5A==", + "dependencies": { + "@smithy/util-buffer-from": "^2.2.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=14.0.0" + } + }, + "node_modules/@aws-sdk/client-kms": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-kms/-/client-kms-3.637.0.tgz", + "integrity": "sha512-bqppLpmIPl6eZkZx/9axnr4CBbhtrRKe3LffW8320DlwCqP3zU+c500vXMjEgYdrAqkqOFyDY/FYMAgZhtHVCQ==", + "dependencies": { + "@aws-crypto/sha256-browser": "5.2.0", + "@aws-crypto/sha256-js": "5.2.0", + "@aws-sdk/client-sso-oidc": "3.637.0", + "@aws-sdk/client-sts": "3.637.0", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/credential-provider-node": "3.637.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "@smithy/util-base64": "^3.0.0", + "@smithy/util-body-length-browser": "^3.0.0", + "@smithy/util-body-length-node": "^3.0.0", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@aws-sdk/client-sso": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso/-/client-sso-3.637.0.tgz", + "integrity": "sha512-+KjLvgX5yJYROWo3TQuwBJlHCY0zz9PsLuEolmXQn0BVK1L/m9GteZHtd+rEdAoDGBpE0Xqjy1oz5+SmtsaRUw==", + "dependencies": { + "@aws-crypto/sha256-browser": "5.2.0", + "@aws-crypto/sha256-js": "5.2.0", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "@smithy/util-base64": "^3.0.0", + "@smithy/util-body-length-browser": "^3.0.0", + "@smithy/util-body-length-node": "^3.0.0", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@aws-sdk/client-sso-oidc": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso-oidc/-/client-sso-oidc-3.637.0.tgz", + "integrity": "sha512-27bHALN6Qb6m6KZmPvRieJ/QRlj1lyac/GT2Rn5kJpre8Mpp+yxrtvp3h9PjNBty4lCeFEENfY4dGNSozBuBcw==", + "dependencies": { + "@aws-crypto/sha256-browser": "5.2.0", + "@aws-crypto/sha256-js": "5.2.0", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/credential-provider-node": "3.637.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "@smithy/util-base64": "^3.0.0", + "@smithy/util-body-length-browser": "^3.0.0", + "@smithy/util-body-length-node": "^3.0.0", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" }, "peerDependencies": { - "@babel/core": "^7.0.0" + "@aws-sdk/client-sts": "^3.637.0" + } + }, + "node_modules/@aws-sdk/client-sts": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-sts/-/client-sts-3.637.0.tgz", + "integrity": "sha512-xUi7x4qDubtA8QREtlblPuAcn91GS/09YVEY/RwU7xCY0aqGuFwgszAANlha4OUIqva8oVj2WO4gJuG+iaSnhw==", + "dependencies": { + "@aws-crypto/sha256-browser": "5.2.0", + "@aws-crypto/sha256-js": "5.2.0", + "@aws-sdk/client-sso-oidc": "3.637.0", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/credential-provider-node": "3.637.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "@smithy/util-base64": "^3.0.0", + "@smithy/util-body-length-browser": "^3.0.0", + "@smithy/util-body-length-node": "^3.0.0", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@aws-sdk/core": { + "version": "3.635.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/core/-/core-3.635.0.tgz", + "integrity": "sha512-i1x/E/sgA+liUE1XJ7rj1dhyXpAKO1UKFUcTTHXok2ARjWTvszHnSXMOsB77aPbmn0fUp1JTx2kHUAZ1LVt5Bg==", + "dependencies": { + "@smithy/core": "^2.4.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/signature-v4": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-middleware": "^3.0.3", + "fast-xml-parser": "4.4.1", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@aws-sdk/credential-provider-env": { + "version": "3.620.1", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-env/-/credential-provider-env-3.620.1.tgz", + "integrity": "sha512-ExuILJ2qLW5ZO+rgkNRj0xiAipKT16Rk77buvPP8csR7kkCflT/gXTyzRe/uzIiETTxM7tr8xuO9MP/DQXqkfg==", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@aws-sdk/credential-provider-http": { + "version": "3.635.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-http/-/credential-provider-http-3.635.0.tgz", + "integrity": "sha512-iJyRgEjOCQlBMXqtwPLIKYc7Bsc6nqjrZybdMDenPDa+kmLg7xh8LxHsu9088e+2/wtLicE34FsJJIfzu3L82g==", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-stream": "^3.1.3", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@aws-sdk/credential-provider-ini": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-ini/-/credential-provider-ini-3.637.0.tgz", + "integrity": "sha512-h+PFCWfZ0Q3Dx84SppET/TFpcQHmxFW8/oV9ArEvMilw4EBN+IlxgbL0CnHwjHW64szcmrM0mbebjEfHf4FXmw==", + "dependencies": { + "@aws-sdk/credential-provider-env": "3.620.1", + "@aws-sdk/credential-provider-http": "3.635.0", + "@aws-sdk/credential-provider-process": "3.620.1", + "@aws-sdk/credential-provider-sso": "3.637.0", + "@aws-sdk/credential-provider-web-identity": "3.621.0", + "@aws-sdk/types": "3.609.0", + "@smithy/credential-provider-imds": "^3.2.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + }, + "peerDependencies": { + "@aws-sdk/client-sts": "^3.637.0" } }, - "node_modules/@babel/helper-plugin-utils": { - "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.8.tgz", - "integrity": "sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==", + "node_modules/@aws-sdk/credential-provider-node": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-node/-/credential-provider-node-3.637.0.tgz", + "integrity": "sha512-yoEhoxJJfs7sPVQ6Is939BDQJZpZCoUgKr/ySse4YKOZ24t4VqgHA6+wV7rYh+7IW24Rd91UTvEzSuHYTlxlNA==", + "dependencies": { + "@aws-sdk/credential-provider-env": "3.620.1", + "@aws-sdk/credential-provider-http": "3.635.0", + "@aws-sdk/credential-provider-ini": "3.637.0", + "@aws-sdk/credential-provider-process": "3.620.1", + "@aws-sdk/credential-provider-sso": "3.637.0", + "@aws-sdk/credential-provider-web-identity": "3.621.0", + "@aws-sdk/types": "3.609.0", + "@smithy/credential-provider-imds": "^3.2.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "node_modules/@babel/helper-simple-access": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz", - "integrity": "sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==", + "node_modules/@aws-sdk/credential-provider-process": { + "version": "3.620.1", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-process/-/credential-provider-process-3.620.1.tgz", + "integrity": "sha512-hWqFMidqLAkaV9G460+1at6qa9vySbjQKKc04p59OT7lZ5cO5VH5S4aI05e+m4j364MBROjjk2ugNvfNf/8ILg==", "dependencies": { - "@babel/traverse": "^7.24.7", - "@babel/types": "^7.24.7" + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "node_modules/@babel/helper-string-parser": { - "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", - "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", + "node_modules/@aws-sdk/credential-provider-sso": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-sso/-/credential-provider-sso-3.637.0.tgz", + "integrity": "sha512-Mvz+h+e62/tl+dVikLafhv+qkZJ9RUb8l2YN/LeKMWkxQylPT83CPk9aimVhCV89zth1zpREArl97+3xsfgQvA==", + "dependencies": { + "@aws-sdk/client-sso": "3.637.0", + "@aws-sdk/token-providers": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "node_modules/@babel/helper-validator-identifier": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", - "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", + "node_modules/@aws-sdk/credential-provider-web-identity": { + "version": "3.621.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-web-identity/-/credential-provider-web-identity-3.621.0.tgz", + "integrity": "sha512-w7ASSyfNvcx7+bYGep3VBgC3K6vEdLmlpjT7nSIHxxQf+WSdvy+HynwJosrpZax0sK5q0D1Jpn/5q+r5lwwW6w==", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" + }, + "peerDependencies": { + "@aws-sdk/client-sts": "^3.621.0" } }, - "node_modules/@babel/helper-validator-option": { - "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.24.8.tgz", - "integrity": "sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==", + "node_modules/@aws-sdk/middleware-host-header": { + "version": "3.620.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-host-header/-/middleware-host-header-3.620.0.tgz", + "integrity": "sha512-VMtPEZwqYrII/oUkffYsNWY9PZ9xpNJpMgmyU0rlDQ25O1c0Hk3fJmZRe6pEkAJ0omD7kLrqGl1DUjQVxpd/Rg==", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "node_modules/@babel/helpers": { - "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.0.tgz", - "integrity": "sha512-MjgLZ42aCm0oGjJj8CtSM3DB8NOOf8h2l7DCTePJs29u+v7yO/RBX9nShlKMgFnRks/Q4tBAe7Hxnov9VkGwLw==", + "node_modules/@aws-sdk/middleware-logger": { + "version": "3.609.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-logger/-/middleware-logger-3.609.0.tgz", + "integrity": "sha512-S62U2dy4jMDhDFDK5gZ4VxFdWzCtLzwbYyFZx2uvPYTECkepLUfzLic2BHg2Qvtu4QjX+oGE3P/7fwaGIsGNuQ==", "dependencies": { - "@babel/template": "^7.25.0", - "@babel/types": "^7.25.0" + "@aws-sdk/types": "3.609.0", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "node_modules/@babel/highlight": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", - "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", + "node_modules/@aws-sdk/middleware-recursion-detection": { + "version": "3.620.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-recursion-detection/-/middleware-recursion-detection-3.620.0.tgz", + "integrity": "sha512-nh91S7aGK3e/o1ck64sA/CyoFw+gAYj2BDOnoNa6ouyCrVJED96ZXWbhye/fz9SgmNUZR2g7GdVpiLpMKZoI5w==", "dependencies": { - "@babel/helper-validator-identifier": "^7.24.7", - "chalk": "^2.4.2", - "js-tokens": "^4.0.0", - "picocolors": "^1.0.0" + "@aws-sdk/types": "3.609.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@aws-sdk/middleware-user-agent": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-user-agent/-/middleware-user-agent-3.637.0.tgz", + "integrity": "sha512-EYo0NE9/da/OY8STDsK2LvM4kNa79DBsf4YVtaG4P5pZ615IeFsD8xOHZeuJmUrSMlVQ8ywPRX7WMucUybsKug==", "dependencies": { - "color-convert": "^1.9.0" + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=4" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@aws-sdk/region-config-resolver": { + "version": "3.614.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/region-config-resolver/-/region-config-resolver-3.614.0.tgz", + "integrity": "sha512-vDCeMXvic/LU0KFIUjpC3RiSTIkkvESsEfbVHiHH0YINfl8HnEqR5rj+L8+phsCeVg2+LmYwYxd5NRz4PHxt5g==", "dependencies": { - "ansi-styles": "^3.2.1", - "escape-string-regexp": "^1.0.5", - "supports-color": "^5.3.0" + "@aws-sdk/types": "3.609.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", + "@smithy/util-config-provider": "^3.0.0", + "@smithy/util-middleware": "^3.0.3", + "tslib": "^2.6.2" }, "engines": { - "node": ">=4" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@aws-sdk/token-providers": { + "version": "3.614.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/token-providers/-/token-providers-3.614.0.tgz", + "integrity": "sha512-okItqyY6L9IHdxqs+Z116y5/nda7rHxLvROxtAJdLavWTYDydxrZstImNgGWTeVdmc0xX2gJCI77UYUTQWnhRw==", "dependencies": { - "color-name": "1.1.3" - } - }, - "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==" - }, - "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==", + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=0.8.0" + "node": ">=16.0.0" + }, + "peerDependencies": { + "@aws-sdk/client-sso-oidc": "^3.614.0" } }, - "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==", + "node_modules/@aws-sdk/types": { + "version": "3.609.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/types/-/types-3.609.0.tgz", + "integrity": "sha512-+Tqnh9w0h2LcrUsdXyT1F8mNhXz+tVYBtP19LpeEGntmvHwa2XzvLUCWpoIAIVsHp5+HdB2X9Sn0KAtmbFXc2Q==", + "dependencies": { + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=4" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@aws-sdk/util-endpoints": { + "version": "3.637.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-endpoints/-/util-endpoints-3.637.0.tgz", + "integrity": "sha512-pAqOKUHeVWHEXXDIp/qoMk/6jyxIb6GGjnK1/f8dKHtKIEs4tKsnnL563gceEvdad53OPXIt86uoevCcCzmBnw==", "dependencies": { - "has-flag": "^3.0.0" + "@aws-sdk/types": "3.609.0", + "@smithy/types": "^3.3.0", + "@smithy/util-endpoints": "^2.0.5", + "tslib": "^2.6.2" }, "engines": { - "node": ">=4" + "node": ">=16.0.0" } }, - "node_modules/@babel/parser": { - "version": "7.25.3", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.3.tgz", - "integrity": "sha512-iLTJKDbJ4hMvFPgQwwsVoxtHyWpKKPBrxkANrSYewDPaPpT5py5yeVkgPIJ7XYXhndxJpaA3PyALSXQ7u8e/Dw==", + "node_modules/@aws-sdk/util-locate-window": { + "version": "3.568.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-locate-window/-/util-locate-window-3.568.0.tgz", + "integrity": "sha512-3nh4TINkXYr+H41QaPelCceEB2FXP3fxp93YZXB/kqJvX0U9j0N0Uk45gvsjmEPzG8XxkPEeLIfT2I1M7A6Lig==", "dependencies": { - "@babel/types": "^7.25.2" - }, - "bin": { - "parser": "bin/babel-parser.js" + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.0.0" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@aws-sdk/util-user-agent-browser": { + "version": "3.609.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-browser/-/util-user-agent-browser-3.609.0.tgz", + "integrity": "sha512-fojPU+mNahzQ0YHYBsx0ZIhmMA96H+ZIZ665ObU9tl+SGdbLneVZVikGve+NmHTQwHzwkFsZYYnVKAkreJLAtA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" - }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "@aws-sdk/types": "3.609.0", + "@smithy/types": "^3.3.0", + "bowser": "^2.11.0", + "tslib": "^2.6.2" } }, - "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==", + "node_modules/@aws-sdk/util-user-agent-node": { + "version": "3.614.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-node/-/util-user-agent-node-3.614.0.tgz", + "integrity": "sha512-15ElZT88peoHnq5TEoEtZwoXTXRxNrk60TZNdpl/TUBJ5oNJ9Dqb5Z4ryb8ofN6nm9aFf59GVAerFDz8iUoHBA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" + "@aws-sdk/types": "3.609.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" }, "peerDependencies": { - "@babel/core": "^7.0.0-0" + "aws-crt": ">=1.0.0" + }, + "peerDependenciesMeta": { + "aws-crt": { + "optional": true + } } }, - "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==", + "node_modules/@azure/abort-controller": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-1.1.0.tgz", + "integrity": "sha512-TrRLIoSQVzfAJX9H1JeFjzAoDGcoK1IYX1UImfceTZpsyYfWr09Ss1aHW1y5TrrR3iq6RZLBwJ3E24uwPhwahw==", "dependencies": { - "@babel/helper-plugin-utils": "^7.12.13" + "tslib": "^2.2.0" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=12.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==", + "node_modules/@azure/core-auth": { + "version": "1.7.2", + "resolved": "https://registry.npmjs.org/@azure/core-auth/-/core-auth-1.7.2.tgz", + "integrity": "sha512-Igm/S3fDYmnMq1uKS38Ae1/m37B3zigdlZw+kocwEhh5GjyKjPrXKO2J6rzpC1wAxrNil/jX9BJRqBshyjnF3g==", "dependencies": { - "@babel/helper-plugin-utils": "^7.10.4" + "@azure/abort-controller": "^2.0.0", + "@azure/core-util": "^1.1.0", + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.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==", + "node_modules/@azure/core-auth/node_modules/@azure/abort-controller": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", + "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "node_modules/@babel/plugin-syntax-jsx": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz", - "integrity": "sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ==", + "node_modules/@azure/core-client": { + "version": "1.9.2", + "resolved": "https://registry.npmjs.org/@azure/core-client/-/core-client-1.9.2.tgz", + "integrity": "sha512-kRdry/rav3fUKHl/aDLd/pDLcB+4pOFwPPTVEExuMyaI5r+JBbMWqRbCY1pn5BniDaU3lRxO9eaQ1AmSMehl/w==", "dependencies": { - "@babel/helper-plugin-utils": "^7.24.7" + "@azure/abort-controller": "^2.0.0", + "@azure/core-auth": "^1.4.0", + "@azure/core-rest-pipeline": "^1.9.1", + "@azure/core-tracing": "^1.0.0", + "@azure/core-util": "^1.6.1", + "@azure/logger": "^1.0.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" - }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-client/node_modules/@azure/abort-controller": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", + "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.10.4" + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-http-compat": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/core-http-compat/-/core-http-compat-2.1.2.tgz", + "integrity": "sha512-5MnV1yqzZwgNLLjlizsU3QqOeQChkIXw781Fwh1xdAqJR5AA32IUaq6xv1BICJvfbHoa+JYcaij2HFkhLbNTJQ==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" + "@azure/abort-controller": "^2.0.0", + "@azure/core-client": "^1.3.0", + "@azure/core-rest-pipeline": "^1.3.0" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-http-compat/node_modules/@azure/abort-controller": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", + "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.10.4" + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-lro": { + "version": "2.7.2", + "resolved": "https://registry.npmjs.org/@azure/core-lro/-/core-lro-2.7.2.tgz", + "integrity": "sha512-0YIpccoX8m/k00O7mDDMdJpbr6mf1yWo2dfmxt5A8XVZVVMz2SSKaEbMCeJRvgQ0IaSlqhjT47p4hVIRRy90xw==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" + "@azure/abort-controller": "^2.0.0", + "@azure/core-util": "^1.2.0", + "@azure/logger": "^1.0.0", + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-lro/node_modules/@azure/abort-controller": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", + "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-paging": { + "version": "1.6.2", + "resolved": "https://registry.npmjs.org/@azure/core-paging/-/core-paging-1.6.2.tgz", + "integrity": "sha512-YKWi9YuCU04B55h25cnOYZHxXYtEvQEbKST5vqRga7hWY9ydd3FZHdeQF8pyh+acWZvppw13M/LMGx0LABUVMA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.8.0" + "tslib": "^2.6.2" }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "engines": { + "node": ">=18.0.0" } }, - "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==", + "node_modules/@azure/core-rest-pipeline": { + "version": "1.16.3", + "resolved": "https://registry.npmjs.org/@azure/core-rest-pipeline/-/core-rest-pipeline-1.16.3.tgz", + "integrity": "sha512-VxLk4AHLyqcHsfKe4MZ6IQ+D+ShuByy+RfStKfSjxJoL3WBWq17VNmrz8aT8etKzqc2nAeIyLxScjpzsS4fz8w==", "dependencies": { - "@babel/helper-plugin-utils": "^7.14.5" + "@azure/abort-controller": "^2.0.0", + "@azure/core-auth": "^1.4.0", + "@azure/core-tracing": "^1.0.1", + "@azure/core-util": "^1.9.0", + "@azure/logger": "^1.0.0", + "http-proxy-agent": "^7.0.0", + "https-proxy-agent": "^7.0.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" - }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "node": ">=18.0.0" } }, - "node_modules/@babel/plugin-syntax-typescript": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.7.tgz", - "integrity": "sha512-c/+fVeJBB0FeKsFvwytYiUD+LBvhHjGSI0g446PRGdSVGZLRNArBUno2PETbAly3tpiNAQR5XaZ+JslxkotsbA==", + "node_modules/@azure/core-rest-pipeline/node_modules/@azure/abort-controller": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", + "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", "dependencies": { - "@babel/helper-plugin-utils": "^7.24.7" + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" - }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" + "node": ">=18.0.0" } }, - "node_modules/@babel/template": { - "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", - "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", + "node_modules/@azure/core-rest-pipeline/node_modules/agent-base": { + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.1.tgz", + "integrity": "sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==", "dependencies": { - "@babel/code-frame": "^7.24.7", - "@babel/parser": "^7.25.0", - "@babel/types": "^7.25.0" + "debug": "^4.3.4" }, "engines": { - "node": ">=6.9.0" + "node": ">= 14" } }, - "node_modules/@babel/traverse": { - "version": "7.25.3", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.3.tgz", - "integrity": "sha512-HefgyP1x754oGCsKmV5reSmtV7IXj/kpaE1XYY+D9G5PvKKoFfSbiS4M77MdjuwlZKDIKFCffq9rPU+H/s3ZdQ==", + "node_modules/@azure/core-rest-pipeline/node_modules/http-proxy-agent": { + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/http-proxy-agent/-/http-proxy-agent-7.0.2.tgz", + "integrity": "sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig==", "dependencies": { - "@babel/code-frame": "^7.24.7", - "@babel/generator": "^7.25.0", - "@babel/parser": "^7.25.3", - "@babel/template": "^7.25.0", - "@babel/types": "^7.25.2", - "debug": "^4.3.1", - "globals": "^11.1.0" + "agent-base": "^7.1.0", + "debug": "^4.3.4" }, "engines": { - "node": ">=6.9.0" + "node": ">= 14" } }, - "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==", + "node_modules/@azure/core-rest-pipeline/node_modules/https-proxy-agent": { + "version": "7.0.5", + "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz", + "integrity": "sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==", + "dependencies": { + "agent-base": "^7.0.2", + "debug": "4" + }, "engines": { - "node": ">=4" + "node": ">= 14" } }, - "node_modules/@babel/types": { - "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.2.tgz", - "integrity": "sha512-YTnYtra7W9e6/oAZEHj0bJehPRUlLH9/fbpT5LfB0NhQXyALCRkRs3zH9v07IYhkgpqX6Z78FnuccZr/l4Fs4Q==", + "node_modules/@azure/core-tracing": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@azure/core-tracing/-/core-tracing-1.1.2.tgz", + "integrity": "sha512-dawW9ifvWAWmUm9/h+/UQ2jrdvjCJ7VJEuCJ6XVNudzcOwm53BFZH4Q845vjfgoUAM8ZxokvVNxNxAITc502YA==", "dependencies": { - "@babel/helper-string-parser": "^7.24.8", - "@babel/helper-validator-identifier": "^7.24.7", - "to-fast-properties": "^2.0.0" + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.9.0" + "node": ">=18.0.0" } }, - "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==" + "node_modules/@azure/core-util": { + "version": "1.9.2", + "resolved": "https://registry.npmjs.org/@azure/core-util/-/core-util-1.9.2.tgz", + "integrity": "sha512-l1Qrqhi4x1aekkV+OlcqsJa4AnAkj5p0JV8omgwjaV9OAbP41lvrMvs+CptfetKkeEaGRGSzby7sjPZEX7+kkQ==", + "dependencies": { + "@azure/abort-controller": "^2.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=18.0.0" + } }, - "node_modules/@bufbuild/buf": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf/-/buf-1.37.0.tgz", - "integrity": "sha512-33Rp90nRa7ebTwAjI8Yc7f8WOQMOEfRM0mg/o1bkNvE6cGQhYiq82Ag27pcegvYIyXl413XCeLMAUegsqjuFqw==", - "hasInstallScript": true, - "bin": { - "buf": "bin/buf", - "protoc-gen-buf-breaking": "bin/protoc-gen-buf-breaking", - "protoc-gen-buf-lint": "bin/protoc-gen-buf-lint" + "node_modules/@azure/core-util/node_modules/@azure/abort-controller": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", + "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "dependencies": { + "tslib": "^2.6.2" }, "engines": { - "node": ">=12" + "node": ">=18.0.0" + } + }, + "node_modules/@azure/identity": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/@azure/identity/-/identity-4.4.1.tgz", + "integrity": "sha512-DwnG4cKFEM7S3T+9u05NstXU/HN0dk45kPOinUyNKsn5VWwpXd9sbPKEg6kgJzGbm1lMuhx9o31PVbCtM5sfBA==", + "dependencies": { + "@azure/abort-controller": "^1.0.0", + "@azure/core-auth": "^1.5.0", + "@azure/core-client": "^1.9.2", + "@azure/core-rest-pipeline": "^1.1.0", + "@azure/core-tracing": "^1.0.0", + "@azure/core-util": "^1.3.0", + "@azure/logger": "^1.0.0", + "@azure/msal-browser": "^3.14.0", + "@azure/msal-node": "^2.9.2", + "events": "^3.0.0", + "jws": "^4.0.0", + "open": "^8.0.0", + "stoppable": "^1.1.0", + "tslib": "^2.2.0" }, - "optionalDependencies": { - "@bufbuild/buf-darwin-arm64": "1.37.0", - "@bufbuild/buf-darwin-x64": "1.37.0", - "@bufbuild/buf-linux-aarch64": "1.37.0", - "@bufbuild/buf-linux-x64": "1.37.0", - "@bufbuild/buf-win32-arm64": "1.37.0", - "@bufbuild/buf-win32-x64": "1.37.0" + "engines": { + "node": ">=18.0.0" } }, - "node_modules/@bufbuild/buf-darwin-arm64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-arm64/-/buf-darwin-arm64-1.37.0.tgz", - "integrity": "sha512-YrT0HqLf7qJJ9VjXmnG900oGOjskerqfY4UTbpLb67b76uRxZm7pfSfqpM4bCaSru4j6GPBDwH9dkwiuNOenEA==", - "cpu": [ - "arm64" - ], - "optional": true, - "os": [ - "darwin" - ], + "node_modules/@azure/keyvault-keys": { + "version": "4.8.0", + "resolved": "https://registry.npmjs.org/@azure/keyvault-keys/-/keyvault-keys-4.8.0.tgz", + "integrity": "sha512-jkuYxgkw0aaRfk40OQhFqDIupqblIOIlYESWB6DKCVDxQet1pyv86Tfk9M+5uFM0+mCs6+MUHU+Hxh3joiUn4Q==", + "dependencies": { + "@azure/abort-controller": "^1.0.0", + "@azure/core-auth": "^1.3.0", + "@azure/core-client": "^1.5.0", + "@azure/core-http-compat": "^2.0.1", + "@azure/core-lro": "^2.2.0", + "@azure/core-paging": "^1.1.1", + "@azure/core-rest-pipeline": "^1.8.1", + "@azure/core-tracing": "^1.0.0", + "@azure/core-util": "^1.0.0", + "@azure/logger": "^1.0.0", + "tslib": "^2.2.0" + }, "engines": { - "node": ">=12" + "node": ">=18.0.0" } }, - "node_modules/@bufbuild/buf-darwin-x64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-x64/-/buf-darwin-x64-1.37.0.tgz", - "integrity": "sha512-c1iDr1LaYcdzAK7v4ZBTiYVRPltPa1UNFQ3JJ6BI/RUaBk/eYFsykuNtsJQb5lXh/usjMz76juzOxAdYikcw7w==", - "cpu": [ - "x64" - ], - "optional": true, - "os": [ - "darwin" - ], + "node_modules/@azure/logger": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/@azure/logger/-/logger-1.1.4.tgz", + "integrity": "sha512-4IXXzcCdLdlXuCG+8UKEwLA1T1NHqUfanhXYHiQTn+6sfWCZXduqbtXDGceg3Ce5QxTGo7EqmbV6Bi+aqKuClQ==", + "dependencies": { + "tslib": "^2.6.2" + }, "engines": { - "node": ">=12" + "node": ">=18.0.0" } }, - "node_modules/@bufbuild/buf-linux-aarch64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-aarch64/-/buf-linux-aarch64-1.37.0.tgz", - "integrity": "sha512-ENaCLU4XZxQFc6+pWjNGeKITlzY0jYeEmz6R2cEeR9kVl+HyNNcTMaKxdffG5HNU5Ci1dC3pSY/ZIF3ck8UHVw==", - "cpu": [ - "arm64" - ], - "optional": true, - "os": [ - "linux" - ], + "node_modules/@azure/msal-browser": { + "version": "3.21.0", + "resolved": "https://registry.npmjs.org/@azure/msal-browser/-/msal-browser-3.21.0.tgz", + "integrity": "sha512-BAwcFsVvOrYzKuUZHhFuvRykUmQGq6lDxst2qGnjxnpNZc3d/tnVPcmhgvUdeKl28VSE0ltgBzT3HkdpDtz9rg==", + "dependencies": { + "@azure/msal-common": "14.14.1" + }, "engines": { - "node": ">=12" + "node": ">=0.8.0" } }, - "node_modules/@bufbuild/buf-linux-x64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-x64/-/buf-linux-x64-1.37.0.tgz", - "integrity": "sha512-/buQuUaBVU4rYqdoS/xmbyohJFaE4uy7boHMabCnI2aLsVBNPHYa/k0JMc+ZDRqgq0Wnth9VbIHn6DSGHG+cmw==", - "cpu": [ - "x64" - ], - "optional": true, - "os": [ - "linux" - ], + "node_modules/@azure/msal-common": { + "version": "14.14.1", + "resolved": "https://registry.npmjs.org/@azure/msal-common/-/msal-common-14.14.1.tgz", + "integrity": "sha512-2Q3tqNz/PZLfSr8BvcHZVpRRfSn4MjGSqjj9J+HlBsmbf1Uu4P0WeXnemjTJwwx9KrmplsrN3UkZ/LPOR720rw==", "engines": { - "node": ">=12" + "node": ">=0.8.0" } }, - "node_modules/@bufbuild/buf-win32-arm64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-arm64/-/buf-win32-arm64-1.37.0.tgz", - "integrity": "sha512-5Y1aaHge9iQP+YNyxUayMd+rSVbjNXLBXniJDQhim56B+6eufPiKzmTgAKjDE9hGTT+mcBdLznrCMI0fzDLHJw==", - "cpu": [ - "arm64" - ], - "optional": true, - "os": [ - "win32" - ], + "node_modules/@azure/msal-node": { + "version": "2.13.0", + "resolved": "https://registry.npmjs.org/@azure/msal-node/-/msal-node-2.13.0.tgz", + "integrity": "sha512-DhP97ycs7qlCVzzzWGzJiwAFyFj5okno74E4FUZ61oCLfKh4IxA1kxirqzrWuYZWpBe9HVPL6GA4NvmlEOBN5Q==", + "dependencies": { + "@azure/msal-common": "14.14.1", + "jsonwebtoken": "^9.0.0", + "uuid": "^8.3.0" + }, "engines": { - "node": ">=12" + "node": ">=16" } }, - "node_modules/@bufbuild/buf-win32-x64": { + "node_modules/@azure/msal-node/node_modules/uuid": { + "version": "8.3.2", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", + "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", + "bin": { + "uuid": "dist/bin/uuid" + } + }, + "node_modules/@babel/code-frame": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", + "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", + "dependencies": { + "@babel/highlight": "^7.24.7", + "picocolors": "^1.0.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/compat-data": { + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.2.tgz", + "integrity": "sha512-bYcppcpKBvX4znYaPEeFau03bp89ShqNMLs+rmdptMw+heSZh9+z84d2YG+K7cYLbWwzdjtDoW/uqZmPjulClQ==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/core": { + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.25.2.tgz", + "integrity": "sha512-BBt3opiCOxUr9euZ5/ro/Xv8/V7yJ5bjYMqG/C1YAo8MIKAnumZalCN+msbci3Pigy4lIQfPUpfMM27HMGaYEA==", + "dependencies": { + "@ampproject/remapping": "^2.2.0", + "@babel/code-frame": "^7.24.7", + "@babel/generator": "^7.25.0", + "@babel/helper-compilation-targets": "^7.25.2", + "@babel/helper-module-transforms": "^7.25.2", + "@babel/helpers": "^7.25.0", + "@babel/parser": "^7.25.0", + "@babel/template": "^7.25.0", + "@babel/traverse": "^7.25.2", + "@babel/types": "^7.25.2", + "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": ">=6.9.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/babel" + } + }, + "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==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/generator": { + "version": "7.25.0", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.0.tgz", + "integrity": "sha512-3LEEcj3PVW8pW2R1SR1M89g/qrYk/m/mB/tLqn7dn4sbBUQyTqnlod+II2U4dqiGtUmkcnAmkMDralTFZttRiw==", + "dependencies": { + "@babel/types": "^7.25.0", + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.25", + "jsesc": "^2.5.1" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-compilation-targets": { + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.25.2.tgz", + "integrity": "sha512-U2U5LsSaZ7TAt3cfaymQ8WHh0pxvdHoEk6HVpaexxixjyEquMh0L0YNJNM6CTGKMXV1iksi0iZkGw4AcFkPaaw==", + "dependencies": { + "@babel/compat-data": "^7.25.2", + "@babel/helper-validator-option": "^7.24.8", + "browserslist": "^4.23.1", + "lru-cache": "^5.1.1", + "semver": "^6.3.1" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-5.1.1.tgz", + "integrity": "sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w==", + "license": "ISC", + "dependencies": { + "yallist": "^3.0.2" + } + }, + "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==", + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/@babel/helper-module-imports": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", + "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", + "dependencies": { + "@babel/traverse": "^7.24.7", + "@babel/types": "^7.24.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-module-transforms": { + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.25.2.tgz", + "integrity": "sha512-BjyRAbix6j/wv83ftcVJmBt72QtHI56C7JXZoG2xATiLpmoC7dpd8WnkikExHDVPpi/3qCmO6WY1EaXOluiecQ==", + "dependencies": { + "@babel/helper-module-imports": "^7.24.7", + "@babel/helper-simple-access": "^7.24.7", + "@babel/helper-validator-identifier": "^7.24.7", + "@babel/traverse": "^7.25.2" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/@babel/helper-plugin-utils": { + "version": "7.24.8", + "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.8.tgz", + "integrity": "sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-simple-access": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz", + "integrity": "sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==", + "dependencies": { + "@babel/traverse": "^7.24.7", + "@babel/types": "^7.24.7" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-string-parser": { + "version": "7.24.8", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", + "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-validator-identifier": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", + "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-validator-option": { + "version": "7.24.8", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.24.8.tgz", + "integrity": "sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==", + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helpers": { + "version": "7.25.0", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.0.tgz", + "integrity": "sha512-MjgLZ42aCm0oGjJj8CtSM3DB8NOOf8h2l7DCTePJs29u+v7yO/RBX9nShlKMgFnRks/Q4tBAe7Hxnov9VkGwLw==", + "dependencies": { + "@babel/template": "^7.25.0", + "@babel/types": "^7.25.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/highlight": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", + "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", + "dependencies": { + "@babel/helper-validator-identifier": "^7.24.7", + "chalk": "^2.4.2", + "js-tokens": "^4.0.0", + "picocolors": "^1.0.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "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==", + "dependencies": { + "color-convert": "^1.9.0" + }, + "engines": { + "node": ">=4" + } + }, + "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==", + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "engines": { + "node": ">=4" + } + }, + "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==", + "dependencies": { + "color-name": "1.1.3" + } + }, + "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==" + }, + "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==", + "engines": { + "node": ">=0.8.0" + } + }, + "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==", + "engines": { + "node": ">=4" + } + }, + "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==", + "dependencies": { + "has-flag": "^3.0.0" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/@babel/parser": { + "version": "7.25.3", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.3.tgz", + "integrity": "sha512-iLTJKDbJ4hMvFPgQwwsVoxtHyWpKKPBrxkANrSYewDPaPpT5py5yeVkgPIJ7XYXhndxJpaA3PyALSXQ7u8e/Dw==", + "dependencies": { + "@babel/types": "^7.25.2" + }, + "bin": { + "parser": "bin/babel-parser.js" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.12.13" + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-jsx": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz", + "integrity": "sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.24.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "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==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-typescript": { + "version": "7.24.7", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.7.tgz", + "integrity": "sha512-c/+fVeJBB0FeKsFvwytYiUD+LBvhHjGSI0g446PRGdSVGZLRNArBUno2PETbAly3tpiNAQR5XaZ+JslxkotsbA==", + "dependencies": { + "@babel/helper-plugin-utils": "^7.24.7" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/template": { + "version": "7.25.0", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", + "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", + "dependencies": { + "@babel/code-frame": "^7.24.7", + "@babel/parser": "^7.25.0", + "@babel/types": "^7.25.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/traverse": { + "version": "7.25.3", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.3.tgz", + "integrity": "sha512-HefgyP1x754oGCsKmV5reSmtV7IXj/kpaE1XYY+D9G5PvKKoFfSbiS4M77MdjuwlZKDIKFCffq9rPU+H/s3ZdQ==", + "dependencies": { + "@babel/code-frame": "^7.24.7", + "@babel/generator": "^7.25.0", + "@babel/parser": "^7.25.3", + "@babel/template": "^7.25.0", + "@babel/types": "^7.25.2", + "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==", + "engines": { + "node": ">=4" + } + }, + "node_modules/@babel/types": { + "version": "7.25.2", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.2.tgz", + "integrity": "sha512-YTnYtra7W9e6/oAZEHj0bJehPRUlLH9/fbpT5LfB0NhQXyALCRkRs3zH9v07IYhkgpqX6Z78FnuccZr/l4Fs4Q==", + "dependencies": { + "@babel/helper-string-parser": "^7.24.8", + "@babel/helper-validator-identifier": "^7.24.7", + "to-fast-properties": "^2.0.0" + }, + "engines": { + "node": ">=6.9.0" + } + }, + "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==" + }, + "node_modules/@bufbuild/buf": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf/-/buf-1.37.0.tgz", + "integrity": "sha512-33Rp90nRa7ebTwAjI8Yc7f8WOQMOEfRM0mg/o1bkNvE6cGQhYiq82Ag27pcegvYIyXl413XCeLMAUegsqjuFqw==", + "dev": true, + "hasInstallScript": true, + "bin": { + "buf": "bin/buf", + "protoc-gen-buf-breaking": "bin/protoc-gen-buf-breaking", + "protoc-gen-buf-lint": "bin/protoc-gen-buf-lint" + }, + "engines": { + "node": ">=12" + }, + "optionalDependencies": { + "@bufbuild/buf-darwin-arm64": "1.37.0", + "@bufbuild/buf-darwin-x64": "1.37.0", + "@bufbuild/buf-linux-aarch64": "1.37.0", + "@bufbuild/buf-linux-x64": "1.37.0", + "@bufbuild/buf-win32-arm64": "1.37.0", + "@bufbuild/buf-win32-x64": "1.37.0" + } + }, + "node_modules/@bufbuild/buf-darwin-arm64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-arm64/-/buf-darwin-arm64-1.37.0.tgz", + "integrity": "sha512-YrT0HqLf7qJJ9VjXmnG900oGOjskerqfY4UTbpLb67b76uRxZm7pfSfqpM4bCaSru4j6GPBDwH9dkwiuNOenEA==", + "cpu": [ + "arm64" + ], + "dev": true, + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-darwin-x64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-x64/-/buf-darwin-x64-1.37.0.tgz", + "integrity": "sha512-c1iDr1LaYcdzAK7v4ZBTiYVRPltPa1UNFQ3JJ6BI/RUaBk/eYFsykuNtsJQb5lXh/usjMz76juzOxAdYikcw7w==", + "cpu": [ + "x64" + ], + "dev": true, + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-linux-aarch64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-aarch64/-/buf-linux-aarch64-1.37.0.tgz", + "integrity": "sha512-ENaCLU4XZxQFc6+pWjNGeKITlzY0jYeEmz6R2cEeR9kVl+HyNNcTMaKxdffG5HNU5Ci1dC3pSY/ZIF3ck8UHVw==", + "cpu": [ + "arm64" + ], + "dev": true, + "optional": true, + "os": [ + "linux" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-linux-x64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-x64/-/buf-linux-x64-1.37.0.tgz", + "integrity": "sha512-/buQuUaBVU4rYqdoS/xmbyohJFaE4uy7boHMabCnI2aLsVBNPHYa/k0JMc+ZDRqgq0Wnth9VbIHn6DSGHG+cmw==", + "cpu": [ + "x64" + ], + "dev": true, + "optional": true, + "os": [ + "linux" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-win32-arm64": { + "version": "1.37.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-arm64/-/buf-win32-arm64-1.37.0.tgz", + "integrity": "sha512-5Y1aaHge9iQP+YNyxUayMd+rSVbjNXLBXniJDQhim56B+6eufPiKzmTgAKjDE9hGTT+mcBdLznrCMI0fzDLHJw==", + "cpu": [ + "arm64" + ], + "dev": true, + "optional": true, + "os": [ + "win32" + ], + "engines": { + "node": ">=12" + } + }, + "node_modules/@bufbuild/buf-win32-x64": { "version": "1.37.0", "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-x64/-/buf-win32-x64-1.37.0.tgz", "integrity": "sha512-2BioEPdC6EM5zEKmTmM14ZJuuPjKuYIvoAwQ4hoCSJBllIhGvshk61NDYBorr1huEEq6baa4ITL/gWHJ86B0bw==", "cpu": [ "x64" ], + "dev": true, "optional": true, "os": [ "win32" ], "engines": { - "node": ">=12" + "node": ">=12" + } + }, + "node_modules/@bufbuild/protobuf": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@bufbuild/protobuf/-/protobuf-2.0.0.tgz", + "integrity": "sha512-sw2JhwJyvyL0zlhG61aDzOVryEfJg2PDZFSV7i7IdC7nAE41WuXCru3QWLGiP87At0BMzKOoKO/FqEGoKygGZQ==" + }, + "node_modules/@bufbuild/protoc-gen-es": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@bufbuild/protoc-gen-es/-/protoc-gen-es-2.0.0.tgz", + "integrity": "sha512-xXDrDFjSD97jGu9RMA8O4Myj66DdCLOFeYG7Z0mX1aCE9hayymfor+BKgKCQouUfty6CYv8709FPYpEtDurLFQ==", + "dev": true, + "dependencies": { + "@bufbuild/protobuf": "^2.0.0", + "@bufbuild/protoplugin": "2.0.0" + }, + "bin": { + "protoc-gen-es": "bin/protoc-gen-es" + }, + "engines": { + "node": ">=14" + }, + "peerDependencies": { + "@bufbuild/protobuf": "2.0.0" + }, + "peerDependenciesMeta": { + "@bufbuild/protobuf": { + "optional": true + } + } + }, + "node_modules/@bufbuild/protoplugin": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@bufbuild/protoplugin/-/protoplugin-2.0.0.tgz", + "integrity": "sha512-y77IarGNbJfr5YZHQ+biO0O9B8rXicKLLOFRJu9mKr9LYE5/CLIKqonIBfkc/lggCK2YOtinjZJEYb39HymIog==", + "dev": true, + "dependencies": { + "@bufbuild/protobuf": "2.0.0", + "@typescript/vfs": "^1.5.2", + "typescript": "5.4.5" + } + }, + "node_modules/@bufbuild/protoplugin/node_modules/typescript": { + "version": "5.4.5", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.5.tgz", + "integrity": "sha512-vcI4UpRgg81oIRUFwR0WSIHKt11nJ7SAVlYNIu+QpqeyXP+gpQJy/Z4+F0aGxSE4MqwjyXvW/TzgkLAx2AGHwQ==", + "dev": true, + "bin": { + "tsc": "bin/tsc", + "tsserver": "bin/tsserver" + }, + "engines": { + "node": ">=14.17" + } + }, + "node_modules/@criteria/json-pointer": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/@criteria/json-pointer/-/json-pointer-0.2.1.tgz", + "integrity": "sha512-+UcEXcbZLkmhB4vuuuLAfnWD7cGmUGrZxzldFFdLwFKglkGuLiE4YTBX0gvEQuVHTwMN/f1R/0TB3YRzf5/BPw==", + "engines": { + "node": ">=18.12.1" + } + }, + "node_modules/@criteria/json-schema": { + "version": "0.10.0", + "resolved": "https://registry.npmjs.org/@criteria/json-schema/-/json-schema-0.10.0.tgz", + "integrity": "sha512-Y+gLIVPdgflQ3zeayUcfYjHOf+KJOp8dx5LWpLfy3XJ0P61JxU5srk2cGpVZYPJNTnixm8tBwb8T26DCMjw+hw==", + "dependencies": { + "@criteria/json-pointer": "^0.2.1", + "toad-uri-js": "^5.0.1" + }, + "engines": { + "node": ">=18.12.1" + } + }, + "node_modules/@criteria/json-schema-validation": { + "version": "0.10.0", + "resolved": "https://registry.npmjs.org/@criteria/json-schema-validation/-/json-schema-validation-0.10.0.tgz", + "integrity": "sha512-QqkO0uEAjMIEdybZPSGZEOa2HExAr83lE5ya3bRTVFMByb9sXq7JK7ZyHHAhSOWQyUN1Tgkk/sozfLygG5VOuw==", + "dependencies": { + "@criteria/json-pointer": "^0.2.1", + "@criteria/json-schema": "^0.10.0", + "fast-deep-equal": "^3.1.3", + "punycode": "^2.3.1", + "smtp-address-parser": "^1.0.10", + "toad-uri-js": "^5.0.1" + }, + "engines": { + "node": ">=18.12.1" + } + }, + "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.11.0", + "resolved": "https://registry.npmjs.org/@eslint-community/regexpp/-/regexpp-4.11.0.tgz", + "integrity": "sha512-G/M/tIiMrTAxEWRfLfQJMmGNX28IxBg4PBz8XqQhqUHLFI6TL2htpIB1iQCj144V5ee/JaKyT9/WZ0MGZWfA7A==", + "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/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/@eslint/eslintrc/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/@eslint/js": { + "version": "9.9.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.9.0.tgz", + "integrity": "sha512-hhetes6ZHP3BlXLxmd8K2SNgkhNSi+UcecbnwWKwpP7kyi/uC75DJ1lOOBO3xrC4jyojtGE3YxKZPHfk4yrgug==", + "dev": true, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.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/@google-cloud/kms": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/@google-cloud/kms/-/kms-4.5.0.tgz", + "integrity": "sha512-i2vC0DI7bdfEhQszqASTw0KVvbB7HsO2CwTBod423NawAu7FWi+gVVa7NLfXVNGJaZZayFfci2Hu+om/HmyEjQ==", + "dependencies": { + "google-gax": "^4.0.3" + }, + "engines": { + "node": ">=14.0.0" + } + }, + "node_modules/@grpc/grpc-js": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.11.1.tgz", + "integrity": "sha512-gyt/WayZrVPH2w/UTLansS7F9Nwld472JxxaETamrM8HNlsa+jSLNyKAZmhxI2Me4c3mQHFiS1wWHDY1g1Kthw==", + "dependencies": { + "@grpc/proto-loader": "^0.7.13", + "@js-sdsl/ordered-map": "^4.4.2" + }, + "engines": { + "node": ">=12.10.0" + } + }, + "node_modules/@grpc/proto-loader": { + "version": "0.7.13", + "resolved": "https://registry.npmjs.org/@grpc/proto-loader/-/proto-loader-0.7.13.tgz", + "integrity": "sha512-AiXO/bfe9bmxBjxxtYxFAXGZvMaN5s8kO+jBHAJCON8rJoB5YS/D6X7ZNc6XQkuHNmyl4CYaMI1fJ/Gn27RGGw==", + "dependencies": { + "lodash.camelcase": "^4.3.0", + "long": "^5.0.0", + "protobufjs": "^7.2.5", + "yargs": "^17.7.2" + }, + "bin": { + "proto-loader-gen-types": "build/bin/proto-loader-gen-types.js" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/@hackbg/miscreant-esm": { + "version": "0.3.2-patch.3", + "resolved": "https://registry.npmjs.org/@hackbg/miscreant-esm/-/miscreant-esm-0.3.2-patch.3.tgz", + "integrity": "sha512-CxWupG9uSc0e0toliMBc+ZSC2cXuHqcSgvwCgCiWXkpF6adzcttNFr6KyumI3utO/WQKQrdgq93LcQEueBjTHQ==" + }, + "node_modules/@humanwhocodes/config-array": { + "version": "0.11.14", + "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.14.tgz", + "integrity": "sha512-3T8LkOmg45BV5FICb15QQMsyUSWrQ8AygVfC7ZG32zOalnqrilm018ZVCw0eapXux8FtA33q8PSRSstjee3jSg==", + "deprecated": "Use @eslint/config-array instead", + "dev": true, + "dependencies": { + "@humanwhocodes/object-schema": "^2.0.2", + "debug": "^4.3.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.3", + "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.3.tgz", + "integrity": "sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA==", + "deprecated": "Use @eslint/object-schema instead", + "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", + "integrity": "sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ==", + "dependencies": { + "camelcase": "^5.3.1", + "find-up": "^4.1.0", + "get-package-type": "^0.1.0", + "js-yaml": "^3.13.1", + "resolve-from": "^5.0.0" + }, + "engines": { + "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==", + "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==", + "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==", + "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==", + "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==", + "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==", + "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==", + "engines": { + "node": ">=8" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/sprintf-js": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", + "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==" + }, + "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==", + "engines": { + "node": ">=8" + } + }, + "node_modules/@jest/console": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/console/-/console-29.7.0.tgz", + "integrity": "sha512-5Ni4CU7XHQi32IJ398EEP4RrB8eV09sXP2ROqD4bksHrnTree52PsxvX8tpL8LvTZ3pFzXyPbNQReSN41CAhOg==", + "dependencies": { + "@jest/types": "^29.6.3", + "@types/node": "*", + "chalk": "^4.0.0", + "jest-message-util": "^29.7.0", + "jest-util": "^29.7.0", + "slash": "^3.0.0" + }, + "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==", + "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": "^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/environment": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/environment/-/environment-29.7.0.tgz", + "integrity": "sha512-aQIfHDq33ExsN4jP1NWGXhxgQ/wixs60gDiKO+XVMd8Mn0NWPWgc34ZQDTb2jKaUWQ7MuwoitXAsN2XVXNMpAw==", + "dependencies": { + "@jest/fake-timers": "^29.7.0", + "@jest/types": "^29.6.3", + "@types/node": "*", + "jest-mock": "^29.7.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "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==", + "dependencies": { + "expect": "^29.7.0", + "jest-snapshot": "^29.7.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "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==", + "dependencies": { + "jest-get-type": "^29.6.3" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "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==", + "dependencies": { + "@jest/types": "^29.6.3", + "@sinonjs/fake-timers": "^10.0.2", + "@types/node": "*", + "jest-message-util": "^29.7.0", + "jest-mock": "^29.7.0", + "jest-util": "^29.7.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/@bufbuild/protobuf": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@bufbuild/protobuf/-/protobuf-2.0.0.tgz", - "integrity": "sha512-sw2JhwJyvyL0zlhG61aDzOVryEfJg2PDZFSV7i7IdC7nAE41WuXCru3QWLGiP87At0BMzKOoKO/FqEGoKygGZQ==" - }, - "node_modules/@bufbuild/protoc-gen-es": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@bufbuild/protoc-gen-es/-/protoc-gen-es-2.0.0.tgz", - "integrity": "sha512-xXDrDFjSD97jGu9RMA8O4Myj66DdCLOFeYG7Z0mX1aCE9hayymfor+BKgKCQouUfty6CYv8709FPYpEtDurLFQ==", + "node_modules/@jest/globals": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/globals/-/globals-29.7.0.tgz", + "integrity": "sha512-mpiz3dutLbkW2MNFubUGUEVLkTGiqW6yLVTA+JbP6fI6J5iL9Y0Nlg8k95pcF8ctKwCS7WVxteBs29hhfAotzQ==", "dependencies": { - "@bufbuild/protobuf": "^2.0.0", - "@bufbuild/protoplugin": "2.0.0" + "@jest/environment": "^29.7.0", + "@jest/expect": "^29.7.0", + "@jest/types": "^29.6.3", + "jest-mock": "^29.7.0" }, - "bin": { - "protoc-gen-es": "bin/protoc-gen-es" + "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==", + "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" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" }, "peerDependencies": { - "@bufbuild/protobuf": "2.0.0" + "node-notifier": "^8.0.1 || ^9.0.0 || ^10.0.0" }, "peerDependenciesMeta": { - "@bufbuild/protobuf": { + "node-notifier": { "optional": true } } }, - "node_modules/@bufbuild/protoplugin": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@bufbuild/protoplugin/-/protoplugin-2.0.0.tgz", - "integrity": "sha512-y77IarGNbJfr5YZHQ+biO0O9B8rXicKLLOFRJu9mKr9LYE5/CLIKqonIBfkc/lggCK2YOtinjZJEYb39HymIog==", + "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==", "dependencies": { - "@bufbuild/protobuf": "2.0.0", - "@typescript/vfs": "^1.5.2", - "typescript": "5.4.5" + "@sinclair/typebox": "^0.27.8" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/@bufbuild/protoplugin/node_modules/typescript": { - "version": "5.4.5", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.5.tgz", - "integrity": "sha512-vcI4UpRgg81oIRUFwR0WSIHKt11nJ7SAVlYNIu+QpqeyXP+gpQJy/Z4+F0aGxSE4MqwjyXvW/TzgkLAx2AGHwQ==", - "bin": { - "tsc": "bin/tsc", - "tsserver": "bin/tsserver" + "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==", + "dependencies": { + "@jridgewell/trace-mapping": "^0.3.18", + "callsites": "^3.0.0", + "graceful-fs": "^4.2.9" }, "engines": { - "node": ">=14.17" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/@criteria/json-pointer": { - "version": "0.2.1", - "resolved": "https://registry.npmjs.org/@criteria/json-pointer/-/json-pointer-0.2.1.tgz", - "integrity": "sha512-+UcEXcbZLkmhB4vuuuLAfnWD7cGmUGrZxzldFFdLwFKglkGuLiE4YTBX0gvEQuVHTwMN/f1R/0TB3YRzf5/BPw==", + "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==", + "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==", + "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==", + "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==", + "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.5", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", + "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", + "dependencies": { + "@jridgewell/set-array": "^1.2.1", + "@jridgewell/sourcemap-codec": "^1.4.10", + "@jridgewell/trace-mapping": "^0.3.24" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/resolve-uri": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", + "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/set-array": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", + "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/sourcemap-codec": { + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==" + }, + "node_modules/@jridgewell/trace-mapping": { + "version": "0.3.25", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", + "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", + "dependencies": { + "@jridgewell/resolve-uri": "^3.1.0", + "@jridgewell/sourcemap-codec": "^1.4.14" + } + }, + "node_modules/@js-sdsl/ordered-map": { + "version": "4.4.2", + "resolved": "https://registry.npmjs.org/@js-sdsl/ordered-map/-/ordered-map-4.4.2.tgz", + "integrity": "sha512-iUKgm52T8HOE/makSxjqoWhe95ZJA1/G1sYsGev2JDKUSS14KAgg1LHb+Ba+IPow0xflbnSkOsZcO08C7w1gYw==", + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/js-sdsl" + } + }, + "node_modules/@jsdoc/salty": { + "version": "0.2.8", + "resolved": "https://registry.npmjs.org/@jsdoc/salty/-/salty-0.2.8.tgz", + "integrity": "sha512-5e+SFVavj1ORKlKaKr2BmTOekmXbelU7dC0cDkQLqag7xfuTPuGMUFx7KWJuv4bYZrTsoL2Z18VVCOKYxzoHcg==", + "dev": true, + "dependencies": { + "lodash": "^4.17.21" + }, + "engines": { + "node": ">=v12.0.0" + } + }, + "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/@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": ">=18.12.1" + "node": ">= 8" } }, - "node_modules/@criteria/json-schema": { - "version": "0.10.0", - "resolved": "https://registry.npmjs.org/@criteria/json-schema/-/json-schema-0.10.0.tgz", - "integrity": "sha512-Y+gLIVPdgflQ3zeayUcfYjHOf+KJOp8dx5LWpLfy3XJ0P61JxU5srk2cGpVZYPJNTnixm8tBwb8T26DCMjw+hw==", + "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": { - "@criteria/json-pointer": "^0.2.1", - "toad-uri-js": "^5.0.1" + "@nodelib/fs.scandir": "2.1.5", + "fastq": "^1.6.0" }, "engines": { - "node": ">=18.12.1" + "node": ">= 8" } }, - "node_modules/@criteria/json-schema-validation": { - "version": "0.10.0", - "resolved": "https://registry.npmjs.org/@criteria/json-schema-validation/-/json-schema-validation-0.10.0.tgz", - "integrity": "sha512-QqkO0uEAjMIEdybZPSGZEOa2HExAr83lE5ya3bRTVFMByb9sXq7JK7ZyHHAhSOWQyUN1Tgkk/sozfLygG5VOuw==", + "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": { - "@criteria/json-pointer": "^0.2.1", - "@criteria/json-schema": "^0.10.0", - "fast-deep-equal": "^3.1.3", - "punycode": "^2.3.1", - "smtp-address-parser": "^1.0.10", - "toad-uri-js": "^5.0.1" + "@gar/promisify": "^1.1.3", + "semver": "^7.3.5" }, "engines": { - "node": ">=18.12.1" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, - "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==", + "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": { - "eslint-visitor-keys": "^3.3.0" + "mkdirp": "^1.0.4", + "rimraf": "^3.0.2" }, "engines": { - "node": "^12.22.0 || ^14.17.0 || >=16.0.0" - }, - "peerDependencies": { - "eslint": "^6.0.0 || ^7.0.0 || >=8.0.0" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, - "node_modules/@eslint-community/regexpp": { - "version": "4.11.0", - "resolved": "https://registry.npmjs.org/@eslint-community/regexpp/-/regexpp-4.11.0.tgz", - "integrity": "sha512-G/M/tIiMrTAxEWRfLfQJMmGNX28IxBg4PBz8XqQhqUHLFI6TL2htpIB1iQCj144V5ee/JaKyT9/WZ0MGZWfA7A==", - "dev": true, + "node_modules/@postman/form-data": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/@postman/form-data/-/form-data-3.1.1.tgz", + "integrity": "sha512-vjh8Q2a8S6UCm/KKs31XFJqEEgmbjBmpPNVV2eVav6905wyFAwaUOBGA1NPBI4ERH9MMZc6w0umFgM6WbEPMdg==", + "dependencies": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.8", + "mime-types": "^2.1.12" + }, "engines": { - "node": "^12.0.0 || ^14.0.0 || >=16.0.0" + "node": ">= 6" } }, - "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, + "node_modules/@postman/tough-cookie": { + "version": "4.1.3-postman.1", + "resolved": "https://registry.npmjs.org/@postman/tough-cookie/-/tough-cookie-4.1.3-postman.1.tgz", + "integrity": "sha512-txpgUqZOnWYnUHZpHjkfb0IwVH4qJmyq77pPnJLlfhMtdCLMFTEeQHlzQiK906aaNCe4NEB5fGJHo9uzGbFMeA==", "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" + "psl": "^1.1.33", + "punycode": "^2.1.1", + "universalify": "^0.2.0", + "url-parse": "^1.5.3" }, "engines": { - "node": "^12.22.0 || ^14.17.0 || >=16.0.0" - }, - "funding": { - "url": "https://opencollective.com/eslint" + "node": ">=6" } }, - "node_modules/@eslint/eslintrc/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, + "node_modules/@postman/tunnel-agent": { + "version": "0.6.4", + "resolved": "https://registry.npmjs.org/@postman/tunnel-agent/-/tunnel-agent-0.6.4.tgz", + "integrity": "sha512-CJJlq8V7rNKhAw4sBfjixKpJW00SHqebqNUQKxMoepgeWZIbdPcD+rguRcivGhS4N12PymDcKgUgSD4rVC+RjQ==", "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" + "safe-buffer": "^5.0.1" }, - "funding": { - "type": "github", - "url": "https://github.com/sponsors/epoberezkin" + "engines": { + "node": "*" } }, - "node_modules/@eslint/eslintrc/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/@protobufjs/aspromise": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@protobufjs/aspromise/-/aspromise-1.1.2.tgz", + "integrity": "sha512-j+gKExEuLmKwvz3OgROXtrJ2UG2x8Ch2YZUxahh+s1F2HZ+wAceUNLkvy6zKCPVRkU++ZWQrdxsUeQXmcg4uoQ==" }, - "node_modules/@eslint/js": { - "version": "9.9.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.9.0.tgz", - "integrity": "sha512-hhetes6ZHP3BlXLxmd8K2SNgkhNSi+UcecbnwWKwpP7kyi/uC75DJ1lOOBO3xrC4jyojtGE3YxKZPHfk4yrgug==", - "dev": true, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - } + "node_modules/@protobufjs/base64": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@protobufjs/base64/-/base64-1.1.2.tgz", + "integrity": "sha512-AZkcAA5vnN/v4PDqKyMR5lx7hZttPDgClv83E//FMNhR2TMcLUhfRUBHCmSl0oi9zMgDDqRUJkSxO3wm85+XLg==" }, - "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/@protobufjs/codegen": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/@protobufjs/codegen/-/codegen-2.0.4.tgz", + "integrity": "sha512-YyFaikqM5sH0ziFZCN3xDC7zeGaB/d0IUb9CATugHWbd1FRFwWwt4ld4OYMPWu5a3Xe01mGAULCdqhMlPl29Jg==" }, - "node_modules/@hackbg/miscreant-esm": { - "version": "0.3.2-patch.3", - "resolved": "https://registry.npmjs.org/@hackbg/miscreant-esm/-/miscreant-esm-0.3.2-patch.3.tgz", - "integrity": "sha512-CxWupG9uSc0e0toliMBc+ZSC2cXuHqcSgvwCgCiWXkpF6adzcttNFr6KyumI3utO/WQKQrdgq93LcQEueBjTHQ==" + "node_modules/@protobufjs/eventemitter": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@protobufjs/eventemitter/-/eventemitter-1.1.0.tgz", + "integrity": "sha512-j9ednRT81vYJ9OfVuXG6ERSTdEL1xVsNgqpkxMsbIabzSo3goCjDIveeGv5d03om39ML71RdmrGNjG5SReBP/Q==" }, - "node_modules/@humanwhocodes/config-array": { - "version": "0.11.14", - "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.14.tgz", - "integrity": "sha512-3T8LkOmg45BV5FICb15QQMsyUSWrQ8AygVfC7ZG32zOalnqrilm018ZVCw0eapXux8FtA33q8PSRSstjee3jSg==", - "deprecated": "Use @eslint/config-array instead", - "dev": true, + "node_modules/@protobufjs/fetch": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@protobufjs/fetch/-/fetch-1.1.0.tgz", + "integrity": "sha512-lljVXpqXebpsijW71PZaCYeIcE5on1w5DlQy5WH6GLbFryLUrBD4932W/E2BSpfRJWseIL4v/KPgBFxDOIdKpQ==", "dependencies": { - "@humanwhocodes/object-schema": "^2.0.2", - "debug": "^4.3.1", - "minimatch": "^3.0.5" - }, - "engines": { - "node": ">=10.10.0" + "@protobufjs/aspromise": "^1.1.1", + "@protobufjs/inquire": "^1.1.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/@protobufjs/float": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/@protobufjs/float/-/float-1.0.2.tgz", + "integrity": "sha512-Ddb+kVXlXst9d+R9PfTIxh1EdNkgoRe5tOX6t01f1lYWOvJnSPDBlG241QLzcyPdoNTsblLUdujGSE4RzrTZGQ==" }, - "node_modules/@humanwhocodes/object-schema": { - "version": "2.0.3", - "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.3.tgz", - "integrity": "sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA==", - "deprecated": "Use @eslint/object-schema instead", - "dev": true + "node_modules/@protobufjs/inquire": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@protobufjs/inquire/-/inquire-1.1.0.tgz", + "integrity": "sha512-kdSefcPdruJiFMVSbn801t4vFK7KB/5gd2fYvrxhuJYg8ILrmn9SKSX2tZdV6V+ksulWqS7aXjBcRXl3wHoD9Q==" }, - "node_modules/@istanbuljs/load-nyc-config": { + "node_modules/@protobufjs/path": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@protobufjs/path/-/path-1.1.2.tgz", + "integrity": "sha512-6JOcJ5Tm08dOHAbdR3GrvP+yUUfkjG5ePsHYczMFLq3ZmMkAD98cDgcT2iA1lJ9NVwFd4tH/iSSoe44YWkltEA==" + }, + "node_modules/@protobufjs/pool": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", - "integrity": "sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ==", + "resolved": "https://registry.npmjs.org/@protobufjs/pool/-/pool-1.1.0.tgz", + "integrity": "sha512-0kELaGSIDBKvcgS4zkjz1PeddatrjYcmMWOlAuAPwAeccUrPHdUqo/J6LiymHHEiJT5NrF1UVwxY14f+fy4WQw==" + }, + "node_modules/@protobufjs/utf8": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@protobufjs/utf8/-/utf8-1.1.0.tgz", + "integrity": "sha512-Vvn3zZrhQZkkBE8LSuW3em98c0FwgO4nxzv6OdSxPKJIEKY2bGbHn+mhGIPerzI4twdxaP8/0+06HBpwf345Lw==" + }, + "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==" + }, + "node_modules/@sinonjs/commons": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.1.tgz", + "integrity": "sha512-K3mCHKQ9sVh8o1C9cxkwxaOmXoAMlDxC1mYyHrjqOWEcBjYr76t96zL2zlj5dUGZ3HSw240X1qgH3Mjf1yJWpQ==", "dependencies": { - "camelcase": "^5.3.1", - "find-up": "^4.1.0", - "get-package-type": "^0.1.0", - "js-yaml": "^3.13.1", - "resolve-from": "^5.0.0" - }, - "engines": { - "node": ">=8" + "type-detect": "4.0.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==", + "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==", "dependencies": { - "sprintf-js": "~1.0.2" + "@sinonjs/commons": "^3.0.0" } }, - "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==", + "node_modules/@smithy/abort-controller": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/@smithy/abort-controller/-/abort-controller-3.1.1.tgz", + "integrity": "sha512-MBJBiidoe+0cTFhyxT8g+9g7CeVccLM0IOKKUMCNQ1CNMJ/eIfoo0RTfVrXOONEI1UCN1W+zkiHSbzUNE9dZtQ==", "dependencies": { - "locate-path": "^5.0.0", - "path-exists": "^4.0.0" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=8" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/config-resolver": { + "version": "3.0.5", + "resolved": "https://registry.npmjs.org/@smithy/config-resolver/-/config-resolver-3.0.5.tgz", + "integrity": "sha512-SkW5LxfkSI1bUC74OtfBbdz+grQXYiPYolyu8VfpLIjEoN/sHVBlLeGXMQ1vX4ejkgfv6sxVbQJ32yF2cl1veA==", "dependencies": { - "argparse": "^1.0.7", - "esprima": "^4.0.0" + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", + "@smithy/util-config-provider": "^3.0.0", + "@smithy/util-middleware": "^3.0.3", + "tslib": "^2.6.2" }, - "bin": { - "js-yaml": "bin/js-yaml.js" + "engines": { + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/core": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/@smithy/core/-/core-2.4.0.tgz", + "integrity": "sha512-cHXq+FneIF/KJbt4q4pjN186+Jf4ZB0ZOqEaZMBhT79srEyGDDBV31NqBRBjazz8ppQ1bJbDJMY9ba5wKFV36w==", "dependencies": { - "p-locate": "^4.1.0" + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-body-length-browser": "^3.0.0", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=8" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/credential-provider-imds": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/@smithy/credential-provider-imds/-/credential-provider-imds-3.2.0.tgz", + "integrity": "sha512-0SCIzgd8LYZ9EJxUjLXBmEKSZR/P/w6l7Rz/pab9culE/RWuqelAKGJvn5qUOl8BgX8Yj5HWM50A5hiB/RzsgA==", "dependencies": { - "p-try": "^2.0.0" + "@smithy/node-config-provider": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "tslib": "^2.6.2" }, "engines": { - "node": ">=6" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/fetch-http-handler": { + "version": "3.2.4", + "resolved": "https://registry.npmjs.org/@smithy/fetch-http-handler/-/fetch-http-handler-3.2.4.tgz", + "integrity": "sha512-kBprh5Gs5h7ug4nBWZi1FZthdqSM+T7zMmsZxx0IBvWUn7dK3diz2SHn7Bs4dQGFDk8plDv375gzenDoNwrXjg==", "dependencies": { - "p-limit": "^2.2.0" + "@smithy/protocol-http": "^4.1.0", + "@smithy/querystring-builder": "^3.0.3", + "@smithy/types": "^3.3.0", + "@smithy/util-base64": "^3.0.0", + "tslib": "^2.6.2" + } + }, + "node_modules/@smithy/hash-node": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/hash-node/-/hash-node-3.0.3.tgz", + "integrity": "sha512-2ctBXpPMG+B3BtWSGNnKELJ7SH9e4TNefJS0cd2eSkOOROeBnnVBnAy9LtJ8tY4vUEoe55N4CNPxzbWvR39iBw==", + "dependencies": { + "@smithy/types": "^3.3.0", + "@smithy/util-buffer-from": "^3.0.0", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">=8" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/invalid-dependency": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/invalid-dependency/-/invalid-dependency-3.0.3.tgz", + "integrity": "sha512-ID1eL/zpDULmHJbflb864k72/SNOZCADRc9i7Exq3RUNJw6raWUSlFEQ+3PX3EYs++bTxZB2dE9mEHTQLv61tw==", + "dependencies": { + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + } + }, + "node_modules/@smithy/is-array-buffer": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/is-array-buffer/-/is-array-buffer-3.0.0.tgz", + "integrity": "sha512-+Fsu6Q6C4RSJiy81Y8eApjEB5gVtM+oFKTffg+jSuwtvomJJrhUJBu2zS8wjXSgH/g1MKEWrzyChTBe6clb5FQ==", + "dependencies": { + "tslib": "^2.6.2" + }, "engines": { - "node": ">=8" + "node": ">=16.0.0" } }, - "node_modules/@istanbuljs/load-nyc-config/node_modules/sprintf-js": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", - "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==" - }, - "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==", + "node_modules/@smithy/middleware-content-length": { + "version": "3.0.5", + "resolved": "https://registry.npmjs.org/@smithy/middleware-content-length/-/middleware-content-length-3.0.5.tgz", + "integrity": "sha512-ILEzC2eyxx6ncej3zZSwMpB5RJ0zuqH7eMptxC4KN3f+v9bqT8ohssKbhNR78k/2tWW+KS5Spw+tbPF4Ejyqvw==", + "dependencies": { + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=8" + "node": ">=16.0.0" } }, - "node_modules/@jest/console": { - "version": "29.7.0", - "resolved": "https://registry.npmjs.org/@jest/console/-/console-29.7.0.tgz", - "integrity": "sha512-5Ni4CU7XHQi32IJ398EEP4RrB8eV09sXP2ROqD4bksHrnTree52PsxvX8tpL8LvTZ3pFzXyPbNQReSN41CAhOg==", + "node_modules/@smithy/middleware-endpoint": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/@smithy/middleware-endpoint/-/middleware-endpoint-3.1.0.tgz", + "integrity": "sha512-5y5aiKCEwg9TDPB4yFE7H6tYvGFf1OJHNczeY10/EFF8Ir8jZbNntQJxMWNfeQjC1mxPsaQ6mR9cvQbf+0YeMw==", "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" + "@smithy/middleware-serde": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "@smithy/util-middleware": "^3.0.3", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/middleware-retry": { + "version": "3.0.15", + "resolved": "https://registry.npmjs.org/@smithy/middleware-retry/-/middleware-retry-3.0.15.tgz", + "integrity": "sha512-iTMedvNt1ApdvkaoE8aSDuwaoc+BhvHqttbA/FO4Ty+y/S5hW6Ci/CTScG7vam4RYJWZxdTElc3MEfHRVH6cgQ==", "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" + "@smithy/node-config-provider": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/service-error-classification": "^3.0.3", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", + "tslib": "^2.6.2", + "uuid": "^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": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/middleware-serde": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/middleware-serde/-/middleware-serde-3.0.3.tgz", + "integrity": "sha512-puUbyJQBcg9eSErFXjKNiGILJGtiqmuuNKEYNYfUD57fUl4i9+mfmThtQhvFXU0hCVG0iEJhvQUipUf+/SsFdA==", "dependencies": { - "@jest/fake-timers": "^29.7.0", - "@jest/types": "^29.6.3", - "@types/node": "*", - "jest-mock": "^29.7.0" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/middleware-stack": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/middleware-stack/-/middleware-stack-3.0.3.tgz", + "integrity": "sha512-r4klY9nFudB0r9UdSMaGSyjyQK5adUyPnQN/ZM6M75phTxOdnc/AhpvGD1fQUvgmqjQEBGCwpnPbDm8pH5PapA==", "dependencies": { - "expect": "^29.7.0", - "jest-snapshot": "^29.7.0" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/node-config-provider": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@smithy/node-config-provider/-/node-config-provider-3.1.4.tgz", + "integrity": "sha512-YvnElQy8HR4vDcAjoy7Xkx9YT8xZP4cBXcbJSgm/kxmiQu08DwUwj8rkGnyoJTpfl/3xYHH+d8zE+eHqoDCSdQ==", "dependencies": { - "jest-get-type": "^29.6.3" + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/node-http-handler": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@smithy/node-http-handler/-/node-http-handler-3.1.4.tgz", + "integrity": "sha512-+UmxgixgOr/yLsUxcEKGH0fMNVteJFGkmRltYFHnBMlogyFdpzn2CwqWmxOrfJELhV34v0WSlaqG1UtE1uXlJg==", "dependencies": { - "@jest/types": "^29.6.3", - "@sinonjs/fake-timers": "^10.0.2", - "@types/node": "*", - "jest-message-util": "^29.7.0", - "jest-mock": "^29.7.0", - "jest-util": "^29.7.0" + "@smithy/abort-controller": "^3.1.1", + "@smithy/protocol-http": "^4.1.0", + "@smithy/querystring-builder": "^3.0.3", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/property-provider": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/@smithy/property-provider/-/property-provider-3.1.3.tgz", + "integrity": "sha512-zahyOVR9Q4PEoguJ/NrFP4O7SMAfYO1HLhB18M+q+Z4KFd4V2obiMnlVoUFzFLSPeVt1POyNWneHHrZaTMoc/g==", "dependencies": { - "@jest/environment": "^29.7.0", - "@jest/expect": "^29.7.0", - "@jest/types": "^29.6.3", - "jest-mock": "^29.7.0" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/protocol-http": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/@smithy/protocol-http/-/protocol-http-4.1.0.tgz", + "integrity": "sha512-dPVoHYQ2wcHooGXg3LQisa1hH0e4y0pAddPMeeUPipI1tEOqL6A4N0/G7abeq+K8wrwSgjk4C0wnD1XZpJm5aA==", "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" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "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": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/querystring-builder": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/querystring-builder/-/querystring-builder-3.0.3.tgz", + "integrity": "sha512-vyWckeUeesFKzCDaRwWLUA1Xym9McaA6XpFfAK5qI9DKJ4M33ooQGqvM4J+LalH4u/Dq9nFiC8U6Qn1qi0+9zw==", "dependencies": { - "@sinclair/typebox": "^0.27.8" + "@smithy/types": "^3.3.0", + "@smithy/util-uri-escape": "^3.0.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/querystring-parser": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/querystring-parser/-/querystring-parser-3.0.3.tgz", + "integrity": "sha512-zahM1lQv2YjmznnfQsWbYojFe55l0SLG/988brlLv1i8z3dubloLF+75ATRsqPBboUXsW6I9CPGE5rQgLfY0vQ==", "dependencies": { - "@jridgewell/trace-mapping": "^0.3.18", - "callsites": "^3.0.0", - "graceful-fs": "^4.2.9" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/service-error-classification": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/service-error-classification/-/service-error-classification-3.0.3.tgz", + "integrity": "sha512-Jn39sSl8cim/VlkLsUhRFq/dKDnRUFlfRkvhOJaUbLBXUsLRLNf9WaxDv/z9BjuQ3A6k/qE8af1lsqcwm7+DaQ==", "dependencies": { - "@jest/console": "^29.7.0", - "@jest/types": "^29.6.3", - "@types/istanbul-lib-coverage": "^2.0.0", - "collect-v8-coverage": "^1.0.0" + "@smithy/types": "^3.3.0" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/shared-ini-file-loader": { + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@smithy/shared-ini-file-loader/-/shared-ini-file-loader-3.1.4.tgz", + "integrity": "sha512-qMxS4hBGB8FY2GQqshcRUy1K6k8aBWP5vwm8qKkCT3A9K2dawUwOIJfqh9Yste/Bl0J2lzosVyrXDj68kLcHXQ==", "dependencies": { - "@jest/test-result": "^29.7.0", - "graceful-fs": "^4.2.9", - "jest-haste-map": "^29.7.0", - "slash": "^3.0.0" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/signature-v4": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/@smithy/signature-v4/-/signature-v4-4.1.0.tgz", + "integrity": "sha512-aRryp2XNZeRcOtuJoxjydO6QTaVhxx/vjaR+gx7ZjaFgrgPRyZ3HCTbfwqYj6ZWEBHkCSUfcaymKPURaByukag==", "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" + "@smithy/is-array-buffer": "^3.0.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "@smithy/util-hex-encoding": "^3.0.0", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-uri-escape": "^3.0.0", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.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==", + "node_modules/@smithy/smithy-client": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/@smithy/smithy-client/-/smithy-client-3.2.0.tgz", + "integrity": "sha512-pDbtxs8WOhJLJSeaF/eAbPgXg4VVYFlRcL/zoNYA5WbG3wBL06CHtBSg53ppkttDpAJ/hdiede+xApip1CwSLw==", "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" + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "@smithy/util-stream": "^3.1.3", + "tslib": "^2.6.2" }, "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + "node": ">=16.0.0" } }, - "node_modules/@jridgewell/gen-mapping": { - "version": "0.3.5", - "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", - "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", + "node_modules/@smithy/types": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/@smithy/types/-/types-3.3.0.tgz", + "integrity": "sha512-IxvBBCTFDHbVoK7zIxqA1ZOdc4QfM5HM7rGleCuHi7L1wnKv5Pn69xXJQ9hgxH60ZVygH9/JG0jRgtUncE3QUA==", "dependencies": { - "@jridgewell/set-array": "^1.2.1", - "@jridgewell/sourcemap-codec": "^1.4.10", - "@jridgewell/trace-mapping": "^0.3.24" + "tslib": "^2.6.2" }, "engines": { - "node": ">=6.0.0" + "node": ">=16.0.0" } }, - "node_modules/@jridgewell/resolve-uri": { - "version": "3.1.2", - "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", - "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", - "engines": { - "node": ">=6.0.0" + "node_modules/@smithy/url-parser": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/url-parser/-/url-parser-3.0.3.tgz", + "integrity": "sha512-pw3VtZtX2rg+s6HMs6/+u9+hu6oY6U7IohGhVNnjbgKy86wcIsSZwgHrFR+t67Uyxvp4Xz3p3kGXXIpTNisq8A==", + "dependencies": { + "@smithy/querystring-parser": "^3.0.3", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" } }, - "node_modules/@jridgewell/set-array": { - "version": "1.2.1", - "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", - "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", + "node_modules/@smithy/util-base64": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-base64/-/util-base64-3.0.0.tgz", + "integrity": "sha512-Kxvoh5Qtt0CDsfajiZOCpJxgtPHXOKwmM+Zy4waD43UoEMA+qPxxa98aE/7ZhdnBFZFXMOiBR5xbcaMhLtznQQ==", + "dependencies": { + "@smithy/util-buffer-from": "^3.0.0", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">=6.0.0" + "node": ">=16.0.0" } }, - "node_modules/@jridgewell/sourcemap-codec": { - "version": "1.5.0", - "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", - "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==" + "node_modules/@smithy/util-body-length-browser": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-body-length-browser/-/util-body-length-browser-3.0.0.tgz", + "integrity": "sha512-cbjJs2A1mLYmqmyVl80uoLTJhAcfzMOyPgjwAYusWKMdLeNtzmMz9YxNl3/jRLoxSS3wkqkf0jwNdtXWtyEBaQ==", + "dependencies": { + "tslib": "^2.6.2" + } }, - "node_modules/@jridgewell/trace-mapping": { - "version": "0.3.25", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", - "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", + "node_modules/@smithy/util-body-length-node": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-body-length-node/-/util-body-length-node-3.0.0.tgz", + "integrity": "sha512-Tj7pZ4bUloNUP6PzwhN7K386tmSmEET9QtQg0TgdNOnxhZvCssHji+oZTUIuzxECRfG8rdm2PMw2WCFs6eIYkA==", + "dependencies": { + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } + }, + "node_modules/@smithy/util-buffer-from": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-buffer-from/-/util-buffer-from-3.0.0.tgz", + "integrity": "sha512-aEOHCgq5RWFbP+UDPvPot26EJHjOC+bRgse5A8V3FSShqd5E5UN4qc7zkwsvJPPAVsf73QwYcHN1/gt/rtLwQA==", "dependencies": { - "@jridgewell/resolve-uri": "^3.1.0", - "@jridgewell/sourcemap-codec": "^1.4.14" + "@smithy/is-array-buffer": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" } }, - "node_modules/@jsdoc/salty": { - "version": "0.2.8", - "resolved": "https://registry.npmjs.org/@jsdoc/salty/-/salty-0.2.8.tgz", - "integrity": "sha512-5e+SFVavj1ORKlKaKr2BmTOekmXbelU7dC0cDkQLqag7xfuTPuGMUFx7KWJuv4bYZrTsoL2Z18VVCOKYxzoHcg==", - "dev": true, + "node_modules/@smithy/util-config-provider": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-config-provider/-/util-config-provider-3.0.0.tgz", + "integrity": "sha512-pbjk4s0fwq3Di/ANL+rCvJMKM5bzAQdE5S/6RL5NXgMExFAi6UgQMPOm5yPaIWPpr+EOXKXRonJ3FoxKf4mCJQ==", "dependencies": { - "lodash": "^4.17.21" + "tslib": "^2.6.2" }, "engines": { - "node": ">=v12.0.0" + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/util-defaults-mode-browser": { + "version": "3.0.15", + "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-browser/-/util-defaults-mode-browser-3.0.15.tgz", + "integrity": "sha512-FZ4Psa3vjp8kOXcd3HJOiDPBCWtiilLl57r0cnNtq/Ga9RSDrM5ERL6xt+tO43+2af6Pn5Yp92x2n5vPuduNfg==", "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" + "@smithy/property-provider": "^3.1.3", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "bowser": "^2.11.0", + "tslib": "^2.6.2" }, - "bin": { - "node-pre-gyp": "bin/node-pre-gyp" + "engines": { + "node": ">= 10.0.0" } }, - "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, + "node_modules/@smithy/util-defaults-mode-node": { + "version": "3.0.15", + "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-node/-/util-defaults-mode-node-3.0.15.tgz", + "integrity": "sha512-KSyAAx2q6d0t6f/S4XB2+3+6aQacm3aLMhs9aLMqn18uYGUepbdssfogW5JQZpc6lXNBnp0tEnR5e9CEKmEd7A==", "dependencies": { - "@nodelib/fs.stat": "2.0.5", - "run-parallel": "^1.1.9" + "@smithy/config-resolver": "^3.0.5", + "@smithy/credential-provider-imds": "^3.2.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": ">= 8" + "node": ">= 10.0.0" } }, - "node_modules/@nodelib/fs.stat": { + "node_modules/@smithy/util-endpoints": { "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, + "resolved": "https://registry.npmjs.org/@smithy/util-endpoints/-/util-endpoints-2.0.5.tgz", + "integrity": "sha512-ReQP0BWihIE68OAblC/WQmDD40Gx+QY1Ez8mTdFMXpmjfxSyz2fVQu3A4zXRfQU9sZXtewk3GmhfOHswvX+eNg==", + "dependencies": { + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" + }, "engines": { - "node": ">= 8" + "node": ">=16.0.0" } }, - "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, + "node_modules/@smithy/util-hex-encoding": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-hex-encoding/-/util-hex-encoding-3.0.0.tgz", + "integrity": "sha512-eFndh1WEK5YMUYvy3lPlVmYY/fZcQE1D8oSf41Id2vCeIkKJXPcYDCZD+4+xViI6b1XSd7tE+s5AmXzz5ilabQ==", "dependencies": { - "@nodelib/fs.scandir": "2.1.5", - "fastq": "^1.6.0" + "tslib": "^2.6.2" }, "engines": { - "node": ">= 8" + "node": ">=16.0.0" } }, - "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, + "node_modules/@smithy/util-middleware": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/util-middleware/-/util-middleware-3.0.3.tgz", + "integrity": "sha512-l+StyYYK/eO3DlVPbU+4Bi06Jjal+PFLSMmlWM1BEwyLxZ3aKkf1ROnoIakfaA7mC6uw3ny7JBkau4Yc+5zfWw==", "dependencies": { - "@gar/promisify": "^1.1.3", - "semver": "^7.3.5" + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + "node": ">=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, + "node_modules/@smithy/util-retry": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@smithy/util-retry/-/util-retry-3.0.3.tgz", + "integrity": "sha512-AFw+hjpbtVApzpNDhbjNG5NA3kyoMs7vx0gsgmlJF4s+yz1Zlepde7J58zpIRIsdjc+emhpAITxA88qLkPF26w==", "dependencies": { - "mkdirp": "^1.0.4", - "rimraf": "^3.0.2" + "@smithy/service-error-classification": "^3.0.3", + "@smithy/types": "^3.3.0", + "tslib": "^2.6.2" }, "engines": { - "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + "node": ">=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==" + "node_modules/@smithy/util-stream": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/@smithy/util-stream/-/util-stream-3.1.3.tgz", + "integrity": "sha512-FIv/bRhIlAxC0U7xM1BCnF2aDRPq0UaelqBHkM2lsCp26mcBbgI0tCVTv+jGdsQLUmAMybua/bjDsSu8RQHbmw==", + "dependencies": { + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/types": "^3.3.0", + "@smithy/util-base64": "^3.0.0", + "@smithy/util-buffer-from": "^3.0.0", + "@smithy/util-hex-encoding": "^3.0.0", + "@smithy/util-utf8": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" + } }, - "node_modules/@sinonjs/commons": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.1.tgz", - "integrity": "sha512-K3mCHKQ9sVh8o1C9cxkwxaOmXoAMlDxC1mYyHrjqOWEcBjYr76t96zL2zlj5dUGZ3HSw240X1qgH3Mjf1yJWpQ==", + "node_modules/@smithy/util-uri-escape": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-uri-escape/-/util-uri-escape-3.0.0.tgz", + "integrity": "sha512-LqR7qYLgZTD7nWLBecUi4aqolw8Mhza9ArpNEQ881MJJIU2sE5iHCK6TdyqqzcDLy0OPe10IY4T8ctVdtynubg==", "dependencies": { - "type-detect": "4.0.8" + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.0.0" } }, - "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==", + "node_modules/@smithy/util-utf8": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@smithy/util-utf8/-/util-utf8-3.0.0.tgz", + "integrity": "sha512-rUeT12bxFnplYDe815GXbq/oixEGHfRFFtcTF3YdDi/JaENIM6aSYYLJydG83UNzLXeRI5K8abYd/8Sp/QM0kA==", "dependencies": { - "@sinonjs/commons": "^3.0.0" + "@smithy/util-buffer-from": "^3.0.0", + "tslib": "^2.6.2" + }, + "engines": { + "node": ">=16.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" } @@ -1452,6 +3063,11 @@ "@babel/types": "^7.20.7" } }, + "node_modules/@types/caseless": { + "version": "0.12.5", + "resolved": "https://registry.npmjs.org/@types/caseless/-/caseless-0.12.5.tgz", + "integrity": "sha512-hWtVTC2q7hc7xZ/RLbxapMvDMgUnDvKvMOpKal4DrMyfGBUfB1oKaZlIRr6mJL+If3bAP6sV/QneGzF6tJjZDg==" + }, "node_modules/@types/eslint": { "version": "9.6.0", "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-9.6.0.tgz", @@ -1518,6 +3134,11 @@ "integrity": "sha512-sVDA58zAw4eWAffKOaQH5/5j3XeayukzDk+ewSsnv3p4yJEZHCCzMDiZM8e0OUrRvmpGZ85jf4yDHkHsgBNr9Q==", "dev": true }, + "node_modules/@types/long": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/@types/long/-/long-4.0.2.tgz", + "integrity": "sha512-MqTGEo5bj5t157U6fA/BiDynNkn0YknVdh48CMPkTSpFTVmvao5UQmm7uEF6xBEo7qIMAlY/JSleYaE6VOdpaA==" + }, "node_modules/@types/markdown-it": { "version": "14.1.2", "resolved": "https://registry.npmjs.org/@types/markdown-it/-/markdown-it-14.1.2.tgz", @@ -1535,11 +3156,35 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.14.14", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.14.14.tgz", - "integrity": "sha512-d64f00982fS9YoOgJkAMolK7MN8Iq3TDdVjchbYHdEmjth/DHowx82GnoA+tVUAN+7vxfYUgAzi+JXbKNd2SDQ==", + "version": "20.16.1", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.16.1.tgz", + "integrity": "sha512-zJDo7wEadFtSyNz5QITDfRcrhqDvQI1xQNQ0VoizPjM/dVAODqqIUWbJPkvsxmTI0MYRGRikcdjMPhOssnPejQ==", "dependencies": { - "undici-types": "~5.26.4" + "undici-types": "~6.19.2" + } + }, + "node_modules/@types/request": { + "version": "2.48.12", + "resolved": "https://registry.npmjs.org/@types/request/-/request-2.48.12.tgz", + "integrity": "sha512-G3sY+NpsA9jnwm0ixhAFQSJ3Q9JkpLZpJbI3GMv0mIAT0y3mRabYeINzal5WOChIiaTEGQYlHOKgkaM9EisWHw==", + "dependencies": { + "@types/caseless": "*", + "@types/node": "*", + "@types/tough-cookie": "*", + "form-data": "^2.5.0" + } + }, + "node_modules/@types/request/node_modules/form-data": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/form-data/-/form-data-2.5.1.tgz", + "integrity": "sha512-m21N3WOmEEURgk6B9GLOE4RuWOFf28Lhh9qGYeNlGq4VDXUlJy2th2slBNU8Gp8EzloYZOibZJ7t5ecIrFSjVA==", + "dependencies": { + "asynckit": "^0.4.0", + "combined-stream": "^1.0.6", + "mime-types": "^2.1.12" + }, + "engines": { + "node": ">= 0.12" } }, "node_modules/@types/semver": { @@ -1555,6 +3200,11 @@ "resolved": "https://registry.npmjs.org/@types/stack-utils/-/stack-utils-2.0.3.tgz", "integrity": "sha512-9aEbYZ3TbYMznPdcdr3SmIrLXwC/AKZXQeCf9Pgao5CKb8CyHuEX5jzWPTkvregvhRJHcpRO6BFoGW9ycaOkYw==" }, + "node_modules/@types/tough-cookie": { + "version": "4.0.5", + "resolved": "https://registry.npmjs.org/@types/tough-cookie/-/tough-cookie-4.0.5.tgz", + "integrity": "sha512-/Ad8+nIOV7Rl++6f1BdKxFSMgmoqEoYbHRpPcx3JEfv8VRsQe9Z4mCXeJBzxs7mbHY/XOZZuXlRNfhpVPbs6ZA==" + }, "node_modules/@types/validator": { "version": "13.12.0", "resolved": "https://registry.npmjs.org/@types/validator/-/validator-13.12.0.tgz", @@ -2181,6 +3831,7 @@ "version": "1.6.0", "resolved": "https://registry.npmjs.org/@typescript/vfs/-/vfs-1.6.0.tgz", "integrity": "sha512-hvJUjNVeBMp77qPINuUvYXj4FyWeeMMKZkxEATEU3hqBAQ7qdTBCUFT7Sp0Zu0faeEtFf+ldXxMEDr/bk73ISg==", + "dev": true, "dependencies": { "debug": "^4.1.1" }, @@ -2199,6 +3850,17 @@ "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.1.1.tgz", "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" }, + "node_modules/abort-controller": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/abort-controller/-/abort-controller-3.0.0.tgz", + "integrity": "sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg==", + "dependencies": { + "event-target-shim": "^5.0.0" + }, + "engines": { + "node": ">=6.5" + } + }, "node_modules/acorn": { "version": "8.12.1", "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", @@ -2372,6 +4034,22 @@ "node": ">=8" } }, + "node_modules/asn1": { + "version": "0.2.6", + "resolved": "https://registry.npmjs.org/asn1/-/asn1-0.2.6.tgz", + "integrity": "sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ==", + "dependencies": { + "safer-buffer": "~2.1.0" + } + }, + "node_modules/assert-plus": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/assert-plus/-/assert-plus-1.0.0.tgz", + "integrity": "sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw==", + "engines": { + "node": ">=0.8" + } + }, "node_modules/async": { "version": "3.2.5", "resolved": "https://registry.npmjs.org/async/-/async-3.2.5.tgz", @@ -2401,6 +4079,19 @@ "node": ">=0.11" } }, + "node_modules/aws-sign2": { + "version": "0.7.0", + "resolved": "https://registry.npmjs.org/aws-sign2/-/aws-sign2-0.7.0.tgz", + "integrity": "sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==", + "engines": { + "node": "*" + } + }, + "node_modules/aws4": { + "version": "1.13.1", + "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.13.1.tgz", + "integrity": "sha512-u5w79Rd7SU4JaIlA/zFqG+gOiuq25q5VLyZ8E+ijJeILuTxVzZgp2CaGw/UTw6pXYN9XMO9yiqj/nEHmhTG5CA==" + }, "node_modules/axios": { "version": "1.7.3", "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.3.tgz", @@ -2526,6 +4217,41 @@ "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" }, + "node_modules/base64-js": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.5.1.tgz", + "integrity": "sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==", + "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/bcrypt-pbkdf": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz", + "integrity": "sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w==", + "dependencies": { + "tweetnacl": "^0.14.3" + } + }, + "node_modules/bignumber.js": { + "version": "9.1.2", + "resolved": "https://registry.npmjs.org/bignumber.js/-/bignumber.js-9.1.2.tgz", + "integrity": "sha512-2/mKyZH9K85bzOEfhXDBFZTGd1CTs+5IHpeFQo9luiBG7hghdC851Pj2WAhb6E3R6b9tZj/XKhbg4fum+Kepug==", + "engines": { + "node": "*" + } + }, "node_modules/binary-extensions": { "version": "2.3.0", "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.3.0.tgz", @@ -2552,6 +4278,11 @@ "integrity": "sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg==", "dev": true }, + "node_modules/bowser": { + "version": "2.11.0", + "resolved": "https://registry.npmjs.org/bowser/-/bowser-2.11.0.tgz", + "integrity": "sha512-AlcaJBi/pqqJBIQ8U9Mcpc9i8Aqxn88Skv5d+xBX006BY5u8N3mGLHa5Lgppa7L/HfwgwLgZ6NYs+Ag6uUmJRA==" + }, "node_modules/brace-expansion": { "version": "1.1.11", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", @@ -2572,6 +4303,14 @@ "node": ">=8" } }, + "node_modules/brotli": { + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/brotli/-/brotli-1.3.3.tgz", + "integrity": "sha512-oTKjJdShmDuGW94SyyaoQvAjf30dZaHnjJ8uAF+u2/vGJkJbJPJAT1gDiOJP5v1Zb6f9KEyW/1HpuaWIXtGHPg==", + "dependencies": { + "base64-js": "^1.1.2" + } + }, "node_modules/browser-stdout": { "version": "1.3.1", "resolved": "https://registry.npmjs.org/browser-stdout/-/browser-stdout-1.3.1.tgz", @@ -2629,6 +4368,11 @@ "node-int64": "^0.4.0" } }, + "node_modules/buffer-equal-constant-time": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz", + "integrity": "sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA==" + }, "node_modules/buffer-from": { "version": "1.1.2", "resolved": "https://registry.npmjs.org/buffer-from/-/buffer-from-1.1.2.tgz", @@ -2748,6 +4492,11 @@ } ] }, + "node_modules/caseless": { + "version": "0.12.0", + "resolved": "https://registry.npmjs.org/caseless/-/caseless-0.12.0.tgz", + "integrity": "sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==" + }, "node_modules/catharsis": { "version": "0.9.0", "resolved": "https://registry.npmjs.org/catharsis/-/catharsis-0.9.0.tgz", @@ -2938,6 +4687,11 @@ "resolved": "https://registry.npmjs.org/convert-source-map/-/convert-source-map-2.0.0.tgz", "integrity": "sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg==" }, + "node_modules/core-util-is": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.2.tgz", + "integrity": "sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ==" + }, "node_modules/create-jest": { "version": "29.7.0", "resolved": "https://registry.npmjs.org/create-jest/-/create-jest-29.7.0.tgz", @@ -2971,6 +4725,17 @@ "node": ">= 8" } }, + "node_modules/dashdash": { + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/dashdash/-/dashdash-1.14.1.tgz", + "integrity": "sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g==", + "dependencies": { + "assert-plus": "^1.0.0" + }, + "engines": { + "node": ">=0.10" + } + }, "node_modules/debug": { "version": "4.3.6", "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.6.tgz", @@ -3026,6 +4791,14 @@ "node": ">=0.10.0" } }, + "node_modules/define-lazy-prop": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz", + "integrity": "sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og==", + "engines": { + "node": ">=8" + } + }, "node_modules/delayed-stream": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz", @@ -3102,6 +4875,39 @@ "node": ">=6.0.0" } }, + "node_modules/duplexify": { + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/duplexify/-/duplexify-4.1.3.tgz", + "integrity": "sha512-M3BmBhwJRZsSx38lZyhE53Csddgzl5R7xGJNk7CVddZD6CcmwMCH8J+7AprIrQKH7TonKxaCjcv27Qmf+sQ+oA==", + "dependencies": { + "end-of-stream": "^1.4.1", + "inherits": "^2.0.3", + "readable-stream": "^3.1.1", + "stream-shift": "^1.0.2" + } + }, + "node_modules/ecc-jsbn": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz", + "integrity": "sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw==", + "dependencies": { + "jsbn": "~0.1.0", + "safer-buffer": "^2.1.0" + } + }, + "node_modules/ecc-jsbn/node_modules/jsbn": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-0.1.1.tgz", + "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==" + }, + "node_modules/ecdsa-sig-formatter": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", + "integrity": "sha512-nagl3RYrbNv6kQkeJIpt6NJZy8twLB/2vtz6yN9Z4vRKHN4/QZJIEbqohALSgwKdnksuY3k5Addp5lg8sVoVcQ==", + "dependencies": { + "safe-buffer": "^5.0.1" + } + }, "node_modules/ejs": { "version": "3.1.10", "resolved": "https://registry.npmjs.org/ejs/-/ejs-3.1.10.tgz", @@ -3147,6 +4953,14 @@ "iconv-lite": "^0.6.2" } }, + "node_modules/end-of-stream": { + "version": "1.4.4", + "resolved": "https://registry.npmjs.org/end-of-stream/-/end-of-stream-1.4.4.tgz", + "integrity": "sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q==", + "dependencies": { + "once": "^1.4.0" + } + }, "node_modules/entities": { "version": "4.5.0", "resolved": "https://registry.npmjs.org/entities/-/entities-4.5.0.tgz", @@ -3412,6 +5226,22 @@ "node": ">=0.10.0" } }, + "node_modules/event-target-shim": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/event-target-shim/-/event-target-shim-5.0.1.tgz", + "integrity": "sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ==", + "engines": { + "node": ">=6" + } + }, + "node_modules/events": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/events/-/events-3.3.0.tgz", + "integrity": "sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q==", + "engines": { + "node": ">=0.8.x" + } + }, "node_modules/execa": { "version": "5.1.1", "resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz", @@ -3463,6 +5293,19 @@ "integrity": "sha512-dX7e/LHVJ6W3DE1MHWi9S1EYzDESENfLrYohG2G++ovZrYOkm4Knwa0mc1cn84xJOR4KEU0WSchhLbd0UklbHw==", "dev": true }, + "node_modules/extend": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/extend/-/extend-3.0.2.tgz", + "integrity": "sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==" + }, + "node_modules/extsprintf": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/extsprintf/-/extsprintf-1.3.0.tgz", + "integrity": "sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==", + "engines": [ + "node >=0.6.0" + ] + }, "node_modules/fast-deep-equal": { "version": "3.1.3", "resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz", @@ -3512,6 +5355,27 @@ "resolved": "https://registry.npmjs.org/fast-uri/-/fast-uri-3.0.1.tgz", "integrity": "sha512-MWipKbbYiYI0UC7cl8m/i/IWTqfC8YXsqjzybjddLsFjStroQzsHXkc73JutMvBiXmOvapk+axIl79ig5t55Bw==" }, + "node_modules/fast-xml-parser": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/fast-xml-parser/-/fast-xml-parser-4.4.1.tgz", + "integrity": "sha512-xkjOecfnKGkSsOwtZ5Pz7Us/T6mrbPQrq0nh+aCO5V9nk5NLWmasAHumTKjiPJPWANe+kAZ84Jc8ooJkzZ88Sw==", + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/NaturalIntelligence" + }, + { + "type": "paypal", + "url": "https://paypal.me/naturalintelligence" + } + ], + "dependencies": { + "strnum": "^1.0.5" + }, + "bin": { + "fxparser": "src/cli/cli.js" + } + }, "node_modules/fastq": { "version": "1.17.1", "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.17.1.tgz", @@ -3652,6 +5516,14 @@ } } }, + "node_modules/forever-agent": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/forever-agent/-/forever-agent-0.6.1.tgz", + "integrity": "sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw==", + "engines": { + "node": "*" + } + }, "node_modules/form-data": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/form-data/-/form-data-4.0.0.tgz", @@ -3720,7 +5592,57 @@ "wide-align": "^1.1.2" }, "engines": { - "node": ">=10" + "node": ">=10" + } + }, + "node_modules/gaxios": { + "version": "6.7.1", + "resolved": "https://registry.npmjs.org/gaxios/-/gaxios-6.7.1.tgz", + "integrity": "sha512-LDODD4TMYx7XXdpwxAVRAIAuB0bzv0s+ywFonY46k126qzQHT9ygyoa9tncmOiQmmDrik65UYsEkv3lbfqQ3yQ==", + "dependencies": { + "extend": "^3.0.2", + "https-proxy-agent": "^7.0.1", + "is-stream": "^2.0.0", + "node-fetch": "^2.6.9", + "uuid": "^9.0.1" + }, + "engines": { + "node": ">=14" + } + }, + "node_modules/gaxios/node_modules/agent-base": { + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.1.tgz", + "integrity": "sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==", + "dependencies": { + "debug": "^4.3.4" + }, + "engines": { + "node": ">= 14" + } + }, + "node_modules/gaxios/node_modules/https-proxy-agent": { + "version": "7.0.5", + "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz", + "integrity": "sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==", + "dependencies": { + "agent-base": "^7.0.2", + "debug": "4" + }, + "engines": { + "node": ">= 14" + } + }, + "node_modules/gcp-metadata": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/gcp-metadata/-/gcp-metadata-6.1.0.tgz", + "integrity": "sha512-Jh/AIwwgaxan+7ZUUmRLCjtchyDiqh4KjBJ5tW3plBZb5iL/BPcso8A5DlzeD9qlw0duCamnNdpFjxwaT0KyKg==", + "dependencies": { + "gaxios": "^6.0.0", + "json-bigint": "^1.0.0" + }, + "engines": { + "node": ">=14" } }, "node_modules/gensync": { @@ -3758,6 +5680,14 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/getpass": { + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/getpass/-/getpass-0.1.7.tgz", + "integrity": "sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng==", + "dependencies": { + "assert-plus": "^1.0.0" + } + }, "node_modules/glob": { "version": "7.2.3", "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz", @@ -3825,6 +5755,44 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/google-auth-library": { + "version": "9.14.0", + "resolved": "https://registry.npmjs.org/google-auth-library/-/google-auth-library-9.14.0.tgz", + "integrity": "sha512-Y/eq+RWVs55Io/anIsm24sDS8X79Tq948zVLGaa7+KlJYYqaGwp1YI37w48nzrNi12RgnzMrQD4NzdmCowT90g==", + "dependencies": { + "base64-js": "^1.3.0", + "ecdsa-sig-formatter": "^1.0.11", + "gaxios": "^6.1.1", + "gcp-metadata": "^6.1.0", + "gtoken": "^7.0.0", + "jws": "^4.0.0" + }, + "engines": { + "node": ">=14" + } + }, + "node_modules/google-gax": { + "version": "4.3.9", + "resolved": "https://registry.npmjs.org/google-gax/-/google-gax-4.3.9.tgz", + "integrity": "sha512-tcjQr7sXVGMdlvcG25wSv98ap1dtF4Z6mcV0rztGIddOcezw4YMb/uTXg72JPrLep+kXcVjaJjg6oo3KLf4itQ==", + "dependencies": { + "@grpc/grpc-js": "^1.10.9", + "@grpc/proto-loader": "^0.7.13", + "@types/long": "^4.0.0", + "abort-controller": "^3.0.0", + "duplexify": "^4.0.0", + "google-auth-library": "^9.3.0", + "node-fetch": "^2.7.0", + "object-hash": "^3.0.0", + "proto3-json-serializer": "^2.0.2", + "protobufjs": "^7.3.2", + "retry-request": "^7.0.0", + "uuid": "^9.0.1" + }, + "engines": { + "node": ">=14" + } + }, "node_modules/graceful-fs": { "version": "4.2.11", "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.11.tgz", @@ -3836,6 +5804,59 @@ "integrity": "sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag==", "dev": true }, + "node_modules/gtoken": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/gtoken/-/gtoken-7.1.0.tgz", + "integrity": "sha512-pCcEwRi+TKpMlxAQObHDQ56KawURgyAf6jtIY046fJ5tIv3zDe/LEIubckAO8fj6JnAxLdmWkUfNyulQ2iKdEw==", + "dependencies": { + "gaxios": "^6.0.0", + "jws": "^4.0.0" + }, + "engines": { + "node": ">=14.0.0" + } + }, + "node_modules/har-schema": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/har-schema/-/har-schema-2.0.0.tgz", + "integrity": "sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q==", + "engines": { + "node": ">=4" + } + }, + "node_modules/har-validator": { + "version": "5.1.5", + "resolved": "https://registry.npmjs.org/har-validator/-/har-validator-5.1.5.tgz", + "integrity": "sha512-nmT2T0lljbxdQZfspsno9hgrG3Uir6Ks5afism62poxqBM6sDnMEuPmzTq8XN0OEwqKLLdh1jQI3qyE66Nzb3w==", + "deprecated": "this library is no longer supported", + "dependencies": { + "ajv": "^6.12.3", + "har-schema": "^2.0.0" + }, + "engines": { + "node": ">=6" + } + }, + "node_modules/har-validator/node_modules/ajv": { + "version": "6.12.6", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.12.6.tgz", + "integrity": "sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==", + "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/har-validator/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==" + }, "node_modules/has-flag": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", @@ -3884,7 +5905,6 @@ "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": { "@tootallnate/once": "2", "agent-base": "6", @@ -3894,6 +5914,19 @@ "node": ">= 6" } }, + "node_modules/http-signature": { + "version": "1.3.6", + "resolved": "https://registry.npmjs.org/http-signature/-/http-signature-1.3.6.tgz", + "integrity": "sha512-3adrsD6zqo4GsTqtO7FyrejHNv+NgiIfAfv68+jVlFmSr9OGy7zrxONceFRLKvnnZA5jbxQBX1u9PpB6Wi32Gw==", + "dependencies": { + "assert-plus": "^1.0.0", + "jsprim": "^2.0.2", + "sshpk": "^1.14.1" + }, + "engines": { + "node": ">=0.10" + } + }, "node_modules/https-proxy-agent": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-5.0.1.tgz", @@ -4060,6 +6093,20 @@ "url": "https://github.com/sponsors/ljharb" } }, + "node_modules/is-docker": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/is-docker/-/is-docker-2.2.1.tgz", + "integrity": "sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ==", + "bin": { + "is-docker": "cli.js" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/is-extglob": { "version": "2.1.1", "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", @@ -4140,6 +6187,11 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/is-typedarray": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz", + "integrity": "sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA==" + }, "node_modules/is-unicode-supported": { "version": "0.1.0", "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz", @@ -4152,11 +6204,27 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/is-wsl": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/is-wsl/-/is-wsl-2.2.0.tgz", + "integrity": "sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww==", + "dependencies": { + "is-docker": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/isexe": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==" }, + "node_modules/isstream": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/isstream/-/isstream-0.1.2.tgz", + "integrity": "sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g==" + }, "node_modules/istanbul-lib-coverage": { "version": "3.2.2", "resolved": "https://registry.npmjs.org/istanbul-lib-coverage/-/istanbul-lib-coverage-3.2.2.tgz", @@ -4858,6 +6926,14 @@ "node": ">=4" } }, + "node_modules/json-bigint": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/json-bigint/-/json-bigint-1.0.0.tgz", + "integrity": "sha512-SiPv/8VpZuWbvLSMtTDU8hEfrZWg/mH/nV/b4o0CYbSxu1UIQPLdwKOCIyLQX+VIPO5vrLX3i8qtqFyhdPSUSQ==", + "dependencies": { + "bignumber.js": "^9.0.0" + } + }, "node_modules/json-buffer": { "version": "3.0.1", "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.1.tgz", @@ -4869,6 +6945,11 @@ "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==" }, + "node_modules/json-schema": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/json-schema/-/json-schema-0.4.0.tgz", + "integrity": "sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA==" + }, "node_modules/json-schema-traverse": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz", @@ -4889,6 +6970,11 @@ "node": ">= 4" } }, + "node_modules/json-stringify-safe": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz", + "integrity": "sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA==" + }, "node_modules/json5": { "version": "2.2.3", "resolved": "https://registry.npmjs.org/json5/-/json5-2.2.3.tgz", @@ -4900,6 +6986,87 @@ "node": ">=6" } }, + "node_modules/jsonata": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/jsonata/-/jsonata-2.0.5.tgz", + "integrity": "sha512-wEse9+QLIIU5IaCgtJCPsFi/H4F3qcikWzF4bAELZiRz08ohfx3Q6CjDRf4ZPF5P/92RI3KIHtb7u3jqPaHXdQ==", + "engines": { + "node": ">= 8" + } + }, + "node_modules/jsonwebtoken": { + "version": "9.0.2", + "resolved": "https://registry.npmjs.org/jsonwebtoken/-/jsonwebtoken-9.0.2.tgz", + "integrity": "sha512-PRp66vJ865SSqOlgqS8hujT5U4AOgMfhrwYIuIhfKaoSCZcirrmASQr8CX7cUg+RMih+hgznrjp99o+W4pJLHQ==", + "dependencies": { + "jws": "^3.2.2", + "lodash.includes": "^4.3.0", + "lodash.isboolean": "^3.0.3", + "lodash.isinteger": "^4.0.4", + "lodash.isnumber": "^3.0.3", + "lodash.isplainobject": "^4.0.6", + "lodash.isstring": "^4.0.1", + "lodash.once": "^4.0.0", + "ms": "^2.1.1", + "semver": "^7.5.4" + }, + "engines": { + "node": ">=12", + "npm": ">=6" + } + }, + "node_modules/jsonwebtoken/node_modules/jwa": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/jwa/-/jwa-1.4.1.tgz", + "integrity": "sha512-qiLX/xhEEFKUAJ6FiBMbes3w9ATzyk5W7Hvzpa/SLYdxNtng+gcurvrI7TbACjIXlsJyr05/S1oUhZrc63evQA==", + "dependencies": { + "buffer-equal-constant-time": "1.0.1", + "ecdsa-sig-formatter": "1.0.11", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jsonwebtoken/node_modules/jws": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/jws/-/jws-3.2.2.tgz", + "integrity": "sha512-YHlZCB6lMTllWDtSPHz/ZXTsi8S00usEV6v1tjq8tOUZzw7DpSDWVXjXDre6ed1w/pd495ODpHZYSdkRTsa0HA==", + "dependencies": { + "jwa": "^1.4.1", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jsprim": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/jsprim/-/jsprim-2.0.2.tgz", + "integrity": "sha512-gqXddjPqQ6G40VdnI6T6yObEC+pDNvyP95wdQhkWkg7crHH3km5qP1FsOXEkzEQwnz6gz5qGTn1c2Y52wP3OyQ==", + "engines": [ + "node >=0.6.0" + ], + "dependencies": { + "assert-plus": "1.0.0", + "extsprintf": "1.3.0", + "json-schema": "0.4.0", + "verror": "1.10.0" + } + }, + "node_modules/jwa": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/jwa/-/jwa-2.0.0.tgz", + "integrity": "sha512-jrZ2Qx916EA+fq9cEAeCROWPTfCwi1IVHqT2tapuqLEVVDKFDENFw1oL+MwrTvH6msKxsd1YTDVw6uKEcsrLEA==", + "dependencies": { + "buffer-equal-constant-time": "1.0.1", + "ecdsa-sig-formatter": "1.0.11", + "safe-buffer": "^5.0.1" + } + }, + "node_modules/jws": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/jws/-/jws-4.0.0.tgz", + "integrity": "sha512-KDncfTmOZoOMTFG4mBlG0qUIOlc03fmzH+ru6RgYVZhPkyiy/92Owlt/8UEN+a4TXR1FQetfIpJE8ApdvdVxTg==", + "dependencies": { + "jwa": "^2.0.0", + "safe-buffer": "^5.0.1" + } + }, "node_modules/keyv": { "version": "4.5.4", "resolved": "https://registry.npmjs.org/keyv/-/keyv-4.5.4.tgz", @@ -4982,6 +7149,41 @@ "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==", "dev": true }, + "node_modules/lodash.camelcase": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/lodash.camelcase/-/lodash.camelcase-4.3.0.tgz", + "integrity": "sha512-TwuEnCnxbc3rAvhf/LbG7tJUDzhqXyFnv3dtzLOPgCG/hODL7WFnsbwktkD7yUV0RrreP/l1PALq/YSg6VvjlA==" + }, + "node_modules/lodash.includes": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/lodash.includes/-/lodash.includes-4.3.0.tgz", + "integrity": "sha512-W3Bx6mdkRTGtlJISOvVD/lbqjTlPPUDTMnlXZFnVwi9NKJ6tiAk6LVdlhZMm17VZisqhKcgzpO5Wz91PCt5b0w==" + }, + "node_modules/lodash.isboolean": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/lodash.isboolean/-/lodash.isboolean-3.0.3.tgz", + "integrity": "sha512-Bz5mupy2SVbPHURB98VAcw+aHh4vRV5IPNhILUCsOzRmsTmSQ17jIuqopAentWoehktxGd9e/hbIXq980/1QJg==" + }, + "node_modules/lodash.isinteger": { + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/lodash.isinteger/-/lodash.isinteger-4.0.4.tgz", + "integrity": "sha512-DBwtEWN2caHQ9/imiNeEA5ys1JoRtRfY3d7V9wkqtbycnAmTvRRmbHKDV4a0EYc678/dia0jrte4tjYwVBaZUA==" + }, + "node_modules/lodash.isnumber": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/lodash.isnumber/-/lodash.isnumber-3.0.3.tgz", + "integrity": "sha512-QYqzpfwO3/CWf3XP+Z+tkQsfaLL/EnUlXWVkIk5FUPc4sBdTehEqZONuyRt2P67PXAk+NXmTBcc97zw9t1FQrw==" + }, + "node_modules/lodash.isplainobject": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/lodash.isplainobject/-/lodash.isplainobject-4.0.6.tgz", + "integrity": "sha512-oSXzaWypCMHkPC3NvBEaPHf0KsA5mvPrOPgQWDsbg8n7orZ290M0BmC/jgRZ4vcJ6DTAhjrsSYgdsW/F+MFOBA==" + }, + "node_modules/lodash.isstring": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/lodash.isstring/-/lodash.isstring-4.0.1.tgz", + "integrity": "sha512-0wJxfxH1wgO3GrbuP+dTTk7op+6L41QCXbGINEmD+ny/G/eCqGzxyCsh7159S+mgDDcoarnBw6PC1PS5+wUGgw==" + }, "node_modules/lodash.memoize": { "version": "4.1.2", "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", @@ -4994,6 +7196,11 @@ "integrity": "sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ==", "dev": true }, + "node_modules/lodash.once": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz", + "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==" + }, "node_modules/log-symbols": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz", @@ -5010,6 +7217,11 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/long": { + "version": "5.2.3", + "resolved": "https://registry.npmjs.org/long/-/long-5.2.3.tgz", + "integrity": "sha512-lcHwpNoggQTObv5apGNCTdJrO69eHOZMi4BNC+rTLER8iHAqGrUVeLh/irVIM7zTw2bOXA8T6uNPeujwOLg/2Q==" + }, "node_modules/lru-cache": { "version": "11.0.0", "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-11.0.0.tgz", @@ -5300,11 +7512,6 @@ "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" }, - "node_modules/miscreant": { - "version": "0.3.2", - "resolved": "https://registry.npmjs.org/miscreant/-/miscreant-0.3.2.tgz", - "integrity": "sha512-fL9KxsQz9BJB2KGPMHFrReioywkiomBiuaLk6EuChijK0BsJsIKJXdVomR+/bPj5mvbFD6wM0CM3bZio9g7OHA==" - }, "node_modules/mkdirp": { "version": "1.0.4", "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-1.0.4.tgz", @@ -5452,6 +7659,14 @@ "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" }, + "node_modules/mustache": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/mustache/-/mustache-4.2.0.tgz", + "integrity": "sha512-71ippSywq5Yb7/tVYyGbkBggbU8H3u5Rz56fH60jGFgr8uHwxs+aSKeqmluIVzM0m0kB7xQjKS6qPfd0b2ZoqQ==", + "bin": { + "mustache": "bin/mustache" + } + }, "node_modules/nan": { "version": "2.20.0", "resolved": "https://registry.npmjs.org/nan/-/nan-2.20.0.tgz", @@ -5611,6 +7826,20 @@ "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.18.tgz", "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==" }, + "node_modules/node-vault": { + "version": "0.10.2", + "resolved": "https://registry.npmjs.org/node-vault/-/node-vault-0.10.2.tgz", + "integrity": "sha512-//uc9/YImE7Dx0QHdwMiAzLaOumiKUnOUP8DymgtkZ8nsq6/V2LKvEu6kw91Lcruw8lWUfj4DO7CIXNPRWBuuA==", + "dependencies": { + "debug": "^4.3.4", + "mustache": "^4.2.0", + "postman-request": "^2.88.1-postman.33", + "tv4": "^1.3.0" + }, + "engines": { + "node": ">= 16.0.0" + } + }, "node_modules/nopt": { "version": "5.0.0", "resolved": "https://registry.npmjs.org/nopt/-/nopt-5.0.0.tgz", @@ -5656,6 +7885,14 @@ "set-blocking": "^2.0.0" } }, + "node_modules/oauth-sign": { + "version": "0.9.0", + "resolved": "https://registry.npmjs.org/oauth-sign/-/oauth-sign-0.9.0.tgz", + "integrity": "sha512-fexhUFFPTGV8ybAtSIGbV6gOkSv8UtRbDBnAyLQw4QPKkgNlsH2ByPGtMUqdWkos6YCRmAqViwgZrJc/mRDzZQ==", + "engines": { + "node": "*" + } + }, "node_modules/object-assign": { "version": "4.1.1", "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", @@ -5664,6 +7901,14 @@ "node": ">=0.10.0" } }, + "node_modules/object-hash": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/object-hash/-/object-hash-3.0.0.tgz", + "integrity": "sha512-RSn9F68PjH9HqtltsSnqYC1XXoWe9Bju5+213R98cNGttag9q9yAOTzdbsqvIa7aNm5WffBZFpWYr2aWrklWAw==", + "engines": { + "node": ">= 6" + } + }, "node_modules/once": { "version": "1.4.0", "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", @@ -5686,6 +7931,22 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/open": { + "version": "8.4.2", + "resolved": "https://registry.npmjs.org/open/-/open-8.4.2.tgz", + "integrity": "sha512-7x81NCL719oNbsq/3mh+hVrAWmFuEYUqrq/Iw3kUzH8ReypT9QQ0BLoJS7/G9k6N81XjW4qHWtjWwe/9eLy1EQ==", + "dependencies": { + "define-lazy-prop": "^2.0.0", + "is-docker": "^2.1.1", + "is-wsl": "^2.2.0" + }, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/optionator": { "version": "0.9.4", "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.4.tgz", @@ -5822,6 +8083,11 @@ "node": ">=8" } }, + "node_modules/performance-now": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/performance-now/-/performance-now-2.1.0.tgz", + "integrity": "sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==" + }, "node_modules/picocolors": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.1.tgz", @@ -5905,6 +8171,46 @@ "node": ">=8" } }, + "node_modules/postman-request": { + "version": "2.88.1-postman.39", + "resolved": "https://registry.npmjs.org/postman-request/-/postman-request-2.88.1-postman.39.tgz", + "integrity": "sha512-rsncxxDlbn1YpygXSgJqbJzIjGlHFcZjbYDzeBPTQHMDfLuSTzZz735JHV8i1+lOROuJ7MjNap4eaSD3UijHzQ==", + "dependencies": { + "@postman/form-data": "~3.1.1", + "@postman/tough-cookie": "~4.1.3-postman.1", + "@postman/tunnel-agent": "^0.6.4", + "aws-sign2": "~0.7.0", + "aws4": "^1.12.0", + "brotli": "^1.3.3", + "caseless": "~0.12.0", + "combined-stream": "~1.0.6", + "extend": "~3.0.2", + "forever-agent": "~0.6.1", + "har-validator": "~5.1.3", + "http-signature": "~1.3.1", + "is-typedarray": "~1.0.0", + "isstream": "~0.1.2", + "json-stringify-safe": "~5.0.1", + "mime-types": "^2.1.35", + "oauth-sign": "~0.9.0", + "performance-now": "^2.1.0", + "qs": "~6.5.3", + "safe-buffer": "^5.1.2", + "stream-length": "^1.0.2", + "uuid": "^8.3.2" + }, + "engines": { + "node": ">= 16" + } + }, + "node_modules/postman-request/node_modules/uuid": { + "version": "8.3.2", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", + "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", + "bin": { + "uuid": "dist/bin/uuid" + } + }, "node_modules/prelude-ls": { "version": "1.2.1", "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.2.1.tgz", @@ -5969,12 +8275,51 @@ "node": ">= 6" } }, + "node_modules/proto3-json-serializer": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/proto3-json-serializer/-/proto3-json-serializer-2.0.2.tgz", + "integrity": "sha512-SAzp/O4Yh02jGdRc+uIrGoe87dkN/XtwxfZ4ZyafJHymd79ozp5VG5nyZ7ygqPM5+cpLDjjGnYFUkngonyDPOQ==", + "dependencies": { + "protobufjs": "^7.2.5" + }, + "engines": { + "node": ">=14.0.0" + } + }, + "node_modules/protobufjs": { + "version": "7.4.0", + "resolved": "https://registry.npmjs.org/protobufjs/-/protobufjs-7.4.0.tgz", + "integrity": "sha512-mRUWCc3KUU4w1jU8sGxICXH/gNS94DvI1gxqDvBzhj1JpcsimQkYiOJfwsPUykUI5ZaspFbSgmBLER8IrQ3tqw==", + "hasInstallScript": true, + "dependencies": { + "@protobufjs/aspromise": "^1.1.2", + "@protobufjs/base64": "^1.1.2", + "@protobufjs/codegen": "^2.0.4", + "@protobufjs/eventemitter": "^1.1.0", + "@protobufjs/fetch": "^1.1.0", + "@protobufjs/float": "^1.0.2", + "@protobufjs/inquire": "^1.1.0", + "@protobufjs/path": "^1.1.2", + "@protobufjs/pool": "^1.1.0", + "@protobufjs/utf8": "^1.1.0", + "@types/node": ">=13.7.0", + "long": "^5.0.0" + }, + "engines": { + "node": ">=12.0.0" + } + }, "node_modules/proxy-from-env": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/proxy-from-env/-/proxy-from-env-1.1.0.tgz", "integrity": "sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg==", "license": "MIT" }, + "node_modules/psl": { + "version": "1.9.0", + "resolved": "https://registry.npmjs.org/psl/-/psl-1.9.0.tgz", + "integrity": "sha512-E/ZsdU4HLs/68gYzgGTkMicWTLPdAftJLfJFlLUAAKZGkStNU72sZjT66SnMDVOfOWY/YAoiD7Jxa9iHvngcag==" + }, "node_modules/punycode": { "version": "2.3.1", "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.3.1.tgz", @@ -6007,6 +8352,19 @@ } ] }, + "node_modules/qs": { + "version": "6.5.3", + "resolved": "https://registry.npmjs.org/qs/-/qs-6.5.3.tgz", + "integrity": "sha512-qxXIEh4pCGfHICj1mAJQ2/2XVZkjCDTcEgfoSQxc/fYivUZxTkk7L3bDBJSoNrEzXI17oUO5Dp07ktqE5KzczA==", + "engines": { + "node": ">=0.6" + } + }, + "node_modules/querystringify": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/querystringify/-/querystringify-2.2.0.tgz", + "integrity": "sha512-FIqgj2EUvTa7R50u0rGsyTftzjYmv/a3hO345bZNrqabNqjtgiDMgmo4mkUjd+nzU5oF3dClKqFIPUKybUyqoQ==" + }, "node_modules/queue-microtask": { "version": "1.2.3", "resolved": "https://registry.npmjs.org/queue-microtask/-/queue-microtask-1.2.3.tgz", @@ -6099,6 +8457,11 @@ "node": ">=0.10.0" } }, + "node_modules/requires-port": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/requires-port/-/requires-port-1.0.0.tgz", + "integrity": "sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ==" + }, "node_modules/requizzle": { "version": "0.2.4", "resolved": "https://registry.npmjs.org/requizzle/-/requizzle-0.2.4.tgz", @@ -6177,6 +8540,19 @@ "node": ">= 4" } }, + "node_modules/retry-request": { + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/retry-request/-/retry-request-7.0.2.tgz", + "integrity": "sha512-dUOvLMJ0/JJYEn8NrpOaGNE7X3vpI5XlZS/u0ANjqtcZVKnIxP7IgCFwrKTxENw29emmwug53awKtaMm4i9g5w==", + "dependencies": { + "@types/request": "^2.48.8", + "extend": "^3.0.2", + "teeny-request": "^9.0.0" + }, + "engines": { + "node": ">=14" + } + }, "node_modules/reusify": { "version": "1.0.4", "resolved": "https://registry.npmjs.org/reusify/-/reusify-1.0.4.tgz", @@ -6247,8 +8623,7 @@ "node_modules/safer-buffer": { "version": "2.1.2", "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz", - "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==", - "optional": true + "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==" }, "node_modules/semver": { "version": "7.6.3", @@ -6384,6 +8759,35 @@ "integrity": "sha512-Oo+0REFV59/rz3gfJNKQiBlwfHaSESl1pcGyABQsnnIfWOFt6JNj5gCog2U6MLZ//IGYD+nA8nI+mTShREReaA==", "dev": true }, + "node_modules/sshpk": { + "version": "1.18.0", + "resolved": "https://registry.npmjs.org/sshpk/-/sshpk-1.18.0.tgz", + "integrity": "sha512-2p2KJZTSqQ/I3+HX42EpYOa2l3f8Erv8MWKsy2I9uf4wA7yFIkXRffYdsx86y6z4vHtV8u7g+pPlr8/4ouAxsQ==", + "dependencies": { + "asn1": "~0.2.3", + "assert-plus": "^1.0.0", + "bcrypt-pbkdf": "^1.0.0", + "dashdash": "^1.12.0", + "ecc-jsbn": "~0.1.1", + "getpass": "^0.1.1", + "jsbn": "~0.1.0", + "safer-buffer": "^2.0.2", + "tweetnacl": "~0.14.0" + }, + "bin": { + "sshpk-conv": "bin/sshpk-conv", + "sshpk-sign": "bin/sshpk-sign", + "sshpk-verify": "bin/sshpk-verify" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/sshpk/node_modules/jsbn": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-0.1.1.tgz", + "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==" + }, "node_modules/ssri": { "version": "9.0.1", "resolved": "https://registry.npmjs.org/ssri/-/ssri-9.0.1.tgz", @@ -6415,6 +8819,41 @@ "node": ">=8" } }, + "node_modules/stoppable": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/stoppable/-/stoppable-1.1.0.tgz", + "integrity": "sha512-KXDYZ9dszj6bzvnEMRYvxgeTHU74QBFL54XKtP3nyMuJ81CFYtABZ3bAzL2EdFUaEwJOBOgENyFj3R7oTzDyyw==", + "engines": { + "node": ">=4", + "npm": ">=6" + } + }, + "node_modules/stream-events": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/stream-events/-/stream-events-1.0.5.tgz", + "integrity": "sha512-E1GUzBSgvct8Jsb3v2X15pjzN1tYebtbLaMg+eBOUOAxgbLoSbT2NS91ckc5lJD1KfLjId+jXJRgo0qnV5Nerg==", + "dependencies": { + "stubs": "^3.0.0" + } + }, + "node_modules/stream-length": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/stream-length/-/stream-length-1.0.2.tgz", + "integrity": "sha512-aI+qKFiwoDV4rsXiS7WRoCt+v2RX1nUj17+KJC5r2gfh5xoSJIfP6Y3Do/HtvesFcTSWthIuJ3l1cvKQY/+nZg==", + "dependencies": { + "bluebird": "^2.6.2" + } + }, + "node_modules/stream-length/node_modules/bluebird": { + "version": "2.11.0", + "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-2.11.0.tgz", + "integrity": "sha512-UfFSr22dmHPQqPP9XWHRhq+gWnHCYguQGkXQlbyPtW5qTnhFWA8/iXg765tH0cAjy7l/zPJ1aBTO0g5XgA7kvQ==" + }, + "node_modules/stream-shift": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/stream-shift/-/stream-shift-1.0.3.tgz", + "integrity": "sha512-76ORR0DO1o1hlKwTbi/DM3EXWGf3ZJYO8cXX5RJwnul2DEg2oyoZyjLNoQM8WsvZiFKCRfC1O0J7iCvie3RZmQ==" + }, "node_modules/string_decoder": { "version": "1.3.0", "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.3.0.tgz", @@ -6486,6 +8925,16 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/strnum": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/strnum/-/strnum-1.0.5.tgz", + "integrity": "sha512-J8bbNyKKXl5qYcR36TIO8W3mVGVHrmmxsd5PAItGkmyzwJvybiw2IVq5nqd0i4LSNSkB/sx9VHllbfFdr9k1JA==" + }, + "node_modules/stubs": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/stubs/-/stubs-3.0.0.tgz", + "integrity": "sha512-PdHt7hHUJKxvTCgbKX9C1V/ftOcjJQgz8BZwNfV5c4B6dcGqlpelTbJ999jBGZ2jYiPAwcX5dP6oBwVlBlUbxw==" + }, "node_modules/supports-color": { "version": "7.2.0", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", @@ -6537,6 +8986,21 @@ "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" }, + "node_modules/teeny-request": { + "version": "9.0.0", + "resolved": "https://registry.npmjs.org/teeny-request/-/teeny-request-9.0.0.tgz", + "integrity": "sha512-resvxdc6Mgb7YEThw6G6bExlXKkv6+YbuzGg9xuXxSgxJF7Ozs+o8Y9+2R3sArdWdW8nOokoQb1yrpFB0pQK2g==", + "dependencies": { + "http-proxy-agent": "^5.0.0", + "https-proxy-agent": "^5.0.0", + "node-fetch": "^2.6.9", + "stream-events": "^1.0.5", + "uuid": "^9.0.0" + }, + "engines": { + "node": ">=14" + } + }, "node_modules/test-exclude": { "version": "6.0.0", "resolved": "https://registry.npmjs.org/test-exclude/-/test-exclude-6.0.0.tgz", @@ -6668,6 +9132,19 @@ "integrity": "sha512-xNvxJEOUiWPGhUuUdQgAJPKOOJfGnIyKySOc09XkKsgdUV/3E2zvwZYdejjmRgPCgcym1juLH3226yA7sEFJKQ==", "license": "0BSD" }, + "node_modules/tv4": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/tv4/-/tv4-1.3.0.tgz", + "integrity": "sha512-afizzfpJgvPr+eDkREK4MxJ/+r8nEEHcmitwgnPUqpaP+FpwQyadnxNoSACbgc/b1LsZYtODGoPiFxQrgJgjvw==", + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/tweetnacl": { + "version": "0.14.5", + "resolved": "https://registry.npmjs.org/tweetnacl/-/tweetnacl-0.14.5.tgz", + "integrity": "sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA==" + }, "node_modules/type-check": { "version": "0.4.0", "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.4.0.tgz", @@ -6954,9 +9431,9 @@ "dev": true }, "node_modules/undici-types": { - "version": "5.26.5", - "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-5.26.5.tgz", - "integrity": "sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==" + "version": "6.19.8", + "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-6.19.8.tgz", + "integrity": "sha512-ve2KP6f/JnbPBFyobGHuerC9g1FYGn/F8n1LWTwNxCEzd6IfqTwUQcNXgEtmmQ6DlRrC1hrSrBnCZPokRrDHjw==" }, "node_modules/unique-filename": { "version": "2.0.1", @@ -6982,6 +9459,14 @@ "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, + "node_modules/universalify": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/universalify/-/universalify-0.2.0.tgz", + "integrity": "sha512-CJ1QgKmNg3CwvAv/kOFmtnEN05f0D/cn9QntgNOQlQF9dgvVTHj3t+8JPdjqawCHk7V/KA+fbUqzZ9XWhcqPUg==", + "engines": { + "node": ">= 4.0.0" + } + }, "node_modules/update-browserslist-db": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.1.0.tgz", @@ -7015,16 +9500,36 @@ "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/url-parse": { + "version": "1.5.10", + "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.5.10.tgz", + "integrity": "sha512-WypcfiRhfeUP9vvF0j6rw0J3hrWrw6iZv3+22h6iRMJ/8z1Tj6XfLP4DsUix5MhMPnXpiHDoKyoZ/bdCkwBCiQ==", + "dependencies": { + "querystringify": "^2.1.1", + "requires-port": "^1.0.0" + } + }, "node_modules/util-deprecate": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", "integrity": "sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==" }, + "node_modules/uuid": { + "version": "9.0.1", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", + "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "bin": { + "uuid": "dist/bin/uuid" + } + }, "node_modules/v8-to-istanbul": { "version": "9.3.0", "resolved": "https://registry.npmjs.org/v8-to-istanbul/-/v8-to-istanbul-9.3.0.tgz", @@ -7046,6 +9551,19 @@ "node": ">= 0.10" } }, + "node_modules/verror": { + "version": "1.10.0", + "resolved": "https://registry.npmjs.org/verror/-/verror-1.10.0.tgz", + "integrity": "sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw==", + "engines": [ + "node >=0.6.0" + ], + "dependencies": { + "assert-plus": "^1.0.0", + "core-util-is": "1.0.2", + "extsprintf": "^1.2.0" + } + }, "node_modules/walker": { "version": "1.0.8", "resolved": "https://registry.npmjs.org/walker/-/walker-1.0.8.tgz", diff --git a/package.json b/package.json index 085efcfa..b133aa13 100644 --- a/package.json +++ b/package.json @@ -32,9 +32,11 @@ }, "license": "MIT", "devDependencies": { + "@bufbuild/buf": "^1.37.0", + "@bufbuild/protoc-gen-es": "^2.0.0", "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", - "@types/node": "^20.4.5", + "@types/node": "^20.16.1", "bluebird": "^3.5.3", "eslint": "^8.57.0", "eslint-plugin-jest": "^28.6.0", @@ -46,13 +48,16 @@ "typescript-eslint": "^8.2.0" }, "dependencies": { - "@bufbuild/buf": "^1.37.0", + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", "@bufbuild/protobuf": "^2.0.0", - "@bufbuild/protoc-gen-es": "^2.0.0", "@criteria/json-schema": "^0.10.0", "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", + "@smithy/types": "^3.3.0", "@types/validator": "^13.12.0", "ajv": "^8.17.1", "async-mutex": "^0.5.0", @@ -60,9 +65,10 @@ "axios": "^1.7.3", "bindings": "^1.3.1", "json-stringify-deterministic": "^1.0.12", + "jsonata": "^2.0.5", "lru-cache": "^11.0.0", - "miscreant": "^0.3.2", "nan": "^2.17.0", + "node-vault": "^0.10.2", "ts-jest": "^29.2.4", "validator": "^13.12.0" }, diff --git a/proto/test/schemaregistry/serde/cycle.proto b/proto/test/schemaregistry/serde/cycle.proto new file mode 100644 index 00000000..a5351102 --- /dev/null +++ b/proto/test/schemaregistry/serde/cycle.proto @@ -0,0 +1,9 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +message LinkedList { + int32 value = 1; + LinkedList next = 10; +} \ No newline at end of file diff --git a/proto/test/schemaregistry/serde/dep.proto b/proto/test/schemaregistry/serde/dep.proto new file mode 100644 index 00000000..c0beccf9 --- /dev/null +++ b/proto/test/schemaregistry/serde/dep.proto @@ -0,0 +1,11 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +import "test/schemaregistry/serde/test.proto"; + +message DependencyMessage { + bool is_active = 1; + TestMessage test_messsage = 2; +} diff --git a/proto/test/schemaregistry/serde/example.proto b/proto/test/schemaregistry/serde/example.proto new file mode 100644 index 00000000..a8e65fca --- /dev/null +++ b/proto/test/schemaregistry/serde/example.proto @@ -0,0 +1,22 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +import "confluent/meta.proto"; + +message Author { + string name = 1 [ + (confluent.field_meta).tags = "PII" + ]; + int32 id = 2; + bytes picture = 3 [ + (confluent.field_meta).tags = "PII" + ]; + repeated string works = 4; +} + +message Pizza { + string size = 1; + repeated string toppings = 2; +} diff --git a/proto/test/schemaregistry/serde/nested.proto b/proto/test/schemaregistry/serde/nested.proto new file mode 100644 index 00000000..3b7ce1b5 --- /dev/null +++ b/proto/test/schemaregistry/serde/nested.proto @@ -0,0 +1,59 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +import "google/protobuf/timestamp.proto"; + +message UserId { + oneof user_id { + string kafka_user_id = 1; + int32 other_user_id = 2; + MessageId another_id = 3; + } +} + +message MessageId { + string id = 1; +} + +enum Status { + ACTIVE = 0; + INACTIVE = 1; +} + +message ComplexType { + oneof some_val { + string one_id = 1; + int32 other_id = 2; + } + bool is_active = 3; +} + +/* + * Complex message using nested protos and repeated fields + */ +message NestedMessage { + UserId user_id = 1; + bool is_active = 2; + repeated string experiments_active = 3; + google.protobuf.Timestamp updated_at = 4; + Status status = 5; + ComplexType complex_type = 6; + map map_type = 7; + InnerMessage inner = 8; + + message InnerMessage { + string id = 1 [json_name="id"]; + repeated int32 ids = 2 [packed=true]; + } + + enum InnerEnum { + option allow_alias = true; + ZERO = 0; + ALSO_ZERO = 0; + } + + reserved 14, 15, 9 to 11; + reserved "foo", "bar"; +} diff --git a/proto/test/schemaregistry/serde/newerwidget.proto b/proto/test/schemaregistry/serde/newerwidget.proto new file mode 100644 index 00000000..4243c0eb --- /dev/null +++ b/proto/test/schemaregistry/serde/newerwidget.proto @@ -0,0 +1,10 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +message NewerWidget { + string name = 1; + int32 length = 2; + int32 version = 3; +} diff --git a/proto/test/schemaregistry/serde/newwidget.proto b/proto/test/schemaregistry/serde/newwidget.proto new file mode 100644 index 00000000..fdca7b28 --- /dev/null +++ b/proto/test/schemaregistry/serde/newwidget.proto @@ -0,0 +1,10 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +message NewWidget { + string name = 1; + int32 height = 2; + int32 version = 3; +} diff --git a/proto/test/schemaregistry/serde/test.proto b/proto/test/schemaregistry/serde/test.proto new file mode 100644 index 00000000..28afb444 --- /dev/null +++ b/proto/test/schemaregistry/serde/test.proto @@ -0,0 +1,24 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +import "google/protobuf/descriptor.proto"; + +message TestMessage { + string test_string = 1; + bool test_bool = 2; + bytes test_bytes = 3; + double test_double = 4; + float test_float = 5; + fixed32 test_fixed32 = 6; + fixed64 test_fixed64 = 7; + int32 test_int32 = 8; + int64 test_int64 = 9; + sfixed32 test_sfixed32 = 10; + sfixed64 test_sfixed64 = 11; + sint32 test_sint32 = 12; + sint64 test_sint64 = 13; + uint32 test_uint32 = 14; + uint64 test_uint64 = 15; +} \ No newline at end of file diff --git a/proto/test/schemaregistry/serde/widget.proto b/proto/test/schemaregistry/serde/widget.proto new file mode 100644 index 00000000..94a99c2d --- /dev/null +++ b/proto/test/schemaregistry/serde/widget.proto @@ -0,0 +1,10 @@ +syntax = "proto3"; + +package test; +option go_package="../test"; + +message Widget { + string name = 1; + int32 size = 2; + int32 version = 3; +} diff --git a/schemaregistry/google/type/calendar_period_pb.ts b/schemaregistry/google/type/calendar_period_pb.ts new file mode 100644 index 00000000..a8ad8988 --- /dev/null +++ b/schemaregistry/google/type/calendar_period_pb.ts @@ -0,0 +1,102 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/calendar_period.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenEnum, GenFile } from "@bufbuild/protobuf/codegenv1"; +import { enumDesc, fileDesc } from "@bufbuild/protobuf/codegenv1"; + +/** + * Describes the file google/type/calendar_period.proto. + */ +export const file_google_type_calendar_period: GenFile = /*@__PURE__*/ + fileDesc("CiFnb29nbGUvdHlwZS9jYWxlbmRhcl9wZXJpb2QucHJvdG8SC2dvb2dsZS50eXBlKn8KDkNhbGVuZGFyUGVyaW9kEh8KG0NBTEVOREFSX1BFUklPRF9VTlNQRUNJRklFRBAAEgcKA0RBWRABEggKBFdFRUsQAhINCglGT1JUTklHSFQQAxIJCgVNT05USBAEEgsKB1FVQVJURVIQBRIICgRIQUxGEAYSCAoEWUVBUhAHQngKD2NvbS5nb29nbGUudHlwZUITQ2FsZW5kYXJQZXJpb2RQcm90b1ABWkhnb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvY2FsZW5kYXJwZXJpb2Q7Y2FsZW5kYXJwZXJpb2SiAgNHVFBiBnByb3RvMw"); + +/** + * A `CalendarPeriod` represents the abstract concept of a time period that has + * a canonical start. Grammatically, "the start of the current + * `CalendarPeriod`." All calendar times begin at midnight UTC. + * + * @generated from enum google.type.CalendarPeriod + */ +export enum CalendarPeriod { + /** + * Undefined period, raises an error. + * + * @generated from enum value: CALENDAR_PERIOD_UNSPECIFIED = 0; + */ + CALENDAR_PERIOD_UNSPECIFIED = 0, + + /** + * A day. + * + * @generated from enum value: DAY = 1; + */ + DAY = 1, + + /** + * A week. Weeks begin on Monday, following + * [ISO 8601](https://en.wikipedia.org/wiki/ISO_week_date). + * + * @generated from enum value: WEEK = 2; + */ + WEEK = 2, + + /** + * A fortnight. The first calendar fortnight of the year begins at the start + * of week 1 according to + * [ISO 8601](https://en.wikipedia.org/wiki/ISO_week_date). + * + * @generated from enum value: FORTNIGHT = 3; + */ + FORTNIGHT = 3, + + /** + * A month. + * + * @generated from enum value: MONTH = 4; + */ + MONTH = 4, + + /** + * A quarter. Quarters start on dates 1-Jan, 1-Apr, 1-Jul, and 1-Oct of each + * year. + * + * @generated from enum value: QUARTER = 5; + */ + QUARTER = 5, + + /** + * A half-year. Half-years start on dates 1-Jan and 1-Jul. + * + * @generated from enum value: HALF = 6; + */ + HALF = 6, + + /** + * A year. + * + * @generated from enum value: YEAR = 7; + */ + YEAR = 7, +} + +/** + * Describes the enum google.type.CalendarPeriod. + */ +export const CalendarPeriodSchema: GenEnum = /*@__PURE__*/ + enumDesc(file_google_type_calendar_period, 0); + diff --git a/schemaregistry/google/type/color_pb.ts b/schemaregistry/google/type/color_pb.ts new file mode 100644 index 00000000..462d70da --- /dev/null +++ b/schemaregistry/google/type/color_pb.ts @@ -0,0 +1,204 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/color.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import { file_google_protobuf_wrappers } from "@bufbuild/protobuf/wkt"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/color.proto. + */ +export const file_google_type_color: GenFile = /*@__PURE__*/ + fileDesc("Chdnb29nbGUvdHlwZS9jb2xvci5wcm90bxILZ29vZ2xlLnR5cGUiXQoFQ29sb3ISCwoDcmVkGAEgASgCEg0KBWdyZWVuGAIgASgCEgwKBGJsdWUYAyABKAISKgoFYWxwaGEYBCABKAsyGy5nb29nbGUucHJvdG9idWYuRmxvYXRWYWx1ZUJgCg9jb20uZ29vZ2xlLnR5cGVCCkNvbG9yUHJvdG9QAVo2Z29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL2NvbG9yO2NvbG9y+AEBogIDR1RQYgZwcm90bzM", [file_google_protobuf_wrappers]); + +/** + * Represents a color in the RGBA color space. This representation is designed + * for simplicity of conversion to/from color representations in various + * languages over compactness. For example, the fields of this representation + * can be trivially provided to the constructor of `java.awt.Color` in Java; it + * can also be trivially provided to UIColor's `+colorWithRed:green:blue:alpha` + * method in iOS; and, with just a little work, it can be easily formatted into + * a CSS `rgba()` string in JavaScript. + * + * This reference page doesn't carry information about the absolute color + * space + * that should be used to interpret the RGB value (e.g. sRGB, Adobe RGB, + * DCI-P3, BT.2020, etc.). By default, applications should assume the sRGB color + * space. + * + * When color equality needs to be decided, implementations, unless + * documented otherwise, treat two colors as equal if all their red, + * green, blue, and alpha values each differ by at most 1e-5. + * + * Example (Java): + * + * import com.google.type.Color; + * + * // ... + * public static java.awt.Color fromProto(Color protocolor) { + * float alpha = protocolor.hasAlpha() + * ? protocolor.getAlpha().getValue() + * : 1.0; + * + * return new java.awt.Color( + * protocolor.getRed(), + * protocolor.getGreen(), + * protocolor.getBlue(), + * alpha); + * } + * + * public static Color toProto(java.awt.Color color) { + * float red = (float) color.getRed(); + * float green = (float) color.getGreen(); + * float blue = (float) color.getBlue(); + * float denominator = 255.0; + * Color.Builder resultBuilder = + * Color + * .newBuilder() + * .setRed(red / denominator) + * .setGreen(green / denominator) + * .setBlue(blue / denominator); + * int alpha = color.getAlpha(); + * if (alpha != 255) { + * result.setAlpha( + * FloatValue + * .newBuilder() + * .setValue(((float) alpha) / denominator) + * .build()); + * } + * return resultBuilder.build(); + * } + * // ... + * + * Example (iOS / Obj-C): + * + * // ... + * static UIColor* fromProto(Color* protocolor) { + * float red = [protocolor red]; + * float green = [protocolor green]; + * float blue = [protocolor blue]; + * FloatValue* alpha_wrapper = [protocolor alpha]; + * float alpha = 1.0; + * if (alpha_wrapper != nil) { + * alpha = [alpha_wrapper value]; + * } + * return [UIColor colorWithRed:red green:green blue:blue alpha:alpha]; + * } + * + * static Color* toProto(UIColor* color) { + * CGFloat red, green, blue, alpha; + * if (![color getRed:&red green:&green blue:&blue alpha:&alpha]) { + * return nil; + * } + * Color* result = [[Color alloc] init]; + * [result setRed:red]; + * [result setGreen:green]; + * [result setBlue:blue]; + * if (alpha <= 0.9999) { + * [result setAlpha:floatWrapperWithValue(alpha)]; + * } + * [result autorelease]; + * return result; + * } + * // ... + * + * Example (JavaScript): + * + * // ... + * + * var protoToCssColor = function(rgb_color) { + * var redFrac = rgb_color.red || 0.0; + * var greenFrac = rgb_color.green || 0.0; + * var blueFrac = rgb_color.blue || 0.0; + * var red = Math.floor(redFrac * 255); + * var green = Math.floor(greenFrac * 255); + * var blue = Math.floor(blueFrac * 255); + * + * if (!('alpha' in rgb_color)) { + * return rgbToCssColor(red, green, blue); + * } + * + * var alphaFrac = rgb_color.alpha.value || 0.0; + * var rgbParams = [red, green, blue].join(','); + * return ['rgba(', rgbParams, ',', alphaFrac, ')'].join(''); + * }; + * + * var rgbToCssColor = function(red, green, blue) { + * var rgbNumber = new Number((red << 16) | (green << 8) | blue); + * var hexString = rgbNumber.toString(16); + * var missingZeros = 6 - hexString.length; + * var resultBuilder = ['#']; + * for (var i = 0; i < missingZeros; i++) { + * resultBuilder.push('0'); + * } + * resultBuilder.push(hexString); + * return resultBuilder.join(''); + * }; + * + * // ... + * + * @generated from message google.type.Color + */ +export type Color = Message<"google.type.Color"> & { + /** + * The amount of red in the color as a value in the interval [0, 1]. + * + * @generated from field: float red = 1; + */ + red: number; + + /** + * The amount of green in the color as a value in the interval [0, 1]. + * + * @generated from field: float green = 2; + */ + green: number; + + /** + * The amount of blue in the color as a value in the interval [0, 1]. + * + * @generated from field: float blue = 3; + */ + blue: number; + + /** + * The fraction of this color that should be applied to the pixel. That is, + * the final pixel color is defined by the equation: + * + * `pixel color = alpha * (this color) + (1.0 - alpha) * (background color)` + * + * This means that a value of 1.0 corresponds to a solid color, whereas + * a value of 0.0 corresponds to a completely transparent color. This + * uses a wrapper message rather than a simple float scalar so that it is + * possible to distinguish between a default value and the value being unset. + * If omitted, this color object is rendered as a solid color + * (as if the alpha value had been explicitly given a value of 1.0). + * + * @generated from field: google.protobuf.FloatValue alpha = 4; + */ + alpha?: number; +}; + +/** + * Describes the message google.type.Color. + * Use `create(ColorSchema)` to create a new message. + */ +export const ColorSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_color, 0); + diff --git a/schemaregistry/google/type/date_pb.ts b/schemaregistry/google/type/date_pb.ts new file mode 100644 index 00000000..eaf34962 --- /dev/null +++ b/schemaregistry/google/type/date_pb.ts @@ -0,0 +1,79 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/date.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/date.proto. + */ +export const file_google_type_date: GenFile = /*@__PURE__*/ + fileDesc("ChZnb29nbGUvdHlwZS9kYXRlLnByb3RvEgtnb29nbGUudHlwZSIwCgREYXRlEgwKBHllYXIYASABKAUSDQoFbW9udGgYAiABKAUSCwoDZGF5GAMgASgFQl0KD2NvbS5nb29nbGUudHlwZUIJRGF0ZVByb3RvUAFaNGdvb2dsZS5nb2xhbmcub3JnL2dlbnByb3RvL2dvb2dsZWFwaXMvdHlwZS9kYXRlO2RhdGX4AQGiAgNHVFBiBnByb3RvMw"); + +/** + * Represents a whole or partial calendar date, such as a birthday. The time of + * day and time zone are either specified elsewhere or are insignificant. The + * date is relative to the Gregorian Calendar. This can represent one of the + * following: + * + * * A full date, with non-zero year, month, and day values + * * A month and day value, with a zero year, such as an anniversary + * * A year on its own, with zero month and day values + * * A year and month value, with a zero day, such as a credit card expiration + * date + * + * Related types are [google.type.TimeOfDay][google.type.TimeOfDay] and + * `google.protobuf.Timestamp`. + * + * @generated from message google.type.Date + */ +export type Date = Message<"google.type.Date"> & { + /** + * Year of the date. Must be from 1 to 9999, or 0 to specify a date without + * a year. + * + * @generated from field: int32 year = 1; + */ + year: number; + + /** + * Month of a year. Must be from 1 to 12, or 0 to specify a year without a + * month and day. + * + * @generated from field: int32 month = 2; + */ + month: number; + + /** + * Day of a month. Must be from 1 to 31 and valid for the year and month, or 0 + * to specify a year by itself or a year and month where the day isn't + * significant. + * + * @generated from field: int32 day = 3; + */ + day: number; +}; + +/** + * Describes the message google.type.Date. + * Use `create(DateSchema)` to create a new message. + */ +export const DateSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_date, 0); + diff --git a/schemaregistry/google/type/datetime_pb.ts b/schemaregistry/google/type/datetime_pb.ts new file mode 100644 index 00000000..511b0859 --- /dev/null +++ b/schemaregistry/google/type/datetime_pb.ts @@ -0,0 +1,180 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/datetime.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Duration } from "@bufbuild/protobuf/wkt"; +import { file_google_protobuf_duration } from "@bufbuild/protobuf/wkt"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/datetime.proto. + */ +export const file_google_type_datetime: GenFile = /*@__PURE__*/ + fileDesc("Chpnb29nbGUvdHlwZS9kYXRldGltZS5wcm90bxILZ29vZ2xlLnR5cGUi4AEKCERhdGVUaW1lEgwKBHllYXIYASABKAUSDQoFbW9udGgYAiABKAUSCwoDZGF5GAMgASgFEg0KBWhvdXJzGAQgASgFEg8KB21pbnV0ZXMYBSABKAUSDwoHc2Vjb25kcxgGIAEoBRINCgVuYW5vcxgHIAEoBRIvCgp1dGNfb2Zmc2V0GAggASgLMhkuZ29vZ2xlLnByb3RvYnVmLkR1cmF0aW9uSAASKgoJdGltZV96b25lGAkgASgLMhUuZ29vZ2xlLnR5cGUuVGltZVpvbmVIAEINCgt0aW1lX29mZnNldCInCghUaW1lWm9uZRIKCgJpZBgBIAEoCRIPCgd2ZXJzaW9uGAIgASgJQmkKD2NvbS5nb29nbGUudHlwZUINRGF0ZVRpbWVQcm90b1ABWjxnb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvZGF0ZXRpbWU7ZGF0ZXRpbWX4AQGiAgNHVFBiBnByb3RvMw", [file_google_protobuf_duration]); + +/** + * Represents civil time (or occasionally physical time). + * + * This type can represent a civil time in one of a few possible ways: + * + * * When utc_offset is set and time_zone is unset: a civil time on a calendar + * day with a particular offset from UTC. + * * When time_zone is set and utc_offset is unset: a civil time on a calendar + * day in a particular time zone. + * * When neither time_zone nor utc_offset is set: a civil time on a calendar + * day in local time. + * + * The date is relative to the Proleptic Gregorian Calendar. + * + * If year is 0, the DateTime is considered not to have a specific year. month + * and day must have valid, non-zero values. + * + * This type may also be used to represent a physical time if all the date and + * time fields are set and either case of the `time_offset` oneof is set. + * Consider using `Timestamp` message for physical time instead. If your use + * case also would like to store the user's timezone, that can be done in + * another field. + * + * This type is more flexible than some applications may want. Make sure to + * document and validate your application's limitations. + * + * @generated from message google.type.DateTime + */ +export type DateTime = Message<"google.type.DateTime"> & { + /** + * Optional. Year of date. Must be from 1 to 9999, or 0 if specifying a + * datetime without a year. + * + * @generated from field: int32 year = 1; + */ + year: number; + + /** + * Required. Month of year. Must be from 1 to 12. + * + * @generated from field: int32 month = 2; + */ + month: number; + + /** + * Required. Day of month. Must be from 1 to 31 and valid for the year and + * month. + * + * @generated from field: int32 day = 3; + */ + day: number; + + /** + * Required. Hours of day in 24 hour format. Should be from 0 to 23. An API + * may choose to allow the value "24:00:00" for scenarios like business + * closing time. + * + * @generated from field: int32 hours = 4; + */ + hours: number; + + /** + * Required. Minutes of hour of day. Must be from 0 to 59. + * + * @generated from field: int32 minutes = 5; + */ + minutes: number; + + /** + * Required. Seconds of minutes of the time. Must normally be from 0 to 59. An + * API may allow the value 60 if it allows leap-seconds. + * + * @generated from field: int32 seconds = 6; + */ + seconds: number; + + /** + * Required. Fractions of seconds in nanoseconds. Must be from 0 to + * 999,999,999. + * + * @generated from field: int32 nanos = 7; + */ + nanos: number; + + /** + * Optional. Specifies either the UTC offset or the time zone of the DateTime. + * Choose carefully between them, considering that time zone data may change + * in the future (for example, a country modifies their DST start/end dates, + * and future DateTimes in the affected range had already been stored). + * If omitted, the DateTime is considered to be in local time. + * + * @generated from oneof google.type.DateTime.time_offset + */ + timeOffset: { + /** + * UTC offset. Must be whole seconds, between -18 hours and +18 hours. + * For example, a UTC offset of -4:00 would be represented as + * { seconds: -14400 }. + * + * @generated from field: google.protobuf.Duration utc_offset = 8; + */ + value: Duration; + case: "utcOffset"; + } | { + /** + * Time zone. + * + * @generated from field: google.type.TimeZone time_zone = 9; + */ + value: TimeZone; + case: "timeZone"; + } | { case: undefined; value?: undefined }; +}; + +/** + * Describes the message google.type.DateTime. + * Use `create(DateTimeSchema)` to create a new message. + */ +export const DateTimeSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_datetime, 0); + +/** + * Represents a time zone from the + * [IANA Time Zone Database](https://www.iana.org/time-zones). + * + * @generated from message google.type.TimeZone + */ +export type TimeZone = Message<"google.type.TimeZone"> & { + /** + * IANA Time Zone Database time zone, e.g. "America/New_York". + * + * @generated from field: string id = 1; + */ + id: string; + + /** + * Optional. IANA Time Zone Database version number, e.g. "2019a". + * + * @generated from field: string version = 2; + */ + version: string; +}; + +/** + * Describes the message google.type.TimeZone. + * Use `create(TimeZoneSchema)` to create a new message. + */ +export const TimeZoneSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_datetime, 1); + diff --git a/schemaregistry/google/type/dayofweek_pb.ts b/schemaregistry/google/type/dayofweek_pb.ts new file mode 100644 index 00000000..8932990c --- /dev/null +++ b/schemaregistry/google/type/dayofweek_pb.ts @@ -0,0 +1,96 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/dayofweek.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenEnum, GenFile } from "@bufbuild/protobuf/codegenv1"; +import { enumDesc, fileDesc } from "@bufbuild/protobuf/codegenv1"; + +/** + * Describes the file google/type/dayofweek.proto. + */ +export const file_google_type_dayofweek: GenFile = /*@__PURE__*/ + fileDesc("Chtnb29nbGUvdHlwZS9kYXlvZndlZWsucHJvdG8SC2dvb2dsZS50eXBlKoQBCglEYXlPZldlZWsSGwoXREFZX09GX1dFRUtfVU5TUEVDSUZJRUQQABIKCgZNT05EQVkQARILCgdUVUVTREFZEAISDQoJV0VETkVTREFZEAMSDAoIVEhVUlNEQVkQBBIKCgZGUklEQVkQBRIMCghTQVRVUkRBWRAGEgoKBlNVTkRBWRAHQmkKD2NvbS5nb29nbGUudHlwZUIORGF5T2ZXZWVrUHJvdG9QAVo+Z29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL2RheW9md2VlaztkYXlvZndlZWuiAgNHVFBiBnByb3RvMw"); + +/** + * Represents a day of the week. + * + * @generated from enum google.type.DayOfWeek + */ +export enum DayOfWeek { + /** + * The day of the week is unspecified. + * + * @generated from enum value: DAY_OF_WEEK_UNSPECIFIED = 0; + */ + DAY_OF_WEEK_UNSPECIFIED = 0, + + /** + * Monday + * + * @generated from enum value: MONDAY = 1; + */ + MONDAY = 1, + + /** + * Tuesday + * + * @generated from enum value: TUESDAY = 2; + */ + TUESDAY = 2, + + /** + * Wednesday + * + * @generated from enum value: WEDNESDAY = 3; + */ + WEDNESDAY = 3, + + /** + * Thursday + * + * @generated from enum value: THURSDAY = 4; + */ + THURSDAY = 4, + + /** + * Friday + * + * @generated from enum value: FRIDAY = 5; + */ + FRIDAY = 5, + + /** + * Saturday + * + * @generated from enum value: SATURDAY = 6; + */ + SATURDAY = 6, + + /** + * Sunday + * + * @generated from enum value: SUNDAY = 7; + */ + SUNDAY = 7, +} + +/** + * Describes the enum google.type.DayOfWeek. + */ +export const DayOfWeekSchema: GenEnum = /*@__PURE__*/ + enumDesc(file_google_type_dayofweek, 0); + diff --git a/schemaregistry/google/type/decimal_pb.ts b/schemaregistry/google/type/decimal_pb.ts new file mode 100644 index 00000000..f3c65377 --- /dev/null +++ b/schemaregistry/google/type/decimal_pb.ts @@ -0,0 +1,114 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/decimal.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/decimal.proto. + */ +export const file_google_type_decimal: GenFile = /*@__PURE__*/ + fileDesc("Chlnb29nbGUvdHlwZS9kZWNpbWFsLnByb3RvEgtnb29nbGUudHlwZSIYCgdEZWNpbWFsEg0KBXZhbHVlGAEgASgJQmYKD2NvbS5nb29nbGUudHlwZUIMRGVjaW1hbFByb3RvUAFaOmdvb2dsZS5nb2xhbmcub3JnL2dlbnByb3RvL2dvb2dsZWFwaXMvdHlwZS9kZWNpbWFsO2RlY2ltYWz4AQGiAgNHVFBiBnByb3RvMw"); + +/** + * A representation of a decimal value, such as 2.5. Clients may convert values + * into language-native decimal formats, such as Java's [BigDecimal][] or + * Python's [decimal.Decimal][]. + * + * [BigDecimal]: + * https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/math/BigDecimal.html + * [decimal.Decimal]: https://docs.python.org/3/library/decimal.html + * + * @generated from message google.type.Decimal + */ +export type Decimal = Message<"google.type.Decimal"> & { + /** + * The decimal value, as a string. + * + * The string representation consists of an optional sign, `+` (`U+002B`) + * or `-` (`U+002D`), followed by a sequence of zero or more decimal digits + * ("the integer"), optionally followed by a fraction, optionally followed + * by an exponent. + * + * The fraction consists of a decimal point followed by zero or more decimal + * digits. The string must contain at least one digit in either the integer + * or the fraction. The number formed by the sign, the integer and the + * fraction is referred to as the significand. + * + * The exponent consists of the character `e` (`U+0065`) or `E` (`U+0045`) + * followed by one or more decimal digits. + * + * Services **should** normalize decimal values before storing them by: + * + * - Removing an explicitly-provided `+` sign (`+2.5` -> `2.5`). + * - Replacing a zero-length integer value with `0` (`.5` -> `0.5`). + * - Coercing the exponent character to lower-case (`2.5E8` -> `2.5e8`). + * - Removing an explicitly-provided zero exponent (`2.5e0` -> `2.5`). + * + * Services **may** perform additional normalization based on its own needs + * and the internal decimal implementation selected, such as shifting the + * decimal point and exponent value together (example: `2.5e-1` <-> `0.25`). + * Additionally, services **may** preserve trailing zeroes in the fraction + * to indicate increased precision, but are not required to do so. + * + * Note that only the `.` character is supported to divide the integer + * and the fraction; `,` **should not** be supported regardless of locale. + * Additionally, thousand separators **should not** be supported. If a + * service does support them, values **must** be normalized. + * + * The ENBF grammar is: + * + * DecimalString = + * [Sign] Significand [Exponent]; + * + * Sign = '+' | '-'; + * + * Significand = + * Digits ['.'] [Digits] | [Digits] '.' Digits; + * + * Exponent = ('e' | 'E') [Sign] Digits; + * + * Digits = { '0' | '1' | '2' | '3' | '4' | '5' | '6' | '7' | '8' | '9' }; + * + * Services **should** clearly document the range of supported values, the + * maximum supported precision (total number of digits), and, if applicable, + * the scale (number of digits after the decimal point), as well as how it + * behaves when receiving out-of-bounds values. + * + * Services **may** choose to accept values passed as input even when the + * value has a higher precision or scale than the service supports, and + * **should** round the value to fit the supported scale. Alternatively, the + * service **may** error with `400 Bad Request` (`INVALID_ARGUMENT` in gRPC) + * if precision would be lost. + * + * Services **should** error with `400 Bad Request` (`INVALID_ARGUMENT` in + * gRPC) if the service receives a value outside of the supported range. + * + * @generated from field: string value = 1; + */ + value: string; +}; + +/** + * Describes the message google.type.Decimal. + * Use `create(DecimalSchema)` to create a new message. + */ +export const DecimalSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_decimal, 0); + diff --git a/schemaregistry/google/type/expr_pb.ts b/schemaregistry/google/type/expr_pb.ts new file mode 100644 index 00000000..a4dd114b --- /dev/null +++ b/schemaregistry/google/type/expr_pb.ts @@ -0,0 +1,105 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/expr.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/expr.proto. + */ +export const file_google_type_expr: GenFile = /*@__PURE__*/ + fileDesc("ChZnb29nbGUvdHlwZS9leHByLnByb3RvEgtnb29nbGUudHlwZSJQCgRFeHByEhIKCmV4cHJlc3Npb24YASABKAkSDQoFdGl0bGUYAiABKAkSEwoLZGVzY3JpcHRpb24YAyABKAkSEAoIbG9jYXRpb24YBCABKAlCWgoPY29tLmdvb2dsZS50eXBlQglFeHByUHJvdG9QAVo0Z29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL2V4cHI7ZXhwcqICA0dUUGIGcHJvdG8z"); + +/** + * Represents a textual expression in the Common Expression Language (CEL) + * syntax. CEL is a C-like expression language. The syntax and semantics of CEL + * are documented at https://github.com/google/cel-spec. + * + * Example (Comparison): + * + * title: "Summary size limit" + * description: "Determines if a summary is less than 100 chars" + * expression: "document.summary.size() < 100" + * + * Example (Equality): + * + * title: "Requestor is owner" + * description: "Determines if requestor is the document owner" + * expression: "document.owner == request.auth.claims.email" + * + * Example (Logic): + * + * title: "Public documents" + * description: "Determine whether the document should be publicly visible" + * expression: "document.type != 'private' && document.type != 'internal'" + * + * Example (Data Manipulation): + * + * title: "Notification string" + * description: "Create a notification string with a timestamp." + * expression: "'New message received at ' + string(document.create_time)" + * + * The exact variables and functions that may be referenced within an expression + * are determined by the service that evaluates it. See the service + * documentation for additional information. + * + * @generated from message google.type.Expr + */ +export type Expr = Message<"google.type.Expr"> & { + /** + * Textual representation of an expression in Common Expression Language + * syntax. + * + * @generated from field: string expression = 1; + */ + expression: string; + + /** + * Optional. Title for the expression, i.e. a short string describing + * its purpose. This can be used e.g. in UIs which allow to enter the + * expression. + * + * @generated from field: string title = 2; + */ + title: string; + + /** + * Optional. Description of the expression. This is a longer text which + * describes the expression, e.g. when hovered over it in a UI. + * + * @generated from field: string description = 3; + */ + description: string; + + /** + * Optional. String indicating the location of the expression for error + * reporting, e.g. a file name and a position in the file. + * + * @generated from field: string location = 4; + */ + location: string; +}; + +/** + * Describes the message google.type.Expr. + * Use `create(ExprSchema)` to create a new message. + */ +export const ExprSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_expr, 0); + diff --git a/schemaregistry/google/type/fraction_pb.ts b/schemaregistry/google/type/fraction_pb.ts new file mode 100644 index 00000000..2d180530 --- /dev/null +++ b/schemaregistry/google/type/fraction_pb.ts @@ -0,0 +1,57 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/fraction.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/fraction.proto. + */ +export const file_google_type_fraction: GenFile = /*@__PURE__*/ + fileDesc("Chpnb29nbGUvdHlwZS9mcmFjdGlvbi5wcm90bxILZ29vZ2xlLnR5cGUiMgoIRnJhY3Rpb24SEQoJbnVtZXJhdG9yGAEgASgDEhMKC2Rlbm9taW5hdG9yGAIgASgDQmYKD2NvbS5nb29nbGUudHlwZUINRnJhY3Rpb25Qcm90b1ABWjxnb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvZnJhY3Rpb247ZnJhY3Rpb26iAgNHVFBiBnByb3RvMw"); + +/** + * Represents a fraction in terms of a numerator divided by a denominator. + * + * @generated from message google.type.Fraction + */ +export type Fraction = Message<"google.type.Fraction"> & { + /** + * The numerator in the fraction, e.g. 2 in 2/3. + * + * @generated from field: int64 numerator = 1; + */ + numerator: bigint; + + /** + * The value by which the numerator is divided, e.g. 3 in 2/3. Must be + * positive. + * + * @generated from field: int64 denominator = 2; + */ + denominator: bigint; +}; + +/** + * Describes the message google.type.Fraction. + * Use `create(FractionSchema)` to create a new message. + */ +export const FractionSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_fraction, 0); + diff --git a/schemaregistry/google/type/interval_pb.ts b/schemaregistry/google/type/interval_pb.ts new file mode 100644 index 00000000..ae5ffa96 --- /dev/null +++ b/schemaregistry/google/type/interval_pb.ts @@ -0,0 +1,69 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/interval.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Timestamp } from "@bufbuild/protobuf/wkt"; +import { file_google_protobuf_timestamp } from "@bufbuild/protobuf/wkt"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/interval.proto. + */ +export const file_google_type_interval: GenFile = /*@__PURE__*/ + fileDesc("Chpnb29nbGUvdHlwZS9pbnRlcnZhbC5wcm90bxILZ29vZ2xlLnR5cGUiaAoISW50ZXJ2YWwSLgoKc3RhcnRfdGltZRgBIAEoCzIaLmdvb2dsZS5wcm90b2J1Zi5UaW1lc3RhbXASLAoIZW5kX3RpbWUYAiABKAsyGi5nb29nbGUucHJvdG9idWYuVGltZXN0YW1wQmkKD2NvbS5nb29nbGUudHlwZUINSW50ZXJ2YWxQcm90b1ABWjxnb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvaW50ZXJ2YWw7aW50ZXJ2YWz4AQGiAgNHVFBiBnByb3RvMw", [file_google_protobuf_timestamp]); + +/** + * Represents a time interval, encoded as a Timestamp start (inclusive) and a + * Timestamp end (exclusive). + * + * The start must be less than or equal to the end. + * When the start equals the end, the interval is empty (matches no time). + * When both start and end are unspecified, the interval matches any time. + * + * @generated from message google.type.Interval + */ +export type Interval = Message<"google.type.Interval"> & { + /** + * Optional. Inclusive start of the interval. + * + * If specified, a Timestamp matching this interval will have to be the same + * or after the start. + * + * @generated from field: google.protobuf.Timestamp start_time = 1; + */ + startTime?: Timestamp; + + /** + * Optional. Exclusive end of the interval. + * + * If specified, a Timestamp matching this interval will have to be before the + * end. + * + * @generated from field: google.protobuf.Timestamp end_time = 2; + */ + endTime?: Timestamp; +}; + +/** + * Describes the message google.type.Interval. + * Use `create(IntervalSchema)` to create a new message. + */ +export const IntervalSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_interval, 0); + diff --git a/schemaregistry/google/type/latlng_pb.ts b/schemaregistry/google/type/latlng_pb.ts new file mode 100644 index 00000000..41c99c15 --- /dev/null +++ b/schemaregistry/google/type/latlng_pb.ts @@ -0,0 +1,60 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/latlng.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/latlng.proto. + */ +export const file_google_type_latlng: GenFile = /*@__PURE__*/ + fileDesc("Chhnb29nbGUvdHlwZS9sYXRsbmcucHJvdG8SC2dvb2dsZS50eXBlIi0KBkxhdExuZxIQCghsYXRpdHVkZRgBIAEoARIRCglsb25naXR1ZGUYAiABKAFCYwoPY29tLmdvb2dsZS50eXBlQgtMYXRMbmdQcm90b1ABWjhnb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvbGF0bG5nO2xhdGxuZ/gBAaICA0dUUGIGcHJvdG8z"); + +/** + * An object that represents a latitude/longitude pair. This is expressed as a + * pair of doubles to represent degrees latitude and degrees longitude. Unless + * specified otherwise, this must conform to the + * WGS84 + * standard. Values must be within normalized ranges. + * + * @generated from message google.type.LatLng + */ +export type LatLng = Message<"google.type.LatLng"> & { + /** + * The latitude in degrees. It must be in the range [-90.0, +90.0]. + * + * @generated from field: double latitude = 1; + */ + latitude: number; + + /** + * The longitude in degrees. It must be in the range [-180.0, +180.0]. + * + * @generated from field: double longitude = 2; + */ + longitude: number; +}; + +/** + * Describes the message google.type.LatLng. + * Use `create(LatLngSchema)` to create a new message. + */ +export const LatLngSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_latlng, 0); + diff --git a/schemaregistry/google/type/localized_text_pb.ts b/schemaregistry/google/type/localized_text_pb.ts new file mode 100644 index 00000000..80235333 --- /dev/null +++ b/schemaregistry/google/type/localized_text_pb.ts @@ -0,0 +1,59 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/localized_text.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/localized_text.proto. + */ +export const file_google_type_localized_text: GenFile = /*@__PURE__*/ + fileDesc("CiBnb29nbGUvdHlwZS9sb2NhbGl6ZWRfdGV4dC5wcm90bxILZ29vZ2xlLnR5cGUiNAoNTG9jYWxpemVkVGV4dBIMCgR0ZXh0GAEgASgJEhUKDWxhbmd1YWdlX2NvZGUYAiABKAlCegoPY29tLmdvb2dsZS50eXBlQhJMb2NhbGl6ZWRUZXh0UHJvdG9QAVpIZ29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL2xvY2FsaXplZF90ZXh0O2xvY2FsaXplZF90ZXh0+AEBogIDR1RQYgZwcm90bzM"); + +/** + * Localized variant of a text in a particular language. + * + * @generated from message google.type.LocalizedText + */ +export type LocalizedText = Message<"google.type.LocalizedText"> & { + /** + * Localized string in the language corresponding to `language_code' below. + * + * @generated from field: string text = 1; + */ + text: string; + + /** + * The text's BCP-47 language code, such as "en-US" or "sr-Latn". + * + * For more information, see + * http://www.unicode.org/reports/tr35/#Unicode_locale_identifier. + * + * @generated from field: string language_code = 2; + */ + languageCode: string; +}; + +/** + * Describes the message google.type.LocalizedText. + * Use `create(LocalizedTextSchema)` to create a new message. + */ +export const LocalizedTextSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_localized_text, 0); + diff --git a/schemaregistry/google/type/money_pb.ts b/schemaregistry/google/type/money_pb.ts new file mode 100644 index 00000000..de6bd31d --- /dev/null +++ b/schemaregistry/google/type/money_pb.ts @@ -0,0 +1,69 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/money.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/money.proto. + */ +export const file_google_type_money: GenFile = /*@__PURE__*/ + fileDesc("Chdnb29nbGUvdHlwZS9tb25leS5wcm90bxILZ29vZ2xlLnR5cGUiPAoFTW9uZXkSFQoNY3VycmVuY3lfY29kZRgBIAEoCRINCgV1bml0cxgCIAEoAxINCgVuYW5vcxgDIAEoBUJgCg9jb20uZ29vZ2xlLnR5cGVCCk1vbmV5UHJvdG9QAVo2Z29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL21vbmV5O21vbmV5+AEBogIDR1RQYgZwcm90bzM"); + +/** + * Represents an amount of money with its currency type. + * + * @generated from message google.type.Money + */ +export type Money = Message<"google.type.Money"> & { + /** + * The three-letter currency code defined in ISO 4217. + * + * @generated from field: string currency_code = 1; + */ + currencyCode: string; + + /** + * The whole units of the amount. + * For example if `currencyCode` is `"USD"`, then 1 unit is one US dollar. + * + * @generated from field: int64 units = 2; + */ + units: bigint; + + /** + * Number of nano (10^-9) units of the amount. + * The value must be between -999,999,999 and +999,999,999 inclusive. + * If `units` is positive, `nanos` must be positive or zero. + * If `units` is zero, `nanos` can be positive, zero, or negative. + * If `units` is negative, `nanos` must be negative or zero. + * For example $-1.75 is represented as `units`=-1 and `nanos`=-750,000,000. + * + * @generated from field: int32 nanos = 3; + */ + nanos: number; +}; + +/** + * Describes the message google.type.Money. + * Use `create(MoneySchema)` to create a new message. + */ +export const MoneySchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_money, 0); + diff --git a/schemaregistry/google/type/month_pb.ts b/schemaregistry/google/type/month_pb.ts new file mode 100644 index 00000000..072f60b1 --- /dev/null +++ b/schemaregistry/google/type/month_pb.ts @@ -0,0 +1,131 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/month.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenEnum, GenFile } from "@bufbuild/protobuf/codegenv1"; +import { enumDesc, fileDesc } from "@bufbuild/protobuf/codegenv1"; + +/** + * Describes the file google/type/month.proto. + */ +export const file_google_type_month: GenFile = /*@__PURE__*/ + fileDesc("Chdnb29nbGUvdHlwZS9tb250aC5wcm90bxILZ29vZ2xlLnR5cGUqsAEKBU1vbnRoEhUKEU1PTlRIX1VOU1BFQ0lGSUVEEAASCwoHSkFOVUFSWRABEgwKCEZFQlJVQVJZEAISCQoFTUFSQ0gQAxIJCgVBUFJJTBAEEgcKA01BWRAFEggKBEpVTkUQBhIICgRKVUxZEAcSCgoGQVVHVVNUEAgSDQoJU0VQVEVNQkVSEAkSCwoHT0NUT0JFUhAKEgwKCE5PVkVNQkVSEAsSDAoIREVDRU1CRVIQDEJdCg9jb20uZ29vZ2xlLnR5cGVCCk1vbnRoUHJvdG9QAVo2Z29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL21vbnRoO21vbnRoogIDR1RQYgZwcm90bzM"); + +/** + * Represents a month in the Gregorian calendar. + * + * @generated from enum google.type.Month + */ +export enum Month { + /** + * The unspecified month. + * + * @generated from enum value: MONTH_UNSPECIFIED = 0; + */ + MONTH_UNSPECIFIED = 0, + + /** + * The month of January. + * + * @generated from enum value: JANUARY = 1; + */ + JANUARY = 1, + + /** + * The month of February. + * + * @generated from enum value: FEBRUARY = 2; + */ + FEBRUARY = 2, + + /** + * The month of March. + * + * @generated from enum value: MARCH = 3; + */ + MARCH = 3, + + /** + * The month of April. + * + * @generated from enum value: APRIL = 4; + */ + APRIL = 4, + + /** + * The month of May. + * + * @generated from enum value: MAY = 5; + */ + MAY = 5, + + /** + * The month of June. + * + * @generated from enum value: JUNE = 6; + */ + JUNE = 6, + + /** + * The month of July. + * + * @generated from enum value: JULY = 7; + */ + JULY = 7, + + /** + * The month of August. + * + * @generated from enum value: AUGUST = 8; + */ + AUGUST = 8, + + /** + * The month of September. + * + * @generated from enum value: SEPTEMBER = 9; + */ + SEPTEMBER = 9, + + /** + * The month of October. + * + * @generated from enum value: OCTOBER = 10; + */ + OCTOBER = 10, + + /** + * The month of November. + * + * @generated from enum value: NOVEMBER = 11; + */ + NOVEMBER = 11, + + /** + * The month of December. + * + * @generated from enum value: DECEMBER = 12; + */ + DECEMBER = 12, +} + +/** + * Describes the enum google.type.Month. + */ +export const MonthSchema: GenEnum = /*@__PURE__*/ + enumDesc(file_google_type_month, 0); + diff --git a/schemaregistry/google/type/phone_number_pb.ts b/schemaregistry/google/type/phone_number_pb.ts new file mode 100644 index 00000000..f11e0e54 --- /dev/null +++ b/schemaregistry/google/type/phone_number_pb.ts @@ -0,0 +1,165 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/phone_number.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/phone_number.proto. + */ +export const file_google_type_phone_number: GenFile = /*@__PURE__*/ + fileDesc("Ch5nb29nbGUvdHlwZS9waG9uZV9udW1iZXIucHJvdG8SC2dvb2dsZS50eXBlIqsBCgtQaG9uZU51bWJlchIVCgtlMTY0X251bWJlchgBIAEoCUgAEjgKCnNob3J0X2NvZGUYAiABKAsyIi5nb29nbGUudHlwZS5QaG9uZU51bWJlci5TaG9ydENvZGVIABIRCglleHRlbnNpb24YAyABKAkaMAoJU2hvcnRDb2RlEhMKC3JlZ2lvbl9jb2RlGAEgASgJEg4KBm51bWJlchgCIAEoCUIGCgRraW5kQnQKD2NvbS5nb29nbGUudHlwZUIQUGhvbmVOdW1iZXJQcm90b1ABWkRnb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvcGhvbmVfbnVtYmVyO3Bob25lX251bWJlcvgBAaICA0dUUGIGcHJvdG8z"); + +/** + * An object representing a phone number, suitable as an API wire format. + * + * This representation: + * + * - should not be used for locale-specific formatting of a phone number, such + * as "+1 (650) 253-0000 ext. 123" + * + * - is not designed for efficient storage + * - may not be suitable for dialing - specialized libraries (see references) + * should be used to parse the number for that purpose + * + * To do something meaningful with this number, such as format it for various + * use-cases, convert it to an `i18n.phonenumbers.PhoneNumber` object first. + * + * For instance, in Java this would be: + * + * com.google.type.PhoneNumber wireProto = + * com.google.type.PhoneNumber.newBuilder().build(); + * com.google.i18n.phonenumbers.Phonenumber.PhoneNumber phoneNumber = + * PhoneNumberUtil.getInstance().parse(wireProto.getE164Number(), "ZZ"); + * if (!wireProto.getExtension().isEmpty()) { + * phoneNumber.setExtension(wireProto.getExtension()); + * } + * + * Reference(s): + * - https://github.com/google/libphonenumber + * + * @generated from message google.type.PhoneNumber + */ +export type PhoneNumber = Message<"google.type.PhoneNumber"> & { + /** + * Required. Either a regular number, or a short code. New fields may be + * added to the oneof below in the future, so clients should ignore phone + * numbers for which none of the fields they coded against are set. + * + * @generated from oneof google.type.PhoneNumber.kind + */ + kind: { + /** + * The phone number, represented as a leading plus sign ('+'), followed by a + * phone number that uses a relaxed ITU E.164 format consisting of the + * country calling code (1 to 3 digits) and the subscriber number, with no + * additional spaces or formatting, e.g.: + * - correct: "+15552220123" + * - incorrect: "+1 (555) 222-01234 x123". + * + * The ITU E.164 format limits the latter to 12 digits, but in practice not + * all countries respect that, so we relax that restriction here. + * National-only numbers are not allowed. + * + * References: + * - https://www.itu.int/rec/T-REC-E.164-201011-I + * - https://en.wikipedia.org/wiki/E.164. + * - https://en.wikipedia.org/wiki/List_of_country_calling_codes + * + * @generated from field: string e164_number = 1; + */ + value: string; + case: "e164Number"; + } | { + /** + * A short code. + * + * Reference(s): + * - https://en.wikipedia.org/wiki/Short_code + * + * @generated from field: google.type.PhoneNumber.ShortCode short_code = 2; + */ + value: PhoneNumber_ShortCode; + case: "shortCode"; + } | { case: undefined; value?: undefined }; + + /** + * The phone number's extension. The extension is not standardized in ITU + * recommendations, except for being defined as a series of numbers with a + * maximum length of 40 digits. Other than digits, some other dialing + * characters such as ',' (indicating a wait) or '#' may be stored here. + * + * Note that no regions currently use extensions with short codes, so this + * field is normally only set in conjunction with an E.164 number. It is held + * separately from the E.164 number to allow for short code extensions in the + * future. + * + * @generated from field: string extension = 3; + */ + extension: string; +}; + +/** + * Describes the message google.type.PhoneNumber. + * Use `create(PhoneNumberSchema)` to create a new message. + */ +export const PhoneNumberSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_phone_number, 0); + +/** + * An object representing a short code, which is a phone number that is + * typically much shorter than regular phone numbers and can be used to + * address messages in MMS and SMS systems, as well as for abbreviated dialing + * (e.g. "Text 611 to see how many minutes you have remaining on your plan."). + * + * Short codes are restricted to a region and are not internationally + * dialable, which means the same short code can exist in different regions, + * with different usage and pricing, even if those regions share the same + * country calling code (e.g. US and CA). + * + * @generated from message google.type.PhoneNumber.ShortCode + */ +export type PhoneNumber_ShortCode = Message<"google.type.PhoneNumber.ShortCode"> & { + /** + * Required. The BCP-47 region code of the location where calls to this + * short code can be made, such as "US" and "BB". + * + * Reference(s): + * - http://www.unicode.org/reports/tr35/#unicode_region_subtag + * + * @generated from field: string region_code = 1; + */ + regionCode: string; + + /** + * Required. The short code digits, without a leading plus ('+') or country + * calling code, e.g. "611". + * + * @generated from field: string number = 2; + */ + number: string; +}; + +/** + * Describes the message google.type.PhoneNumber.ShortCode. + * Use `create(PhoneNumber_ShortCodeSchema)` to create a new message. + */ +export const PhoneNumber_ShortCodeSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_phone_number, 0, 0); + diff --git a/schemaregistry/google/type/postal_address_pb.ts b/schemaregistry/google/type/postal_address_pb.ts new file mode 100644 index 00000000..2bde15eb --- /dev/null +++ b/schemaregistry/google/type/postal_address_pb.ts @@ -0,0 +1,193 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/postal_address.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/postal_address.proto. + */ +export const file_google_type_postal_address: GenFile = /*@__PURE__*/ + fileDesc("CiBnb29nbGUvdHlwZS9wb3N0YWxfYWRkcmVzcy5wcm90bxILZ29vZ2xlLnR5cGUi/QEKDVBvc3RhbEFkZHJlc3MSEAoIcmV2aXNpb24YASABKAUSEwoLcmVnaW9uX2NvZGUYAiABKAkSFQoNbGFuZ3VhZ2VfY29kZRgDIAEoCRITCgtwb3N0YWxfY29kZRgEIAEoCRIUCgxzb3J0aW5nX2NvZGUYBSABKAkSGwoTYWRtaW5pc3RyYXRpdmVfYXJlYRgGIAEoCRIQCghsb2NhbGl0eRgHIAEoCRITCgtzdWJsb2NhbGl0eRgIIAEoCRIVCg1hZGRyZXNzX2xpbmVzGAkgAygJEhIKCnJlY2lwaWVudHMYCiADKAkSFAoMb3JnYW5pemF0aW9uGAsgASgJQngKD2NvbS5nb29nbGUudHlwZUISUG9zdGFsQWRkcmVzc1Byb3RvUAFaRmdvb2dsZS5nb2xhbmcub3JnL2dlbnByb3RvL2dvb2dsZWFwaXMvdHlwZS9wb3N0YWxhZGRyZXNzO3Bvc3RhbGFkZHJlc3P4AQGiAgNHVFBiBnByb3RvMw"); + +/** + * Represents a postal address, e.g. for postal delivery or payments addresses. + * Given a postal address, a postal service can deliver items to a premise, P.O. + * Box or similar. + * It is not intended to model geographical locations (roads, towns, + * mountains). + * + * In typical usage an address would be created via user input or from importing + * existing data, depending on the type of process. + * + * Advice on address input / editing: + * - Use an i18n-ready address widget such as + * https://github.com/google/libaddressinput) + * - Users should not be presented with UI elements for input or editing of + * fields outside countries where that field is used. + * + * For more guidance on how to use this schema, please see: + * https://support.google.com/business/answer/6397478 + * + * @generated from message google.type.PostalAddress + */ +export type PostalAddress = Message<"google.type.PostalAddress"> & { + /** + * The schema revision of the `PostalAddress`. This must be set to 0, which is + * the latest revision. + * + * All new revisions **must** be backward compatible with old revisions. + * + * @generated from field: int32 revision = 1; + */ + revision: number; + + /** + * Required. CLDR region code of the country/region of the address. This + * is never inferred and it is up to the user to ensure the value is + * correct. See http://cldr.unicode.org/ and + * http://www.unicode.org/cldr/charts/30/supplemental/territory_information.html + * for details. Example: "CH" for Switzerland. + * + * @generated from field: string region_code = 2; + */ + regionCode: string; + + /** + * Optional. BCP-47 language code of the contents of this address (if + * known). This is often the UI language of the input form or is expected + * to match one of the languages used in the address' country/region, or their + * transliterated equivalents. + * This can affect formatting in certain countries, but is not critical + * to the correctness of the data and will never affect any validation or + * other non-formatting related operations. + * + * If this value is not known, it should be omitted (rather than specifying a + * possibly incorrect default). + * + * Examples: "zh-Hant", "ja", "ja-Latn", "en". + * + * @generated from field: string language_code = 3; + */ + languageCode: string; + + /** + * Optional. Postal code of the address. Not all countries use or require + * postal codes to be present, but where they are used, they may trigger + * additional validation with other parts of the address (e.g. state/zip + * validation in the U.S.A.). + * + * @generated from field: string postal_code = 4; + */ + postalCode: string; + + /** + * Optional. Additional, country-specific, sorting code. This is not used + * in most regions. Where it is used, the value is either a string like + * "CEDEX", optionally followed by a number (e.g. "CEDEX 7"), or just a number + * alone, representing the "sector code" (Jamaica), "delivery area indicator" + * (Malawi) or "post office indicator" (e.g. Côte d'Ivoire). + * + * @generated from field: string sorting_code = 5; + */ + sortingCode: string; + + /** + * Optional. Highest administrative subdivision which is used for postal + * addresses of a country or region. + * For example, this can be a state, a province, an oblast, or a prefecture. + * Specifically, for Spain this is the province and not the autonomous + * community (e.g. "Barcelona" and not "Catalonia"). + * Many countries don't use an administrative area in postal addresses. E.g. + * in Switzerland this should be left unpopulated. + * + * @generated from field: string administrative_area = 6; + */ + administrativeArea: string; + + /** + * Optional. Generally refers to the city/town portion of the address. + * Examples: US city, IT comune, UK post town. + * In regions of the world where localities are not well defined or do not fit + * into this structure well, leave locality empty and use address_lines. + * + * @generated from field: string locality = 7; + */ + locality: string; + + /** + * Optional. Sublocality of the address. + * For example, this can be neighborhoods, boroughs, districts. + * + * @generated from field: string sublocality = 8; + */ + sublocality: string; + + /** + * Unstructured address lines describing the lower levels of an address. + * + * Because values in address_lines do not have type information and may + * sometimes contain multiple values in a single field (e.g. + * "Austin, TX"), it is important that the line order is clear. The order of + * address lines should be "envelope order" for the country/region of the + * address. In places where this can vary (e.g. Japan), address_language is + * used to make it explicit (e.g. "ja" for large-to-small ordering and + * "ja-Latn" or "en" for small-to-large). This way, the most specific line of + * an address can be selected based on the language. + * + * The minimum permitted structural representation of an address consists + * of a region_code with all remaining information placed in the + * address_lines. It would be possible to format such an address very + * approximately without geocoding, but no semantic reasoning could be + * made about any of the address components until it was at least + * partially resolved. + * + * Creating an address only containing a region_code and address_lines, and + * then geocoding is the recommended way to handle completely unstructured + * addresses (as opposed to guessing which parts of the address should be + * localities or administrative areas). + * + * @generated from field: repeated string address_lines = 9; + */ + addressLines: string[]; + + /** + * Optional. The recipient at the address. + * This field may, under certain circumstances, contain multiline information. + * For example, it might contain "care of" information. + * + * @generated from field: repeated string recipients = 10; + */ + recipients: string[]; + + /** + * Optional. The name of the organization at the address. + * + * @generated from field: string organization = 11; + */ + organization: string; +}; + +/** + * Describes the message google.type.PostalAddress. + * Use `create(PostalAddressSchema)` to create a new message. + */ +export const PostalAddressSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_postal_address, 0); + diff --git a/schemaregistry/google/type/quaternion_pb.ts b/schemaregistry/google/type/quaternion_pb.ts new file mode 100644 index 00000000..b0047bef --- /dev/null +++ b/schemaregistry/google/type/quaternion_pb.ts @@ -0,0 +1,125 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/quaternion.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/quaternion.proto. + */ +export const file_google_type_quaternion: GenFile = /*@__PURE__*/ + fileDesc("Chxnb29nbGUvdHlwZS9xdWF0ZXJuaW9uLnByb3RvEgtnb29nbGUudHlwZSI4CgpRdWF0ZXJuaW9uEgkKAXgYASABKAESCQoBeRgCIAEoARIJCgF6GAMgASgBEgkKAXcYBCABKAFCbwoPY29tLmdvb2dsZS50eXBlQg9RdWF0ZXJuaW9uUHJvdG9QAVpAZ29vZ2xlLmdvbGFuZy5vcmcvZ2VucHJvdG8vZ29vZ2xlYXBpcy90eXBlL3F1YXRlcm5pb247cXVhdGVybmlvbvgBAaICA0dUUGIGcHJvdG8z"); + +/** + * A quaternion is defined as the quotient of two directed lines in a + * three-dimensional space or equivalently as the quotient of two Euclidean + * vectors (https://en.wikipedia.org/wiki/Quaternion). + * + * Quaternions are often used in calculations involving three-dimensional + * rotations (https://en.wikipedia.org/wiki/Quaternions_and_spatial_rotation), + * as they provide greater mathematical robustness by avoiding the gimbal lock + * problems that can be encountered when using Euler angles + * (https://en.wikipedia.org/wiki/Gimbal_lock). + * + * Quaternions are generally represented in this form: + * + * w + xi + yj + zk + * + * where x, y, z, and w are real numbers, and i, j, and k are three imaginary + * numbers. + * + * Our naming choice `(x, y, z, w)` comes from the desire to avoid confusion for + * those interested in the geometric properties of the quaternion in the 3D + * Cartesian space. Other texts often use alternative names or subscripts, such + * as `(a, b, c, d)`, `(1, i, j, k)`, or `(0, 1, 2, 3)`, which are perhaps + * better suited for mathematical interpretations. + * + * To avoid any confusion, as well as to maintain compatibility with a large + * number of software libraries, the quaternions represented using the protocol + * buffer below *must* follow the Hamilton convention, which defines `ij = k` + * (i.e. a right-handed algebra), and therefore: + * + * i^2 = j^2 = k^2 = ijk = −1 + * ij = −ji = k + * jk = −kj = i + * ki = −ik = j + * + * Please DO NOT use this to represent quaternions that follow the JPL + * convention, or any of the other quaternion flavors out there. + * + * Definitions: + * + * - Quaternion norm (or magnitude): `sqrt(x^2 + y^2 + z^2 + w^2)`. + * - Unit (or normalized) quaternion: a quaternion whose norm is 1. + * - Pure quaternion: a quaternion whose scalar component (`w`) is 0. + * - Rotation quaternion: a unit quaternion used to represent rotation. + * - Orientation quaternion: a unit quaternion used to represent orientation. + * + * A quaternion can be normalized by dividing it by its norm. The resulting + * quaternion maintains the same direction, but has a norm of 1, i.e. it moves + * on the unit sphere. This is generally necessary for rotation and orientation + * quaternions, to avoid rounding errors: + * https://en.wikipedia.org/wiki/Rotation_formalisms_in_three_dimensions + * + * Note that `(x, y, z, w)` and `(-x, -y, -z, -w)` represent the same rotation, + * but normalization would be even more useful, e.g. for comparison purposes, if + * it would produce a unique representation. It is thus recommended that `w` be + * kept positive, which can be achieved by changing all the signs when `w` is + * negative. + * + * + * @generated from message google.type.Quaternion + */ +export type Quaternion = Message<"google.type.Quaternion"> & { + /** + * The x component. + * + * @generated from field: double x = 1; + */ + x: number; + + /** + * The y component. + * + * @generated from field: double y = 2; + */ + y: number; + + /** + * The z component. + * + * @generated from field: double z = 3; + */ + z: number; + + /** + * The scalar component. + * + * @generated from field: double w = 4; + */ + w: number; +}; + +/** + * Describes the message google.type.Quaternion. + * Use `create(QuaternionSchema)` to create a new message. + */ +export const QuaternionSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_quaternion, 0); + diff --git a/schemaregistry/google/type/timeofday_pb.ts b/schemaregistry/google/type/timeofday_pb.ts new file mode 100644 index 00000000..ddcca796 --- /dev/null +++ b/schemaregistry/google/type/timeofday_pb.ts @@ -0,0 +1,75 @@ +// Copyright 2021-2024 Buf Technologies, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file google/type/timeofday.proto (package google.type, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file google/type/timeofday.proto. + */ +export const file_google_type_timeofday: GenFile = /*@__PURE__*/ + fileDesc("Chtnb29nbGUvdHlwZS90aW1lb2ZkYXkucHJvdG8SC2dvb2dsZS50eXBlIksKCVRpbWVPZkRheRINCgVob3VycxgBIAEoBRIPCgdtaW51dGVzGAIgASgFEg8KB3NlY29uZHMYAyABKAUSDQoFbmFub3MYBCABKAVCbAoPY29tLmdvb2dsZS50eXBlQg5UaW1lT2ZEYXlQcm90b1ABWj5nb29nbGUuZ29sYW5nLm9yZy9nZW5wcm90by9nb29nbGVhcGlzL3R5cGUvdGltZW9mZGF5O3RpbWVvZmRhefgBAaICA0dUUGIGcHJvdG8z"); + +/** + * Represents a time of day. The date and time zone are either not significant + * or are specified elsewhere. An API may choose to allow leap seconds. Related + * types are [google.type.Date][google.type.Date] and + * `google.protobuf.Timestamp`. + * + * @generated from message google.type.TimeOfDay + */ +export type TimeOfDay = Message<"google.type.TimeOfDay"> & { + /** + * Hours of day in 24 hour format. Should be from 0 to 23. An API may choose + * to allow the value "24:00:00" for scenarios like business closing time. + * + * @generated from field: int32 hours = 1; + */ + hours: number; + + /** + * Minutes of hour of day. Must be from 0 to 59. + * + * @generated from field: int32 minutes = 2; + */ + minutes: number; + + /** + * Seconds of minutes of the time. Must normally be from 0 to 59. An API may + * allow the value 60 if it allows leap-seconds. + * + * @generated from field: int32 seconds = 3; + */ + seconds: number; + + /** + * Fractions of seconds in nanoseconds. Must be from 0 to 999,999,999. + * + * @generated from field: int32 nanos = 4; + */ + nanos: number; +}; + +/** + * Describes the message google.type.TimeOfDay. + * Use `create(TimeOfDaySchema)` to create a new message. + */ +export const TimeOfDaySchema: GenMessage = /*@__PURE__*/ + messageDesc(file_google_type_timeofday, 0); + diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts index 7e018983..c2d754f5 100644 --- a/schemaregistry/mock-schemaregistry-client.ts +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -1,7 +1,15 @@ -import { Client, Compatibility, SchemaInfo, SchemaMetadata, ServerConfig } from './schemaregistry-client'; +import { + Client, + Compatibility, + minimize, + SchemaInfo, + SchemaMetadata, + ServerConfig +} from './schemaregistry-client'; import stringify from "json-stringify-deterministic"; import {ClientConfig} from "./rest-service"; +import {RestError} from "./rest-error"; interface VersionCacheEntry { version: number; @@ -57,13 +65,13 @@ class MockClient implements Client { async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const metadata = await this.registerFullResponse(subject, schema, normalize); if (!metadata) { - throw new Error("Failed to register schema"); + throw new RestError("Failed to register schema", 422, 42200); } return metadata.id; } async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); const cacheEntry = this.infoToSchemaCache.get(cacheKey); if (cacheEntry && !cacheEntry.softDeleted) { @@ -72,7 +80,7 @@ class MockClient implements Client { const id = await this.getIDFromRegistry(subject, schema); if (id === -1) { - throw new Error("Failed to retrieve schema ID from registry"); + throw new RestError("Failed to retrieve schema ID from registry", 422, 42200); } const metadata: SchemaMetadata = { ...schema, id }; @@ -112,49 +120,49 @@ class MockClient implements Client { newVersion = versions[versions.length - 1] + 1; } - const cacheKey = stringify({ subject, schema: schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); this.schemaToVersionCache.set(cacheKey, { version: newVersion, softDeleted: false }); } - async getBySubjectAndId(subject: string, id: number): Promise { + async getBySubjectAndId(subject: string, id: number, format?: string): Promise { const cacheKey = stringify({ subject, id }); const cacheEntry = this.idToSchemaCache.get(cacheKey); if (!cacheEntry || cacheEntry.softDeleted) { - throw new Error("Schema not found"); + throw new RestError("Schema not found", 404, 40400); } return cacheEntry.info; } async getId(subject: string, schema: SchemaInfo): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); const cacheEntry = this.infoToSchemaCache.get(cacheKey); if (!cacheEntry || cacheEntry.softDeleted) { - throw new Error("Schema not found"); + throw new RestError("Schema not found", 404, 40400); } return cacheEntry.metadata.id; } - async getLatestSchemaMetadata(subject: string): Promise { + async getLatestSchemaMetadata(subject: string, format?: string): Promise { const version = await this.latestVersion(subject); if (version === -1) { - throw new Error("No versions found for subject"); + throw new RestError("No versions found for subject", 404, 40400); } return this.getSchemaMetadata(subject, version); } - async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { + async getSchemaMetadata(subject: string, version: number, deleted: boolean = false, format?: string): Promise { let json; for (const [key, value] of this.schemaToVersionCache.entries()) { const parsedKey = JSON.parse(key); - if (parsedKey.subject === subject && value.version === version && value.softDeleted === deleted) { + if (parsedKey.subject === subject && value.version === version) { json = parsedKey; } } if (!json) { - throw new Error("Schema not found"); + throw new RestError("Schema not found", 404, 40400); } let id: number = -1; @@ -165,7 +173,7 @@ class MockClient implements Client { } } if (id === -1) { - throw new Error("Schema not found"); + throw new RestError("Schema not found", 404, 40400); } @@ -173,11 +181,12 @@ class MockClient implements Client { id, version, subject, - schema: json.schema.schema + ...json.schema, }; } - async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { + async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, + deleted: boolean = false, format?: string): Promise { let metadataStr = ''; for (const key in metadata) { @@ -198,7 +207,7 @@ class MockClient implements Client { } if (results.length === 0) { - throw new Error("Schema not found"); + throw new RestError("Schema not found", 404, 40400); } let latest: SchemaMetadata = results[0]; @@ -225,7 +234,7 @@ class MockClient implements Client { const results = await this.allVersions(subject); if (results.length === 0) { - throw new Error("No versions found for subject"); + throw new RestError("No versions found for subject", 404, 40400); } return results; } @@ -275,11 +284,11 @@ class MockClient implements Client { } async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); const cacheEntry = this.schemaToVersionCache.get(cacheKey); if (!cacheEntry || cacheEntry.softDeleted) { - throw new Error("Schema not found"); + throw new RestError("Schema not found", 404, 40400); } return cacheEntry.version; @@ -333,7 +342,7 @@ class MockClient implements Client { if (parsedKey.subject === subject && value.version === version) { await this.deleteVersion(key, version, permanent); - const cacheKeySchema = stringify({ subject, schema: parsedKey.schema }); + const cacheKeySchema = stringify({ subject, schema: minimize(parsedKey.schema) }); const cacheEntry = this.infoToSchemaCache.get(cacheKeySchema); if (cacheEntry) { await this.deleteMetadata(cacheKeySchema, cacheEntry.metadata, permanent); @@ -363,7 +372,7 @@ class MockClient implements Client { async getCompatibility(subject: string): Promise { const cacheEntry = this.configCache.get(subject); if (!cacheEntry) { - throw new Error("Subject not found"); + throw new RestError("Subject not found", 404, 40400); } return cacheEntry.compatibilityLevel as Compatibility; } @@ -376,7 +385,7 @@ class MockClient implements Client { async getDefaultCompatibility(): Promise { const cacheEntry = this.configCache.get(noSubject); if (!cacheEntry) { - throw new Error("Default compatibility not found"); + throw new RestError("Default compatibility not found", 404, 40400); } return cacheEntry.compatibilityLevel as Compatibility; } @@ -389,7 +398,7 @@ class MockClient implements Client { async getConfig(subject: string): Promise { const cacheEntry = this.configCache.get(subject); if (!cacheEntry) { - throw new Error("Subject not found"); + throw new RestError("Subject not found", 404, 40400); } return cacheEntry; } @@ -402,7 +411,7 @@ class MockClient implements Client { async getDefaultConfig(): Promise { const cacheEntry = this.configCache.get(noSubject); if (!cacheEntry) { - throw new Error("Default config not found"); + throw new RestError("Default config not found", 404, 40400); } return cacheEntry; } diff --git a/schemaregistry/rules/encryption/awskms/aws-client.ts b/schemaregistry/rules/encryption/awskms/aws-client.ts new file mode 100644 index 00000000..c1f2f6f5 --- /dev/null +++ b/schemaregistry/rules/encryption/awskms/aws-client.ts @@ -0,0 +1,46 @@ +import {KmsClient} from "../kms-registry"; +import {AwsKmsDriver} from "./aws-driver"; +import { + DecryptCommand, + EncryptCommand, + KMSClient +} from '@aws-sdk/client-kms' +import {AwsCredentialIdentity} from "@smithy/types"; + +export class AwsKmsClient implements KmsClient { + + private kmsClient: KMSClient + private keyId: string + + constructor(keyUri: string, creds?: AwsCredentialIdentity) { + if (!keyUri.startsWith(AwsKmsDriver.PREFIX)) { + throw new Error(`key uri must start with ${AwsKmsDriver.PREFIX}`) + } + this.keyId = keyUri.substring(AwsKmsDriver.PREFIX.length) + const tokens = this.keyId.split(':') + if (tokens.length < 4) { + throw new Error(`invalid key uri ${this.keyId}`) + } + const regionName = tokens[3] + this.kmsClient = new KMSClient({ + region: regionName, + ...creds && {credentials: creds} + }) + } + + supported(keyUri: string): boolean { + return keyUri.startsWith(AwsKmsDriver.PREFIX) + } + + async encrypt(plaintext: Buffer): Promise { + const encryptCommand = new EncryptCommand({KeyId: this.keyId, Plaintext: plaintext}); + const data = await this.kmsClient.send(encryptCommand) + return Buffer.from(data.CiphertextBlob!); + } + + async decrypt(ciphertext: Buffer): Promise { + const decryptCommand = new DecryptCommand({KeyId: this.keyId, CiphertextBlob: ciphertext}); + const data = await this.kmsClient.send(decryptCommand); + return Buffer.from(data.Plaintext!) + } +} diff --git a/schemaregistry/rules/encryption/awskms/aws-driver.ts b/schemaregistry/rules/encryption/awskms/aws-driver.ts new file mode 100644 index 00000000..e01d8afa --- /dev/null +++ b/schemaregistry/rules/encryption/awskms/aws-driver.ts @@ -0,0 +1,29 @@ +import {KmsClient, KmsDriver, registerKmsDriver} from "../kms-registry"; +import {AwsKmsClient} from "./aws-client"; +import {AwsCredentialIdentity} from "@smithy/types"; + +export class AwsKmsDriver implements KmsDriver { + + static PREFIX = 'aws-kms://' + static ACCESS_KEY_ID = 'access.key.id' + static SECRET_ACCESS_KEY = 'secret.access.key' + + static register(): void { + registerKmsDriver(new AwsKmsDriver()) + } + + getKeyUrlPrefix(): string { + return AwsKmsDriver.PREFIX + } + + newKmsClient(config: Map, keyUrl?: string): KmsClient { + const uriPrefix = keyUrl != null ? keyUrl : AwsKmsDriver.PREFIX + const key = config.get(AwsKmsDriver.ACCESS_KEY_ID) + const secret = config.get(AwsKmsDriver.SECRET_ACCESS_KEY) + let creds: AwsCredentialIdentity | undefined + if (key != null && secret != null) { + creds = {accessKeyId: key, secretAccessKey: secret} + } + return new AwsKmsClient(uriPrefix, creds) + } +} diff --git a/schemaregistry/rules/encryption/azurekms/azure-client.ts b/schemaregistry/rules/encryption/azurekms/azure-client.ts new file mode 100644 index 00000000..da4c3312 --- /dev/null +++ b/schemaregistry/rules/encryption/azurekms/azure-client.ts @@ -0,0 +1,33 @@ +import {KmsClient} from "../kms-registry"; +import {AzureKmsDriver} from "./azure-driver"; +import {TokenCredential} from "@azure/identity"; +import {CryptographyClient, EncryptionAlgorithm} from "@azure/keyvault-keys"; + +export class AzureKmsClient implements KmsClient { + private static ALGORITHM: EncryptionAlgorithm = 'RSA-OAEP-256' + + private kmsClient: CryptographyClient + private keyId: string + + constructor(keyUri: string, creds: TokenCredential) { + if (!keyUri.startsWith(AzureKmsDriver.PREFIX)) { + throw new Error(`key uri must start with ${AzureKmsDriver.PREFIX}`) + } + this.keyId = keyUri.substring(AzureKmsDriver.PREFIX.length) + this.kmsClient = new CryptographyClient(this.keyId, creds) + } + + supported(keyUri: string): boolean { + return keyUri.startsWith(AzureKmsDriver.PREFIX) + } + + async encrypt(plaintext: Buffer): Promise { + const result = await this.kmsClient.encrypt(AzureKmsClient.ALGORITHM, plaintext) + return Buffer.from(result.result) + } + + async decrypt(ciphertext: Buffer): Promise { + const result = await this.kmsClient.decrypt(AzureKmsClient.ALGORITHM, ciphertext) + return Buffer.from(result.result) + } +} diff --git a/schemaregistry/rules/encryption/azurekms/azure-driver.ts b/schemaregistry/rules/encryption/azurekms/azure-driver.ts new file mode 100644 index 00000000..42d6cc84 --- /dev/null +++ b/schemaregistry/rules/encryption/azurekms/azure-driver.ts @@ -0,0 +1,33 @@ +import {KmsClient, KmsDriver, registerKmsDriver} from "../kms-registry"; +import {ClientSecretCredential, DefaultAzureCredential, TokenCredential} from '@azure/identity' +import {AzureKmsClient} from "./azure-client"; + +export class AzureKmsDriver implements KmsDriver { + + static PREFIX = 'azure-kms://' + static TENANT_ID = 'tenant_id' + static CLIENT_ID = 'client.id' + static CLIENT_SECRET = 'client.secret' + + static register(): void { + registerKmsDriver(new AzureKmsDriver()) + } + + getKeyUrlPrefix(): string { + return AzureKmsDriver.PREFIX + } + + newKmsClient(config: Map, keyUrl?: string): KmsClient { + const uriPrefix = keyUrl != null ? keyUrl : AzureKmsDriver.PREFIX + const tenantId = config.get(AzureKmsDriver.TENANT_ID) + const clientId = config.get(AzureKmsDriver.CLIENT_ID) + const clientSecret = config.get(AzureKmsDriver.CLIENT_SECRET) + let creds: TokenCredential + if (tenantId != null && clientId != null && clientSecret != null) { + creds = new ClientSecretCredential(tenantId, clientId, clientSecret) + } else { + creds = new DefaultAzureCredential() + } + return new AzureKmsClient(uriPrefix, creds) + } +} diff --git a/schemaregistry/dekregistry/constants.ts b/schemaregistry/rules/encryption/dekregistry/constants.ts similarity index 100% rename from schemaregistry/dekregistry/constants.ts rename to schemaregistry/rules/encryption/dekregistry/constants.ts diff --git a/schemaregistry/dekregistry/dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts similarity index 95% rename from schemaregistry/dekregistry/dekregistry-client.ts rename to schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts index 84216a62..00f49886 100644 --- a/schemaregistry/dekregistry/dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts @@ -1,7 +1,8 @@ import { LRUCache } from 'lru-cache'; import { Mutex } from 'async-mutex'; -import { ClientConfig, RestService } from '../rest-service'; +import { ClientConfig, RestService } from '../../../rest-service'; import stringify from 'json-stringify-deterministic'; +import {MockDekRegistryClient} from "./mock-dekregistry-client"; /* * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry @@ -76,6 +77,14 @@ class DekRegistryClient implements Client { this.dekMutex = new Mutex(); } + static newClient(config: ClientConfig): Client { + let url = config.baseURLs[0] + if (url.startsWith("mock://")) { + return new MockDekRegistryClient() + } + return new DekRegistryClient(config) + } + static getEncryptedKeyMaterialBytes(dek: Dek): Buffer | null { if (!dek.encryptedKeyMaterial) { return null; diff --git a/schemaregistry/dekregistry/mock-dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts similarity index 90% rename from schemaregistry/dekregistry/mock-dekregistry-client.ts rename to schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts index f0145275..febfbb66 100644 --- a/schemaregistry/dekregistry/mock-dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts @@ -1,6 +1,7 @@ import { Client, Dek, Kek } from "./dekregistry-client"; import { MOCK_TS } from "./constants"; import stringify from "json-stringify-deterministic"; +import {RestError} from "../../../rest-error"; class MockDekRegistryClient implements Client { private kekCache: Map; @@ -39,7 +40,7 @@ class MockDekRegistryClient implements Client { return cachedKek; } - throw new Error(`Kek not found: ${name}`); + throw new RestError(`Kek not found: ${name}`, 404, 40400); } async registerDek(kekName: string, subject: string, algorithm: string, @@ -75,18 +76,18 @@ class MockDekRegistryClient implements Client { } } if (latestVersion === 0) { - throw new Error(`Dek not found: ${subject}`); + throw new RestError(`Dek not found: ${subject}`, 404, 40400); } version = latestVersion; } - const cacheKey = stringify({ kekName, subject, version, algorithm, deleted }); + const cacheKey = stringify({ kekName, subject, version, algorithm, deleted: false }); const cachedDek = this.dekCache.get(cacheKey); if (cachedDek) { return cachedDek; } - throw new Error(`Dek not found: ${subject}`); + throw new RestError(`Dek not found: ${subject}`, 404, 40400); } async close() { diff --git a/schemaregistry/rules/encryption/encrypt-executor.ts b/schemaregistry/rules/encryption/encrypt-executor.ts index 6ba09148..cd5b7fb1 100644 --- a/schemaregistry/rules/encryption/encrypt-executor.ts +++ b/schemaregistry/rules/encryption/encrypt-executor.ts @@ -5,11 +5,11 @@ import { FieldType, MAGIC_BYTE, RuleContext, - RuleError + RuleError, } from "../../serde/serde"; import {RuleMode,} from "../../schemaregistry-client"; -import {Client, Dek, DekRegistryClient, Kek} from "../../dekregistry/dekregistry-client"; -import {registerRuleExecutor} from "../../serde/rule-registry"; +import {Client, Dek, DekRegistryClient, Kek} from "./dekregistry/dekregistry-client"; +import {RuleRegistry} from "../../serde/rule-registry"; import {ClientConfig} from "../../rest-service"; import {RestError} from "../../rest-error"; import * as Random from './tink/random'; @@ -35,7 +35,7 @@ const ENCRYPT_DEK_EXPIRY_DAYS = 'encrypt.dek.expiry.days' // MillisInDay represents number of milliseconds in a day const MILLIS_IN_DAY = 24 * 60 * 60 * 1000 -enum DekFormat { +export enum DekFormat { AES128_GCM = 'AES128_GCM', AES256_GCM = 'AES256_GCM', AES256_SIV = 'AES256_SIV', @@ -57,13 +57,14 @@ interface DekId { export class FieldEncryptionExecutor extends FieldRuleExecutor { client: Client | null = null - static register() { - registerRuleExecutor(new FieldEncryptionExecutor()) + static register(): FieldEncryptionExecutor { + const executor = new FieldEncryptionExecutor() + RuleRegistry.registerRuleExecutor(executor) + return executor } override configure(clientConfig: ClientConfig, config: Map) { - // TODO use mock - this.client = new DekRegistryClient(clientConfig) + this.client = DekRegistryClient.newClient(clientConfig) this.config = config } @@ -80,9 +81,9 @@ export class FieldEncryptionExecutor extends FieldRuleExecutor { return transform } - override close() { + async close(): Promise { if (this.client != null) { - this.client.close() + await this.client.close() } } diff --git a/schemaregistry/rules/encryption/gcpkms/gcp-client.ts b/schemaregistry/rules/encryption/gcpkms/gcp-client.ts new file mode 100644 index 00000000..4b51f4f4 --- /dev/null +++ b/schemaregistry/rules/encryption/gcpkms/gcp-client.ts @@ -0,0 +1,43 @@ +import {KmsClient} from "../kms-registry"; +import {GcpCredentials, GcpKmsDriver} from "./gcp-driver"; +import {KeyManagementServiceClient} from "@google-cloud/kms"; + +export class GcpKmsClient implements KmsClient { + + private kmsClient: KeyManagementServiceClient + private keyId: string + + constructor(keyUri: string, creds?: GcpCredentials) { + if (!keyUri.startsWith(GcpKmsDriver.PREFIX)) { + throw new Error(`key uri must start with ${GcpKmsDriver.PREFIX}`) + } + this.keyId = keyUri.substring(GcpKmsDriver.PREFIX.length) + const tokens = this.keyId.split(':') + if (tokens.length < 4) { + throw new Error(`invalid key uri ${this.keyId}`) + } + this.kmsClient = creds != null + ? new KeyManagementServiceClient() + : new KeyManagementServiceClient({credentials: creds}) + } + + supported(keyUri: string): boolean { + return keyUri.startsWith(GcpKmsDriver.PREFIX) + } + + async encrypt(plaintext: Buffer): Promise { + const [result] = await this.kmsClient.encrypt({ + name: this.keyId, + plaintext: plaintext + }) + return Buffer.from(result.ciphertext as string) + } + + async decrypt(ciphertext: Buffer): Promise { + const [result] = await this.kmsClient.decrypt({ + name: this.keyId, + ciphertext: ciphertext + }) + return Buffer.from(result.plaintext as string) + } +} diff --git a/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts b/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts new file mode 100644 index 00000000..73021678 --- /dev/null +++ b/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts @@ -0,0 +1,51 @@ +import {KmsClient, KmsDriver, registerKmsDriver} from "../kms-registry"; +import {GcpKmsClient} from "./gcp-client"; + +export class GcpKmsDriver implements KmsDriver { + + static PREFIX = 'gcp-kms://' + static ACCOUNT_TYPE = "account.type"; + static CLIENT_ID= "client.id"; + static CLIENT_EMAIL = "client.email"; + static PRIVATE_KEY_ID = "private.key.id"; + static PRIVATE_KEY= "private.key"; + + static register(): void { + registerKmsDriver(new GcpKmsDriver()) + } + + getKeyUrlPrefix(): string { + return GcpKmsDriver.PREFIX + } + + newKmsClient(config: Map, keyUrl?: string): KmsClient { + const uriPrefix = keyUrl != null ? keyUrl : GcpKmsDriver.PREFIX + let accountType = config.get(GcpKmsDriver.ACCOUNT_TYPE) + const clientId = config.get(GcpKmsDriver.CLIENT_ID) + const clientEmail = config.get(GcpKmsDriver.CLIENT_EMAIL) + const privateKeyId = config.get(GcpKmsDriver.PRIVATE_KEY_ID) + const privateKey = config.get(GcpKmsDriver.PRIVATE_KEY) + let creds: GcpCredentials | undefined + if (clientId != null && clientEmail != null && privateKeyId != null && privateKey != null) { + if (accountType == null) { + accountType = "service_account" + } + creds = { + ...accountType && {type: accountType}, + private_key_id: privateKeyId, + private_key: privateKey, + client_email: clientEmail, + client_id: clientId, + } + } + return new GcpKmsClient(uriPrefix, creds) + } +} + +export interface GcpCredentials { + type?: string + private_key_id?: string + private_key?: string + client_email?: string + client_id?: string +} diff --git a/schemaregistry/rules/encryption/hcvault/hcvault-client.ts b/schemaregistry/rules/encryption/hcvault/hcvault-client.ts new file mode 100644 index 00000000..53434b14 --- /dev/null +++ b/schemaregistry/rules/encryption/hcvault/hcvault-client.ts @@ -0,0 +1,47 @@ +import {KmsClient} from "../kms-registry"; +import {HcVaultDriver} from "./hcvault-driver"; +import NodeVault from "node-vault"; + +export class HcVaultClient implements KmsClient { + + private kmsClient: NodeVault.client + private keyId: string + private keyName: string + + constructor(keyUri: string, namespace?: string, token?: string) { + if (token == null) + { + namespace = process.env["VAULT_NAMESPACE"] + } + if (!keyUri.startsWith(HcVaultDriver.PREFIX)) { + throw new Error(`key uri must start with ${HcVaultDriver.PREFIX}`) + } + this.keyId = keyUri.substring(HcVaultDriver.PREFIX.length) + let url = new URL(this.keyId) + let parts = url.pathname.split('/') + if (parts.length === 0) { + throw new Error('key uri must contain a key name') + } + this.keyName = parts.pop()! + this.kmsClient = NodeVault({ + endpoint: url.protocol + '//' + url.host, + ...namespace && { namespace }, + ...token && { token }, + apiVersion: 'v1', + }) + } + + supported(keyUri: string): boolean { + return keyUri.startsWith(HcVaultDriver.PREFIX) + } + + async encrypt(plaintext: Buffer): Promise { + const data = await this.kmsClient.encryptData({name: this.keyName, plainText: plaintext.toString('base64') }) + return Buffer.from(data.ciphertext, 'base64') + } + + async decrypt(ciphertext: Buffer): Promise { + const data = await this.kmsClient.decryptData({name: this.keyName, cipherText: ciphertext.toString('base64') }) + return Buffer.from(data.plaintext, 'base64') + } +} diff --git a/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts b/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts new file mode 100644 index 00000000..ccdcdc90 --- /dev/null +++ b/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts @@ -0,0 +1,24 @@ +import {KmsClient, KmsDriver, registerKmsDriver} from "../kms-registry"; +import {HcVaultClient} from "./hcvault-client"; + +export class HcVaultDriver implements KmsDriver { + + static PREFIX = 'hcvault-kms://' + static TOKEN_ID = 'token.id' + static NAMESPACE = 'namespace' + + static register(): void { + registerKmsDriver(new HcVaultDriver()) + } + + getKeyUrlPrefix(): string { + return HcVaultDriver.PREFIX + } + + newKmsClient(config: Map, keyUrl?: string): KmsClient { + const uriPrefix = keyUrl != null ? keyUrl : HcVaultDriver.PREFIX + const tokenId = config.get(HcVaultDriver.TOKEN_ID) + const ns = config.get(HcVaultDriver.NAMESPACE) + return new HcVaultClient(uriPrefix, ns, tokenId) + } +} diff --git a/schemaregistry/rules/encryption/localkms/local-client.ts b/schemaregistry/rules/encryption/localkms/local-client.ts new file mode 100644 index 00000000..c65cc089 --- /dev/null +++ b/schemaregistry/rules/encryption/localkms/local-client.ts @@ -0,0 +1,44 @@ +import {KmsClient} from "../kms-registry"; +import {Cryptor, DekFormat} from "../encrypt-executor"; +import * as Hkdf from '../tink/hkdf'; +import {LocalKmsDriver} from "./local-driver"; +import {AesGcmKey, AesGcmKeySchema} from "../tink/proto/aes_gcm_pb"; +import {create, toBinary} from "@bufbuild/protobuf"; + +export class LocalKmsClient implements KmsClient { + + private secret: string + private cryptor: Cryptor + + constructor(secret?: string) { + if (secret == null) { + secret = process.env['LOCAL_SECRET'] + } + if (secret == null) { + throw new Error('cannot load secret') + } + this.secret = secret + this.cryptor = new Cryptor(DekFormat.AES128_GCM) + } + + async getKey(): Promise { + const rawKey = await Hkdf.compute(16, 'SHA-256', Buffer.from(this.secret, 'utf8'), new Uint8Array(0)); + const aesGcmKey: AesGcmKey = create(AesGcmKeySchema, { + version: 0, + keyValue: rawKey + }); + return Buffer.from(toBinary(AesGcmKeySchema, aesGcmKey)) + } + + supported(keyUri: string): boolean { + return keyUri.startsWith(LocalKmsDriver.PREFIX) + } + + async encrypt(plaintext: Buffer): Promise { + return this.cryptor.encrypt(await this.getKey(), plaintext) + } + + async decrypt(ciphertext: Buffer): Promise { + return this.cryptor.decrypt(await this.getKey(), ciphertext) + } +} diff --git a/schemaregistry/rules/encryption/localkms/local-driver.ts b/schemaregistry/rules/encryption/localkms/local-driver.ts new file mode 100644 index 00000000..96ecb481 --- /dev/null +++ b/schemaregistry/rules/encryption/localkms/local-driver.ts @@ -0,0 +1,21 @@ +import {KmsClient, KmsDriver, registerKmsDriver} from "../kms-registry"; +import {LocalKmsClient} from "./local-client"; + +export class LocalKmsDriver implements KmsDriver { + + static PREFIX: string = 'local-kms://' + static SECRET: string = 'secret' + + static register(): void { + registerKmsDriver(new LocalKmsDriver()) + } + + getKeyUrlPrefix(): string { + return LocalKmsDriver.PREFIX + } + + newKmsClient(config: Map, keyUrl: string): KmsClient { + const secret = config.get(LocalKmsDriver.SECRET) + return new LocalKmsClient(secret) + } +} diff --git a/schemaregistry/rules/encryption/tink/aes_gcm.ts b/schemaregistry/rules/encryption/tink/aes_gcm.ts index d9723abe..fcce0110 100644 --- a/schemaregistry/rules/encryption/tink/aes_gcm.ts +++ b/schemaregistry/rules/encryption/tink/aes_gcm.ts @@ -10,6 +10,7 @@ import {SecurityException} from './exception/security_exception'; import * as Bytes from './bytes'; import * as Random from './random'; import * as Validators from './validators'; +import * as crypto from 'crypto'; /** * The only supported IV size. @@ -51,7 +52,7 @@ export class AesGcm extends Aead { alg['additionalData'] = associatedData; } const ciphertext = - await self.crypto.subtle.encrypt(alg, this.key, plaintext); + await crypto.subtle.encrypt(alg, this.key, plaintext); return Bytes.concat(iv, new Uint8Array(ciphertext)); } @@ -77,7 +78,7 @@ export class AesGcm extends Aead { alg['additionalData'] = associatedData; } try { - return new Uint8Array(await self.crypto.subtle.decrypt( + return new Uint8Array(await crypto.subtle.decrypt( alg, this.key, new Uint8Array(ciphertext.subarray(IV_SIZE_IN_BYTES)))); // Preserving old behavior when moving to @@ -92,7 +93,7 @@ export class AesGcm extends Aead { export async function fromRawKey(key: Uint8Array): Promise { Validators.requireUint8Array(key); Validators.validateAesKeySize(key.length); - const webCryptoKey = await self.crypto.subtle.importKey( + const webCryptoKey = await crypto.subtle.importKey( /* format */ 'raw', key, /* keyData */ diff --git a/schemaregistry/rules/encryption/tink/aes_siv.ts b/schemaregistry/rules/encryption/tink/aes_siv.ts index 5f9521b6..5d2fc4a0 100644 --- a/schemaregistry/rules/encryption/tink/aes_siv.ts +++ b/schemaregistry/rules/encryption/tink/aes_siv.ts @@ -7,7 +7,8 @@ import {Aead} from './aead'; // @ts-expect-error miscreant does not have types -import {AEAD} from "@hackbg/miscreant-esm"; +import {SIV, WebCryptoProvider} from "@hackbg/miscreant-esm"; +import * as crypto from 'crypto'; /** * Implementation of AES-SIV. @@ -23,16 +24,16 @@ export class AesSiv extends Aead { */ async encrypt(plaintext: Uint8Array, associatedData?: Uint8Array): Promise { - let key = await AEAD.importKey(this.key, "AES-CMAC-SIV"); - return key.seal(plaintext, null, associatedData); + let key = await SIV.importKey(this.key, "AES-CMAC-SIV", new WebCryptoProvider(crypto)); + return key.seal(plaintext, [associatedData]); } /** */ async decrypt(ciphertext: Uint8Array, associatedData?: Uint8Array): Promise { - let key = await AEAD.importKey(this.key, "AES-CMAC-SIV"); - return key.open(ciphertext, null, associatedData); + let key = await SIV.importKey(this.key, "AES-CMAC-SIV", new WebCryptoProvider(crypto)); + return key.open(ciphertext, [associatedData]); } } diff --git a/schemaregistry/rules/encryption/tink/hkdf.ts b/schemaregistry/rules/encryption/tink/hkdf.ts new file mode 100644 index 00000000..5a411222 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/hkdf.ts @@ -0,0 +1,99 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + + +/** + * @fileoverview An implementation of HKDF, RFC 5869. + */ +import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; + +import {fromRawKey as hmacFromRawKey} from './hmac'; +import * as Validators from './validators'; + +/** + * Computes an HKDF. + * + * @param size The length of the generated pseudorandom string in + * bytes. The maximal size is 255 * DigestSize, where DigestSize is the size + * of the underlying HMAC. + * @param hash the name of the hash function. Accepted names are SHA-1, + * SHA-256 and SHA-512 + * @param ikm Input keying material. + * @param info Context and application specific + * information (can be a zero-length array). + * @param opt_salt Salt value (a non-secret random + * value). If not provided, it is set to a string of hash length zeros. + * @return Output keying material (okm). + */ +export async function compute( + size: number, hash: string, ikm: Uint8Array, info: Uint8Array, + opt_salt?: Uint8Array): Promise { + let digestSize; + if (!Number.isInteger(size)) { + throw new InvalidArgumentsException('size must be an integer'); + } + if (size <= 0) { + throw new InvalidArgumentsException('size must be positive'); + } + switch (hash) { + case 'SHA-1': + digestSize = 20; + if (size > 255 * 20) { + throw new InvalidArgumentsException('size too large'); + } + break; + case 'SHA-256': + digestSize = 32; + if (size > 255 * 32) { + throw new InvalidArgumentsException('size too large'); + } + break; + case 'SHA-512': + digestSize = 64; + if (size > 255 * 64) { + throw new InvalidArgumentsException('size too large'); + } + break; + default: + throw new InvalidArgumentsException(hash + ' is not supported'); + } + Validators.requireUint8Array(ikm); + Validators.requireUint8Array(info); + let salt = opt_salt; + if (opt_salt == null || salt === undefined || salt.length == 0) { + salt = new Uint8Array(digestSize); + } + Validators.requireUint8Array(salt); + + // Extract. + let hmac = await hmacFromRawKey(hash, salt, digestSize); + const prk = await hmac.computeMac( + // Pseudorandom Key + ikm); + + // Expand + hmac = await hmacFromRawKey(hash, prk, digestSize); + let ctr = 1; + let pos = 0; + let digest = new Uint8Array(0); + const result = new Uint8Array(size); + while (true) { + const input = new Uint8Array(digest.length + info.length + 1); + input.set(digest, 0); + input.set(info, digest.length); + input[input.length - 1] = ctr; + digest = await hmac.computeMac(input); + if (pos + digest.length < size) { + result.set(digest, pos); + pos += digest.length; + ctr++; + } else { + result.set(digest.subarray(0, size - pos), pos); + break; + } + } + return result; +} diff --git a/schemaregistry/rules/encryption/tink/hmac.ts b/schemaregistry/rules/encryption/tink/hmac.ts new file mode 100644 index 00000000..15045e68 --- /dev/null +++ b/schemaregistry/rules/encryption/tink/hmac.ts @@ -0,0 +1,98 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + +import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; +import {Mac} from './mac'; + +import * as Bytes from './bytes'; +import * as Validators from './validators'; +import * as crypto from 'crypto'; + +/** + * The minimum tag size. + * + */ +const MIN_TAG_SIZE_IN_BYTES: number = 10; + +/** + * Implementation of HMAC. + * + * @final + */ +export class Hmac extends Mac { + /** + * @param hash accepted names are SHA-1, SHA-256 and SHA-512 + * @param tagSize the size of the tag + */ + constructor( + private readonly hash: string, private readonly key: CryptoKey, + private readonly tagSize: number) { + super(); + } + + /** + */ + async computeMac(data: Uint8Array): Promise { + Validators.requireUint8Array(data); + const tag = await crypto.subtle.sign( + {'name': 'HMAC', 'hash': {'name': this.hash}}, this.key, data); + return new Uint8Array(tag.slice(0, this.tagSize)); + } + + /** + */ + async verifyMac(tag: Uint8Array, data: Uint8Array): Promise { + Validators.requireUint8Array(tag); + Validators.requireUint8Array(data); + const computedTag = await this.computeMac(data); + return Bytes.isEqual(tag, computedTag); + } +} + +/** + * @param hash accepted names are SHA-1, SHA-256 and SHA-512 + * @param tagSize the size of the tag + */ +export async function fromRawKey( + hash: string, key: Uint8Array, tagSize: number): Promise { + Validators.requireUint8Array(key); + if (!Number.isInteger(tagSize)) { + throw new InvalidArgumentsException('invalid tag size, must be an integer'); + } + if (tagSize < MIN_TAG_SIZE_IN_BYTES) { + throw new InvalidArgumentsException( + 'tag too short, must be at least ' + MIN_TAG_SIZE_IN_BYTES + ' bytes'); + } + switch (hash) { + case 'SHA-1': + if (tagSize > 20) { + throw new InvalidArgumentsException( + 'tag too long, must not be larger than 20 bytes'); + } + break; + case 'SHA-256': + if (tagSize > 32) { + throw new InvalidArgumentsException( + 'tag too long, must not be larger than 32 bytes'); + } + break; + case 'SHA-512': + if (tagSize > 64) { + throw new InvalidArgumentsException( + 'tag too long, must not be larger than 64 bytes'); + } + break; + default: + throw new InvalidArgumentsException(hash + ' is not supported'); + } + + // TODO(b/115974209): Add check that key.length > 16. + const cryptoKey = await crypto.subtle.importKey( + 'raw', key, + {'name': 'HMAC', 'hash': {'name': hash}, 'length': key.length * 8}, false, + ['sign', 'verify']); + return new Hmac(hash, cryptoKey, tagSize); +} diff --git a/schemaregistry/rules/encryption/tink/mac.ts b/schemaregistry/rules/encryption/tink/mac.ts new file mode 100644 index 00000000..31e422da --- /dev/null +++ b/schemaregistry/rules/encryption/tink/mac.ts @@ -0,0 +1,34 @@ +/** + * @license + * Copyright 2020 Google LLC + * SPDX-License-Identifier: Apache-2.0 + */ + + +/** + * Interface for Message Authentication Codes (MAC). + * + * Security guarantees: Message Authentication Codes provide symmetric message + * authentication. Instances implementing this interface are secure against + * existential forgery under chosen plaintext attack, and can be deterministic + * or randomized. This interface should be used for authentication only, and not + * for other purposes like generation of pseudorandom bytes. + * + */ +export abstract class Mac { + /** + * Computes message authentication code (MAC) for `data`. + * + * @param data the data to compute MAC + * @return the MAC tag + */ + abstract computeMac(data: Uint8Array): Promise; + + /** + * Verifies whether `tag` is a correct authentication code for `data`. + * + * @param tag the MAC tag + * @param data the data to compute MAC + */ + abstract verifyMac(tag: Uint8Array, data: Uint8Array): Promise; +} diff --git a/schemaregistry/rules/encryption/tink/random.ts b/schemaregistry/rules/encryption/tink/random.ts index 89315cf2..b139bc08 100644 --- a/schemaregistry/rules/encryption/tink/random.ts +++ b/schemaregistry/rules/encryption/tink/random.ts @@ -9,6 +9,7 @@ * @fileoverview Several simple wrappers of crypto.getRandomValues. */ import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; +import * as crypto from 'crypto'; /** * Randomly generates `n` bytes. diff --git a/schemaregistry/rules/jsonata/jsonata-executor.ts b/schemaregistry/rules/jsonata/jsonata-executor.ts new file mode 100644 index 00000000..cd9209dd --- /dev/null +++ b/schemaregistry/rules/jsonata/jsonata-executor.ts @@ -0,0 +1,40 @@ +import {RuleRegistry} from "../../serde/rule-registry"; +import {RuleContext, RuleExecutor} from "../../serde/serde"; +import {ClientConfig} from "../../rest-service"; +import {LRUCache} from "lru-cache"; +import jsonata, {Expression} from "jsonata"; + +export class JsonataExecutor implements RuleExecutor { + config: Map | null = null + cache: LRUCache = new LRUCache({max: 1000}) + + static register(): JsonataExecutor { + const executor = new JsonataExecutor() + RuleRegistry.registerRuleExecutor(executor) + return executor + } + + configure(clientConfig: ClientConfig, config: Map) { + this.config = config + } + + type(): string { + return "JSONATA" + } + + async transform(ctx: RuleContext, msg: any): Promise { + let expr = ctx.rule.expr + if (expr == null) { + return msg + } + let jsonataExpr = this.cache.get(expr) + if (jsonataExpr == null) { + jsonataExpr = jsonata(expr) + this.cache.set(expr, jsonataExpr) + } + return jsonataExpr.evaluate(msg) + } + + async close(): Promise { + } +} diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index 2e0145a5..89d64b39 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -3,6 +3,7 @@ import { AxiosResponse } from 'axios'; import stringify from "json-stringify-deterministic"; import { LRUCache } from 'lru-cache'; import { Mutex } from 'async-mutex'; +import {MockClient} from "./mock-schemaregistry-client"; /* * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry @@ -34,7 +35,7 @@ export interface Rule { kind?: string mode?: RuleMode type: string - tags?: Set + tags?: string[] params?: { [key: string]: string } expr?: string onSuccess?: string @@ -59,6 +60,17 @@ export interface SchemaInfo { ruleSet?: RuleSet; } +// Ensure that SchemaMetadata fields are removed +export function minimize(info: SchemaInfo): SchemaInfo { + return { + schemaType: info.schemaType, + schema: info.schema, + references: info.references, + metadata: info.metadata, + ruleSet: info.ruleSet + } +} + export interface SchemaMetadata extends SchemaInfo { id: number; subject?: string; @@ -72,9 +84,9 @@ export interface Reference { } export interface Metadata { - tags?: { [key: string]: Set }; + tags?: { [key: string]: string[] }; properties?: { [key: string]: string }; - sensitive?: Set; + sensitive?: string[]; } export interface RuleSet { @@ -102,11 +114,12 @@ export interface Client { config(): ClientConfig; register(subject: string, schema: SchemaInfo, normalize: boolean): Promise; registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean): Promise; - getBySubjectAndId(subject: string, id: number): Promise; + getBySubjectAndId(subject: string, id: number, format?: string): Promise; getId(subject: string, schema: SchemaInfo, normalize: boolean): Promise; - getLatestSchemaMetadata(subject: string): Promise; - getSchemaMetadata(subject: string, version: number, deleted: boolean): Promise; - getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean): Promise; + getLatestSchemaMetadata(subject: string, format?: string): Promise; + getSchemaMetadata(subject: string, version: number, deleted: boolean, format?: string): Promise; + getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, + deleted: boolean, format?: string): Promise; getAllVersions(subject: string): Promise; getVersion(subject: string, schema: SchemaInfo, normalize: boolean): Promise; getAllSubjects(): Promise; @@ -170,6 +183,14 @@ export class SchemaRegistryClient implements Client { this.metadataToSchemaMutex = new Mutex(); } + static newClient(config: ClientConfig): Client { + let url = config.baseURLs[0] + if (url.startsWith("mock://")) { + return new MockClient(config) + } + return new SchemaRegistryClient(config) + } + config(): ClientConfig { return this.clientConfig } @@ -181,7 +202,7 @@ export class SchemaRegistryClient implements Client { } async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); return await this.infoToSchemaMutex.runExclusive(async () => { const cachedSchemaMetadata: SchemaMetadata | undefined = this.infoToSchemaCache.get(cacheKey); @@ -201,7 +222,7 @@ export class SchemaRegistryClient implements Client { }); } - async getBySubjectAndId(subject: string, id: number): Promise { + async getBySubjectAndId(subject: string, id: number, format?: string): Promise { const cacheKey = stringify({ subject, id }); return await this.idToSchemaInfoMutex.runExclusive(async () => { const cachedSchema: SchemaInfo | undefined = this.idToSchemaInfoCache.get(cacheKey); @@ -211,8 +232,10 @@ export class SchemaRegistryClient implements Client { subject = encodeURIComponent(subject); + let formatStr = format != null ? `&format=${format}` : ''; + const response: AxiosResponse = await this.restService.handleRequest( - `/schemas/ids/${id}?subject=${subject}`, + `/schemas/ids/${id}?subject=${subject}${formatStr}`, 'GET' ); this.idToSchemaInfoCache.set(cacheKey, response.data); @@ -221,7 +244,7 @@ export class SchemaRegistryClient implements Client { } async getId(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); return await this.schemaToIdMutex.runExclusive(async () => { const cachedId: number | undefined = this.schemaToIdCache.get(cacheKey); @@ -241,7 +264,7 @@ export class SchemaRegistryClient implements Client { }); } - async getLatestSchemaMetadata(subject: string): Promise { + async getLatestSchemaMetadata(subject: string, format?: string): Promise { return await this.latestToSchemaMutex.runExclusive(async () => { const cachedSchema: SchemaMetadata | undefined = this.latestToSchemaCache.get(subject); if (cachedSchema) { @@ -250,8 +273,10 @@ export class SchemaRegistryClient implements Client { subject = encodeURIComponent(subject); + let formatStr = format != null ? `?format=${format}` : ''; + const response: AxiosResponse = await this.restService.handleRequest( - `/subjects/${subject}/versions/latest`, + `/subjects/${subject}/versions/latest${formatStr}`, 'GET' ); this.latestToSchemaCache.set(subject, response.data); @@ -259,7 +284,7 @@ export class SchemaRegistryClient implements Client { }); } - async getSchemaMetadata(subject: string, version: number, deleted: boolean = false): Promise { + async getSchemaMetadata(subject: string, version: number, deleted: boolean = false, format?: string): Promise { const cacheKey = stringify({ subject, version, deleted }); return await this.versionToSchemaMutex.runExclusive(async () => { @@ -270,8 +295,10 @@ export class SchemaRegistryClient implements Client { subject = encodeURIComponent(subject); + let formatStr = format != null ? `&format=${format}` : ''; + const response: AxiosResponse = await this.restService.handleRequest( - `/subjects/${subject}/versions/${version}?deleted=${deleted}`, + `/subjects/${subject}/versions/${version}?deleted=${deleted}${formatStr}`, 'GET' ); this.versionToSchemaCache.set(cacheKey, response.data); @@ -279,7 +306,8 @@ export class SchemaRegistryClient implements Client { }); } - async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false): Promise { + async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, + deleted: boolean = false, format?: string): Promise { const cacheKey = stringify({ subject, metadata, deleted }); return await this.metadataToSchemaMutex.runExclusive(async () => { @@ -298,8 +326,10 @@ export class SchemaRegistryClient implements Client { metadataStr += `&key=${encodedKey}&value=${encodedValue}`; } + let formatStr = format != null ? `&format=${format}` : ''; + const response: AxiosResponse = await this.restService.handleRequest( - `/subjects/${subject}/metadata?deleted=${deleted}&${metadataStr}`, + `/subjects/${subject}/metadata?deleted=${deleted}&${metadataStr}${formatStr}`, 'GET' ); this.metadataToSchemaCache.set(cacheKey, response.data); @@ -317,7 +347,7 @@ export class SchemaRegistryClient implements Client { } async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); return await this.schemaToVersionMutex.runExclusive(async () => { const cachedVersion: number | undefined = this.schemaToVersionCache.get(cacheKey); @@ -399,7 +429,7 @@ export class SchemaRegistryClient implements Client { const parsedKey = JSON.parse(key); if (parsedKey.subject === subject && value === version) { this.schemaToVersionCache.delete(key); - const infoToSchemaCacheKey = stringify({ subject: subject, schema: parsedKey.schema }); + const infoToSchemaCacheKey = stringify({ subject: subject, schema: minimize(parsedKey.schema) }); this.infoToSchemaMutex.runExclusive(async () => { metadataValue = this.infoToSchemaCache.get(infoToSchemaCacheKey); @@ -537,14 +567,14 @@ export class SchemaRegistryClient implements Client { // Cache methods for testing async addToInfoToSchemaCache(subject: string, schema: SchemaInfo, metadata: SchemaMetadata): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); await this.infoToSchemaMutex.runExclusive(async () => { this.infoToSchemaCache.set(cacheKey, metadata); }); } async addToSchemaToVersionCache(subject: string, schema: SchemaInfo, version: number): Promise { - const cacheKey = stringify({ subject, schema }); + const cacheKey = stringify({ subject, schema: minimize(schema) }); await this.schemaToVersionMutex.runExclusive(async () => { this.schemaToVersionCache.set(cacheKey, version); }); diff --git a/schemaregistry/serde/avro.ts b/schemaregistry/serde/avro.ts index d1590607..5444e938 100644 --- a/schemaregistry/serde/avro.ts +++ b/schemaregistry/serde/avro.ts @@ -10,7 +10,7 @@ import { Client, RuleMode, SchemaInfo } from "../schemaregistry-client"; -import avro, { ForSchemaOptions, Type, types } from "avsc"; +import avro, {ForSchemaOptions, Type, types} from "avsc"; import UnwrappedUnionType = types.UnwrappedUnionType import WrappedUnionType = types.WrappedUnionType import ArrayType = types.ArrayType @@ -18,7 +18,7 @@ import MapType = types.MapType import RecordType = types.RecordType import Field = types.Field import { LRUCache } from 'lru-cache' -import {getRuleExecutors} from "./rule-registry"; +import {RuleRegistry} from "./rule-registry"; import stringify from "json-stringify-deterministic"; type TypeHook = (schema: avro.Schema, opts: ForSchemaOptions) => Type | undefined @@ -26,22 +26,22 @@ type TypeHook = (schema: avro.Schema, opts: ForSchemaOptions) => Type | undefine export type AvroSerdeConfig = Partial export interface AvroSerde { - schemaToTypeCache: LRUCache + schemaToTypeCache: LRUCache]> } export type AvroSerializerConfig = SerializerConfig & AvroSerdeConfig export class AvroSerializer extends Serializer implements AvroSerde { - schemaToTypeCache: LRUCache + schemaToTypeCache: LRUCache]> - constructor(client: Client, serdeType: SerdeType, conf: AvroSerializerConfig) { - super(client, serdeType, conf) - this.schemaToTypeCache = new LRUCache({ max: this.conf.cacheCapacity ?? 1000 }) + constructor(client: Client, serdeType: SerdeType, conf: AvroSerializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) + this.schemaToTypeCache = new LRUCache]>({ max: this.conf.cacheCapacity ?? 1000 }) this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { return await this.fieldTransform(ctx, fieldTransform, msg) } - for (const rule of getRuleExecutors()) { - rule.configure(client.config(), conf.ruleConfig ?? new Map) + for (const rule of this.ruleRegistry.getExecutors()) { + rule.configure(client.config(), new Map(Object.entries(conf.ruleConfig ?? {}))) } } @@ -53,25 +53,51 @@ export class AvroSerializer extends Serializer implements AvroSerde { throw new Error('message is empty') } - let avroSchema = Type.forValue(msg) + let enumIndex = 1 + let fixedIndex = 1 + let recordIndex = 1 + + const namingHook: TypeHook = ( + avroSchema: avro.Schema, + opts: ForSchemaOptions, + ) => { + let schema = avroSchema as any + switch (schema.type) { + case 'enum': + schema.name = `Enum${enumIndex++}`; + break; + case 'fixed': + schema.name = `Fixed${fixedIndex++}`; + break; + case 'record': + schema.name = `Record${recordIndex++}`; + break; + default: + } + return undefined + } + + let avroSchema = Type.forValue(msg, { typeHook: namingHook }) const schema: SchemaInfo = { schemaType: 'AVRO', schema: JSON.stringify(avroSchema), } const [id, info] = await this.getId(topic, msg, schema) - avroSchema = await this.toType(info) + let deps: Map + [avroSchema, deps] = await this.toType(info) const subject = this.subjectName(topic, info) - msg = await this.executeRules(subject, topic, RuleMode.WRITE, null, info, msg, getInlineTags(avroSchema)) + msg = await this.executeRules( + subject, topic, RuleMode.WRITE, null, info, msg, getInlineTags(info, deps)) const msgBytes = avroSchema.toBuffer(msg) return this.writeBytes(id, msgBytes) } async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { - const schema = await this.toType(ctx.target) + const [schema, ] = await this.toType(ctx.target) return await transform(ctx, schema, msg, fieldTransform) } - async toType(info: SchemaInfo): Promise { + async toType(info: SchemaInfo): Promise<[Type, Map]> { return toType(this.client, this.conf as AvroDeserializerConfig, this, info, async (client, info) => { const deps = new Map() await this.resolveReferences(client, info, deps) @@ -83,16 +109,16 @@ export class AvroSerializer extends Serializer implements AvroSerde { export type AvroDeserializerConfig = DeserializerConfig & AvroSerdeConfig export class AvroDeserializer extends Deserializer implements AvroSerde { - schemaToTypeCache: LRUCache + schemaToTypeCache: LRUCache]> - constructor(client: Client, serdeType: SerdeType, conf: AvroDeserializerConfig) { - super(client, serdeType, conf) - this.schemaToTypeCache = new LRUCache({ max: this.conf.cacheCapacity ?? 1000 }) + constructor(client: Client, serdeType: SerdeType, conf: AvroDeserializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) + this.schemaToTypeCache = new LRUCache]>({ max: this.conf.cacheCapacity ?? 1000 }) this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { return await this.fieldTransform(ctx, fieldTransform, msg) } - for (const rule of getRuleExecutors()) { - rule.configure(client.config(), conf.ruleConfig ?? new Map) + for (const rule of this.ruleRegistry.getExecutors()) { + rule.configure(client.config(), new Map(Object.entries(conf.ruleConfig ?? {}))) } } @@ -111,7 +137,7 @@ export class AvroDeserializer extends Deserializer implements AvroSerde { if (readerMeta != null) { migrations = await this.getMigrations(subject, info, readerMeta) } - const writer = await this.toType(info) + const [writer, deps] = await this.toType(info) let msg: any const msgBytes = payload.subarray(5) @@ -120,7 +146,7 @@ export class AvroDeserializer extends Deserializer implements AvroSerde { msg = await this.executeMigrations(migrations, subject, topic, msg) } else { if (readerMeta != null) { - const reader = await this.toType(readerMeta) + const [reader, ] = await this.toType(readerMeta) if (reader.equals(writer)) { msg = reader.fromBuffer(msgBytes) } else { @@ -136,16 +162,17 @@ export class AvroDeserializer extends Deserializer implements AvroSerde { } else { target = info } - msg = await this.executeRules(subject, topic, RuleMode.READ, null, target, msg, getInlineTags(writer)) + msg = await this.executeRules( + subject, topic, RuleMode.READ, null, target, msg, getInlineTags(info, deps)) return msg } async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { - const schema = await this.toType(ctx.target) + const [schema, ] = await this.toType(ctx.target) return await transform(ctx, schema, msg, fieldTransform) } - async toType(info: SchemaInfo): Promise { + async toType(info: SchemaInfo): Promise<[Type, Map]> { return toType(this.client, this.conf as AvroDeserializerConfig, this, info, async (client, info) => { const deps = new Map() await this.resolveReferences(client, info, deps) @@ -160,10 +187,10 @@ async function toType( serde: AvroSerde, info: SchemaInfo, refResolver: RefResolver, -): Promise { - let type = serde.schemaToTypeCache.get(stringify(info.schema)) - if (type != null) { - return type +): Promise<[Type, Map]> { + let tuple = serde.schemaToTypeCache.get(stringify(info.schema)) + if (tuple != null) { + return tuple } const deps = await refResolver(client, info) @@ -172,8 +199,10 @@ async function toType( schema: avro.Schema, opts: ForSchemaOptions, ) => { - deps.forEach((_name, schema) => { - avro.Type.forSchema(JSON.parse(schema), opts) + const avroOpts = opts as AvroSerdeConfig + deps.forEach((schema, _name) => { + avroOpts.typeHook = userHook + avro.Type.forSchema(JSON.parse(schema), avroOpts) }) if (userHook) { return userHook(schema, opts) @@ -182,12 +211,12 @@ async function toType( } const avroOpts = conf - type = avro.Type.forSchema(JSON.parse(info.schema), { + let type = avro.Type.forSchema(JSON.parse(info.schema), { ...avroOpts, typeHook: addReferencedSchemas(avroOpts?.typeHook), }) - serde.schemaToTypeCache.set(stringify(info.schema), type) - return type + serde.schemaToTypeCache.set(stringify(info.schema), [type, deps]) + return [type, deps] } async function transform(ctx: RuleContext, schema: Type, msg: any, fieldTransform: FieldTransform): Promise { @@ -226,8 +255,8 @@ async function transform(ctx: RuleContext, schema: Type, msg: any, fieldTransfor return record default: if (fieldCtx != null) { - const ruleTags = ctx.rule.tags - if (ruleTags == null || ruleTags.size === 0 || !disjoint(ruleTags, fieldCtx.tags)) { + const ruleTags = ctx.rule.tags ?? [] + if (ruleTags == null || ruleTags.length === 0 || !disjoint(new Set(ruleTags), fieldCtx.tags)) { return await fieldTransform.transform(ctx, fieldCtx, msg) } } @@ -246,7 +275,7 @@ async function transformField( const fullName = recordSchema.name + '.' + field.name try { ctx.enterField( - val.Interface(), + val, fullName, field.name, getType(field.type), @@ -330,9 +359,12 @@ function resolveUnion(schema: Type, msg: any): Type | null { return null } -function getInlineTags(schema: object): Map> { +function getInlineTags(info: SchemaInfo, deps: Map): Map> { const inlineTags = new Map>() - getInlineTagsRecursively('', '', schema, inlineTags) + getInlineTagsRecursively('', '', JSON.parse(info.schema), inlineTags) + for (const depSchema of Object.values(deps)) { + getInlineTagsRecursively('', '', JSON.parse(depSchema), inlineTags) + } return inlineTags } @@ -378,3 +410,5 @@ function impliedNamespace(name: string): string | null { const match = /^(.*)\.[^.]+$/.exec(name) return match ? match[1] : null } + + diff --git a/schemaregistry/serde/json_util.ts b/schemaregistry/serde/json-util.ts similarity index 100% rename from schemaregistry/serde/json_util.ts rename to schemaregistry/serde/json-util.ts diff --git a/schemaregistry/serde/json.ts b/schemaregistry/serde/json.ts index 7c53d1db..a40632d9 100644 --- a/schemaregistry/serde/json.ts +++ b/schemaregistry/serde/json.ts @@ -27,8 +27,8 @@ import { } from '@criteria/json-schema/draft-07' import { validateJSON } from '@criteria/json-schema-validation' import { LRUCache } from "lru-cache"; -import { generateSchema } from "./json_util"; -import {getRuleExecutors} from "./rule-registry"; +import { generateSchema } from "./json-util"; +import {RuleRegistry} from "./rule-registry"; import stringify from "json-stringify-deterministic"; export interface ValidateFunction { @@ -53,15 +53,15 @@ export class JsonSerializer extends Serializer implements JsonSerde { schemaToTypeCache: LRUCache schemaToValidateCache: LRUCache - constructor(client: Client, serdeType: SerdeType, conf: JsonSerializerConfig) { - super(client, serdeType, conf) + constructor(client: Client, serdeType: SerdeType, conf: JsonSerializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) this.schemaToTypeCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) this.schemaToValidateCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { return await this.fieldTransform(ctx, fieldTransform, msg) } - for (const rule of getRuleExecutors()) { - rule.configure(client.config(), conf.ruleConfig ?? new Map) + for (const rule of this.ruleRegistry.getExecutors()) { + rule.configure(client.config(), new Map(Object.entries(conf.ruleConfig ?? {}))) } } @@ -123,15 +123,15 @@ export class JsonDeserializer extends Deserializer implements JsonSerde { schemaToTypeCache: LRUCache schemaToValidateCache: LRUCache - constructor(client: Client, serdeType: SerdeType, conf: JsonDeserializerConfig) { - super(client, serdeType, conf) + constructor(client: Client, serdeType: SerdeType, conf: JsonDeserializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) this.schemaToTypeCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) this.schemaToValidateCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { return await this.fieldTransform(ctx, fieldTransform, msg) } - for (const rule of getRuleExecutors()) { - rule.configure(client.config(), conf.ruleConfig ?? new Map) + for (const rule of this.ruleRegistry.getExecutors()) { + rule.configure(client.config(), new Map(Object.entries(conf.ruleConfig ?? {}))) } } @@ -213,7 +213,7 @@ async function toValidateFunction( const spec = json.$schema if (spec === 'http://json-schema.org/draft/2020-12/schema') { const ajv2020 = new Ajv2020(conf as JsonSerdeConfig) - deps.forEach((name, schema) => { + deps.forEach((schema, name) => { ajv2020.addSchema(JSON.parse(schema), name) }) fn = ajv2020.compile(json) @@ -221,7 +221,7 @@ async function toValidateFunction( const ajv = new Ajv2019(conf as JsonSerdeConfig) ajv.addMetaSchema(draft6MetaSchema) ajv.addMetaSchema(draft7MetaSchema) - deps.forEach((name, schema) => { + deps.forEach((schema, name) => { ajv.addSchema(JSON.parse(schema), name) }) fn = ajv.compile(json) @@ -317,7 +317,7 @@ async function transform(ctx: RuleContext, schema: DereferencedJSONSchema, path: case FieldType.BOOLEAN: if (fieldCtx != null) { const ruleTags = ctx.rule.tags - if (ruleTags == null || ruleTags.size === 0 || !disjoint(ruleTags, fieldCtx.tags)) { + if (ruleTags == null || ruleTags.length === 0 || !disjoint(new Set(ruleTags), fieldCtx.tags)) { return await fieldTransform.transform(ctx, fieldCtx, msg) } } diff --git a/schemaregistry/serde/protobuf.ts b/schemaregistry/serde/protobuf.ts index 363a322c..cfa59ce7 100644 --- a/schemaregistry/serde/protobuf.ts +++ b/schemaregistry/serde/protobuf.ts @@ -14,47 +14,99 @@ import { SchemaMetadata } from "../schemaregistry-client"; import { - createFileRegistry, + createFileRegistry, createMutableRegistry, DescField, DescFile, DescMessage, FileRegistry, - fromBinary, getExtension, hasExtension, - Registry, + fromBinary, getExtension, hasExtension, MutableRegistry, ScalarType, - toBinary + toBinary, } from "@bufbuild/protobuf"; -import { FileDescriptorProtoSchema } from "@bufbuild/protobuf/wkt"; +import { + file_google_protobuf_any, + file_google_protobuf_api, + file_google_protobuf_descriptor, + file_google_protobuf_duration, + file_google_protobuf_empty, + file_google_protobuf_field_mask, + file_google_protobuf_source_context, + file_google_protobuf_struct, + file_google_protobuf_timestamp, file_google_protobuf_type, file_google_protobuf_wrappers, + FileDescriptorProtoSchema +} from "@bufbuild/protobuf/wkt"; import { BufferWrapper, MAX_VARINT_LEN_64 } from "./buffer-wrapper"; import { LRUCache } from "lru-cache"; -import {field_meta, Meta} from "../confluent/meta_pb"; -import {getRuleExecutors} from "./rule-registry"; +import {field_meta, file_confluent_meta, Meta} from "../confluent/meta_pb"; +import {RuleRegistry} from "./rule-registry"; import stringify from "json-stringify-deterministic"; +import {file_confluent_types_decimal} from "../confluent/types/decimal_pb"; +import {file_google_type_calendar_period} from "../google/type/calendar_period_pb"; +import {file_google_type_color} from "../google/type/color_pb"; +import {file_google_type_date} from "../google/type/date_pb"; +import {file_google_type_datetime} from "../google/type/datetime_pb"; +import {file_google_type_dayofweek} from "../google/type/dayofweek_pb"; +import {file_google_type_fraction} from "../google/type/fraction_pb"; +import {file_google_type_expr} from "../google/type/expr_pb"; +import {file_google_type_latlng} from "../google/type/latlng_pb"; +import {file_google_type_money} from "../google/type/money_pb"; +import {file_google_type_postal_address} from "../google/type/postal_address_pb"; +import {file_google_type_quaternion} from "../google/type/quaternion_pb"; +import {file_google_type_timeofday} from "../google/type/timeofday_pb"; +import {file_google_type_month} from "../google/type/month_pb"; + +const builtinDeps = new Map([ + ['confluent/meta.proto', file_confluent_meta], + ['confluent/type/decimal.proto', file_confluent_types_decimal], + ['google/type/calendar_period.proto', file_google_type_calendar_period], + ['google/type/color.proto', file_google_type_color], + ['google/type/date.proto', file_google_type_date], + ['google/type/datetime.proto', file_google_type_datetime], + ['google/type/dayofweek.proto', file_google_type_dayofweek], + ['google/type/expr.proto', file_google_type_expr], + ['google/type/fraction.proto', file_google_type_fraction], + ['google/type/latlng.proto', file_google_type_latlng], + ['google/type/money.proto', file_google_type_money], + ['google/type/month.proto', file_google_type_month], + ['google/type/postal_address.proto', file_google_type_postal_address], + ['google/type/quaternion.proto', file_google_type_quaternion], + ['google/type/timeofday.proto', file_google_type_timeofday], + ['google/protobuf/any.proto', file_google_protobuf_any], + ['google/protobuf/api.proto', file_google_protobuf_api], + ['google/protobuf/descriptor.proto', file_google_protobuf_descriptor], + ['google/protobuf/duration.proto', file_google_protobuf_duration], + ['google/protobuf/empty.proto', file_google_protobuf_empty], + ['google/protobuf/field_mask.proto', file_google_protobuf_field_mask], + ['google/protobuf/source_context.proto', file_google_protobuf_source_context], + ['google/protobuf/struct.proto', file_google_protobuf_struct], + ['google/protobuf/timestamp.proto', file_google_protobuf_timestamp], + ['google/protobuf/type.proto', file_google_protobuf_type], + ['google/protobuf/wrappers.proto', file_google_protobuf_wrappers], +]) export interface ProtobufSerde { schemaToDescCache: LRUCache } export type ProtobufSerializerConfig = SerializerConfig & { - registry: Registry - descToSchemaCache: LRUCache + registry?: MutableRegistry } export class ProtobufSerializer extends Serializer implements ProtobufSerde { - registry: Registry + registry: MutableRegistry schemaToDescCache: LRUCache descToSchemaCache: LRUCache - constructor(client: Client, serdeType: SerdeType, conf: ProtobufSerializerConfig) { - super(client, serdeType, conf) - this.registry = conf.registry + constructor(client: Client, serdeType: SerdeType, conf: ProtobufSerializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) + this.registry = conf.registry ?? createMutableRegistry() this.schemaToDescCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 } ) this.descToSchemaCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 } ) this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { return await this.fieldTransform(ctx, fieldTransform, msg) } - for (const rule of getRuleExecutors()) { - rule.configure(client.config(), conf.ruleConfig ?? new Map) + for (const rule of this.ruleRegistry.getExecutors()) { + rule.configure(client.config(), new Map(Object.entries(conf.ruleConfig ?? {}))) } } @@ -76,7 +128,7 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { } const fileDesc = messageDesc.file const schema = await this.getSchemaInfo(fileDesc) - const [id, info] = await this.getId(topic, msg, schema) + const [id, info] = await this.getId(topic, msg, schema, 'serialized') const subject = this.subjectName(topic, info) msg = await this.executeRules(subject, topic, RuleMode.WRITE, null, info, msg, null) const msgIndexBytes = this.toMessageIndexBytes(messageDesc) @@ -114,7 +166,7 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { toDependencies(fileDesc: DescFile, deps: Map) { deps.set(fileDesc.name, Buffer.from(toBinary(FileDescriptorProtoSchema, fileDesc.proto)).toString('base64')) fileDesc.dependencies.forEach((dep) => { - if (!this.ignoreFile(dep.name)) { + if (!isBuiltin(dep.name)) { this.toDependencies(dep, deps) } }) @@ -123,17 +175,17 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { async resolveDependencies(fileDesc: DescFile, deps: Map, subject: string, autoRegister: boolean, normalize: boolean): Promise { const refs: Reference[] = [] - refs.length = fileDesc.dependencies.length for (let i = 0; i < fileDesc.dependencies.length; i++) { const dep = fileDesc.dependencies[i] - if (this.ignoreFile(dep.name)) { + const depName = dep.name + '.proto' + if (isBuiltin(depName)) { continue } - const ref = await this.resolveDependencies(dep, deps, dep.name, autoRegister, normalize) + const ref = await this.resolveDependencies(dep, deps, depName, autoRegister, normalize) if (ref == null) { throw new SerializationError('dependency not found') } - refs.push({name: dep.name, subject: ref.subject!, version: ref.version!}) + refs.push({name: depName, subject: ref.subject!, version: ref.version!}) } const info: SchemaInfo = { schema: deps.get(fileDesc.name)!, @@ -182,8 +234,7 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { // done. Allocate an array large enough to hold count+1 entries and // populate first value with index const msgIndexes: number[] = [] - msgIndexes.length = count + 1 - msgIndexes[0] = index + msgIndexes.push(index) return msgIndexes } else { const msgIndexes = this.toMessageIndexes(parent, count + 1) @@ -211,12 +262,6 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { throw new SerializationError('message descriptor not found in file descriptor'); } - ignoreFile(name: string): boolean { - return name.startsWith('confluent/') || - name.startsWith('google/protobuf/') || - name.startsWith('google/type/') - } - async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { const typeName = msg.$typeName if (typeName == null) { @@ -236,15 +281,15 @@ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde registry: FileRegistry schemaToDescCache: LRUCache - constructor(client: Client, serdeType: SerdeType, conf: ProtobufDeserializerConfig) { - super(client, serdeType, conf) + constructor(client: Client, serdeType: SerdeType, conf: ProtobufDeserializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) this.registry = createFileRegistry() this.schemaToDescCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 } ) this.fieldTransformer = async (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => { return await this.fieldTransform(ctx, fieldTransform, msg) } - for (const rule of getRuleExecutors()) { - rule.configure(client.config(), conf.ruleConfig ?? new Map) + for (const rule of this.ruleRegistry.getExecutors()) { + rule.configure(client.config(), new Map(Object.entries(conf.ruleConfig ?? {}))) } } @@ -256,13 +301,13 @@ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde return null } - const info = await this.getSchema(topic, payload) + const info = await this.getSchema(topic, payload, 'serialized') const fd = await this.toFileDesc(this.client, info) const [bytesRead, msgIndexes] = this.readMessageIndexes(payload.subarray(5)) const messageDesc = this.toMessageDesc(fd, msgIndexes) const subject = this.subjectName(topic, info) - const readerMeta = await this.getReaderSchema(subject) + const readerMeta = await this.getReaderSchema(subject, 'serialized') const msgBytes = payload.subarray(5 + bytesRead) let msg = fromBinary(messageDesc, msgBytes) @@ -306,16 +351,25 @@ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde async parseFileDesc(client: Client, info: SchemaInfo): Promise { const deps = new Map() - await this.resolveReferences(client, info, deps) + await this.resolveReferences(client, info, deps, 'serialized') const fileDesc = fromBinary(FileDescriptorProtoSchema, Buffer.from(info.schema, 'base64')) const resolve = (depName: string) => { - const dep = deps.get(depName) - if (dep == null) { - throw new SerializationError('dependency not found') + if (isBuiltin(depName)) { + const dep = builtinDeps.get(depName) + if (dep == null) { + throw new SerializationError(`dependency ${depName} not found`) + } + return dep + } else { + const dep = deps.get(depName) + if (dep == null) { + throw new SerializationError(`dependency ${depName} not found`) + } + const fileDesc = fromBinary(FileDescriptorProtoSchema, Buffer.from(dep, 'base64')) + fileDesc.name = depName + return fileDesc } - return fromBinary(FileDescriptorProtoSchema, Buffer.from(dep, 'base64')) } - // TODO check google protos already in registry const fileRegistry = createFileRegistry(fileDesc, resolve) this.registry = createFileRegistry(this.registry, fileRegistry) return this.registry.getFile(fileDesc.name) @@ -325,9 +379,8 @@ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde const bw = new BufferWrapper(payload) const count = bw.readVarInt() const msgIndexes = [] - msgIndexes.length = count for (let i = 0; i < count; i++) { - msgIndexes[i] = bw.readVarInt() + msgIndexes.push(bw.readVarInt()) } return [bw.pos, msgIndexes] } @@ -372,8 +425,8 @@ async function transform(ctx: RuleContext, descriptor: DescMessage, msg: any, fi } const fieldCtx = ctx.currentField() if (fieldCtx != null) { - const ruleTags = ctx.rule.tags - if (ruleTags == null || ruleTags.size === 0 || !disjoint(ruleTags, fieldCtx.tags)) { + const ruleTags = ctx.rule.tags ?? [] + if (ruleTags == null || ruleTags.length === 0 || !disjoint(new Set(ruleTags), fieldCtx.tags)) { return await fieldTransform.transform(ctx, fieldCtx, msg) } } @@ -461,5 +514,8 @@ function disjoint(tags1: Set, tags2: Set): boolean { return true } - - +function isBuiltin(name: string): boolean { + return name.startsWith('confluent/') || + name.startsWith('google/protobuf/') || + name.startsWith('google/type/') +} diff --git a/schemaregistry/serde/rule-registry.ts b/schemaregistry/serde/rule-registry.ts index e1fb1514..d10320f8 100644 --- a/schemaregistry/serde/rule-registry.ts +++ b/schemaregistry/serde/rule-registry.ts @@ -1,42 +1,56 @@ import {RuleAction, RuleExecutor} from "./serde"; -const ruleExecutors = new Map - -const ruleActions = new Map - - -// registerRuleExecutor is used to register a new rule executor. -export function registerRuleExecutor(ruleExecutor: RuleExecutor): void { - ruleExecutors.set(ruleExecutor.type(), ruleExecutor) -} - -// getRuleExecutor fetches a rule executor by a given name. -export function getRuleExecutor(name: string): RuleExecutor | undefined { - return ruleExecutors.get(name) -} - -// getRuleExecutors fetches all rule executors -export function getRuleExecutors(): RuleExecutor[] { - return Array.from(ruleExecutors.values()) -} - -// registerRuleAction is used to register a new rule action. -export function registerRuleAction(ruleAction: RuleAction): void { - ruleActions.set(ruleAction.type(), ruleAction) -} - -// getRuleAction fetches a rule action by a given name. -export function getRuleAction(name: string): RuleAction | undefined { - return ruleActions.get(name) -} - -// getRuleActions fetches all rule actions -export function getRuleActions(): RuleAction[] { - return Array.from(ruleActions.values()) -} - -// clearRules clears all registered rules -export function clearRules(): void { - ruleExecutors.clear() - ruleActions.clear() +export class RuleRegistry { + private ruleExecutors: Map = new Map() + private ruleActions: Map = new Map() + + private static globalInstance: RuleRegistry = new RuleRegistry() + + // registerExecutor is used to register a new rule executor. + public registerExecutor(ruleExecutor: RuleExecutor): void { + this.ruleExecutors.set(ruleExecutor.type(), ruleExecutor) + } + + // getExecutor fetches a rule executor by a given name. + public getExecutor(name: string): RuleExecutor | undefined { + return this.ruleExecutors.get(name) + } + + // getExecutors fetches all rule executors + public getExecutors(): RuleExecutor[] { + return Array.from(this.ruleExecutors.values()) + } + + // registerAction is used to register a new rule action. + public registerAction(ruleAction: RuleAction): void { + this.ruleActions.set(ruleAction.type(), ruleAction) + } + + // getAction fetches a rule action by a given name. + public getAction(name: string): RuleAction | undefined { + return this.ruleActions.get(name) + } + + // getActions fetches all rule actions + public getActions(): RuleAction[] { + return Array.from(this.ruleActions.values()) + } + + // clear clears all registered rules + public clear(): void { + this.ruleExecutors.clear() + this.ruleActions.clear() + } + + public static getGlobalInstance(): RuleRegistry { + return RuleRegistry.globalInstance + } + + public static registerRuleExecutor(ruleExecutor: RuleExecutor): void { + RuleRegistry.globalInstance.registerExecutor(ruleExecutor) + } + + public static registerRuleAction(ruleAction: RuleAction): void { + RuleRegistry.globalInstance.registerAction(ruleAction) + } } diff --git a/schemaregistry/serde/serde.ts b/schemaregistry/serde/serde.ts index b550c941..18a5321a 100644 --- a/schemaregistry/serde/serde.ts +++ b/schemaregistry/serde/serde.ts @@ -5,11 +5,10 @@ import { RuleMode, RuleSet, SchemaInfo, - SchemaMetadata, SchemaRegistryClient + SchemaMetadata } from "../schemaregistry-client"; -import {getRuleAction, getRuleExecutor} from "./rule-registry"; +import {RuleRegistry} from "./rule-registry"; import {ClientConfig} from "../rest-service"; -import {MockClient} from "../mock-schemaregistry-client"; export enum SerdeType { KEY = 'KEY', @@ -29,13 +28,13 @@ export interface SerdeConfig { // useLatestVersion specifies whether to use the latest schema version useLatestVersion?: boolean // useLatestWithMetadata specifies whether to use the latest schema with metadata - useLatestWithMetadata?: Map + useLatestWithMetadata?: { [key: string]: string }; // cacheCapacity specifies the cache capacity cacheCapacity?: number, // cacheLatestTtlSecs specifies the cache latest TTL in seconds cacheLatestTtlSecs?: number // ruleConfig specifies configuration options to the rules - ruleConfig?: Map + ruleConfig?: { [key: string]: string }; // subjectNameStrategy specifies a function to generate a subject name subjectNameStrategy?: SubjectNameStrategyFunc } @@ -47,11 +46,13 @@ export abstract class Serde { serdeType: SerdeType conf: SerdeConfig fieldTransformer: FieldTransformer | null = null + ruleRegistry: RuleRegistry - protected constructor(client: Client, serdeType: SerdeType, conf: SerdeConfig) { + protected constructor(client: Client, serdeType: SerdeType, conf: SerdeConfig, ruleRegistry?: RuleRegistry) { this.client = client this.serdeType = serdeType this.conf = conf + this.ruleRegistry = ruleRegistry ?? RuleRegistry.getGlobalInstance() } abstract config(): SerdeConfig @@ -65,22 +66,15 @@ export abstract class Serde { return strategy(topic, this.serdeType, info) } - async resolveReferences(client: Client, schema: SchemaInfo, deps: Map): Promise { + async resolveReferences(client: Client, schema: SchemaInfo, deps: Map, format?: string): Promise { let references = schema.references if (references == null) { return } for (let ref of references) { - let metadata = await client.getSchemaMetadata(ref.subject, ref.version, true) - let info = { - schema: schema.schema, - schemaType: schema.schemaType, - references: schema.references, - metadata: schema.metadata, - ruleSet: schema.ruleSet, - } + let metadata = await client.getSchemaMetadata(ref.subject, ref.version, true, format) deps.set(ref.name, metadata.schema) - await this.resolveReferences(client, info, deps) + await this.resolveReferences(client, metadata, deps) } } @@ -134,7 +128,7 @@ export abstract class Serde { } let ctx = new RuleContext(source, target, subject, topic, this.serdeType === SerdeType.KEY, ruleMode, rule, i, rules, inlineTags, this.fieldTransformer!) - let ruleExecutor = getRuleExecutor(rule.type) + let ruleExecutor = this.ruleRegistry.getExecutor(rule.type) if (ruleExecutor == null) { await this.runAction(ctx, ruleMode, rule, rule.onFailure, msg, new Error(`could not find rule executor of type ${rule.type}`), 'ERROR') @@ -208,7 +202,7 @@ export abstract class Serde { } else if (actionName === 'NONE') { return new NoneAction() } - return getRuleAction(actionName) + return this.ruleRegistry.getAction(actionName) } } @@ -222,8 +216,8 @@ export interface SerializerConfig extends SerdeConfig { } export abstract class Serializer extends Serde { - protected constructor(client: Client, serdeType: SerdeType, conf: SerializerConfig) { - super(client, serdeType, conf) + protected constructor(client: Client, serdeType: SerdeType, conf: SerializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) } override config(): SerializerConfig { @@ -234,7 +228,7 @@ export abstract class Serializer extends Serde { abstract serialize(topic: string, msg: any): Promise // GetID returns a schema ID for the given schema - async getId(topic: string, msg: any, info: SchemaInfo): Promise<[number, SchemaInfo]> { + async getId(topic: string, msg: any, info: SchemaInfo, format?: string): Promise<[number, SchemaInfo]> { let autoRegister = this.config().autoRegisterSchemas let useSchemaId = this.config().useSchemaId let useLatestWithMetadata = this.conf.useLatestWithMetadata @@ -246,17 +240,16 @@ export abstract class Serializer extends Serde { if (autoRegister) { id = await this.client.register(subject, info, Boolean(normalizeSchema)) } else if (useSchemaId != null && useSchemaId >= 0) { - info = await this.client.getBySubjectAndId(subject, useSchemaId) + info = await this.client.getBySubjectAndId(subject, useSchemaId, format) id = await this.client.getId(subject, info, false) if (id !== useSchemaId) { throw new SerializationError(`failed to match schema ID (${id} != ${useSchemaId})`) } - } else if (useLatestWithMetadata != null && useLatestWithMetadata.size !== 0) { - info = await this.client.getLatestWithMetadata( - subject, Object.fromEntries(useLatestWithMetadata), true) + } else if (useLatestWithMetadata != null && Object.keys(useLatestWithMetadata).length !== 0) { + info = await this.client.getLatestWithMetadata(subject, useLatestWithMetadata, true, format) id = await this.client.getId(subject, info, false) } else if (useLatest) { - info = await this.client.getLatestSchemaMetadata(subject) + info = await this.client.getLatestSchemaMetadata(subject, format) id = await this.client.getId(subject, info, false) } else { id = await this.client.getId(subject, info, Boolean(normalizeSchema)) @@ -281,15 +274,15 @@ export interface Migration { } export abstract class Deserializer extends Serde { - protected constructor(client: Client, serdeType: SerdeType, conf: DeserializerConfig) { - super(client, serdeType, conf) + protected constructor(client: Client, serdeType: SerdeType, conf: DeserializerConfig, ruleRegistry?: RuleRegistry) { + super(client, serdeType, conf, ruleRegistry) } override config(): DeserializerConfig { return this.conf as DeserializerConfig } - async getSchema(topic: string, payload: Buffer): Promise { + async getSchema(topic: string, payload: Buffer, format?: string): Promise { const magicByte = payload.subarray(0, 1) if (!magicByte.equals(MAGIC_BYTE)) { throw new SerializationError( @@ -300,18 +293,17 @@ export abstract class Deserializer extends Serde { } const id = payload.subarray(1, 5).readInt32BE(0) let subject = this.subjectName(topic) - return await this.client.getBySubjectAndId(subject, id) + return await this.client.getBySubjectAndId(subject, id, format) } - async getReaderSchema(subject: string): Promise { + async getReaderSchema(subject: string, format?: string): Promise { let useLatestWithMetadata = this.config().useLatestWithMetadata let useLatest = this.config().useLatestVersion - if (useLatestWithMetadata != null && useLatestWithMetadata.size !== 0) { - return await this.client.getLatestWithMetadata( - subject, Object.fromEntries(useLatestWithMetadata), true) + if (useLatestWithMetadata != null && Object.keys(useLatestWithMetadata).length !== 0) { + return await this.client.getLatestWithMetadata(subject, useLatestWithMetadata, true, format) } if (useLatest) { - return await this.client.getLatestSchemaMetadata(subject) + return await this.client.getLatestSchemaMetadata(subject, format) } return null } @@ -349,7 +341,7 @@ export abstract class Deserializer extends Serde { } async getMigrations(subject: string, sourceInfo: SchemaInfo, - target: SchemaMetadata): Promise { + target: SchemaMetadata, format?: string): Promise { let version = await this.client.getVersion(subject, sourceInfo, false) let source: SchemaMetadata = { id: 0, @@ -375,7 +367,7 @@ export abstract class Deserializer extends Serde { return migrations } let previous: SchemaMetadata | null = null - let versions = await this.getSchemasBetween(subject, first, last) + let versions = await this.getSchemasBetween(subject, first, last, format) for (let i = 0; i < versions.length; i++) { let version = versions[i] if (i === 0) { @@ -408,7 +400,7 @@ export abstract class Deserializer extends Serde { } async getSchemasBetween(subject: string, first: SchemaMetadata, - last: SchemaMetadata): Promise { + last: SchemaMetadata, format?: string): Promise { if (last.version!-first.version! <= 1) { return [first, last] } @@ -416,7 +408,7 @@ export abstract class Deserializer extends Serde { let version2 = last.version! let result = [first] for (let i = version1 + 1; i < version2; i++) { - let meta = await this.client.getSchemaMetadata(subject, i, true) + let meta = await this.client.getSchemaMetadata(subject, i, true, format) result.push(meta) } result.push(last) @@ -615,7 +607,7 @@ export abstract class FieldRuleExecutor implements RuleExecutor { } function areTransformsWithSameTag(rule1: Rule, rule2: Rule): boolean { - return rule1.tags != null && rule1.tags.size > 0 + return rule1.tags != null && rule1.tags.length > 0 && rule1.kind === 'TRANSFORM' && rule1.kind === rule2.kind && rule1.mode === rule2.mode @@ -736,11 +728,3 @@ export class RuleConditionError extends RuleError { return errMsg } } - -export function newClient(config: ClientConfig): Client { - let url = config.baseURLs[0] - if (url.startsWith("mock://")) { - return new MockClient(config) - } - return new SchemaRegistryClient(config) -} diff --git a/test/schemaregistry/dekregistry/dekregistry-client.spec.ts b/test/schemaregistry/rules/encryption/dekregistry/dekregistry-client.spec.ts similarity index 94% rename from test/schemaregistry/dekregistry/dekregistry-client.spec.ts rename to test/schemaregistry/rules/encryption/dekregistry/dekregistry-client.spec.ts index 4a7eb47c..5dc3da20 100644 --- a/test/schemaregistry/dekregistry/dekregistry-client.spec.ts +++ b/test/schemaregistry/rules/encryption/dekregistry/dekregistry-client.spec.ts @@ -1,14 +1,14 @@ -import { DekRegistryClient, Dek, Kek } from "../../../schemaregistry/dekregistry/dekregistry-client"; -import { RestService } from "../../../schemaregistry/rest-service"; +import { DekRegistryClient, Dek, Kek } from "../../../../../schemaregistry/rules/encryption/dekregistry/dekregistry-client"; +import { RestService } from "../../../../../schemaregistry/rest-service"; import { AxiosResponse } from 'axios'; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; import { TEST_KEK, TEST_KEK_2, TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_2, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_SUBJECT, TEST_VERSION, TEST_DEK_LATEST} from "./test-constants"; -import { mockClientConfig } from "../test-constants"; +import { mockClientConfig } from "../../../test-constants"; -jest.mock('../../../schemaregistry/rest-service'); +jest.mock('../../../../../schemaregistry/rest-service'); let client: DekRegistryClient; diff --git a/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts b/test/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.spec.ts similarity index 90% rename from test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts rename to test/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.spec.ts index 71176814..a5ed3cb2 100644 --- a/test/schemaregistry/dekregistry/mock-dekregistry-client.spec.ts +++ b/test/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.spec.ts @@ -1,5 +1,5 @@ -import { Dek, Kek } from "../../../schemaregistry/dekregistry/dekregistry-client"; -import { MockDekRegistryClient } from "../../../schemaregistry/dekregistry/mock-dekregistry-client"; +import { Dek, Kek } from "../../../../../schemaregistry/rules/encryption/dekregistry/dekregistry-client"; +import { MockDekRegistryClient } from "../../../../../schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client"; import { beforeEach, afterEach, describe, expect, it, jest } from '@jest/globals'; import { TEST_KEK, TEST_KEK_NAME, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, TEST_DEK, TEST_DEK_V2, TEST_ALGORITHM, diff --git a/test/schemaregistry/dekregistry/test-constants.ts b/test/schemaregistry/rules/encryption/dekregistry/test-constants.ts similarity index 90% rename from test/schemaregistry/dekregistry/test-constants.ts rename to test/schemaregistry/rules/encryption/dekregistry/test-constants.ts index 4719fdce..b5cae073 100644 --- a/test/schemaregistry/dekregistry/test-constants.ts +++ b/test/schemaregistry/rules/encryption/dekregistry/test-constants.ts @@ -1,5 +1,5 @@ -import { MOCK_TS } from "../../../schemaregistry/dekregistry/constants"; -import { Kek, Dek } from "../../../schemaregistry/dekregistry/dekregistry-client"; +import { MOCK_TS } from "../../../../../schemaregistry/rules/encryption/dekregistry/constants"; +import { Kek, Dek } from "../../../../../schemaregistry/rules/encryption/dekregistry/dekregistry-client"; const TEST_KEK_NAME: string = 'test-kek-name'; const TEST_KEK_NAME_2: string = 'test-kek-name2'; @@ -71,4 +71,4 @@ export { TEST_KEK_NAME, TEST_KEK_NAME_2, TEST_KMS_TYPE, TEST_KMS_KEY_ID, TEST_KMS_PROPS, TEST_DOC, TEST_SUBJECT, TEST_ALGORITHM, TEST_ENCRYPTED_KEY_MATERIAL, TEST_VERSION, TEST_KEK, TEST_KEK_2, TEST_DEK, TEST_DEK_V2, TEST_DEK_2, TEST_DEK_LATEST -}; \ No newline at end of file +}; diff --git a/test/schemaregistry/serde/avro.spec.ts b/test/schemaregistry/serde/avro.spec.ts index 068d99e9..ec7abec9 100644 --- a/test/schemaregistry/serde/avro.spec.ts +++ b/test/schemaregistry/serde/avro.spec.ts @@ -1,32 +1,516 @@ -import {describe, expect, it} from '@jest/globals'; +import {afterEach, describe, expect, it} from '@jest/globals'; import {ClientConfig} from "../../../schemaregistry/rest-service"; -import {AvroDeserializer, AvroSerializer} from "../../../schemaregistry/serde/avro"; -import {newClient, SerdeType} from "../../../schemaregistry/serde/serde"; +import { + AvroDeserializer, AvroDeserializerConfig, + AvroSerializer, + AvroSerializerConfig +} from "../../../schemaregistry/serde/avro"; +import {SerdeType} from "../../../schemaregistry/serde/serde"; +import { + Rule, + RuleMode, + RuleSet, + SchemaInfo, + SchemaRegistryClient +} from "../../../schemaregistry/schemaregistry-client"; +import {LocalKmsDriver} from "../../../schemaregistry/rules/encryption/localkms/local-driver"; +import {FieldEncryptionExecutor} from "../../../schemaregistry/rules/encryption/encrypt-executor"; + +const demoSchema = ` +{ + "name": "DemoSchema", + "type": "record", + "fields": [ + { + "name": "intField", + "type": "int" + }, + { + "name": "doubleField", + "type": "double" + }, + { + "name": "stringField", + "type": "string", + "confluent:tags": [ "PII" ] + }, + { + "name": "boolField", + "type": "boolean" + }, + { + "name": "bytesField", + "type": "bytes", + "confluent:tags": [ "PII" ] + } + ] +} +` +const demoSchemaWithLogicalType = ` +{ + "name": "DemoSchema", + "type": "record", + "fields": [ + { + "name": "intField", + "type": "int" + }, + { + "name": "doubleField", + "type": "double" + }, + { + "name": "stringField", + "type": { + "type": "string", + "logicalType": "uuid" + }, + "confluent:tags": [ "PII" ] + }, + { + "name": "boolField", + "type": "boolean" + }, + { + "name": "bytesField", + "type": "bytes", + "confluent:tags": [ "PII" ] + } + ] +} +` +const rootPointerSchema = ` +{ + "name": "NestedTestPointerRecord", + "type": "record", + "fields": [ + { + "name": "otherField", + "type": ["null", "DemoSchema"] + } +] +} +` +const f1Schema = ` +{ + "name": "F1Schema", + "type": "record", + "fields": [ + { + "name": "f1", + "type": "string", + "confluent:tags": [ "PII" ] + } + ] +} +` + +const fieldEncryptionExecutor = FieldEncryptionExecutor.register() +LocalKmsDriver.register() + +//const baseURL = 'http://localhost:8081' +const baseURL = 'mock://' + +const topic = 'topic1' +const subject = topic + '-value' describe('AvroSerializer', () => { + afterEach(async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + await client.deleteSubject(subject, false) + await client.deleteSubject(subject, true) + }) it('basic serialization', async () => { let conf: ClientConfig = { - baseURLs: ['mock://'], - cacheCapacity: 1000, - createAxiosDefaults: {} + baseURLs: [baseURL], + cacheCapacity: 1000 } - let client = newClient(conf) + let client = SchemaRegistryClient.newClient(conf) let ser = new AvroSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) let obj = { intField: 123, doubleField: 45.67, stringField: 'hi', boolField: true, - bytesField: new Buffer([1, 2]), + bytesField: Buffer.from([1, 2]), + } + let bytes = await ser.serialize(topic, obj) + + let deser = new AvroDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + }) + it('serialize nested', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new AvroSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + + let nested = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), + } + let obj = { + otherField: nested + } + let bytes = await ser.serialize(topic, obj) + + let deser = new AvroDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.otherField.intField).toEqual(nested.intField); + expect(obj2.otherField.doubleField).toBeCloseTo(nested.doubleField, 0.001); + expect(obj2.otherField.stringField).toEqual(nested.stringField); + expect(obj2.otherField.boolField).toEqual(nested.boolField); + expect(obj2.otherField.bytesField).toEqual(nested.bytesField); + }) + it('serialize reference', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new AvroSerializer(client, SerdeType.VALUE, {useLatestVersion: true}) + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: demoSchema, + } + await client.register('demo-value', info , false) + + info = { + schemaType: 'AVRO', + schema: rootPointerSchema, + references: [{ + name: 'DemoSchema', + subject: 'demo-value', + version: 1 + }] + } + await client.register(subject, info , false) + + let nested = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), } - let bytes = await ser.serialize("topic1", obj) + let obj = { + otherField: nested + } + let bytes = await ser.serialize(topic, obj) let deser = new AvroDeserializer(client, SerdeType.VALUE, {}) - let obj2 = await deser.deserialize("topic1", bytes) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.otherField.intField).toEqual(nested.intField); + expect(obj2.otherField.doubleField).toBeCloseTo(nested.doubleField, 0.001); + expect(obj2.otherField.stringField).toEqual(nested.stringField); + expect(obj2.otherField.boolField).toEqual(nested.boolField); + expect(obj2.otherField.bytesField).toEqual(nested.bytesField); + }) + it('basic encryption', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: demoSchema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + obj.bytesField = Buffer.from([1, 2]) + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) expect(obj2.intField).toEqual(obj.intField); expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); expect(obj2.stringField).toEqual(obj.stringField); expect(obj2.boolField).toEqual(obj.boolField); expect(obj2.bytesField).toEqual(obj.bytesField); }) + it('basic encryption with logical type', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: demoSchemaWithLogicalType, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + obj.bytesField = Buffer.from([1, 2]) + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + }) + it('basic encryption with preserialized data', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: f1Schema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + f1: 'hello world' + } + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + let dekClient = fieldEncryptionExecutor.client! + + await dekClient.registerKek("kek1", "local-kms", "mykey", false) + const encryptedDek = "07V2ndh02DA73p+dTybwZFm7DKQSZN1tEwQh+FoX1DZLk4Yj2LLu4omYjp/84tAg3BYlkfGSz+zZacJHIE4=" + await dekClient.registerDek("kek1", subject, "AES256_GCM", 1, encryptedDek) + + const bytes = Buffer.from([0, 0, 0, 0, 1, 104, 122, 103, 121, 47, 106, 70, 78, 77, 86, 47, 101, 70, 105, 108, 97, 72, 114, 77, 121, 101, 66, 103, 100, 97, 86, 122, 114, 82, 48, 117, 100, 71, 101, 111, 116, 87, 56, 99, 65, 47, 74, 97, 108, 55, 117, 107, 114, 43, 77, 47, 121, 122]) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.f1).toEqual(obj.f1); + }) + it('deterministic encryption with preserialized data', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + 'encrypt.dek.algorithm': 'AES256_SIV', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: f1Schema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + f1: 'hello world' + } + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + let dekClient = fieldEncryptionExecutor.client! + + await dekClient.registerKek("kek1", "local-kms", "mykey", false) + const encryptedDek = "YSx3DTlAHrmpoDChquJMifmPntBzxgRVdMzgYL82rgWBKn7aUSnG+WIu9ozBNS3y2vXd++mBtK07w4/W/G6w0da39X9hfOVZsGnkSvry/QRht84V8yz3dqKxGMOK5A==" + await dekClient.registerDek("kek1", subject, "AES256_SIV", 1, encryptedDek) + + const bytes = Buffer.from([0, 0, 0, 0, 1, 72, 68, 54, 89, 116, 120, 114, 108, 66, 110, 107, 84, 87, 87, 57, 78, 54, 86, 98, 107, 51, 73, 73, 110, 106, 87, 72, 56, 49, 120, 109, 89, 104, 51, 107, 52, 100]) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.f1).toEqual(obj.f1); + }) + it('dek rotation encryption with preserialized data', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + 'encrypt.dek.expiry.days': '1', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: f1Schema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + f1: 'hello world' + } + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + let dekClient = fieldEncryptionExecutor.client! + + await dekClient.registerKek("kek1", "local-kms", "mykey", false) + const encryptedDek = "W/v6hOQYq1idVAcs1pPWz9UUONMVZW4IrglTnG88TsWjeCjxmtRQ4VaNe/I5dCfm2zyY9Cu0nqdvqImtUk4=" + await dekClient.registerDek("kek1", subject, "AES256_GCM", 1, encryptedDek) + + const bytes = Buffer.from([0, 0, 0, 0, 1, 120, 65, 65, 65, 65, 65, 65, 71, 52, 72, 73, 54, 98, 49, 110, 88, 80, 88, 113, 76, 121, 71, 56, 99, 73, 73, 51, 53, 78, 72, 81, 115, 101, 113, 113, 85, 67, 100, 43, 73, 101, 76, 101, 70, 86, 65, 101, 78, 112, 83, 83, 51, 102, 120, 80, 110, 74, 51, 50, 65, 61]) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.f1).toEqual(obj.f1); + }) }) diff --git a/test/schemaregistry/serde/buffer-wrapper.spec.ts b/test/schemaregistry/serde/buffer-wrapper.spec.ts index 5c4ed3af..f3ac5f76 100644 --- a/test/schemaregistry/serde/buffer-wrapper.spec.ts +++ b/test/schemaregistry/serde/buffer-wrapper.spec.ts @@ -3,21 +3,21 @@ import { BufferWrapper, MAX_VARINT_LEN_32 } from "../../../schemaregistry/serde/ describe('BufferWrapper', () => { it('write and read 100', () => { - const buf = new Buffer(MAX_VARINT_LEN_32) + const buf = Buffer.alloc(MAX_VARINT_LEN_32) const bw = new BufferWrapper(buf) bw.writeVarInt(100) const bw2 = new BufferWrapper(bw.buf.subarray(0, bw.pos)) expect(bw2.readVarInt()).toBe(100) }) it('write and read max pos int', () => { - const buf = new Buffer(MAX_VARINT_LEN_32) + const buf = Buffer.alloc(MAX_VARINT_LEN_32) const bw = new BufferWrapper(buf) bw.writeVarInt(2147483647) const bw2 = new BufferWrapper(bw.buf.subarray(0, bw.pos)) expect(bw2.readVarInt()).toBe(2147483647) }) it('write and read max neg int', () => { - const buf = new Buffer(MAX_VARINT_LEN_32) + const buf = Buffer.alloc(MAX_VARINT_LEN_32) const bw = new BufferWrapper(buf) bw.writeVarInt(-2147483648) const bw2 = new BufferWrapper(bw.buf.subarray(0, bw.pos)) diff --git a/test/schemaregistry/serde/json.spec.ts b/test/schemaregistry/serde/json.spec.ts new file mode 100644 index 00000000..c28f6f9d --- /dev/null +++ b/test/schemaregistry/serde/json.spec.ts @@ -0,0 +1,209 @@ +import {afterEach, describe, expect, it} from '@jest/globals'; +import {ClientConfig} from "../../../schemaregistry/rest-service"; +import {SerdeType} from "../../../schemaregistry/serde/serde"; +import { + Rule, + RuleMode, + RuleSet, + SchemaInfo, + SchemaRegistryClient +} from "../../../schemaregistry/schemaregistry-client"; +import {LocalKmsDriver} from "../../../schemaregistry/rules/encryption/localkms/local-driver"; +import {FieldEncryptionExecutor} from "../../../schemaregistry/rules/encryption/encrypt-executor"; +import { + JsonDeserializer, JsonDeserializerConfig, + JsonSerializer, + JsonSerializerConfig +} from "../../../schemaregistry/serde/json"; + +const fieldEncryptionExecutor = FieldEncryptionExecutor.register() +LocalKmsDriver.register() + +//const baseURL = 'http://localhost:8081' +const baseURL = 'mock://' + +const topic = 'topic1' +const subject = topic + '-value' + +const rootSchema = ` +{ + "type": "object", + "properties": { + "otherField": { "$ref": "DemoSchema" } + } +} +` + +const demoSchema = ` +{ + "type": "object", + "properties": { + "intField": { "type": "integer" }, + "doubleField": { "type": "number" }, + "stringField": { + "type": "string", + "confluent:tags": [ "PII" ] + }, + "boolField": { "type": "boolean" }, + "bytesField": { + "type": "string", + "contentEncoding": "base64", + "confluent:tags": [ "PII" ] + } + } +} +` + +describe('JsonSerializer', () => { + afterEach(async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + await client.deleteSubject(subject, false) + await client.deleteSubject(subject, true) + }) + it('basic serialization', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('serialize nested', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('serialize reference', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, {useLatestVersion: true}) + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchema + } + await client.register('demo-value', info, false) + + info = { + schemaType: 'JSON', + schema: rootSchema, + references: [{ + name: 'DemoSchema', + subject: 'demo-value', + version: 1 + }] + } + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('basic encryption', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: JsonSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new JsonSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + obj.bytesField = Buffer.from([0, 0, 0, 1]).toString('base64') + + let deserConfig: JsonDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new JsonDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) +}) diff --git a/test/schemaregistry/serde/protobuf.spec.ts b/test/schemaregistry/serde/protobuf.spec.ts new file mode 100644 index 00000000..de238e35 --- /dev/null +++ b/test/schemaregistry/serde/protobuf.spec.ts @@ -0,0 +1,205 @@ +import {afterEach, describe, expect, it} from '@jest/globals'; +import {ClientConfig} from "../../../schemaregistry/rest-service"; +import { + ProtobufDeserializer, ProtobufDeserializerConfig, + ProtobufSerializer, ProtobufSerializerConfig, +} from "../../../schemaregistry/serde/protobuf"; +import {SerdeType} from "../../../schemaregistry/serde/serde"; +import { + Rule, + RuleMode, + RuleSet, + SchemaInfo, + SchemaRegistryClient +} from "../../../schemaregistry/schemaregistry-client"; +import {LocalKmsDriver} from "../../../schemaregistry/rules/encryption/localkms/local-driver"; +import {FieldEncryptionExecutor} from "../../../schemaregistry/rules/encryption/encrypt-executor"; +import {AuthorSchema, file_test_schemaregistry_serde_example, PizzaSchema} from "./test/example_pb"; +import {create, toBinary} from "@bufbuild/protobuf"; +import {FileDescriptorProtoSchema} from "@bufbuild/protobuf/wkt"; +import { + NestedMessage_InnerMessageSchema +} from "./test/nested_pb"; +import {TestMessageSchema} from "./test/test_pb"; +import {DependencyMessageSchema} from "./test/dep_pb"; + +const fieldEncryptionExecutor = FieldEncryptionExecutor.register() +LocalKmsDriver.register() + +//const baseURL = 'http://localhost:8081' +const baseURL = 'mock://' + +const topic = 'topic1' +const subject = topic + '-value' + +describe('ProtobufSerializer', () => { + afterEach(async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + await client.deleteSubject(subject, false) + await client.deleteSubject(subject, true) + }) + it('basic serialization', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new ProtobufSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + ser.registry.add(AuthorSchema) + let obj = create(AuthorSchema, { + name: 'Kafka', + id: 123, + picture: Buffer.from([1, 2]), + works: ['The Castle', 'The Trial'] + }) + let bytes = await ser.serialize(topic, obj) + + let deser = new ProtobufDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('serialize second messsage', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new ProtobufSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + ser.registry.add(PizzaSchema) + let obj = create(PizzaSchema, { + size: 'Extra extra large', + toppings: ['anchovies', 'mushrooms'] + }) + let bytes = await ser.serialize(topic, obj) + + let deser = new ProtobufDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('serialize nested messsage', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new ProtobufSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + ser.registry.add(NestedMessage_InnerMessageSchema) + let obj = create(NestedMessage_InnerMessageSchema, { + id: "inner" + }) + let bytes = await ser.serialize(topic, obj) + + let deser = new ProtobufDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('serialize reference', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new ProtobufSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + ser.registry.add(TestMessageSchema) + ser.registry.add(DependencyMessageSchema) + let msg = create(TestMessageSchema, { + testString: "hi", + testBool: true, + testBytes: Buffer.from([1, 2]), + testDouble: 1.23, + testFloat: 3.45, + testFixed32: 67, + testFixed64: 89n, + testInt32: 100, + testInt64: 200n, + testSfixed32: 300, + testSfixed64: 400n, + testSint32: 500, + testSint64: 600n, + testUint32: 700, + testUint64: 800n, + }) + let obj = create(DependencyMessageSchema, { + isActive: true, + testMesssage: msg + }) + let bytes = await ser.serialize(topic, obj) + + let deser = new ProtobufDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.testMesssage.testString).toEqual(msg.testString); + expect(obj2.testMesssage.testBool).toEqual(msg.testBool); + expect(obj2.testMesssage.testBytes).toEqual(msg.testBytes); + expect(obj2.testMesssage.testDouble).toBeCloseTo(msg.testDouble, 0.001); + expect(obj2.testMesssage.testFloat).toBeCloseTo(msg.testFloat, 0.001); + expect(obj2.testMesssage.testFixed32).toEqual(msg.testFixed32); + expect(obj2.testMesssage.testFixed64).toEqual(msg.testFixed64); + }) + it('basic encryption', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: ProtobufSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new ProtobufSerializer(client, SerdeType.VALUE, serConfig) + ser.registry.add(AuthorSchema) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'PROTOBUF', + schema: Buffer.from(toBinary(FileDescriptorProtoSchema, file_test_schemaregistry_serde_example.proto)).toString('base64'), + ruleSet + } + + await client.register(subject, info, false) + + let obj = create(AuthorSchema, { + name: 'Kafka', + id: 123, + picture: Buffer.from([1, 2]), + works: ['The Castle', 'The Trial'] + }) + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.name = 'Kafka' + obj.picture = Buffer.from([1, 2]) + + let deserConfig: ProtobufDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new ProtobufDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) +}) diff --git a/test/schemaregistry/serde/test/cycle_pb.ts b/test/schemaregistry/serde/test/cycle_pb.ts new file mode 100644 index 00000000..73e60993 --- /dev/null +++ b/test/schemaregistry/serde/test/cycle_pb.ts @@ -0,0 +1,36 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/cycle.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/cycle.proto. + */ +export const file_test_schemaregistry_serde_cycle: GenFile = /*@__PURE__*/ + fileDesc("CiV0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL2N5Y2xlLnByb3RvEgR0ZXN0IjsKCkxpbmtlZExpc3QSDQoFdmFsdWUYASABKAUSHgoEbmV4dBgKIAEoCzIQLnRlc3QuTGlua2VkTGlzdEIJWgcuLi90ZXN0YgZwcm90bzM"); + +/** + * @generated from message test.LinkedList + */ +export type LinkedList = Message<"test.LinkedList"> & { + /** + * @generated from field: int32 value = 1; + */ + value: number; + + /** + * @generated from field: test.LinkedList next = 10; + */ + next?: LinkedList; +}; + +/** + * Describes the message test.LinkedList. + * Use `create(LinkedListSchema)` to create a new message. + */ +export const LinkedListSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_cycle, 0); + diff --git a/test/schemaregistry/serde/test/dep_pb.ts b/test/schemaregistry/serde/test/dep_pb.ts new file mode 100644 index 00000000..80783aee --- /dev/null +++ b/test/schemaregistry/serde/test/dep_pb.ts @@ -0,0 +1,38 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/dep.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { TestMessage } from "./test_pb"; +import { file_test_schemaregistry_serde_test } from "./test_pb"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/dep.proto. + */ +export const file_test_schemaregistry_serde_dep: GenFile = /*@__PURE__*/ + fileDesc("CiN0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL2RlcC5wcm90bxIEdGVzdCJQChFEZXBlbmRlbmN5TWVzc2FnZRIRCglpc19hY3RpdmUYASABKAgSKAoNdGVzdF9tZXNzc2FnZRgCIAEoCzIRLnRlc3QuVGVzdE1lc3NhZ2VCCVoHLi4vdGVzdGIGcHJvdG8z", [file_test_schemaregistry_serde_test]); + +/** + * @generated from message test.DependencyMessage + */ +export type DependencyMessage = Message<"test.DependencyMessage"> & { + /** + * @generated from field: bool is_active = 1; + */ + isActive: boolean; + + /** + * @generated from field: test.TestMessage test_messsage = 2; + */ + testMesssage?: TestMessage; +}; + +/** + * Describes the message test.DependencyMessage. + * Use `create(DependencyMessageSchema)` to create a new message. + */ +export const DependencyMessageSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_dep, 0); + diff --git a/test/schemaregistry/serde/test/example_pb.ts b/test/schemaregistry/serde/test/example_pb.ts new file mode 100644 index 00000000..2120dfd8 --- /dev/null +++ b/test/schemaregistry/serde/test/example_pb.ts @@ -0,0 +1,69 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/example.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import { file_confluent_meta } from "../../../../schemaregistry/confluent/meta_pb"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/example.proto. + */ +export const file_test_schemaregistry_serde_example: GenFile = /*@__PURE__*/ + fileDesc("Cid0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL2V4YW1wbGUucHJvdG8SBHRlc3QiVgoGQXV0aG9yEhYKBG5hbWUYASABKAlCCIJEBRoDUElJEgoKAmlkGAIgASgFEhkKB3BpY3R1cmUYAyABKAxCCIJEBRoDUElJEg0KBXdvcmtzGAQgAygJIicKBVBpenphEgwKBHNpemUYASABKAkSEAoIdG9wcGluZ3MYAiADKAlCCVoHLi4vdGVzdGIGcHJvdG8z", [file_confluent_meta]); + +/** + * @generated from message test.Author + */ +export type Author = Message<"test.Author"> & { + /** + * @generated from field: string name = 1; + */ + name: string; + + /** + * @generated from field: int32 id = 2; + */ + id: number; + + /** + * @generated from field: bytes picture = 3; + */ + picture: Uint8Array; + + /** + * @generated from field: repeated string works = 4; + */ + works: string[]; +}; + +/** + * Describes the message test.Author. + * Use `create(AuthorSchema)` to create a new message. + */ +export const AuthorSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_example, 0); + +/** + * @generated from message test.Pizza + */ +export type Pizza = Message<"test.Pizza"> & { + /** + * @generated from field: string size = 1; + */ + size: string; + + /** + * @generated from field: repeated string toppings = 2; + */ + toppings: string[]; +}; + +/** + * Describes the message test.Pizza. + * Use `create(PizzaSchema)` to create a new message. + */ +export const PizzaSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_example, 1); + diff --git a/test/schemaregistry/serde/test/nested_pb.ts b/test/schemaregistry/serde/test/nested_pb.ts new file mode 100644 index 00000000..a76d1dfb --- /dev/null +++ b/test/schemaregistry/serde/test/nested_pb.ts @@ -0,0 +1,221 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/nested.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenEnum, GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { enumDesc, fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Timestamp } from "@bufbuild/protobuf/wkt"; +import { file_google_protobuf_timestamp } from "@bufbuild/protobuf/wkt"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/nested.proto. + */ +export const file_test_schemaregistry_serde_nested: GenFile = /*@__PURE__*/ + fileDesc("CiZ0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL25lc3RlZC5wcm90bxIEdGVzdCJsCgZVc2VySWQSFwoNa2Fma2FfdXNlcl9pZBgBIAEoCUgAEhcKDW90aGVyX3VzZXJfaWQYAiABKAVIABIlCgphbm90aGVyX2lkGAMgASgLMg8udGVzdC5NZXNzYWdlSWRIAEIJCgd1c2VyX2lkIhcKCU1lc3NhZ2VJZBIKCgJpZBgBIAEoCSJSCgtDb21wbGV4VHlwZRIQCgZvbmVfaWQYASABKAlIABISCghvdGhlcl9pZBgCIAEoBUgAEhEKCWlzX2FjdGl2ZRgDIAEoCEIKCghzb21lX3ZhbCLcAwoNTmVzdGVkTWVzc2FnZRIdCgd1c2VyX2lkGAEgASgLMgwudGVzdC5Vc2VySWQSEQoJaXNfYWN0aXZlGAIgASgIEhoKEmV4cGVyaW1lbnRzX2FjdGl2ZRgDIAMoCRIuCgp1cGRhdGVkX2F0GAQgASgLMhouZ29vZ2xlLnByb3RvYnVmLlRpbWVzdGFtcBIcCgZzdGF0dXMYBSABKA4yDC50ZXN0LlN0YXR1cxInCgxjb21wbGV4X3R5cGUYBiABKAsyES50ZXN0LkNvbXBsZXhUeXBlEjIKCG1hcF90eXBlGAcgAygLMiAudGVzdC5OZXN0ZWRNZXNzYWdlLk1hcFR5cGVFbnRyeRIvCgVpbm5lchgIIAEoCzIgLnRlc3QuTmVzdGVkTWVzc2FnZS5Jbm5lck1lc3NhZ2UaLgoMTWFwVHlwZUVudHJ5EgsKA2tleRgBIAEoCRINCgV2YWx1ZRgCIAEoCToCOAEaKwoMSW5uZXJNZXNzYWdlEgoKAmlkGAEgASgJEg8KA2lkcxgCIAMoBUICEAEiKAoJSW5uZXJFbnVtEggKBFpFUk8QABINCglBTFNPX1pFUk8QABoCEAFKBAgOEA9KBAgPEBBKBAgJEAxSA2Zvb1IDYmFyKiIKBlN0YXR1cxIKCgZBQ1RJVkUQABIMCghJTkFDVElWRRABQglaBy4uL3Rlc3RiBnByb3RvMw", [file_google_protobuf_timestamp]); + +/** + * @generated from message test.UserId + */ +export type UserId = Message<"test.UserId"> & { + /** + * @generated from oneof test.UserId.user_id + */ + userId: { + /** + * @generated from field: string kafka_user_id = 1; + */ + value: string; + case: "kafkaUserId"; + } | { + /** + * @generated from field: int32 other_user_id = 2; + */ + value: number; + case: "otherUserId"; + } | { + /** + * @generated from field: test.MessageId another_id = 3; + */ + value: MessageId; + case: "anotherId"; + } | { case: undefined; value?: undefined }; +}; + +/** + * Describes the message test.UserId. + * Use `create(UserIdSchema)` to create a new message. + */ +export const UserIdSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_nested, 0); + +/** + * @generated from message test.MessageId + */ +export type MessageId = Message<"test.MessageId"> & { + /** + * @generated from field: string id = 1; + */ + id: string; +}; + +/** + * Describes the message test.MessageId. + * Use `create(MessageIdSchema)` to create a new message. + */ +export const MessageIdSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_nested, 1); + +/** + * @generated from message test.ComplexType + */ +export type ComplexType = Message<"test.ComplexType"> & { + /** + * @generated from oneof test.ComplexType.some_val + */ + someVal: { + /** + * @generated from field: string one_id = 1; + */ + value: string; + case: "oneId"; + } | { + /** + * @generated from field: int32 other_id = 2; + */ + value: number; + case: "otherId"; + } | { case: undefined; value?: undefined }; + + /** + * @generated from field: bool is_active = 3; + */ + isActive: boolean; +}; + +/** + * Describes the message test.ComplexType. + * Use `create(ComplexTypeSchema)` to create a new message. + */ +export const ComplexTypeSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_nested, 2); + +/** + * + * Complex message using nested protos and repeated fields + * + * @generated from message test.NestedMessage + */ +export type NestedMessage = Message<"test.NestedMessage"> & { + /** + * @generated from field: test.UserId user_id = 1; + */ + userId?: UserId; + + /** + * @generated from field: bool is_active = 2; + */ + isActive: boolean; + + /** + * @generated from field: repeated string experiments_active = 3; + */ + experimentsActive: string[]; + + /** + * @generated from field: google.protobuf.Timestamp updated_at = 4; + */ + updatedAt?: Timestamp; + + /** + * @generated from field: test.Status status = 5; + */ + status: Status; + + /** + * @generated from field: test.ComplexType complex_type = 6; + */ + complexType?: ComplexType; + + /** + * @generated from field: map map_type = 7; + */ + mapType: { [key: string]: string }; + + /** + * @generated from field: test.NestedMessage.InnerMessage inner = 8; + */ + inner?: NestedMessage_InnerMessage; +}; + +/** + * Describes the message test.NestedMessage. + * Use `create(NestedMessageSchema)` to create a new message. + */ +export const NestedMessageSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_nested, 3); + +/** + * @generated from message test.NestedMessage.InnerMessage + */ +export type NestedMessage_InnerMessage = Message<"test.NestedMessage.InnerMessage"> & { + /** + * @generated from field: string id = 1; + */ + id: string; + + /** + * @generated from field: repeated int32 ids = 2 [packed = true]; + */ + ids: number[]; +}; + +/** + * Describes the message test.NestedMessage.InnerMessage. + * Use `create(NestedMessage_InnerMessageSchema)` to create a new message. + */ +export const NestedMessage_InnerMessageSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_nested, 3, 0); + +/** + * @generated from enum test.NestedMessage.InnerEnum + */ +export enum NestedMessage_InnerEnum { + /** + * @generated from enum value: ZERO = 0; + */ + ZERO = 0, + + /** + * @generated from enum value: ALSO_ZERO = 0; + */ + ALSO_ZERO = 0, +} + +/** + * Describes the enum test.NestedMessage.InnerEnum. + */ +export const NestedMessage_InnerEnumSchema: GenEnum = /*@__PURE__*/ + enumDesc(file_test_schemaregistry_serde_nested, 3, 0); + +/** + * @generated from enum test.Status + */ +export enum Status { + /** + * @generated from enum value: ACTIVE = 0; + */ + ACTIVE = 0, + + /** + * @generated from enum value: INACTIVE = 1; + */ + INACTIVE = 1, +} + +/** + * Describes the enum test.Status. + */ +export const StatusSchema: GenEnum = /*@__PURE__*/ + enumDesc(file_test_schemaregistry_serde_nested, 0); + diff --git a/test/schemaregistry/serde/test/newerwidget_pb.ts b/test/schemaregistry/serde/test/newerwidget_pb.ts new file mode 100644 index 00000000..69749e4d --- /dev/null +++ b/test/schemaregistry/serde/test/newerwidget_pb.ts @@ -0,0 +1,41 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/newerwidget.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/newerwidget.proto. + */ +export const file_test_schemaregistry_serde_newerwidget: GenFile = /*@__PURE__*/ + fileDesc("Cit0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL25ld2Vyd2lkZ2V0LnByb3RvEgR0ZXN0IjwKC05ld2VyV2lkZ2V0EgwKBG5hbWUYASABKAkSDgoGbGVuZ3RoGAIgASgFEg8KB3ZlcnNpb24YAyABKAVCCVoHLi4vdGVzdGIGcHJvdG8z"); + +/** + * @generated from message test.NewerWidget + */ +export type NewerWidget = Message<"test.NewerWidget"> & { + /** + * @generated from field: string name = 1; + */ + name: string; + + /** + * @generated from field: int32 length = 2; + */ + length: number; + + /** + * @generated from field: int32 version = 3; + */ + version: number; +}; + +/** + * Describes the message test.NewerWidget. + * Use `create(NewerWidgetSchema)` to create a new message. + */ +export const NewerWidgetSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_newerwidget, 0); + diff --git a/test/schemaregistry/serde/test/newwidget_pb.ts b/test/schemaregistry/serde/test/newwidget_pb.ts new file mode 100644 index 00000000..a24ad209 --- /dev/null +++ b/test/schemaregistry/serde/test/newwidget_pb.ts @@ -0,0 +1,41 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/newwidget.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/newwidget.proto. + */ +export const file_test_schemaregistry_serde_newwidget: GenFile = /*@__PURE__*/ + fileDesc("Cil0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL25ld3dpZGdldC5wcm90bxIEdGVzdCI6CglOZXdXaWRnZXQSDAoEbmFtZRgBIAEoCRIOCgZoZWlnaHQYAiABKAUSDwoHdmVyc2lvbhgDIAEoBUIJWgcuLi90ZXN0YgZwcm90bzM"); + +/** + * @generated from message test.NewWidget + */ +export type NewWidget = Message<"test.NewWidget"> & { + /** + * @generated from field: string name = 1; + */ + name: string; + + /** + * @generated from field: int32 height = 2; + */ + height: number; + + /** + * @generated from field: int32 version = 3; + */ + version: number; +}; + +/** + * Describes the message test.NewWidget. + * Use `create(NewWidgetSchema)` to create a new message. + */ +export const NewWidgetSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_newwidget, 0); + diff --git a/test/schemaregistry/serde/test/test_pb.ts b/test/schemaregistry/serde/test/test_pb.ts new file mode 100644 index 00000000..09126158 --- /dev/null +++ b/test/schemaregistry/serde/test/test_pb.ts @@ -0,0 +1,102 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/test.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import { file_google_protobuf_descriptor } from "@bufbuild/protobuf/wkt"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/test.proto. + */ +export const file_test_schemaregistry_serde_test: GenFile = /*@__PURE__*/ + fileDesc("CiR0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL3Rlc3QucHJvdG8SBHRlc3QiyAIKC1Rlc3RNZXNzYWdlEhMKC3Rlc3Rfc3RyaW5nGAEgASgJEhEKCXRlc3RfYm9vbBgCIAEoCBISCgp0ZXN0X2J5dGVzGAMgASgMEhMKC3Rlc3RfZG91YmxlGAQgASgBEhIKCnRlc3RfZmxvYXQYBSABKAISFAoMdGVzdF9maXhlZDMyGAYgASgHEhQKDHRlc3RfZml4ZWQ2NBgHIAEoBhISCgp0ZXN0X2ludDMyGAggASgFEhIKCnRlc3RfaW50NjQYCSABKAMSFQoNdGVzdF9zZml4ZWQzMhgKIAEoDxIVCg10ZXN0X3NmaXhlZDY0GAsgASgQEhMKC3Rlc3Rfc2ludDMyGAwgASgREhMKC3Rlc3Rfc2ludDY0GA0gASgSEhMKC3Rlc3RfdWludDMyGA4gASgNEhMKC3Rlc3RfdWludDY0GA8gASgEQglaBy4uL3Rlc3RiBnByb3RvMw", [file_google_protobuf_descriptor]); + +/** + * @generated from message test.TestMessage + */ +export type TestMessage = Message<"test.TestMessage"> & { + /** + * @generated from field: string test_string = 1; + */ + testString: string; + + /** + * @generated from field: bool test_bool = 2; + */ + testBool: boolean; + + /** + * @generated from field: bytes test_bytes = 3; + */ + testBytes: Uint8Array; + + /** + * @generated from field: double test_double = 4; + */ + testDouble: number; + + /** + * @generated from field: float test_float = 5; + */ + testFloat: number; + + /** + * @generated from field: fixed32 test_fixed32 = 6; + */ + testFixed32: number; + + /** + * @generated from field: fixed64 test_fixed64 = 7; + */ + testFixed64: bigint; + + /** + * @generated from field: int32 test_int32 = 8; + */ + testInt32: number; + + /** + * @generated from field: int64 test_int64 = 9; + */ + testInt64: bigint; + + /** + * @generated from field: sfixed32 test_sfixed32 = 10; + */ + testSfixed32: number; + + /** + * @generated from field: sfixed64 test_sfixed64 = 11; + */ + testSfixed64: bigint; + + /** + * @generated from field: sint32 test_sint32 = 12; + */ + testSint32: number; + + /** + * @generated from field: sint64 test_sint64 = 13; + */ + testSint64: bigint; + + /** + * @generated from field: uint32 test_uint32 = 14; + */ + testUint32: number; + + /** + * @generated from field: uint64 test_uint64 = 15; + */ + testUint64: bigint; +}; + +/** + * Describes the message test.TestMessage. + * Use `create(TestMessageSchema)` to create a new message. + */ +export const TestMessageSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_test, 0); + diff --git a/test/schemaregistry/serde/test/widget_pb.ts b/test/schemaregistry/serde/test/widget_pb.ts new file mode 100644 index 00000000..a8924d86 --- /dev/null +++ b/test/schemaregistry/serde/test/widget_pb.ts @@ -0,0 +1,41 @@ +// @generated by protoc-gen-es v2.0.0 with parameter "target=ts" +// @generated from file test/schemaregistry/serde/widget.proto (package test, syntax proto3) +/* eslint-disable */ + +import type { GenFile, GenMessage } from "@bufbuild/protobuf/codegenv1"; +import { fileDesc, messageDesc } from "@bufbuild/protobuf/codegenv1"; +import type { Message } from "@bufbuild/protobuf"; + +/** + * Describes the file test/schemaregistry/serde/widget.proto. + */ +export const file_test_schemaregistry_serde_widget: GenFile = /*@__PURE__*/ + fileDesc("CiZ0ZXN0L3NjaGVtYXJlZ2lzdHJ5L3NlcmRlL3dpZGdldC5wcm90bxIEdGVzdCI1CgZXaWRnZXQSDAoEbmFtZRgBIAEoCRIMCgRzaXplGAIgASgFEg8KB3ZlcnNpb24YAyABKAVCCVoHLi4vdGVzdGIGcHJvdG8z"); + +/** + * @generated from message test.Widget + */ +export type Widget = Message<"test.Widget"> & { + /** + * @generated from field: string name = 1; + */ + name: string; + + /** + * @generated from field: int32 size = 2; + */ + size: number; + + /** + * @generated from field: int32 version = 3; + */ + version: number; +}; + +/** + * Describes the message test.Widget. + * Use `create(WidgetSchema)` to create a new message. + */ +export const WidgetSchema: GenMessage = /*@__PURE__*/ + messageDesc(file_test_schemaregistry_serde_widget, 0); + diff --git a/tsconfig.json b/tsconfig.json index 52031e99..b3650ce2 100644 --- a/tsconfig.json +++ b/tsconfig.json @@ -4,12 +4,12 @@ "lib": [ "es2021", "dom" ], - "module": "commonjs", + "module": "preserve", "target": "es2021", "strict": true, "esModuleInterop": true, "forceConsistentCasingInFileNames": true, - "moduleResolution": "node16", + "moduleResolution": "bundler", "allowUnusedLabels": false, "allowUnreachableCode": false, "noFallthroughCasesInSwitch": true, @@ -18,10 +18,11 @@ "noPropertyAccessFromIndexSignature": true, "noUnusedLocals": true, "useUnknownInCatchVariables": true, - "types": ["node_modules/@types/node", "jest"], + "types": ["node_modules/@types/node"], "typeRoots": ["."], "noEmit": true, - "resolveJsonModule": true + "resolveJsonModule": true, + "skipLibCheck": true }, "include": [ "index.d.ts", From 7e09cb7527fd2cd2963407fcf5c8a137d8722d97 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 12 Sep 2024 11:11:44 -0700 Subject: [PATCH 189/224] Separate SR into a different workspace (#78) Co-authored-by: Milind L --- .npmignore | 1 + eslint.config.js | 2 +- package-lock.json | 2993 ++++++++--------- package.json | 30 +- .../Makefile.schemaregistry | 14 +- schemaregistry/jest.config.js | 6 + schemaregistry/package.json | 59 + schemaregistry/tsconfig.json | 31 + tsconfig.json | 34 +- 9 files changed, 1472 insertions(+), 1698 deletions(-) rename Makefile.schemaregistry => schemaregistry/Makefile.schemaregistry (50%) create mode 100644 schemaregistry/jest.config.js create mode 100644 schemaregistry/package.json create mode 100644 schemaregistry/tsconfig.json diff --git a/.npmignore b/.npmignore index 5130a557..f551518a 100644 --- a/.npmignore +++ b/.npmignore @@ -5,6 +5,7 @@ deps/* .gitmodules Dockerfile deps/librdkafka/config.h +schemaregistry build .github .vscode diff --git a/eslint.config.js b/eslint.config.js index 823b769b..4f83933a 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -54,7 +54,7 @@ module.exports = ts.config( }, ...ts.configs.recommended.map((config) => ({ ...config, - ignores: ["**/*.js"], + ignores: ["**/*.js", "types/rdkafka.d.ts", "types/kafkajs.d.ts", "types/config.d.ts"], rules: { ...config.rules, "prefer-const": "warn", diff --git a/package-lock.json b/package-lock.json index aacda59a..bce3b217 100644 --- a/package-lock.json +++ b/package-lock.json @@ -9,30 +9,13 @@ "version": "v0.1.16-devel", "hasInstallScript": true, "license": "MIT", + "workspaces": [ + "schemaregistry" + ], "dependencies": { - "@aws-sdk/client-kms": "^3.637.0", - "@azure/identity": "^4.4.1", - "@azure/keyvault-keys": "^4.8.0", - "@bufbuild/protobuf": "^2.0.0", - "@criteria/json-schema": "^0.10.0", - "@criteria/json-schema-validation": "^0.10.0", - "@google-cloud/kms": "^4.5.0", - "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", - "@smithy/types": "^3.3.0", - "@types/validator": "^13.12.0", - "ajv": "^8.17.1", - "async-mutex": "^0.5.0", - "avsc": "^5.7.7", - "axios": "^1.7.3", "bindings": "^1.3.1", - "json-stringify-deterministic": "^1.0.12", - "jsonata": "^2.0.5", - "lru-cache": "^11.0.0", - "nan": "^2.17.0", - "node-vault": "^0.10.2", - "ts-jest": "^29.2.4", - "validator": "^13.12.0" + "nan": "^2.17.0" }, "devDependencies": { "@bufbuild/buf": "^1.37.0", @@ -58,6 +41,7 @@ "version": "2.3.0", "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.3.0.tgz", "integrity": "sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw==", + "dev": true, "dependencies": { "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.24" @@ -182,49 +166,49 @@ } }, "node_modules/@aws-sdk/client-kms": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-kms/-/client-kms-3.637.0.tgz", - "integrity": "sha512-bqppLpmIPl6eZkZx/9axnr4CBbhtrRKe3LffW8320DlwCqP3zU+c500vXMjEgYdrAqkqOFyDY/FYMAgZhtHVCQ==", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-kms/-/client-kms-3.650.0.tgz", + "integrity": "sha512-7J/DW9/+CAdCop36IhiGGPLx4rclMyzQrI95EIN3FU5dTUFZ8aDHN+euTMfVSy7dfbwCsTxESx5/U/ZeOrFvPA==", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/client-sso-oidc": "3.637.0", - "@aws-sdk/client-sts": "3.637.0", - "@aws-sdk/core": "3.635.0", - "@aws-sdk/credential-provider-node": "3.637.0", - "@aws-sdk/middleware-host-header": "3.620.0", - "@aws-sdk/middleware-logger": "3.609.0", - "@aws-sdk/middleware-recursion-detection": "3.620.0", - "@aws-sdk/middleware-user-agent": "3.637.0", - "@aws-sdk/region-config-resolver": "3.614.0", - "@aws-sdk/types": "3.609.0", - "@aws-sdk/util-endpoints": "3.637.0", - "@aws-sdk/util-user-agent-browser": "3.609.0", - "@aws-sdk/util-user-agent-node": "3.614.0", - "@smithy/config-resolver": "^3.0.5", - "@smithy/core": "^2.4.0", - "@smithy/fetch-http-handler": "^3.2.4", - "@smithy/hash-node": "^3.0.3", - "@smithy/invalid-dependency": "^3.0.3", - "@smithy/middleware-content-length": "^3.0.5", - "@smithy/middleware-endpoint": "^3.1.0", - "@smithy/middleware-retry": "^3.0.15", - "@smithy/middleware-serde": "^3.0.3", - "@smithy/middleware-stack": "^3.0.3", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/node-http-handler": "^3.1.4", - "@smithy/protocol-http": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/url-parser": "^3.0.3", + "@aws-sdk/client-sso-oidc": "3.650.0", + "@aws-sdk/client-sts": "3.650.0", + "@aws-sdk/core": "3.649.0", + "@aws-sdk/credential-provider-node": "3.650.0", + "@aws-sdk/middleware-host-header": "3.649.0", + "@aws-sdk/middleware-logger": "3.649.0", + "@aws-sdk/middleware-recursion-detection": "3.649.0", + "@aws-sdk/middleware-user-agent": "3.649.0", + "@aws-sdk/region-config-resolver": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@aws-sdk/util-endpoints": "3.649.0", + "@aws-sdk/util-user-agent-browser": "3.649.0", + "@aws-sdk/util-user-agent-node": "3.649.0", + "@smithy/config-resolver": "^3.0.6", + "@smithy/core": "^2.4.1", + "@smithy/fetch-http-handler": "^3.2.5", + "@smithy/hash-node": "^3.0.4", + "@smithy/invalid-dependency": "^3.0.4", + "@smithy/middleware-content-length": "^3.0.6", + "@smithy/middleware-endpoint": "^3.1.1", + "@smithy/middleware-retry": "^3.0.16", + "@smithy/middleware-serde": "^3.0.4", + "@smithy/middleware-stack": "^3.0.4", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/node-http-handler": "^3.2.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/url-parser": "^3.0.4", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.15", - "@smithy/util-defaults-mode-node": "^3.0.15", - "@smithy/util-endpoints": "^2.0.5", - "@smithy/util-middleware": "^3.0.3", - "@smithy/util-retry": "^3.0.3", + "@smithy/util-defaults-mode-browser": "^3.0.16", + "@smithy/util-defaults-mode-node": "^3.0.16", + "@smithy/util-endpoints": "^2.1.0", + "@smithy/util-middleware": "^3.0.4", + "@smithy/util-retry": "^3.0.4", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -233,46 +217,46 @@ } }, "node_modules/@aws-sdk/client-sso": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso/-/client-sso-3.637.0.tgz", - "integrity": "sha512-+KjLvgX5yJYROWo3TQuwBJlHCY0zz9PsLuEolmXQn0BVK1L/m9GteZHtd+rEdAoDGBpE0Xqjy1oz5+SmtsaRUw==", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso/-/client-sso-3.650.0.tgz", + "integrity": "sha512-YKm14gCMChD/jlCisFlsVqB8HJujR41bl4Fup2crHwNJxhD/9LTnzwMiVVlBqlXr41Sfa6fSxczX2AMP8NM14A==", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/core": "3.635.0", - "@aws-sdk/middleware-host-header": "3.620.0", - "@aws-sdk/middleware-logger": "3.609.0", - "@aws-sdk/middleware-recursion-detection": "3.620.0", - "@aws-sdk/middleware-user-agent": "3.637.0", - "@aws-sdk/region-config-resolver": "3.614.0", - "@aws-sdk/types": "3.609.0", - "@aws-sdk/util-endpoints": "3.637.0", - "@aws-sdk/util-user-agent-browser": "3.609.0", - "@aws-sdk/util-user-agent-node": "3.614.0", - "@smithy/config-resolver": "^3.0.5", - "@smithy/core": "^2.4.0", - "@smithy/fetch-http-handler": "^3.2.4", - "@smithy/hash-node": "^3.0.3", - "@smithy/invalid-dependency": "^3.0.3", - "@smithy/middleware-content-length": "^3.0.5", - "@smithy/middleware-endpoint": "^3.1.0", - "@smithy/middleware-retry": "^3.0.15", - "@smithy/middleware-serde": "^3.0.3", - "@smithy/middleware-stack": "^3.0.3", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/node-http-handler": "^3.1.4", - "@smithy/protocol-http": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/url-parser": "^3.0.3", + "@aws-sdk/core": "3.649.0", + "@aws-sdk/middleware-host-header": "3.649.0", + "@aws-sdk/middleware-logger": "3.649.0", + "@aws-sdk/middleware-recursion-detection": "3.649.0", + "@aws-sdk/middleware-user-agent": "3.649.0", + "@aws-sdk/region-config-resolver": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@aws-sdk/util-endpoints": "3.649.0", + "@aws-sdk/util-user-agent-browser": "3.649.0", + "@aws-sdk/util-user-agent-node": "3.649.0", + "@smithy/config-resolver": "^3.0.6", + "@smithy/core": "^2.4.1", + "@smithy/fetch-http-handler": "^3.2.5", + "@smithy/hash-node": "^3.0.4", + "@smithy/invalid-dependency": "^3.0.4", + "@smithy/middleware-content-length": "^3.0.6", + "@smithy/middleware-endpoint": "^3.1.1", + "@smithy/middleware-retry": "^3.0.16", + "@smithy/middleware-serde": "^3.0.4", + "@smithy/middleware-stack": "^3.0.4", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/node-http-handler": "^3.2.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/url-parser": "^3.0.4", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.15", - "@smithy/util-defaults-mode-node": "^3.0.15", - "@smithy/util-endpoints": "^2.0.5", - "@smithy/util-middleware": "^3.0.3", - "@smithy/util-retry": "^3.0.3", + "@smithy/util-defaults-mode-browser": "^3.0.16", + "@smithy/util-defaults-mode-node": "^3.0.16", + "@smithy/util-endpoints": "^2.1.0", + "@smithy/util-middleware": "^3.0.4", + "@smithy/util-retry": "^3.0.4", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -281,47 +265,47 @@ } }, "node_modules/@aws-sdk/client-sso-oidc": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso-oidc/-/client-sso-oidc-3.637.0.tgz", - "integrity": "sha512-27bHALN6Qb6m6KZmPvRieJ/QRlj1lyac/GT2Rn5kJpre8Mpp+yxrtvp3h9PjNBty4lCeFEENfY4dGNSozBuBcw==", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso-oidc/-/client-sso-oidc-3.650.0.tgz", + "integrity": "sha512-6J7IS0f8ovhvbIAZaynOYP+jPX8344UlTjwHxjaXHgFvI8axu3+NslKtEEV5oHLhgzDvrKbinsu5lgE2n4Sqng==", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/core": "3.635.0", - "@aws-sdk/credential-provider-node": "3.637.0", - "@aws-sdk/middleware-host-header": "3.620.0", - "@aws-sdk/middleware-logger": "3.609.0", - "@aws-sdk/middleware-recursion-detection": "3.620.0", - "@aws-sdk/middleware-user-agent": "3.637.0", - "@aws-sdk/region-config-resolver": "3.614.0", - "@aws-sdk/types": "3.609.0", - "@aws-sdk/util-endpoints": "3.637.0", - "@aws-sdk/util-user-agent-browser": "3.609.0", - "@aws-sdk/util-user-agent-node": "3.614.0", - "@smithy/config-resolver": "^3.0.5", - "@smithy/core": "^2.4.0", - "@smithy/fetch-http-handler": "^3.2.4", - "@smithy/hash-node": "^3.0.3", - "@smithy/invalid-dependency": "^3.0.3", - "@smithy/middleware-content-length": "^3.0.5", - "@smithy/middleware-endpoint": "^3.1.0", - "@smithy/middleware-retry": "^3.0.15", - "@smithy/middleware-serde": "^3.0.3", - "@smithy/middleware-stack": "^3.0.3", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/node-http-handler": "^3.1.4", - "@smithy/protocol-http": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/url-parser": "^3.0.3", + "@aws-sdk/core": "3.649.0", + "@aws-sdk/credential-provider-node": "3.650.0", + "@aws-sdk/middleware-host-header": "3.649.0", + "@aws-sdk/middleware-logger": "3.649.0", + "@aws-sdk/middleware-recursion-detection": "3.649.0", + "@aws-sdk/middleware-user-agent": "3.649.0", + "@aws-sdk/region-config-resolver": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@aws-sdk/util-endpoints": "3.649.0", + "@aws-sdk/util-user-agent-browser": "3.649.0", + "@aws-sdk/util-user-agent-node": "3.649.0", + "@smithy/config-resolver": "^3.0.6", + "@smithy/core": "^2.4.1", + "@smithy/fetch-http-handler": "^3.2.5", + "@smithy/hash-node": "^3.0.4", + "@smithy/invalid-dependency": "^3.0.4", + "@smithy/middleware-content-length": "^3.0.6", + "@smithy/middleware-endpoint": "^3.1.1", + "@smithy/middleware-retry": "^3.0.16", + "@smithy/middleware-serde": "^3.0.4", + "@smithy/middleware-stack": "^3.0.4", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/node-http-handler": "^3.2.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/url-parser": "^3.0.4", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.15", - "@smithy/util-defaults-mode-node": "^3.0.15", - "@smithy/util-endpoints": "^2.0.5", - "@smithy/util-middleware": "^3.0.3", - "@smithy/util-retry": "^3.0.3", + "@smithy/util-defaults-mode-browser": "^3.0.16", + "@smithy/util-defaults-mode-node": "^3.0.16", + "@smithy/util-endpoints": "^2.1.0", + "@smithy/util-middleware": "^3.0.4", + "@smithy/util-retry": "^3.0.4", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -329,52 +313,52 @@ "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sts": "^3.637.0" + "@aws-sdk/client-sts": "^3.650.0" } }, "node_modules/@aws-sdk/client-sts": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-sts/-/client-sts-3.637.0.tgz", - "integrity": "sha512-xUi7x4qDubtA8QREtlblPuAcn91GS/09YVEY/RwU7xCY0aqGuFwgszAANlha4OUIqva8oVj2WO4gJuG+iaSnhw==", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/client-sts/-/client-sts-3.650.0.tgz", + "integrity": "sha512-ISK0ZQYA7O5/WYgslpWy956lUBudGC9d7eL0FFbiL0j50N80Gx3RUv22ezvZgxJWE0W3DqNr4CE19sPYn4Lw8g==", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/client-sso-oidc": "3.637.0", - "@aws-sdk/core": "3.635.0", - "@aws-sdk/credential-provider-node": "3.637.0", - "@aws-sdk/middleware-host-header": "3.620.0", - "@aws-sdk/middleware-logger": "3.609.0", - "@aws-sdk/middleware-recursion-detection": "3.620.0", - "@aws-sdk/middleware-user-agent": "3.637.0", - "@aws-sdk/region-config-resolver": "3.614.0", - "@aws-sdk/types": "3.609.0", - "@aws-sdk/util-endpoints": "3.637.0", - "@aws-sdk/util-user-agent-browser": "3.609.0", - "@aws-sdk/util-user-agent-node": "3.614.0", - "@smithy/config-resolver": "^3.0.5", - "@smithy/core": "^2.4.0", - "@smithy/fetch-http-handler": "^3.2.4", - "@smithy/hash-node": "^3.0.3", - "@smithy/invalid-dependency": "^3.0.3", - "@smithy/middleware-content-length": "^3.0.5", - "@smithy/middleware-endpoint": "^3.1.0", - "@smithy/middleware-retry": "^3.0.15", - "@smithy/middleware-serde": "^3.0.3", - "@smithy/middleware-stack": "^3.0.3", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/node-http-handler": "^3.1.4", - "@smithy/protocol-http": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/url-parser": "^3.0.3", + "@aws-sdk/client-sso-oidc": "3.650.0", + "@aws-sdk/core": "3.649.0", + "@aws-sdk/credential-provider-node": "3.650.0", + "@aws-sdk/middleware-host-header": "3.649.0", + "@aws-sdk/middleware-logger": "3.649.0", + "@aws-sdk/middleware-recursion-detection": "3.649.0", + "@aws-sdk/middleware-user-agent": "3.649.0", + "@aws-sdk/region-config-resolver": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@aws-sdk/util-endpoints": "3.649.0", + "@aws-sdk/util-user-agent-browser": "3.649.0", + "@aws-sdk/util-user-agent-node": "3.649.0", + "@smithy/config-resolver": "^3.0.6", + "@smithy/core": "^2.4.1", + "@smithy/fetch-http-handler": "^3.2.5", + "@smithy/hash-node": "^3.0.4", + "@smithy/invalid-dependency": "^3.0.4", + "@smithy/middleware-content-length": "^3.0.6", + "@smithy/middleware-endpoint": "^3.1.1", + "@smithy/middleware-retry": "^3.0.16", + "@smithy/middleware-serde": "^3.0.4", + "@smithy/middleware-stack": "^3.0.4", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/node-http-handler": "^3.2.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/url-parser": "^3.0.4", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.15", - "@smithy/util-defaults-mode-node": "^3.0.15", - "@smithy/util-endpoints": "^2.0.5", - "@smithy/util-middleware": "^3.0.3", - "@smithy/util-retry": "^3.0.3", + "@smithy/util-defaults-mode-browser": "^3.0.16", + "@smithy/util-defaults-mode-node": "^3.0.16", + "@smithy/util-endpoints": "^2.1.0", + "@smithy/util-middleware": "^3.0.4", + "@smithy/util-retry": "^3.0.4", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -383,18 +367,18 @@ } }, "node_modules/@aws-sdk/core": { - "version": "3.635.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/core/-/core-3.635.0.tgz", - "integrity": "sha512-i1x/E/sgA+liUE1XJ7rj1dhyXpAKO1UKFUcTTHXok2ARjWTvszHnSXMOsB77aPbmn0fUp1JTx2kHUAZ1LVt5Bg==", - "dependencies": { - "@smithy/core": "^2.4.0", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/property-provider": "^3.1.3", - "@smithy/protocol-http": "^4.1.0", - "@smithy/signature-v4": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/util-middleware": "^3.0.3", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/core/-/core-3.649.0.tgz", + "integrity": "sha512-dheG/X2y25RHE7K+TlS32kcy7TgDg1OpWV44BQRoE0OBPAWmFR1D1qjjTZ7WWrdqRPKzcnDj1qED8ncyncOX8g==", + "dependencies": { + "@smithy/core": "^2.4.1", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/property-provider": "^3.1.4", + "@smithy/protocol-http": "^4.1.1", + "@smithy/signature-v4": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/util-middleware": "^3.0.4", "fast-xml-parser": "4.4.1", "tslib": "^2.6.2" }, @@ -403,13 +387,13 @@ } }, "node_modules/@aws-sdk/credential-provider-env": { - "version": "3.620.1", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-env/-/credential-provider-env-3.620.1.tgz", - "integrity": "sha512-ExuILJ2qLW5ZO+rgkNRj0xiAipKT16Rk77buvPP8csR7kkCflT/gXTyzRe/uzIiETTxM7tr8xuO9MP/DQXqkfg==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-env/-/credential-provider-env-3.649.0.tgz", + "integrity": "sha512-tViwzM1dauksA3fdRjsg0T8mcHklDa8EfveyiQKK6pUJopkqV6FQx+X5QNda0t/LrdEVlFZvwHNdXqOEfc83TA==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -417,18 +401,18 @@ } }, "node_modules/@aws-sdk/credential-provider-http": { - "version": "3.635.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-http/-/credential-provider-http-3.635.0.tgz", - "integrity": "sha512-iJyRgEjOCQlBMXqtwPLIKYc7Bsc6nqjrZybdMDenPDa+kmLg7xh8LxHsu9088e+2/wtLicE34FsJJIfzu3L82g==", - "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/fetch-http-handler": "^3.2.4", - "@smithy/node-http-handler": "^3.1.4", - "@smithy/property-provider": "^3.1.3", - "@smithy/protocol-http": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/util-stream": "^3.1.3", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-http/-/credential-provider-http-3.649.0.tgz", + "integrity": "sha512-ODAJ+AJJq6ozbns6ejGbicpsQ0dyMOpnGlg0J9J0jITQ05DKQZ581hdB8APDOZ9N8FstShP6dLZflSj8jb5fNA==", + "dependencies": { + "@aws-sdk/types": "3.649.0", + "@smithy/fetch-http-handler": "^3.2.5", + "@smithy/node-http-handler": "^3.2.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/protocol-http": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/util-stream": "^3.1.4", "tslib": "^2.6.2" }, "engines": { @@ -436,45 +420,45 @@ } }, "node_modules/@aws-sdk/credential-provider-ini": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-ini/-/credential-provider-ini-3.637.0.tgz", - "integrity": "sha512-h+PFCWfZ0Q3Dx84SppET/TFpcQHmxFW8/oV9ArEvMilw4EBN+IlxgbL0CnHwjHW64szcmrM0mbebjEfHf4FXmw==", - "dependencies": { - "@aws-sdk/credential-provider-env": "3.620.1", - "@aws-sdk/credential-provider-http": "3.635.0", - "@aws-sdk/credential-provider-process": "3.620.1", - "@aws-sdk/credential-provider-sso": "3.637.0", - "@aws-sdk/credential-provider-web-identity": "3.621.0", - "@aws-sdk/types": "3.609.0", - "@smithy/credential-provider-imds": "^3.2.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-ini/-/credential-provider-ini-3.650.0.tgz", + "integrity": "sha512-x2M9buZxIsKuUbuDgkGHhAKYBpn0/rYdKlwuFuOhXyyAcnhvPj0lgNF2KE4ld/GF1mKr7FF/uV3G9lM6PFaYmA==", + "dependencies": { + "@aws-sdk/credential-provider-env": "3.649.0", + "@aws-sdk/credential-provider-http": "3.649.0", + "@aws-sdk/credential-provider-process": "3.649.0", + "@aws-sdk/credential-provider-sso": "3.650.0", + "@aws-sdk/credential-provider-web-identity": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@smithy/credential-provider-imds": "^3.2.1", + "@smithy/property-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sts": "^3.637.0" + "@aws-sdk/client-sts": "^3.650.0" } }, "node_modules/@aws-sdk/credential-provider-node": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-node/-/credential-provider-node-3.637.0.tgz", - "integrity": "sha512-yoEhoxJJfs7sPVQ6Is939BDQJZpZCoUgKr/ySse4YKOZ24t4VqgHA6+wV7rYh+7IW24Rd91UTvEzSuHYTlxlNA==", - "dependencies": { - "@aws-sdk/credential-provider-env": "3.620.1", - "@aws-sdk/credential-provider-http": "3.635.0", - "@aws-sdk/credential-provider-ini": "3.637.0", - "@aws-sdk/credential-provider-process": "3.620.1", - "@aws-sdk/credential-provider-sso": "3.637.0", - "@aws-sdk/credential-provider-web-identity": "3.621.0", - "@aws-sdk/types": "3.609.0", - "@smithy/credential-provider-imds": "^3.2.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-node/-/credential-provider-node-3.650.0.tgz", + "integrity": "sha512-uBra5YjzS/gWSekAogfqJfY6c+oKQkkou7Cjc4d/cpMNvQtF1IBdekJ7NaE1RfsDEz3uH1+Myd07YWZAJo/2Qw==", + "dependencies": { + "@aws-sdk/credential-provider-env": "3.649.0", + "@aws-sdk/credential-provider-http": "3.649.0", + "@aws-sdk/credential-provider-ini": "3.650.0", + "@aws-sdk/credential-provider-process": "3.649.0", + "@aws-sdk/credential-provider-sso": "3.650.0", + "@aws-sdk/credential-provider-web-identity": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@smithy/credential-provider-imds": "^3.2.1", + "@smithy/property-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -482,14 +466,14 @@ } }, "node_modules/@aws-sdk/credential-provider-process": { - "version": "3.620.1", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-process/-/credential-provider-process-3.620.1.tgz", - "integrity": "sha512-hWqFMidqLAkaV9G460+1at6qa9vySbjQKKc04p59OT7lZ5cO5VH5S4aI05e+m4j364MBROjjk2ugNvfNf/8ILg==", - "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-process/-/credential-provider-process-3.649.0.tgz", + "integrity": "sha512-6VYPQpEVpU+6DDS/gLoI40ppuNM5RPIEprK30qZZxnhTr5wyrGOeJ7J7wbbwPOZ5dKwta290BiJDU2ipV8Y9BQ==", + "dependencies": { + "@aws-sdk/types": "3.649.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -497,16 +481,16 @@ } }, "node_modules/@aws-sdk/credential-provider-sso": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-sso/-/credential-provider-sso-3.637.0.tgz", - "integrity": "sha512-Mvz+h+e62/tl+dVikLafhv+qkZJ9RUb8l2YN/LeKMWkxQylPT83CPk9aimVhCV89zth1zpREArl97+3xsfgQvA==", - "dependencies": { - "@aws-sdk/client-sso": "3.637.0", - "@aws-sdk/token-providers": "3.614.0", - "@aws-sdk/types": "3.609.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", + "version": "3.650.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-sso/-/credential-provider-sso-3.650.0.tgz", + "integrity": "sha512-069nkhcwximbvyGiAC6Fr2G+yrG/p1S3NQ5BZ2cMzB1hgUKo6TvgFK7nriYI4ljMQ+UWxqPwIdTqiUmn2iJmhg==", + "dependencies": { + "@aws-sdk/client-sso": "3.650.0", + "@aws-sdk/token-providers": "3.649.0", + "@aws-sdk/types": "3.649.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -514,30 +498,30 @@ } }, "node_modules/@aws-sdk/credential-provider-web-identity": { - "version": "3.621.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-web-identity/-/credential-provider-web-identity-3.621.0.tgz", - "integrity": "sha512-w7ASSyfNvcx7+bYGep3VBgC3K6vEdLmlpjT7nSIHxxQf+WSdvy+HynwJosrpZax0sK5q0D1Jpn/5q+r5lwwW6w==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-web-identity/-/credential-provider-web-identity-3.649.0.tgz", + "integrity": "sha512-XVk3WsDa0g3kQFPmnCH/LaCtGY/0R2NDv7gscYZSXiBZcG/fixasglTprgWSp8zcA0t7tEIGu9suyjz8ZwhymQ==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sts": "^3.621.0" + "@aws-sdk/client-sts": "^3.649.0" } }, "node_modules/@aws-sdk/middleware-host-header": { - "version": "3.620.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-host-header/-/middleware-host-header-3.620.0.tgz", - "integrity": "sha512-VMtPEZwqYrII/oUkffYsNWY9PZ9xpNJpMgmyU0rlDQ25O1c0Hk3fJmZRe6pEkAJ0omD7kLrqGl1DUjQVxpd/Rg==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-host-header/-/middleware-host-header-3.649.0.tgz", + "integrity": "sha512-PjAe2FocbicHVgNNwdSZ05upxIO7AgTPFtQLpnIAmoyzMcgv/zNB5fBn3uAnQSAeEPPCD+4SYVEUD1hw1ZBvEg==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/protocol-http": "^4.1.0", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -545,12 +529,12 @@ } }, "node_modules/@aws-sdk/middleware-logger": { - "version": "3.609.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-logger/-/middleware-logger-3.609.0.tgz", - "integrity": "sha512-S62U2dy4jMDhDFDK5gZ4VxFdWzCtLzwbYyFZx2uvPYTECkepLUfzLic2BHg2Qvtu4QjX+oGE3P/7fwaGIsGNuQ==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-logger/-/middleware-logger-3.649.0.tgz", + "integrity": "sha512-qdqRx6q7lYC6KL/NT9x3ShTL0TBuxdkCczGzHzY3AnOoYUjnCDH7Vlq867O6MAvb4EnGNECFzIgtkZkQ4FhY5w==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -558,13 +542,13 @@ } }, "node_modules/@aws-sdk/middleware-recursion-detection": { - "version": "3.620.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-recursion-detection/-/middleware-recursion-detection-3.620.0.tgz", - "integrity": "sha512-nh91S7aGK3e/o1ck64sA/CyoFw+gAYj2BDOnoNa6ouyCrVJED96ZXWbhye/fz9SgmNUZR2g7GdVpiLpMKZoI5w==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-recursion-detection/-/middleware-recursion-detection-3.649.0.tgz", + "integrity": "sha512-IPnO4wlmaLRf6IYmJW2i8gJ2+UPXX0hDRv1it7Qf8DpBW+lGyF2rnoN7NrFX0WIxdGOlJF1RcOr/HjXb2QeXfQ==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/protocol-http": "^4.1.0", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -572,14 +556,14 @@ } }, "node_modules/@aws-sdk/middleware-user-agent": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-user-agent/-/middleware-user-agent-3.637.0.tgz", - "integrity": "sha512-EYo0NE9/da/OY8STDsK2LvM4kNa79DBsf4YVtaG4P5pZ615IeFsD8xOHZeuJmUrSMlVQ8ywPRX7WMucUybsKug==", - "dependencies": { - "@aws-sdk/types": "3.609.0", - "@aws-sdk/util-endpoints": "3.637.0", - "@smithy/protocol-http": "^4.1.0", - "@smithy/types": "^3.3.0", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-user-agent/-/middleware-user-agent-3.649.0.tgz", + "integrity": "sha512-q6sO10dnCXoxe9thobMJxekhJumzd1j6dxcE1+qJdYKHJr6yYgWbogJqrLCpWd30w0lEvnuAHK8lN2kWLdJxJw==", + "dependencies": { + "@aws-sdk/types": "3.649.0", + "@aws-sdk/util-endpoints": "3.649.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -587,15 +571,15 @@ } }, "node_modules/@aws-sdk/region-config-resolver": { - "version": "3.614.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/region-config-resolver/-/region-config-resolver-3.614.0.tgz", - "integrity": "sha512-vDCeMXvic/LU0KFIUjpC3RiSTIkkvESsEfbVHiHH0YINfl8HnEqR5rj+L8+phsCeVg2+LmYwYxd5NRz4PHxt5g==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/region-config-resolver/-/region-config-resolver-3.649.0.tgz", + "integrity": "sha512-xURBvdQXvRvca5Du8IlC5FyCj3pkw8Z75+373J3Wb+vyg8GjD14HfKk1Je1HCCQDyIE9VB/scYDcm9ri0ppePw==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/types": "^3.4.0", "@smithy/util-config-provider": "^3.0.0", - "@smithy/util-middleware": "^3.0.3", + "@smithy/util-middleware": "^3.0.4", "tslib": "^2.6.2" }, "engines": { @@ -603,29 +587,29 @@ } }, "node_modules/@aws-sdk/token-providers": { - "version": "3.614.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/token-providers/-/token-providers-3.614.0.tgz", - "integrity": "sha512-okItqyY6L9IHdxqs+Z116y5/nda7rHxLvROxtAJdLavWTYDydxrZstImNgGWTeVdmc0xX2gJCI77UYUTQWnhRw==", - "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/property-provider": "^3.1.3", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/token-providers/-/token-providers-3.649.0.tgz", + "integrity": "sha512-ZBqr+JuXI9RiN+4DSZykMx5gxpL8Dr3exIfFhxMiwAP3DQojwl0ub8ONjMuAjq9OvmX6n+jHZL6fBnNgnNFC8w==", + "dependencies": { + "@aws-sdk/types": "3.649.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sso-oidc": "^3.614.0" + "@aws-sdk/client-sso-oidc": "^3.649.0" } }, "node_modules/@aws-sdk/types": { - "version": "3.609.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/types/-/types-3.609.0.tgz", - "integrity": "sha512-+Tqnh9w0h2LcrUsdXyT1F8mNhXz+tVYBtP19LpeEGntmvHwa2XzvLUCWpoIAIVsHp5+HdB2X9Sn0KAtmbFXc2Q==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/types/-/types-3.649.0.tgz", + "integrity": "sha512-PuPw8RysbhJNlaD2d/PzOTf8sbf4Dsn2b7hwyGh7YVG3S75yTpxSAZxrnhKsz9fStgqFmnw/jUfV/G+uQAeTVw==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -633,13 +617,13 @@ } }, "node_modules/@aws-sdk/util-endpoints": { - "version": "3.637.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-endpoints/-/util-endpoints-3.637.0.tgz", - "integrity": "sha512-pAqOKUHeVWHEXXDIp/qoMk/6jyxIb6GGjnK1/f8dKHtKIEs4tKsnnL563gceEvdad53OPXIt86uoevCcCzmBnw==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-endpoints/-/util-endpoints-3.649.0.tgz", + "integrity": "sha512-bZI1Wc3R/KibdDVWFxX/N4AoJFG4VJ92Dp4WYmOrVD6VPkb8jPz7ZeiYc7YwPl8NoDjYyPneBV0lEoK/V8OKAA==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/types": "^3.3.0", - "@smithy/util-endpoints": "^2.0.5", + "@aws-sdk/types": "3.649.0", + "@smithy/types": "^3.4.0", + "@smithy/util-endpoints": "^2.1.0", "tslib": "^2.6.2" }, "engines": { @@ -658,24 +642,24 @@ } }, "node_modules/@aws-sdk/util-user-agent-browser": { - "version": "3.609.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-browser/-/util-user-agent-browser-3.609.0.tgz", - "integrity": "sha512-fojPU+mNahzQ0YHYBsx0ZIhmMA96H+ZIZ665ObU9tl+SGdbLneVZVikGve+NmHTQwHzwkFsZYYnVKAkreJLAtA==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-browser/-/util-user-agent-browser-3.649.0.tgz", + "integrity": "sha512-IY43r256LhKAvdEVQO/FPdUyVpcZS5EVxh/WHVdNzuN1bNLoUK2rIzuZqVA0EGguvCxoXVmQv9m50GvG7cGktg==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/types": "^3.4.0", "bowser": "^2.11.0", "tslib": "^2.6.2" } }, "node_modules/@aws-sdk/util-user-agent-node": { - "version": "3.614.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-node/-/util-user-agent-node-3.614.0.tgz", - "integrity": "sha512-15ElZT88peoHnq5TEoEtZwoXTXRxNrk60TZNdpl/TUBJ5oNJ9Dqb5Z4ryb8ofN6nm9aFf59GVAerFDz8iUoHBA==", + "version": "3.649.0", + "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-node/-/util-user-agent-node-3.649.0.tgz", + "integrity": "sha512-x5DiLpZDG/AJmCIBnE3Xhpwy35QIo3WqNiOpw6ExVs1NydbM/e90zFPSfhME0FM66D/WorigvluBxxwjxDm/GA==", "dependencies": { - "@aws-sdk/types": "3.609.0", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/types": "^3.3.0", + "@aws-sdk/types": "3.649.0", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -842,41 +826,6 @@ "node": ">=18.0.0" } }, - "node_modules/@azure/core-rest-pipeline/node_modules/agent-base": { - "version": "7.1.1", - "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.1.tgz", - "integrity": "sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==", - "dependencies": { - "debug": "^4.3.4" - }, - "engines": { - "node": ">= 14" - } - }, - "node_modules/@azure/core-rest-pipeline/node_modules/http-proxy-agent": { - "version": "7.0.2", - "resolved": "https://registry.npmjs.org/http-proxy-agent/-/http-proxy-agent-7.0.2.tgz", - "integrity": "sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig==", - "dependencies": { - "agent-base": "^7.1.0", - "debug": "^4.3.4" - }, - "engines": { - "node": ">= 14" - } - }, - "node_modules/@azure/core-rest-pipeline/node_modules/https-proxy-agent": { - "version": "7.0.5", - "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz", - "integrity": "sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==", - "dependencies": { - "agent-base": "^7.0.2", - "debug": "4" - }, - "engines": { - "node": ">= 14" - } - }, "node_modules/@azure/core-tracing": { "version": "1.1.2", "resolved": "https://registry.npmjs.org/@azure/core-tracing/-/core-tracing-1.1.2.tgz", @@ -968,30 +917,30 @@ } }, "node_modules/@azure/msal-browser": { - "version": "3.21.0", - "resolved": "https://registry.npmjs.org/@azure/msal-browser/-/msal-browser-3.21.0.tgz", - "integrity": "sha512-BAwcFsVvOrYzKuUZHhFuvRykUmQGq6lDxst2qGnjxnpNZc3d/tnVPcmhgvUdeKl28VSE0ltgBzT3HkdpDtz9rg==", + "version": "3.23.0", + "resolved": "https://registry.npmjs.org/@azure/msal-browser/-/msal-browser-3.23.0.tgz", + "integrity": "sha512-+QgdMvaeEpdtgRTD7AHHq9aw8uga7mXVHV1KshO1RQ2uI5B55xJ4aEpGlg/ga3H+0arEVcRfT4ZVmX7QLXiCVw==", "dependencies": { - "@azure/msal-common": "14.14.1" + "@azure/msal-common": "14.14.2" }, "engines": { "node": ">=0.8.0" } }, "node_modules/@azure/msal-common": { - "version": "14.14.1", - "resolved": "https://registry.npmjs.org/@azure/msal-common/-/msal-common-14.14.1.tgz", - "integrity": "sha512-2Q3tqNz/PZLfSr8BvcHZVpRRfSn4MjGSqjj9J+HlBsmbf1Uu4P0WeXnemjTJwwx9KrmplsrN3UkZ/LPOR720rw==", + "version": "14.14.2", + "resolved": "https://registry.npmjs.org/@azure/msal-common/-/msal-common-14.14.2.tgz", + "integrity": "sha512-XV0P5kSNwDwCA/SjIxTe9mEAsKB0NqGNSuaVrkCCE2lAyBr/D6YtD80Vkdp4tjWnPFwjzkwldjr1xU/facOJog==", "engines": { "node": ">=0.8.0" } }, "node_modules/@azure/msal-node": { - "version": "2.13.0", - "resolved": "https://registry.npmjs.org/@azure/msal-node/-/msal-node-2.13.0.tgz", - "integrity": "sha512-DhP97ycs7qlCVzzzWGzJiwAFyFj5okno74E4FUZ61oCLfKh4IxA1kxirqzrWuYZWpBe9HVPL6GA4NvmlEOBN5Q==", + "version": "2.13.1", + "resolved": "https://registry.npmjs.org/@azure/msal-node/-/msal-node-2.13.1.tgz", + "integrity": "sha512-sijfzPNorKt6+9g1/miHwhj6Iapff4mPQx1azmmZExgzUROqWTM1o3ACyxDja0g47VpowFy/sxTM/WsuCyXTiw==", "dependencies": { - "@azure/msal-common": "14.14.1", + "@azure/msal-common": "14.14.2", "jsonwebtoken": "^9.0.0", "uuid": "^8.3.0" }, @@ -999,18 +948,11 @@ "node": ">=16" } }, - "node_modules/@azure/msal-node/node_modules/uuid": { - "version": "8.3.2", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", - "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", - "bin": { - "uuid": "dist/bin/uuid" - } - }, "node_modules/@babel/code-frame": { "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", + "dev": true, "dependencies": { "@babel/highlight": "^7.24.7", "picocolors": "^1.0.0" @@ -1020,9 +962,10 @@ } }, "node_modules/@babel/compat-data": { - "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.2.tgz", - "integrity": "sha512-bYcppcpKBvX4znYaPEeFau03bp89ShqNMLs+rmdptMw+heSZh9+z84d2YG+K7cYLbWwzdjtDoW/uqZmPjulClQ==", + "version": "7.25.4", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.4.tgz", + "integrity": "sha512-+LGRog6RAsCJrrrg/IO6LGmpphNe5DiK30dGjCoxxeGv49B10/3XYGxPsAwrDlMFcFEvdAUavDT8r9k/hSyQqQ==", + "dev": true, "engines": { "node": ">=6.9.0" } @@ -1031,6 +974,7 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.25.2.tgz", "integrity": "sha512-BBt3opiCOxUr9euZ5/ro/Xv8/V7yJ5bjYMqG/C1YAo8MIKAnumZalCN+msbci3Pigy4lIQfPUpfMM27HMGaYEA==", + "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", "@babel/code-frame": "^7.24.7", @@ -1060,16 +1004,18 @@ "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/generator": { - "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.0.tgz", - "integrity": "sha512-3LEEcj3PVW8pW2R1SR1M89g/qrYk/m/mB/tLqn7dn4sbBUQyTqnlod+II2U4dqiGtUmkcnAmkMDralTFZttRiw==", + "version": "7.25.6", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.6.tgz", + "integrity": "sha512-VPC82gr1seXOpkjAAKoLhP50vx4vGNlF4msF64dSFq1P8RfB+QAuJWGHPXXPc8QyfVWwwB/TNNU4+ayZmHNbZw==", + "dev": true, "dependencies": { - "@babel/types": "^7.25.0", + "@babel/types": "^7.25.6", "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.25", "jsesc": "^2.5.1" @@ -1082,6 +1028,7 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.25.2.tgz", "integrity": "sha512-U2U5LsSaZ7TAt3cfaymQ8WHh0pxvdHoEk6HVpaexxixjyEquMh0L0YNJNM6CTGKMXV1iksi0iZkGw4AcFkPaaw==", + "dev": true, "dependencies": { "@babel/compat-data": "^7.25.2", "@babel/helper-validator-option": "^7.24.8", @@ -1097,7 +1044,7 @@ "version": "5.1.1", "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-5.1.1.tgz", "integrity": "sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w==", - "license": "ISC", + "dev": true, "dependencies": { "yallist": "^3.0.2" } @@ -1106,14 +1053,22 @@ "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/helper-compilation-targets/node_modules/yallist": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-3.1.1.tgz", + "integrity": "sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g==", + "dev": true + }, "node_modules/@babel/helper-module-imports": { "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", + "dev": true, "dependencies": { "@babel/traverse": "^7.24.7", "@babel/types": "^7.24.7" @@ -1126,6 +1081,7 @@ "version": "7.25.2", "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.25.2.tgz", "integrity": "sha512-BjyRAbix6j/wv83ftcVJmBt72QtHI56C7JXZoG2xATiLpmoC7dpd8WnkikExHDVPpi/3qCmO6WY1EaXOluiecQ==", + "dev": true, "dependencies": { "@babel/helper-module-imports": "^7.24.7", "@babel/helper-simple-access": "^7.24.7", @@ -1143,6 +1099,7 @@ "version": "7.24.8", "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.8.tgz", "integrity": "sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==", + "dev": true, "engines": { "node": ">=6.9.0" } @@ -1151,6 +1108,7 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz", "integrity": "sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==", + "dev": true, "dependencies": { "@babel/traverse": "^7.24.7", "@babel/types": "^7.24.7" @@ -1163,6 +1121,7 @@ "version": "7.24.8", "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", + "dev": true, "engines": { "node": ">=6.9.0" } @@ -1171,6 +1130,7 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", + "dev": true, "engines": { "node": ">=6.9.0" } @@ -1179,17 +1139,19 @@ "version": "7.24.8", "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.24.8.tgz", "integrity": "sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==", + "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helpers": { - "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.0.tgz", - "integrity": "sha512-MjgLZ42aCm0oGjJj8CtSM3DB8NOOf8h2l7DCTePJs29u+v7yO/RBX9nShlKMgFnRks/Q4tBAe7Hxnov9VkGwLw==", + "version": "7.25.6", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.6.tgz", + "integrity": "sha512-Xg0tn4HcfTijTwfDwYlvVCl43V6h4KyVVX2aEm4qdO/PC6L2YvzLHFdmxhoeSA3eslcE6+ZVXHgWwopXYLNq4Q==", + "dev": true, "dependencies": { "@babel/template": "^7.25.0", - "@babel/types": "^7.25.0" + "@babel/types": "^7.25.6" }, "engines": { "node": ">=6.9.0" @@ -1199,6 +1161,7 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", + "dev": true, "dependencies": { "@babel/helper-validator-identifier": "^7.24.7", "chalk": "^2.4.2", @@ -1213,6 +1176,7 @@ "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": { "color-convert": "^1.9.0" }, @@ -1224,6 +1188,7 @@ "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": { "ansi-styles": "^3.2.1", "escape-string-regexp": "^1.0.5", @@ -1237,6 +1202,7 @@ "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" } @@ -1244,12 +1210,14 @@ "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==" + "integrity": "sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==", + "dev": true }, "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": ">=0.8.0" } @@ -1258,6 +1226,7 @@ "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" } @@ -1266,6 +1235,7 @@ "version": "5.5.0", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==", + "dev": true, "dependencies": { "has-flag": "^3.0.0" }, @@ -1274,11 +1244,12 @@ } }, "node_modules/@babel/parser": { - "version": "7.25.3", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.3.tgz", - "integrity": "sha512-iLTJKDbJ4hMvFPgQwwsVoxtHyWpKKPBrxkANrSYewDPaPpT5py5yeVkgPIJ7XYXhndxJpaA3PyALSXQ7u8e/Dw==", + "version": "7.25.6", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.6.tgz", + "integrity": "sha512-trGdfBdbD0l1ZPmcJ83eNxB9rbEax4ALFTF7fN386TMYbeCQbyme5cOEXQhbGXKebwGaB/J52w1mrklMcbgy6Q==", + "dev": true, "dependencies": { - "@babel/types": "^7.25.2" + "@babel/types": "^7.25.6" }, "bin": { "parser": "bin/babel-parser.js" @@ -1291,6 +1262,7 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1302,6 +1274,7 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1313,6 +1286,7 @@ "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": { "@babel/helper-plugin-utils": "^7.12.13" }, @@ -1320,10 +1294,41 @@ "@babel/core": "^7.0.0-0" } }, + "node_modules/@babel/plugin-syntax-class-static-block": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz", + "integrity": "sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw==", + "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-import-attributes": { + "version": "7.25.6", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-import-attributes/-/plugin-syntax-import-attributes-7.25.6.tgz", + "integrity": "sha512-sXaDXaJN9SNLymBdlWFA+bjzBhFD617ZaFiY13dGt7TVslVvVgA6fkZOP7Ki3IGElC45lwHdOTrCtKZGVAWeLQ==", + "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.24.8" + }, + "engines": { + "node": ">=6.9.0" + }, + "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" }, @@ -1335,6 +1340,7 @@ "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" }, @@ -1346,6 +1352,7 @@ "version": "7.24.7", "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz", "integrity": "sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ==", + "dev": true, "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" }, @@ -1360,6 +1367,7 @@ "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, "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -1371,6 +1379,7 @@ "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, "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1382,6 +1391,7 @@ "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": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -1393,6 +1403,7 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1404,6 +1415,7 @@ "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": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1415,6 +1427,7 @@ "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, "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1422,10 +1435,26 @@ "@babel/core": "^7.0.0-0" } }, + "node_modules/@babel/plugin-syntax-private-property-in-object": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz", + "integrity": "sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg==", + "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, + "engines": { + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, "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" }, @@ -1437,11 +1466,12 @@ } }, "node_modules/@babel/plugin-syntax-typescript": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.24.7.tgz", - "integrity": "sha512-c/+fVeJBB0FeKsFvwytYiUD+LBvhHjGSI0g446PRGdSVGZLRNArBUno2PETbAly3tpiNAQR5XaZ+JslxkotsbA==", + "version": "7.25.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.25.4.tgz", + "integrity": "sha512-uMOCoHVU52BsSWxPOMVv5qKRdeSlPuImUCB2dlPuBSU+W2/ROE7/Zg8F2Kepbk+8yBa68LlRKxO+xgEVWorsDg==", + "dev": true, "dependencies": { - "@babel/helper-plugin-utils": "^7.24.7" + "@babel/helper-plugin-utils": "^7.24.8" }, "engines": { "node": ">=6.9.0" @@ -1454,6 +1484,7 @@ "version": "7.25.0", "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", + "dev": true, "dependencies": { "@babel/code-frame": "^7.24.7", "@babel/parser": "^7.25.0", @@ -1464,15 +1495,16 @@ } }, "node_modules/@babel/traverse": { - "version": "7.25.3", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.3.tgz", - "integrity": "sha512-HefgyP1x754oGCsKmV5reSmtV7IXj/kpaE1XYY+D9G5PvKKoFfSbiS4M77MdjuwlZKDIKFCffq9rPU+H/s3ZdQ==", + "version": "7.25.6", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.6.tgz", + "integrity": "sha512-9Vrcx5ZW6UwK5tvqsj0nGpp/XzqthkT0dqIc9g1AdtygFToNtTF67XzYS//dm+SAK9cp3B9R4ZO/46p63SCjlQ==", + "dev": true, "dependencies": { "@babel/code-frame": "^7.24.7", - "@babel/generator": "^7.25.0", - "@babel/parser": "^7.25.3", + "@babel/generator": "^7.25.6", + "@babel/parser": "^7.25.6", "@babel/template": "^7.25.0", - "@babel/types": "^7.25.2", + "@babel/types": "^7.25.6", "debug": "^4.3.1", "globals": "^11.1.0" }, @@ -1484,14 +1516,16 @@ "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.25.2", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.2.tgz", - "integrity": "sha512-YTnYtra7W9e6/oAZEHj0bJehPRUlLH9/fbpT5LfB0NhQXyALCRkRs3zH9v07IYhkgpqX6Z78FnuccZr/l4Fs4Q==", + "version": "7.25.6", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.6.tgz", + "integrity": "sha512-/l42B1qxpG6RdfYf343Uw1vmDjeNhneUXtzhojE7pDgfpEypmRhI6j1kr17XCVv4Cgl9HdAiQY2x0GwKm7rWCw==", + "dev": true, "dependencies": { "@babel/helper-string-parser": "^7.24.8", "@babel/helper-validator-identifier": "^7.24.7", @@ -1504,12 +1538,13 @@ "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==" + "integrity": "sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw==", + "dev": true }, "node_modules/@bufbuild/buf": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf/-/buf-1.37.0.tgz", - "integrity": "sha512-33Rp90nRa7ebTwAjI8Yc7f8WOQMOEfRM0mg/o1bkNvE6cGQhYiq82Ag27pcegvYIyXl413XCeLMAUegsqjuFqw==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf/-/buf-1.41.0.tgz", + "integrity": "sha512-6pN2fqMrPqnIkrC1q9KpXpu7fv3Rul0ZPhT4MSYYj+8VcyR3kbLVk6K+CzzPvYhr4itfotnI3ZVGQ/X/vupECg==", "dev": true, "hasInstallScript": true, "bin": { @@ -1521,18 +1556,18 @@ "node": ">=12" }, "optionalDependencies": { - "@bufbuild/buf-darwin-arm64": "1.37.0", - "@bufbuild/buf-darwin-x64": "1.37.0", - "@bufbuild/buf-linux-aarch64": "1.37.0", - "@bufbuild/buf-linux-x64": "1.37.0", - "@bufbuild/buf-win32-arm64": "1.37.0", - "@bufbuild/buf-win32-x64": "1.37.0" + "@bufbuild/buf-darwin-arm64": "1.41.0", + "@bufbuild/buf-darwin-x64": "1.41.0", + "@bufbuild/buf-linux-aarch64": "1.41.0", + "@bufbuild/buf-linux-x64": "1.41.0", + "@bufbuild/buf-win32-arm64": "1.41.0", + "@bufbuild/buf-win32-x64": "1.41.0" } }, "node_modules/@bufbuild/buf-darwin-arm64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-arm64/-/buf-darwin-arm64-1.37.0.tgz", - "integrity": "sha512-YrT0HqLf7qJJ9VjXmnG900oGOjskerqfY4UTbpLb67b76uRxZm7pfSfqpM4bCaSru4j6GPBDwH9dkwiuNOenEA==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-arm64/-/buf-darwin-arm64-1.41.0.tgz", + "integrity": "sha512-+G5DwpIgnm0AkqgxORxoYXVT0RGDcw8P4SXFXcovgvDBkk9rPvEI1dbPF83n3SUxzcu2A2OxC7DxlXszWIh2Gw==", "cpu": [ "arm64" ], @@ -1546,9 +1581,9 @@ } }, "node_modules/@bufbuild/buf-darwin-x64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-x64/-/buf-darwin-x64-1.37.0.tgz", - "integrity": "sha512-c1iDr1LaYcdzAK7v4ZBTiYVRPltPa1UNFQ3JJ6BI/RUaBk/eYFsykuNtsJQb5lXh/usjMz76juzOxAdYikcw7w==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-x64/-/buf-darwin-x64-1.41.0.tgz", + "integrity": "sha512-qjkJ/LAWqNk3HX65n+JTt18WtKrhrrAhIu3Dpfbe0eujsxafFZKoPzlWJYybxvsaF9CdEyMMm/OalBPpoosMOA==", "cpu": [ "x64" ], @@ -1562,9 +1597,9 @@ } }, "node_modules/@bufbuild/buf-linux-aarch64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-aarch64/-/buf-linux-aarch64-1.37.0.tgz", - "integrity": "sha512-ENaCLU4XZxQFc6+pWjNGeKITlzY0jYeEmz6R2cEeR9kVl+HyNNcTMaKxdffG5HNU5Ci1dC3pSY/ZIF3ck8UHVw==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-aarch64/-/buf-linux-aarch64-1.41.0.tgz", + "integrity": "sha512-5E+MLAF4QHPwAjwVVRRP3Is2U3zpIpQQR7S3di9HlKACbgvefJEBrUfRqQZvHrMuuynQRqjFuZD16Sfvxn9rCQ==", "cpu": [ "arm64" ], @@ -1578,9 +1613,9 @@ } }, "node_modules/@bufbuild/buf-linux-x64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-x64/-/buf-linux-x64-1.37.0.tgz", - "integrity": "sha512-/buQuUaBVU4rYqdoS/xmbyohJFaE4uy7boHMabCnI2aLsVBNPHYa/k0JMc+ZDRqgq0Wnth9VbIHn6DSGHG+cmw==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-x64/-/buf-linux-x64-1.41.0.tgz", + "integrity": "sha512-W4T+uqmdtypzzatv6OXjUzGacZiNzGECogr+qDkJF38MSZd3jHXhTEN2KhRckl3i9rRAnfHBwG68BjCTxxBCOQ==", "cpu": [ "x64" ], @@ -1594,9 +1629,9 @@ } }, "node_modules/@bufbuild/buf-win32-arm64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-arm64/-/buf-win32-arm64-1.37.0.tgz", - "integrity": "sha512-5Y1aaHge9iQP+YNyxUayMd+rSVbjNXLBXniJDQhim56B+6eufPiKzmTgAKjDE9hGTT+mcBdLznrCMI0fzDLHJw==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-arm64/-/buf-win32-arm64-1.41.0.tgz", + "integrity": "sha512-OsRVoTZHJZYGIphAwaRqcCeYR9Sk5VEMjpCJiFt/dkHxx2acKH4u/7O+633gcCxQL8EnsU2l8AfdbW7sQaOvlg==", "cpu": [ "arm64" ], @@ -1610,9 +1645,9 @@ } }, "node_modules/@bufbuild/buf-win32-x64": { - "version": "1.37.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-x64/-/buf-win32-x64-1.37.0.tgz", - "integrity": "sha512-2BioEPdC6EM5zEKmTmM14ZJuuPjKuYIvoAwQ4hoCSJBllIhGvshk61NDYBorr1huEEq6baa4ITL/gWHJ86B0bw==", + "version": "1.41.0", + "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-x64/-/buf-win32-x64-1.41.0.tgz", + "integrity": "sha512-2KJLp7Py0GsfRjDxwBzS17RMpaYFGCvzkwY5CtxfPMw8cg6cE7E36r+vcjHh5dBOj/CumaiXLTwxhCSBtp0V1g==", "cpu": [ "x64" ], @@ -1678,6 +1713,10 @@ "node": ">=14.17" } }, + "node_modules/@confluentinc/schemaregistry": { + "resolved": "schemaregistry", + "link": true + }, "node_modules/@criteria/json-pointer": { "version": "0.2.1", "resolved": "https://registry.npmjs.org/@criteria/json-pointer/-/json-pointer-0.2.1.tgz", @@ -1784,9 +1823,9 @@ "dev": true }, "node_modules/@eslint/js": { - "version": "9.9.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.9.0.tgz", - "integrity": "sha512-hhetes6ZHP3BlXLxmd8K2SNgkhNSi+UcecbnwWKwpP7kyi/uC75DJ1lOOBO3xrC4jyojtGE3YxKZPHfk4yrgug==", + "version": "9.10.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.10.0.tgz", + "integrity": "sha512-fuXtbiP5GWIn8Fz+LWoOMVf/Jxm+aajZYkhi6CuEm4SxymFM+eUWzbO9qXT+L0iCkL5+KGYMCSGxo686H19S1g==", "dev": true, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -1810,9 +1849,9 @@ } }, "node_modules/@grpc/grpc-js": { - "version": "1.11.1", - "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.11.1.tgz", - "integrity": "sha512-gyt/WayZrVPH2w/UTLansS7F9Nwld472JxxaETamrM8HNlsa+jSLNyKAZmhxI2Me4c3mQHFiS1wWHDY1g1Kthw==", + "version": "1.11.2", + "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.11.2.tgz", + "integrity": "sha512-DWp92gDD7/Qkj7r8kus6/HCINeo3yPZWZ3paKgDgsbKbSpoxKg1yvN8xe2Q8uE3zOsPe3bX8FQX2+XValq2yTw==", "dependencies": { "@grpc/proto-loader": "^0.7.13", "@js-sdsl/ordered-map": "^4.4.2" @@ -1882,6 +1921,7 @@ "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", "integrity": "sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ==", + "dev": true, "dependencies": { "camelcase": "^5.3.1", "find-up": "^4.1.0", @@ -1897,6 +1937,7 @@ "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" } @@ -1905,6 +1946,7 @@ "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" @@ -1917,6 +1959,7 @@ "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" @@ -1929,6 +1972,7 @@ "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" }, @@ -1940,6 +1984,7 @@ "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" }, @@ -1954,6 +1999,7 @@ "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" }, @@ -1965,6 +2011,7 @@ "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" } @@ -1972,12 +2019,14 @@ "node_modules/@istanbuljs/load-nyc-config/node_modules/sprintf-js": { "version": "1.0.3", "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", - "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==" + "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==", + "dev": true }, "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, "engines": { "node": ">=8" } @@ -1986,6 +2035,7 @@ "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": "*", @@ -2002,6 +2052,7 @@ "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", @@ -2048,6 +2099,7 @@ "version": "29.7.0", "resolved": "https://registry.npmjs.org/@jest/environment/-/environment-29.7.0.tgz", "integrity": "sha512-aQIfHDq33ExsN4jP1NWGXhxgQ/wixs60gDiKO+XVMd8Mn0NWPWgc34ZQDTb2jKaUWQ7MuwoitXAsN2XVXNMpAw==", + "dev": true, "dependencies": { "@jest/fake-timers": "^29.7.0", "@jest/types": "^29.6.3", @@ -2062,6 +2114,7 @@ "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" @@ -2074,6 +2127,7 @@ "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" }, @@ -2085,6 +2139,7 @@ "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": { "@jest/types": "^29.6.3", "@sinonjs/fake-timers": "^10.0.2", @@ -2101,6 +2156,7 @@ "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", @@ -2115,6 +2171,7 @@ "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", @@ -2157,6 +2214,7 @@ "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" }, @@ -2168,6 +2226,7 @@ "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", @@ -2181,6 +2240,7 @@ "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", @@ -2195,6 +2255,7 @@ "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", @@ -2209,6 +2270,7 @@ "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", @@ -2234,6 +2296,7 @@ "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", @@ -2250,6 +2313,7 @@ "version": "0.3.5", "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", + "dev": true, "dependencies": { "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", @@ -2263,6 +2327,7 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", + "dev": true, "engines": { "node": ">=6.0.0" } @@ -2271,6 +2336,7 @@ "version": "1.2.1", "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", + "dev": true, "engines": { "node": ">=6.0.0" } @@ -2278,12 +2344,14 @@ "node_modules/@jridgewell/sourcemap-codec": { "version": "1.5.0", "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", - "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==" + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", + "dev": true }, "node_modules/@jridgewell/trace-mapping": { "version": "0.3.25", "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", + "dev": true, "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", "@jridgewell/sourcemap-codec": "^1.4.14" @@ -2329,6 +2397,29 @@ "node-pre-gyp": "bin/node-pre-gyp" } }, + "node_modules/@mapbox/node-pre-gyp/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/@mapbox/node-pre-gyp/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/@nodelib/fs.scandir": { "version": "2.1.5", "resolved": "https://registry.npmjs.org/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz", @@ -2486,12 +2577,14 @@ "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==" + "integrity": "sha512-+Fj43pSMwJs4KRrH/938Uf+uAELIgVBmQzg/q1YG10djyfA3TnrU8N8XzqCh/okZdszqBQTZf96idMfE5lnwTA==", + "dev": true }, "node_modules/@sinonjs/commons": { "version": "3.0.1", "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.1.tgz", "integrity": "sha512-K3mCHKQ9sVh8o1C9cxkwxaOmXoAMlDxC1mYyHrjqOWEcBjYr76t96zL2zlj5dUGZ3HSw240X1qgH3Mjf1yJWpQ==", + "dev": true, "dependencies": { "type-detect": "4.0.8" } @@ -2500,16 +2593,17 @@ "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/@smithy/abort-controller": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@smithy/abort-controller/-/abort-controller-3.1.1.tgz", - "integrity": "sha512-MBJBiidoe+0cTFhyxT8g+9g7CeVccLM0IOKKUMCNQ1CNMJ/eIfoo0RTfVrXOONEI1UCN1W+zkiHSbzUNE9dZtQ==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@smithy/abort-controller/-/abort-controller-3.1.2.tgz", + "integrity": "sha512-b5g+PNujlfqIib9BjkNB108NyO5aZM/RXjfOCXRCqXQ1oPnIkfvdORrztbGgCZdPe/BN/MKDlrGA7PafKPM2jw==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2517,14 +2611,14 @@ } }, "node_modules/@smithy/config-resolver": { - "version": "3.0.5", - "resolved": "https://registry.npmjs.org/@smithy/config-resolver/-/config-resolver-3.0.5.tgz", - "integrity": "sha512-SkW5LxfkSI1bUC74OtfBbdz+grQXYiPYolyu8VfpLIjEoN/sHVBlLeGXMQ1vX4ejkgfv6sxVbQJ32yF2cl1veA==", + "version": "3.0.6", + "resolved": "https://registry.npmjs.org/@smithy/config-resolver/-/config-resolver-3.0.6.tgz", + "integrity": "sha512-j7HuVNoRd8EhcFp0MzcUb4fG40C7BcyshH+fAd3Jhd8bINNFvEQYBrZoS/SK6Pun9WPlfoI8uuU2SMz8DsEGlA==", "dependencies": { - "@smithy/node-config-provider": "^3.1.4", - "@smithy/types": "^3.3.0", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/types": "^3.4.0", "@smithy/util-config-provider": "^3.0.0", - "@smithy/util-middleware": "^3.0.3", + "@smithy/util-middleware": "^3.0.4", "tslib": "^2.6.2" }, "engines": { @@ -2532,18 +2626,18 @@ } }, "node_modules/@smithy/core": { - "version": "2.4.0", - "resolved": "https://registry.npmjs.org/@smithy/core/-/core-2.4.0.tgz", - "integrity": "sha512-cHXq+FneIF/KJbt4q4pjN186+Jf4ZB0ZOqEaZMBhT79srEyGDDBV31NqBRBjazz8ppQ1bJbDJMY9ba5wKFV36w==", - "dependencies": { - "@smithy/middleware-endpoint": "^3.1.0", - "@smithy/middleware-retry": "^3.0.15", - "@smithy/middleware-serde": "^3.0.3", - "@smithy/protocol-http": "^4.1.0", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/@smithy/core/-/core-2.4.1.tgz", + "integrity": "sha512-7cts7/Oni7aCHebHGiBeWoz5z+vmH+Vx2Z/UW3XtXMslcxI3PEwBZxNinepwZjixS3n12fPc247PHWmjU7ndsQ==", + "dependencies": { + "@smithy/middleware-endpoint": "^3.1.1", + "@smithy/middleware-retry": "^3.0.16", + "@smithy/middleware-serde": "^3.0.4", + "@smithy/protocol-http": "^4.1.1", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", "@smithy/util-body-length-browser": "^3.0.0", - "@smithy/util-middleware": "^3.0.3", + "@smithy/util-middleware": "^3.0.4", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -2552,14 +2646,14 @@ } }, "node_modules/@smithy/credential-provider-imds": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/@smithy/credential-provider-imds/-/credential-provider-imds-3.2.0.tgz", - "integrity": "sha512-0SCIzgd8LYZ9EJxUjLXBmEKSZR/P/w6l7Rz/pab9culE/RWuqelAKGJvn5qUOl8BgX8Yj5HWM50A5hiB/RzsgA==", + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/@smithy/credential-provider-imds/-/credential-provider-imds-3.2.1.tgz", + "integrity": "sha512-4z/oTWpRF2TqQI3aCM89/PWu3kim58XU4kOCTtuTJnoaS4KT95cPWMxbQfTN2vzcOe96SOKO8QouQW/+ESB1fQ==", "dependencies": { - "@smithy/node-config-provider": "^3.1.4", - "@smithy/property-provider": "^3.1.3", - "@smithy/types": "^3.3.0", - "@smithy/url-parser": "^3.0.3", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/property-provider": "^3.1.4", + "@smithy/types": "^3.4.0", + "@smithy/url-parser": "^3.0.4", "tslib": "^2.6.2" }, "engines": { @@ -2567,23 +2661,23 @@ } }, "node_modules/@smithy/fetch-http-handler": { - "version": "3.2.4", - "resolved": "https://registry.npmjs.org/@smithy/fetch-http-handler/-/fetch-http-handler-3.2.4.tgz", - "integrity": "sha512-kBprh5Gs5h7ug4nBWZi1FZthdqSM+T7zMmsZxx0IBvWUn7dK3diz2SHn7Bs4dQGFDk8plDv375gzenDoNwrXjg==", + "version": "3.2.5", + "resolved": "https://registry.npmjs.org/@smithy/fetch-http-handler/-/fetch-http-handler-3.2.5.tgz", + "integrity": "sha512-DjRtGmK8pKQMIo9+JlAKUt14Z448bg8nAN04yKIvlrrpmpRSG57s5d2Y83npks1r4gPtTRNbAFdQCoj9l3P2KQ==", "dependencies": { - "@smithy/protocol-http": "^4.1.0", - "@smithy/querystring-builder": "^3.0.3", - "@smithy/types": "^3.3.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/querystring-builder": "^3.0.4", + "@smithy/types": "^3.4.0", "@smithy/util-base64": "^3.0.0", "tslib": "^2.6.2" } }, "node_modules/@smithy/hash-node": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/hash-node/-/hash-node-3.0.3.tgz", - "integrity": "sha512-2ctBXpPMG+B3BtWSGNnKELJ7SH9e4TNefJS0cd2eSkOOROeBnnVBnAy9LtJ8tY4vUEoe55N4CNPxzbWvR39iBw==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/hash-node/-/hash-node-3.0.4.tgz", + "integrity": "sha512-6FgTVqEfCr9z/7+Em8BwSkJKA2y3krf1em134x3yr2NHWVCo2KYI8tcA53cjeO47y41jwF84ntsEE0Pe6pNKlg==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "@smithy/util-buffer-from": "^3.0.0", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" @@ -2593,11 +2687,11 @@ } }, "node_modules/@smithy/invalid-dependency": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/invalid-dependency/-/invalid-dependency-3.0.3.tgz", - "integrity": "sha512-ID1eL/zpDULmHJbflb864k72/SNOZCADRc9i7Exq3RUNJw6raWUSlFEQ+3PX3EYs++bTxZB2dE9mEHTQLv61tw==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/invalid-dependency/-/invalid-dependency-3.0.4.tgz", + "integrity": "sha512-MJBUrojC4SEXi9aJcnNOE3oNAuYNphgCGFXscaCj2TA/59BTcXhzHACP8jnnEU3n4yir/NSLKzxqez0T4x4tjA==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" } }, @@ -2613,12 +2707,12 @@ } }, "node_modules/@smithy/middleware-content-length": { - "version": "3.0.5", - "resolved": "https://registry.npmjs.org/@smithy/middleware-content-length/-/middleware-content-length-3.0.5.tgz", - "integrity": "sha512-ILEzC2eyxx6ncej3zZSwMpB5RJ0zuqH7eMptxC4KN3f+v9bqT8ohssKbhNR78k/2tWW+KS5Spw+tbPF4Ejyqvw==", + "version": "3.0.6", + "resolved": "https://registry.npmjs.org/@smithy/middleware-content-length/-/middleware-content-length-3.0.6.tgz", + "integrity": "sha512-AFyHCfe8rumkJkz+hCOVJmBagNBj05KypyDwDElA4TgMSA4eYDZRjVePFZuyABrJZFDc7uVj3dpFIDCEhf59SA==", "dependencies": { - "@smithy/protocol-http": "^4.1.0", - "@smithy/types": "^3.3.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2626,16 +2720,16 @@ } }, "node_modules/@smithy/middleware-endpoint": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/@smithy/middleware-endpoint/-/middleware-endpoint-3.1.0.tgz", - "integrity": "sha512-5y5aiKCEwg9TDPB4yFE7H6tYvGFf1OJHNczeY10/EFF8Ir8jZbNntQJxMWNfeQjC1mxPsaQ6mR9cvQbf+0YeMw==", - "dependencies": { - "@smithy/middleware-serde": "^3.0.3", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", - "@smithy/url-parser": "^3.0.3", - "@smithy/util-middleware": "^3.0.3", + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/@smithy/middleware-endpoint/-/middleware-endpoint-3.1.1.tgz", + "integrity": "sha512-Irv+soW8NKluAtFSEsF8O3iGyLxa5oOevJb/e1yNacV9H7JP/yHyJuKST5YY2ORS1+W34VR8EuUrOF+K29Pl4g==", + "dependencies": { + "@smithy/middleware-serde": "^3.0.4", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", + "@smithy/url-parser": "^3.0.4", + "@smithy/util-middleware": "^3.0.4", "tslib": "^2.6.2" }, "engines": { @@ -2643,17 +2737,17 @@ } }, "node_modules/@smithy/middleware-retry": { - "version": "3.0.15", - "resolved": "https://registry.npmjs.org/@smithy/middleware-retry/-/middleware-retry-3.0.15.tgz", - "integrity": "sha512-iTMedvNt1ApdvkaoE8aSDuwaoc+BhvHqttbA/FO4Ty+y/S5hW6Ci/CTScG7vam4RYJWZxdTElc3MEfHRVH6cgQ==", - "dependencies": { - "@smithy/node-config-provider": "^3.1.4", - "@smithy/protocol-http": "^4.1.0", - "@smithy/service-error-classification": "^3.0.3", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", - "@smithy/util-middleware": "^3.0.3", - "@smithy/util-retry": "^3.0.3", + "version": "3.0.16", + "resolved": "https://registry.npmjs.org/@smithy/middleware-retry/-/middleware-retry-3.0.16.tgz", + "integrity": "sha512-08kI36p1yB4CWO3Qi+UQxjzobt8iQJpnruF0K5BkbZmA/N/sJ51A1JJGJ36GgcbFyPfWw2FU48S5ZoqXt0h0jw==", + "dependencies": { + "@smithy/node-config-provider": "^3.1.5", + "@smithy/protocol-http": "^4.1.1", + "@smithy/service-error-classification": "^3.0.4", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", + "@smithy/util-middleware": "^3.0.4", + "@smithy/util-retry": "^3.0.4", "tslib": "^2.6.2", "uuid": "^9.0.1" }, @@ -2661,12 +2755,24 @@ "node": ">=16.0.0" } }, - "node_modules/@smithy/middleware-serde": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/middleware-serde/-/middleware-serde-3.0.3.tgz", - "integrity": "sha512-puUbyJQBcg9eSErFXjKNiGILJGtiqmuuNKEYNYfUD57fUl4i9+mfmThtQhvFXU0hCVG0iEJhvQUipUf+/SsFdA==", + "node_modules/@smithy/middleware-retry/node_modules/uuid": { + "version": "9.0.1", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", + "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "bin": { + "uuid": "dist/bin/uuid" + } + }, + "node_modules/@smithy/middleware-serde": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/middleware-serde/-/middleware-serde-3.0.4.tgz", + "integrity": "sha512-1lPDB2O6IJ50Ucxgn7XrvZXbbuI48HmPCcMTuSoXT1lDzuTUfIuBjgAjpD8YLVMfnrjdepi/q45556LA51Pubw==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2674,11 +2780,11 @@ } }, "node_modules/@smithy/middleware-stack": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/middleware-stack/-/middleware-stack-3.0.3.tgz", - "integrity": "sha512-r4klY9nFudB0r9UdSMaGSyjyQK5adUyPnQN/ZM6M75phTxOdnc/AhpvGD1fQUvgmqjQEBGCwpnPbDm8pH5PapA==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/middleware-stack/-/middleware-stack-3.0.4.tgz", + "integrity": "sha512-sLMRjtMCqtVcrOqaOZ10SUnlFE25BSlmLsi4bRSGFD7dgR54eqBjfqkVkPBQyrKBortfGM0+2DJoUPcGECR+nQ==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2686,13 +2792,13 @@ } }, "node_modules/@smithy/node-config-provider": { - "version": "3.1.4", - "resolved": "https://registry.npmjs.org/@smithy/node-config-provider/-/node-config-provider-3.1.4.tgz", - "integrity": "sha512-YvnElQy8HR4vDcAjoy7Xkx9YT8xZP4cBXcbJSgm/kxmiQu08DwUwj8rkGnyoJTpfl/3xYHH+d8zE+eHqoDCSdQ==", + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/@smithy/node-config-provider/-/node-config-provider-3.1.5.tgz", + "integrity": "sha512-dq/oR3/LxgCgizVk7in7FGTm0w9a3qM4mg3IIXLTCHeW3fV+ipssSvBZ2bvEx1+asfQJTyCnVLeYf7JKfd9v3Q==", "dependencies": { - "@smithy/property-provider": "^3.1.3", - "@smithy/shared-ini-file-loader": "^3.1.4", - "@smithy/types": "^3.3.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2700,14 +2806,14 @@ } }, "node_modules/@smithy/node-http-handler": { - "version": "3.1.4", - "resolved": "https://registry.npmjs.org/@smithy/node-http-handler/-/node-http-handler-3.1.4.tgz", - "integrity": "sha512-+UmxgixgOr/yLsUxcEKGH0fMNVteJFGkmRltYFHnBMlogyFdpzn2CwqWmxOrfJELhV34v0WSlaqG1UtE1uXlJg==", + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/@smithy/node-http-handler/-/node-http-handler-3.2.0.tgz", + "integrity": "sha512-5TFqaABbiY7uJMKbqR4OARjwI/l4TRoysDJ75pLpVQyO3EcmeloKYwDGyCtgB9WJniFx3BMkmGCB9+j+QiB+Ww==", "dependencies": { - "@smithy/abort-controller": "^3.1.1", - "@smithy/protocol-http": "^4.1.0", - "@smithy/querystring-builder": "^3.0.3", - "@smithy/types": "^3.3.0", + "@smithy/abort-controller": "^3.1.2", + "@smithy/protocol-http": "^4.1.1", + "@smithy/querystring-builder": "^3.0.4", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2715,11 +2821,11 @@ } }, "node_modules/@smithy/property-provider": { - "version": "3.1.3", - "resolved": "https://registry.npmjs.org/@smithy/property-provider/-/property-provider-3.1.3.tgz", - "integrity": "sha512-zahyOVR9Q4PEoguJ/NrFP4O7SMAfYO1HLhB18M+q+Z4KFd4V2obiMnlVoUFzFLSPeVt1POyNWneHHrZaTMoc/g==", + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@smithy/property-provider/-/property-provider-3.1.4.tgz", + "integrity": "sha512-BmhefQbfkSl9DeU0/e6k9N4sT5bya5etv2epvqLUz3eGyfRBhtQq60nDkc1WPp4c+KWrzK721cUc/3y0f2psPQ==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2727,11 +2833,11 @@ } }, "node_modules/@smithy/protocol-http": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/@smithy/protocol-http/-/protocol-http-4.1.0.tgz", - "integrity": "sha512-dPVoHYQ2wcHooGXg3LQisa1hH0e4y0pAddPMeeUPipI1tEOqL6A4N0/G7abeq+K8wrwSgjk4C0wnD1XZpJm5aA==", + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/@smithy/protocol-http/-/protocol-http-4.1.1.tgz", + "integrity": "sha512-Fm5+8LkeIus83Y8jTL1XHsBGP8sPvE1rEVyKf/87kbOPTbzEDMcgOlzcmYXat2h+nC3wwPtRy8hFqtJS71+Wow==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2739,11 +2845,11 @@ } }, "node_modules/@smithy/querystring-builder": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/querystring-builder/-/querystring-builder-3.0.3.tgz", - "integrity": "sha512-vyWckeUeesFKzCDaRwWLUA1Xym9McaA6XpFfAK5qI9DKJ4M33ooQGqvM4J+LalH4u/Dq9nFiC8U6Qn1qi0+9zw==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/querystring-builder/-/querystring-builder-3.0.4.tgz", + "integrity": "sha512-NEoPAsZPdpfVbF98qm8i5k1XMaRKeEnO47CaL5ja6Y1Z2DgJdwIJuJkTJypKm/IKfp8gc0uimIFLwhml8+/pAw==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "@smithy/util-uri-escape": "^3.0.0", "tslib": "^2.6.2" }, @@ -2752,11 +2858,11 @@ } }, "node_modules/@smithy/querystring-parser": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/querystring-parser/-/querystring-parser-3.0.3.tgz", - "integrity": "sha512-zahM1lQv2YjmznnfQsWbYojFe55l0SLG/988brlLv1i8z3dubloLF+75ATRsqPBboUXsW6I9CPGE5rQgLfY0vQ==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/querystring-parser/-/querystring-parser-3.0.4.tgz", + "integrity": "sha512-7CHPXffFcakFzhO0OZs/rn6fXlTHrSDdLhIT6/JIk1u2bvwguTL3fMCc1+CfcbXA7TOhjWXu3TcB1EGMqJQwHg==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2764,22 +2870,22 @@ } }, "node_modules/@smithy/service-error-classification": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/service-error-classification/-/service-error-classification-3.0.3.tgz", - "integrity": "sha512-Jn39sSl8cim/VlkLsUhRFq/dKDnRUFlfRkvhOJaUbLBXUsLRLNf9WaxDv/z9BjuQ3A6k/qE8af1lsqcwm7+DaQ==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/service-error-classification/-/service-error-classification-3.0.4.tgz", + "integrity": "sha512-KciDHHKFVTb9A1KlJHBt2F26PBaDtoE23uTZy5qRvPzHPqrooXFi6fmx98lJb3Jl38PuUTqIuCUmmY3pacuMBQ==", "dependencies": { - "@smithy/types": "^3.3.0" + "@smithy/types": "^3.4.0" }, "engines": { "node": ">=16.0.0" } }, "node_modules/@smithy/shared-ini-file-loader": { - "version": "3.1.4", - "resolved": "https://registry.npmjs.org/@smithy/shared-ini-file-loader/-/shared-ini-file-loader-3.1.4.tgz", - "integrity": "sha512-qMxS4hBGB8FY2GQqshcRUy1K6k8aBWP5vwm8qKkCT3A9K2dawUwOIJfqh9Yste/Bl0J2lzosVyrXDj68kLcHXQ==", + "version": "3.1.5", + "resolved": "https://registry.npmjs.org/@smithy/shared-ini-file-loader/-/shared-ini-file-loader-3.1.5.tgz", + "integrity": "sha512-6jxsJ4NOmY5Du4FD0enYegNJl4zTSuKLiChIMqIkh+LapxiP7lmz5lYUNLE9/4cvA65mbBmtdzZ8yxmcqM5igg==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2787,15 +2893,15 @@ } }, "node_modules/@smithy/signature-v4": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/@smithy/signature-v4/-/signature-v4-4.1.0.tgz", - "integrity": "sha512-aRryp2XNZeRcOtuJoxjydO6QTaVhxx/vjaR+gx7ZjaFgrgPRyZ3HCTbfwqYj6ZWEBHkCSUfcaymKPURaByukag==", + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/@smithy/signature-v4/-/signature-v4-4.1.1.tgz", + "integrity": "sha512-SH9J9be81TMBNGCmjhrgMWu4YSpQ3uP1L06u/K9SDrE2YibUix1qxedPCxEQu02At0P0SrYDjvz+y91vLG0KRQ==", "dependencies": { "@smithy/is-array-buffer": "^3.0.0", - "@smithy/protocol-http": "^4.1.0", - "@smithy/types": "^3.3.0", + "@smithy/protocol-http": "^4.1.1", + "@smithy/types": "^3.4.0", "@smithy/util-hex-encoding": "^3.0.0", - "@smithy/util-middleware": "^3.0.3", + "@smithy/util-middleware": "^3.0.4", "@smithy/util-uri-escape": "^3.0.0", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" @@ -2805,15 +2911,15 @@ } }, "node_modules/@smithy/smithy-client": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/@smithy/smithy-client/-/smithy-client-3.2.0.tgz", - "integrity": "sha512-pDbtxs8WOhJLJSeaF/eAbPgXg4VVYFlRcL/zoNYA5WbG3wBL06CHtBSg53ppkttDpAJ/hdiede+xApip1CwSLw==", - "dependencies": { - "@smithy/middleware-endpoint": "^3.1.0", - "@smithy/middleware-stack": "^3.0.3", - "@smithy/protocol-http": "^4.1.0", - "@smithy/types": "^3.3.0", - "@smithy/util-stream": "^3.1.3", + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/@smithy/smithy-client/-/smithy-client-3.3.0.tgz", + "integrity": "sha512-H32nVo8tIX82kB0xI2LBrIcj8jx/3/ITotNLbeG1UL0b3b440YPR/hUvqjFJiaB24pQrMjRbU8CugqH5sV0hkw==", + "dependencies": { + "@smithy/middleware-endpoint": "^3.1.1", + "@smithy/middleware-stack": "^3.0.4", + "@smithy/protocol-http": "^4.1.1", + "@smithy/types": "^3.4.0", + "@smithy/util-stream": "^3.1.4", "tslib": "^2.6.2" }, "engines": { @@ -2821,9 +2927,9 @@ } }, "node_modules/@smithy/types": { - "version": "3.3.0", - "resolved": "https://registry.npmjs.org/@smithy/types/-/types-3.3.0.tgz", - "integrity": "sha512-IxvBBCTFDHbVoK7zIxqA1ZOdc4QfM5HM7rGleCuHi7L1wnKv5Pn69xXJQ9hgxH60ZVygH9/JG0jRgtUncE3QUA==", + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/@smithy/types/-/types-3.4.0.tgz", + "integrity": "sha512-0shOWSg/pnFXPcsSU8ZbaJ4JBHZJPPzLCJxafJvbMVFo9l1w81CqpgUqjlKGNHVrVB7fhIs+WS82JDTyzaLyLA==", "dependencies": { "tslib": "^2.6.2" }, @@ -2832,12 +2938,12 @@ } }, "node_modules/@smithy/url-parser": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/url-parser/-/url-parser-3.0.3.tgz", - "integrity": "sha512-pw3VtZtX2rg+s6HMs6/+u9+hu6oY6U7IohGhVNnjbgKy86wcIsSZwgHrFR+t67Uyxvp4Xz3p3kGXXIpTNisq8A==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/url-parser/-/url-parser-3.0.4.tgz", + "integrity": "sha512-XdXfObA8WrloavJYtDuzoDhJAYc5rOt+FirFmKBRKaihu7QtU/METAxJgSo7uMK6hUkx0vFnqxV75urtRaLkLg==", "dependencies": { - "@smithy/querystring-parser": "^3.0.3", - "@smithy/types": "^3.3.0", + "@smithy/querystring-parser": "^3.0.4", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" } }, @@ -2897,13 +3003,13 @@ } }, "node_modules/@smithy/util-defaults-mode-browser": { - "version": "3.0.15", - "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-browser/-/util-defaults-mode-browser-3.0.15.tgz", - "integrity": "sha512-FZ4Psa3vjp8kOXcd3HJOiDPBCWtiilLl57r0cnNtq/Ga9RSDrM5ERL6xt+tO43+2af6Pn5Yp92x2n5vPuduNfg==", + "version": "3.0.16", + "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-browser/-/util-defaults-mode-browser-3.0.16.tgz", + "integrity": "sha512-Os8ddfNBe7hmc5UMWZxygIHCyAqY0aWR8Wnp/aKbti3f8Df/r0J9ttMZIxeMjsFgtVjEryB0q7SGcwBsHk8WEw==", "dependencies": { - "@smithy/property-provider": "^3.1.3", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", + "@smithy/property-provider": "^3.1.4", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", "bowser": "^2.11.0", "tslib": "^2.6.2" }, @@ -2912,16 +3018,16 @@ } }, "node_modules/@smithy/util-defaults-mode-node": { - "version": "3.0.15", - "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-node/-/util-defaults-mode-node-3.0.15.tgz", - "integrity": "sha512-KSyAAx2q6d0t6f/S4XB2+3+6aQacm3aLMhs9aLMqn18uYGUepbdssfogW5JQZpc6lXNBnp0tEnR5e9CEKmEd7A==", - "dependencies": { - "@smithy/config-resolver": "^3.0.5", - "@smithy/credential-provider-imds": "^3.2.0", - "@smithy/node-config-provider": "^3.1.4", - "@smithy/property-provider": "^3.1.3", - "@smithy/smithy-client": "^3.2.0", - "@smithy/types": "^3.3.0", + "version": "3.0.16", + "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-node/-/util-defaults-mode-node-3.0.16.tgz", + "integrity": "sha512-rNhFIYRtrOrrhRlj6RL8jWA6/dcwrbGYAmy8+OAHjjzQ6zdzUBB1P+3IuJAgwWN6Y5GxI+mVXlM/pOjaoIgHow==", + "dependencies": { + "@smithy/config-resolver": "^3.0.6", + "@smithy/credential-provider-imds": "^3.2.1", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/property-provider": "^3.1.4", + "@smithy/smithy-client": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2929,12 +3035,12 @@ } }, "node_modules/@smithy/util-endpoints": { - "version": "2.0.5", - "resolved": "https://registry.npmjs.org/@smithy/util-endpoints/-/util-endpoints-2.0.5.tgz", - "integrity": "sha512-ReQP0BWihIE68OAblC/WQmDD40Gx+QY1Ez8mTdFMXpmjfxSyz2fVQu3A4zXRfQU9sZXtewk3GmhfOHswvX+eNg==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/@smithy/util-endpoints/-/util-endpoints-2.1.0.tgz", + "integrity": "sha512-ilS7/0jcbS2ELdg0fM/4GVvOiuk8/U3bIFXUW25xE1Vh1Ol4DP6vVHQKqM40rCMizCLmJ9UxK+NeJrKlhI3HVA==", "dependencies": { - "@smithy/node-config-provider": "^3.1.4", - "@smithy/types": "^3.3.0", + "@smithy/node-config-provider": "^3.1.5", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2953,11 +3059,11 @@ } }, "node_modules/@smithy/util-middleware": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/util-middleware/-/util-middleware-3.0.3.tgz", - "integrity": "sha512-l+StyYYK/eO3DlVPbU+4Bi06Jjal+PFLSMmlWM1BEwyLxZ3aKkf1ROnoIakfaA7mC6uw3ny7JBkau4Yc+5zfWw==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/util-middleware/-/util-middleware-3.0.4.tgz", + "integrity": "sha512-uSXHTBhstb1c4nHdmQEdkNMv9LiRNaJ/lWV2U/GO+5F236YFpdPw+hyWI9Zc0Rp9XKzwD9kVZvhZmEgp0UCVnA==", "dependencies": { - "@smithy/types": "^3.3.0", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2965,12 +3071,12 @@ } }, "node_modules/@smithy/util-retry": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/@smithy/util-retry/-/util-retry-3.0.3.tgz", - "integrity": "sha512-AFw+hjpbtVApzpNDhbjNG5NA3kyoMs7vx0gsgmlJF4s+yz1Zlepde7J58zpIRIsdjc+emhpAITxA88qLkPF26w==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@smithy/util-retry/-/util-retry-3.0.4.tgz", + "integrity": "sha512-JJr6g0tO1qO2tCQyK+n3J18r34ZpvatlFN5ULcLranFIBZPxqoivb77EPyNTVwTGMEvvq2qMnyjm4jMIxjdLFg==", "dependencies": { - "@smithy/service-error-classification": "^3.0.3", - "@smithy/types": "^3.3.0", + "@smithy/service-error-classification": "^3.0.4", + "@smithy/types": "^3.4.0", "tslib": "^2.6.2" }, "engines": { @@ -2978,13 +3084,13 @@ } }, "node_modules/@smithy/util-stream": { - "version": "3.1.3", - "resolved": "https://registry.npmjs.org/@smithy/util-stream/-/util-stream-3.1.3.tgz", - "integrity": "sha512-FIv/bRhIlAxC0U7xM1BCnF2aDRPq0UaelqBHkM2lsCp26mcBbgI0tCVTv+jGdsQLUmAMybua/bjDsSu8RQHbmw==", + "version": "3.1.4", + "resolved": "https://registry.npmjs.org/@smithy/util-stream/-/util-stream-3.1.4.tgz", + "integrity": "sha512-txU3EIDLhrBZdGfon6E9V6sZz/irYnKFMblz4TLVjyq8hObNHNS2n9a2t7GIrl7d85zgEPhwLE0gANpZsvpsKg==", "dependencies": { - "@smithy/fetch-http-handler": "^3.2.4", - "@smithy/node-http-handler": "^3.1.4", - "@smithy/types": "^3.3.0", + "@smithy/fetch-http-handler": "^3.2.5", + "@smithy/node-http-handler": "^3.2.0", + "@smithy/types": "^3.4.0", "@smithy/util-base64": "^3.0.0", "@smithy/util-buffer-from": "^3.0.0", "@smithy/util-hex-encoding": "^3.0.0", @@ -3030,6 +3136,7 @@ "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", @@ -3042,6 +3149,7 @@ "version": "7.6.8", "resolved": "https://registry.npmjs.org/@types/babel__generator/-/babel__generator-7.6.8.tgz", "integrity": "sha512-ASsj+tpEDsEiFr1arWrlN6V3mdfjRMZt6LtK/Vp/kreFLnr5QH5+DhvD5nINYZXzwJvXeGq+05iUXcAzVrqWtw==", + "dev": true, "dependencies": { "@babel/types": "^7.0.0" } @@ -3050,6 +3158,7 @@ "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" @@ -3059,6 +3168,7 @@ "version": "7.20.6", "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.6.tgz", "integrity": "sha512-r1bzfrm0tomOI8g1SzvCaQHo6Lcv6zu0EA+W2kHrt8dyrHQxGzBBL4kdkzIS+jBMV+EYcMAEAqXqYaLJq5rOZg==", + "dev": true, "dependencies": { "@babel/types": "^7.20.7" } @@ -3069,9 +3179,9 @@ "integrity": "sha512-hWtVTC2q7hc7xZ/RLbxapMvDMgUnDvKvMOpKal4DrMyfGBUfB1oKaZlIRr6mJL+If3bAP6sV/QneGzF6tJjZDg==" }, "node_modules/@types/eslint": { - "version": "9.6.0", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-9.6.0.tgz", - "integrity": "sha512-gi6WQJ7cHRgZxtkQEoyHMppPjq9Kxo5Tjn2prSKDSmZrCz8TZ3jSRCeTJm+WoM+oB0WG37bRqLzaaU3q7JypGg==", + "version": "9.6.1", + "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-9.6.1.tgz", + "integrity": "sha512-FXx2pKgId/WyYo2jXw63kk7/+TY7u7AziEJxJAnSFzHlqTAS3Ync6SvgYAN/k4/PQpnnVuzoMuVnByKK2qp0ag==", "dev": true, "dependencies": { "@types/estree": "*", @@ -3097,6 +3207,7 @@ "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": "*" } @@ -3104,12 +3215,14 @@ "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==" + "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": "*" } @@ -3118,6 +3231,7 @@ "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": "*" } @@ -3156,9 +3270,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.16.1", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.16.1.tgz", - "integrity": "sha512-zJDo7wEadFtSyNz5QITDfRcrhqDvQI1xQNQ0VoizPjM/dVAODqqIUWbJPkvsxmTI0MYRGRikcdjMPhOssnPejQ==", + "version": "20.16.5", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.16.5.tgz", + "integrity": "sha512-VwYCweNo3ERajwy0IUlqqcyZ8/A7Zwa9ZP3MnENWcB11AejO+tLy3pu850goUW2FC/IJMdZUfKpX/yxL1gymCA==", "dependencies": { "undici-types": "~6.19.2" } @@ -3187,18 +3301,11 @@ "node": ">= 0.12" } }, - "node_modules/@types/semver": { - "version": "7.5.8", - "resolved": "https://registry.npmjs.org/@types/semver/-/semver-7.5.8.tgz", - "integrity": "sha512-I8EUhyrgfLrcTkzV3TSsGyl1tSuPrEDzr0yd5m90UgNxQkyDXULk3b6MlQqTCpZpNtWe1K0hzclnZkTcLBe2UQ==", - "dev": true, - "optional": true, - "peer": true - }, "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==" + "integrity": "sha512-9aEbYZ3TbYMznPdcdr3SmIrLXwC/AKZXQeCf9Pgao5CKb8CyHuEX5jzWPTkvregvhRJHcpRO6BFoGW9ycaOkYw==", + "dev": true }, "node_modules/@types/tough-cookie": { "version": "4.0.5", @@ -3206,14 +3313,15 @@ "integrity": "sha512-/Ad8+nIOV7Rl++6f1BdKxFSMgmoqEoYbHRpPcx3JEfv8VRsQe9Z4mCXeJBzxs7mbHY/XOZZuXlRNfhpVPbs6ZA==" }, "node_modules/@types/validator": { - "version": "13.12.0", - "resolved": "https://registry.npmjs.org/@types/validator/-/validator-13.12.0.tgz", - "integrity": "sha512-nH45Lk7oPIJ1RVOF6JgFI6Dy0QpHEzq4QecZhvguxYPDwT8c93prCMqAtiIttm39voZ+DDR+qkNnMpJmMBRqag==" + "version": "13.12.1", + "resolved": "https://registry.npmjs.org/@types/validator/-/validator-13.12.1.tgz", + "integrity": "sha512-w0URwf7BQb0rD/EuiG12KP0bailHKHP5YVviJG9zw3ykAokL0TuxU2TUqMB7EwZ59bDHYdeTIvjI5m0S7qHfOA==" }, "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==", + "version": "17.0.33", + "resolved": "https://registry.npmjs.org/@types/yargs/-/yargs-17.0.33.tgz", + "integrity": "sha512-WpxBCKWPLr4xSsHgz511rFJAM+wS28w2zEO1QDNY5zM/S8ok70NNfztH0xwhqKyaK0OHCbN98LDAZuy1ctxDkA==", + "dev": true, "dependencies": { "@types/yargs-parser": "*" } @@ -3221,38 +3329,35 @@ "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==" + "integrity": "sha512-I4q9QU9MQv4oEOz4tAHJtNz1cwuLxn2F3xcc2iV5WdqLPpUnj30aUuxt1mAxYTG+oe8CZMV/+6rU4S4gRDzqtQ==", + "dev": true }, "node_modules/@typescript-eslint/eslint-plugin": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-7.2.0.tgz", - "integrity": "sha512-mdekAHOqS9UjlmyF/LSs6AIEvfceV749GFxoBAjwAv0nkevfKHWQFDMcBZWUiIC5ft6ePWivXoS36aKQ0Cy3sw==", + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.5.0.tgz", + "integrity": "sha512-lHS5hvz33iUFQKuPFGheAB84LwcJ60G8vKnEhnfcK1l8kGVLro2SFYW6K0/tj8FUhRJ0VHyg1oAfg50QGbPPHw==", "dev": true, - "optional": true, - "peer": true, "dependencies": { - "@eslint-community/regexpp": "^4.5.1", - "@typescript-eslint/scope-manager": "7.2.0", - "@typescript-eslint/type-utils": "7.2.0", - "@typescript-eslint/utils": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0", - "debug": "^4.3.4", + "@eslint-community/regexpp": "^4.10.0", + "@typescript-eslint/scope-manager": "8.5.0", + "@typescript-eslint/type-utils": "8.5.0", + "@typescript-eslint/utils": "8.5.0", + "@typescript-eslint/visitor-keys": "8.5.0", "graphemer": "^1.4.0", - "ignore": "^5.2.4", + "ignore": "^5.3.1", "natural-compare": "^1.4.0", - "semver": "^7.5.4", - "ts-api-utils": "^1.0.1" + "ts-api-utils": "^1.3.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "@typescript-eslint/parser": "^7.0.0", - "eslint": "^8.56.0" + "@typescript-eslint/parser": "^8.0.0 || ^8.0.0-alpha.0", + "eslint": "^8.57.0 || ^9.0.0" }, "peerDependenciesMeta": { "typescript": { @@ -3260,59 +3365,64 @@ } } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/scope-manager": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", - "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", + "node_modules/@typescript-eslint/parser": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.5.0.tgz", + "integrity": "sha512-gF77eNv0Xz2UJg/NbpWJ0kqAm35UMsvZf1GHj8D9MRFTj/V3tAciIWXfmPLsAAF/vUlpWPvUDyH1jjsr0cMVWw==", "dev": true, - "optional": true, - "peer": true, "dependencies": { - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0" + "@typescript-eslint/scope-manager": "8.5.0", + "@typescript-eslint/types": "8.5.0", + "@typescript-eslint/typescript-estree": "8.5.0", + "@typescript-eslint/visitor-keys": "8.5.0", + "debug": "^4.3.4" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/types": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", - "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", + "node_modules/@typescript-eslint/scope-manager": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.5.0.tgz", + "integrity": "sha512-06JOQ9Qgj33yvBEx6tpC8ecP9o860rsR22hWMEd12WcTRrfaFgHr2RB/CA/B+7BMhHkXT4chg2MyboGdFGawYg==", "dev": true, - "optional": true, - "peer": true, + "dependencies": { + "@typescript-eslint/types": "8.5.0", + "@typescript-eslint/visitor-keys": "8.5.0" + }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/typescript-estree": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", - "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", + "node_modules/@typescript-eslint/type-utils": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.5.0.tgz", + "integrity": "sha512-N1K8Ix+lUM+cIDhL2uekVn/ZD7TZW+9/rwz8DclQpcQ9rk4sIL5CAlBC0CugWKREmDjBzI/kQqU4wkg46jWLYA==", "dev": true, - "optional": true, - "peer": true, "dependencies": { - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0", + "@typescript-eslint/typescript-estree": "8.5.0", + "@typescript-eslint/utils": "8.5.0", "debug": "^4.3.4", - "globby": "^11.1.0", - "is-glob": "^4.0.3", - "minimatch": "9.0.3", - "semver": "^7.5.4", - "ts-api-utils": "^1.0.1" + "ts-api-utils": "^1.3.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", @@ -3324,70 +3434,61 @@ } } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/utils": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-7.2.0.tgz", - "integrity": "sha512-YfHpnMAGb1Eekpm3XRK8hcMwGLGsnT6L+7b2XyRv6ouDuJU1tZir1GS2i0+VXRatMwSI1/UfcyPe53ADkU+IuA==", + "node_modules/@typescript-eslint/types": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.5.0.tgz", + "integrity": "sha512-qjkormnQS5wF9pjSi6q60bKUHH44j2APxfh9TQRXK8wbYVeDYYdYJGIROL87LGZZ2gz3Rbmjc736qyL8deVtdw==", "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@eslint-community/eslint-utils": "^4.4.0", - "@types/json-schema": "^7.0.12", - "@types/semver": "^7.5.0", - "@typescript-eslint/scope-manager": "7.2.0", - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/typescript-estree": "7.2.0", - "semver": "^7.5.4" - }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^8.56.0" } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/visitor-keys": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", - "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", + "node_modules/@typescript-eslint/typescript-estree": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.5.0.tgz", + "integrity": "sha512-vEG2Sf9P8BPQ+d0pxdfndw3xIXaoSjliG0/Ejk7UggByZPKXmJmw3GW5jV2gHNQNawBUyfahoSiCFVov0Ruf7Q==", "dev": true, - "optional": true, - "peer": true, "dependencies": { - "@typescript-eslint/types": "7.2.0", - "eslint-visitor-keys": "^3.4.1" + "@typescript-eslint/types": "8.5.0", + "@typescript-eslint/visitor-keys": "8.5.0", + "debug": "^4.3.4", + "fast-glob": "^3.3.2", + "is-glob": "^4.0.3", + "minimatch": "^9.0.4", + "semver": "^7.6.0", + "ts-api-utils": "^1.3.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/brace-expansion": { + "node_modules/@typescript-eslint/typescript-estree/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, - "optional": true, - "peer": true, "dependencies": { "balanced-match": "^1.0.0" } }, - "node_modules/@typescript-eslint/eslint-plugin/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/@typescript-eslint/typescript-estree/node_modules/minimatch": { + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", + "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", "dev": true, - "optional": true, - "peer": true, "dependencies": { "brace-expansion": "^2.0.1" }, @@ -3398,508 +3499,118 @@ "url": "https://github.com/sponsors/isaacs" } }, - "node_modules/@typescript-eslint/parser": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-7.2.0.tgz", - "integrity": "sha512-5FKsVcHTk6TafQKQbuIVkXq58Fnbkd2wDL4LB7AURN7RUOu1utVP+G8+6u3ZhEroW3DF6hyo3ZEXxgKgp4KeCg==", + "node_modules/@typescript-eslint/utils": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.5.0.tgz", + "integrity": "sha512-6yyGYVL0e+VzGYp60wvkBHiqDWOpT63pdMV2CVG4LVDd5uR6q1qQN/7LafBZtAtNIn/mqXjsSeS5ggv/P0iECw==", "dev": true, - "optional": true, - "peer": true, "dependencies": { - "@typescript-eslint/scope-manager": "7.2.0", - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/typescript-estree": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0", - "debug": "^4.3.4" + "@eslint-community/eslint-utils": "^4.4.0", + "@typescript-eslint/scope-manager": "8.5.0", + "@typescript-eslint/types": "8.5.0", + "@typescript-eslint/typescript-estree": "8.5.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "eslint": "^8.56.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } + "eslint": "^8.57.0 || ^9.0.0" } }, - "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/scope-manager": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", - "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", + "node_modules/@typescript-eslint/visitor-keys": { + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.5.0.tgz", + "integrity": "sha512-yTPqMnbAZJNy2Xq2XU8AdtOW9tJIr+UQb64aXB9f3B1498Zx9JorVgFJcZpEc9UBuCCrdzKID2RGAMkYcDtZOw==", "dev": true, - "optional": true, - "peer": true, "dependencies": { - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0" + "@typescript-eslint/types": "8.5.0", + "eslint-visitor-keys": "^3.4.3" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" } }, - "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/types": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", - "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", + "node_modules/@typescript/vfs": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/@typescript/vfs/-/vfs-1.6.0.tgz", + "integrity": "sha512-hvJUjNVeBMp77qPINuUvYXj4FyWeeMMKZkxEATEU3hqBAQ7qdTBCUFT7Sp0Zu0faeEtFf+ldXxMEDr/bk73ISg==", "dev": true, - "optional": true, - "peer": true, - "engines": { - "node": "^16.0.0 || >=18.0.0" + "dependencies": { + "debug": "^4.1.1" }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" + "peerDependencies": { + "typescript": "*" } }, - "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/typescript-estree": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", - "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", - "dev": true, - "optional": true, - "peer": 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/abort-controller": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/abort-controller/-/abort-controller-3.0.0.tgz", + "integrity": "sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg==", "dependencies": { - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0", - "debug": "^4.3.4", - "globby": "^11.1.0", - "is-glob": "^4.0.3", - "minimatch": "9.0.3", - "semver": "^7.5.4", - "ts-api-utils": "^1.0.1" + "event-target-shim": "^5.0.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } + "node": ">=6.5" } }, - "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/visitor-keys": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", - "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", + "node_modules/acorn": { + "version": "8.12.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", + "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@typescript-eslint/types": "7.2.0", - "eslint-visitor-keys": "^3.4.1" + "bin": { + "acorn": "bin/acorn" }, "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" + "node": ">=0.4.0" } }, - "node_modules/@typescript-eslint/parser/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/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, - "optional": true, - "peer": true, - "dependencies": { - "balanced-match": "^1.0.0" + "peerDependencies": { + "acorn": "^6.0.0 || ^7.0.0 || ^8.0.0" } }, - "node_modules/@typescript-eslint/parser/node_modules/minimatch": { - "version": "9.0.3", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", - "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", - "dev": true, - "optional": true, - "peer": true, + "node_modules/agent-base": { + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.1.tgz", + "integrity": "sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==", "dependencies": { - "brace-expansion": "^2.0.1" + "debug": "^4.3.4" }, "engines": { - "node": ">=16 || 14 >=14.17" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" + "node": ">= 14" } }, - "node_modules/@typescript-eslint/scope-manager": { - "version": "7.18.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.18.0.tgz", - "integrity": "sha512-jjhdIE/FPF2B7Z1uzc6i3oWKbGcHb87Qw7AWj6jmEqNOfDFbJWtjt/XfwCpvNkpGWlcJaog5vTR+VV8+w9JflA==", + "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": { - "@typescript-eslint/types": "7.18.0", - "@typescript-eslint/visitor-keys": "7.18.0" - }, - "engines": { - "node": "^18.18.0 || >=20.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/@typescript-eslint/type-utils": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-7.2.0.tgz", - "integrity": "sha512-xHi51adBHo9O9330J8GQYQwrKBqbIPJGZZVQTHHmy200hvkLZFWJIFtAG/7IYTWUyun6DE6w5InDReePJYJlJA==", - "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@typescript-eslint/typescript-estree": "7.2.0", - "@typescript-eslint/utils": "7.2.0", - "debug": "^4.3.4", - "ts-api-utils": "^1.0.1" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^8.56.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/scope-manager": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", - "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", - "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/types": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", - "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", - "dev": true, - "optional": true, - "peer": true, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/typescript-estree": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", - "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", - "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/visitor-keys": "7.2.0", - "debug": "^4.3.4", - "globby": "^11.1.0", - "is-glob": "^4.0.3", - "minimatch": "9.0.3", - "semver": "^7.5.4", - "ts-api-utils": "^1.0.1" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/utils": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-7.2.0.tgz", - "integrity": "sha512-YfHpnMAGb1Eekpm3XRK8hcMwGLGsnT6L+7b2XyRv6ouDuJU1tZir1GS2i0+VXRatMwSI1/UfcyPe53ADkU+IuA==", - "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@eslint-community/eslint-utils": "^4.4.0", - "@types/json-schema": "^7.0.12", - "@types/semver": "^7.5.0", - "@typescript-eslint/scope-manager": "7.2.0", - "@typescript-eslint/types": "7.2.0", - "@typescript-eslint/typescript-estree": "7.2.0", - "semver": "^7.5.4" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^8.56.0" - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/visitor-keys": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", - "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", - "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "@typescript-eslint/types": "7.2.0", - "eslint-visitor-keys": "^3.4.1" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/@typescript-eslint/type-utils/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, - "optional": true, - "peer": true, - "dependencies": { - "balanced-match": "^1.0.0" - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/minimatch": { - "version": "9.0.3", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", - "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", - "dev": true, - "optional": true, - "peer": true, - "dependencies": { - "brace-expansion": "^2.0.1" - }, - "engines": { - "node": ">=16 || 14 >=14.17" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" - } - }, - "node_modules/@typescript-eslint/types": { - "version": "7.18.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.18.0.tgz", - "integrity": "sha512-iZqi+Ds1y4EDYUtlOOC+aUmxnE9xS/yCigkjA7XpTKV6nCBd3Hp/PRGGmdwnfkV2ThMyYldP1wRpm/id99spTQ==", - "dev": true, - "engines": { - "node": "^18.18.0 || >=20.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/@typescript-eslint/typescript-estree": { - "version": "7.18.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.18.0.tgz", - "integrity": "sha512-aP1v/BSPnnyhMHts8cf1qQ6Q1IFwwRvAQGRvBFkWlo3/lH29OXA3Pts+c10nxRxIBrDnoMqzhgdwVe5f2D6OzA==", - "dev": true, - "dependencies": { - "@typescript-eslint/types": "7.18.0", - "@typescript-eslint/visitor-keys": "7.18.0", - "debug": "^4.3.4", - "globby": "^11.1.0", - "is-glob": "^4.0.3", - "minimatch": "^9.0.4", - "semver": "^7.6.0", - "ts-api-utils": "^1.3.0" - }, - "engines": { - "node": "^18.18.0 || >=20.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/typescript-estree/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/@typescript-eslint/typescript-estree/node_modules/minimatch": { - "version": "9.0.5", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", - "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", - "dev": true, - "dependencies": { - "brace-expansion": "^2.0.1" - }, - "engines": { - "node": ">=16 || 14 >=14.17" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" - } - }, - "node_modules/@typescript-eslint/utils": { - "version": "7.18.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-7.18.0.tgz", - "integrity": "sha512-kK0/rNa2j74XuHVcoCZxdFBMF+aq/vH83CXAOHieC+2Gis4mF8jJXT5eAfyD3K0sAxtPuwxaIOIOvhwzVDt/kw==", - "dev": true, - "dependencies": { - "@eslint-community/eslint-utils": "^4.4.0", - "@typescript-eslint/scope-manager": "7.18.0", - "@typescript-eslint/types": "7.18.0", - "@typescript-eslint/typescript-estree": "7.18.0" - }, - "engines": { - "node": "^18.18.0 || >=20.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^8.56.0" - } - }, - "node_modules/@typescript-eslint/visitor-keys": { - "version": "7.18.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.18.0.tgz", - "integrity": "sha512-cDF0/Gf81QpY3xYyJKDV14Zwdmid5+uuENhjH2EqFaF0ni+yAyq/LzMaIJdhNJXZI7uLzwIlA+V7oWoyn6Curg==", - "dev": true, - "dependencies": { - "@typescript-eslint/types": "7.18.0", - "eslint-visitor-keys": "^3.4.3" - }, - "engines": { - "node": "^18.18.0 || >=20.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/@typescript/vfs": { - "version": "1.6.0", - "resolved": "https://registry.npmjs.org/@typescript/vfs/-/vfs-1.6.0.tgz", - "integrity": "sha512-hvJUjNVeBMp77qPINuUvYXj4FyWeeMMKZkxEATEU3hqBAQ7qdTBCUFT7Sp0Zu0faeEtFf+ldXxMEDr/bk73ISg==", - "dev": true, - "dependencies": { - "debug": "^4.1.1" - }, - "peerDependencies": { - "typescript": "*" - } - }, - "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/abort-controller": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/abort-controller/-/abort-controller-3.0.0.tgz", - "integrity": "sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg==", - "dependencies": { - "event-target-shim": "^5.0.0" - }, - "engines": { - "node": ">=6.5" - } - }, - "node_modules/acorn": { - "version": "8.12.1", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", - "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", - "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", - "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" + "humanize-ms": "^1.2.1" }, "engines": { "node": ">= 8.0.0" @@ -3946,6 +3657,7 @@ "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" }, @@ -3960,6 +3672,7 @@ "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" }, @@ -3993,6 +3706,7 @@ "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" @@ -4025,15 +3739,6 @@ "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", "dev": true }, - "node_modules/array-union": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/array-union/-/array-union-2.1.0.tgz", - "integrity": "sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw==", - "dev": true, - "engines": { - "node": ">=8" - } - }, "node_modules/asn1": { "version": "0.2.6", "resolved": "https://registry.npmjs.org/asn1/-/asn1-0.2.6.tgz", @@ -4051,16 +3756,15 @@ } }, "node_modules/async": { - "version": "3.2.5", - "resolved": "https://registry.npmjs.org/async/-/async-3.2.5.tgz", - "integrity": "sha512-baNZyqaaLhyLVKm/DlvdW051MSgO6b8eVfIezl9E5PqWxFgzLm/wQntEW4zOytVburDEr0JlALEpdOFwvErLsg==", - "license": "MIT" + "version": "3.2.6", + "resolved": "https://registry.npmjs.org/async/-/async-3.2.6.tgz", + "integrity": "sha512-htCUDlxyyCLMgaM3xXg0C0LW2xqfuQ6p05pCEIsXuyQ+a1koYKTuBMzRNwmybfLgvJDMd0r1LTn4+E0Ti6C2AA==", + "dev": true }, "node_modules/async-mutex": { "version": "0.5.0", "resolved": "https://registry.npmjs.org/async-mutex/-/async-mutex-0.5.0.tgz", "integrity": "sha512-1A94B18jkJ3DYq284ohPxoXbfTA5HsQ7/Mf4DEhcyLx3Bz27Rh59iScbB6EPiP+B+joue6YCxcMXSbFC1tZKwA==", - "license": "MIT", "dependencies": { "tslib": "^2.4.0" } @@ -4068,8 +3772,7 @@ "node_modules/asynckit": { "version": "0.4.0", "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz", - "integrity": "sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==", - "license": "MIT" + "integrity": "sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==" }, "node_modules/avsc": { "version": "5.7.7", @@ -4088,15 +3791,14 @@ } }, "node_modules/aws4": { - "version": "1.13.1", - "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.13.1.tgz", - "integrity": "sha512-u5w79Rd7SU4JaIlA/zFqG+gOiuq25q5VLyZ8E+ijJeILuTxVzZgp2CaGw/UTw6pXYN9XMO9yiqj/nEHmhTG5CA==" + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.13.2.tgz", + "integrity": "sha512-lHe62zvbTB5eEABUVi/AwVh0ZKY9rMMDhmm+eeyuuUQbQ3+J+fONVQOZyj+DdrvD4BY33uYniyRJ4UJIaSKAfw==" }, "node_modules/axios": { - "version": "1.7.3", - "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.3.tgz", - "integrity": "sha512-Ar7ND9pU99eJ9GpoGQKhKf58GpUOgnzuaB7ueNQ5BMi0p+LZ5oaEnfF999fAArcTIBwXTCHAmGcHOZJaWPq9Nw==", - "license": "MIT", + "version": "1.7.7", + "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.7.tgz", + "integrity": "sha512-S4kL7XrjgBmvdGut0sN3yJxqYzrDOnivkBiN0OFs6hLiUam3UPvswUo0kqGyhqUZGEOytHyumEdXsAkgCOUf3Q==", "dependencies": { "follow-redirects": "^1.15.6", "form-data": "^4.0.0", @@ -4107,6 +3809,7 @@ "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", @@ -4127,6 +3830,7 @@ "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", @@ -4142,6 +3846,7 @@ "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", @@ -4157,6 +3862,7 @@ "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" } @@ -4165,6 +3871,7 @@ "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", @@ -4176,22 +3883,26 @@ } }, "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==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/babel-preset-current-node-syntax/-/babel-preset-current-node-syntax-1.1.0.tgz", + "integrity": "sha512-ldYss8SbBlWva1bs28q78Ju5Zq1F+8BrqBZZ0VFhLBvhh6lCpC2o3gDJi/5DRLs9FgYZCnmPYIVFU4lRXCkyUw==", + "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-class-properties": "^7.12.13", + "@babel/plugin-syntax-class-static-block": "^7.14.5", + "@babel/plugin-syntax-import-attributes": "^7.24.7", + "@babel/plugin-syntax-import-meta": "^7.10.4", "@babel/plugin-syntax-json-strings": "^7.8.3", - "@babel/plugin-syntax-logical-assignment-operators": "^7.8.3", + "@babel/plugin-syntax-logical-assignment-operators": "^7.10.4", "@babel/plugin-syntax-nullish-coalescing-operator": "^7.8.3", - "@babel/plugin-syntax-numeric-separator": "^7.8.3", + "@babel/plugin-syntax-numeric-separator": "^7.10.4", "@babel/plugin-syntax-object-rest-spread": "^7.8.3", "@babel/plugin-syntax-optional-catch-binding": "^7.8.3", "@babel/plugin-syntax-optional-chaining": "^7.8.3", - "@babel/plugin-syntax-top-level-await": "^7.8.3" + "@babel/plugin-syntax-private-property-in-object": "^7.14.5", + "@babel/plugin-syntax-top-level-await": "^7.14.5" }, "peerDependencies": { "@babel/core": "^7.0.0" @@ -4201,6 +3912,7 @@ "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" @@ -4296,6 +4008,7 @@ "version": "3.0.3", "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", + "dev": true, "dependencies": { "fill-range": "^7.1.1" }, @@ -4321,6 +4034,7 @@ "version": "4.23.3", "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.3.tgz", "integrity": "sha512-btwCFJVjI4YWDNfau8RhZ+B1Q/VLoUITrm3RlP6y1tYGWIOa+InuYiRGXUBXo8nA1qKmHMyLB/iVQg5TT4eFoA==", + "dev": true, "funding": [ { "type": "opencollective", @@ -4352,7 +4066,7 @@ "version": "0.2.6", "resolved": "https://registry.npmjs.org/bs-logger/-/bs-logger-0.2.6.tgz", "integrity": "sha512-pd8DCoxmbgc7hyPKOvxtqNcjYoOsABPQdcCUjGp3d42VR2CX1ORhk2A87oqqu5R1kk+76nsxZupkmyd+MVtCog==", - "license": "MIT", + "dev": true, "dependencies": { "fast-json-stable-stringify": "2.x" }, @@ -4364,6 +4078,7 @@ "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" } @@ -4376,7 +4091,8 @@ "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==" + "integrity": "sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ==", + "dev": true }, "node_modules/cacache": { "version": "16.1.3", @@ -4461,6 +4177,7 @@ "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" } @@ -4469,14 +4186,16 @@ "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.30001646", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001646.tgz", - "integrity": "sha512-dRg00gudiBDDTmUhClSdv3hqRfpbOnU28IpI1T6PBTLWa+kOj0681C8uML3PifYfREuBrVjDGhL3adYpBT6spw==", + "version": "1.0.30001660", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001660.tgz", + "integrity": "sha512-GacvNTTuATm26qC74pt+ad1fW15mlQ/zuTzzY1ZoIzECTP8HURDfF43kNxPgf7H1jmelCBQTTbBNxdSXOA7Bqg==", + "dev": true, "funding": [ { "type": "opencollective", @@ -4513,6 +4232,7 @@ "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" @@ -4528,6 +4248,7 @@ "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" } @@ -4580,6 +4301,7 @@ "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", @@ -4591,9 +4313,10 @@ } }, "node_modules/cjs-module-lexer": { - "version": "1.3.1", - "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.3.1.tgz", - "integrity": "sha512-a3KdPAANPbNE4ZUv9h6LckSl9zLsYOP4MBmhIPkRaeyybt+r4UghLvq+xw/YwUcC1gqylCkL4rdVs3Lwupjm4Q==" + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.4.1.tgz", + "integrity": "sha512-cuSVIHi9/9E/+821Qjdvngor+xpnlwnuwIyZOaLmHBVdXL+gP+I6QQB9VkO7RI77YIcTV+S1W9AreJ5eN63JBA==", + "dev": true }, "node_modules/clean-stack": { "version": "2.2.0", @@ -4621,6 +4344,7 @@ "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" @@ -4629,7 +4353,8 @@ "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==" + "integrity": "sha512-lHl4d5/ONEbLlJvaJNtsF/Lz+WvB07u2ycqTYbdrq7UypDXailES4valYb2eWiJFxZlVmpGekfqoxQhzyFdT4Q==", + "dev": true }, "node_modules/color-convert": { "version": "2.0.1", @@ -4659,7 +4384,6 @@ "version": "1.0.8", "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.8.tgz", "integrity": "sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg==", - "license": "MIT", "dependencies": { "delayed-stream": "~1.0.0" }, @@ -4685,7 +4409,8 @@ "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==" + "integrity": "sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg==", + "dev": true }, "node_modules/core-util-is": { "version": "1.0.2", @@ -4696,6 +4421,7 @@ "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", @@ -4716,6 +4442,7 @@ "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", @@ -4737,11 +4464,11 @@ } }, "node_modules/debug": { - "version": "4.3.6", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.6.tgz", - "integrity": "sha512-O/09Bd4Z1fBrU4VzkhFqVgpPzaGbw6Sm9FEkBT1A/YBXQFGuuSxa1dN2nxgxS34JmKXqYx8CZAwEVoJFImUXIg==", + "version": "4.3.7", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.7.tgz", + "integrity": "sha512-Er2nc/H7RrMXZBFCEim6TCmMk02Z8vLC2Rbi1KEBggpo0fS6l0S1nnapwmIi3yW/+GOJap1Krg4w0Hg80oCqgQ==", "dependencies": { - "ms": "2.1.2" + "ms": "^2.1.3" }, "engines": { "node": ">=6.0" @@ -4768,6 +4495,7 @@ "version": "1.5.3", "resolved": "https://registry.npmjs.org/dedent/-/dedent-1.5.3.tgz", "integrity": "sha512-NHQtfOOW68WD8lgypbLA5oT+Bt0xXJhiYvoR6SmmNXZfpzOGXwdKWmcwG8N7PwVVWV3eF/68nmD9BaJSsTBhyQ==", + "dev": true, "peerDependencies": { "babel-plugin-macros": "^3.1.0" }, @@ -4787,6 +4515,7 @@ "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" } @@ -4803,7 +4532,6 @@ "version": "1.0.0", "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz", "integrity": "sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ==", - "license": "MIT", "engines": { "node": ">=0.4.0" } @@ -4825,6 +4553,7 @@ "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" } @@ -4842,20 +4571,9 @@ "version": "29.6.3", "resolved": "https://registry.npmjs.org/diff-sequences/-/diff-sequences-29.6.3.tgz", "integrity": "sha512-EjePK1srD3P08o2j4f0ExnylqRs5B9tJjcp9t1krH2qRi8CCdsYfwe9JgSLurFBWwq4uOlipzfk5fHNvwFKr8Q==", - "engines": { - "node": "^14.15.0 || ^16.10.0 || >=18.0.0" - } - }, - "node_modules/dir-glob": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-3.0.1.tgz", - "integrity": "sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA==", "dev": true, - "dependencies": { - "path-type": "^4.0.0" - }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, "node_modules/discontinuous-range": { @@ -4912,7 +4630,7 @@ "version": "3.1.10", "resolved": "https://registry.npmjs.org/ejs/-/ejs-3.1.10.tgz", "integrity": "sha512-UeJmFfOrAQS8OJWPZ4qtgHyWExa088/MtK5UEyoJGFH67cDEXkZSviOiKRCZ4Xij0zxI3JECgYs3oKx+AizQBA==", - "license": "Apache-2.0", + "dev": true, "dependencies": { "jake": "^10.8.5" }, @@ -4924,14 +4642,16 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.5.4", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.4.tgz", - "integrity": "sha512-orzA81VqLyIGUEA77YkVA1D+N+nNfl2isJVjjmOyrlxuooZ19ynb+dOlaDTqd/idKRS9lDCSBmtzM+kyCsMnkA==" + "version": "1.5.20", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.20.tgz", + "integrity": "sha512-74mdl6Fs1HHzK9SUX4CKFxAtAe3nUns48y79TskHNAG6fGOlLfyKA4j855x+0b5u8rWJIrlaG9tcTPstMlwjIw==", + "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" }, @@ -4992,14 +4712,15 @@ "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.2", - "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.2.tgz", - "integrity": "sha512-ErCHMCae19vR8vQGe50xIsVomy19rg6gFu3+r3jkEO46suLMWBksvVyoGgQV+jOfl84ZSOSlmv6Gxa89PmTGmA==", + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.2.0.tgz", + "integrity": "sha512-WUj2qlxaQtO4g6Pq5c29GTcWGDyd8itL8zTlipgECz3JesAiiOKotd8JU6otB3PACgG6xkJUyVhboMS+bje/jA==", "engines": { "node": ">=6" } @@ -5072,18 +4793,18 @@ } }, "node_modules/eslint-plugin-jest": { - "version": "28.6.0", - "resolved": "https://registry.npmjs.org/eslint-plugin-jest/-/eslint-plugin-jest-28.6.0.tgz", - "integrity": "sha512-YG28E1/MIKwnz+e2H7VwYPzHUYU4aMa19w0yGcwXnnmJH6EfgHahTJ2un3IyraUxNfnz/KUhJAFXNNwWPo12tg==", + "version": "28.8.3", + "resolved": "https://registry.npmjs.org/eslint-plugin-jest/-/eslint-plugin-jest-28.8.3.tgz", + "integrity": "sha512-HIQ3t9hASLKm2IhIOqnu+ifw7uLZkIlR7RYNv7fMcEi/p0CIiJmfriStQS2LDkgtY4nyLbIZAD+JL347Yc2ETQ==", "dev": true, "dependencies": { - "@typescript-eslint/utils": "^6.0.0 || ^7.0.0" + "@typescript-eslint/utils": "^6.0.0 || ^7.0.0 || ^8.0.0" }, "engines": { "node": "^16.10.0 || ^18.12.0 || >=20.0.0" }, "peerDependencies": { - "@typescript-eslint/eslint-plugin": "^6.0.0 || ^7.0.0", + "@typescript-eslint/eslint-plugin": "^6.0.0 || ^7.0.0 || ^8.0.0", "eslint": "^7.0.0 || ^8.0.0 || ^9.0.0", "jest": "*" }, @@ -5176,6 +4897,7 @@ "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" @@ -5246,6 +4968,7 @@ "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", @@ -5268,6 +4991,7 @@ "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" } @@ -5276,6 +5000,7 @@ "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", @@ -5389,6 +5114,7 @@ "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" } @@ -5414,7 +5140,7 @@ "version": "1.0.4", "resolved": "https://registry.npmjs.org/filelist/-/filelist-1.0.4.tgz", "integrity": "sha512-w1cEuf3S+DrLCQL7ET6kz+gmlJdbq9J7yXCSjK/OZCPA+qEN1WyF4ZAf0YYJa4/shHJra2t/d/r8SV4Ji+x+8Q==", - "license": "Apache-2.0", + "dev": true, "dependencies": { "minimatch": "^5.0.1" } @@ -5423,7 +5149,7 @@ "version": "2.0.1", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-2.0.1.tgz", "integrity": "sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA==", - "license": "MIT", + "dev": true, "dependencies": { "balanced-match": "^1.0.0" } @@ -5432,7 +5158,7 @@ "version": "5.1.6", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.1.6.tgz", "integrity": "sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g==", - "license": "ISC", + "dev": true, "dependencies": { "brace-expansion": "^2.0.1" }, @@ -5444,6 +5170,7 @@ "version": "7.1.1", "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", + "dev": true, "dependencies": { "to-regex-range": "^5.0.1" }, @@ -5497,16 +5224,15 @@ "dev": true }, "node_modules/follow-redirects": { - "version": "1.15.6", - "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.6.tgz", - "integrity": "sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA==", + "version": "1.15.9", + "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.9.tgz", + "integrity": "sha512-gew4GsXizNgdoRyqmyfMHyAmXsZDk6mHkSxZFCzW9gwlbtOW44CDtYavM+y+72qD/Vq2l550kMF52DT8fOLJqQ==", "funding": [ { "type": "individual", "url": "https://github.com/sponsors/RubenVerborgh" } ], - "license": "MIT", "engines": { "node": ">=4.0" }, @@ -5528,7 +5254,6 @@ "version": "4.0.0", "resolved": "https://registry.npmjs.org/form-data/-/form-data-4.0.0.tgz", "integrity": "sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww==", - "license": "MIT", "dependencies": { "asynckit": "^0.4.0", "combined-stream": "^1.0.8", @@ -5558,6 +5283,7 @@ "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": [ @@ -5571,6 +5297,7 @@ "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" } @@ -5610,27 +5337,16 @@ "node": ">=14" } }, - "node_modules/gaxios/node_modules/agent-base": { - "version": "7.1.1", - "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.1.tgz", - "integrity": "sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==", - "dependencies": { - "debug": "^4.3.4" - }, - "engines": { - "node": ">= 14" - } - }, - "node_modules/gaxios/node_modules/https-proxy-agent": { - "version": "7.0.5", - "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz", - "integrity": "sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==", - "dependencies": { - "agent-base": "^7.0.2", - "debug": "4" - }, - "engines": { - "node": ">= 14" + "node_modules/gaxios/node_modules/uuid": { + "version": "9.0.1", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", + "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "bin": { + "uuid": "dist/bin/uuid" } }, "node_modules/gcp-metadata": { @@ -5649,6 +5365,7 @@ "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" } @@ -5665,6 +5382,7 @@ "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" } @@ -5673,6 +5391,7 @@ "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" }, @@ -5735,30 +5454,10 @@ "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/globby": { - "version": "11.1.0", - "resolved": "https://registry.npmjs.org/globby/-/globby-11.1.0.tgz", - "integrity": "sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g==", - "dev": true, - "dependencies": { - "array-union": "^2.1.0", - "dir-glob": "^3.0.1", - "fast-glob": "^3.2.9", - "ignore": "^5.2.0", - "merge2": "^1.4.1", - "slash": "^3.0.0" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, "node_modules/google-auth-library": { - "version": "9.14.0", - "resolved": "https://registry.npmjs.org/google-auth-library/-/google-auth-library-9.14.0.tgz", - "integrity": "sha512-Y/eq+RWVs55Io/anIsm24sDS8X79Tq948zVLGaa7+KlJYYqaGwp1YI37w48nzrNi12RgnzMrQD4NzdmCowT90g==", + "version": "9.14.1", + "resolved": "https://registry.npmjs.org/google-auth-library/-/google-auth-library-9.14.1.tgz", + "integrity": "sha512-Rj+PMjoNFGFTmtItH7gHfbHpGVSb3vmnGK3nwNBqxQF9NoBpttSZI/rc0WiM63ma2uGDQtYEkMHkK9U6937NiA==", "dependencies": { "base64-js": "^1.3.0", "ecdsa-sig-formatter": "^1.0.11", @@ -5772,9 +5471,9 @@ } }, "node_modules/google-gax": { - "version": "4.3.9", - "resolved": "https://registry.npmjs.org/google-gax/-/google-gax-4.3.9.tgz", - "integrity": "sha512-tcjQr7sXVGMdlvcG25wSv98ap1dtF4Z6mcV0rztGIddOcezw4YMb/uTXg72JPrLep+kXcVjaJjg6oo3KLf4itQ==", + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/google-gax/-/google-gax-4.4.1.tgz", + "integrity": "sha512-Phyp9fMfA00J3sZbJxbbB4jC55b7DBjE3F6poyL3wKMEBVKA79q6BGuHcTiM28yOzVql0NDbRL8MLLh8Iwk9Dg==", "dependencies": { "@grpc/grpc-js": "^1.10.9", "@grpc/proto-loader": "^0.7.13", @@ -5793,10 +5492,23 @@ "node": ">=14" } }, + "node_modules/google-gax/node_modules/uuid": { + "version": "9.0.1", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", + "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "bin": { + "uuid": "dist/bin/uuid" + } + }, "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==" + "integrity": "sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ==", + "dev": true }, "node_modules/graphemer": { "version": "1.4.0", @@ -5861,6 +5573,7 @@ "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" } @@ -5874,6 +5587,7 @@ "version": "2.0.2", "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.2.tgz", "integrity": "sha512-0hJU9SCPvmMzIBdZFqNPXWa6dqh7WdH0cII9y+CyS8rG3nL48Bclra9HmKhVVUHyPWNH5Y7xDwAB7bfgSjkUMQ==", + "dev": true, "dependencies": { "function-bind": "^1.1.2" }, @@ -5893,7 +5607,8 @@ "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==" + "integrity": "sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg==", + "dev": true }, "node_modules/http-cache-semantics": { "version": "4.1.1", @@ -5902,16 +5617,15 @@ "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==", + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/http-proxy-agent/-/http-proxy-agent-7.0.2.tgz", + "integrity": "sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig==", "dependencies": { - "@tootallnate/once": "2", - "agent-base": "6", - "debug": "4" + "agent-base": "^7.1.0", + "debug": "^4.3.4" }, "engines": { - "node": ">= 6" + "node": ">= 14" } }, "node_modules/http-signature": { @@ -5928,21 +5642,22 @@ } }, "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==", + "version": "7.0.5", + "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz", + "integrity": "sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==", "dependencies": { - "agent-base": "6", + "agent-base": "^7.0.2", "debug": "4" }, "engines": { - "node": ">= 6" + "node": ">= 14" } }, "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" } @@ -5969,9 +5684,9 @@ } }, "node_modules/ignore": { - "version": "5.3.1", - "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.3.1.tgz", - "integrity": "sha512-5Fytz/IraMjqpwfd34ke28PTVMjZjJG2MPn5t7OE4eUCUNf8BAa7b5WUS9/Qvr6mwOQS7Mk6vdsMno5he+T8Xw==", + "version": "5.3.2", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.3.2.tgz", + "integrity": "sha512-hsBTNUqQTDwkWtcdYI2i06Y/nUBEsNEDJKjWdigLvegy8kDuJAS8uRlpkkcQpyEXL0Z/pjDy5HBmMjRCJ2gq+g==", "dev": true, "engines": { "node": ">= 4" @@ -5997,6 +5712,7 @@ "version": "3.2.0", "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.2.0.tgz", "integrity": "sha512-2SPlun1JUPWoM6t3F0dw0FkCF/jWY8kttcY4f599GLTSjh2OCuuhdTkJQsEcZzBqbXZGKMK2OqW1oZsjtf/gQA==", + "dev": true, "dependencies": { "pkg-dir": "^4.2.0", "resolve-cwd": "^3.0.0" @@ -6015,6 +5731,7 @@ "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" } @@ -6065,7 +5782,8 @@ "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==" + "integrity": "sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg==", + "dev": true }, "node_modules/is-binary-path": { "version": "2.1.0", @@ -6080,9 +5798,10 @@ } }, "node_modules/is-core-module": { - "version": "2.15.0", - "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.15.0.tgz", - "integrity": "sha512-Dd+Lb2/zvk9SKy1TGCt1wFJFo/MWBPMX5x7KcvLajWTGuomczdQX61PvY5yK6SVACwpoexWo81IfFyoKY2QnTA==", + "version": "2.15.1", + "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.15.1.tgz", + "integrity": "sha512-z0vtXSwucUJtANQWldhbtbt7BnL0vxiFjIdDLAatwhDYty2bad6s+rijD6Ri4YuYJubLzIJLUidCh09e1djEVQ==", + "dev": true, "dependencies": { "hasown": "^2.0.2" }, @@ -6128,6 +5847,7 @@ "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" } @@ -6154,6 +5874,7 @@ "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" } @@ -6218,7 +5939,8 @@ "node_modules/isexe": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", - "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==" + "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==", + "dev": true }, "node_modules/isstream": { "version": "0.1.2", @@ -6229,6 +5951,7 @@ "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, "engines": { "node": ">=8" } @@ -6237,6 +5960,7 @@ "version": "6.0.3", "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.3.tgz", "integrity": "sha512-Vtgk7L/R2JHyyGW07spoFlB8/lpjiOLTjMdms6AFMraYt3BaJauod/NGrfnVG/y4Ix1JEuMRPDPEj2ua+zz1/Q==", + "dev": true, "dependencies": { "@babel/core": "^7.23.9", "@babel/parser": "^7.23.9", @@ -6252,6 +5976,7 @@ "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": { "istanbul-lib-coverage": "^3.0.0", "make-dir": "^4.0.0", @@ -6265,6 +5990,7 @@ "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" }, @@ -6279,6 +6005,7 @@ "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": { "debug": "^4.1.1", "istanbul-lib-coverage": "^3.0.0", @@ -6292,6 +6019,7 @@ "version": "3.1.7", "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-3.1.7.tgz", "integrity": "sha512-BewmUXImeuRk2YY0PVbxgKAysvhRPUQE0h5QRM++nVWyubKGV0l8qQ5op8+B2DOmwSe63Jivj0BjkPQVf8fP5g==", + "dev": true, "dependencies": { "html-escaper": "^2.0.0", "istanbul-lib-report": "^3.0.0" @@ -6304,7 +6032,7 @@ "version": "10.9.2", "resolved": "https://registry.npmjs.org/jake/-/jake-10.9.2.tgz", "integrity": "sha512-2P4SQ0HrLQ+fw6llpLnOaGAvN2Zu6778SJMrCUwns4fOoG9ayrTiZk3VV8sCPkVZF8ab0zksVpS8FDY5pRCNBA==", - "license": "Apache-2.0", + "dev": true, "dependencies": { "async": "^3.2.3", "chalk": "^4.0.2", @@ -6322,6 +6050,7 @@ "version": "29.7.0", "resolved": "https://registry.npmjs.org/jest/-/jest-29.7.0.tgz", "integrity": "sha512-NIy3oAFp9shda19hy4HK0HRTWKtPJmGdnvywu01nOqNC2vZg+Z+fvJDxpMQA88eb2I9EcafcdjYgsDthnYTvGw==", + "dev": true, "dependencies": { "@jest/core": "^29.7.0", "@jest/types": "^29.6.3", @@ -6347,6 +6076,7 @@ "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, "dependencies": { "execa": "^5.0.0", "jest-util": "^29.7.0", @@ -6360,6 +6090,7 @@ "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": { "@jest/environment": "^29.7.0", "@jest/expect": "^29.7.0", @@ -6390,6 +6121,7 @@ "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": { "@jest/core": "^29.7.0", "@jest/test-result": "^29.7.0", @@ -6422,6 +6154,7 @@ "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", @@ -6466,6 +6199,7 @@ "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", @@ -6480,6 +6214,7 @@ "version": "29.7.0", "resolved": "https://registry.npmjs.org/jest-docblock/-/jest-docblock-29.7.0.tgz", "integrity": "sha512-q617Auw3A612guyaFgsbFeYpNP5t2aoUNLwBUbc/0kD1R4t9ixDbyFTHd1nok4epoVFpr7PmeWHrhvuV3XaJ4g==", + "dev": true, "dependencies": { "detect-newline": "^3.0.0" }, @@ -6491,6 +6226,7 @@ "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": { "@jest/types": "^29.6.3", "chalk": "^4.0.0", @@ -6506,6 +6242,7 @@ "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": { "@jest/environment": "^29.7.0", "@jest/fake-timers": "^29.7.0", @@ -6522,6 +6259,7 @@ "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": "^14.15.0 || ^16.10.0 || >=18.0.0" } @@ -6530,6 +6268,7 @@ "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": { "@jest/types": "^29.6.3", "@types/graceful-fs": "^4.1.3", @@ -6554,6 +6293,7 @@ "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": { "jest-get-type": "^29.6.3", "pretty-format": "^29.7.0" @@ -6566,6 +6306,7 @@ "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", @@ -6580,6 +6321,7 @@ "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", @@ -6599,6 +6341,7 @@ "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": { "@jest/types": "^29.6.3", "@types/node": "*", @@ -6612,6 +6355,7 @@ "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" }, @@ -6628,6 +6372,7 @@ "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" } @@ -6636,6 +6381,7 @@ "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": { "chalk": "^4.0.0", "graceful-fs": "^4.2.9", @@ -6655,6 +6401,7 @@ "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" @@ -6667,6 +6414,7 @@ "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", @@ -6698,6 +6446,7 @@ "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": { "@jest/environment": "^29.7.0", "@jest/fake-timers": "^29.7.0", @@ -6730,6 +6479,7 @@ "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", @@ -6760,6 +6510,7 @@ "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": "*", @@ -6776,6 +6527,7 @@ "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", @@ -6792,6 +6544,7 @@ "version": "6.3.0", "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", + "dev": true, "engines": { "node": ">=10" }, @@ -6803,6 +6556,7 @@ "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", @@ -6821,6 +6575,7 @@ "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", @@ -6835,6 +6590,7 @@ "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" }, @@ -6848,7 +6604,8 @@ "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==" + "integrity": "sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==", + "dev": true }, "node_modules/js-yaml": { "version": "4.1.0", @@ -6919,6 +6676,7 @@ "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" }, @@ -6943,7 +6701,8 @@ "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==" + "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==", + "dev": true }, "node_modules/json-schema": { "version": "0.4.0", @@ -6965,7 +6724,6 @@ "version": "1.0.12", "resolved": "https://registry.npmjs.org/json-stringify-deterministic/-/json-stringify-deterministic-1.0.12.tgz", "integrity": "sha512-q3PN0lbUdv0pmurkBNdJH3pfFvOTL/Zp0lquqpvcjfKzt6Y0j49EPHAmVHCAS4Ceq/Y+PejWTzyiVpoY71+D6g==", - "license": "MIT", "engines": { "node": ">= 4" } @@ -6979,6 +6737,7 @@ "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" }, @@ -7089,6 +6848,7 @@ "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" } @@ -7097,6 +6857,7 @@ "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" } @@ -7117,7 +6878,8 @@ "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==" + "integrity": "sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg==", + "dev": true }, "node_modules/linkify-it": { "version": "5.0.0", @@ -7188,7 +6950,7 @@ "version": "4.1.2", "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", "integrity": "sha512-t7j+NzmgnQzTAYXcsHYLgimltOV1MXHtlOWf6GjL9Kj8GK5FInw5JotxvbOs+IvV1/Dzo04/fCGfLVs7aXb4Ag==", - "license": "MIT" + "dev": true }, "node_modules/lodash.merge": { "version": "4.6.2", @@ -7223,10 +6985,9 @@ "integrity": "sha512-lcHwpNoggQTObv5apGNCTdJrO69eHOZMi4BNC+rTLER8iHAqGrUVeLh/irVIM7zTw2bOXA8T6uNPeujwOLg/2Q==" }, "node_modules/lru-cache": { - "version": "11.0.0", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-11.0.0.tgz", - "integrity": "sha512-Qv32eSV1RSCfhY3fpPE2GNZ8jgM9X7rdAfemLWqTUxwiyIC4jJ6Sy0fZ8H+oLWevO6i4/bizg7c8d8i6bxrzbA==", - "license": "ISC", + "version": "11.0.1", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-11.0.1.tgz", + "integrity": "sha512-CgeuL5uom6j/ZVrg7G/+1IXqRY8JXX4Hghfy5YE0EhoYQWvndP1kufu58cmZLNIDKnRhZrXfdS9urVWx98AipQ==", "engines": { "node": "20 || >=22" } @@ -7257,7 +7018,7 @@ "version": "1.3.6", "resolved": "https://registry.npmjs.org/make-error/-/make-error-1.3.6.tgz", "integrity": "sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw==", - "license": "ISC" + "dev": true }, "node_modules/make-fetch-happen": { "version": "10.2.1", @@ -7286,6 +7047,45 @@ "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, + "node_modules/make-fetch-happen/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==", + "dev": true, + "dependencies": { + "debug": "4" + }, + "engines": { + "node": ">= 6.0.0" + } + }, + "node_modules/make-fetch-happen/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": { + "@tootallnate/once": "2", + "agent-base": "6", + "debug": "4" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/make-fetch-happen/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==", + "dev": true, + "dependencies": { + "agent-base": "6", + "debug": "4" + }, + "engines": { + "node": ">= 6" + } + }, "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", @@ -7299,6 +7099,7 @@ "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" } @@ -7351,7 +7152,8 @@ "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==" + "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==", + "dev": true }, "node_modules/merge2": { "version": "1.4.1", @@ -7363,9 +7165,10 @@ } }, "node_modules/micromatch": { - "version": "4.0.7", - "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.7.tgz", - "integrity": "sha512-LPP/3KorzCwBxfeUuZmaR6bG2kdeHSbe0P2tY3FLRU4vYrjYz5hI4QZwV0njUx3jeuKe67YukQ1LSPZBKDqO/Q==", + "version": "4.0.8", + "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz", + "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==", + "dev": true, "dependencies": { "braces": "^3.0.3", "picomatch": "^2.3.1" @@ -7378,7 +7181,6 @@ "version": "1.52.0", "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.52.0.tgz", "integrity": "sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==", - "license": "MIT", "engines": { "node": ">= 0.6" } @@ -7387,7 +7189,6 @@ "version": "2.1.35", "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.35.tgz", "integrity": "sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==", - "license": "MIT", "dependencies": { "mime-db": "1.52.0" }, @@ -7399,6 +7200,7 @@ "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": ">=6" } @@ -7490,11 +7292,6 @@ "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", "resolved": "https://registry.npmjs.org/minizlib/-/minizlib-2.1.2.tgz", @@ -7507,11 +7304,6 @@ "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", "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-1.0.4.tgz", @@ -7524,9 +7316,9 @@ } }, "node_modules/mocha": { - "version": "10.7.0", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.7.0.tgz", - "integrity": "sha512-v8/rBWr2VO5YkspYINnvu81inSz2y3ODJrhO175/Exzor1RcEZZkizgE2A+w/CAXXoESS8Kys5E62dOHGHzULA==", + "version": "10.7.3", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.7.3.tgz", + "integrity": "sha512-uQWxAu44wwiACGqjbPYmjo7Lg8sFrS3dQe7PP2FQI+woptP4vZXSMcfMyFL/e1yFEeEpV4RtyTpZROOKmxis+A==", "dev": true, "dependencies": { "ansi-colors": "^4.1.3", @@ -7610,12 +7402,6 @@ "node": ">=10" } }, - "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/supports-color": { "version": "8.1.1", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", @@ -7655,9 +7441,9 @@ "integrity": "sha512-iSAJLHYKnX41mKcJKjqvnAN9sf0LMDTXDEvFv+ffuRR9a1MIuXLjMNL6EsnDHSkKLTWNqQQ5uo61P4EbU4NU+Q==" }, "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==" + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==" }, "node_modules/mustache": { "version": "4.2.0", @@ -7675,7 +7461,8 @@ "node_modules/natural-compare": { "version": "1.4.0", "resolved": "https://registry.npmjs.org/natural-compare/-/natural-compare-1.4.0.tgz", - "integrity": "sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw==" + "integrity": "sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw==", + "dev": true }, "node_modules/nearley": { "version": "2.20.1", @@ -7819,12 +7606,14 @@ "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==" + "integrity": "sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw==", + "dev": true }, "node_modules/node-releases": { "version": "2.0.18", "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.18.tgz", - "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==" + "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==", + "dev": true }, "node_modules/node-vault": { "version": "0.10.2", @@ -7858,6 +7647,7 @@ "version": "3.0.0", "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", + "dev": true, "engines": { "node": ">=0.10.0" } @@ -7866,6 +7656,7 @@ "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" }, @@ -7921,6 +7712,7 @@ "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" }, @@ -7968,6 +7760,7 @@ "version": "3.1.0", "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dev": true, "dependencies": { "yocto-queue": "^0.1.0" }, @@ -8012,6 +7805,7 @@ "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" } @@ -8032,6 +7826,7 @@ "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", @@ -8049,6 +7844,7 @@ "version": "4.0.0", "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==", + "dev": true, "engines": { "node": ">=8" } @@ -8065,6 +7861,7 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", + "dev": true, "engines": { "node": ">=8" } @@ -8072,16 +7869,8 @@ "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==" - }, - "node_modules/path-type": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", - "integrity": "sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==", - "dev": true, - "engines": { - "node": ">=8" - } + "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==", + "dev": true }, "node_modules/performance-now": { "version": "2.1.0", @@ -8089,14 +7878,16 @@ "integrity": "sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==" }, "node_modules/picocolors": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.1.tgz", - "integrity": "sha512-anP1Z8qwhkbmu7MFP5iTt+wQKXgwzf7zTyGlcdzabySa9vd0Xt392U0rVmz9poOaBj0uHJKyyo9/upk0HrEQew==" + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.1.0.tgz", + "integrity": "sha512-TQ92mBOW0l3LeMeyLV6mzy/kWr8lkd/hp3mTg7wYK7zJhuBStmGMBG0BdeDZS/dZx1IukaX6Bk11zcln25o1Aw==", + "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, "engines": { "node": ">=8.6" }, @@ -8108,6 +7899,7 @@ "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": ">= 6" } @@ -8116,6 +7908,7 @@ "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" }, @@ -8127,6 +7920,7 @@ "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" @@ -8139,6 +7933,7 @@ "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" }, @@ -8150,6 +7945,7 @@ "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" }, @@ -8164,6 +7960,7 @@ "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" }, @@ -8172,9 +7969,9 @@ } }, "node_modules/postman-request": { - "version": "2.88.1-postman.39", - "resolved": "https://registry.npmjs.org/postman-request/-/postman-request-2.88.1-postman.39.tgz", - "integrity": "sha512-rsncxxDlbn1YpygXSgJqbJzIjGlHFcZjbYDzeBPTQHMDfLuSTzZz735JHV8i1+lOROuJ7MjNap4eaSD3UijHzQ==", + "version": "2.88.1-postman.40", + "resolved": "https://registry.npmjs.org/postman-request/-/postman-request-2.88.1-postman.40.tgz", + "integrity": "sha512-uE4AiIqhjtHKp4pj9ei7fkdfNXEX9IqDBlK1plGAQne6y79UUlrTdtYLhwXoO0AMOvqyl9Ar+BU6Eo6P/MPgfg==", "dependencies": { "@postman/form-data": "~3.1.1", "@postman/tough-cookie": "~4.1.3-postman.1", @@ -8203,14 +8000,6 @@ "node": ">= 16" } }, - "node_modules/postman-request/node_modules/uuid": { - "version": "8.3.2", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", - "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", - "bin": { - "uuid": "dist/bin/uuid" - } - }, "node_modules/prelude-ls": { "version": "1.2.1", "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.2.1.tgz", @@ -8224,6 +8013,7 @@ "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", @@ -8237,6 +8027,7 @@ "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": ">=10" }, @@ -8267,6 +8058,7 @@ "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" @@ -8312,8 +8104,7 @@ "node_modules/proxy-from-env": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/proxy-from-env/-/proxy-from-env-1.1.0.tgz", - "integrity": "sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg==", - "license": "MIT" + "integrity": "sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg==" }, "node_modules/psl": { "version": "1.9.0", @@ -8341,6 +8132,7 @@ "version": "6.1.0", "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.1.0.tgz", "integrity": "sha512-bVWawvoZoBYpp6yIoQtQXHZjmz35RSVHnUOTefl8Vcjr8snTPY1wnpSPMWekcFwbxI6gtmT7rSYPFvz71ldiOA==", + "dev": true, "funding": [ { "type": "individual", @@ -8414,7 +8206,8 @@ "node_modules/react-is": { "version": "18.3.1", "resolved": "https://registry.npmjs.org/react-is/-/react-is-18.3.1.tgz", - "integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==" + "integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==", + "dev": true }, "node_modules/readable-stream": { "version": "3.6.2", @@ -8475,6 +8268,7 @@ "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", @@ -8491,6 +8285,7 @@ "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" }, @@ -8502,6 +8297,7 @@ "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" } @@ -8519,6 +8315,7 @@ "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" } @@ -8654,6 +8451,7 @@ "version": "2.0.0", "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", + "dev": true, "dependencies": { "shebang-regex": "^3.0.0" }, @@ -8665,6 +8463,7 @@ "version": "3.0.0", "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", + "dev": true, "engines": { "node": ">=8" } @@ -8677,12 +8476,14 @@ "node_modules/sisteransi": { "version": "1.0.5", "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", - "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==" + "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" } @@ -8736,10 +8537,23 @@ "node": ">= 10" } }, + "node_modules/socks-proxy-agent/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==", + "dev": true, + "dependencies": { + "debug": "4" + }, + "engines": { + "node": ">= 6.0.0" + } + }, "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" } @@ -8748,6 +8562,7 @@ "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" @@ -8804,6 +8619,7 @@ "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" }, @@ -8815,6 +8631,7 @@ "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" } @@ -8866,6 +8683,7 @@ "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": { "char-regex": "^1.0.2", "strip-ansi": "^6.0.0" @@ -8902,6 +8720,7 @@ "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, "engines": { "node": ">=8" } @@ -8910,6 +8729,7 @@ "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" } @@ -8918,6 +8738,7 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==", + "dev": true, "engines": { "node": ">=8" }, @@ -8939,6 +8760,7 @@ "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" }, @@ -8950,6 +8772,7 @@ "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" }, @@ -8981,30 +8804,74 @@ "node": ">=8" } }, - "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/teeny-request": { + "version": "9.0.0", + "resolved": "https://registry.npmjs.org/teeny-request/-/teeny-request-9.0.0.tgz", + "integrity": "sha512-resvxdc6Mgb7YEThw6G6bExlXKkv6+YbuzGg9xuXxSgxJF7Ozs+o8Y9+2R3sArdWdW8nOokoQb1yrpFB0pQK2g==", + "dependencies": { + "http-proxy-agent": "^5.0.0", + "https-proxy-agent": "^5.0.0", + "node-fetch": "^2.6.9", + "stream-events": "^1.0.5", + "uuid": "^9.0.0" + }, + "engines": { + "node": ">=14" + } + }, + "node_modules/teeny-request/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/teeny-request/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==", + "dependencies": { + "@tootallnate/once": "2", + "agent-base": "6", + "debug": "4" + }, + "engines": { + "node": ">= 6" + } }, - "node_modules/teeny-request": { - "version": "9.0.0", - "resolved": "https://registry.npmjs.org/teeny-request/-/teeny-request-9.0.0.tgz", - "integrity": "sha512-resvxdc6Mgb7YEThw6G6bExlXKkv6+YbuzGg9xuXxSgxJF7Ozs+o8Y9+2R3sArdWdW8nOokoQb1yrpFB0pQK2g==", + "node_modules/teeny-request/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": { - "http-proxy-agent": "^5.0.0", - "https-proxy-agent": "^5.0.0", - "node-fetch": "^2.6.9", - "stream-events": "^1.0.5", - "uuid": "^9.0.0" + "agent-base": "6", + "debug": "4" }, "engines": { - "node": ">=14" + "node": ">= 6" + } + }, + "node_modules/teeny-request/node_modules/uuid": { + "version": "9.0.1", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", + "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "bin": { + "uuid": "dist/bin/uuid" } }, "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": { "@istanbuljs/schema": "^0.1.2", "glob": "^7.1.4", @@ -9023,12 +8890,14 @@ "node_modules/tmpl": { "version": "1.0.5", "resolved": "https://registry.npmjs.org/tmpl/-/tmpl-1.0.5.tgz", - "integrity": "sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw==" + "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" } @@ -9037,6 +8906,7 @@ "version": "5.0.1", "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", + "dev": true, "dependencies": { "is-number": "^7.0.0" }, @@ -9070,20 +8940,20 @@ } }, "node_modules/ts-jest": { - "version": "29.2.4", - "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.4.tgz", - "integrity": "sha512-3d6tgDyhCI29HlpwIq87sNuI+3Q6GLTTCeYRHCs7vDz+/3GCMwEtV9jezLyl4ZtnBgx00I7hm8PCP8cTksMGrw==", - "license": "MIT", + "version": "29.2.5", + "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.5.tgz", + "integrity": "sha512-KD8zB2aAZrcKIdGk4OwpJggeLcH1FgrICqDSROWqlnJXGCXK4Mn6FcdK2B6670Xr73lHMG1kHw8R87A0ecZ+vA==", + "dev": true, "dependencies": { - "bs-logger": "0.x", + "bs-logger": "^0.2.6", "ejs": "^3.1.10", - "fast-json-stable-stringify": "2.x", + "fast-json-stable-stringify": "^2.1.0", "jest-util": "^29.0.0", "json5": "^2.2.3", - "lodash.memoize": "4.x", - "make-error": "1.x", - "semver": "^7.5.3", - "yargs-parser": "^21.0.1" + "lodash.memoize": "^4.1.2", + "make-error": "^1.3.6", + "semver": "^7.6.3", + "yargs-parser": "^21.1.1" }, "bin": { "ts-jest": "cli.js" @@ -9121,16 +8991,15 @@ "version": "21.1.1", "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-21.1.1.tgz", "integrity": "sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw==", - "license": "ISC", + "dev": true, "engines": { "node": ">=12" } }, "node_modules/tslib": { - "version": "2.6.3", - "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.6.3.tgz", - "integrity": "sha512-xNvxJEOUiWPGhUuUdQgAJPKOOJfGnIyKySOc09XkKsgdUV/3E2zvwZYdejjmRgPCgcym1juLH3226yA7sEFJKQ==", - "license": "0BSD" + "version": "2.7.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.7.0.tgz", + "integrity": "sha512-gLXCKdN1/j47AiHiOkJN69hJmcbGTHI0ImLmbYLHykhgeN0jVGola9yVjFgzCUklsZQMW55o+dW7IXv3RCXDzA==" }, "node_modules/tv4": { "version": "1.3.0", @@ -9161,6 +9030,7 @@ "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" } @@ -9178,9 +9048,10 @@ } }, "node_modules/typescript": { - "version": "5.5.4", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.5.4.tgz", - "integrity": "sha512-Mtq29sKDAEYP7aljRgtPOpTvOfbwRWlS6dPRzwjdE+C0R4brX/GUyhHSecbHMFLNBLcJIPt9nl9yG5TZ1weH+Q==", + "version": "5.6.2", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.6.2.tgz", + "integrity": "sha512-NW8ByodCSNCwZeghjN3o+JX5OFH0Ojg6sadjEKY4huZ52TqbJTJnDo5+Tw98lSy63NZvi4n+ez5m2u5d4PkZyw==", + "dev": true, "bin": { "tsc": "bin/tsc", "tsserver": "bin/tsserver" @@ -9190,157 +9061,14 @@ } }, "node_modules/typescript-eslint": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/typescript-eslint/-/typescript-eslint-8.2.0.tgz", - "integrity": "sha512-DmnqaPcML0xYwUzgNbM1XaKXpEb7BShYf2P1tkUmmcl8hyeG7Pj08Er7R9bNy6AufabywzJcOybQAtnD/c9DGw==", - "dev": true, - "dependencies": { - "@typescript-eslint/eslint-plugin": "8.2.0", - "@typescript-eslint/parser": "8.2.0", - "@typescript-eslint/utils": "8.2.0" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/eslint-plugin": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.2.0.tgz", - "integrity": "sha512-02tJIs655em7fvt9gps/+4k4OsKULYGtLBPJfOsmOq1+3cdClYiF0+d6mHu6qDnTcg88wJBkcPLpQhq7FyDz0A==", - "dev": true, - "dependencies": { - "@eslint-community/regexpp": "^4.10.0", - "@typescript-eslint/scope-manager": "8.2.0", - "@typescript-eslint/type-utils": "8.2.0", - "@typescript-eslint/utils": "8.2.0", - "@typescript-eslint/visitor-keys": "8.2.0", - "graphemer": "^1.4.0", - "ignore": "^5.3.1", - "natural-compare": "^1.4.0", - "ts-api-utils": "^1.3.0" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "@typescript-eslint/parser": "^8.0.0 || ^8.0.0-alpha.0", - "eslint": "^8.57.0 || ^9.0.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/parser": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.2.0.tgz", - "integrity": "sha512-j3Di+o0lHgPrb7FxL3fdEy6LJ/j2NE8u+AP/5cQ9SKb+JLH6V6UHDqJ+e0hXBkHP1wn1YDFjYCS9LBQsZDlDEg==", - "dev": true, - "dependencies": { - "@typescript-eslint/scope-manager": "8.2.0", - "@typescript-eslint/types": "8.2.0", - "@typescript-eslint/typescript-estree": "8.2.0", - "@typescript-eslint/visitor-keys": "8.2.0", - "debug": "^4.3.4" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^8.57.0 || ^9.0.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/scope-manager": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.2.0.tgz", - "integrity": "sha512-OFn80B38yD6WwpoHU2Tz/fTz7CgFqInllBoC3WP+/jLbTb4gGPTy9HBSTsbDWkMdN55XlVU0mMDYAtgvlUspGw==", - "dev": true, - "dependencies": { - "@typescript-eslint/types": "8.2.0", - "@typescript-eslint/visitor-keys": "8.2.0" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/type-utils": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.2.0.tgz", - "integrity": "sha512-g1CfXGFMQdT5S+0PSO0fvGXUaiSkl73U1n9LTK5aRAFnPlJ8dLKkXr4AaLFvPedW8lVDoMgLLE3JN98ZZfsj0w==", - "dev": true, - "dependencies": { - "@typescript-eslint/typescript-estree": "8.2.0", - "@typescript-eslint/utils": "8.2.0", - "debug": "^4.3.4", - "ts-api-utils": "^1.3.0" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/types": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.2.0.tgz", - "integrity": "sha512-6a9QSK396YqmiBKPkJtxsgZZZVjYQ6wQ/TlI0C65z7vInaETuC6HAHD98AGLC8DyIPqHytvNuS8bBVvNLKyqvQ==", - "dev": true, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/typescript-estree": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.2.0.tgz", - "integrity": "sha512-kiG4EDUT4dImplOsbh47B1QnNmXSoUqOjWDvCJw/o8LgfD0yr7k2uy54D5Wm0j4t71Ge1NkynGhpWdS0dEIAUA==", + "version": "8.5.0", + "resolved": "https://registry.npmjs.org/typescript-eslint/-/typescript-eslint-8.5.0.tgz", + "integrity": "sha512-uD+XxEoSIvqtm4KE97etm32Tn5MfaZWgWfMMREStLxR6JzvHkc2Tkj7zhTEK5XmtpTmKHNnG8Sot6qDfhHtR1Q==", "dev": true, "dependencies": { - "@typescript-eslint/types": "8.2.0", - "@typescript-eslint/visitor-keys": "8.2.0", - "debug": "^4.3.4", - "globby": "^11.1.0", - "is-glob": "^4.0.3", - "minimatch": "^9.0.4", - "semver": "^7.6.0", - "ts-api-utils": "^1.3.0" + "@typescript-eslint/eslint-plugin": "8.5.0", + "@typescript-eslint/parser": "8.5.0", + "@typescript-eslint/utils": "8.5.0" }, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -9355,69 +9083,6 @@ } } }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/utils": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.2.0.tgz", - "integrity": "sha512-O46eaYKDlV3TvAVDNcoDzd5N550ckSe8G4phko++OCSC1dYIb9LTc3HDGYdWqWIAT5qDUKphO6sd9RrpIJJPfg==", - "dev": true, - "dependencies": { - "@eslint-community/eslint-utils": "^4.4.0", - "@typescript-eslint/scope-manager": "8.2.0", - "@typescript-eslint/types": "8.2.0", - "@typescript-eslint/typescript-estree": "8.2.0" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^8.57.0 || ^9.0.0" - } - }, - "node_modules/typescript-eslint/node_modules/@typescript-eslint/visitor-keys": { - "version": "8.2.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.2.0.tgz", - "integrity": "sha512-sbgsPMW9yLvS7IhCi8IpuK1oBmtbWUNP+hBdwl/I9nzqVsszGnNGti5r9dUtF5RLivHUFFIdRvLiTsPhzSyJ3Q==", - "dev": true, - "dependencies": { - "@typescript-eslint/types": "8.2.0", - "eslint-visitor-keys": "^3.4.3" - }, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - } - }, - "node_modules/typescript-eslint/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/typescript-eslint/node_modules/minimatch": { - "version": "9.0.5", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", - "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", - "dev": true, - "dependencies": { - "brace-expansion": "^2.0.1" - }, - "engines": { - "node": ">=16 || 14 >=14.17" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" - } - }, "node_modules/uc.micro": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/uc.micro/-/uc.micro-2.1.0.tgz", @@ -9471,6 +9136,7 @@ "version": "1.1.0", "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.1.0.tgz", "integrity": "sha512-EdRAaAyk2cUE1wOf2DkEhzxqOQvFOoRJFNS6NeyJ01Gp2beMRpBAINjM2iDXE3KCuKhwnvHIQCJm6ThL2Z+HzQ==", + "dev": true, "funding": [ { "type": "opencollective", @@ -9519,13 +9185,9 @@ "integrity": "sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==" }, "node_modules/uuid": { - "version": "9.0.1", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", - "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", - "funding": [ - "https://github.com/sponsors/broofa", - "https://github.com/sponsors/ctavan" - ], + "version": "8.3.2", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", + "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", "bin": { "uuid": "dist/bin/uuid" } @@ -9534,6 +9196,7 @@ "version": "9.3.0", "resolved": "https://registry.npmjs.org/v8-to-istanbul/-/v8-to-istanbul-9.3.0.tgz", "integrity": "sha512-kiGUalWN+rgBJ/1OHZsBtU4rXZOfj/7rKQxULKlIzwzQSvMJUUNgPwJEEh7gU6xEVxC0ahoOBvN2YI8GH6FNgA==", + "dev": true, "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", "@types/istanbul-lib-coverage": "^2.0.1", @@ -9568,6 +9231,7 @@ "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" } @@ -9590,6 +9254,7 @@ "version": "2.0.2", "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dev": true, "dependencies": { "isexe": "^2.0.0" }, @@ -9648,6 +9313,7 @@ "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": { "imurmurhash": "^0.1.4", "signal-exit": "^3.0.7" @@ -9671,10 +9337,9 @@ } }, "node_modules/yallist": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/yallist/-/yallist-3.1.1.tgz", - "integrity": "sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g==", - "license": "ISC" + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" }, "node_modules/yargs": { "version": "17.7.2", @@ -9741,12 +9406,56 @@ "version": "0.1.0", "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==", + "dev": true, "engines": { "node": ">=10" }, "funding": { "url": "https://github.com/sponsors/sindresorhus" } + }, + "schemaregistry": { + "name": "@confluentinc/schemaregistry", + "version": "v0.1.16-devel", + "license": "MIT", + "dependencies": { + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", + "@bufbuild/protobuf": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", + "@smithy/types": "^3.3.0", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", + "async-mutex": "^0.5.0", + "avsc": "^5.7.7", + "axios": "^1.7.3", + "json-stringify-deterministic": "^1.0.12", + "jsonata": "^2.0.5", + "lru-cache": "^11.0.0", + "node-vault": "^0.10.2", + "validator": "^13.12.0" + }, + "devDependencies": { + "@bufbuild/buf": "^1.37.0", + "@bufbuild/protoc-gen-es": "^2.0.0", + "@eslint/js": "^9.9.0", + "@types/eslint__js": "^8.42.3", + "@types/node": "^20.16.1", + "bluebird": "^3.5.3", + "eslint": "^8.57.0", + "eslint-plugin-jest": "^28.6.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", + "mocha": "^10.7.0", + "node-gyp": "^9.3.1", + "ts-jest": "^29.2.4", + "typescript": "^5.5.4", + "typescript-eslint": "^8.2.0" + } } } } diff --git a/package.json b/package.json index b133aa13..9916f632 100644 --- a/package.json +++ b/package.json @@ -12,8 +12,7 @@ "test": "make test", "install": "node-pre-gyp install --fallback-to-build", "prepack": "node ./ci/prepublish.js", - "test:types": "tsc -p .", - "test:schemaregistry": "make -f Makefile.schemaregistry test" + "test:types": "tsc -p ." }, "binary": { "module_name": "confluent-kafka-javascript", @@ -48,31 +47,14 @@ "typescript-eslint": "^8.2.0" }, "dependencies": { - "@aws-sdk/client-kms": "^3.637.0", - "@azure/identity": "^4.4.1", - "@azure/keyvault-keys": "^4.8.0", - "@bufbuild/protobuf": "^2.0.0", - "@criteria/json-schema": "^0.10.0", - "@criteria/json-schema-validation": "^0.10.0", - "@google-cloud/kms": "^4.5.0", - "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", - "@smithy/types": "^3.3.0", - "@types/validator": "^13.12.0", - "ajv": "^8.17.1", - "async-mutex": "^0.5.0", - "avsc": "^5.7.7", - "axios": "^1.7.3", "bindings": "^1.3.1", - "json-stringify-deterministic": "^1.0.12", - "jsonata": "^2.0.5", - "lru-cache": "^11.0.0", - "nan": "^2.17.0", - "node-vault": "^0.10.2", - "ts-jest": "^29.2.4", - "validator": "^13.12.0" + "nan": "^2.17.0" }, "engines": { "node": ">=18.0.0" - } + }, + "workspaces": [ + "schemaregistry" + ] } diff --git a/Makefile.schemaregistry b/schemaregistry/Makefile.schemaregistry similarity index 50% rename from Makefile.schemaregistry rename to schemaregistry/Makefile.schemaregistry index 37f002bc..aebf1d1e 100644 --- a/Makefile.schemaregistry +++ b/schemaregistry/Makefile.schemaregistry @@ -2,15 +2,15 @@ # Variables NODE ?= node -ESLINT ?= ./node_modules/.bin/eslint -JEST ?= ./node_modules/.bin/jest -TS_NODE ?= ./node_modules/.bin/ts-node +ESLINT ?= ../node_modules/.bin/eslint +JEST ?= ../node_modules/.bin/jest +TS_NODE ?= ../node_modules/.bin/ts-node # Paths -SRC_DIR = schemaregistry -SR_TEST_DIR = test/schemaregistry -DEK_TEST_DIR = test/schemaregistry/rules/encryption/dekregistry -INTEG_DIR = e2e/schemaregistry +SRC_DIR = . +SR_TEST_DIR = ../test/schemaregistry +DEK_TEST_DIR = ../test/schemaregistry/rules/encryption/dekregistry +INTEG_DIR = ../e2e/schemaregistry # Tasks .PHONY: all lint test integtest diff --git a/schemaregistry/jest.config.js b/schemaregistry/jest.config.js new file mode 100644 index 00000000..c4caa2fb --- /dev/null +++ b/schemaregistry/jest.config.js @@ -0,0 +1,6 @@ +module.exports = { + roots: [".."], + transform: { + '^.+\\.tsx?$': 'ts-jest', + }, + }; diff --git a/schemaregistry/package.json b/schemaregistry/package.json new file mode 100644 index 00000000..0938e555 --- /dev/null +++ b/schemaregistry/package.json @@ -0,0 +1,59 @@ +{ + "name": "@confluentinc/schemaregistry", + "version": "v0.1.16-devel", + "description": "Node.js client for Confluent Schema Registry", + "main": "lib/index.js", + "devDependencies": { + "@bufbuild/buf": "^1.37.0", + "@bufbuild/protoc-gen-es": "^2.0.0", + "@eslint/js": "^9.9.0", + "@types/eslint__js": "^8.42.3", + "@types/node": "^20.16.1", + "bluebird": "^3.5.3", + "eslint": "^8.57.0", + "eslint-plugin-jest": "^28.6.0", + "jest": "^29.7.0", + "jsdoc": "^4.0.2", + "mocha": "^10.7.0", + "node-gyp": "^9.3.1", + "ts-jest": "^29.2.4", + "typescript": "^5.5.4", + "typescript-eslint": "^8.2.0" + }, + "dependencies": { + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", + "@bufbuild/protobuf": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", + "@smithy/types": "^3.3.0", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", + "async-mutex": "^0.5.0", + "avsc": "^5.7.7", + "axios": "^1.7.3", + "json-stringify-deterministic": "^1.0.12", + "jsonata": "^2.0.5", + "lru-cache": "^11.0.0", + "node-vault": "^0.10.2", + "validator": "^13.12.0" + }, + "scripts": { + "test:types": "tsc -p .", + "test:schemaregistry": "make -f Makefile.schemaregistry test" + }, + + "keywords": [ + "schemaregistry", + "confluent" + ], + "repository": { + "type": "git", + "url": "git@github.com:confluentinc/confluent-kafka-javascript.git" + }, + + "license": "MIT" +} diff --git a/schemaregistry/tsconfig.json b/schemaregistry/tsconfig.json new file mode 100644 index 00000000..1a256bdd --- /dev/null +++ b/schemaregistry/tsconfig.json @@ -0,0 +1,31 @@ +{ + "compilerOptions": { + "baseUrl": ".", + "lib": [ + "es2021", "dom" + ], + "module": "preserve", + "target": "es2021", + "strict": true, + "esModuleInterop": true, + "forceConsistentCasingInFileNames": true, + "moduleResolution": "bundler", + "allowUnusedLabels": false, + "allowUnreachableCode": false, + "noFallthroughCasesInSwitch": true, + "noImplicitOverride": true, + "noImplicitReturns": true, + "noPropertyAccessFromIndexSignature": true, + "noUnusedLocals": true, + "useUnknownInCatchVariables": true, + "types": ["../node_modules/@types/node"], + "typeRoots": ["."], + "noEmit": true, + "resolveJsonModule": true, + "skipLibCheck": true + }, + "include": [ + "**/*", + "../test/**/*" + ] +} diff --git a/tsconfig.json b/tsconfig.json index b3650ce2..c9e6c6fe 100644 --- a/tsconfig.json +++ b/tsconfig.json @@ -1,32 +1,18 @@ { "compilerOptions": { + "module": "commonjs", + "lib": ["es6"], + "noImplicitAny": true, + "noImplicitThis": true, + "strictNullChecks": true, "baseUrl": ".", - "lib": [ - "es2021", "dom" - ], - "module": "preserve", - "target": "es2021", - "strict": true, - "esModuleInterop": true, - "forceConsistentCasingInFileNames": true, - "moduleResolution": "bundler", - "allowUnusedLabels": false, - "allowUnreachableCode": false, - "noFallthroughCasesInSwitch": true, - "noImplicitOverride": true, - "noImplicitReturns": true, - "noPropertyAccessFromIndexSignature": true, - "noUnusedLocals": true, - "useUnknownInCatchVariables": true, - "types": ["node_modules/@types/node"], + "types": ["./node_modules/@types/node"], "typeRoots": ["."], "noEmit": true, - "resolveJsonModule": true, - "skipLibCheck": true + "forceConsistentCasingInFileNames": true, + "strictFunctionTypes": true }, - "include": [ - "index.d.ts", - "schemaregistry/**/*", - "test/**/*" + "files": [ + "index.d.ts" ] } From f5e7f3e569e8257582a9e5b947a29b6fd0c54af4 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 13 Sep 2024 12:03:59 +0200 Subject: [PATCH 190/224] Refactor to always use a barrier for pending operation (#26) Readers writer lock with reentrant calls Refactor to always use a barrier for pending operation to avoid problems or regressions with async pause or resume calls that will be introduced later. Uses linked lists for the cache. Removes heap-js code. Test name improvement to run single parametric tests Seek signature and add partitionsConsumedConcurrently to ConsumerRunConfig type definition Make final max poll interval double of configured value so even last message processed before cache cleanup can take that time to process. Fix to restart max poll interval timer on fetch. Marking batch stale after cache clear was requested and max poll interval is reached before it's cleared. Add assignmentLost function to the rebalance callback. Fix to nextN size, version with max.poll.interval.ms applied to each message or batch (only for messages after cache reset) Performance test, removing outliers Start performance timer from first message received after resuming --- LICENSE.heap-js | 36 - LICENSE.kafkajs | 2 +- eslint.config.js | 4 +- .../performance-primitives-kafkajs.js | 6 +- .../performance/performance-primitives.js | 6 +- lib/kafka-consumer.js | 13 + lib/kafkajs/_common.js | 252 +- lib/kafkajs/_consumer.js | 923 ++++--- lib/kafkajs/_consumer_cache.js | 519 +--- lib/kafkajs/_heap.js | 2353 ----------------- lib/kafkajs/_linked-list.js | 219 ++ src/kafka-consumer.cc | 17 + src/kafka-consumer.h | 2 + test/promisified/consumer/commit.spec.js | 2 +- .../consumer/consumeMessages.spec.js | 255 +- .../consumer/consumerCacheTests.spec.js | 6 +- test/promisified/consumer/pause.spec.js | 1 + test/promisified/testhelpers.js | 26 + test/promisified/unit/cache.spec.js | 588 ++-- test/promisified/unit/common.spec.js | 227 ++ types/config.d.ts | 6 +- types/kafkajs.d.ts | 3 +- 22 files changed, 1915 insertions(+), 3551 deletions(-) delete mode 100644 LICENSE.heap-js delete mode 100644 lib/kafkajs/_heap.js create mode 100644 lib/kafkajs/_linked-list.js create mode 100644 test/promisified/unit/common.spec.js diff --git a/LICENSE.heap-js b/LICENSE.heap-js deleted file mode 100644 index a2a9eba1..00000000 --- a/LICENSE.heap-js +++ /dev/null @@ -1,36 +0,0 @@ -Code from https://github.com/ignlg/heap-js/ | Commit Hash: a3bb403 | dist/heap-js.es5.js - -Code used in lib/kafkajs/_heap.js - ----- - - -BSD 3-Clause License - -Copyright (c) 2017, Ignacio Lago -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -* Neither the name of the copyright holder nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE -FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER -CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, -OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/LICENSE.kafkajs b/LICENSE.kafkajs index d303b06e..3a91a6dc 100644 --- a/LICENSE.kafkajs +++ b/LICENSE.kafkajs @@ -1,6 +1,6 @@ The promisified API (lib/kafkajs) is inspired by kafkajs (github.com/tulios/kafkajs). The promisified tests (test/promisified) are also adapted from there. -An async lock implementation and many error types are also adapted from there. +Many error types are also adapted from there. The license notice is reproduced below. ---- diff --git a/eslint.config.js b/eslint.config.js index 4f83933a..0dd90f8f 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -37,12 +37,12 @@ module.exports = ts.config( { ...js.configs.recommended, files: ["lib/**/*.js", "test/promisified/**/*.js"], - ignores: ["lib/kafkajs/_heap.js"] + ignores: [] }, { ...ckjsSpecificSettings, files: ["lib/**/*.js", "test/promisified/**/*.js"], - ignores: ["lib/kafkajs/_heap.js"] + ignores: [] }, { ...jest.configs['flat/recommended'], diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 1dba3588..76e44da9 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -128,6 +128,8 @@ async function runConsumer(brokers, topic, totalMessageCnt) { totalMessageSize += message.value.length; if (messagesReceived === 1) { consumer.pause([{ topic }]); + } else if (messagesReceived === 2) { + startTime = hrtime(); } else if (messagesReceived === totalMessageCnt) { let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; @@ -153,7 +155,6 @@ async function runConsumer(brokers, topic, totalMessageCnt) { console.log("Starting consumer.") totalMessageSize = 0; - startTime = hrtime(); consumer.resume([{ topic }]); await new Promise((resolve) => { let interval = setInterval(() => { @@ -203,6 +204,8 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t totalMessageSize += message.value.length; if (messagesReceived === 1) { consumer.pause([{ topic }]); + } else if (messagesReceived === 2) { + startTime = hrtime(); } else if (messagesReceived === totalMessageCnt) { let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; @@ -228,7 +231,6 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t console.log("Starting consume-transform-produce.") totalMessageSize = 0; - startTime = hrtime(); consumer.resume([{ topic: consumeTopic }]); await new Promise((resolve) => { let interval = setInterval(() => { diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 4de54ac4..58ee48e1 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -134,6 +134,8 @@ async function runConsumer(brokers, topic, totalMessageCnt) { totalMessageSize += message.value.length; if (messagesReceived === 1) { consumer.pause([{ topic }]); + } else if (messagesReceived === 2) { + startTime = hrtime(); } else if (messagesReceived === totalMessageCnt) { let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; @@ -159,7 +161,6 @@ async function runConsumer(brokers, topic, totalMessageCnt) { console.log("Starting consumer.") totalMessageSize = 0; - startTime = hrtime(); consumer.resume([{ topic }]); await new Promise((resolve) => { let interval = setInterval(() => { @@ -222,6 +223,8 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t totalMessageSize += message.value.length; if (messagesReceived === 1) { consumer.pause([{ topic }]); + } else if (messagesReceived === 2) { + startTime = hrtime(); } else if (messagesReceived === totalMessageCnt) { let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; @@ -247,7 +250,6 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t console.log("Starting consume-transform-produce.") totalMessageSize = 0; - startTime = hrtime(); consumer.resume([{ topic: consumeTopic }]); await new Promise((resolve) => { let interval = setInterval(() => { diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index 03b1f20a..e6e66a94 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -340,6 +340,19 @@ KafkaConsumer.prototype.assignments = function() { return this._errorWrap(this._client.assignments(), true); }; +/** + * Is current assignment in rebalance callback lost? + * + * @note This method should only be called from within the rebalance callback + * when partitions are revoked. + * + * @return {boolean} true if assignment was lost + */ + +KafkaConsumer.prototype.assignmentLost = function() { + return this._client.assignmentLost(); +}; + /** * Get the type of rebalance protocol used in the consumer group. * diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 39cfcc39..256566b9 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,5 +1,6 @@ const error = require("./_error"); const process = require("process"); +const { AsyncLocalStorage } = require('node:async_hooks'); /* A list of kafkaJS compatible properties that we process. * All of these are not necessarily supported, and an error will be @@ -627,81 +628,203 @@ function notImplemented(msg = 'Not implemented') { throw new error.KafkaJSError(msg, { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } -/* Code from the async lock is from github.com/tulios/kafkajs. - * For more details, see LICENSE.kafkajs */ -const LockStates = Object.freeze({ - LOCKED: 'locked', - TIMEOUT: 'timeout', - WAITING: 'waiting', - TIMEOUT_ERROR_MESSAGE: 'timeoutErrorMessage', -}); +/** + * A promise that can be resolved externally. + */ +class DeferredPromise extends Promise{ + #resolved = false; + + /** + * JS expects a resolver function to be passed to classes extending Promise. + * that takes the same parameter a normal Promise constructor does. + * The DeferredPromise cannot be rejected to avoid unhandled rejections + * entirely. + * @param {(resolve: (value: any) => void, reject: (error: Error) => void) => void} resolver + */ + constructor(resolver) { + let resolveF; + super((resolve) => { + resolveF = resolve; + }); + this.resolve = (...args) => { + this.#resolved = true; + resolveF(...args); + }; + if (resolver) + resolver(this.resolve, () => {}); + } + + get resolved() { + return this.#resolved; + } +} +/** + * Utility class for time related functions + */ +class Timer { + /** + * Function that resolves when the given timeout is reached + * or the passed promise resolves, when it's passed, clearing the timeout + * in any case. + * + * @param {number} timeoutMs The timeout in milliseconds. + * @param {Promise|undefined} promise The promise to wait for, + * alternatively to the timeout, or `undefined` to just wait for the timeout. + */ + static async withTimeout(timeoutMs, promise) { + const timer = new DeferredPromise(); + const registration = setTimeout(timer.resolve, timeoutMs); + if (!promise) + await timer; + else { + await Promise.race([ + promise, + timer + ]); + } + if (!timer.resolved) { + timer.resolve(); + } + clearTimeout(registration); + } +} + +/** + * Readers-writer lock with reentrant calls. + * Upgrading from a read to a write lock is supported. + * Acquiring a read lock while holding a write lock is a no-op. + */ class Lock { - constructor({ timeout, description = null } = {}) { - if (typeof timeout !== 'number') { - throw new TypeError(`'timeout' is not a number, received '${typeof timeout}'`); + // Total number of readers, not increases when already holding a write lock + #readers = 0; + + // Total number of writers, increased only by a single write and + // its reentrant calls + #writers = 0; + + #asyncLocalStorage = new AsyncLocalStorage(); + + // Promise to resolve and recreate when there are no readers or writers + // This is used to notify all waiting writers so at least one can proceed. + // It's also used to notify all waiting readers so they can can check + // the writer has finished. + #zeroReadersAndWritersPromise = new DeferredPromise(); + + #notifyZeroReadersAndWriters() { + if (this.#readers === 0 && this.#writers === 0) { + this.#zeroReadersAndWritersPromise.resolve(); + this.#zeroReadersAndWritersPromise = new DeferredPromise(); } + } - this[LockStates.LOCKED] = false; - this[LockStates.TIMEOUT] = timeout; - this[LockStates.WAITING] = new Set(); - this[LockStates.TIMEOUT_ERROR_MESSAGE] = () => { - const timeoutMessage = `Timeout while acquiring lock (${this[LockStates.WAITING].size} waiting locks)`; - return description ? `${timeoutMessage}: "${description}"` : timeoutMessage; + #createAsyncLocalStorageStore() { + return { + // All reentrant calls + stack: [], + // Number of write locks in reentrant calls + writers: 0, + // Number of read locks in reentrant calls + readers: 0, }; } - async acquire() { - return new Promise((resolve, reject) => { - if (!this[LockStates.LOCKED]) { - this[LockStates.LOCKED] = true; - return resolve(); - } - - let timeoutId = null; - const tryToAcquire = async () => { - if (!this[LockStates.LOCKED]) { - this[LockStates.LOCKED] = true; - clearTimeout(timeoutId); - this[LockStates.WAITING].delete(tryToAcquire); - return resolve(); - } - }; - - this[LockStates.WAITING].add(tryToAcquire); - timeoutId = setTimeout(() => { - // The message should contain the number of waiters _including_ this one - const e = new error.KafkaJSLockTimeout(this[LockStates.TIMEOUT_ERROR_MESSAGE]()); - this[LockStates.WAITING].delete(tryToAcquire); - reject(e); - }, this[LockStates.TIMEOUT]); - }); + async #runAsyncStack(type, f) { + let store = this.#asyncLocalStorage.getStore(); + if (store) { + let promise = f(); + store.stack.push(promise); + await promise; + } else { + await this.#asyncLocalStorage.run(this.#createAsyncLocalStorageStore(type), + async () => { + store = this.#asyncLocalStorage.getStore(); + let promise = f(); + store.stack.push(promise); + // Await all promises are settled + await Promise.allSettled(store.stack); + // Reject if any promise is rejected + await Promise.all(store.stack); + }); + } } - async release() { - this[LockStates.LOCKED] = false; - const waitingLock = this[LockStates.WAITING].values().next().value; - - if (waitingLock) { - return waitingLock(); + async #acquireRead() { + let store = this.#asyncLocalStorage.getStore(); + if (!store.writers) { + while (this.#writers > 0) { + await this.#zeroReadersAndWritersPromise; + } + this.#readers++; + store.readers++; } } -} -/** - * Acquires a lock, or logs an error if it fails. - * @param {Lock} lock - * @param {import("../../types/kafkajs").Logger} logger - * @returns {boolean} true if the lock was acquired, false otherwise. - */ -async function acquireOrLog(lock, logger) { - try { - await lock.acquire(); - return true; - } catch (e) { - logger.error(`Failed to acquire lock: ${e.message}`); - } - return false; + async #acquireWrite() { + let store = this.#asyncLocalStorage.getStore(); + // We remove current stack readers and writers so it + // becomes reentrant + let readers = this.#readers - store.readers; + let writers = this.#writers - store.writers; + while (readers > 0 || writers > 0) { + await this.#zeroReadersAndWritersPromise; + writers = this.#writers - store.writers; + readers = this.#readers - store.readers; + } + this.#writers++; + store.writers++; + } + + async #releaseRead() { + let store = this.#asyncLocalStorage.getStore(); + this.#readers--; + store.readers--; + this.#notifyZeroReadersAndWriters(); + } + + async #releaseWrite() { + let store = this.#asyncLocalStorage.getStore(); + this.#writers--; + store.writers--; + this.#notifyZeroReadersAndWriters(); + } + + /** + * Acquire a write (exclusive) lock while executing + * the given task. + * @param {function} task The task to execute. + * @returns {Promise} The result of the task. + */ + async write(task) { + let withWriteLock = async () => { + try { + await this.#acquireWrite(); + return await task(); + } finally { + await this.#releaseWrite(); + } + }; + await this.#runAsyncStack(1, withWriteLock); + } + + + /** + * Acquire a read (shared) lock while executing + * the given task. + * @param {function} task The task to execute. + * @returns {Promise} The result of the task. + */ + async read(task) { + let withReadLock = async () => { + try { + await this.#acquireRead(); + return await task(); + } finally { + await this.#releaseRead(); + } + }; + await this.#runAsyncStack(0, withReadLock); + } } /** @@ -731,6 +854,7 @@ module.exports = { checkAllowedKeys, checkIfKafkaJsKeysPresent, Lock, - acquireOrLog, + DeferredPromise, + Timer, partitionKey, }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index fca14570..51053902 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -16,11 +16,13 @@ const { checkAllowedKeys, logLevel, Lock, - acquireOrLog, partitionKey, + DeferredPromise, + Timer } = require('./_common'); const { Buffer } = require('buffer'); const MessageCache = require('./_consumer_cache'); +const { hrtime } = require('process'); const ConsumerState = Object.freeze({ INIT: 0, @@ -67,12 +69,6 @@ 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 "|". @@ -81,10 +77,10 @@ class Consumer { #pendingSeeks = new Map(); /** - * Stores the list of paused partitions, as a set of JSON.stringify'd TopicPartition objects. - * @type {Set} + * Stores the map of paused partitions keys to TopicPartition objects. + * @type {Map} */ - #pausedPartitions = new Set(); + #pausedPartitions = new Map(); /** * Contains a list of stored topics/regexes that the user has subscribed to. @@ -109,10 +105,9 @@ class Consumer { * A lock for consuming and disconnecting. * This lock should be held whenever we want to change the state from CONNECTED to any state other than CONNECTED. * In practical terms, this lock is held whenever we're consuming a message, or disconnecting. - * We set the timeout to 5 seconds, after which we log an error, but keep trying to acquire the lock. * @type {Lock} */ - #lock = new Lock({ timeout: 5000 }); + #lock = new Lock(); /** * Whether the consumer is running. @@ -126,6 +121,17 @@ class Consumer { */ #messageCache = null; + /** + * The maximum size of the message cache. + * Will be adjusted dynamically. + */ + #messageCacheMaxSize = 1; + + /** + * Number of times we tried to increase the cache. + */ + #increaseCount = 0; + /** * Whether the user has enabled manual offset management (commits). */ @@ -145,7 +151,7 @@ class Consumer { /** * Whether worker termination has been scheduled. */ - #workerTerminationScheduled = false; + #workerTerminationScheduled = new DeferredPromise(); /** * The worker functions currently running in the consumer. @@ -158,9 +164,45 @@ class Consumer { #concurrency = 1; /** - * Whether any call to the internalClient's consume() method is in progress. + * Promise that resolves together with last in progress fetch. + * It's set to null when no fetch is in progress. + */ + #fetchInProgress; + + /** + * Whether any rebalance callback is in progress. + * That can last more than the fetch itself given it's not awaited. + * So we await it after fetch is done. + */ + #rebalanceCbInProgress; + + /** + * Promise that is resolved on fetch to restart max poll interval timer. */ - #fetchInProgress = false; + #maxPollIntervalRestart = new DeferredPromise(); + + /** + * Initial default value for max poll interval. + */ + #maxPollIntervalMs = 300000; + /** + * Maximum interval between poll calls from workers, + * if exceeded, the cache is cleared so a new poll can be made + * before reaching the max poll interval. + * It's set to max poll interval value. + */ + #cacheExpirationTimeoutMs = 300000; + + /** + * Last fetch real time clock in nanoseconds. + */ + #lastFetchClockNs = 0; + + /** + * List of pending operations to be executed after + * all workers reach the end of their current processing. + */ + #pendingOperations = []; /** * Maps topic-partition key to the batch payload for marking staleness. @@ -205,47 +247,50 @@ class Consumer { * * @param {Array<{topic: string, partition: number}>|null} topicPartitions to clear the cache for, if null, then clear all assigned. */ - async #clearCacheAndResetPositions(topicPartitions = null) { + async #clearCacheAndResetPositions() { /* Seek to stored offset for each topic partition. It's possible that we've * consumed messages upto N from the internalClient, but the user has stale'd the cache - * after consuming just k (< N) messages. We seek to k+1. */ - - const clearPartitions = topicPartitions ? topicPartitions : this.assignment(); - const seekPromises = []; - for (const topicPartitionOffset of clearPartitions) { - const key = partitionKey(topicPartitionOffset); + * after consuming just k (< N) messages. We seek back to last consumed offset + 1. */ + this.#messageCache.clear(); + this.#messageCacheMaxSize = 1; + this.#increaseCount = 0; + const clearPartitions = this.assignment(); + const seeks = []; + for (const topicPartition of clearPartitions) { + const key = partitionKey(topicPartition); if (!this.#lastConsumedOffsets.has(key)) continue; - /* Fire off a seek */ - const seekPromise = new Promise((resolve, reject) => { - this.#internalClient.seek({ - topic: topicPartitionOffset.topic, - partition: topicPartitionOffset.partition, - offset: +this.#lastConsumedOffsets.get(key) - }, 10000, err => { - if (err) { - reject(err); - } else { - resolve(); - } - }); - - this.#lastConsumedOffsets.delete(key); - }); - seekPromises.push(seekPromise); + const lastConsumedOffsets = this.#lastConsumedOffsets.get(key); + const topicPartitionOffsets = [ + { + topic: topicPartition.topic, + partition: topicPartition.partition, + offset: lastConsumedOffsets.offset, + leaderEpoch: lastConsumedOffsets.leaderEpoch, + } + ]; + seeks.push(this.#seekInternal(topicPartitionOffsets)); } - /* TODO: we should cry more about this and render the consumer unusable. */ - await Promise.all(seekPromises) - .catch(err => this.#logger.error(`Seek error. This is effectively a fatal error: ${err}`), this.#createConsumerBindingMessageMetadata()); + await Promise.allSettled(seeks); + try { + await Promise.all(seeks); + } catch (err) { + /* TODO: we should cry more about this and render the consumer unusable. */ + this.#logger.error(`Seek error. This is effectively a fatal error: ${err.stack}`); + } + } - /* Clear the cache and stored offsets. - * We need to do this only if topicPartitions = null (global cache expiry). - * This is because in case of a local cache expiry, MessageCache handles - * skipping that (and clearing that later before getting new messages). */ - if (!topicPartitions) { + #unassign(assignment) { + if (this.#internalClient.rebalanceProtocol() === "EAGER") { + this.#internalClient.unassign(); this.#messageCache.clear(); + this.#partitionCount = 0; + } else { + this.#internalClient.incrementalUnassign(assignment); + this.#messageCache.markStale(assignment); + this.#partitionCount -= assignment.length; } } @@ -254,15 +299,20 @@ class Consumer { * @param {Error} err - error in rebalance * @param {import("../../types").TopicPartition[]} assignment */ - #rebalanceCallback(err, assignment) { - err = LibrdKafkaError.create(err); - const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; + async #rebalanceCallback(err, assignment) { + const isLost = this.#internalClient.assignmentLost(); + this.#rebalanceCbInProgress = new DeferredPromise(); + let assignmentFnCalled = false; this.#logger.info( - `Received rebalance event with message: '${err.message}' and ${assignment.length} partition(s)`, + `Received rebalance event with message: '${err.message}' and ${assignment.length} partition(s), isLost: ${isLost}`, this.#createConsumerBindingMessageMetadata()); + /* We allow the user to modify the assignment by returning it. If a truthy + * value is returned, we use that and do not apply any pending seeks to it either. + * The user can alternatively use the assignmentFns argument. + * Precedence is given to the calling of functions within assignmentFns. */ + let assignmentModified = false; - let assignmentFnCalled = false; - function assignmentFn(userAssignment) { + const assignmentFn = (userAssignment) => { if (assignmentFnCalled) return; assignmentFnCalled = true; @@ -274,124 +324,95 @@ class Consumer { this.#internalClient.incrementalAssign(userAssignment); this.#partitionCount += userAssignment.length; } - } + }; - function unassignmentFn(userAssignment) { + const unassignmentFn = (userAssignment) => { if (assignmentFnCalled) return; assignmentFnCalled = true; - if (this.#internalClient.rebalanceProtocol() === "EAGER") { - this.#internalClient.unassign(); - this.#messageCache.removeTopicPartitions(); - this.#partitionCount = 0; - } else { - this.#internalClient.incrementalUnassign(userAssignment); - this.#messageCache.removeTopicPartitions(userAssignment); - this.#partitionCount -= userAssignment.length; - } - } - - let call = Promise.resolve(); + if (this.#disconnectStarted) + this.#unassign(userAssignment); + else + this.#addPendingOperation(() => this.#unassign(userAssignment)); + }; + + try { + err = LibrdKafkaError.create(err); + const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; - /* We allow the user to modify the assignment by returning it. If a truthy - * value is returned, we use that and do not apply any pending seeks to it either. - * The user can alternatively use the assignmentFns argument. - * Precedence is given to the calling of functions within assignmentFns. */ - let assignmentModified = false; - if (typeof userSpecifiedRebalanceCb === 'function') { - call = new Promise((resolve, reject) => { + if (typeof userSpecifiedRebalanceCb === 'function') { const assignmentFns = { - assign: assignmentFn.bind(this), - unassign: unassignmentFn.bind(this), + assign: assignmentFn, + unassign: unassignmentFn, + assignmentLost: () => isLost, }; - /* The user specified callback may be async, or sync. Wrapping it in a - * Promise.resolve ensures that we always get a promise back. */ - return Promise.resolve( - userSpecifiedRebalanceCb(err, assignment, assignmentFns) - ).then(alternateAssignment => { - if (alternateAssignment) { - assignment = alternateAssignment; - assignmentModified = true; - } - resolve(); - }).catch(reject); - }); - } else if (err.code !== LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS && err.code !== LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS) { - call = Promise.reject(`Unexpected rebalance_cb error code ${err.code}`).catch((e) => { - this.#logger.error(e); - }); - } - - call - .finally(async () => { - /* Emit the event */ - this.#internalClient.emit('rebalance', err, assignment); - - /** - * We never need to clear the cache in case of a rebalance. - * This is because rebalances are triggered ONLY when we call the consume() - * method of the internalClient. - * In case consume() is being called, we've already either consumed all the messages - * in the cache, or timed out (this.#messageCache.cachedTime is going to exceed max.poll.interval) - * and marked the cache stale. This means that the cache is always expired when a rebalance - * is triggered. - * This is applicable both for incremental and non-incremental rebalances. - * Multiple consume()s cannot be called together, too, because we make sure that only - * one worker is calling into the internal consumer at a time. - */ - + let alternateAssignment = null; try { - if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { + alternateAssignment = await userSpecifiedRebalanceCb(err, assignment, assignmentFns); + } catch (e) { + this.#logger.error(`Error from user's rebalance callback: ${e.stack}, `+ + 'continuing with the default rebalance behavior.'); + } - const checkPendingSeeks = this.#pendingSeeks.size !== 0; - if (checkPendingSeeks && !assignmentModified && !assignmentFnCalled) - assignment = this.#assignAsPerSeekedOffsets(assignment); + if (alternateAssignment) { + assignment = alternateAssignment; + assignmentModified = true; + } + } else if (err.code !== LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS && err.code !== LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS) { + throw new Error(`Unexpected rebalance_cb error code ${err.code}`); + } - assignmentFn.call(this, assignment); + } finally { + /* Emit the event */ + this.#internalClient.emit('rebalance', err, assignment); + + /** + * We never need to clear the cache in case of a rebalance. + * This is because rebalances are triggered ONLY when we call the consume() + * method of the internalClient. + * In case consume() is being called, we've already either consumed all the messages + * in the cache, or timed out (this.#messageCache.cachedTime is going to exceed max.poll.interval) + * and marked the cache stale. This means that the cache is always expired when a rebalance + * is triggered. + * This is applicable both for incremental and non-incremental rebalances. + * Multiple consume()s cannot be called together, too, because we make sure that only + * one worker is calling into the internal consumer at a time. + */ + try { - if (checkPendingSeeks) { - const offsetsToCommit = assignment - .filter((topicPartition) => topicPartition.offset !== undefined) - .map((topicPartition) => ({ - topic: topicPartition.topic, - partition: topicPartition.partition, - offset: String(topicPartition.offset), - })); + if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { - if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { - await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); - } - } + const checkPendingSeeks = this.#pendingSeeks.size !== 0; + if (checkPendingSeeks && !assignmentModified && !assignmentFnCalled) + assignment = this.#assignAsPerSeekedOffsets(assignment); - // Populate per-partion caches. - // For cooperative sticky, just add the newly recieved partitions. - // If it's eager, it's already empty, so we can add all the partitions. - this.#messageCache.addTopicPartitions(assignment); + assignmentFn(assignment); - } else { - unassignmentFn.call(this, assignment); - } - } catch (e) { - // Ignore exceptions if we are not connected - if (this.#internalClient.isConnected()) { - this.#internalClient.emit('rebalance.error', e); - } + } else { + unassignmentFn(assignment); } - - /** - * Schedule worker termination here, in case the number of workers is not equal to the target concurrency. - * We need to do this so we will respawn workers with the correct concurrency count. - */ - const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); - if (workersToSpawn !== this.#workers.length) { - this.#workerTerminationScheduled = true; - /* We don't need to await the workers here. We are OK if the termination and respawning - * occurs later, since even if we have a few more or few less workers for a while, it's - * not a big deal. */ + } catch (e) { + // Ignore exceptions if we are not connected + if (this.#internalClient.isConnected()) { + this.#internalClient.emit('rebalance.error', e); } - }); + } + + /** + * Schedule worker termination here, in case the number of workers is not equal to the target concurrency. + * We need to do this so we will respawn workers with the correct concurrency count. + */ + const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); + if (workersToSpawn !== this.#workers.length) { + this.#workerTerminationScheduled.resolve(); + /* We don't need to await the workers here. We are OK if the termination and respawning + * occurs later, since even if we have a few more or few less workers for a while, it's + * not a big deal. */ + } + this.#rebalanceCbInProgress.resolve(); + } } #kafkaJSToConsumerConfig(kjsConfig) { @@ -441,7 +462,7 @@ class Consumer { if (Object.hasOwn(kjsConfig, 'rebalanceTimeout')) { /* In librdkafka, we use the max poll interval as the rebalance timeout as well. */ rdKafkaConfig['max.poll.interval.ms'] = +kjsConfig.rebalanceTimeout; - } else { + } else if (!rdKafkaConfig['max.poll.interval.ms']) { rdKafkaConfig['max.poll.interval.ms'] = 300000; /* librdkafka default */ } @@ -547,7 +568,11 @@ class Consumer { /* Certain properties that the user has set are overridden. We use trampolines to accommodate the user's callbacks. * TODO: add trampoline method for offset commit callback. */ rdKafkaConfig['offset_commit_cb'] = true; - rdKafkaConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); + rdKafkaConfig['rebalance_cb'] = (err, assignment) => this.#rebalanceCallback(err, assignment).catch(e => + { + if (this.#logger) + this.#logger.error(`Error from rebalance callback: ${e.stack}`); + }); /* We handle offset storage within the promisified API by ourselves. Thus we don't allow the user to change this * setting and set it to false. */ @@ -564,6 +589,20 @@ class Consumer { this.#autoCommit = rdKafkaConfig['enable.auto.commit']; } + /** + * Actual max poll interval is twice the configured max poll interval, + * because we want to ensure that when we ask for worker termination, + * and there is one last message to be processed, we can process it in + * the configured max poll interval time. + * This will cause the rebalance callback timeout to be double + * the value of the configured max poll interval. + * But it's expected otherwise we cannot have a cache and need to consider + * max poll interval reached on processing the very first message. + */ + this.#maxPollIntervalMs = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; + this.#cacheExpirationTimeoutMs = this.#maxPollIntervalMs; + rdKafkaConfig['max.poll.interval.ms'] = this.#maxPollIntervalMs * 2; + return rdKafkaConfig; } @@ -668,7 +707,6 @@ class Consumer { const topic = payload.batch.topic; const partition = payload.batch.partition; - const key = partitionKey({ topic, partition }); payload._lastResolvedOffset = { offset, leaderEpoch }; @@ -678,7 +716,6 @@ class Consumer { partition, offset + 1, leaderEpoch); - this.#lastConsumedOffsets.set(key, offset + 1); } catch (e) { /* Not much we can do, except log the error. */ this.#logger.error(`Consumer encountered error while storing offset. Error details: ${e}:${e.stack}`, this.#createConsumerBindingMessageMetadata()); @@ -699,6 +736,31 @@ class Consumer { await this.commitOffsets(); } + /** + * Request a size increase. + * It increases the size by 2x, but only if the size is less than 1024, + * only if the size has been requested to be increased twice in a row. + */ + #increaseMaxSize() { + if (this.#messageCacheMaxSize === 1024) + return; + this.#increaseCount++; + if (this.#increaseCount <= 1) + return; + this.#messageCacheMaxSize = Math.min(this.#messageCacheMaxSize << 1, 1024); + this.#increaseCount = 0; + } + + /** + * Request a size decrease. + * It decreases the size to 80% of the last received size, with a minimum of 1. + * @param {number} recvdSize - the number of messages received in the last poll. + */ + #decreaseMaxSize(recvdSize) { + this.#messageCacheMaxSize = Math.max(Math.floor((recvdSize * 8) / 10), 1); + this.#increaseCount = 0; + } + /** * Converts a list of messages returned by node-rdkafka into a message that can be used by the eachBatch callback. * @param {import("../..").Message[]} messages - must not be empty. Must contain messages from the same topic and partition. @@ -777,14 +839,53 @@ class Consumer { return returnPayload; } + async #fetchAndResolveWith(takeFromCache, size) { + if (this.#fetchInProgress) { + return null; + } + + try { + this.#fetchInProgress = new DeferredPromise(); + const fetchResult = new DeferredPromise(); + this.#logger.debug(`Attempting to fetch ${size} messages to the message cache`, + this.#createConsumerBindingMessageMetadata()); + this.#internalClient.consume(size, (err, messages) => + fetchResult.resolve([err, messages])); + + let [err, messages] = await fetchResult; + if (this.#rebalanceCbInProgress) { + await this.#rebalanceCbInProgress; + this.#rebalanceCbInProgress = null; + } + + if (err) { + throw createKafkaJsErrorFromLibRdKafkaError(err); + } + + this.#messageCache.addMessages(messages); + const res = takeFromCache(); + this.#lastFetchClockNs = hrtime.bigint(); + this.#maxPollIntervalRestart.resolve(); + if (messages.length === this.#messageCacheMaxSize) { + this.#increaseMaxSize(); + } else { + this.#decreaseMaxSize(messages.length); + } + return res; + } finally { + this.#fetchInProgress.resolve(); + this.#fetchInProgress = null; + } + } + /** * Consumes a single message from the internal consumer. - * @param {number} savedIndex - the index of the message in the cache to return. + * @param {PerPartitionCache} ppc Per partition cache to use or null|undefined . * @returns {Promise} a promise that resolves to a single message or null. * @note this method caches messages as well, but returns only a single message. */ - async #consumeSingleCached(savedIndex) { - const msg = this.#messageCache.next(savedIndex); + async #consumeSingleCached(ppc) { + const msg = this.#messageCache.next(ppc); if (msg) { return msg; } @@ -793,33 +894,12 @@ class Consumer { * exceeds the number of partitions containing messages. So in this case, * we should not call for new fetches, rather, try to focus on what we have left. */ - if (!msg && this.#messageCache.pendingSize() !== 0) { + if (!msg && this.#messageCache.assignedSize !== 0) { return null; } - if (this.#fetchInProgress) { - return null; - } - - this.#fetchInProgress = true; - this.#logger.debug(`Attempting to fetch ${this.#messageCache.maxSize} messages to the message cache`, this.#createConsumerBindingMessageMetadata()); - return new Promise((resolve, reject) => { - this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { - this.#fetchInProgress = false; - if (err) { - reject(createKafkaJsErrorFromLibRdKafkaError(err)); - return; - } - this.#messageCache.addMessages(messages); - const message = this.#messageCache.next(); - if (messages.length === this.#messageCache.maxSize) { - this.#messageCache.increaseMaxSize(); - } else { - this.#messageCache.decreaseMaxSize(messages.length); - } - resolve(message); - }); - }); + return this.#fetchAndResolveWith(() => this.#messageCache.next(), + this.#messageCacheMaxSize); } /** @@ -830,8 +910,8 @@ class Consumer { * @note this method caches messages as well. * @sa #consumeSingleCached */ - async #consumeCachedN(savedIndex, size) { - const msgs = this.#messageCache.nextN(savedIndex, size); + async #consumeCachedN(ppc, size) { + const msgs = this.#messageCache.nextN(ppc, size); if (msgs) { return msgs; } @@ -840,33 +920,13 @@ class Consumer { * exceeds the number of partitions containing messages. So in this case, * we should not call for new fetches, rather, try to focus on what we have left. */ - if (!msgs && this.#messageCache.pendingSize() !== 0) { - return null; - } - - if (this.#fetchInProgress) { + if (!msgs && this.#messageCache.assignedSize !== 0) { return null; } - this.#fetchInProgress = true; - this.#logger.debug(`Attempting to fetch ${this.#messageCache.maxSize} messages to the message cache`, this.#createConsumerBindingMessageMetadata()); - return new Promise((resolve, reject) => { - this.#internalClient.consume(this.#messageCache.maxSize, (err, messages) => { - this.#fetchInProgress = false; - if (err) { - reject(createKafkaJsErrorFromLibRdKafkaError(err)); - return; - } - this.#messageCache.addMessages(messages); - const msgsList = this.#messageCache.nextN(-1, size); - if (messages.length === this.#messageCache.maxSize) { - this.#messageCache.increaseMaxSize(); - } else { - this.#messageCache.decreaseMaxSize(messages.length); - } - resolve(msgsList); - }); - }); + return this.#fetchAndResolveWith(() => + this.#messageCache.nextN(null, size), + this.#messageCacheMaxSize); } /** @@ -1036,10 +1096,7 @@ class Consumer { configCopy.partitionsConsumedConcurrently = 1; } - const rdKafkaConfig = this.#config(); - const maxPollInterval = rdKafkaConfig['max.poll.interval.ms'] ?? 300000; - this.#messageCache = new MessageCache(Math.floor(maxPollInterval * 0.8), configCopy.partitionsConsumedConcurrently, this.#logger); - + this.#messageCache = new MessageCache(this.#logger); /* We deliberately don't await this because we want to return from this method immediately. */ this.#runInternal(configCopy); } @@ -1052,10 +1109,14 @@ class Consumer { * @returns {Promise} the cache index of the message that was processed. */ async #messageProcessor(m, config) { + let ppc; + [m, ppc] = m; + let key = partitionKey(m); let eachMessageProcessed = false; const payload = this.#createPayload(m); try { + this.#lastConsumedOffsets.set(key, m); await config.eachMessage(payload); eachMessageProcessed = true; } catch (e) { @@ -1076,10 +1137,11 @@ class Consumer { /* If the message is unprocessed, due to an error, or because the user has not resolved it, we seek back. */ if (!eachMessageProcessed) { - await this.seek({ + this.seek({ topic: m.topic, partition: m.partition, offset: m.offset, + leaderEpoch: m.leaderEpoch, }); } @@ -1087,7 +1149,6 @@ class Consumer { if (eachMessageProcessed) { try { this.#internalClient._offsetsStoreSingle(m.topic, m.partition, Number(m.offset) + 1, m.leaderEpoch); - this.#lastConsumedOffsets.set(partitionKey(m), Number(m.offset) + 1); } catch (e) { /* Not much we can do, except log the error. */ this.#logger.error(`Consumer encountered error while storing offset. Error details: ${JSON.stringify(e)}`, this.#createConsumerBindingMessageMetadata()); @@ -1095,32 +1156,30 @@ class Consumer { } - /* 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. - * TOOD: this block can probably be common and not per message. */ - if (this.#checkPendingSeeks) - await this.#seekInternal(); - - return m.index; + return ppc; } /** * Processes a batch of messages. * - * @param ms Messages as obtained from #consumeCachedN (ms.length !== 0). + * @param {[[Message], PerPartitionCache]} ms Messages as obtained from #consumeCachedN (ms.length !== 0). * @param config Config as passed to run(). - * @returns {Promise} the cache index of the message that was processed. + * @returns {Promise} the PPC corresponding to + * the passed batch. */ async #batchProcessor(ms, config) { + let ppc; + [ms, ppc] = ms; const key = partitionKey(ms[0]); const payload = this.#createBatchPayload(ms); this.#topicPartitionToBatchPayload.set(key, payload); let lastOffsetProcessed = { offset: -1, leaderEpoch: -1 }; - const lastOffset = +(ms[ms.length - 1].offset); - const lastLeaderEpoch = ms[ms.length - 1].leaderEpoch; + const firstMessage = ms[0]; + const lastMessage = ms[ms.length - 1]; + const lastOffset = +(lastMessage.offset); + const lastLeaderEpoch = lastMessage.leaderEpoch; try { await config.eachBatch(payload); @@ -1164,21 +1223,46 @@ class Consumer { /* If any message is unprocessed, either due to an error or due to the user not marking it processed, we must seek * back to get it so it can be reprocessed. */ if (lastOffsetProcessed.offset !== lastOffset) { - const offsetToSeekTo = lastOffsetProcessed.offset === -1 ? ms[0].offset : (lastOffsetProcessed.offset + 1); - await this.seek({ - topic: ms[0].topic, - partition: ms[0].partition, + const offsetToSeekTo = lastOffsetProcessed.offset === -1 ? firstMessage.offset : (lastOffsetProcessed.offset + 1); + const leaderEpoch = lastOffsetProcessed.offset === -1 ? firstMessage.leaderEpoch : lastOffsetProcessed.leaderEpoch; + this.seek({ + topic: firstMessage.topic, + partition: firstMessage.partition, offset: offsetToSeekTo, + leaderEpoch: leaderEpoch, }); } - /* 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(); + return ppc; + } + + #discardMessages(ms, ppc) { + if (ms) { + let m = ms[0]; + if (m.constructor === Array) { + m = m[0]; + } + ppc = ms[1]; + if (m && !this.#lastConsumedOffsets.has(ppc.key)) { + this.#lastConsumedOffsets.set(ppc.key, { + topic: m.topic, + partition: m.partition, + offset: m.offset - 1, + }); + } + } + return ppc; + } - return ms.index; + async #nextFetchRetry() { + if (this.#fetchInProgress) { + await this.#fetchInProgress; + } else { + /* Backoff a little. If m is null, we might be without messages + * or in available partition starvation, and calling consumeSingleCached + * in a tight loop will help no one. */ + await Timer.withTimeout(1); + } } /** @@ -1194,46 +1278,96 @@ class Consumer { * Worker termination acts as a async barrier. */ async #worker(config, perMessageProcessor, fetcher) { - let nextIdx = -1; - while (!this.#workerTerminationScheduled) { - /* Invalidate the message cache if needed */ - const locallyStale = this.#messageCache.popLocallyStale(); - if (this.#messageCache.isStale()) { - this.#logger.debug("Scheduling worker termination", this.#createConsumerBindingMessageMetadata()); - this.#workerTerminationScheduled = true; - break; - } else if (locallyStale.length !== 0) { - // TODO: is it correct to await some concurrent promises for eachMessage here? - // to be safe we can do it, but I don't think we really need to do that for - // any correctness reason. - await this.#clearCacheAndResetPositions(locallyStale); - continue; - } + let ppc = null; - const m = await fetcher(nextIdx).catch(e => { + while (!this.#workerTerminationScheduled.resolved) { + + const ms = await fetcher(ppc).catch(e => { /* Since this error cannot be exposed to the user in the current situation, just log and retry. - * This is due to restartOnFailure being set to always true. */ + * This is due to restartOnFailure being set to always true. */ if (this.#logger) this.#logger.error(`Consumer encountered error while consuming. Retrying. Error details: ${e} : ${e.stack}`, this.#createConsumerBindingMessageMetadata()); }); - nextIdx = -1; + if (this.#pendingOperations.length) { + ppc = this.#discardMessages(ms, ppc); + break; + } - if (!m) { - /* Backoff a little. If m is null, we might be fetching from the internal consumer (fetch in progress), - * and calling consumeSingleCached in a tight loop will help no one. */ - await new Promise((resolve) => setTimeout(resolve, 1)); + if (!ms) { + await this.#nextFetchRetry(); continue; } - nextIdx = await perMessageProcessor(m, config); + ppc = await perMessageProcessor(ms, config); } - if (nextIdx !== -1) { - this.#messageCache.return(nextIdx); + if (ppc) + this.#messageCache.return(ppc); + } + + async #checkMaxPollIntervalNotExceeded(now) { + const maxPollExpiration = this.#lastFetchClockNs + + BigInt((this.#cacheExpirationTimeoutMs + this.#maxPollIntervalMs) + * 1e6); + + let interval = Number(maxPollExpiration - now) / 1e6; + if (interval < 1) + interval = 1; + await Timer.withTimeout(interval, + this.#maxPollIntervalRestart); + now = hrtime.bigint(); + + if (now > (maxPollExpiration - 1000000n)) { + this.#markBatchPayloadsStale(this.assignment()); } } + /** + * Clears the cache and resets the positions when + * the internal client hasn't been polled for more than + * max poll interval since the last fetch. + * After that it waits until barrier is reached or + * max poll interval is reached. In the latter case it + * marks the batch payloads as stale. + */ + async #cacheExpirationLoop() { + while (!this.#workerTerminationScheduled.resolved) { + let now = hrtime.bigint(); + const cacheExpiration = this.#lastFetchClockNs + + BigInt(this.#cacheExpirationTimeoutMs * 1e6); + + if (now > cacheExpiration) { + this.#addPendingOperation(() => + this.#clearCacheAndResetPositions()); + await this.#checkMaxPollIntervalNotExceeded(now); + break; + } + + let interval = Number(cacheExpiration - now) / 1e6; + if (interval < 100) + interval = 100; + const promises = Promise.race([this.#workerTerminationScheduled, + this.#maxPollIntervalRestart]); + await Timer.withTimeout(interval, + promises); + if (this.#maxPollIntervalRestart.resolved) + this.#maxPollIntervalRestart = new DeferredPromise(); + } + if (this.#maxPollIntervalRestart.resolved) + this.#maxPollIntervalRestart = new DeferredPromise(); + } + + /** + * Executes all pending operations and clears the list. + */ + async #executePendingOperations() { + for (const op of this.#pendingOperations) { + await op(); + } + this.#pendingOperations = []; + } + /** * Internal polling loop. * Spawns and awaits workers until disconnect is initiated. @@ -1242,38 +1376,45 @@ class Consumer { this.#concurrency = config.partitionsConsumedConcurrently; const perMessageProcessor = config.eachMessage ? this.#messageProcessor : this.#batchProcessor; /* TODO: make this dynamic, based on max batch size / size of last message seen. */ - const maxBatchSize = 30; + const maxBatchSize = 32; const fetcher = config.eachMessage ? (savedIdx) => this.#consumeSingleCached(savedIdx) : (savedIdx) => this.#consumeCachedN(savedIdx, maxBatchSize); this.#workers = []; - while (!(await acquireOrLog(this.#lock, this.#logger))); - while (!this.#disconnectStarted) { - this.#workerTerminationScheduled = false; - const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); - this.#logger.debug(`Spawning ${workersToSpawn} workers`, this.#createConsumerBindingMessageMetadata()); - this.#workers = - Array(workersToSpawn) - .fill() - .map((_, i) => - this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this)) - .catch(e => { - if (this.#logger) - this.#logger.error(`Worker ${i} encountered an error: ${e}:${e.stack}`, this.#createConsumerBindingMessageMetadata()); - })); - - /* Best we can do is log errors on worker issues - handled by the catch block above. */ - await Promise.allSettled(this.#workers); - - /* One of the possible reasons for the workers to end is that the cache is globally stale. - * We need to take care of expiring it. */ - if (this.#messageCache.isStale()) { - await this.#clearCacheAndResetPositions(); + await this.#lock.write(async () => { + + while (!this.#disconnectStarted) { + if (this.#maxPollIntervalRestart.resolved) + this.#maxPollIntervalRestart = new DeferredPromise(); + + this.#workerTerminationScheduled = new DeferredPromise(); + this.#lastFetchClockNs = hrtime.bigint(); + if (this.#pendingOperations.length === 0) { + const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); + const cacheExpirationLoop = this.#cacheExpirationLoop(); + this.#logger.debug(`Spawning ${workersToSpawn} workers`, this.#createConsumerBindingMessageMetadata()); + this.#workers = + Array(workersToSpawn) + .fill() + .map((_, i) => + this.#worker(config, perMessageProcessor.bind(this), fetcher.bind(this)) + .catch(e => { + if (this.#logger) + this.#logger.error(`Worker ${i} encountered an error: ${e}:${e.stack}`); + })); + + /* Best we can do is log errors on worker issues - handled by the catch block above. */ + await Promise.allSettled(this.#workers); + this.#maxPollIntervalRestart.resolve(); + await cacheExpirationLoop; + } + + await this.#executePendingOperations(); } - } - this.#lock.release(); + }); + this.#maxPollIntervalRestart.resolve(); } /** @@ -1305,7 +1446,6 @@ class Consumer { } throw new error.KafkaJSError('consume() is not implemented.' + m, { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); - // return m ?? null; } async #commitOffsetsUntilNoStateErr(offsetsToCommit) { @@ -1389,91 +1529,126 @@ class Consumer { * @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 = partitionKey(topicPartition); if (!this.#pendingSeeks.has(key)) continue; - const offset = this.#pendingSeeks.get(key); + const tpo = this.#pendingSeeks.get(key); this.#pendingSeeks.delete(key); - assignment[i].offset = offset; - - offsetsToCommit.push({ - topic: topicPartition.topic, - partition: topicPartition.partition, - offset: String(offset), - }); + assignment[i].offset = tpo.offset; + assignment[i].leaderEpoch = tpo.leaderEpoch; } 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; + #addPendingOperation(fun) { + if (this.#pendingOperations.length === 0) { + this.#workerTerminationScheduled.resolve(); + } + this.#pendingOperations.push(fun); + } + + async #seekInternal(topicPartitionOffsets) { + if (topicPartitionOffsets.length === 0) { + return; + } + + // Uncomment to test an additional delay in seek + // await Timer.withTimeout(1000); + + const seekedPartitions = []; + const pendingSeeks = new Map(); + const assignmentSet = new Set(); + for (const topicPartitionOffset of topicPartitionOffsets) { + const key = partitionKey(topicPartitionOffset); + pendingSeeks.set(key, topicPartitionOffset); + } + const assignment = this.assignment(); for (const topicPartition of assignment) { const key = partitionKey(topicPartition); - if (!this.#pendingSeeks.has(key)) + assignmentSet.add(key); + if (!pendingSeeks.has(key)) continue; + seekedPartitions.push([key, pendingSeeks.get(key)]); + } - const offset = this.#pendingSeeks.get(key); - this.#pendingSeeks.delete(key); - - const topicPartitionOffset = { - topic: topicPartition.topic, - partition: topicPartition.partition, - offset - }; + for (const topicPartitionOffset of topicPartitionOffsets) { + const key = partitionKey(topicPartitionOffset); + if (!assignmentSet.has(key)) + this.#pendingSeeks.set(key, topicPartitionOffset); + } - /* The ideal sequence of events here is to: - * 1. Mark the cache as stale so we don't consume from it any further. - * 2. Call clearCacheAndResetPositions() for the topic partition, which is supposed - * to be called after each cache invalidation. - * - * However, what (2) does is to pop lastConsumedOffsets[topic partition], and seeks to - * the said popped value. Seeking is redundant since we seek here anyway. So, we can skip - * the seek by just clearing the lastConsumedOffsets[topic partition]. - */ - this.#messageCache.markStale([topicPartition]); + const offsetsToCommit = []; + const librdkafkaSeekPromises = []; + for (const [key, topicPartitionOffset] of seekedPartitions) { this.#lastConsumedOffsets.delete(key); + this.#messageCache.markStale([topicPartitionOffset]); + offsetsToCommit.push(topicPartitionOffset); + + const librdkafkaSeekPromise = new DeferredPromise(); + this.#internalClient.seek(topicPartitionOffset, 1000, + (err) => { + if (err) + this.#logger.error(`Error while calling seek from within seekInternal: ${err}`, this.#createConsumerBindingMessageMetadata()); + librdkafkaSeekPromise.resolve(); + }); + librdkafkaSeekPromises.push(librdkafkaSeekPromise); + } + await Promise.allSettled(librdkafkaSeekPromises); + await Promise.all(librdkafkaSeekPromises); - /* 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. */ - /* TODO: is it worth awaiting seeks to finish? */ - this.#internalClient.seek(topicPartitionOffset, 0, err => { - if (err) - this.#logger.error(`Error while calling seek from within seekInternal: ${err}`, this.#createConsumerBindingMessageMetadata()); - }); - 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; - } + for (const [key, ] of seekedPartitions) { + this.#pendingSeeks.delete(key); } /* Offsets are committed on seek only when in compatibility mode. */ if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); + } + } + + #markBatchPayloadsStale(topicPartitions) { + for (const topicPartition of topicPartitions) { + const key = partitionKey(topicPartition); + if (this.#topicPartitionToBatchPayload.has(key)) + this.#topicPartitionToBatchPayload.get(key)._stale = true; + } + } + + async #pauseInternal(topicPartitions) { + // Uncomment to test future async pause + // await Timer.withTimeout(1000); + + this.#messageCache.markStale(topicPartitions); + this.#internalClient.pause(topicPartitions); + + const seekOffsets = []; + for (let topicPartition of topicPartitions) { + const key = partitionKey(topicPartition); + if (this.#lastConsumedOffsets.has(key)) { + const seekOffset = this.#lastConsumedOffsets.get(key); + const topicPartitionOffset = { + topic: topicPartition.topic, + partition: topicPartition.partition, + offset: seekOffset.offset + 1, + leaderEpoch: seekOffset.leaderEpoch, + }; + seekOffsets.push(topicPartitionOffset); + } + } + if (seekOffsets.length) { + await this.#seekInternal(seekOffsets, false); } + } - return invalidateMessage; + async #resumeInternal(topicPartitions) { + // Uncomment to test future async resume + // await Timer.withTimeout(1000); + this.#internalClient.resume(topicPartitions); } /** @@ -1483,7 +1658,6 @@ class Consumer { * 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|null} a promise that resolves when the seek has been performed. */ seek(topicPartitionOffset) { if (this.#state !== ConsumerState.CONNECTED) { @@ -1501,18 +1675,11 @@ class Consumer { throw new error.KafkaJSError('Offset must be >= 0, or a special value.', { code: error.ErrorCodes.ERR__INVALID_ARG }); } - this.#checkPendingSeeks = true; - const key = partitionKey(rdKafkaTopicPartitionOffset); - this.#pendingSeeks.set(key, rdKafkaTopicPartitionOffset.offset); + /* If anyone's using eachBatch, mark the batch as stale. */ + this.#markBatchPayloadsStale([rdKafkaTopicPartitionOffset]); - /* Only for eachBatch: - * Immediately mark the batch it's associated with as stale, even if we don't - * do the actual 'seekInternal' at this time. This is because we need read-after-write - * consistency for eachBatch, and calling seek(toppar) from within eachBatch(toppar) - * should change the result of batch.isStale() immediately. */ - if (this.#topicPartitionToBatchPayload.has(key)) { - this.#topicPartitionToBatchPayload.get(key)._stale = true; - } + this.#addPendingOperation(() => + this.#seekInternal([rdKafkaTopicPartitionOffset])); } async describeGroup() { @@ -1591,19 +1758,16 @@ class Consumer { return; } - /* TODO: error handling is lacking for pause, including partition level errors. */ - this.#internalClient.pause(flattenedToppars); - - /* Mark the messages in the cache as stale, runInternal* will deal with - * making it unusable. */ - this.#messageCache.markStale(flattenedToppars); - /* If anyone's using eachBatch, mark the batch as stale. */ - flattenedToppars.map(partitionKey) - .filter(key => this.#topicPartitionToBatchPayload.has(key)) - .forEach(key => this.#topicPartitionToBatchPayload.get(key)._stale = true); + this.#markBatchPayloadsStale(flattenedToppars); + + flattenedToppars.forEach( + topicPartition => this.#pausedPartitions.set( + partitionKey(topicPartition), + topicPartition)); - flattenedToppars.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); + this.#addPendingOperation(() => + this.#pauseInternal(flattenedToppars)); /* Note: we don't use flattenedToppars here because resume flattens them again. */ return () => this.resume(toppars); @@ -1616,7 +1780,6 @@ class Consumer { paused() { const topicToPartitions = Array .from(this.#pausedPartitions.values()) - .map(JSON.parse) .reduce( (acc, { topic, partition }) => { if (!acc[topic]) { @@ -1663,9 +1826,11 @@ class Consumer { if (flattenedToppars.length === 0) { return; } - this.#internalClient.resume(flattenedToppars); - - flattenedToppars.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.delete(topicPartition)); + flattenedToppars.map(partitionKey). + forEach(key => this.#pausedPartitions.delete(key)); + + this.#addPendingOperation(() => + this.#resumeInternal(flattenedToppars)); } on(/* eventName, listener */) { @@ -1707,15 +1872,13 @@ class Consumer { } this.#disconnectStarted = true; - this.#workerTerminationScheduled = true; - + this.#workerTerminationScheduled.resolve(); this.#logger.debug("Signalling disconnection attempt to workers", this.#createConsumerBindingMessageMetadata()); - while (!(await acquireOrLog(this.#lock, this.#logger))); /* Just retry... */ + await this.#lock.write(async () => { - this.#state = ConsumerState.DISCONNECTING; + this.#state = ConsumerState.DISCONNECTING; - /* Since there are state-checks before everything, we are safe to proceed without the lock. */ - await this.#lock.release(); + }); await new Promise((resolve, reject) => { const cb = (err) => { diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 06077250..0c9f43eb 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -1,339 +1,200 @@ -const { hrtime } = require('process'); const { partitionKey, } = require('./_common'); -const { Heap } = require('./_heap'); +const { LinkedList } = require('./_linked-list'); /** * A PerPartitionMessageCache is a cache for messages for a single partition. */ class PerPartitionMessageCache { /* The cache is a list of messages. */ - cache = []; - /* Index of next element to be fetched in the cache. */ - currentIndex = 0; - /* Whether the cache is stale. */ - stale = false; + #cache = new LinkedList(); + /* The key for the partition. */ + #key = null; + /* Whether the cache is assigned to a consumer. */ + _assigned = false; + + constructor(key) { + this.#key = key; + } /** * Returns the number of total elements in the cache. */ size() { - return this.cache.length; - } - - /** - * Clears the cache. - */ - clear() { - this.cache = []; - this.currentIndex = 0; - this.stale = false; + return this.#cache.length; } /** * Adds a message to the cache. */ - add(message) { - this.cache.push(message); + _add(message) { + this.#cache.addLast(message); } - /** - * Returns whether the cache is stale. - */ - isStale() { - return this.stale; + get key() { + return this.#key; } /** * @returns The next element in the cache or null if none exists. - * @warning Does not check for staleness. */ - next() { - return this.currentIndex < this.cache.length ? this.cache[this.currentIndex++] : null; + _next() { + return this.#cache.removeFirst(); } /** - * @returns Upto `n` next elements in the cache or an null if none available. - * @warning Does not check for staleness. + * @returns Upto `n` next elements in the cache or an empty array if none exists. */ - nextN(n) { - if (this.currentIndex >= this.cache.length) { - return null; - } + _nextN(n) { + const len = this.#cache.length; + n = (n < 0 || len < n) ? len : n; - if (this.currentIndex + n >= this.cache.length) { - const res = this.cache.slice(this.currentIndex); - this.currentIndex = this.cache.length; - return res; + const ret = new Array(n); + for (let i = 0; i < n; i++) { + ret[i] = this.#cache.removeFirst(); } - - const res = this.cache.slice(this.currentIndex, this.currentIndex + n); - this.currentIndex += n; - return res; + return ret; } } /** * MessageCache defines a dynamically sized cache for messages. - * Internally, it uses PerPartitionMessageCache to store messages for each partition. - * The capacity is increased or decreased according to whether the last fetch of messages - * was less than the current capacity or saturated the current capacity. + * Internally, it uses PerPartitionMessageCache to store messages for each partition. */ class MessageCache { + #size; + /* Map of topic+partition to PerPartitionMessageCache. */ + #tpToPpc; + /* LinkedList of available partitions. */ + #availablePartitions; + /* LinkedList of assigned partitions. */ + #assignedPartitions; - constructor(expiryDurationMs, maxConcurrency, logger) { - /* Per partition cache list containing non-empty PPCs */ - this.ppcList = []; - /* Map of topic+partition to PerPartitionMessageCache. */ - this.tpToPpc = new Map(); - /* Index of the current PPC in the ppcList. */ - this.currentPpcTODO_remove_this = 0; - /* Maximum size of the cache. (Capacity) */ - this.maxSize = 1; - /* Number of times the size has been increased in a row, used for accounting for maxSize. */ - this.increaseCount = 0; - /* Last cached time */ - this.cachedTime = hrtime(); - /* Whether the cache is stale. */ - this.stale = false; - /* Expiry duration for this cache */ - this.expiryDurationMs = expiryDurationMs; - /* A list of caches which have been marked stale since the last call to popLocallyStale or addMessages. */ - this.locallyStaleCaches = []; - /* Max allowed concurrency */ - this.maxConcurrency = maxConcurrency; - /* Contains a list of indices of ppcList from which we are allowed to consume. */ - this.indices = new Heap(); - /* Largest ppc index we are allowed to consume from (inclusive). */ - this.maxIndicesIndex = 0; - /* Contains a list of indices of ppcList from which we have sent a message returned through next, but - * the user has not returned the index back to us via next(idx) */ - this.pendingIndices = new Set(); - /* Logger provided by cache user. Must have 'error' function defined on it. `console` is used by default. */ - this.logger = logger ?? console; - } - pendingSize() { - return this.pendingIndices.size; + constructor(logger) { + this.logger = logger ?? console; + this.#reinit(); } /** - * Add a set of topic partitions to the cache (empty PPCs). - * Pre-conditions: ppcList must be empty (cache is inactive) + * Reinitializes the cache. */ - addTopicPartitions(topicPartitions) { - if (this.ppcList.some(ppc => (ppc.currentIndex < ppc.size()) && !ppc.stale)) { - throw new Error('Cannot add topic partitions to a cache which contains unprocessed, unstale elements.'); - } - for (const topicPartition of topicPartitions) { - const key = partitionKey(topicPartition); - const existing = this.tpToPpc.get(key); - /* We're erring on the side of caution by including this check, as in the current model, - * a rebalance occurs only if all the caches are drained. */ - if (existing && existing.cache.length > 0 && !existing.stale && existing.currentIndex < existing.cache.length) { - this.logger.error("Cache already exists for key " + key + " with messages in it."); - throw new Error("Cache already exists for key " + key + " with messages in it."); - } - this.tpToPpc.set(key, new PerPartitionMessageCache()); - } + #reinit() { + this.#tpToPpc = new Map(); + this.#availablePartitions = new LinkedList(); + this.#assignedPartitions = new LinkedList(); + this.#size = 0; } /** - * Remove a set of topic partitions from the cache. - * If topicPartitions is null, removes everything. - * Pre-conditions: ppcList must be empty (cache is inactive) + * Assign a new partition to the consumer, if available. + * + * @returns {PerPartitionMessageCache} - the partition assigned to the consumer, or null if none available. */ - removeTopicPartitions(topicPartitions = null) { - if (this.ppcList.some(ppc => (ppc.currentIndex < ppc.size()) && !ppc.stale)) { - throw new Error('Cannot remove topic partitions from a cache which contains unprocessed, unstale elements.'); - } + #assignNewPartition() { + let ppc = this.#availablePartitions.removeFirst(); + if (!ppc) + return null; - if (topicPartitions === null) { - for (const key of this.tpToPpc.keys()) { - const existing = this.tpToPpc.get(key); - /* We're erring on the side of caution by including this check, as in the current model, - * a rebalance occurs only if all the caches are drained. */ - if (existing && existing.cache.length > 0 && !existing.stale && existing.currentIndex < existing.cache.length) { - this.logger.error("Cache already exists for key " + key + " with messages in it."); - throw new Error("Cache already exists for key " + key + " with messages in it."); - } - } - this.tpToPpc.clear(); - return; - } - for (const topicPartition of topicPartitions) { - const key = partitionKey(topicPartition); - const existing = this.tpToPpc.get(key); - /* We're erring on the side of caution by including this check, as in the current model, - * a rebalance occurs only if all the caches are drained. */ - if (existing && existing.cache.length > 0 && !existing.stale && existing.currentIndex < existing.cache.length) { - this.logger.error("Cache already exists for key " + key + " with messages in it."); - throw new Error("Cache already exists for key " + key + " with messages in it."); - } - this.tpToPpc.delete(key); - } + ppc._node = this.#assignedPartitions.addLast(ppc); + ppc._assigned = true; + return ppc; } /** - * Returns whether the cache is globally stale. + * Remove an empty partition from the cache. + * + * @param {PerPartitionMessageCache} ppc The partition to remove from the cache. */ - isStale() { - if (this.stale) - return true; - - const cacheTime = hrtime(this.cachedTime); - const cacheTimeMs = Math.floor(cacheTime[0] * 1000 + cacheTime[1] / 1000000); - this.stale = cacheTimeMs > this.expiryDurationMs; - - return this.stale; + #removeEmptyPartition(ppc) { + this.#assignedPartitions.remove(ppc._node); + ppc._assigned = false; + ppc._node = null; + this.#tpToPpc.delete(ppc.key); } /** - * If there are any locally stale caches, return them, and clear - * the list of locally stale caches. + * Add a single message to a PPC. + * In case the PPC does not exist, it is created. + * + * @param {Object} message - the message to add to the cache. */ - popLocallyStale() { - if (this.locallyStaleCaches.length > 0) { - const locallyStale = this.locallyStaleCaches; - this.locallyStaleCaches = []; - return locallyStale; + #add(message) { + const key = partitionKey(message); + let cache = this.#tpToPpc.get(key); + if (!cache) { + cache = new PerPartitionMessageCache(key); + this.#tpToPpc.set(key, cache); + cache._node = this.#availablePartitions.addLast(cache); } - return []; + cache._add(message); + } + + get availableSize() { + return this.#availablePartitions.length; + } + + get assignedSize() { + return this.#assignedPartitions.length; + } + + get size() { + return this.#size; } /** * Mark a set of topic partitions 'stale'. - * If no topic partitions are provided, marks the entire cache as stale globally. * - * Pre-conditions: toppars must be in tpToPpc, may or may not be in ppcList. - * Post-conditions: PPCs marked stale, locally stale caches updated to contain said toppars. + * Post-conditions: PPCs are removed from their currently assigned list + * and deleted from the PPC map. Cache size is decremented accordingly. + * PPCs are marked as not assigned. */ - markStale(topicPartitions = null) { - if (!topicPartitions) { - this.stale = true; - return; - } - + markStale(topicPartitions) { for (const topicPartition of topicPartitions) { const key = partitionKey(topicPartition); - const cache = this.tpToPpc.get(key); - if (!cache) + const ppc = this.#tpToPpc.get(key); + if (!ppc) continue; - if (!cache.stale) { - /* Newly stale cache, so add it into list of such caches. */ - this.locallyStaleCaches.push(topicPartition); + this.#size -= ppc.size(); + if (ppc._assigned) { + this.#assignedPartitions.remove(ppc._node); + } else { + this.#availablePartitions.remove(ppc._node); } - cache.stale = true; - } - } - - /** - * Request a size increase. - * It increases the size by 2x, but only if the size is less than 1024, - * only if the size has been requested to be increased twice in a row. - */ - increaseMaxSize() { - if (this.maxSize === 1024) - return; - - this.increaseCount++; - if (this.increaseCount <= 1) - return; - - this.maxSize = Math.min(this.maxSize << 1, 1024); - this.increaseCount = 0; - } - - /** - * Request a size decrease. - * It decreases the size to 80% of the last received size, with a minimum of 1. - * @param {number} recvdSize - the number of messages received in the last poll. - */ - decreaseMaxSize(recvdSize) { - this.maxSize = Math.max(Math.floor((recvdSize * 8) / 10), 1); - this.increaseCount = 0; - } - - /** - * Add a single message to a PPC. - * Pre-conditions: PPC does not have stale messages. - * Post-conditions: PPC is unstale, ppcList contains all caches with messages in them. - */ - #add(message) { - const key = partitionKey(message); - const cache = this.tpToPpc.get(key); - if (!cache) { - this.logger.error("No cache found for message", message); - throw new Error("Inconsistency between fetched message and partition map"); - } - cache.add(message); - if (cache.size() === 1) { - this.ppcList.push(cache); - /* Just in case this cache was marked stale by pause or seek, we unstale it now - * that there are fresh messages in here. It is possible because markStale() can - * mark toppar caches as stale without checking if they're in ppcList. */ - cache.stale = false; + this.#tpToPpc.delete(key); + ppc._assigned = false; } } /** * Adds many messages into the cache, partitioning them as per their toppar. - * Pre-conditions: no locally stale caches with messages in them. - * Post-conditions: all caches are unstale, (todo: ppcList is sorted by timestamp). + * Increases cache size by the number of messages added. + * + * @param {Array} messages - the messages to add to the cache. */ addMessages(messages) { - if (this.pendingSize() > 0) { - throw new Error(`Cache cannot be added to with ${this.pendingSize()} pending indices.`); - } - - /* There will be caches in the ppcList which are either stale, or have - * run out of messages. We need to clear them, else #add() will not add - * them back to the ppcList since they're not empty. */ - this.ppcList.forEach(cache => cache.clear()); - this.ppcList = []; - - if (this.locallyStaleCaches.length !== 0 && this.locallyStaleCaches.some(tp => { - const key = partitionKey(tp); - return this.tpToPpc.get(key).size() !== 0; - })) { - throw new Error('Locally stale caches should have been cleared before adding messages.'); - } - - this.stale = false; - this.cachedTime = hrtime(); - for (const message of messages) this.#add(message); - - // TODO: add ppcList sort step. - // Rationale: ideally it's best to consume in the ascending order of timestamps. - - /* Reset the indices and pendingIndices because ppcList is being created newly. */ - this.indices.clear(); - this.pendingIndices.clear(); - this.maxIndicesIndex = Math.min(this.maxConcurrency, this.ppcList.length - 1); - for (let i = 0; i <= this.maxIndicesIndex; i++) { - this.indices.push(i); - } + this.#size += messages.length; } /** - * Allows returning the cache index of a consumed message without asking for another message. - * @param {number} idx - the index of the message that was consumed. - * @note This is a no-op if the index is not in the pendingIndices set. + * Allows returning the PPC without asking for another message. + * + * @param {PerPartitionMessageCache} ppc - the partition to return. + * + * @note this is a no-op if the PPC is not assigned. */ - return(idx) { - if (!this.pendingIndices.has(idx)) { - /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible - * that we ran out of messages and fetched a new batch. So we just discard what the user is - * returning to us. */ - this.logger.error("Returning unowned index", idx, "to cache. Discarding it."); - } else { - this.pendingIndices.delete(idx); - this.indices.add(idx); + return(ppc) { + if (!ppc._assigned) + return; + if (ppc._node) { + this.#assignedPartitions.remove(ppc._node); + ppc._node = this.#availablePartitions.addLast(ppc); + ppc._assigned = false; } } @@ -342,67 +203,27 @@ class MessageCache { * * If the current PPC is exhausted, it moves to the next PPC. * If all PPCs are exhausted, it returns null. - * @param {number} idx - after a consumer has consumed a message, it must return the index back to us via this parameter. + * + * @param {PerPartitionMessageCache} ppc - after a consumer has consumed a message, it must return the PPC back to us via this parameter. * otherwise, no messages from that topic partition will be consumed. - * @returns {Object} - the next message in the cache, or null if none exists. An `index` field is added to the message. - * @warning Does not check for global staleness. That is left up to the user. - * Skips locally stale messages. - * The topicPartition, if provided, MUST be one such that the user has fetched - * the message from the same topicPartition earlier. + * @returns {Array} - the next message in the cache, or null if none exists, and the corresponding PPC. * @note Whenever making changes to this function, ensure that you benchmark perf. */ - next(idx = -1) { - let index = idx; - if (index !== -1 && !this.pendingIndices.has(index)) { - /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible - * that we ran out of messages and fetched a new batch. So we just discard what the user is - * returning to us. */ - this.logger.error("Returning unowned index", idx, "to cache. Discarding it."); - index = -1; - } else if (index !== -1) { - this.pendingIndices.delete(index); - /* We don't add the index back to the this.indices here because we're just going to remove it again the - * first thing in the loop below, so it's slightly better to just avoid doing it. */ - } - - if (index === -1) { - if (this.indices.size() === 0 || this.pendingIndices.size === this.maxConcurrency) { - return null; - } - index = this.indices.pop(); // index cannot be undefined here since indices.size > 0 - } + next(ppc = null) { + if (!ppc|| !ppc._assigned) + ppc = this.#assignNewPartition(); + if (!ppc) + return null; - /* This loop will always terminate. Why? - * On each iteration: - * 1. We either return (if next is not null). - * 2. We change the PPC index we're interested in, and there are a finite number of PPCs. - * (PPCs don't repeat within the loop since the indices of the PPC are popped from within the - * heap and not put back in, or else a new index is created bounded by ppcList.length). - */ - while (true) { - const next = this.ppcList[index].next(); - if (this.ppcList[index].isStale() || next === null) { - /* If the current PPC is stale or empty, then we move on to the next one. - * It is equally valid to choose any PPC available within this.indices, or else - * move on to the next PPC (maxIndicesIndex + 1) if available. - * We prefer the second option a bit more since we don't have to do a heap operation. */ - const toAdd = this.maxIndicesIndex + 1; - if (toAdd < this.ppcList.length) { - this.maxIndicesIndex = toAdd; - index = toAdd; - } else if (!this.indices.isEmpty()) { - index = this.indices.pop(); - } else { - break; // nothing left. - } - continue; - } + let next = ppc._next(); - this.pendingIndices.add(index); - next.index = index; - return next; + if (!next) { + this.#removeEmptyPartition(ppc); + return this.next(); } - return null; // Caller is responsible for triggering fetch logic here if next == null. + + this.#size--; + return [next, ppc]; } /** @@ -410,83 +231,33 @@ class MessageCache { * * @sa next, the behaviour is similar in other aspects. */ - nextN(idx = -1, size = 1) { - let index = idx; - if (index !== -1 && !this.pendingIndices.has(index)) { - /* The user is behaving well by returning the index to us, but in the meanwhile, it's possible - * that we ran out of messages and fetched a new batch. So we just discard what the user is - * returning to us. */ - this.logger.error("Returning unowned index", idx, "to cache. Discarding it."); - index = -1; - } else if (index !== -1) { - this.pendingIndices.delete(index); - /* We don't add the index back to the this.indices here because we're just going to remove it again the - * first thing in the loop below, so it's slightly better to just avoid doing it. */ - } - - if (index === -1) { - if (this.indices.size() === 0 || this.pendingIndices.size === this.maxConcurrency) { - return null; - } - index = this.indices.pop(); // index cannot be undefined here since indices.size > 0 - } + nextN(ppc = null, size = -1) { + if (!ppc || !ppc._assigned) + ppc = this.#assignNewPartition(); + if (!ppc) + return null; - /* This loop will always terminate. Why? - * On each iteration: - * 1. We either return (if next is not null). - * 2. We change the PPC index we're interested in, and there are a finite number of PPCs. - * (PPCs don't repeat within the loop since the indices of the PPC are popped from within the - * heap and not put back in, or else a new index is created bounded by ppcList.length). - */ - while (true) { - const next = this.ppcList[index].nextN(size); - if (this.ppcList[index].isStale() || next === null) { - /* If the current PPC is stale or empty, then we move on to the next one. - * It is equally valid to choose any PPC available within this.indices, or else - * move on to the next PPC (maxIndicesIndex + 1) if available. - * We prefer the second option a bit more since we don't have to do a heap operation. */ - const toAdd = this.maxIndicesIndex + 1; - if (toAdd < this.ppcList.length) { - this.maxIndicesIndex = toAdd; - index = toAdd; - } else if (!this.indices.isEmpty()) { - index = this.indices.pop(); - } else { - break; // nothing left. - } - continue; - } + let nextN = ppc._nextN(size); - this.pendingIndices.add(index); - /* Arrays are just objects. Setting a property is odd, but not disallowed. */ - next.index = index; - return next; + if (size === -1 || nextN.length < size) { + this.#removeEmptyPartition(ppc); } - return null; // Caller is responsible for triggering fetch logic here if next == null. + if (!nextN.length) + return this.nextN(null, size); + + this.#size -= nextN.length; + return [nextN, ppc]; } /** * Clears the cache completely. - * This resets it to a base state, and reduces the capacity of the cache back to 1. - * Pre-conditions: none - * Post-conditions: maxSize = 1, all caches are unstale, ppcList is empty, locallyStaleCaches is empty. + * This resets it to a base state. */ clear() { - if (this.pendingSize() > 0) { - this.logger.error(`Cache cannot be cleared with ${this.pendingSize()} pending indices.`); - throw new Error(`Cache cannot be cleared with ${this.pendingSize()} pending indices.`); - } - for (const cache of this.ppcList) { - cache.clear(); + for (const ppc of this.#tpToPpc.values()) { + ppc._assigned = false; } - this.ppcList = []; - this.maxSize = 1; - this.increaseCount = 0; - this.stale = false; - this.cachedTime = hrtime(); - this.locallyStaleCaches = []; - this.indices.clear(); - this.currentIndex = 0; + this.#reinit(); } } diff --git a/lib/kafkajs/_heap.js b/lib/kafkajs/_heap.js deleted file mode 100644 index cd486661..00000000 --- a/lib/kafkajs/_heap.js +++ /dev/null @@ -1,2353 +0,0 @@ -/** -Code from https://github.com/ignlg/heap-js/ | Commit Hash: a3bb403 | dist/heap-js.es5.js - - ----- - - -BSD 3-Clause License - -Copyright (c) 2017, Ignacio Lago -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -* Neither the name of the copyright holder nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE -FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER -CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, -OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -*/ - -var __awaiter = (undefined && undefined.__awaiter) || function (thisArg, _arguments, P, generator) { - function adopt(value) { return value instanceof P ? value : new P(function (resolve) { resolve(value); }); } - return new (P || (P = Promise))(function (resolve, reject) { - function fulfilled(value) { try { step(generator.next(value)); } catch (e) { reject(e); } } - function rejected(value) { try { step(generator["throw"](value)); } catch (e) { reject(e); } } - function step(result) { result.done ? resolve(result.value) : adopt(result.value).then(fulfilled, rejected); } - step((generator = generator.apply(thisArg, _arguments || [])).next()); - }); -}; -var __generator$1 = (undefined && undefined.__generator) || function (thisArg, body) { - var _ = { label: 0, sent: function() { if (t[0] & 1) throw t[1]; return t[1]; }, trys: [], ops: [] }, f, y, t, g; - return g = { next: verb(0), "throw": verb(1), "return": verb(2) }, typeof Symbol === "function" && (g[Symbol.iterator] = function() { return this; }), g; - function verb(n) { return function (v) { return step([n, v]); }; } - function step(op) { - if (f) throw new TypeError("Generator is already executing."); - while (g && (g = 0, op[0] && (_ = 0)), _) try { - if (f = 1, y && (t = op[0] & 2 ? y["return"] : op[0] ? y["throw"] || ((t = y["return"]) && t.call(y), 0) : y.next) && !(t = t.call(y, op[1])).done) return t; - if (y = 0, t) op = [op[0] & 2, t.value]; - switch (op[0]) { - case 0: case 1: t = op; break; - case 4: _.label++; return { value: op[1], done: false }; - case 5: _.label++; y = op[1]; op = [0]; continue; - case 7: op = _.ops.pop(); _.trys.pop(); continue; - default: - if (!(t = _.trys, t = t.length > 0 && t[t.length - 1]) && (op[0] === 6 || op[0] === 2)) { _ = 0; continue; } - if (op[0] === 3 && (!t || (op[1] > t[0] && op[1] < t[3]))) { _.label = op[1]; break; } - if (op[0] === 6 && _.label < t[1]) { _.label = t[1]; t = op; break; } - if (t && _.label < t[2]) { _.label = t[2]; _.ops.push(op); break; } - if (t[2]) _.ops.pop(); - _.trys.pop(); continue; - } - op = body.call(thisArg, _); - } catch (e) { op = [6, e]; y = 0; } finally { f = t = 0; } - if (op[0] & 5) throw op[1]; return { value: op[0] ? op[1] : void 0, done: true }; - } -}; -var __read$1 = (undefined && undefined.__read) || function (o, n) { - var m = typeof Symbol === "function" && o[Symbol.iterator]; - if (!m) return o; - var i = m.call(o), r, ar = [], e; - try { - while ((n === void 0 || n-- > 0) && !(r = i.next()).done) ar.push(r.value); - } - catch (error) { e = { error: error }; } - finally { - try { - if (r && !r.done && (m = i["return"])) m.call(i); - } - finally { if (e) throw e.error; } - } - return ar; -}; -var __spreadArray$1 = (undefined && undefined.__spreadArray) || function (to, from, pack) { - if (pack || arguments.length === 2) for (var i = 0, l = from.length, ar; i < l; i++) { - if (ar || !(i in from)) { - if (!ar) ar = Array.prototype.slice.call(from, 0, i); - ar[i] = from[i]; - } - } - return to.concat(ar || Array.prototype.slice.call(from)); -}; -var __values = (undefined && undefined.__values) || function(o) { - var s = typeof Symbol === "function" && Symbol.iterator, m = s && o[s], i = 0; - if (m) return m.call(o); - if (o && typeof o.length === "number") return { - next: function () { - if (o && i >= o.length) o = void 0; - return { value: o && o[i++], done: !o }; - } - }; - throw new TypeError(s ? "Object is not iterable." : "Symbol.iterator is not defined."); -}; -/** - * Heap - * @type {Class} - */ -var HeapAsync = /** @class */ (function () { - /** - * Heap instance constructor. - * @param {Function} compare Optional comparison function, defaults to Heap.minComparator - */ - function HeapAsync(compare) { - if (compare === void 0) { compare = HeapAsync.minComparator; } - var _this = this; - this.compare = compare; - this.heapArray = []; - this._limit = 0; - /** - * Alias of add - */ - this.offer = this.add; - /** - * Alias of peek - */ - this.element = this.peek; - /** - * Alias of pop - */ - this.poll = this.pop; - /** - * Returns the inverse to the comparison function. - * @return {Number} - */ - this._invertedCompare = function (a, b) { - return _this.compare(a, b).then(function (res) { return -1 * res; }); - }; - } - /* - Static methods - */ - /** - * Gets children indices for given index. - * @param {Number} idx Parent index - * @return {Array(Number)} Array of children indices - */ - HeapAsync.getChildrenIndexOf = function (idx) { - return [idx * 2 + 1, idx * 2 + 2]; - }; - /** - * Gets parent index for given index. - * @param {Number} idx Children index - * @return {Number | undefined} Parent index, -1 if idx is 0 - */ - HeapAsync.getParentIndexOf = function (idx) { - if (idx <= 0) { - return -1; - } - var whichChildren = idx % 2 ? 1 : 2; - return Math.floor((idx - whichChildren) / 2); - }; - /** - * Gets sibling index for given index. - * @param {Number} idx Children index - * @return {Number | undefined} Sibling index, -1 if idx is 0 - */ - HeapAsync.getSiblingIndexOf = function (idx) { - if (idx <= 0) { - return -1; - } - var whichChildren = idx % 2 ? 1 : -1; - return idx + whichChildren; - }; - /** - * Min heap comparison function, default. - * @param {any} a First element - * @param {any} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - HeapAsync.minComparator = function (a, b) { - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - if (a > b) { - return [2 /*return*/, 1]; - } - else if (a < b) { - return [2 /*return*/, -1]; - } - else { - return [2 /*return*/, 0]; - } - }); - }); - }; - /** - * Max heap comparison function. - * @param {any} a First element - * @param {any} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - HeapAsync.maxComparator = function (a, b) { - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - if (b > a) { - return [2 /*return*/, 1]; - } - else if (b < a) { - return [2 /*return*/, -1]; - } - else { - return [2 /*return*/, 0]; - } - }); - }); - }; - /** - * Min number heap comparison function, default. - * @param {Number} a First element - * @param {Number} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - HeapAsync.minComparatorNumber = function (a, b) { - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - return [2 /*return*/, a - b]; - }); - }); - }; - /** - * Max number heap comparison function. - * @param {Number} a First element - * @param {Number} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - HeapAsync.maxComparatorNumber = function (a, b) { - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - return [2 /*return*/, b - a]; - }); - }); - }; - /** - * Default equality function. - * @param {any} a First element - * @param {any} b Second element - * @return {Boolean} True if equal, false otherwise - */ - HeapAsync.defaultIsEqual = function (a, b) { - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - return [2 /*return*/, a === b]; - }); - }); - }; - /** - * Prints a heap. - * @param {HeapAsync} heap Heap to be printed - * @returns {String} - */ - HeapAsync.print = function (heap) { - function deep(i) { - var pi = HeapAsync.getParentIndexOf(i); - return Math.floor(Math.log2(pi + 1)); - } - function repeat(str, times) { - var out = ''; - for (; times > 0; --times) { - out += str; - } - return out; - } - var node = 0; - var lines = []; - var maxLines = deep(heap.length - 1) + 2; - var maxLength = 0; - while (node < heap.length) { - var i = deep(node) + 1; - if (node === 0) { - i = 0; - } - // Text representation - var nodeText = String(heap.get(node)); - if (nodeText.length > maxLength) { - maxLength = nodeText.length; - } - // Add to line - lines[i] = lines[i] || []; - lines[i].push(nodeText); - node += 1; - } - return lines - .map(function (line, i) { - var times = Math.pow(2, maxLines - i) - 1; - return (repeat(' ', Math.floor(times / 2) * maxLength) + - line - .map(function (el) { - // centered - var half = (maxLength - el.length) / 2; - return repeat(' ', Math.ceil(half)) + el + repeat(' ', Math.floor(half)); - }) - .join(repeat(' ', times * maxLength))); - }) - .join('\n'); - }; - /* - Python style - */ - /** - * Converts an array into an array-heap, in place - * @param {Array} arr Array to be modified - * @param {Function} compare Optional compare function - * @return {HeapAsync} For convenience, it returns a Heap instance - */ - HeapAsync.heapify = function (arr, compare) { - return __awaiter(this, void 0, void 0, function () { - var heap; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - heap = new HeapAsync(compare); - heap.heapArray = arr; - return [4 /*yield*/, heap.init()]; - case 1: - _a.sent(); - return [2 /*return*/, heap]; - } - }); - }); - }; - /** - * Extract the peek of an array-heap - * @param {Array} heapArr Array to be modified, should be a heap - * @param {Function} compare Optional compare function - * @return {any} Returns the extracted peek - */ - HeapAsync.heappop = function (heapArr, compare) { - var heap = new HeapAsync(compare); - heap.heapArray = heapArr; - return heap.pop(); - }; - /** - * Pushes a item into an array-heap - * @param {Array} heapArr Array to be modified, should be a heap - * @param {any} item Item to push - * @param {Function} compare Optional compare function - */ - HeapAsync.heappush = function (heapArr, item, compare) { - return __awaiter(this, void 0, void 0, function () { - var heap; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - heap = new HeapAsync(compare); - heap.heapArray = heapArr; - return [4 /*yield*/, heap.push(item)]; - case 1: - _a.sent(); - return [2 /*return*/]; - } - }); - }); - }; - /** - * Push followed by pop, faster - * @param {Array} heapArr Array to be modified, should be a heap - * @param {any} item Item to push - * @param {Function} compare Optional compare function - * @return {any} Returns the extracted peek - */ - HeapAsync.heappushpop = function (heapArr, item, compare) { - var heap = new HeapAsync(compare); - heap.heapArray = heapArr; - return heap.pushpop(item); - }; - /** - * Replace peek with item - * @param {Array} heapArr Array to be modified, should be a heap - * @param {any} item Item as replacement - * @param {Function} compare Optional compare function - * @return {any} Returns the extracted peek - */ - HeapAsync.heapreplace = function (heapArr, item, compare) { - var heap = new HeapAsync(compare); - heap.heapArray = heapArr; - return heap.replace(item); - }; - /** - * Return the `n` most valuable elements of a heap-like Array - * @param {Array} heapArr Array, should be an array-heap - * @param {number} n Max number of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - HeapAsync.heaptop = function (heapArr, n, compare) { - if (n === void 0) { n = 1; } - var heap = new HeapAsync(compare); - heap.heapArray = heapArr; - return heap.top(n); - }; - /** - * Return the `n` least valuable elements of a heap-like Array - * @param {Array} heapArr Array, should be an array-heap - * @param {number} n Max number of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - HeapAsync.heapbottom = function (heapArr, n, compare) { - if (n === void 0) { n = 1; } - var heap = new HeapAsync(compare); - heap.heapArray = heapArr; - return heap.bottom(n); - }; - /** - * Return the `n` most valuable elements of an iterable - * @param {number} n Max number of elements - * @param {Iterable} Iterable Iterable list of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - HeapAsync.nlargest = function (n, iterable, compare) { - return __awaiter(this, void 0, void 0, function () { - var heap; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - heap = new HeapAsync(compare); - heap.heapArray = __spreadArray$1([], __read$1(iterable), false); - return [4 /*yield*/, heap.init()]; - case 1: - _a.sent(); - return [2 /*return*/, heap.top(n)]; - } - }); - }); - }; - /** - * Return the `n` least valuable elements of an iterable - * @param {number} n Max number of elements - * @param {Iterable} Iterable Iterable list of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - HeapAsync.nsmallest = function (n, iterable, compare) { - return __awaiter(this, void 0, void 0, function () { - var heap; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - heap = new HeapAsync(compare); - heap.heapArray = __spreadArray$1([], __read$1(iterable), false); - return [4 /*yield*/, heap.init()]; - case 1: - _a.sent(); - return [2 /*return*/, heap.bottom(n)]; - } - }); - }); - }; - /* - Instance methods - */ - /** - * Adds an element to the heap. Aliases: `offer`. - * Same as: push(element) - * @param {any} element Element to be added - * @return {Boolean} true - */ - HeapAsync.prototype.add = function (element) { - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: return [4 /*yield*/, this._sortNodeUp(this.heapArray.push(element) - 1)]; - case 1: - _a.sent(); - this._applyLimit(); - return [2 /*return*/, true]; - } - }); - }); - }; - /** - * Adds an array of elements to the heap. - * Similar as: push(element, element, ...). - * @param {Array} elements Elements to be added - * @return {Boolean} true - */ - HeapAsync.prototype.addAll = function (elements) { - return __awaiter(this, void 0, void 0, function () { - var i, l; - var _a; - return __generator$1(this, function (_b) { - switch (_b.label) { - case 0: - i = this.length; - (_a = this.heapArray).push.apply(_a, __spreadArray$1([], __read$1(elements), false)); - l = this.length; - _b.label = 1; - case 1: - if (!(i < l)) return [3 /*break*/, 4]; - return [4 /*yield*/, this._sortNodeUp(i)]; - case 2: - _b.sent(); - _b.label = 3; - case 3: - ++i; - return [3 /*break*/, 1]; - case 4: - this._applyLimit(); - return [2 /*return*/, true]; - } - }); - }); - }; - /** - * Return the bottom (lowest value) N elements of the heap. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - HeapAsync.prototype.bottom = function (n) { - if (n === void 0) { n = 1; } - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - if (this.heapArray.length === 0 || n <= 0) { - // Nothing to do - return [2 /*return*/, []]; - } - else if (this.heapArray.length === 1) { - // Just the peek - return [2 /*return*/, [this.heapArray[0]]]; - } - else if (n >= this.heapArray.length) { - // The whole heap - return [2 /*return*/, __spreadArray$1([], __read$1(this.heapArray), false)]; - } - else { - // Some elements - return [2 /*return*/, this._bottomN_push(~~n)]; - } - }); - }); - }; - /** - * Check if the heap is sorted, useful for testing purposes. - * @return {Undefined | Element} Returns an element if something wrong is found, otherwise it's undefined - */ - HeapAsync.prototype.check = function () { - return __awaiter(this, void 0, void 0, function () { - var j, el, children, children_1, children_1_1, ch, e_1_1; - var e_1, _a; - return __generator$1(this, function (_b) { - switch (_b.label) { - case 0: - j = 0; - _b.label = 1; - case 1: - if (!(j < this.heapArray.length)) return [3 /*break*/, 10]; - el = this.heapArray[j]; - children = this.getChildrenOf(j); - _b.label = 2; - case 2: - _b.trys.push([2, 7, 8, 9]); - children_1 = (e_1 = void 0, __values(children)), children_1_1 = children_1.next(); - _b.label = 3; - case 3: - if (!!children_1_1.done) return [3 /*break*/, 6]; - ch = children_1_1.value; - return [4 /*yield*/, this.compare(el, ch)]; - case 4: - if ((_b.sent()) > 0) { - return [2 /*return*/, el]; - } - _b.label = 5; - case 5: - children_1_1 = children_1.next(); - return [3 /*break*/, 3]; - case 6: return [3 /*break*/, 9]; - case 7: - e_1_1 = _b.sent(); - e_1 = { error: e_1_1 }; - return [3 /*break*/, 9]; - case 8: - try { - if (children_1_1 && !children_1_1.done && (_a = children_1.return)) _a.call(children_1); - } - finally { if (e_1) throw e_1.error; } - return [7 /*endfinally*/]; - case 9: - ++j; - return [3 /*break*/, 1]; - case 10: return [2 /*return*/]; - } - }); - }); - }; - /** - * Remove all of the elements from this heap. - */ - HeapAsync.prototype.clear = function () { - this.heapArray = []; - }; - /** - * Clone this heap - * @return {HeapAsync} - */ - HeapAsync.prototype.clone = function () { - var cloned = new HeapAsync(this.comparator()); - cloned.heapArray = this.toArray(); - cloned._limit = this._limit; - return cloned; - }; - /** - * Returns the comparison function. - * @return {Function} - */ - HeapAsync.prototype.comparator = function () { - return this.compare; - }; - /** - * Returns true if this queue contains the specified element. - * @param {any} o Element to be found - * @param {Function} fn Optional comparison function, receives (element, needle) - * @return {Boolean} - */ - HeapAsync.prototype.contains = function (o, fn) { - if (fn === void 0) { fn = HeapAsync.defaultIsEqual; } - return __awaiter(this, void 0, void 0, function () { - var _a, _b, el, e_2_1; - var e_2, _c; - return __generator$1(this, function (_d) { - switch (_d.label) { - case 0: - _d.trys.push([0, 5, 6, 7]); - _a = __values(this.heapArray), _b = _a.next(); - _d.label = 1; - case 1: - if (!!_b.done) return [3 /*break*/, 4]; - el = _b.value; - return [4 /*yield*/, fn(el, o)]; - case 2: - if (_d.sent()) { - return [2 /*return*/, true]; - } - _d.label = 3; - case 3: - _b = _a.next(); - return [3 /*break*/, 1]; - case 4: return [3 /*break*/, 7]; - case 5: - e_2_1 = _d.sent(); - e_2 = { error: e_2_1 }; - return [3 /*break*/, 7]; - case 6: - try { - if (_b && !_b.done && (_c = _a.return)) _c.call(_a); - } - finally { if (e_2) throw e_2.error; } - return [7 /*endfinally*/]; - case 7: return [2 /*return*/, false]; - } - }); - }); - }; - /** - * Initialise a heap, sorting nodes - * @param {Array} array Optional initial state array - */ - HeapAsync.prototype.init = function (array) { - return __awaiter(this, void 0, void 0, function () { - var i; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - if (array) { - this.heapArray = __spreadArray$1([], __read$1(array), false); - } - i = Math.floor(this.heapArray.length); - _a.label = 1; - case 1: - if (!(i >= 0)) return [3 /*break*/, 4]; - return [4 /*yield*/, this._sortNodeDown(i)]; - case 2: - _a.sent(); - _a.label = 3; - case 3: - --i; - return [3 /*break*/, 1]; - case 4: - this._applyLimit(); - return [2 /*return*/]; - } - }); - }); - }; - /** - * Test if the heap has no elements. - * @return {Boolean} True if no elements on the heap - */ - HeapAsync.prototype.isEmpty = function () { - return this.length === 0; - }; - /** - * Get the leafs of the tree (no children nodes) - */ - HeapAsync.prototype.leafs = function () { - if (this.heapArray.length === 0) { - return []; - } - var pi = HeapAsync.getParentIndexOf(this.heapArray.length - 1); - return this.heapArray.slice(pi + 1); - }; - Object.defineProperty(HeapAsync.prototype, "length", { - /** - * Length of the heap. - * @return {Number} - */ - get: function () { - return this.heapArray.length; - }, - enumerable: false, - configurable: true - }); - Object.defineProperty(HeapAsync.prototype, "limit", { - /** - * Get length limit of the heap. - * @return {Number} - */ - get: function () { - return this._limit; - }, - /** - * Set length limit of the heap. - * @return {Number} - */ - set: function (_l) { - this._limit = ~~_l; - this._applyLimit(); - }, - enumerable: false, - configurable: true - }); - /** - * Top node. Aliases: `element`. - * Same as: `top(1)[0]` - * @return {any} Top node - */ - HeapAsync.prototype.peek = function () { - return this.heapArray[0]; - }; - /** - * Extract the top node (root). Aliases: `poll`. - * @return {any} Extracted top node, undefined if empty - */ - HeapAsync.prototype.pop = function () { - return __awaiter(this, void 0, void 0, function () { - var last; - return __generator$1(this, function (_a) { - last = this.heapArray.pop(); - if (this.length > 0 && last !== undefined) { - return [2 /*return*/, this.replace(last)]; - } - return [2 /*return*/, last]; - }); - }); - }; - /** - * Pushes element(s) to the heap. - * @param {...any} elements Elements to insert - * @return {Boolean} True if elements are present - */ - HeapAsync.prototype.push = function () { - var elements = []; - for (var _i = 0; _i < arguments.length; _i++) { - elements[_i] = arguments[_i]; - } - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - if (elements.length < 1) { - return [2 /*return*/, false]; - } - else if (elements.length === 1) { - return [2 /*return*/, this.add(elements[0])]; - } - else { - return [2 /*return*/, this.addAll(elements)]; - } - }); - }); - }; - /** - * Same as push & pop in sequence, but faster - * @param {any} element Element to insert - * @return {any} Extracted top node - */ - HeapAsync.prototype.pushpop = function (element) { - return __awaiter(this, void 0, void 0, function () { - var _a; - return __generator$1(this, function (_b) { - switch (_b.label) { - case 0: return [4 /*yield*/, this.compare(this.heapArray[0], element)]; - case 1: - if (!((_b.sent()) < 0)) return [3 /*break*/, 3]; - _a = __read$1([this.heapArray[0], element], 2), element = _a[0], this.heapArray[0] = _a[1]; - return [4 /*yield*/, this._sortNodeDown(0)]; - case 2: - _b.sent(); - _b.label = 3; - case 3: return [2 /*return*/, element]; - } - }); - }); - }; - /** - * Remove an element from the heap. - * @param {any} o Element to be found - * @param {Function} fn Optional function to compare - * @return {Boolean} True if the heap was modified - */ - HeapAsync.prototype.remove = function (o, fn) { - if (fn === void 0) { fn = HeapAsync.defaultIsEqual; } - return __awaiter(this, void 0, void 0, function () { - var idx, i; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - if (!(this.length > 0)) return [3 /*break*/, 13]; - if (!(o === undefined)) return [3 /*break*/, 2]; - return [4 /*yield*/, this.pop()]; - case 1: - _a.sent(); - return [2 /*return*/, true]; - case 2: - idx = -1; - i = 0; - _a.label = 3; - case 3: - if (!(i < this.heapArray.length)) return [3 /*break*/, 6]; - return [4 /*yield*/, fn(this.heapArray[i], o)]; - case 4: - if (_a.sent()) { - idx = i; - return [3 /*break*/, 6]; - } - _a.label = 5; - case 5: - ++i; - return [3 /*break*/, 3]; - case 6: - if (!(idx >= 0)) return [3 /*break*/, 13]; - if (!(idx === 0)) return [3 /*break*/, 8]; - return [4 /*yield*/, this.pop()]; - case 7: - _a.sent(); - return [3 /*break*/, 12]; - case 8: - if (!(idx === this.length - 1)) return [3 /*break*/, 9]; - this.heapArray.pop(); - return [3 /*break*/, 12]; - case 9: - this.heapArray.splice(idx, 1, this.heapArray.pop()); - return [4 /*yield*/, this._sortNodeUp(idx)]; - case 10: - _a.sent(); - return [4 /*yield*/, this._sortNodeDown(idx)]; - case 11: - _a.sent(); - _a.label = 12; - case 12: return [2 /*return*/, true]; - case 13: return [2 /*return*/, false]; - } - }); - }); - }; - /** - * Pop the current peek value, and add the new item. - * @param {any} element Element to replace peek - * @return {any} Old peek - */ - HeapAsync.prototype.replace = function (element) { - return __awaiter(this, void 0, void 0, function () { - var peek; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - peek = this.heapArray[0]; - this.heapArray[0] = element; - return [4 /*yield*/, this._sortNodeDown(0)]; - case 1: - _a.sent(); - return [2 /*return*/, peek]; - } - }); - }); - }; - /** - * Size of the heap - * @return {Number} - */ - HeapAsync.prototype.size = function () { - return this.length; - }; - /** - * Return the top (highest value) N elements of the heap. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - HeapAsync.prototype.top = function (n) { - if (n === void 0) { n = 1; } - return __awaiter(this, void 0, void 0, function () { - return __generator$1(this, function (_a) { - if (this.heapArray.length === 0 || n <= 0) { - // Nothing to do - return [2 /*return*/, []]; - } - else if (this.heapArray.length === 1 || n === 1) { - // Just the peek - return [2 /*return*/, [this.heapArray[0]]]; - } - else if (n >= this.heapArray.length) { - // The whole peek - return [2 /*return*/, __spreadArray$1([], __read$1(this.heapArray), false)]; - } - else { - // Some elements - return [2 /*return*/, this._topN_push(~~n)]; - } - }); - }); - }; - /** - * Clone the heap's internal array - * @return {Array} - */ - HeapAsync.prototype.toArray = function () { - return __spreadArray$1([], __read$1(this.heapArray), false); - }; - /** - * String output, call to Array.prototype.toString() - * @return {String} - */ - HeapAsync.prototype.toString = function () { - return this.heapArray.toString(); - }; - /** - * Get the element at the given index. - * @param {Number} i Index to get - * @return {any} Element at that index - */ - HeapAsync.prototype.get = function (i) { - return this.heapArray[i]; - }; - /** - * Get the elements of these node's children - * @param {Number} idx Node index - * @return {Array(any)} Children elements - */ - HeapAsync.prototype.getChildrenOf = function (idx) { - var _this = this; - return HeapAsync.getChildrenIndexOf(idx) - .map(function (i) { return _this.heapArray[i]; }) - .filter(function (e) { return e !== undefined; }); - }; - /** - * Get the element of this node's parent - * @param {Number} idx Node index - * @return {any} Parent element - */ - HeapAsync.prototype.getParentOf = function (idx) { - var pi = HeapAsync.getParentIndexOf(idx); - return this.heapArray[pi]; - }; - /** - * Iterator interface - */ - HeapAsync.prototype[Symbol.iterator] = function () { - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - if (!this.length) return [3 /*break*/, 2]; - return [4 /*yield*/, this.pop()]; - case 1: - _a.sent(); - return [3 /*break*/, 0]; - case 2: return [2 /*return*/]; - } - }); - }; - /** - * Returns an iterator. To comply with Java interface. - */ - HeapAsync.prototype.iterator = function () { - return this; - }; - /** - * Limit heap size if needed - */ - HeapAsync.prototype._applyLimit = function () { - if (this._limit && this._limit < this.heapArray.length) { - var rm = this.heapArray.length - this._limit; - // It's much faster than splice - while (rm) { - this.heapArray.pop(); - --rm; - } - } - }; - /** - * Return the bottom (lowest value) N elements of the heap, without corner cases, unsorted - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - HeapAsync.prototype._bottomN_push = function (n) { - return __awaiter(this, void 0, void 0, function () { - var bottomHeap, startAt, parentStartAt, indices, i, arr, i; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - bottomHeap = new HeapAsync(this.compare); - bottomHeap.limit = n; - bottomHeap.heapArray = this.heapArray.slice(-n); - return [4 /*yield*/, bottomHeap.init()]; - case 1: - _a.sent(); - startAt = this.heapArray.length - 1 - n; - parentStartAt = HeapAsync.getParentIndexOf(startAt); - indices = []; - for (i = startAt; i > parentStartAt; --i) { - indices.push(i); - } - arr = this.heapArray; - _a.label = 2; - case 2: - if (!indices.length) return [3 /*break*/, 6]; - i = indices.shift(); - return [4 /*yield*/, this.compare(arr[i], bottomHeap.peek())]; - case 3: - if (!((_a.sent()) > 0)) return [3 /*break*/, 5]; - return [4 /*yield*/, bottomHeap.replace(arr[i])]; - case 4: - _a.sent(); - if (i % 2) { - indices.push(HeapAsync.getParentIndexOf(i)); - } - _a.label = 5; - case 5: return [3 /*break*/, 2]; - case 6: return [2 /*return*/, bottomHeap.toArray()]; - } - }); - }); - }; - /** - * Move a node to a new index, switching places - * @param {Number} j First node index - * @param {Number} k Another node index - */ - HeapAsync.prototype._moveNode = function (j, k) { - var _a; - _a = __read$1([this.heapArray[k], this.heapArray[j]], 2), this.heapArray[j] = _a[0], this.heapArray[k] = _a[1]; - }; - /** - * Move a node down the tree (to the leaves) to find a place where the heap is sorted. - * @param {Number} i Index of the node - */ - HeapAsync.prototype._sortNodeDown = function (i) { - return __awaiter(this, void 0, void 0, function () { - var moveIt, self, getPotentialParent, childrenIdx, bestChildIndex, j, bestChild, _a; - var _this = this; - return __generator$1(this, function (_b) { - switch (_b.label) { - case 0: - moveIt = i < this.heapArray.length - 1; - self = this.heapArray[i]; - getPotentialParent = function (best, j) { return __awaiter(_this, void 0, void 0, function () { - var _a; - return __generator$1(this, function (_b) { - switch (_b.label) { - case 0: - _a = this.heapArray.length > j; - if (!_a) return [3 /*break*/, 2]; - return [4 /*yield*/, this.compare(this.heapArray[j], this.heapArray[best])]; - case 1: - _a = (_b.sent()) < 0; - _b.label = 2; - case 2: - if (_a) { - best = j; - } - return [2 /*return*/, best]; - } - }); - }); }; - _b.label = 1; - case 1: - if (!moveIt) return [3 /*break*/, 8]; - childrenIdx = HeapAsync.getChildrenIndexOf(i); - bestChildIndex = childrenIdx[0]; - j = 1; - _b.label = 2; - case 2: - if (!(j < childrenIdx.length)) return [3 /*break*/, 5]; - return [4 /*yield*/, getPotentialParent(bestChildIndex, childrenIdx[j])]; - case 3: - bestChildIndex = _b.sent(); - _b.label = 4; - case 4: - ++j; - return [3 /*break*/, 2]; - case 5: - bestChild = this.heapArray[bestChildIndex]; - _a = typeof bestChild !== 'undefined'; - if (!_a) return [3 /*break*/, 7]; - return [4 /*yield*/, this.compare(self, bestChild)]; - case 6: - _a = (_b.sent()) > 0; - _b.label = 7; - case 7: - if (_a) { - this._moveNode(i, bestChildIndex); - i = bestChildIndex; - } - else { - moveIt = false; - } - return [3 /*break*/, 1]; - case 8: return [2 /*return*/]; - } - }); - }); - }; - /** - * Move a node up the tree (to the root) to find a place where the heap is sorted. - * @param {Number} i Index of the node - */ - HeapAsync.prototype._sortNodeUp = function (i) { - return __awaiter(this, void 0, void 0, function () { - var moveIt, pi, _a; - return __generator$1(this, function (_b) { - switch (_b.label) { - case 0: - moveIt = i > 0; - _b.label = 1; - case 1: - if (!moveIt) return [3 /*break*/, 4]; - pi = HeapAsync.getParentIndexOf(i); - _a = pi >= 0; - if (!_a) return [3 /*break*/, 3]; - return [4 /*yield*/, this.compare(this.heapArray[pi], this.heapArray[i])]; - case 2: - _a = (_b.sent()) > 0; - _b.label = 3; - case 3: - if (_a) { - this._moveNode(i, pi); - i = pi; - } - else { - moveIt = false; - } - return [3 /*break*/, 1]; - case 4: return [2 /*return*/]; - } - }); - }); - }; - /** - * Return the top (highest value) N elements of the heap, without corner cases, unsorted - * Implementation: push. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - HeapAsync.prototype._topN_push = function (n) { - return __awaiter(this, void 0, void 0, function () { - var topHeap, indices, arr, i; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - topHeap = new HeapAsync(this._invertedCompare); - topHeap.limit = n; - indices = [0]; - arr = this.heapArray; - _a.label = 1; - case 1: - if (!indices.length) return [3 /*break*/, 7]; - i = indices.shift(); - if (!(i < arr.length)) return [3 /*break*/, 6]; - if (!(topHeap.length < n)) return [3 /*break*/, 3]; - return [4 /*yield*/, topHeap.push(arr[i])]; - case 2: - _a.sent(); - indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i)), false)); - return [3 /*break*/, 6]; - case 3: return [4 /*yield*/, this.compare(arr[i], topHeap.peek())]; - case 4: - if (!((_a.sent()) < 0)) return [3 /*break*/, 6]; - return [4 /*yield*/, topHeap.replace(arr[i])]; - case 5: - _a.sent(); - indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i)), false)); - _a.label = 6; - case 6: return [3 /*break*/, 1]; - case 7: return [2 /*return*/, topHeap.toArray()]; - } - }); - }); - }; - /** - * Return the top (highest value) N elements of the heap, without corner cases, unsorted - * Implementation: init + push. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - HeapAsync.prototype._topN_fill = function (n) { - return __awaiter(this, void 0, void 0, function () { - var heapArray, topHeap, branch, indices, i, i; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - heapArray = this.heapArray; - topHeap = new HeapAsync(this._invertedCompare); - topHeap.limit = n; - topHeap.heapArray = heapArray.slice(0, n); - return [4 /*yield*/, topHeap.init()]; - case 1: - _a.sent(); - branch = HeapAsync.getParentIndexOf(n - 1) + 1; - indices = []; - for (i = branch; i < n; ++i) { - indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i).filter(function (l) { return l < heapArray.length; })), false)); - } - if ((n - 1) % 2) { - indices.push(n); - } - _a.label = 2; - case 2: - if (!indices.length) return [3 /*break*/, 6]; - i = indices.shift(); - if (!(i < heapArray.length)) return [3 /*break*/, 5]; - return [4 /*yield*/, this.compare(heapArray[i], topHeap.peek())]; - case 3: - if (!((_a.sent()) < 0)) return [3 /*break*/, 5]; - return [4 /*yield*/, topHeap.replace(heapArray[i])]; - case 4: - _a.sent(); - indices.push.apply(indices, __spreadArray$1([], __read$1(HeapAsync.getChildrenIndexOf(i)), false)); - _a.label = 5; - case 5: return [3 /*break*/, 2]; - case 6: return [2 /*return*/, topHeap.toArray()]; - } - }); - }); - }; - /** - * Return the top (highest value) N elements of the heap, without corner cases, unsorted - * Implementation: heap. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - HeapAsync.prototype._topN_heap = function (n) { - return __awaiter(this, void 0, void 0, function () { - var topHeap, result, i, _a, _b; - return __generator$1(this, function (_c) { - switch (_c.label) { - case 0: - topHeap = this.clone(); - result = []; - i = 0; - _c.label = 1; - case 1: - if (!(i < n)) return [3 /*break*/, 4]; - _b = (_a = result).push; - return [4 /*yield*/, topHeap.pop()]; - case 2: - _b.apply(_a, [(_c.sent())]); - _c.label = 3; - case 3: - ++i; - return [3 /*break*/, 1]; - case 4: return [2 /*return*/, result]; - } - }); - }); - }; - /** - * Return index of the top element - * @param list - */ - HeapAsync.prototype._topIdxOf = function (list) { - return __awaiter(this, void 0, void 0, function () { - var idx, top, i, comp; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - if (!list.length) { - return [2 /*return*/, -1]; - } - idx = 0; - top = list[idx]; - i = 1; - _a.label = 1; - case 1: - if (!(i < list.length)) return [3 /*break*/, 4]; - return [4 /*yield*/, this.compare(list[i], top)]; - case 2: - comp = _a.sent(); - if (comp < 0) { - idx = i; - top = list[i]; - } - _a.label = 3; - case 3: - ++i; - return [3 /*break*/, 1]; - case 4: return [2 /*return*/, idx]; - } - }); - }); - }; - /** - * Return the top element - * @param list - */ - HeapAsync.prototype._topOf = function () { - var list = []; - for (var _i = 0; _i < arguments.length; _i++) { - list[_i] = arguments[_i]; - } - return __awaiter(this, void 0, void 0, function () { - var heap; - return __generator$1(this, function (_a) { - switch (_a.label) { - case 0: - heap = new HeapAsync(this.compare); - return [4 /*yield*/, heap.init(list)]; - case 1: - _a.sent(); - return [2 /*return*/, heap.peek()]; - } - }); - }); - }; - return HeapAsync; -}()); - -var __generator = (undefined && undefined.__generator) || function (thisArg, body) { - var _ = { label: 0, sent: function() { if (t[0] & 1) throw t[1]; return t[1]; }, trys: [], ops: [] }, f, y, t, g; - return g = { next: verb(0), "throw": verb(1), "return": verb(2) }, typeof Symbol === "function" && (g[Symbol.iterator] = function() { return this; }), g; - function verb(n) { return function (v) { return step([n, v]); }; } - function step(op) { - if (f) throw new TypeError("Generator is already executing."); - while (g && (g = 0, op[0] && (_ = 0)), _) try { - if (f = 1, y && (t = op[0] & 2 ? y["return"] : op[0] ? y["throw"] || ((t = y["return"]) && t.call(y), 0) : y.next) && !(t = t.call(y, op[1])).done) return t; - if (y = 0, t) op = [op[0] & 2, t.value]; - switch (op[0]) { - case 0: case 1: t = op; break; - case 4: _.label++; return { value: op[1], done: false }; - case 5: _.label++; y = op[1]; op = [0]; continue; - case 7: op = _.ops.pop(); _.trys.pop(); continue; - default: - if (!(t = _.trys, t = t.length > 0 && t[t.length - 1]) && (op[0] === 6 || op[0] === 2)) { _ = 0; continue; } - if (op[0] === 3 && (!t || (op[1] > t[0] && op[1] < t[3]))) { _.label = op[1]; break; } - if (op[0] === 6 && _.label < t[1]) { _.label = t[1]; t = op; break; } - if (t && _.label < t[2]) { _.label = t[2]; _.ops.push(op); break; } - if (t[2]) _.ops.pop(); - _.trys.pop(); continue; - } - op = body.call(thisArg, _); - } catch (e) { op = [6, e]; y = 0; } finally { f = t = 0; } - if (op[0] & 5) throw op[1]; return { value: op[0] ? op[1] : void 0, done: true }; - } -}; -var __read = (undefined && undefined.__read) || function (o, n) { - var m = typeof Symbol === "function" && o[Symbol.iterator]; - if (!m) return o; - var i = m.call(o), r, ar = [], e; - try { - while ((n === void 0 || n-- > 0) && !(r = i.next()).done) ar.push(r.value); - } - catch (error) { e = { error: error }; } - finally { - try { - if (r && !r.done && (m = i["return"])) m.call(i); - } - finally { if (e) throw e.error; } - } - return ar; -}; -var __spreadArray = (undefined && undefined.__spreadArray) || function (to, from, pack) { - if (pack || arguments.length === 2) for (var i = 0, l = from.length, ar; i < l; i++) { - if (ar || !(i in from)) { - if (!ar) ar = Array.prototype.slice.call(from, 0, i); - ar[i] = from[i]; - } - } - return to.concat(ar || Array.prototype.slice.call(from)); -}; -var toInt = function (n) { return ~~n; }; -/** - * Heap - * @type {Class} - */ -var Heap = /** @class */ (function () { - /** - * Heap instance constructor. - * @param {Function} compare Optional comparison function, defaults to Heap.minComparator - */ - function Heap(compare) { - if (compare === void 0) { compare = Heap.minComparator; } - var _this = this; - this.compare = compare; - this.heapArray = []; - this._limit = 0; - /** - * Alias of {@link add} - * @see add - */ - this.offer = this.add; - /** - * Alias of {@link peek} - * @see peek - */ - this.element = this.peek; - /** - * Alias of {@link pop} - * @see pop - */ - this.poll = this.pop; - /** - * Alias of {@link clear} - * @see clear - */ - this.removeAll = this.clear; - /** - * Returns the inverse to the comparison function. - * @return {Function} - */ - this._invertedCompare = function (a, b) { - return -1 * _this.compare(a, b); - }; - } - /* - Static methods - */ - /** - * Gets children indices for given index. - * @param {Number} idx Parent index - * @return {Array(Number)} Array of children indices - */ - Heap.getChildrenIndexOf = function (idx) { - return [idx * 2 + 1, idx * 2 + 2]; - }; - /** - * Gets parent index for given index. - * @param {Number} idx Children index - * @return {Number | undefined} Parent index, -1 if idx is 0 - */ - Heap.getParentIndexOf = function (idx) { - if (idx <= 0) { - return -1; - } - var whichChildren = idx % 2 ? 1 : 2; - return Math.floor((idx - whichChildren) / 2); - }; - /** - * Gets sibling index for given index. - * @param {Number} idx Children index - * @return {Number | undefined} Sibling index, -1 if idx is 0 - */ - Heap.getSiblingIndexOf = function (idx) { - if (idx <= 0) { - return -1; - } - var whichChildren = idx % 2 ? 1 : -1; - return idx + whichChildren; - }; - /** - * Min heap comparison function, default. - * @param {any} a First element - * @param {any} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - Heap.minComparator = function (a, b) { - if (a > b) { - return 1; - } - else if (a < b) { - return -1; - } - else { - return 0; - } - }; - /** - * Max heap comparison function. - * @param {any} a First element - * @param {any} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - Heap.maxComparator = function (a, b) { - if (b > a) { - return 1; - } - else if (b < a) { - return -1; - } - else { - return 0; - } - }; - /** - * Min number heap comparison function, default. - * @param {Number} a First element - * @param {Number} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - Heap.minComparatorNumber = function (a, b) { - return a - b; - }; - /** - * Max number heap comparison function. - * @param {Number} a First element - * @param {Number} b Second element - * @return {Number} 0 if they're equal, positive if `a` goes up, negative if `b` goes up - */ - Heap.maxComparatorNumber = function (a, b) { - return b - a; - }; - /** - * Default equality function. - * @param {any} a First element - * @param {any} b Second element - * @return {Boolean} True if equal, false otherwise - */ - Heap.defaultIsEqual = function (a, b) { - return a === b; - }; - /** - * Prints a heap. - * @param {Heap} heap Heap to be printed - * @returns {String} - */ - Heap.print = function (heap) { - function deep(i) { - var pi = Heap.getParentIndexOf(i); - return Math.floor(Math.log2(pi + 1)); - } - function repeat(str, times) { - var out = ''; - for (; times > 0; --times) { - out += str; - } - return out; - } - var node = 0; - var lines = []; - var maxLines = deep(heap.length - 1) + 2; - var maxLength = 0; - while (node < heap.length) { - var i = deep(node) + 1; - if (node === 0) { - i = 0; - } - // Text representation - var nodeText = String(heap.get(node)); - if (nodeText.length > maxLength) { - maxLength = nodeText.length; - } - // Add to line - lines[i] = lines[i] || []; - lines[i].push(nodeText); - node += 1; - } - return lines - .map(function (line, i) { - var times = Math.pow(2, maxLines - i) - 1; - return (repeat(' ', Math.floor(times / 2) * maxLength) + - line - .map(function (el) { - // centered - var half = (maxLength - el.length) / 2; - return repeat(' ', Math.ceil(half)) + el + repeat(' ', Math.floor(half)); - }) - .join(repeat(' ', times * maxLength))); - }) - .join('\n'); - }; - /* - Python style - */ - /** - * Converts an array into an array-heap, in place - * @param {Array} arr Array to be modified - * @param {Function} compare Optional compare function - * @return {Heap} For convenience, it returns a Heap instance - */ - Heap.heapify = function (arr, compare) { - var heap = new Heap(compare); - heap.heapArray = arr; - heap.init(); - return heap; - }; - /** - * Extract the peek of an array-heap - * @param {Array} heapArr Array to be modified, should be a heap - * @param {Function} compare Optional compare function - * @return {any} Returns the extracted peek - */ - Heap.heappop = function (heapArr, compare) { - var heap = new Heap(compare); - heap.heapArray = heapArr; - return heap.pop(); - }; - /** - * Pushes a item into an array-heap - * @param {Array} heapArr Array to be modified, should be a heap - * @param {any} item Item to push - * @param {Function} compare Optional compare function - */ - Heap.heappush = function (heapArr, item, compare) { - var heap = new Heap(compare); - heap.heapArray = heapArr; - heap.push(item); - }; - /** - * Push followed by pop, faster - * @param {Array} heapArr Array to be modified, should be a heap - * @param {any} item Item to push - * @param {Function} compare Optional compare function - * @return {any} Returns the extracted peek - */ - Heap.heappushpop = function (heapArr, item, compare) { - var heap = new Heap(compare); - heap.heapArray = heapArr; - return heap.pushpop(item); - }; - /** - * Replace peek with item - * @param {Array} heapArr Array to be modified, should be a heap - * @param {any} item Item as replacement - * @param {Function} compare Optional compare function - * @return {any} Returns the extracted peek - */ - Heap.heapreplace = function (heapArr, item, compare) { - var heap = new Heap(compare); - heap.heapArray = heapArr; - return heap.replace(item); - }; - /** - * Return the `n` most valuable elements of a heap-like Array - * @param {Array} heapArr Array, should be an array-heap - * @param {number} n Max number of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - Heap.heaptop = function (heapArr, n, compare) { - if (n === void 0) { n = 1; } - var heap = new Heap(compare); - heap.heapArray = heapArr; - return heap.top(n); - }; - /** - * Return the `n` least valuable elements of a heap-like Array - * @param {Array} heapArr Array, should be an array-heap - * @param {number} n Max number of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - Heap.heapbottom = function (heapArr, n, compare) { - if (n === void 0) { n = 1; } - var heap = new Heap(compare); - heap.heapArray = heapArr; - return heap.bottom(n); - }; - /** - * Return the `n` most valuable elements of an iterable - * @param {number} n Max number of elements - * @param {Iterable} Iterable Iterable list of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - Heap.nlargest = function (n, iterable, compare) { - var heap = new Heap(compare); - heap.heapArray = __spreadArray([], __read(iterable), false); - heap.init(); - return heap.top(n); - }; - /** - * Return the `n` least valuable elements of an iterable - * @param {number} n Max number of elements - * @param {Iterable} Iterable Iterable list of elements - * @param {Function} compare Optional compare function - * @return {any} Elements - */ - Heap.nsmallest = function (n, iterable, compare) { - var heap = new Heap(compare); - heap.heapArray = __spreadArray([], __read(iterable), false); - heap.init(); - return heap.bottom(n); - }; - /* - Instance methods - */ - /** - * Adds an element to the heap. Aliases: {@link offer}. - * Same as: {@link push}(element). - * @param {any} element Element to be added - * @return {Boolean} true - */ - Heap.prototype.add = function (element) { - this._sortNodeUp(this.heapArray.push(element) - 1); - this._applyLimit(); - return true; - }; - /** - * Adds an array of elements to the heap. - * Similar as: {@link push}(element, element, ...). - * @param {Array} elements Elements to be added - * @return {Boolean} true - */ - Heap.prototype.addAll = function (elements) { - var _a; - var i = this.length; - (_a = this.heapArray).push.apply(_a, __spreadArray([], __read(elements), false)); - for (var l = this.length; i < l; ++i) { - this._sortNodeUp(i); - } - this._applyLimit(); - return true; - }; - /** - * Return the bottom (lowest value) N elements of the heap. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - Heap.prototype.bottom = function (n) { - if (n === void 0) { n = 1; } - if (this.heapArray.length === 0 || n <= 0) { - // Nothing to do - return []; - } - else if (this.heapArray.length === 1) { - // Just the peek - return [this.heapArray[0]]; - } - else if (n >= this.heapArray.length) { - // The whole heap - return __spreadArray([], __read(this.heapArray), false); - } - else { - // Some elements - return this._bottomN_push(~~n); - } - }; - /** - * Check if the heap is sorted, useful for testing purposes. - * @return {Undefined | Element} Returns an element if something wrong is found, otherwise it's undefined - */ - Heap.prototype.check = function () { - var _this = this; - return this.heapArray.find(function (el, j) { return !!_this.getChildrenOf(j).find(function (ch) { return _this.compare(el, ch) > 0; }); }); - }; - /** - * Remove all of the elements from this heap. - */ - Heap.prototype.clear = function () { - this.heapArray = []; - }; - /** - * Clone this heap - * @return {Heap} - */ - Heap.prototype.clone = function () { - var cloned = new Heap(this.comparator()); - cloned.heapArray = this.toArray(); - cloned._limit = this._limit; - return cloned; - }; - /** - * Returns the comparison function. - * @return {Function} - */ - Heap.prototype.comparator = function () { - return this.compare; - }; - /** - * Returns true if this queue contains the specified element. - * @param {any} o Element to be found - * @param {Function} callbackFn Optional comparison function, receives (element, needle) - * @return {Boolean} - */ - Heap.prototype.contains = function (o, callbackFn) { - if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } - return this.indexOf(o, callbackFn) !== -1; - }; - /** - * Initialize a heap, sorting nodes - * @param {Array} array Optional initial state array - */ - Heap.prototype.init = function (array) { - if (array) { - this.heapArray = __spreadArray([], __read(array), false); - } - for (var i = Math.floor(this.heapArray.length); i >= 0; --i) { - this._sortNodeDown(i); - } - this._applyLimit(); - }; - /** - * Test if the heap has no elements. - * @return {Boolean} True if no elements on the heap - */ - Heap.prototype.isEmpty = function () { - return this.length === 0; - }; - /** - * Get the index of the first occurrence of the element in the heap (using the comparator). - * @param {any} element Element to be found - * @param {Function} callbackFn Optional comparison function, receives (element, needle) - * @return {Number} Index or -1 if not found - */ - Heap.prototype.indexOf = function (element, callbackFn) { - if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } - if (this.heapArray.length === 0) { - return -1; - } - var indexes = []; - var currentIndex = 0; - while (currentIndex < this.heapArray.length) { - var currentElement = this.heapArray[currentIndex]; - if (callbackFn(currentElement, element)) { - return currentIndex; - } - else if (this.compare(currentElement, element) <= 0) { - indexes.push.apply(indexes, __spreadArray([], __read(Heap.getChildrenIndexOf(currentIndex)), false)); - } - currentIndex = indexes.shift() || this.heapArray.length; - } - return -1; - }; - /** - * Get the indexes of the every occurrence of the element in the heap (using the comparator). - * @param {any} element Element to be found - * @param {Function} callbackFn Optional comparison function, receives (element, needle) - * @return {Array} Array of indexes or empty array if not found - */ - Heap.prototype.indexOfEvery = function (element, callbackFn) { - if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } - if (this.heapArray.length === 0) { - return []; - } - var indexes = []; - var foundIndexes = []; - var currentIndex = 0; - while (currentIndex < this.heapArray.length) { - var currentElement = this.heapArray[currentIndex]; - if (callbackFn(currentElement, element)) { - foundIndexes.push(currentIndex); - indexes.push.apply(indexes, __spreadArray([], __read(Heap.getChildrenIndexOf(currentIndex)), false)); - } - else if (this.compare(currentElement, element) <= 0) { - indexes.push.apply(indexes, __spreadArray([], __read(Heap.getChildrenIndexOf(currentIndex)), false)); - } - currentIndex = indexes.shift() || this.heapArray.length; - } - return foundIndexes; - }; - /** - * Get the leafs of the tree (no children nodes). - * See also: {@link getChildrenOf} and {@link bottom}. - * @return {Array} - * @see getChildrenOf - * @see bottom - */ - Heap.prototype.leafs = function () { - if (this.heapArray.length === 0) { - return []; - } - var pi = Heap.getParentIndexOf(this.heapArray.length - 1); - return this.heapArray.slice(pi + 1); - }; - Object.defineProperty(Heap.prototype, "length", { - /** - * Length of the heap. Aliases: {@link size}. - * @return {Number} - * @see size - */ - get: function () { - return this.heapArray.length; - }, - enumerable: false, - configurable: true - }); - Object.defineProperty(Heap.prototype, "limit", { - /** - * Get length limit of the heap. - * Use {@link setLimit} or {@link limit} to set the limit. - * @return {Number} - * @see setLimit - */ - get: function () { - return this._limit; - }, - /** - * Set length limit of the heap. Same as using {@link setLimit}. - * @description If the heap is longer than the limit, the needed amount of leafs are removed. - * @param {Number} _l Limit, defaults to 0 (no limit). Negative, Infinity, or NaN values set the limit to 0. - * @see setLimit - */ - set: function (_l) { - if (_l < 0 || isNaN(_l)) { - // NaN, negative, and Infinity are treated as 0 - this._limit = 0; - } - else { - // truncating a floating-point number to an integer - this._limit = ~~_l; - } - this._applyLimit(); - }, - enumerable: false, - configurable: true - }); - /** - * Set length limit of the heap. - * Same as assigning to {@link limit} but returns NaN if the value was invalid. - * @param {Number} _l Limit. Negative, Infinity, or NaN values set the limit to 0. - * @return {Number} The limit or NaN if the value was negative, or NaN. - * @see limit - */ - Heap.prototype.setLimit = function (_l) { - this.limit = _l; - if (_l < 0 || isNaN(_l)) { - return NaN; - } - else { - return this._limit; - } - }; - /** - * Top node. Aliases: {@link element}. - * Same as: {@link top}(1)[0]. - * @return {any} Top node - * @see top - */ - Heap.prototype.peek = function () { - return this.heapArray[0]; - }; - /** - * Extract the top node (root). Aliases: {@link poll}. - * @return {any} Extracted top node, undefined if empty - */ - Heap.prototype.pop = function () { - var last = this.heapArray.pop(); - if (this.length > 0 && last !== undefined) { - return this.replace(last); - } - return last; - }; - /** - * Pushes element(s) to the heap. - * See also: {@link add} and {@link addAll}. - * @param {...any} elements Elements to insert - * @return {Boolean} True if elements are present - */ - Heap.prototype.push = function () { - var elements = []; - for (var _i = 0; _i < arguments.length; _i++) { - elements[_i] = arguments[_i]; - } - if (elements.length < 1) { - return false; - } - else if (elements.length === 1) { - return this.add(elements[0]); - } - else { - return this.addAll(elements); - } - }; - /** - * Same as push & pop in sequence, but faster - * @param {any} element Element to insert - * @return {any} Extracted top node - */ - Heap.prototype.pushpop = function (element) { - var _a; - if (this.compare(this.heapArray[0], element) < 0) { - _a = __read([this.heapArray[0], element], 2), element = _a[0], this.heapArray[0] = _a[1]; - this._sortNodeDown(0); - } - return element; - }; - /** - * Remove the first occurrence of an element from the heap. - * @param {any} o Element to be found - * @param {Function} callbackFn Optional equality function, receives (element, needle) - * @return {Boolean} True if the heap was modified - */ - Heap.prototype.remove = function (o, callbackFn) { - if (callbackFn === void 0) { callbackFn = Heap.defaultIsEqual; } - if (this.length > 0) { - if (o === undefined) { - this.pop(); - return true; - } - else { - var idx = this.indexOf(o, callbackFn); - if (idx >= 0) { - if (idx === 0) { - this.pop(); - } - else if (idx === this.length - 1) { - this.heapArray.pop(); - } - else { - this.heapArray.splice(idx, 1, this.heapArray.pop()); - this._sortNodeUp(idx); - this._sortNodeDown(idx); - } - return true; - } - } - } - return false; - }; - /** - * Pop the current peek value, and add the new item. - * @param {any} element Element to replace peek - * @return {any} Old peek - */ - Heap.prototype.replace = function (element) { - var peek = this.heapArray[0]; - this.heapArray[0] = element; - this._sortNodeDown(0); - return peek; - }; - /** - * Size of the heap - * @return {Number} - */ - Heap.prototype.size = function () { - return this.length; - }; - /** - * Return the top (highest value) N elements of the heap. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - Heap.prototype.top = function (n) { - if (n === void 0) { n = 1; } - if (this.heapArray.length === 0 || n <= 0) { - // Nothing to do - return []; - } - else if (this.heapArray.length === 1 || n === 1) { - // Just the peek - return [this.heapArray[0]]; - } - else if (n >= this.heapArray.length) { - // The whole peek - return __spreadArray([], __read(this.heapArray), false); - } - else { - // Some elements - return this._topN_push(~~n); - } - }; - /** - * Clone the heap's internal array - * @return {Array} - */ - Heap.prototype.toArray = function () { - return __spreadArray([], __read(this.heapArray), false); - }; - /** - * String output, call to Array.prototype.toString() - * @return {String} - */ - Heap.prototype.toString = function () { - return this.heapArray.toString(); - }; - /** - * Get the element at the given index. - * @param {Number} i Index to get - * @return {any} Element at that index - */ - Heap.prototype.get = function (i) { - return this.heapArray[i]; - }; - /** - * Get the elements of these node's children - * @param {Number} idx Node index - * @return {Array(any)} Children elements - */ - Heap.prototype.getChildrenOf = function (idx) { - var _this = this; - return Heap.getChildrenIndexOf(idx) - .map(function (i) { return _this.heapArray[i]; }) - .filter(function (e) { return e !== undefined; }); - }; - /** - * Get the element of this node's parent - * @param {Number} idx Node index - * @return {any} Parent element - */ - Heap.prototype.getParentOf = function (idx) { - var pi = Heap.getParentIndexOf(idx); - return this.heapArray[pi]; - }; - /** - * Iterator interface - */ - Heap.prototype[Symbol.iterator] = function () { - return __generator(this, function (_a) { - switch (_a.label) { - case 0: - if (!this.length) return [3 /*break*/, 2]; - return [4 /*yield*/, this.pop()]; - case 1: - _a.sent(); - return [3 /*break*/, 0]; - case 2: return [2 /*return*/]; - } - }); - }; - /** - * Returns an iterator. To comply with Java interface. - */ - Heap.prototype.iterator = function () { - return this.toArray(); - }; - /** - * Limit heap size if needed - */ - Heap.prototype._applyLimit = function () { - if (this._limit > 0 && this._limit < this.heapArray.length) { - var rm = this.heapArray.length - this._limit; - // It's much faster than splice - while (rm) { - this.heapArray.pop(); - --rm; - } - } - }; - /** - * Return the bottom (lowest value) N elements of the heap, without corner cases, unsorted - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - Heap.prototype._bottomN_push = function (n) { - // Use an inverted heap - var bottomHeap = new Heap(this.compare); - bottomHeap.limit = n; - bottomHeap.heapArray = this.heapArray.slice(-n); - bottomHeap.init(); - var startAt = this.heapArray.length - 1 - n; - var parentStartAt = Heap.getParentIndexOf(startAt); - var indices = []; - for (var i = startAt; i > parentStartAt; --i) { - indices.push(i); - } - var arr = this.heapArray; - while (indices.length) { - var i = indices.shift(); - if (this.compare(arr[i], bottomHeap.peek()) > 0) { - bottomHeap.replace(arr[i]); - if (i % 2) { - indices.push(Heap.getParentIndexOf(i)); - } - } - } - return bottomHeap.toArray(); - }; - /** - * Move a node to a new index, switching places - * @param {Number} j First node index - * @param {Number} k Another node index - */ - Heap.prototype._moveNode = function (j, k) { - var _a; - _a = __read([this.heapArray[k], this.heapArray[j]], 2), this.heapArray[j] = _a[0], this.heapArray[k] = _a[1]; - }; - /** - * Move a node down the tree (to the leaves) to find a place where the heap is sorted. - * @param {Number} i Index of the node - */ - Heap.prototype._sortNodeDown = function (i) { - var _this = this; - var moveIt = i < this.heapArray.length - 1; - var self = this.heapArray[i]; - var getPotentialParent = function (best, j) { - if (_this.heapArray.length > j && _this.compare(_this.heapArray[j], _this.heapArray[best]) < 0) { - best = j; - } - return best; - }; - while (moveIt) { - var childrenIdx = Heap.getChildrenIndexOf(i); - var bestChildIndex = childrenIdx.reduce(getPotentialParent, childrenIdx[0]); - var bestChild = this.heapArray[bestChildIndex]; - if (typeof bestChild !== 'undefined' && this.compare(self, bestChild) > 0) { - this._moveNode(i, bestChildIndex); - i = bestChildIndex; - } - else { - moveIt = false; - } - } - }; - /** - * Move a node up the tree (to the root) to find a place where the heap is sorted. - * @param {Number} i Index of the node - */ - Heap.prototype._sortNodeUp = function (i) { - var moveIt = i > 0; - while (moveIt) { - var pi = Heap.getParentIndexOf(i); - if (pi >= 0 && this.compare(this.heapArray[pi], this.heapArray[i]) > 0) { - this._moveNode(i, pi); - i = pi; - } - else { - moveIt = false; - } - } - }; - /** - * Return the top (highest value) N elements of the heap, without corner cases, unsorted - * Implementation: push. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - Heap.prototype._topN_push = function (n) { - // Use an inverted heap - var topHeap = new Heap(this._invertedCompare); - topHeap.limit = n; - var indices = [0]; - var arr = this.heapArray; - while (indices.length) { - var i = indices.shift(); - if (i < arr.length) { - if (topHeap.length < n) { - topHeap.push(arr[i]); - indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i)), false)); - } - else if (this.compare(arr[i], topHeap.peek()) < 0) { - topHeap.replace(arr[i]); - indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i)), false)); - } - } - } - return topHeap.toArray(); - }; - /** - * Return the top (highest value) N elements of the heap, without corner cases, unsorted - * Implementation: init + push. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - Heap.prototype._topN_fill = function (n) { - // Use an inverted heap - var heapArray = this.heapArray; - var topHeap = new Heap(this._invertedCompare); - topHeap.limit = n; - topHeap.heapArray = heapArray.slice(0, n); - topHeap.init(); - var branch = Heap.getParentIndexOf(n - 1) + 1; - var indices = []; - for (var i = branch; i < n; ++i) { - indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i).filter(function (l) { return l < heapArray.length; })), false)); - } - if ((n - 1) % 2) { - indices.push(n); - } - while (indices.length) { - var i = indices.shift(); - if (i < heapArray.length) { - if (this.compare(heapArray[i], topHeap.peek()) < 0) { - topHeap.replace(heapArray[i]); - indices.push.apply(indices, __spreadArray([], __read(Heap.getChildrenIndexOf(i)), false)); - } - } - } - return topHeap.toArray(); - }; - /** - * Return the top (highest value) N elements of the heap, without corner cases, unsorted - * Implementation: heap. - * - * @param {Number} n Number of elements. - * @return {Array} Array of length <= N. - */ - Heap.prototype._topN_heap = function (n) { - var topHeap = this.clone(); - var result = []; - for (var i = 0; i < n; ++i) { - result.push(topHeap.pop()); - } - return result; - }; - /** - * Return index of the top element - * @param list - */ - Heap.prototype._topIdxOf = function (list) { - if (!list.length) { - return -1; - } - var idx = 0; - var top = list[idx]; - for (var i = 1; i < list.length; ++i) { - var comp = this.compare(list[i], top); - if (comp < 0) { - idx = i; - top = list[i]; - } - } - return idx; - }; - /** - * Return the top element - * @param list - */ - Heap.prototype._topOf = function () { - var list = []; - for (var _i = 0; _i < arguments.length; _i++) { - list[_i] = arguments[_i]; - } - var heap = new Heap(this.compare); - heap.init(list); - return heap.peek(); - }; - return Heap; -}()); - -module.exports = { Heap, HeapAsync, Heap, toInt }; diff --git a/lib/kafkajs/_linked-list.js b/lib/kafkajs/_linked-list.js new file mode 100644 index 00000000..79d48171 --- /dev/null +++ b/lib/kafkajs/_linked-list.js @@ -0,0 +1,219 @@ +/** + * Node class for linked list, after being removed + * it cannot be used again. + */ +class LinkedListNode { + // Value contained by the node. + #value; + // Node was removed from the list. + _removed = false; + // Next node in the list. + _prev = null; + // Previous node in the list. + _next = null; + + constructor(value) { + this.#value = value; + } + + get value() { + return this.#value; + } + + get prev() { + return this._prev; + } + + get next() { + return this._next; + } +} + +class LinkedList { + _head = null; + _tail = null; + #count = 0; + + *#iterator() { + let node = this._head; + while (node) { + yield node.value; + node = node._next; + } + } + + #insertInBetween(node, prev, next) { + node._next = next; + node._prev = prev; + if (prev) + prev._next = node; + else + this._head = node; + + if (next) + next._prev = node; + else + this._tail = node; + + this.#count++; + return node; + } + + /** + * Removes given node from the list, + * if it is not already removed. + * + * @param {LinkedListNode} node + */ + remove(node) { + if (node._removed) { + return; + } + + if (node._prev) + node._prev._next = node._next; + else + this._head = node._next; + + if (node._next) + node._next._prev = node._prev; + else + this._tail = node._prev; + + node._next = null; + node._prev = null; + node._removed = true; + this.#count--; + } + + /** + * Removes the first node from the list and returns it, + * or null if the list is empty. + * + * @returns {any} The value of the first node in the list or null. + */ + removeFirst() { + if (this._head === null) { + return null; + } + + const node = this._head; + this.remove(node); + return node.value; + } + + /** + * Removes the last node from the list and returns its value, + * or null if the list is empty. + * + * @returns {any} The value of the last node in the list or null. + */ + removeLast() { + if (this._tail === null) { + return null; + } + + const node = this._tail; + this.remove(node); + return node.value; + } + + /** + * Add a new node to the beginning of the list and returns it. + * + * @param {any} value + * @returns {LinkedListNode} The new node. + */ + addFirst(value) { + const node = new LinkedListNode(value); + return this.#insertInBetween(node, null, + this._head); + } + + /** + * Add a new node to the end of the list and returns it. + * + * @param {any} value Node value. + * @returns {LinkedListNode} The new node. + */ + addLast(value) { + const node = new LinkedListNode(value); + return this.#insertInBetween(node, this._tail, null); + } + + /** + * Add a new node before the given node and returns it. + * Given node must not be removed. + * + * @param {LinkedListNode} node Reference node. + * @param {any} value New node value. + * @returns {LinkedListNode} The new node. + */ + addBefore(node, value) { + if (node._removed) + throw new Error('Node was removed'); + const newNode = new LinkedListNode(value); + return this.#insertInBetween(newNode, node._prev, node); + } + + /** + * Add a new node after the given node and returns it. + * Given node must not be removed. + * + * @param {LinkedListNode} node Reference node. + * @param {any} value New node value. + * @returns {LinkedListNode} The new node. + */ + addAfter(node, value) { + if (node._removed) + throw new Error('Node was removed'); + const newNode = new LinkedListNode(value); + return this.#insertInBetween(newNode, node, node._next); + } + + /** + * Concatenates the given list to the end of this list. + * + * @param {LinkedList} list List to concatenate. + */ + concat(list) { + if (list.length === 0) { + return; + } + + if (this._tail) { + this._tail._next = list._head; + } + + if (list._head) { + list._head._prev = this._tail; + } + + this._tail = list._tail; + this.#count += list.length; + list.#count = 0; + list._head = null; + list._tail = null; + } + + get first() { + return this._head; + } + + get last() { + return this._tail; + } + + get length() { + return this.#count; + } + + [Symbol.iterator]() { + return this.#iterator(); + } +} + +module.exports = { + LinkedList, + LinkedListNode +}; diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index d93a49c4..36789a25 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -488,6 +488,11 @@ Baton KafkaConsumer::RefreshAssignments() { } } +Baton KafkaConsumer::AssignmentLost() { + bool lost = m_consumer->assignment_lost(); + return Baton(reinterpret_cast(lost)); +} + std::string KafkaConsumer::RebalanceProtocol() { if (!IsConnected()) { return std::string("NONE"); @@ -555,6 +560,7 @@ void KafkaConsumer::Init(v8::Local exports) { Nan::SetPrototypeMethod(tpl, "incrementalAssign", NodeIncrementalAssign); Nan::SetPrototypeMethod(tpl, "incrementalUnassign", NodeIncrementalUnassign); Nan::SetPrototypeMethod(tpl, "assignments", NodeAssignments); + Nan::SetPrototypeMethod(tpl, "assignmentLost", NodeAssignmentLost); Nan::SetPrototypeMethod(tpl, "rebalanceProtocol", NodeRebalanceProtocol); Nan::SetPrototypeMethod(tpl, "commit", NodeCommit); @@ -730,6 +736,17 @@ NAN_METHOD(KafkaConsumer::NodeAssignments) { Conversion::TopicPartition::ToV8Array(consumer->m_partitions)); } +NAN_METHOD(KafkaConsumer::NodeAssignmentLost) { + Nan::HandleScope scope; + + KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); + + Baton b = consumer->AssignmentLost(); + + bool lost = b.data(); + info.GetReturnValue().Set(Nan::New(lost)); +} + NAN_METHOD(KafkaConsumer::NodeRebalanceProtocol) { KafkaConsumer* consumer = ObjectWrap::Unwrap(info.This()); std::string protocol = consumer->RebalanceProtocol(); diff --git a/src/kafka-consumer.h b/src/kafka-consumer.h index e4b04fee..9da6f2cb 100644 --- a/src/kafka-consumer.h +++ b/src/kafka-consumer.h @@ -68,6 +68,7 @@ class KafkaConsumer : public Connection { Baton Position(std::vector &); Baton RefreshAssignments(); + Baton AssignmentLost(); bool HasAssignedPartitions(); int AssignedPartitionCount(); @@ -117,6 +118,7 @@ class KafkaConsumer : public Connection { static NAN_METHOD(NodeIncrementalAssign); static NAN_METHOD(NodeIncrementalUnassign); static NAN_METHOD(NodeAssignments); + static NAN_METHOD(NodeAssignmentLost); static NAN_METHOD(NodeRebalanceProtocol); static NAN_METHOD(NodeUnsubscribe); static NAN_METHOD(NodeCommit); diff --git a/test/promisified/consumer/commit.spec.js b/test/promisified/consumer/commit.spec.js index 2bc3957e..bebece53 100644 --- a/test/promisified/consumer/commit.spec.js +++ b/test/promisified/consumer/commit.spec.js @@ -150,7 +150,7 @@ describe('Consumer commit', () => { ]); }); - it.each([[true], [false]])('should commit only resolved offsets while using eachBatch', async (isAutoCommit) => { + it.each([[true], [false]])('should commit only resolved offsets while using eachBatch - isAutocommit: %s', async (isAutoCommit) => { /* Evenly distribute 3*30 messages across 3 partitions */ const numMsgs = 30; let i = 0; diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 4e0ad4ee..c59ae78a 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -1,6 +1,6 @@ jest.setTimeout(30000); -const { CompressionTypes } = require('../../../lib').KafkaJS; +const { CompressionTypes, ErrorCodes } = require('../../../lib').KafkaJS; const { secureRandom, createTopic, @@ -15,12 +15,12 @@ const { Buffer } = require('buffer'); /* All variations of partitionsConsumedConcurrently */ const cases = Array(3).fill().map((_, i) => [(i % 3) + 1]); -describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { +describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partitionsConsumedConcurrently) => { let topicName, groupId, producer, consumer; const partitions = 3; beforeEach(async () => { - console.log("Starting:", expect.getState().currentTestName, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); + console.log("Starting:", expect.getState().currentTestName); topicName = `test-topic-${secureRandom()}`; groupId = `consumer-group-id-${secureRandom()}`; @@ -38,7 +38,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { afterEach(async () => { consumer && (await consumer.disconnect()); producer && (await producer.disconnect()); - console.log("Ending:", expect.getState().currentTestName, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); + console.log("Ending:", expect.getState().currentTestName); }); it('consume messages', async () => { @@ -134,7 +134,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { ); }); - it.each([[true], [false]])('consumes messages using eachBatch', async (isAutoResolve) => { + it.each([[true], [false]])('consumes messages using eachBatch - isAutoResolve: %s', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); await consumer.subscribe({ topic: topicName }); @@ -287,7 +287,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { await waitForMessages(messagesConsumed, { number: messages.length }); }); - it.each([[true], [false]])('is able to reconsume messages when an error is thrown', async (isAutoResolve) => { + it.each([[true], [false]])('is able to reconsume messages when an error is thrown - isAutoResolve: %s', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); await consumer.subscribe({ topic: topicName }); @@ -322,7 +322,7 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { await waitForMessages(messagesConsumed, { number: messages.length }); }); - it.each([[true], [false]])('does not reconsume resolved messages even on error', async (isAutoResolve) => { + it.each([[true], [false]])('does not reconsume resolved messages even on error - isAutoResolve: %s', async (isAutoResolve) => { await consumer.connect(); await producer.connect(); await consumer.subscribe({ topic: topicName }); @@ -501,4 +501,245 @@ describe.each(cases)('Consumer', (partitionsConsumedConcurrently) => { await producer.disconnect(); }); + + it('max.poll.interval.ms should not be exceeded when per-message processing time < max.poll.interval.ms', async () => { + let rebalanceCount = 0; + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + rebalanceTimeout: 7000, /* also changes max.poll.interval.ms */ + sessionTimeout: 6000, /* minimum default value, must be less than + * rebalanceTimeout */ + autoCommitInterval: 1000, + }, { + rebalance_cb: () => { + rebalanceCount++; + }, + }); + + await producer.connect(); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + consumer.run({ + partitionsConsumedConcurrently, + eachMessage: async event => { + messagesConsumed.push(event); + await sleep(7500); /* 7.5s 'processing' + * after each message cache is cleared + * and max poll interval isn't reached */ + } + }); + + const messages = Array(5) + .fill() + .map(() => { + const value = secureRandom(); + return { value: `value-${value}`, partition: 0 }; + }); + + await producer.send({ topic: topicName, messages }); + + await waitForMessages(messagesConsumed, { number: 5, delay: 100 }); + expect(rebalanceCount).toEqual(1); /* Just the assign and nothing else at this point. */ + }, 60000); + + it('max.poll.interval.ms should not be exceeded when batch processing time < max.poll.interval.ms', async () => { + if (partitionsConsumedConcurrently !== 1) { + return; + } + let assigns = 0; + let revokes = 0; + let lost = 0; + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + rebalanceTimeout: 7000, /* also changes max.poll.interval.ms */ + sessionTimeout: 6000, /* minimum default value, must be less than + * rebalanceTimeout */ + autoCommitInterval: 1000, + }, { + rebalance_cb: async (err, assignment, { assignmentLost }) => { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assigns++; + expect(assignment.length).toBe(3); + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + revokes++; + if (assignmentLost()) + lost++; + expect(assignment.length).toBe(3); + } + } + }); + + await producer.connect(); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + let errors = false; + let receivedMessages = 0; + const batchLengths = [1, 1, 2, + /* cache reset */ + 1, 1]; + consumer.run({ + partitionsConsumedConcurrently, + eachBatchAutoResolve: true, + eachBatch: async (event) => { + receivedMessages++; + + try { + console.log(event.batch.messages.length); + expect(event.batch.messages.length) + .toEqual(batchLengths[receivedMessages - 1]); + + if (receivedMessages === 3) { + expect(event.isStale()).toEqual(false); + await sleep(7500); + /* 7.5s 'processing' + * doesn't exceed max poll interval. + * Cache reset is transparent */ + expect(event.isStale()).toEqual(false); + } + } catch (e) { + console.error(e); + errors = true; + } + messagesConsumed.push(...event.batch.messages); + } + }); + + const messages = Array(6) + .fill() + .map(() => { + const value = secureRandom(); + return { value: `value-${value}`, partition: 0 }; + }); + + await producer.send({ topic: topicName, messages }); + + await waitForMessages(messagesConsumed, { number: 6, delay: 100 }); + expect(messagesConsumed.length).toEqual(6); + + /* Triggers revocation */ + await consumer.disconnect(); + + /* First assignment */ + expect(assigns).toEqual(1); + /* Revocation on disconnect */ + expect(revokes).toEqual(1); + expect(lost).toEqual(0); + expect(errors).toEqual(false); + }, 60000); + + it('max.poll.interval.ms should be exceeded when batch processing time > max.poll.interval.ms', async () => { + if (partitionsConsumedConcurrently !== 1) { + return; + } + let assigns = 0; + let revokes = 0; + let lost = 0; + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + fromBeginning: true, + sessionTimeout: 6000, /* minimum default value, must be less than + * rebalanceTimeout */ + autoCommitInterval: 1000, + }, { + /* Testing direct librdkafka configuration here */ + 'max.poll.interval.ms': 7000, + rebalance_cb: async (err, assignment, { assignmentLost }) => { + if (err.code === ErrorCodes.ERR__ASSIGN_PARTITIONS) { + assigns++; + expect(assignment.length).toBe(3); + } else if (err.code === ErrorCodes.ERR__REVOKE_PARTITIONS) { + revokes++; + if (assignmentLost()) + lost++; + expect(assignment.length).toBe(3); + } + } + }); + + await producer.connect(); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + let errors = false; + let receivedMessages = 0; + const batchLengths = [/* first we reach batches of 32 message and fetches of 64 + * max poll interval exceeded happens on second + * 32 messages batch of the 64 msg fetch. */ + 1, 1, 2, 2, 4, 4, 8, 8, 16, 16, 32, 32, 32, 32, + /* max poll interval exceeded, 32 reprocessed + + * 1 new message. */ + 1, 1, 2, 2, 4, 4, 8, 8, 3]; + consumer.run({ + partitionsConsumedConcurrently, + eachBatchAutoResolve: true, + eachBatch: async (event) => { + receivedMessages++; + + try { + expect(event.batch.messages.length) + .toEqual(batchLengths[receivedMessages - 1]); + + if (receivedMessages === 13) { + expect(event.isStale()).toEqual(false); + await sleep(6000); + /* 6s 'processing' + * cache clearance starts at 7000 */ + expect(event.isStale()).toEqual(false); + } + if ( receivedMessages === 14) { + expect(event.isStale()).toEqual(false); + await sleep(10000); + /* 10s 'processing' + * 16s in total exceeds max poll interval. + * in this last batch after clearance. + * Batch is marked stale + * and partitions are lost */ + expect(event.isStale()).toEqual(true); + } + } catch (e) { + console.error(e); + errors = true; + } + messagesConsumed.push(...event.batch.messages); + } + }); + + const totalMessages = 191; /* without reprocessed messages */ + const messages = Array(totalMessages) + .fill() + .map(() => { + const value = secureRandom(); + return { value: `value-${value}`, partition: 0 }; + }); + + await producer.send({ topic: topicName, messages }); + /* 32 message are re-consumed after not being resolved + * because of the stale batch */ + await waitForMessages(messagesConsumed, { number: totalMessages + 32, delay: 100 }); + expect(messagesConsumed.length).toEqual(totalMessages + 32); + + /* Triggers revocation */ + await consumer.disconnect(); + + /* First assignment + assignment after partitions lost */ + expect(assigns).toEqual(2); + /* Partitions lost + revocation on disconnect */ + expect(revokes).toEqual(2); + /* Only one of the revocations has the lost flag */ + expect(lost).toEqual(1); + expect(errors).toEqual(false); + }, 60000); }); diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index 0e15a8d6..aabcd90c 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -18,11 +18,11 @@ const cases = [ [false, 3], ]; -describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumedConcurrently) => { +describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsConsumedConcurrently = %s -', (isAutoCommit, partitionsConsumedConcurrently) => { let topicName, groupId, producer, consumer; beforeEach(async () => { - console.log("Starting:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); + console.log("Starting:", expect.getState().currentTestName); topicName = `test-topic-${secureRandom()}`; groupId = `consumer-group-id-${secureRandom()}`; @@ -41,7 +41,7 @@ describe.each(cases)('Consumer message cache', (isAutoCommit, partitionsConsumed afterEach(async () => { consumer && (await consumer.disconnect()); producer && (await producer.disconnect()); - console.log("Ending:", expect.getState().currentTestName, "| isAutoCommit =", isAutoCommit, "| partitionsConsumedConcurrently =", partitionsConsumedConcurrently); + console.log("Ending:", expect.getState().currentTestName); }); it('is cleared on pause', async () => { diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index c16bad39..d5bbfe6c 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -240,6 +240,7 @@ describe('Consumer', () => { await waitForConsumerToJoinGroup(consumer); await waitForMessages(messagesConsumed, { number: 5 }); expect(messagesConsumed.length).toEqual(5); + await waitFor(() => resumeCallbacks.length >= 2, () => null, { delay: 100 }); expect(consumer.paused()).toContainEqual({ topic: topics[0], partitions: [0] }); expect(consumer.paused()).toContainEqual({ topic: topics[1], partitions: [1] }); shouldPause = false; diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index bb7cb062..0331f209 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -1,6 +1,7 @@ const crypto = require('crypto'); const process = require('process'); const { Kafka } = require('../../lib').KafkaJS; +const { DeferredPromise } = require('../../lib/kafkajs/_common'); // TODO: pick this up from a file const clusterInformation = { @@ -99,6 +100,30 @@ const generateMessages = options => { }); }; +/** + * Represents a list of promises that can be resolved in sequence or + * in a different order and awaited multiple times. + * Useful for testing particular ordering of async operations without + * relying of timing. + */ +class SequentialPromises { + #promises; + #current = 0; + + constructor(num) { + this.#promises = Array(num).fill().map(() => new DeferredPromise()); + } + + get(index) { + return this.#promises[index]; + } + + resolveNext(value) { + this.#promises[this.#current].resolve(value); + this.#current++; + } +} + module.exports = { createConsumer, createProducer, @@ -111,4 +136,5 @@ module.exports = { sleep, generateMessages, clusterInformation, + SequentialPromises }; diff --git a/test/promisified/unit/cache.spec.js b/test/promisified/unit/cache.spec.js index 5beb0ab6..2a0a76b5 100644 --- a/test/promisified/unit/cache.spec.js +++ b/test/promisified/unit/cache.spec.js @@ -1,349 +1,293 @@ const MessageCache = require('../../../lib/kafkajs/_consumer_cache'); describe('MessageCache', () => { - const expiryTime = 300000; // Long time. - const toppars = [{ topic: 'topic', partition: 0 }, { topic: 'topic', partition: 1 }, { topic: 'topic', partition: 2 }]; const messages = Array(5000) .fill() .map((_, i) => ({ topic: 'topic', partition: i % 3, number: i })); - describe("with concurrency", () => { - let cache; - beforeEach(() => { - cache = new MessageCache(expiryTime, 1); - cache.addTopicPartitions(toppars); - }); - - it('caches messages and retrieves them', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - for (let i = 0; i < 90; i++) { - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - } - - /* Results are on a per-partition basis and well-ordered */ - expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - }); - - it('caches messages and retrieves N of them', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - const expectedFetchedSizes = [11, 11, 8]; - for (let i = 0; i < (90/11); i++) { - /* We choose to fetch 11 messages together rather than 10 so that we can test the case where - * remaining messages > 0 but less than requested size. */ - const next = cache.nextN(nextIdx, 11); - /* There are 30 messages per partition, the first fetch will get 11, the second 11, and the last one - * 8, and then it repeats for each partition. */ - expect(next.length).toBe(expectedFetchedSizes[i % 3]); - expect(next).not.toBeNull(); - receivedMessages.push(...next); - nextIdx = next.index; - } - - /* Results are on a per-partition basis and well-ordered */ - expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - }); - - it('does not allow fetching more than 1 message at a time', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - let next = cache.next(-1); - let savedIndex = next.index; + let cache; + beforeEach(() => { + cache = new MessageCache(); + }); + + it('caches messages and retrieves them', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let ppc = null, next = null; + for (let i = 0; i < 90; i++) { + next = cache.next(ppc); expect(next).not.toBeNull(); - next = cache.next(-1); - expect(next).toBeNull(); - expect(cache.pendingSize()).toBeGreaterThan(0); + [next, ppc] = next; + expect(next).not.toBeNull(); + receivedMessages.push(next); + } + + /* Results are on a per-partition basis and well-ordered */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); - // Fetch after returning index works. - next = cache.next(savedIndex); + it('caches messages and retrieves N of them', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let ppc = null, next = null; + const expectedFetchedSizes = [11, 11, 8]; + for (let i = 0; i < (90/11); i++) { + /* We choose to fetch 11 messages together rather than 10 so that we can test the case where + * remaining messages > 0 but less than requested size. */ + next = cache.nextN(ppc, 11); expect(next).not.toBeNull(); - }); - - it('stops fetching from stale partition', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - for (let i = 0; i < 3; i++) { - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - cache.markStale([{topic: next.topic, partition: next.partition}]); - } - - // We should not be able to get anything more. - expect(cache.next(nextIdx)).toBeNull(); - // Nothing should be pending, we've returned everything. - expect(cache.pendingSize()).toBe(0); - // The first 3 messages from different toppars are what we should get. - expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 3))); - }); + [next, ppc] = next; + /* There are 30 messages per partition, the first fetch will get 11, the second 11, and the last one + * 8, and then it repeats for each partition. */ + expect(next.length).toBe(expectedFetchedSizes[i % 3]); + expect(next).not.toBeNull(); + receivedMessages.push(...next); + } + + /* Results are on a per-partition basis and well-ordered */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + + it('stops fetching from stale partition', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let ppc = null, next = null; + for (let i = 0; i < 3; i++) { + next = cache.next(null); + expect(next).not.toBeNull(); + [next, ppc] = next; + expect(next).not.toBeNull(); + receivedMessages.push(next); + cache.markStale([{topic: next.topic, partition: next.partition}]); + } + + // We should not be able to get anything more. + expect(cache.next(ppc)).toBeNull(); + // Nothing should be pending, we've returned everything. + expect(cache.assignedSize).toBe(0); + // The first 3 messages from different toppars are what we should get. + expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 3))); + }); + + it('caches messages and retrieves 2-at-a-time', () => { + const msgs = messages.slice(0, 90).filter(msg => msg.partition !== 3); + cache.addMessages(msgs); + + const receivedMessages = []; + let next = [null, null]; + let nextPpc = [null, null]; + for (let i = 0; i < 30; i++) { + next[0] = cache.next(nextPpc[0]); + next[1] = cache.next(nextPpc[1]); + expect(next[0]).not.toBeNull(); + expect(next[1]).not.toBeNull(); + [next[0], nextPpc[0]] = next[0]; + [next[1], nextPpc[1]] = next[1]; + receivedMessages.push(next[0]); + receivedMessages.push(next[1]); + } + + expect(receivedMessages.length).toBe(60); + expect(receivedMessages.filter(msg => msg.partition === 0).length).toBe(30); + expect(receivedMessages.filter(msg => msg.partition === 1).length).toBe(30); + }); + + it('caches messages and retrieves N of them 2-at-a-time', () => { + const msgs = messages.slice(0, 90).filter(msg => msg.partition !== 3); + cache.addMessages(msgs); + + const receivedMessages = []; + let next = [null, null]; + let nextPpc = [null, null]; + for (let i = 0; i < 30/11; i++) { + next[0] = cache.nextN(nextPpc[0], 11); + next[1] = cache.nextN(nextPpc[1], 11); + expect(next[0]).not.toBeNull(); + expect(next[1]).not.toBeNull(); + [next[0], nextPpc[0]] = next[0]; + [next[1], nextPpc[1]] = next[1]; + receivedMessages.push(...next[0]); + receivedMessages.push(...next[1]); + } + + expect(receivedMessages.length).toBe(60); + expect(receivedMessages.filter(msg => msg.partition === 0).length).toBe(30); + expect(receivedMessages.filter(msg => msg.partition === 1).length).toBe(30); + }); + + it('does not allow fetching messages more than available partitions at a time', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + let next = cache.next(); + let ppc = next[1]; + expect(next).not.toBeNull(); + next = cache.next(); + expect(next).not.toBeNull(); + next = cache.next(); + expect(next).not.toBeNull(); + next = cache.next(); + expect(next).toBeNull(); + expect(cache.assignedSize).toBe(3); + + // Fetch after returning ppc works. + cache.return(ppc); + next = cache.next(); + expect(next).not.toBeNull(); + }); + + it('does not allow fetching message sets more than available partitions at a time', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + let next = cache.nextN(null, 11); + let ppc = next[1]; + expect(next).not.toBeNull(); + next = cache.nextN(null, 11); + expect(next).not.toBeNull(); + next = cache.nextN(null, 11); + expect(next).not.toBeNull(); + next = cache.nextN(null, 11); + expect(next).toBeNull(); + expect(cache.assignedSize).toBe(3); + + // Fetch after returning ppc works. + cache.return(ppc); + next = cache.nextN(null, 11); + expect(next).not.toBeNull(); }); - describe("with concurrency = 2", () => { - let cache; - beforeEach(() => { - cache = new MessageCache(expiryTime, 2); - cache.addTopicPartitions(toppars); - }); - - it('caches messages and retrieves them', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - for (let i = 0; i < 90; i++) { - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - } - - /* Results are on a per-partition basis and well-ordered */ - expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(61, 30).every((msg, i) => msg.partition === receivedMessages[60].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - }); - - it('caches messages and retrieves 2-at-a-time', () => { - const msgs = messages.slice(0, 90).filter(msg => msg.partition !== 3); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdxs = [-1, -1]; - for (let i = 0; i < 30; i++) { - const next0 = cache.next(nextIdxs[0]); - const next1 = cache.next(nextIdxs[1]); - expect(next0).not.toBeNull(); - expect(next1).not.toBeNull(); - receivedMessages.push(next0); - receivedMessages.push(next1); - nextIdxs = [next0.index, next1.index]; - } - - expect(receivedMessages.length).toBe(60); - expect(receivedMessages.filter(msg => msg.partition === 0).length).toBe(30); - expect(receivedMessages.filter(msg => msg.partition === 1).length).toBe(30); - }); - - it('caches messages and retrieves N of them 2-at-a-time', () => { - const msgs = messages.slice(0, 90).filter(msg => msg.partition !== 3); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdxs = [-1, -1]; - for (let i = 0; i < 30/11; i++) { - const next0 = cache.nextN(nextIdxs[0], 11); - const next1 = cache.nextN(nextIdxs[1], 11); - expect(next0).not.toBeNull(); - expect(next1).not.toBeNull(); - receivedMessages.push(...next0); - receivedMessages.push(...next1); - nextIdxs = [next0.index, next1.index]; - } - - expect(receivedMessages.length).toBe(60); - expect(receivedMessages.filter(msg => msg.partition === 0).length).toBe(30); - expect(receivedMessages.filter(msg => msg.partition === 1).length).toBe(30); - }); - - it('does not allow fetching more than 2 message at a time', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - let next = cache.next(-1); - let savedIndex = next.index; + it('stops fetching message sets from stale partition', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let next, ppc; + for (let i = 0; i < 3; i++) { + next = cache.nextN(null, 11); + expect(next).not.toBeNull(); + [next, ppc] = next; + receivedMessages.push(...next); + cache.markStale([{topic: next[0].topic, partition: next[0].partition}]); + cache.return(ppc); + } + + // We should not be able to get anything more. + expect(cache.nextN(null, 11)).toBeNull(); + // Nothing should be pending, we've returned everything. + expect(cache.assignedSize).toBe(0); + // The first [11, 11, 11] messages from different toppars. + expect(receivedMessages.length).toBe(33); + expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 33))); + }); + + it('one slow processing message should not slow down others', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let next, ppc; + cache.next(ppc); + for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ + next = cache.next(ppc); expect(next).not.toBeNull(); - next = cache.next(-1); + [next, ppc] = next; expect(next).not.toBeNull(); - next = cache.next(-1); - expect(next).toBeNull(); - expect(cache.pendingSize()).toBe(2); + receivedMessages.push(next); + } + + // We should not be able to get anything more. + expect(cache.next(ppc)).toBeNull(); + // The slowMsg should be pending. + expect(cache.assignedSize).toBe(1); - // Fetch after returning index works. - next = cache.next(savedIndex); + /* Messages should be partition-wise and well-ordered. */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); + + it('one slow processing message set should not slow down others', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let next, ppc; + cache.nextN(ppc, 11); + for (let i = 0; i < 60/11; i++) { /* 60 - for non-partition 0 msgs */ + next = cache.nextN(ppc, 11); expect(next).not.toBeNull(); - }); + [next, ppc] = next; + receivedMessages.push(...next); + } - it('does not allow fetching more than 2 message sets at a time', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); + // We should not be able to get anything more. + expect(cache.nextN(ppc, 11)).toBeNull(); + // The slowMsg should be pending. + expect(cache.assignedSize).toBe(1); + + /* Messages should be partition-wise and well-ordered. */ + expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); + }); - let next = cache.nextN(-1, 11); - let savedIndex = next.index; + it('should be able to handle cache-clearance in the middle of processing', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let next, ppc; + cache.next(); + for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ + next = cache.next(ppc); expect(next).not.toBeNull(); - next = cache.nextN(-1, 11); + [next, ppc] = next; expect(next).not.toBeNull(); - next = cache.nextN(-1, 11); - expect(next).toBeNull(); - expect(cache.pendingSize()).toBe(2); + receivedMessages.push(next); + } + + // We should not be able to get anything more. + expect(cache.next(ppc)).toBeNull(); + + // The slowMsg should be pending. + expect(cache.assignedSize).toBe(1); + + expect(() => cache.clear()).not.toThrow(); + }); - // Fetch after returning index works. - next = cache.nextN(savedIndex, 11); + it('should be able to handle message adds in the middle of processing', () => { + const msgs = messages.slice(0, 90); + cache.addMessages(msgs); + + const receivedMessages = []; + let next, ppc; + cache.next(); + for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ + next = cache.next(ppc); + expect(next).not.toBeNull(); + [next, ppc] = next; expect(next).not.toBeNull(); - }); - - it('stops fetching from stale partition', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - for (let i = 0; i < 3; i++) { - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - cache.markStale([{topic: next.topic, partition: next.partition}]); - } - - // We should not be able to get anything more. - expect(cache.next(nextIdx)).toBeNull(); - // Nothing should be pending, we've returned everything. - expect(cache.pendingSize()).toBe(0); - // The first 3 messages from different toppars are what we should get. - expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 3))); - }); - - it('stops fetching message sets from stale partition', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - for (let i = 0; i < 3; i++) { - const next = cache.nextN(nextIdx, 11); - expect(next).not.toBeNull(); - receivedMessages.push(...next); - nextIdx = next.index; - cache.markStale([{topic: next[0].topic, partition: next[0].partition}]); - } - - // We should not be able to get anything more. - expect(cache.nextN(nextIdx, 11)).toBeNull(); - // Nothing should be pending, we've returned everything. - expect(cache.pendingSize()).toBe(0); - // The first [11, 11, 11] messages from different toppars. - expect(receivedMessages.length).toBe(33); - expect(receivedMessages).toEqual(expect.arrayContaining(msgs.slice(0, 33))); - }); - - it('one slow processing message should not slow down others', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - cache.next(nextIdx); - for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - } - - - // We should not be able to get anything more. - expect(cache.next(nextIdx)).toBeNull(); - // The slowMsg should be pending. - expect(cache.pendingSize()).toBe(1); - - /* Messages should be partition-wise and well-ordered. */ - expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - }); - - it('one slow processing message set should not slow down others', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - cache.nextN(nextIdx, 11); - for (let i = 0; i < 60/11; i++) { /* 60 - for non-partition 0 msgs */ - const next = cache.nextN(nextIdx, 11); - expect(next).not.toBeNull(); - receivedMessages.push(...next); - nextIdx = next.index; - } - - - // We should not be able to get anything more. - expect(cache.nextN(nextIdx, 11)).toBeNull(); - // The slowMsg should be pending. - expect(cache.pendingSize()).toBe(1); - - /* Messages should be partition-wise and well-ordered. */ - expect(receivedMessages.slice(1, 30).every((msg, i) => msg.partition === receivedMessages[0].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - expect(receivedMessages.slice(31, 30).every((msg, i) => msg.partition === receivedMessages[30].partition && (msg.number - 3) === receivedMessages[i].number)).toBeTruthy(); - }); - - it('should not be able to handle cache-clearance in the middle of processing', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - cache.next(nextIdx); - for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - } - - - // We should not be able to get anything more. - expect(cache.next(nextIdx)).toBeNull(); - - // The slowMsg should be pending. - expect(cache.pendingSize()).toBe(1); - - expect(() => cache.clear()).toThrow(); - }); - - it('should not be able to handle message adds in the middle of processing', () => { - const msgs = messages.slice(0, 90); - cache.addMessages(msgs); - - const receivedMessages = []; - let nextIdx = -1; - cache.next(nextIdx); - for (let i = 0; i < 60; i++) { /* 60 - for non-partition 0 msgs */ - const next = cache.next(nextIdx); - expect(next).not.toBeNull(); - receivedMessages.push(next); - nextIdx = next.index; - } - - // We should not be able to get anything more. - expect(cache.next(nextIdx)).toBeNull(); - - // The slowMsg should be pending. - expect(cache.pendingSize()).toBe(1); - - expect(() => cache.addMessages(msgs)).toThrow(); - }); + receivedMessages.push(next); + } + + // We should not be able to get anything more. + expect(cache.next(ppc)).toBeNull(); + + // The slowMsg should be pending. + expect(cache.assignedSize).toBe(1); + + expect(() => cache.addMessages(msgs)).not.toThrow(); }); }); \ No newline at end of file diff --git a/test/promisified/unit/common.spec.js b/test/promisified/unit/common.spec.js new file mode 100644 index 00000000..5fbba1d6 --- /dev/null +++ b/test/promisified/unit/common.spec.js @@ -0,0 +1,227 @@ +const { Lock } = require('../../../lib/kafkajs/_common'); +const { SequentialPromises } = require('../testhelpers'); + +describe('Lock', () => { + + it('allows multiple concurrent readers', async () => { + let lock = new Lock(); + let sequentialPromises = new SequentialPromises(1); + let events = []; + let tasks = []; + let concurrency = 50; + + for (let i = 0; i < concurrency; i++) { + let task = lock.read(async () => { + events.push(i * 2); + await sequentialPromises.get(0); + events.push(i * 2 + 1); + }); + tasks.push(task); + } + + /* Make sure all tasks can reach the promise. */ + await new Promise((r) => setTimeout(r, 10)); + sequentialPromises.resolveNext(); + await Promise.all(tasks); + + for (let event of events.slice(0, 50)) { + expect(event % 2).toEqual(0); + } + for (let event of events.slice(50)) { + expect(event % 2).toEqual(1); + } + }); + + it('prevents multiple concurrent write locks', async () => { + let lock = new Lock(); + let sequentialPromises = new SequentialPromises(1); + let events = []; + let tasks = []; + let concurrency = 50; + + for (let i = 0; i < concurrency; i++) { + let task = lock.write(async () => { + events.push(i * 2); + await sequentialPromises.get(0); + events.push(i * 2 + 1); + }); + tasks.push(task); + } + + /* Make sure all tasks can reach the promise in case + * the lock wasn't working. */ + await new Promise((r) => setTimeout(r, 10)); + sequentialPromises.resolveNext(); + await Promise.all(tasks); + + for (let i = 0; i < concurrency; i++) { + expect(events[i * 2]).toBe(events[i * 2 + 1] - 1); + } + }); + + it('allows either multiple readers or a single writer', async () => { + let lock = new Lock(); + let sequentialPromises = new SequentialPromises(3); + let events = []; + let promises = []; + sequentialPromises.resolveNext(); + + let read1 = lock.read(async () => { + events.push(0); + await sequentialPromises.get(0); + events.push(1); + sequentialPromises.resolveNext(); + }); + promises.push(read1); + + let read2 = lock.read(async () => { + events.push(2); + await sequentialPromises.get(1); + events.push(3); + sequentialPromises.resolveNext(); + }); + promises.push(read2); + + let write1 = lock.write(async () => { + events.push(4); + await sequentialPromises.get(2); + events.push(5); + }); + promises.push(write1); + + await Promise.all(promises); + + expect(events).toEqual([0, 2, 1, 3, 4, 5]); + }); + + + it('allows reentrant read locks', async () => { + let lock = new Lock(); + let sequentialPromises = new SequentialPromises(2); + let events = []; + let promises = []; + sequentialPromises.resolveNext(); + + let read1 = lock.read(async () => { + events.push(0); + await lock.read(async () => { + events.push(1); + await sequentialPromises.get(0); + events.push(2); + }); + events.push(3); + sequentialPromises.resolveNext(); + }); + promises.push(read1); + + let read2 = lock.read(async () => { + events.push(4); + await lock.read(async () => { + events.push(5); + await sequentialPromises.get(1); + events.push(6); + }); + events.push(7); + }); + promises.push(read2); + + await Promise.all(promises); + + expect(events).toEqual([0, 4, 1, 5, 2, 3, 6, 7]); + }); + + it('allows reentrant write locks', async () => { + let lock = new Lock(); + let sequentialPromises = new SequentialPromises(2); + let events = []; + let promises = []; + sequentialPromises.resolveNext(); + + let write1 = lock.write(async () => { + events.push(0); + await lock.write(async () => { + events.push(1); + await sequentialPromises.get(0); + events.push(2); + }); + events.push(3); + sequentialPromises.resolveNext(); + }); + promises.push(write1); + + let write2 = lock.write(async () => { + events.push(4); + await lock.write(async () => { + events.push(5); + await sequentialPromises.get(1); + events.push(6); + }); + events.push(7); + }); + promises.push(write2); + + await Promise.allSettled(promises); + + expect(events).toEqual([0, 1, 2, 3, 4, 5, 6, 7]); + }); + + it('can upgrade to a write lock while holding a read lock', + async () => { + let lock = new Lock(); + await lock.read(async () => { + await lock.read(async () => { + await lock.write(async () => { + await lock.write(async () => { + await lock.read(async () => { + + }); + }); + }); + }); + }); + }); + + it('can acquire a read lock with holding a write lock', async () => { + let lock = new Lock(); + await lock.write(async () => { + await lock.write(async () => { + await lock.read(async () => { + await lock.read(async () => { + await lock.write(async () => { + }); + }); + }); + }); + }); + }); + + it('awaits locks the called function doesn\'t await', async () => { + let lock = new Lock(); + let sequentialPromises = new SequentialPromises(2); + let events = []; + await lock.write(async () => { + events.push(0); + lock.read(async () => { + await sequentialPromises.get(1); + events.push(1); + }); + lock.write(async () => { + await sequentialPromises.get(0); + events.push(2); + sequentialPromises.resolveNext(); + }); + sequentialPromises.resolveNext(); + }); + + expect(events).toEqual([0, 2, 1]); + }); + + it('propagates errors', async () => { + let lock = new Lock(); + let throwing = + lock.read(async () => { + throw new Error('shouldn\'t happen'); + }); + await expect(throwing).rejects.toThrow('shouldn\'t happen'); + }); +}); \ No newline at end of file diff --git a/types/config.d.ts b/types/config.d.ts index 930b611f..f0b1f708 100644 --- a/types/config.d.ts +++ b/types/config.d.ts @@ -762,7 +762,7 @@ export interface ProducerGlobalConfig extends GlobalConfig { /** * Delivery report callback (set with rd_kafka_conf_set_dr_cb()) */ - "dr_cb"?: boolean | Function; + "dr_cb"?: boolean | ((...args: any[]) => any); /** * Delivery report callback (set with rd_kafka_conf_set_dr_msg_cb()) @@ -936,12 +936,12 @@ export interface ConsumerGlobalConfig extends GlobalConfig { /** * Called after consumer group has been rebalanced (set with rd_kafka_conf_set_rebalance_cb()) */ - "rebalance_cb"?: boolean | Function; + "rebalance_cb"?: boolean | ((...args: any[]) => any); /** * Offset commit result propagation callback. (set with rd_kafka_conf_set_offset_commit_cb()) */ - "offset_commit_cb"?: boolean | Function; + "offset_commit_cb"?: boolean | ((...args: any[]) => any); /** * Emit RD_KAFKA_RESP_ERR__PARTITION_EOF event whenever the consumer reaches the end of a partition. diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index b3387e84..bf233db1 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -287,6 +287,7 @@ export type ConsumerSubscribeTopics = { topics: (string | RegExp)[]; replace?: b export type ConsumerRunConfig = { eachBatchAutoResolve?: boolean, + partitionsConsumedConcurrently?: number, eachMessage?: EachMessageHandler eachBatch?: EachBatchHandler } @@ -317,7 +318,7 @@ export type Consumer = Client & { storeOffsets(topicPartitions: Array): void commitOffsets(topicPartitions?: Array): Promise committed(topicPartitions?: Array, timeout?: number): Promise - seek(topicPartitionOffset: TopicPartitionOffset): Promise + seek(topicPartitionOffset: TopicPartitionOffset): void pause(topics: Array<{ topic: string; partitions?: number[] }>): void paused(): TopicPartitions[] resume(topics: Array<{ topic: string; partitions?: number[] }>): void From 65c601d2ffeaba6a8e8b4354d69b5532690ba045 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Fri, 13 Sep 2024 13:09:36 -0400 Subject: [PATCH 191/224] Schemaregistry rebase (#33) (#80) * rebase dev_early_access * Add OAuth Support to Rest Service (#25) * Add OAuth client support * Add optional chaining for token * Fix merge conflict * add simple-oauth2 dependency --- package-lock.json | 4790 ++++++++--------- package.json | 23 +- schemaregistry/oauth/oauth-client.ts | 56 + schemaregistry/package.json | 3 +- schemaregistry/rest-service.ts | 44 +- .../dekregistry/dekregistry-client.ts | 2 +- .../dekregistry/mock-dekregistry-client.ts | 2 +- schemaregistry/schemaregistry-client.ts | 5 +- 8 files changed, 2293 insertions(+), 2632 deletions(-) create mode 100644 schemaregistry/oauth/oauth-client.ts diff --git a/package-lock.json b/package-lock.json index bce3b217..a3e3a19f 100644 --- a/package-lock.json +++ b/package-lock.json @@ -13,9 +13,30 @@ "schemaregistry" ], "dependencies": { + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", + "@bufbuild/protobuf": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", + "@smithy/types": "^3.3.0", + "@types/simple-oauth2": "^5.0.7", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", + "async-mutex": "^0.5.0", + "avsc": "^5.7.7", + "axios": "^1.7.3", "bindings": "^1.3.1", - "nan": "^2.17.0" + "json-stringify-deterministic": "^1.0.12", + "lru-cache": "^11.0.0", + "nan": "^2.17.0", + "node-vault": "^0.10.2", + "simple-oauth2": "^5.1.0", + "ts-jest": "^29.2.4", + "validator": "^13.12.0" }, "devDependencies": { "@bufbuild/buf": "^1.37.0", @@ -39,9 +60,7 @@ }, "node_modules/@ampproject/remapping": { "version": "2.3.0", - "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.3.0.tgz", - "integrity": "sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw==", - "dev": true, + "license": "Apache-2.0", "dependencies": { "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.24" @@ -52,8 +71,7 @@ }, "node_modules/@aws-crypto/sha256-browser": { "version": "5.2.0", - "resolved": "https://registry.npmjs.org/@aws-crypto/sha256-browser/-/sha256-browser-5.2.0.tgz", - "integrity": "sha512-AXfN/lGotSQwu6HNcEsIASo7kWXZ5HYWvfOmSNKDsEqC4OashTp8alTmaz+F7TC2L083SFv5RdB+qU3Vs1kZqw==", + "license": "Apache-2.0", "dependencies": { "@aws-crypto/sha256-js": "^5.2.0", "@aws-crypto/supports-web-crypto": "^5.2.0", @@ -66,8 +84,7 @@ }, "node_modules/@aws-crypto/sha256-browser/node_modules/@smithy/is-array-buffer": { "version": "2.2.0", - "resolved": "https://registry.npmjs.org/@smithy/is-array-buffer/-/is-array-buffer-2.2.0.tgz", - "integrity": "sha512-GGP3O9QFD24uGeAXYUjwSTXARoqpZykHadOmA8G5vfJPK0/DC67qa//0qvqrJzL1xc8WQWX7/yc7fwudjPHPhA==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -77,8 +94,7 @@ }, "node_modules/@aws-crypto/sha256-browser/node_modules/@smithy/util-buffer-from": { "version": "2.2.0", - "resolved": "https://registry.npmjs.org/@smithy/util-buffer-from/-/util-buffer-from-2.2.0.tgz", - "integrity": "sha512-IJdWBbTcMQ6DA0gdNhh/BwrLkDR+ADW5Kr1aZmd4k3DIF6ezMV4R2NIAmT08wQJ3yUK82thHWmC/TnK/wpMMIA==", + "license": "Apache-2.0", "dependencies": { "@smithy/is-array-buffer": "^2.2.0", "tslib": "^2.6.2" @@ -89,8 +105,7 @@ }, "node_modules/@aws-crypto/sha256-browser/node_modules/@smithy/util-utf8": { "version": "2.3.0", - "resolved": "https://registry.npmjs.org/@smithy/util-utf8/-/util-utf8-2.3.0.tgz", - "integrity": "sha512-R8Rdn8Hy72KKcebgLiv8jQcQkXoLMOGGv5uI1/k0l+snqkOzQ1R0ChUBCxWMlBsFMekWjq0wRudIweFs7sKT5A==", + "license": "Apache-2.0", "dependencies": { "@smithy/util-buffer-from": "^2.2.0", "tslib": "^2.6.2" @@ -101,8 +116,7 @@ }, "node_modules/@aws-crypto/sha256-js": { "version": "5.2.0", - "resolved": "https://registry.npmjs.org/@aws-crypto/sha256-js/-/sha256-js-5.2.0.tgz", - "integrity": "sha512-FFQQyu7edu4ufvIZ+OadFpHHOt+eSTBaYaki44c+akjg7qZg9oOQeLlk77F6tSYqjDAFClrHJk9tMf0HdVyOvA==", + "license": "Apache-2.0", "dependencies": { "@aws-crypto/util": "^5.2.0", "@aws-sdk/types": "^3.222.0", @@ -114,16 +128,14 @@ }, "node_modules/@aws-crypto/supports-web-crypto": { "version": "5.2.0", - "resolved": "https://registry.npmjs.org/@aws-crypto/supports-web-crypto/-/supports-web-crypto-5.2.0.tgz", - "integrity": "sha512-iAvUotm021kM33eCdNfwIN//F77/IADDSs58i+MDaOqFrVjZo9bAal0NK7HurRuWLLpF1iLX7gbWrjHjeo+YFg==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" } }, "node_modules/@aws-crypto/util": { "version": "5.2.0", - "resolved": "https://registry.npmjs.org/@aws-crypto/util/-/util-5.2.0.tgz", - "integrity": "sha512-4RkU9EsI6ZpBve5fseQlGNUWKMa1RLPQ1dnjnQoe07ldfIzcsGb5hC5W0Dm7u423KWzawlrpbjXBrXCEv9zazQ==", + "license": "Apache-2.0", "dependencies": { "@aws-sdk/types": "^3.222.0", "@smithy/util-utf8": "^2.0.0", @@ -132,8 +144,7 @@ }, "node_modules/@aws-crypto/util/node_modules/@smithy/is-array-buffer": { "version": "2.2.0", - "resolved": "https://registry.npmjs.org/@smithy/is-array-buffer/-/is-array-buffer-2.2.0.tgz", - "integrity": "sha512-GGP3O9QFD24uGeAXYUjwSTXARoqpZykHadOmA8G5vfJPK0/DC67qa//0qvqrJzL1xc8WQWX7/yc7fwudjPHPhA==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -143,8 +154,7 @@ }, "node_modules/@aws-crypto/util/node_modules/@smithy/util-buffer-from": { "version": "2.2.0", - "resolved": "https://registry.npmjs.org/@smithy/util-buffer-from/-/util-buffer-from-2.2.0.tgz", - "integrity": "sha512-IJdWBbTcMQ6DA0gdNhh/BwrLkDR+ADW5Kr1aZmd4k3DIF6ezMV4R2NIAmT08wQJ3yUK82thHWmC/TnK/wpMMIA==", + "license": "Apache-2.0", "dependencies": { "@smithy/is-array-buffer": "^2.2.0", "tslib": "^2.6.2" @@ -155,8 +165,7 @@ }, "node_modules/@aws-crypto/util/node_modules/@smithy/util-utf8": { "version": "2.3.0", - "resolved": "https://registry.npmjs.org/@smithy/util-utf8/-/util-utf8-2.3.0.tgz", - "integrity": "sha512-R8Rdn8Hy72KKcebgLiv8jQcQkXoLMOGGv5uI1/k0l+snqkOzQ1R0ChUBCxWMlBsFMekWjq0wRudIweFs7sKT5A==", + "license": "Apache-2.0", "dependencies": { "@smithy/util-buffer-from": "^2.2.0", "tslib": "^2.6.2" @@ -166,49 +175,48 @@ } }, "node_modules/@aws-sdk/client-kms": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-kms/-/client-kms-3.650.0.tgz", - "integrity": "sha512-7J/DW9/+CAdCop36IhiGGPLx4rclMyzQrI95EIN3FU5dTUFZ8aDHN+euTMfVSy7dfbwCsTxESx5/U/ZeOrFvPA==", + "version": "3.637.0", + "license": "Apache-2.0", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/client-sso-oidc": "3.650.0", - "@aws-sdk/client-sts": "3.650.0", - "@aws-sdk/core": "3.649.0", - "@aws-sdk/credential-provider-node": "3.650.0", - "@aws-sdk/middleware-host-header": "3.649.0", - "@aws-sdk/middleware-logger": "3.649.0", - "@aws-sdk/middleware-recursion-detection": "3.649.0", - "@aws-sdk/middleware-user-agent": "3.649.0", - "@aws-sdk/region-config-resolver": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@aws-sdk/util-endpoints": "3.649.0", - "@aws-sdk/util-user-agent-browser": "3.649.0", - "@aws-sdk/util-user-agent-node": "3.649.0", - "@smithy/config-resolver": "^3.0.6", - "@smithy/core": "^2.4.1", - "@smithy/fetch-http-handler": "^3.2.5", - "@smithy/hash-node": "^3.0.4", - "@smithy/invalid-dependency": "^3.0.4", - "@smithy/middleware-content-length": "^3.0.6", - "@smithy/middleware-endpoint": "^3.1.1", - "@smithy/middleware-retry": "^3.0.16", - "@smithy/middleware-serde": "^3.0.4", - "@smithy/middleware-stack": "^3.0.4", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/node-http-handler": "^3.2.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/url-parser": "^3.0.4", + "@aws-sdk/client-sso-oidc": "3.637.0", + "@aws-sdk/client-sts": "3.637.0", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/credential-provider-node": "3.637.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.16", - "@smithy/util-defaults-mode-node": "^3.0.16", - "@smithy/util-endpoints": "^2.1.0", - "@smithy/util-middleware": "^3.0.4", - "@smithy/util-retry": "^3.0.4", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -217,46 +225,45 @@ } }, "node_modules/@aws-sdk/client-sso": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso/-/client-sso-3.650.0.tgz", - "integrity": "sha512-YKm14gCMChD/jlCisFlsVqB8HJujR41bl4Fup2crHwNJxhD/9LTnzwMiVVlBqlXr41Sfa6fSxczX2AMP8NM14A==", + "version": "3.637.0", + "license": "Apache-2.0", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/core": "3.649.0", - "@aws-sdk/middleware-host-header": "3.649.0", - "@aws-sdk/middleware-logger": "3.649.0", - "@aws-sdk/middleware-recursion-detection": "3.649.0", - "@aws-sdk/middleware-user-agent": "3.649.0", - "@aws-sdk/region-config-resolver": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@aws-sdk/util-endpoints": "3.649.0", - "@aws-sdk/util-user-agent-browser": "3.649.0", - "@aws-sdk/util-user-agent-node": "3.649.0", - "@smithy/config-resolver": "^3.0.6", - "@smithy/core": "^2.4.1", - "@smithy/fetch-http-handler": "^3.2.5", - "@smithy/hash-node": "^3.0.4", - "@smithy/invalid-dependency": "^3.0.4", - "@smithy/middleware-content-length": "^3.0.6", - "@smithy/middleware-endpoint": "^3.1.1", - "@smithy/middleware-retry": "^3.0.16", - "@smithy/middleware-serde": "^3.0.4", - "@smithy/middleware-stack": "^3.0.4", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/node-http-handler": "^3.2.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/url-parser": "^3.0.4", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.16", - "@smithy/util-defaults-mode-node": "^3.0.16", - "@smithy/util-endpoints": "^2.1.0", - "@smithy/util-middleware": "^3.0.4", - "@smithy/util-retry": "^3.0.4", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -265,47 +272,46 @@ } }, "node_modules/@aws-sdk/client-sso-oidc": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-sso-oidc/-/client-sso-oidc-3.650.0.tgz", - "integrity": "sha512-6J7IS0f8ovhvbIAZaynOYP+jPX8344UlTjwHxjaXHgFvI8axu3+NslKtEEV5oHLhgzDvrKbinsu5lgE2n4Sqng==", + "version": "3.637.0", + "license": "Apache-2.0", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/core": "3.649.0", - "@aws-sdk/credential-provider-node": "3.650.0", - "@aws-sdk/middleware-host-header": "3.649.0", - "@aws-sdk/middleware-logger": "3.649.0", - "@aws-sdk/middleware-recursion-detection": "3.649.0", - "@aws-sdk/middleware-user-agent": "3.649.0", - "@aws-sdk/region-config-resolver": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@aws-sdk/util-endpoints": "3.649.0", - "@aws-sdk/util-user-agent-browser": "3.649.0", - "@aws-sdk/util-user-agent-node": "3.649.0", - "@smithy/config-resolver": "^3.0.6", - "@smithy/core": "^2.4.1", - "@smithy/fetch-http-handler": "^3.2.5", - "@smithy/hash-node": "^3.0.4", - "@smithy/invalid-dependency": "^3.0.4", - "@smithy/middleware-content-length": "^3.0.6", - "@smithy/middleware-endpoint": "^3.1.1", - "@smithy/middleware-retry": "^3.0.16", - "@smithy/middleware-serde": "^3.0.4", - "@smithy/middleware-stack": "^3.0.4", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/node-http-handler": "^3.2.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/url-parser": "^3.0.4", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/credential-provider-node": "3.637.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.16", - "@smithy/util-defaults-mode-node": "^3.0.16", - "@smithy/util-endpoints": "^2.1.0", - "@smithy/util-middleware": "^3.0.4", - "@smithy/util-retry": "^3.0.4", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -313,52 +319,51 @@ "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sts": "^3.650.0" + "@aws-sdk/client-sts": "^3.637.0" } }, "node_modules/@aws-sdk/client-sts": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/client-sts/-/client-sts-3.650.0.tgz", - "integrity": "sha512-ISK0ZQYA7O5/WYgslpWy956lUBudGC9d7eL0FFbiL0j50N80Gx3RUv22ezvZgxJWE0W3DqNr4CE19sPYn4Lw8g==", + "version": "3.637.0", + "license": "Apache-2.0", "dependencies": { "@aws-crypto/sha256-browser": "5.2.0", "@aws-crypto/sha256-js": "5.2.0", - "@aws-sdk/client-sso-oidc": "3.650.0", - "@aws-sdk/core": "3.649.0", - "@aws-sdk/credential-provider-node": "3.650.0", - "@aws-sdk/middleware-host-header": "3.649.0", - "@aws-sdk/middleware-logger": "3.649.0", - "@aws-sdk/middleware-recursion-detection": "3.649.0", - "@aws-sdk/middleware-user-agent": "3.649.0", - "@aws-sdk/region-config-resolver": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@aws-sdk/util-endpoints": "3.649.0", - "@aws-sdk/util-user-agent-browser": "3.649.0", - "@aws-sdk/util-user-agent-node": "3.649.0", - "@smithy/config-resolver": "^3.0.6", - "@smithy/core": "^2.4.1", - "@smithy/fetch-http-handler": "^3.2.5", - "@smithy/hash-node": "^3.0.4", - "@smithy/invalid-dependency": "^3.0.4", - "@smithy/middleware-content-length": "^3.0.6", - "@smithy/middleware-endpoint": "^3.1.1", - "@smithy/middleware-retry": "^3.0.16", - "@smithy/middleware-serde": "^3.0.4", - "@smithy/middleware-stack": "^3.0.4", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/node-http-handler": "^3.2.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/url-parser": "^3.0.4", + "@aws-sdk/client-sso-oidc": "3.637.0", + "@aws-sdk/core": "3.635.0", + "@aws-sdk/credential-provider-node": "3.637.0", + "@aws-sdk/middleware-host-header": "3.620.0", + "@aws-sdk/middleware-logger": "3.609.0", + "@aws-sdk/middleware-recursion-detection": "3.620.0", + "@aws-sdk/middleware-user-agent": "3.637.0", + "@aws-sdk/region-config-resolver": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@aws-sdk/util-user-agent-browser": "3.609.0", + "@aws-sdk/util-user-agent-node": "3.614.0", + "@smithy/config-resolver": "^3.0.5", + "@smithy/core": "^2.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/hash-node": "^3.0.3", + "@smithy/invalid-dependency": "^3.0.3", + "@smithy/middleware-content-length": "^3.0.5", + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", "@smithy/util-base64": "^3.0.0", "@smithy/util-body-length-browser": "^3.0.0", "@smithy/util-body-length-node": "^3.0.0", - "@smithy/util-defaults-mode-browser": "^3.0.16", - "@smithy/util-defaults-mode-node": "^3.0.16", - "@smithy/util-endpoints": "^2.1.0", - "@smithy/util-middleware": "^3.0.4", - "@smithy/util-retry": "^3.0.4", + "@smithy/util-defaults-mode-browser": "^3.0.15", + "@smithy/util-defaults-mode-node": "^3.0.15", + "@smithy/util-endpoints": "^2.0.5", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -367,18 +372,17 @@ } }, "node_modules/@aws-sdk/core": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/core/-/core-3.649.0.tgz", - "integrity": "sha512-dheG/X2y25RHE7K+TlS32kcy7TgDg1OpWV44BQRoE0OBPAWmFR1D1qjjTZ7WWrdqRPKzcnDj1qED8ncyncOX8g==", - "dependencies": { - "@smithy/core": "^2.4.1", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/property-provider": "^3.1.4", - "@smithy/protocol-http": "^4.1.1", - "@smithy/signature-v4": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/util-middleware": "^3.0.4", + "version": "3.635.0", + "license": "Apache-2.0", + "dependencies": { + "@smithy/core": "^2.4.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/signature-v4": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-middleware": "^3.0.3", "fast-xml-parser": "4.4.1", "tslib": "^2.6.2" }, @@ -387,13 +391,12 @@ } }, "node_modules/@aws-sdk/credential-provider-env": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-env/-/credential-provider-env-3.649.0.tgz", - "integrity": "sha512-tViwzM1dauksA3fdRjsg0T8mcHklDa8EfveyiQKK6pUJopkqV6FQx+X5QNda0t/LrdEVlFZvwHNdXqOEfc83TA==", + "version": "3.620.1", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/property-provider": "^3.1.4", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -401,18 +404,17 @@ } }, "node_modules/@aws-sdk/credential-provider-http": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-http/-/credential-provider-http-3.649.0.tgz", - "integrity": "sha512-ODAJ+AJJq6ozbns6ejGbicpsQ0dyMOpnGlg0J9J0jITQ05DKQZ581hdB8APDOZ9N8FstShP6dLZflSj8jb5fNA==", - "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/fetch-http-handler": "^3.2.5", - "@smithy/node-http-handler": "^3.2.0", - "@smithy/property-provider": "^3.1.4", - "@smithy/protocol-http": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/util-stream": "^3.1.4", + "version": "3.635.0", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-stream": "^3.1.3", "tslib": "^2.6.2" }, "engines": { @@ -420,45 +422,43 @@ } }, "node_modules/@aws-sdk/credential-provider-ini": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-ini/-/credential-provider-ini-3.650.0.tgz", - "integrity": "sha512-x2M9buZxIsKuUbuDgkGHhAKYBpn0/rYdKlwuFuOhXyyAcnhvPj0lgNF2KE4ld/GF1mKr7FF/uV3G9lM6PFaYmA==", - "dependencies": { - "@aws-sdk/credential-provider-env": "3.649.0", - "@aws-sdk/credential-provider-http": "3.649.0", - "@aws-sdk/credential-provider-process": "3.649.0", - "@aws-sdk/credential-provider-sso": "3.650.0", - "@aws-sdk/credential-provider-web-identity": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@smithy/credential-provider-imds": "^3.2.1", - "@smithy/property-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", + "version": "3.637.0", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/credential-provider-env": "3.620.1", + "@aws-sdk/credential-provider-http": "3.635.0", + "@aws-sdk/credential-provider-process": "3.620.1", + "@aws-sdk/credential-provider-sso": "3.637.0", + "@aws-sdk/credential-provider-web-identity": "3.621.0", + "@aws-sdk/types": "3.609.0", + "@smithy/credential-provider-imds": "^3.2.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sts": "^3.650.0" + "@aws-sdk/client-sts": "^3.637.0" } }, "node_modules/@aws-sdk/credential-provider-node": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-node/-/credential-provider-node-3.650.0.tgz", - "integrity": "sha512-uBra5YjzS/gWSekAogfqJfY6c+oKQkkou7Cjc4d/cpMNvQtF1IBdekJ7NaE1RfsDEz3uH1+Myd07YWZAJo/2Qw==", - "dependencies": { - "@aws-sdk/credential-provider-env": "3.649.0", - "@aws-sdk/credential-provider-http": "3.649.0", - "@aws-sdk/credential-provider-ini": "3.650.0", - "@aws-sdk/credential-provider-process": "3.649.0", - "@aws-sdk/credential-provider-sso": "3.650.0", - "@aws-sdk/credential-provider-web-identity": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@smithy/credential-provider-imds": "^3.2.1", - "@smithy/property-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", + "version": "3.637.0", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/credential-provider-env": "3.620.1", + "@aws-sdk/credential-provider-http": "3.635.0", + "@aws-sdk/credential-provider-ini": "3.637.0", + "@aws-sdk/credential-provider-process": "3.620.1", + "@aws-sdk/credential-provider-sso": "3.637.0", + "@aws-sdk/credential-provider-web-identity": "3.621.0", + "@aws-sdk/types": "3.609.0", + "@smithy/credential-provider-imds": "^3.2.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -466,14 +466,13 @@ } }, "node_modules/@aws-sdk/credential-provider-process": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-process/-/credential-provider-process-3.649.0.tgz", - "integrity": "sha512-6VYPQpEVpU+6DDS/gLoI40ppuNM5RPIEprK30qZZxnhTr5wyrGOeJ7J7wbbwPOZ5dKwta290BiJDU2ipV8Y9BQ==", - "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/property-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", + "version": "3.620.1", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -481,16 +480,15 @@ } }, "node_modules/@aws-sdk/credential-provider-sso": { - "version": "3.650.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-sso/-/credential-provider-sso-3.650.0.tgz", - "integrity": "sha512-069nkhcwximbvyGiAC6Fr2G+yrG/p1S3NQ5BZ2cMzB1hgUKo6TvgFK7nriYI4ljMQ+UWxqPwIdTqiUmn2iJmhg==", - "dependencies": { - "@aws-sdk/client-sso": "3.650.0", - "@aws-sdk/token-providers": "3.649.0", - "@aws-sdk/types": "3.649.0", - "@smithy/property-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", + "version": "3.637.0", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/client-sso": "3.637.0", + "@aws-sdk/token-providers": "3.614.0", + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -498,30 +496,28 @@ } }, "node_modules/@aws-sdk/credential-provider-web-identity": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/credential-provider-web-identity/-/credential-provider-web-identity-3.649.0.tgz", - "integrity": "sha512-XVk3WsDa0g3kQFPmnCH/LaCtGY/0R2NDv7gscYZSXiBZcG/fixasglTprgWSp8zcA0t7tEIGu9suyjz8ZwhymQ==", + "version": "3.621.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/property-provider": "^3.1.4", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sts": "^3.649.0" + "@aws-sdk/client-sts": "^3.621.0" } }, "node_modules/@aws-sdk/middleware-host-header": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-host-header/-/middleware-host-header-3.649.0.tgz", - "integrity": "sha512-PjAe2FocbicHVgNNwdSZ05upxIO7AgTPFtQLpnIAmoyzMcgv/zNB5fBn3uAnQSAeEPPCD+4SYVEUD1hw1ZBvEg==", + "version": "3.620.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -529,12 +525,11 @@ } }, "node_modules/@aws-sdk/middleware-logger": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-logger/-/middleware-logger-3.649.0.tgz", - "integrity": "sha512-qdqRx6q7lYC6KL/NT9x3ShTL0TBuxdkCczGzHzY3AnOoYUjnCDH7Vlq867O6MAvb4EnGNECFzIgtkZkQ4FhY5w==", + "version": "3.609.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -542,13 +537,12 @@ } }, "node_modules/@aws-sdk/middleware-recursion-detection": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-recursion-detection/-/middleware-recursion-detection-3.649.0.tgz", - "integrity": "sha512-IPnO4wlmaLRf6IYmJW2i8gJ2+UPXX0hDRv1it7Qf8DpBW+lGyF2rnoN7NrFX0WIxdGOlJF1RcOr/HjXb2QeXfQ==", + "version": "3.620.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -556,14 +550,13 @@ } }, "node_modules/@aws-sdk/middleware-user-agent": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/middleware-user-agent/-/middleware-user-agent-3.649.0.tgz", - "integrity": "sha512-q6sO10dnCXoxe9thobMJxekhJumzd1j6dxcE1+qJdYKHJr6yYgWbogJqrLCpWd30w0lEvnuAHK8lN2kWLdJxJw==", - "dependencies": { - "@aws-sdk/types": "3.649.0", - "@aws-sdk/util-endpoints": "3.649.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/types": "^3.4.0", + "version": "3.637.0", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@aws-sdk/util-endpoints": "3.637.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -571,15 +564,14 @@ } }, "node_modules/@aws-sdk/region-config-resolver": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/region-config-resolver/-/region-config-resolver-3.649.0.tgz", - "integrity": "sha512-xURBvdQXvRvca5Du8IlC5FyCj3pkw8Z75+373J3Wb+vyg8GjD14HfKk1Je1HCCQDyIE9VB/scYDcm9ri0ppePw==", + "version": "3.614.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", "@smithy/util-config-provider": "^3.0.0", - "@smithy/util-middleware": "^3.0.4", + "@smithy/util-middleware": "^3.0.3", "tslib": "^2.6.2" }, "engines": { @@ -587,29 +579,27 @@ } }, "node_modules/@aws-sdk/token-providers": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/token-providers/-/token-providers-3.649.0.tgz", - "integrity": "sha512-ZBqr+JuXI9RiN+4DSZykMx5gxpL8Dr3exIfFhxMiwAP3DQojwl0ub8ONjMuAjq9OvmX6n+jHZL6fBnNgnNFC8w==", - "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/property-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", + "version": "3.614.0", + "license": "Apache-2.0", + "dependencies": { + "@aws-sdk/types": "3.609.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { "node": ">=16.0.0" }, "peerDependencies": { - "@aws-sdk/client-sso-oidc": "^3.649.0" + "@aws-sdk/client-sso-oidc": "^3.614.0" } }, "node_modules/@aws-sdk/types": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/types/-/types-3.649.0.tgz", - "integrity": "sha512-PuPw8RysbhJNlaD2d/PzOTf8sbf4Dsn2b7hwyGh7YVG3S75yTpxSAZxrnhKsz9fStgqFmnw/jUfV/G+uQAeTVw==", + "version": "3.609.0", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -617,13 +607,12 @@ } }, "node_modules/@aws-sdk/util-endpoints": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-endpoints/-/util-endpoints-3.649.0.tgz", - "integrity": "sha512-bZI1Wc3R/KibdDVWFxX/N4AoJFG4VJ92Dp4WYmOrVD6VPkb8jPz7ZeiYc7YwPl8NoDjYyPneBV0lEoK/V8OKAA==", + "version": "3.637.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/types": "^3.4.0", - "@smithy/util-endpoints": "^2.1.0", + "@aws-sdk/types": "3.609.0", + "@smithy/types": "^3.3.0", + "@smithy/util-endpoints": "^2.0.5", "tslib": "^2.6.2" }, "engines": { @@ -632,8 +621,7 @@ }, "node_modules/@aws-sdk/util-locate-window": { "version": "3.568.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-locate-window/-/util-locate-window-3.568.0.tgz", - "integrity": "sha512-3nh4TINkXYr+H41QaPelCceEB2FXP3fxp93YZXB/kqJvX0U9j0N0Uk45gvsjmEPzG8XxkPEeLIfT2I1M7A6Lig==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -642,24 +630,22 @@ } }, "node_modules/@aws-sdk/util-user-agent-browser": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-browser/-/util-user-agent-browser-3.649.0.tgz", - "integrity": "sha512-IY43r256LhKAvdEVQO/FPdUyVpcZS5EVxh/WHVdNzuN1bNLoUK2rIzuZqVA0EGguvCxoXVmQv9m50GvG7cGktg==", + "version": "3.609.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/types": "^3.3.0", "bowser": "^2.11.0", "tslib": "^2.6.2" } }, "node_modules/@aws-sdk/util-user-agent-node": { - "version": "3.649.0", - "resolved": "https://registry.npmjs.org/@aws-sdk/util-user-agent-node/-/util-user-agent-node-3.649.0.tgz", - "integrity": "sha512-x5DiLpZDG/AJmCIBnE3Xhpwy35QIo3WqNiOpw6ExVs1NydbM/e90zFPSfhME0FM66D/WorigvluBxxwjxDm/GA==", + "version": "3.614.0", + "license": "Apache-2.0", "dependencies": { - "@aws-sdk/types": "3.649.0", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/types": "^3.4.0", + "@aws-sdk/types": "3.609.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -676,8 +662,7 @@ }, "node_modules/@azure/abort-controller": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-1.1.0.tgz", - "integrity": "sha512-TrRLIoSQVzfAJX9H1JeFjzAoDGcoK1IYX1UImfceTZpsyYfWr09Ss1aHW1y5TrrR3iq6RZLBwJ3E24uwPhwahw==", + "license": "MIT", "dependencies": { "tslib": "^2.2.0" }, @@ -687,8 +672,7 @@ }, "node_modules/@azure/core-auth": { "version": "1.7.2", - "resolved": "https://registry.npmjs.org/@azure/core-auth/-/core-auth-1.7.2.tgz", - "integrity": "sha512-Igm/S3fDYmnMq1uKS38Ae1/m37B3zigdlZw+kocwEhh5GjyKjPrXKO2J6rzpC1wAxrNil/jX9BJRqBshyjnF3g==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^2.0.0", "@azure/core-util": "^1.1.0", @@ -700,8 +684,7 @@ }, "node_modules/@azure/core-auth/node_modules/@azure/abort-controller": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", - "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -711,8 +694,7 @@ }, "node_modules/@azure/core-client": { "version": "1.9.2", - "resolved": "https://registry.npmjs.org/@azure/core-client/-/core-client-1.9.2.tgz", - "integrity": "sha512-kRdry/rav3fUKHl/aDLd/pDLcB+4pOFwPPTVEExuMyaI5r+JBbMWqRbCY1pn5BniDaU3lRxO9eaQ1AmSMehl/w==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^2.0.0", "@azure/core-auth": "^1.4.0", @@ -728,8 +710,7 @@ }, "node_modules/@azure/core-client/node_modules/@azure/abort-controller": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", - "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -739,8 +720,7 @@ }, "node_modules/@azure/core-http-compat": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/core-http-compat/-/core-http-compat-2.1.2.tgz", - "integrity": "sha512-5MnV1yqzZwgNLLjlizsU3QqOeQChkIXw781Fwh1xdAqJR5AA32IUaq6xv1BICJvfbHoa+JYcaij2HFkhLbNTJQ==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^2.0.0", "@azure/core-client": "^1.3.0", @@ -752,8 +732,7 @@ }, "node_modules/@azure/core-http-compat/node_modules/@azure/abort-controller": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", - "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -763,8 +742,7 @@ }, "node_modules/@azure/core-lro": { "version": "2.7.2", - "resolved": "https://registry.npmjs.org/@azure/core-lro/-/core-lro-2.7.2.tgz", - "integrity": "sha512-0YIpccoX8m/k00O7mDDMdJpbr6mf1yWo2dfmxt5A8XVZVVMz2SSKaEbMCeJRvgQ0IaSlqhjT47p4hVIRRy90xw==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^2.0.0", "@azure/core-util": "^1.2.0", @@ -777,8 +755,7 @@ }, "node_modules/@azure/core-lro/node_modules/@azure/abort-controller": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", - "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -788,8 +765,7 @@ }, "node_modules/@azure/core-paging": { "version": "1.6.2", - "resolved": "https://registry.npmjs.org/@azure/core-paging/-/core-paging-1.6.2.tgz", - "integrity": "sha512-YKWi9YuCU04B55h25cnOYZHxXYtEvQEbKST5vqRga7hWY9ydd3FZHdeQF8pyh+acWZvppw13M/LMGx0LABUVMA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -799,8 +775,7 @@ }, "node_modules/@azure/core-rest-pipeline": { "version": "1.16.3", - "resolved": "https://registry.npmjs.org/@azure/core-rest-pipeline/-/core-rest-pipeline-1.16.3.tgz", - "integrity": "sha512-VxLk4AHLyqcHsfKe4MZ6IQ+D+ShuByy+RfStKfSjxJoL3WBWq17VNmrz8aT8etKzqc2nAeIyLxScjpzsS4fz8w==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^2.0.0", "@azure/core-auth": "^1.4.0", @@ -817,8 +792,7 @@ }, "node_modules/@azure/core-rest-pipeline/node_modules/@azure/abort-controller": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", - "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -826,10 +800,41 @@ "node": ">=18.0.0" } }, + "node_modules/@azure/core-rest-pipeline/node_modules/agent-base": { + "version": "7.1.1", + "license": "MIT", + "dependencies": { + "debug": "^4.3.4" + }, + "engines": { + "node": ">= 14" + } + }, + "node_modules/@azure/core-rest-pipeline/node_modules/http-proxy-agent": { + "version": "7.0.2", + "license": "MIT", + "dependencies": { + "agent-base": "^7.1.0", + "debug": "^4.3.4" + }, + "engines": { + "node": ">= 14" + } + }, + "node_modules/@azure/core-rest-pipeline/node_modules/https-proxy-agent": { + "version": "7.0.5", + "license": "MIT", + "dependencies": { + "agent-base": "^7.0.2", + "debug": "4" + }, + "engines": { + "node": ">= 14" + } + }, "node_modules/@azure/core-tracing": { "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@azure/core-tracing/-/core-tracing-1.1.2.tgz", - "integrity": "sha512-dawW9ifvWAWmUm9/h+/UQ2jrdvjCJ7VJEuCJ6XVNudzcOwm53BFZH4Q845vjfgoUAM8ZxokvVNxNxAITc502YA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -839,8 +844,7 @@ }, "node_modules/@azure/core-util": { "version": "1.9.2", - "resolved": "https://registry.npmjs.org/@azure/core-util/-/core-util-1.9.2.tgz", - "integrity": "sha512-l1Qrqhi4x1aekkV+OlcqsJa4AnAkj5p0JV8omgwjaV9OAbP41lvrMvs+CptfetKkeEaGRGSzby7sjPZEX7+kkQ==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^2.0.0", "tslib": "^2.6.2" @@ -851,8 +855,7 @@ }, "node_modules/@azure/core-util/node_modules/@azure/abort-controller": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/@azure/abort-controller/-/abort-controller-2.1.2.tgz", - "integrity": "sha512-nBrLsEWm4J2u5LpAPjxADTlq3trDgVZZXHNKabeXZtpq3d3AbN/KGO82R87rdDz5/lYB024rtEf10/q0urNgsA==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -862,8 +865,7 @@ }, "node_modules/@azure/identity": { "version": "4.4.1", - "resolved": "https://registry.npmjs.org/@azure/identity/-/identity-4.4.1.tgz", - "integrity": "sha512-DwnG4cKFEM7S3T+9u05NstXU/HN0dk45kPOinUyNKsn5VWwpXd9sbPKEg6kgJzGbm1lMuhx9o31PVbCtM5sfBA==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^1.0.0", "@azure/core-auth": "^1.5.0", @@ -886,8 +888,7 @@ }, "node_modules/@azure/keyvault-keys": { "version": "4.8.0", - "resolved": "https://registry.npmjs.org/@azure/keyvault-keys/-/keyvault-keys-4.8.0.tgz", - "integrity": "sha512-jkuYxgkw0aaRfk40OQhFqDIupqblIOIlYESWB6DKCVDxQet1pyv86Tfk9M+5uFM0+mCs6+MUHU+Hxh3joiUn4Q==", + "license": "MIT", "dependencies": { "@azure/abort-controller": "^1.0.0", "@azure/core-auth": "^1.3.0", @@ -907,8 +908,7 @@ }, "node_modules/@azure/logger": { "version": "1.1.4", - "resolved": "https://registry.npmjs.org/@azure/logger/-/logger-1.1.4.tgz", - "integrity": "sha512-4IXXzcCdLdlXuCG+8UKEwLA1T1NHqUfanhXYHiQTn+6sfWCZXduqbtXDGceg3Ce5QxTGo7EqmbV6Bi+aqKuClQ==", + "license": "MIT", "dependencies": { "tslib": "^2.6.2" }, @@ -917,30 +917,27 @@ } }, "node_modules/@azure/msal-browser": { - "version": "3.23.0", - "resolved": "https://registry.npmjs.org/@azure/msal-browser/-/msal-browser-3.23.0.tgz", - "integrity": "sha512-+QgdMvaeEpdtgRTD7AHHq9aw8uga7mXVHV1KshO1RQ2uI5B55xJ4aEpGlg/ga3H+0arEVcRfT4ZVmX7QLXiCVw==", + "version": "3.21.0", + "license": "MIT", "dependencies": { - "@azure/msal-common": "14.14.2" + "@azure/msal-common": "14.14.1" }, "engines": { "node": ">=0.8.0" } }, "node_modules/@azure/msal-common": { - "version": "14.14.2", - "resolved": "https://registry.npmjs.org/@azure/msal-common/-/msal-common-14.14.2.tgz", - "integrity": "sha512-XV0P5kSNwDwCA/SjIxTe9mEAsKB0NqGNSuaVrkCCE2lAyBr/D6YtD80Vkdp4tjWnPFwjzkwldjr1xU/facOJog==", + "version": "14.14.1", + "license": "MIT", "engines": { "node": ">=0.8.0" } }, "node_modules/@azure/msal-node": { - "version": "2.13.1", - "resolved": "https://registry.npmjs.org/@azure/msal-node/-/msal-node-2.13.1.tgz", - "integrity": "sha512-sijfzPNorKt6+9g1/miHwhj6Iapff4mPQx1azmmZExgzUROqWTM1o3ACyxDja0g47VpowFy/sxTM/WsuCyXTiw==", + "version": "2.13.0", + "license": "MIT", "dependencies": { - "@azure/msal-common": "14.14.2", + "@azure/msal-common": "14.14.1", "jsonwebtoken": "^9.0.0", "uuid": "^8.3.0" }, @@ -948,11 +945,16 @@ "node": ">=16" } }, + "node_modules/@azure/msal-node/node_modules/uuid": { + "version": "8.3.2", + "license": "MIT", + "bin": { + "uuid": "dist/bin/uuid" + } + }, "node_modules/@babel/code-frame": { "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", - "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/highlight": "^7.24.7", "picocolors": "^1.0.0" @@ -962,19 +964,15 @@ } }, "node_modules/@babel/compat-data": { - "version": "7.25.4", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.25.4.tgz", - "integrity": "sha512-+LGRog6RAsCJrrrg/IO6LGmpphNe5DiK30dGjCoxxeGv49B10/3XYGxPsAwrDlMFcFEvdAUavDT8r9k/hSyQqQ==", - "dev": true, + "version": "7.25.2", + "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/core": { "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.25.2.tgz", - "integrity": "sha512-BBt3opiCOxUr9euZ5/ro/Xv8/V7yJ5bjYMqG/C1YAo8MIKAnumZalCN+msbci3Pigy4lIQfPUpfMM27HMGaYEA==", - "dev": true, + "license": "MIT", "dependencies": { "@ampproject/remapping": "^2.2.0", "@babel/code-frame": "^7.24.7", @@ -1002,20 +1000,16 @@ }, "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, + "license": "ISC", "bin": { "semver": "bin/semver.js" } }, "node_modules/@babel/generator": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.6.tgz", - "integrity": "sha512-VPC82gr1seXOpkjAAKoLhP50vx4vGNlF4msF64dSFq1P8RfB+QAuJWGHPXXPc8QyfVWwwB/TNNU4+ayZmHNbZw==", - "dev": true, + "version": "7.25.0", + "license": "MIT", "dependencies": { - "@babel/types": "^7.25.6", + "@babel/types": "^7.25.0", "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.25", "jsesc": "^2.5.1" @@ -1026,9 +1020,7 @@ }, "node_modules/@babel/helper-compilation-targets": { "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.25.2.tgz", - "integrity": "sha512-U2U5LsSaZ7TAt3cfaymQ8WHh0pxvdHoEk6HVpaexxixjyEquMh0L0YNJNM6CTGKMXV1iksi0iZkGw4AcFkPaaw==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/compat-data": "^7.25.2", "@babel/helper-validator-option": "^7.24.8", @@ -1042,33 +1034,21 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { "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, + "license": "ISC", "dependencies": { "yallist": "^3.0.2" } }, "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, + "license": "ISC", "bin": { "semver": "bin/semver.js" } }, - "node_modules/@babel/helper-compilation-targets/node_modules/yallist": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/yallist/-/yallist-3.1.1.tgz", - "integrity": "sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g==", - "dev": true - }, "node_modules/@babel/helper-module-imports": { "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", - "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", "@babel/types": "^7.24.7" @@ -1079,9 +1059,7 @@ }, "node_modules/@babel/helper-module-transforms": { "version": "7.25.2", - "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.25.2.tgz", - "integrity": "sha512-BjyRAbix6j/wv83ftcVJmBt72QtHI56C7JXZoG2xATiLpmoC7dpd8WnkikExHDVPpi/3qCmO6WY1EaXOluiecQ==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/helper-module-imports": "^7.24.7", "@babel/helper-simple-access": "^7.24.7", @@ -1097,18 +1075,14 @@ }, "node_modules/@babel/helper-plugin-utils": { "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.24.8.tgz", - "integrity": "sha512-FFWx5142D8h2Mgr/iPVGH5G7w6jDn4jUSpZTyDnQO0Yn7Ks2Kuz6Pci8H6MPCoUJegd/UZQ3tAvfLCxQSnWWwg==", - "dev": true, + "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-simple-access": { "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.24.7.tgz", - "integrity": "sha512-zBAIvbCMh5Ts+b86r/CjU+4XGYIs+R1j951gxI3KmmxBMhCg4oQMsv6ZXQ64XOm/cvzfU1FmoCyt6+owc5QMYg==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", "@babel/types": "^7.24.7" @@ -1119,39 +1093,31 @@ }, "node_modules/@babel/helper-string-parser": { "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", - "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", - "dev": true, + "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-identifier": { "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", - "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", - "dev": true, + "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-option": { "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.24.8.tgz", - "integrity": "sha512-xb8t9tD1MHLungh/AIoWYN+gVHaB9kwlu8gffXGSt3FFEIT7RjS+xWbc2vUD1UTZdIpKj/ab3rdqJ7ufngyi2Q==", - "dev": true, + "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helpers": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.25.6.tgz", - "integrity": "sha512-Xg0tn4HcfTijTwfDwYlvVCl43V6h4KyVVX2aEm4qdO/PC6L2YvzLHFdmxhoeSA3eslcE6+ZVXHgWwopXYLNq4Q==", - "dev": true, + "version": "7.25.0", + "license": "MIT", "dependencies": { "@babel/template": "^7.25.0", - "@babel/types": "^7.25.6" + "@babel/types": "^7.25.0" }, "engines": { "node": ">=6.9.0" @@ -1159,9 +1125,7 @@ }, "node_modules/@babel/highlight": { "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", - "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/helper-validator-identifier": "^7.24.7", "chalk": "^2.4.2", @@ -1174,9 +1138,7 @@ }, "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, + "license": "MIT", "dependencies": { "color-convert": "^1.9.0" }, @@ -1186,9 +1148,7 @@ }, "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, + "license": "MIT", "dependencies": { "ansi-styles": "^3.2.1", "escape-string-regexp": "^1.0.5", @@ -1200,42 +1160,32 @@ }, "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, + "license": "MIT", "dependencies": { "color-name": "1.1.3" } }, "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 + "license": "MIT" }, "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, + "license": "MIT", "engines": { "node": ">=0.8.0" } }, "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, + "license": "MIT", "engines": { "node": ">=4" } }, "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, + "license": "MIT", "dependencies": { "has-flag": "^3.0.0" }, @@ -1244,12 +1194,10 @@ } }, "node_modules/@babel/parser": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.6.tgz", - "integrity": "sha512-trGdfBdbD0l1ZPmcJ83eNxB9rbEax4ALFTF7fN386TMYbeCQbyme5cOEXQhbGXKebwGaB/J52w1mrklMcbgy6Q==", - "dev": true, + "version": "7.25.3", + "license": "MIT", "dependencies": { - "@babel/types": "^7.25.6" + "@babel/types": "^7.25.2" }, "bin": { "parser": "bin/babel-parser.js" @@ -1260,9 +1208,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1272,9 +1218,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1284,9 +1228,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.12.13" }, @@ -1294,41 +1236,9 @@ "@babel/core": "^7.0.0-0" } }, - "node_modules/@babel/plugin-syntax-class-static-block": { - "version": "7.14.5", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz", - "integrity": "sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw==", - "dev": true, - "dependencies": { - "@babel/helper-plugin-utils": "^7.14.5" - }, - "engines": { - "node": ">=6.9.0" - }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" - } - }, - "node_modules/@babel/plugin-syntax-import-attributes": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-import-attributes/-/plugin-syntax-import-attributes-7.25.6.tgz", - "integrity": "sha512-sXaDXaJN9SNLymBdlWFA+bjzBhFD617ZaFiY13dGt7TVslVvVgA6fkZOP7Ki3IGElC45lwHdOTrCtKZGVAWeLQ==", - "dev": true, - "dependencies": { - "@babel/helper-plugin-utils": "^7.24.8" - }, - "engines": { - "node": ">=6.9.0" - }, - "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -1338,9 +1248,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1350,9 +1258,7 @@ }, "node_modules/@babel/plugin-syntax-jsx": { "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.24.7.tgz", - "integrity": "sha512-6ddciUPe/mpMnOKv/U+RSd2vvVy+Yw/JfBB0ZHYjEZt9NLHmCUylNYlsbqCCS1Bffjlb0fCwC9Vqz+sBz6PsiQ==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" }, @@ -1365,9 +1271,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -1377,9 +1281,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1389,9 +1291,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" }, @@ -1401,9 +1301,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1413,9 +1311,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1425,9 +1321,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" }, @@ -1435,26 +1329,9 @@ "@babel/core": "^7.0.0-0" } }, - "node_modules/@babel/plugin-syntax-private-property-in-object": { - "version": "7.14.5", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz", - "integrity": "sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg==", - "dev": true, - "dependencies": { - "@babel/helper-plugin-utils": "^7.14.5" - }, - "engines": { - "node": ">=6.9.0" - }, - "peerDependencies": { - "@babel/core": "^7.0.0-0" - } - }, "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, + "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.14.5" }, @@ -1466,12 +1343,10 @@ } }, "node_modules/@babel/plugin-syntax-typescript": { - "version": "7.25.4", - "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.25.4.tgz", - "integrity": "sha512-uMOCoHVU52BsSWxPOMVv5qKRdeSlPuImUCB2dlPuBSU+W2/ROE7/Zg8F2Kepbk+8yBa68LlRKxO+xgEVWorsDg==", - "dev": true, + "version": "7.24.7", + "license": "MIT", "dependencies": { - "@babel/helper-plugin-utils": "^7.24.8" + "@babel/helper-plugin-utils": "^7.24.7" }, "engines": { "node": ">=6.9.0" @@ -1482,9 +1357,7 @@ }, "node_modules/@babel/template": { "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", - "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", "@babel/parser": "^7.25.0", @@ -1495,16 +1368,14 @@ } }, "node_modules/@babel/traverse": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.6.tgz", - "integrity": "sha512-9Vrcx5ZW6UwK5tvqsj0nGpp/XzqthkT0dqIc9g1AdtygFToNtTF67XzYS//dm+SAK9cp3B9R4ZO/46p63SCjlQ==", - "dev": true, + "version": "7.25.3", + "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", - "@babel/generator": "^7.25.6", - "@babel/parser": "^7.25.6", + "@babel/generator": "^7.25.0", + "@babel/parser": "^7.25.3", "@babel/template": "^7.25.0", - "@babel/types": "^7.25.6", + "@babel/types": "^7.25.2", "debug": "^4.3.1", "globals": "^11.1.0" }, @@ -1514,18 +1385,14 @@ }, "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, + "license": "MIT", "engines": { "node": ">=4" } }, "node_modules/@babel/types": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.6.tgz", - "integrity": "sha512-/l42B1qxpG6RdfYf343Uw1vmDjeNhneUXtzhojE7pDgfpEypmRhI6j1kr17XCVv4Cgl9HdAiQY2x0GwKm7rWCw==", - "dev": true, + "version": "7.25.2", + "license": "MIT", "dependencies": { "@babel/helper-string-parser": "^7.24.8", "@babel/helper-validator-identifier": "^7.24.7", @@ -1537,16 +1404,13 @@ }, "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 + "license": "MIT" }, "node_modules/@bufbuild/buf": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf/-/buf-1.41.0.tgz", - "integrity": "sha512-6pN2fqMrPqnIkrC1q9KpXpu7fv3Rul0ZPhT4MSYYj+8VcyR3kbLVk6K+CzzPvYhr4itfotnI3ZVGQ/X/vupECg==", + "version": "1.37.0", "dev": true, "hasInstallScript": true, + "license": "Apache-2.0", "bin": { "buf": "bin/buf", "protoc-gen-buf-breaking": "bin/protoc-gen-buf-breaking", @@ -1556,22 +1420,21 @@ "node": ">=12" }, "optionalDependencies": { - "@bufbuild/buf-darwin-arm64": "1.41.0", - "@bufbuild/buf-darwin-x64": "1.41.0", - "@bufbuild/buf-linux-aarch64": "1.41.0", - "@bufbuild/buf-linux-x64": "1.41.0", - "@bufbuild/buf-win32-arm64": "1.41.0", - "@bufbuild/buf-win32-x64": "1.41.0" + "@bufbuild/buf-darwin-arm64": "1.37.0", + "@bufbuild/buf-darwin-x64": "1.37.0", + "@bufbuild/buf-linux-aarch64": "1.37.0", + "@bufbuild/buf-linux-x64": "1.37.0", + "@bufbuild/buf-win32-arm64": "1.37.0", + "@bufbuild/buf-win32-x64": "1.37.0" } }, "node_modules/@bufbuild/buf-darwin-arm64": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-arm64/-/buf-darwin-arm64-1.41.0.tgz", - "integrity": "sha512-+G5DwpIgnm0AkqgxORxoYXVT0RGDcw8P4SXFXcovgvDBkk9rPvEI1dbPF83n3SUxzcu2A2OxC7DxlXszWIh2Gw==", + "version": "1.37.0", "cpu": [ "arm64" ], "dev": true, + "license": "Apache-2.0", "optional": true, "os": [ "darwin" @@ -1580,120 +1443,37 @@ "node": ">=12" } }, - "node_modules/@bufbuild/buf-darwin-x64": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-darwin-x64/-/buf-darwin-x64-1.41.0.tgz", - "integrity": "sha512-qjkJ/LAWqNk3HX65n+JTt18WtKrhrrAhIu3Dpfbe0eujsxafFZKoPzlWJYybxvsaF9CdEyMMm/OalBPpoosMOA==", - "cpu": [ - "x64" - ], + "node_modules/@bufbuild/protobuf": { + "version": "2.0.0", + "license": "(Apache-2.0 AND BSD-3-Clause)" + }, + "node_modules/@bufbuild/protoc-gen-es": { + "version": "2.0.0", "dev": true, - "optional": true, - "os": [ - "darwin" - ], + "license": "Apache-2.0", + "dependencies": { + "@bufbuild/protobuf": "^2.0.0", + "@bufbuild/protoplugin": "2.0.0" + }, + "bin": { + "protoc-gen-es": "bin/protoc-gen-es" + }, "engines": { - "node": ">=12" - } - }, - "node_modules/@bufbuild/buf-linux-aarch64": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-aarch64/-/buf-linux-aarch64-1.41.0.tgz", - "integrity": "sha512-5E+MLAF4QHPwAjwVVRRP3Is2U3zpIpQQR7S3di9HlKACbgvefJEBrUfRqQZvHrMuuynQRqjFuZD16Sfvxn9rCQ==", - "cpu": [ - "arm64" - ], - "dev": true, - "optional": true, - "os": [ - "linux" - ], - "engines": { - "node": ">=12" - } - }, - "node_modules/@bufbuild/buf-linux-x64": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-linux-x64/-/buf-linux-x64-1.41.0.tgz", - "integrity": "sha512-W4T+uqmdtypzzatv6OXjUzGacZiNzGECogr+qDkJF38MSZd3jHXhTEN2KhRckl3i9rRAnfHBwG68BjCTxxBCOQ==", - "cpu": [ - "x64" - ], - "dev": true, - "optional": true, - "os": [ - "linux" - ], - "engines": { - "node": ">=12" - } - }, - "node_modules/@bufbuild/buf-win32-arm64": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-arm64/-/buf-win32-arm64-1.41.0.tgz", - "integrity": "sha512-OsRVoTZHJZYGIphAwaRqcCeYR9Sk5VEMjpCJiFt/dkHxx2acKH4u/7O+633gcCxQL8EnsU2l8AfdbW7sQaOvlg==", - "cpu": [ - "arm64" - ], - "dev": true, - "optional": true, - "os": [ - "win32" - ], - "engines": { - "node": ">=12" - } - }, - "node_modules/@bufbuild/buf-win32-x64": { - "version": "1.41.0", - "resolved": "https://registry.npmjs.org/@bufbuild/buf-win32-x64/-/buf-win32-x64-1.41.0.tgz", - "integrity": "sha512-2KJLp7Py0GsfRjDxwBzS17RMpaYFGCvzkwY5CtxfPMw8cg6cE7E36r+vcjHh5dBOj/CumaiXLTwxhCSBtp0V1g==", - "cpu": [ - "x64" - ], - "dev": true, - "optional": true, - "os": [ - "win32" - ], - "engines": { - "node": ">=12" - } - }, - "node_modules/@bufbuild/protobuf": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@bufbuild/protobuf/-/protobuf-2.0.0.tgz", - "integrity": "sha512-sw2JhwJyvyL0zlhG61aDzOVryEfJg2PDZFSV7i7IdC7nAE41WuXCru3QWLGiP87At0BMzKOoKO/FqEGoKygGZQ==" - }, - "node_modules/@bufbuild/protoc-gen-es": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@bufbuild/protoc-gen-es/-/protoc-gen-es-2.0.0.tgz", - "integrity": "sha512-xXDrDFjSD97jGu9RMA8O4Myj66DdCLOFeYG7Z0mX1aCE9hayymfor+BKgKCQouUfty6CYv8709FPYpEtDurLFQ==", - "dev": true, - "dependencies": { - "@bufbuild/protobuf": "^2.0.0", - "@bufbuild/protoplugin": "2.0.0" - }, - "bin": { - "protoc-gen-es": "bin/protoc-gen-es" - }, - "engines": { - "node": ">=14" - }, - "peerDependencies": { - "@bufbuild/protobuf": "2.0.0" - }, - "peerDependenciesMeta": { - "@bufbuild/protobuf": { - "optional": true - } + "node": ">=14" + }, + "peerDependencies": { + "@bufbuild/protobuf": "2.0.0" + }, + "peerDependenciesMeta": { + "@bufbuild/protobuf": { + "optional": true + } } }, "node_modules/@bufbuild/protoplugin": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@bufbuild/protoplugin/-/protoplugin-2.0.0.tgz", - "integrity": "sha512-y77IarGNbJfr5YZHQ+biO0O9B8rXicKLLOFRJu9mKr9LYE5/CLIKqonIBfkc/lggCK2YOtinjZJEYb39HymIog==", "dev": true, + "license": "Apache-2.0", "dependencies": { "@bufbuild/protobuf": "2.0.0", "@typescript/vfs": "^1.5.2", @@ -1702,9 +1482,8 @@ }, "node_modules/@bufbuild/protoplugin/node_modules/typescript": { "version": "5.4.5", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.5.tgz", - "integrity": "sha512-vcI4UpRgg81oIRUFwR0WSIHKt11nJ7SAVlYNIu+QpqeyXP+gpQJy/Z4+F0aGxSE4MqwjyXvW/TzgkLAx2AGHwQ==", "dev": true, + "license": "Apache-2.0", "bin": { "tsc": "bin/tsc", "tsserver": "bin/tsserver" @@ -1719,16 +1498,14 @@ }, "node_modules/@criteria/json-pointer": { "version": "0.2.1", - "resolved": "https://registry.npmjs.org/@criteria/json-pointer/-/json-pointer-0.2.1.tgz", - "integrity": "sha512-+UcEXcbZLkmhB4vuuuLAfnWD7cGmUGrZxzldFFdLwFKglkGuLiE4YTBX0gvEQuVHTwMN/f1R/0TB3YRzf5/BPw==", + "license": "MIT", "engines": { "node": ">=18.12.1" } }, "node_modules/@criteria/json-schema": { "version": "0.10.0", - "resolved": "https://registry.npmjs.org/@criteria/json-schema/-/json-schema-0.10.0.tgz", - "integrity": "sha512-Y+gLIVPdgflQ3zeayUcfYjHOf+KJOp8dx5LWpLfy3XJ0P61JxU5srk2cGpVZYPJNTnixm8tBwb8T26DCMjw+hw==", + "license": "MIT", "dependencies": { "@criteria/json-pointer": "^0.2.1", "toad-uri-js": "^5.0.1" @@ -1739,8 +1516,7 @@ }, "node_modules/@criteria/json-schema-validation": { "version": "0.10.0", - "resolved": "https://registry.npmjs.org/@criteria/json-schema-validation/-/json-schema-validation-0.10.0.tgz", - "integrity": "sha512-QqkO0uEAjMIEdybZPSGZEOa2HExAr83lE5ya3bRTVFMByb9sXq7JK7ZyHHAhSOWQyUN1Tgkk/sozfLygG5VOuw==", + "license": "MIT", "dependencies": { "@criteria/json-pointer": "^0.2.1", "@criteria/json-schema": "^0.10.0", @@ -1755,9 +1531,8 @@ }, "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, + "license": "MIT", "dependencies": { "eslint-visitor-keys": "^3.3.0" }, @@ -1770,18 +1545,16 @@ }, "node_modules/@eslint-community/regexpp": { "version": "4.11.0", - "resolved": "https://registry.npmjs.org/@eslint-community/regexpp/-/regexpp-4.11.0.tgz", - "integrity": "sha512-G/M/tIiMrTAxEWRfLfQJMmGNX28IxBg4PBz8XqQhqUHLFI6TL2htpIB1iQCj144V5ee/JaKyT9/WZ0MGZWfA7A==", "dev": true, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "ajv": "^6.12.4", "debug": "^4.3.2", @@ -1802,9 +1575,8 @@ }, "node_modules/@eslint/eslintrc/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, + "license": "MIT", "dependencies": { "fast-deep-equal": "^3.1.1", "fast-json-stable-stringify": "^2.0.0", @@ -1818,29 +1590,25 @@ }, "node_modules/@eslint/eslintrc/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 + "dev": true, + "license": "MIT" }, "node_modules/@eslint/js": { - "version": "9.10.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.10.0.tgz", - "integrity": "sha512-fuXtbiP5GWIn8Fz+LWoOMVf/Jxm+aajZYkhi6CuEm4SxymFM+eUWzbO9qXT+L0iCkL5+KGYMCSGxo686H19S1g==", + "version": "9.9.0", "dev": true, + "license": "MIT", "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.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 + "dev": true, + "license": "MIT" }, "node_modules/@google-cloud/kms": { "version": "4.5.0", - "resolved": "https://registry.npmjs.org/@google-cloud/kms/-/kms-4.5.0.tgz", - "integrity": "sha512-i2vC0DI7bdfEhQszqASTw0KVvbB7HsO2CwTBod423NawAu7FWi+gVVa7NLfXVNGJaZZayFfci2Hu+om/HmyEjQ==", + "license": "Apache-2.0", "dependencies": { "google-gax": "^4.0.3" }, @@ -1849,9 +1617,8 @@ } }, "node_modules/@grpc/grpc-js": { - "version": "1.11.2", - "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.11.2.tgz", - "integrity": "sha512-DWp92gDD7/Qkj7r8kus6/HCINeo3yPZWZ3paKgDgsbKbSpoxKg1yvN8xe2Q8uE3zOsPe3bX8FQX2+XValq2yTw==", + "version": "1.11.1", + "license": "Apache-2.0", "dependencies": { "@grpc/proto-loader": "^0.7.13", "@js-sdsl/ordered-map": "^4.4.2" @@ -1862,8 +1629,7 @@ }, "node_modules/@grpc/proto-loader": { "version": "0.7.13", - "resolved": "https://registry.npmjs.org/@grpc/proto-loader/-/proto-loader-0.7.13.tgz", - "integrity": "sha512-AiXO/bfe9bmxBjxxtYxFAXGZvMaN5s8kO+jBHAJCON8rJoB5YS/D6X7ZNc6XQkuHNmyl4CYaMI1fJ/Gn27RGGw==", + "license": "Apache-2.0", "dependencies": { "lodash.camelcase": "^4.3.0", "long": "^5.0.0", @@ -1879,15 +1645,59 @@ }, "node_modules/@hackbg/miscreant-esm": { "version": "0.3.2-patch.3", - "resolved": "https://registry.npmjs.org/@hackbg/miscreant-esm/-/miscreant-esm-0.3.2-patch.3.tgz", - "integrity": "sha512-CxWupG9uSc0e0toliMBc+ZSC2cXuHqcSgvwCgCiWXkpF6adzcttNFr6KyumI3utO/WQKQrdgq93LcQEueBjTHQ==" + "license": "MIT" + }, + "node_modules/@hapi/boom": { + "version": "10.0.1", + "resolved": "https://registry.npmjs.org/@hapi/boom/-/boom-10.0.1.tgz", + "integrity": "sha512-ERcCZaEjdH3OgSJlyjVk8pHIFeus91CjKP3v+MpgBNp5IvGzP2l/bRiD78nqYcKPaZdbKkK5vDBVPd2ohHBlsA==", + "license": "BSD-3-Clause", + "dependencies": { + "@hapi/hoek": "^11.0.2" + } + }, + "node_modules/@hapi/bourne": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@hapi/bourne/-/bourne-3.0.0.tgz", + "integrity": "sha512-Waj1cwPXJDucOib4a3bAISsKJVb15MKi9IvmTI/7ssVEm6sywXGjVJDhl6/umt1pK1ZS7PacXU3A1PmFKHEZ2w==", + "license": "BSD-3-Clause" + }, + "node_modules/@hapi/hoek": { + "version": "11.0.4", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-11.0.4.tgz", + "integrity": "sha512-PnsP5d4q7289pS2T2EgGz147BFJ2Jpb4yrEdkpz2IhgEUzos1S7HTl7ezWh1yfYzYlj89KzLdCRkqsP6SIryeQ==", + "license": "BSD-3-Clause" + }, + "node_modules/@hapi/topo": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/@hapi/topo/-/topo-5.1.0.tgz", + "integrity": "sha512-foQZKJig7Ob0BMAYBfcJk8d77QtOe7Wo4ox7ff1lQYoNNAb6jwcY1ncdoy2e9wQZzvNy7ODZCYJkK8kzmcAnAg==", + "license": "BSD-3-Clause", + "dependencies": { + "@hapi/hoek": "^9.0.0" + } + }, + "node_modules/@hapi/topo/node_modules/@hapi/hoek": { + "version": "9.3.0", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-9.3.0.tgz", + "integrity": "sha512-/c6rf4UJlmHlC9b5BaNvzAcFv7HZ2QHaV0D4/HNlBdvFnvQq8RI4kYdhyPCl7Xj+oWvTWQ8ujhqS53LIgAe6KQ==", + "license": "BSD-3-Clause" + }, + "node_modules/@hapi/wreck": { + "version": "18.1.0", + "resolved": "https://registry.npmjs.org/@hapi/wreck/-/wreck-18.1.0.tgz", + "integrity": "sha512-0z6ZRCmFEfV/MQqkQomJ7sl/hyxvcZM7LtuVqN3vdAO4vM9eBbowl0kaqQj9EJJQab+3Uuh1GxbGIBFy4NfJ4w==", + "license": "BSD-3-Clause", + "dependencies": { + "@hapi/boom": "^10.0.1", + "@hapi/bourne": "^3.0.0", + "@hapi/hoek": "^11.0.2" + } }, "node_modules/@humanwhocodes/config-array": { "version": "0.11.14", - "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.14.tgz", - "integrity": "sha512-3T8LkOmg45BV5FICb15QQMsyUSWrQ8AygVfC7ZG32zOalnqrilm018ZVCw0eapXux8FtA33q8PSRSstjee3jSg==", - "deprecated": "Use @eslint/config-array instead", "dev": true, + "license": "Apache-2.0", "dependencies": { "@humanwhocodes/object-schema": "^2.0.2", "debug": "^4.3.1", @@ -1899,9 +1709,8 @@ }, "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, + "license": "Apache-2.0", "engines": { "node": ">=12.22" }, @@ -1912,16 +1721,12 @@ }, "node_modules/@humanwhocodes/object-schema": { "version": "2.0.3", - "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.3.tgz", - "integrity": "sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA==", - "deprecated": "Use @eslint/object-schema instead", - "dev": true + "dev": true, + "license": "BSD-3-Clause" }, "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", - "integrity": "sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ==", - "dev": true, + "license": "ISC", "dependencies": { "camelcase": "^5.3.1", "find-up": "^4.1.0", @@ -1935,18 +1740,14 @@ }, "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, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "locate-path": "^5.0.0", "path-exists": "^4.0.0" @@ -1957,9 +1758,7 @@ }, "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, + "license": "MIT", "dependencies": { "argparse": "^1.0.7", "esprima": "^4.0.0" @@ -1970,9 +1769,7 @@ }, "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, + "license": "MIT", "dependencies": { "p-locate": "^4.1.0" }, @@ -1982,9 +1779,7 @@ }, "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, + "license": "MIT", "dependencies": { "p-try": "^2.0.0" }, @@ -1997,9 +1792,7 @@ }, "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, + "license": "MIT", "dependencies": { "p-limit": "^2.2.0" }, @@ -2009,33 +1802,25 @@ }, "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, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/@istanbuljs/load-nyc-config/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 + "license": "BSD-3-Clause" }, "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, + "license": "MIT", "engines": { "node": ">=8" } }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "@types/node": "*", @@ -2050,9 +1835,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", "@jest/reporters": "^29.7.0", @@ -2097,9 +1880,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/fake-timers": "^29.7.0", "@jest/types": "^29.6.3", @@ -2112,9 +1893,7 @@ }, "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, + "license": "MIT", "dependencies": { "expect": "^29.7.0", "jest-snapshot": "^29.7.0" @@ -2125,9 +1904,7 @@ }, "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, + "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3" }, @@ -2137,9 +1914,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "@sinonjs/fake-timers": "^10.0.2", @@ -2154,9 +1929,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", "@jest/expect": "^29.7.0", @@ -2169,9 +1942,7 @@ }, "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, + "license": "MIT", "dependencies": { "@bcoe/v8-coverage": "^0.2.3", "@jest/console": "^29.7.0", @@ -2212,9 +1983,7 @@ }, "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, + "license": "MIT", "dependencies": { "@sinclair/typebox": "^0.27.8" }, @@ -2224,9 +1993,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jridgewell/trace-mapping": "^0.3.18", "callsites": "^3.0.0", @@ -2238,9 +2005,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", "@jest/types": "^29.6.3", @@ -2253,9 +2018,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", "graceful-fs": "^4.2.9", @@ -2268,9 +2031,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", "@jest/types": "^29.6.3", @@ -2294,9 +2055,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", "@types/istanbul-lib-coverage": "^2.0.0", @@ -2311,9 +2070,7 @@ }, "node_modules/@jridgewell/gen-mapping": { "version": "0.3.5", - "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", - "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", - "dev": true, + "license": "MIT", "dependencies": { "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", @@ -2325,33 +2082,25 @@ }, "node_modules/@jridgewell/resolve-uri": { "version": "3.1.2", - "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", - "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", - "dev": true, + "license": "MIT", "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/set-array": { "version": "1.2.1", - "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", - "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", - "dev": true, + "license": "MIT", "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/sourcemap-codec": { "version": "1.5.0", - "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", - "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", - "dev": true + "license": "MIT" }, "node_modules/@jridgewell/trace-mapping": { "version": "0.3.25", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", - "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", - "dev": true, + "license": "MIT", "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", "@jridgewell/sourcemap-codec": "^1.4.14" @@ -2359,8 +2108,7 @@ }, "node_modules/@js-sdsl/ordered-map": { "version": "4.4.2", - "resolved": "https://registry.npmjs.org/@js-sdsl/ordered-map/-/ordered-map-4.4.2.tgz", - "integrity": "sha512-iUKgm52T8HOE/makSxjqoWhe95ZJA1/G1sYsGev2JDKUSS14KAgg1LHb+Ba+IPow0xflbnSkOsZcO08C7w1gYw==", + "license": "MIT", "funding": { "type": "opencollective", "url": "https://opencollective.com/js-sdsl" @@ -2368,9 +2116,8 @@ }, "node_modules/@jsdoc/salty": { "version": "0.2.8", - "resolved": "https://registry.npmjs.org/@jsdoc/salty/-/salty-0.2.8.tgz", - "integrity": "sha512-5e+SFVavj1ORKlKaKr2BmTOekmXbelU7dC0cDkQLqag7xfuTPuGMUFx7KWJuv4bYZrTsoL2Z18VVCOKYxzoHcg==", "dev": true, + "license": "Apache-2.0", "dependencies": { "lodash": "^4.17.21" }, @@ -2380,8 +2127,7 @@ }, "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==", + "license": "BSD-3-Clause", "dependencies": { "detect-libc": "^2.0.0", "https-proxy-agent": "^5.0.0", @@ -2397,34 +2143,10 @@ "node-pre-gyp": "bin/node-pre-gyp" } }, - "node_modules/@mapbox/node-pre-gyp/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/@mapbox/node-pre-gyp/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/@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, + "license": "MIT", "dependencies": { "@nodelib/fs.stat": "2.0.5", "run-parallel": "^1.1.9" @@ -2435,18 +2157,16 @@ }, "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, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "@nodelib/fs.scandir": "2.1.5", "fastq": "^1.6.0" @@ -2457,9 +2177,8 @@ }, "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, + "license": "ISC", "dependencies": { "@gar/promisify": "^1.1.3", "semver": "^7.3.5" @@ -2470,10 +2189,8 @@ }, "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, + "license": "MIT", "dependencies": { "mkdirp": "^1.0.4", "rimraf": "^3.0.2" @@ -2484,8 +2201,7 @@ }, "node_modules/@postman/form-data": { "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@postman/form-data/-/form-data-3.1.1.tgz", - "integrity": "sha512-vjh8Q2a8S6UCm/KKs31XFJqEEgmbjBmpPNVV2eVav6905wyFAwaUOBGA1NPBI4ERH9MMZc6w0umFgM6WbEPMdg==", + "license": "MIT", "dependencies": { "asynckit": "^0.4.0", "combined-stream": "^1.0.8", @@ -2497,8 +2213,7 @@ }, "node_modules/@postman/tough-cookie": { "version": "4.1.3-postman.1", - "resolved": "https://registry.npmjs.org/@postman/tough-cookie/-/tough-cookie-4.1.3-postman.1.tgz", - "integrity": "sha512-txpgUqZOnWYnUHZpHjkfb0IwVH4qJmyq77pPnJLlfhMtdCLMFTEeQHlzQiK906aaNCe4NEB5fGJHo9uzGbFMeA==", + "license": "BSD-3-Clause", "dependencies": { "psl": "^1.1.33", "punycode": "^2.1.1", @@ -2511,8 +2226,7 @@ }, "node_modules/@postman/tunnel-agent": { "version": "0.6.4", - "resolved": "https://registry.npmjs.org/@postman/tunnel-agent/-/tunnel-agent-0.6.4.tgz", - "integrity": "sha512-CJJlq8V7rNKhAw4sBfjixKpJW00SHqebqNUQKxMoepgeWZIbdPcD+rguRcivGhS4N12PymDcKgUgSD4rVC+RjQ==", + "license": "Apache-2.0", "dependencies": { "safe-buffer": "^5.0.1" }, @@ -2522,28 +2236,23 @@ }, "node_modules/@protobufjs/aspromise": { "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@protobufjs/aspromise/-/aspromise-1.1.2.tgz", - "integrity": "sha512-j+gKExEuLmKwvz3OgROXtrJ2UG2x8Ch2YZUxahh+s1F2HZ+wAceUNLkvy6zKCPVRkU++ZWQrdxsUeQXmcg4uoQ==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/base64": { "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@protobufjs/base64/-/base64-1.1.2.tgz", - "integrity": "sha512-AZkcAA5vnN/v4PDqKyMR5lx7hZttPDgClv83E//FMNhR2TMcLUhfRUBHCmSl0oi9zMgDDqRUJkSxO3wm85+XLg==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/codegen": { "version": "2.0.4", - "resolved": "https://registry.npmjs.org/@protobufjs/codegen/-/codegen-2.0.4.tgz", - "integrity": "sha512-YyFaikqM5sH0ziFZCN3xDC7zeGaB/d0IUb9CATugHWbd1FRFwWwt4ld4OYMPWu5a3Xe01mGAULCdqhMlPl29Jg==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/eventemitter": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@protobufjs/eventemitter/-/eventemitter-1.1.0.tgz", - "integrity": "sha512-j9ednRT81vYJ9OfVuXG6ERSTdEL1xVsNgqpkxMsbIabzSo3goCjDIveeGv5d03om39ML71RdmrGNjG5SReBP/Q==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/fetch": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@protobufjs/fetch/-/fetch-1.1.0.tgz", - "integrity": "sha512-lljVXpqXebpsijW71PZaCYeIcE5on1w5DlQy5WH6GLbFryLUrBD4932W/E2BSpfRJWseIL4v/KPgBFxDOIdKpQ==", + "license": "BSD-3-Clause", "dependencies": { "@protobufjs/aspromise": "^1.1.1", "@protobufjs/inquire": "^1.1.0" @@ -2551,59 +2260,74 @@ }, "node_modules/@protobufjs/float": { "version": "1.0.2", - "resolved": "https://registry.npmjs.org/@protobufjs/float/-/float-1.0.2.tgz", - "integrity": "sha512-Ddb+kVXlXst9d+R9PfTIxh1EdNkgoRe5tOX6t01f1lYWOvJnSPDBlG241QLzcyPdoNTsblLUdujGSE4RzrTZGQ==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/inquire": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@protobufjs/inquire/-/inquire-1.1.0.tgz", - "integrity": "sha512-kdSefcPdruJiFMVSbn801t4vFK7KB/5gd2fYvrxhuJYg8ILrmn9SKSX2tZdV6V+ksulWqS7aXjBcRXl3wHoD9Q==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/path": { "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@protobufjs/path/-/path-1.1.2.tgz", - "integrity": "sha512-6JOcJ5Tm08dOHAbdR3GrvP+yUUfkjG5ePsHYczMFLq3ZmMkAD98cDgcT2iA1lJ9NVwFd4tH/iSSoe44YWkltEA==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/pool": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@protobufjs/pool/-/pool-1.1.0.tgz", - "integrity": "sha512-0kELaGSIDBKvcgS4zkjz1PeddatrjYcmMWOlAuAPwAeccUrPHdUqo/J6LiymHHEiJT5NrF1UVwxY14f+fy4WQw==" + "license": "BSD-3-Clause" }, "node_modules/@protobufjs/utf8": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@protobufjs/utf8/-/utf8-1.1.0.tgz", - "integrity": "sha512-Vvn3zZrhQZkkBE8LSuW3em98c0FwgO4nxzv6OdSxPKJIEKY2bGbHn+mhGIPerzI4twdxaP8/0+06HBpwf345Lw==" + "license": "BSD-3-Clause" + }, + "node_modules/@sideway/address": { + "version": "4.1.5", + "resolved": "https://registry.npmjs.org/@sideway/address/-/address-4.1.5.tgz", + "integrity": "sha512-IqO/DUQHUkPeixNQ8n0JA6102hT9CmaljNTPmQ1u8MEhBo/R4Q8eKLN/vGZxuebwOroDB4cbpjheD4+/sKFK4Q==", + "license": "BSD-3-Clause", + "dependencies": { + "@hapi/hoek": "^9.0.0" + } + }, + "node_modules/@sideway/address/node_modules/@hapi/hoek": { + "version": "9.3.0", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-9.3.0.tgz", + "integrity": "sha512-/c6rf4UJlmHlC9b5BaNvzAcFv7HZ2QHaV0D4/HNlBdvFnvQq8RI4kYdhyPCl7Xj+oWvTWQ8ujhqS53LIgAe6KQ==", + "license": "BSD-3-Clause" + }, + "node_modules/@sideway/formula": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/@sideway/formula/-/formula-3.0.1.tgz", + "integrity": "sha512-/poHZJJVjx3L+zVD6g9KgHfYnb443oi7wLu/XKojDviHy6HOEOA6z1Trk5aR1dGcmPenJEgb2sK2I80LeS3MIg==", + "license": "BSD-3-Clause" + }, + "node_modules/@sideway/pinpoint": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@sideway/pinpoint/-/pinpoint-2.0.0.tgz", + "integrity": "sha512-RNiOoTPkptFtSVzQevY/yWtZwf/RxyVnPy/OcA9HBM3MlGDnBEYL5B41H0MTn0Uec8Hi+2qUtTfG2WWZBmMejQ==", + "license": "BSD-3-Clause" }, "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 + "license": "MIT" }, "node_modules/@sinonjs/commons": { "version": "3.0.1", - "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.1.tgz", - "integrity": "sha512-K3mCHKQ9sVh8o1C9cxkwxaOmXoAMlDxC1mYyHrjqOWEcBjYr76t96zL2zlj5dUGZ3HSw240X1qgH3Mjf1yJWpQ==", - "dev": true, + "license": "BSD-3-Clause", "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, + "license": "BSD-3-Clause", "dependencies": { "@sinonjs/commons": "^3.0.0" } }, "node_modules/@smithy/abort-controller": { - "version": "3.1.2", - "resolved": "https://registry.npmjs.org/@smithy/abort-controller/-/abort-controller-3.1.2.tgz", - "integrity": "sha512-b5g+PNujlfqIib9BjkNB108NyO5aZM/RXjfOCXRCqXQ1oPnIkfvdORrztbGgCZdPe/BN/MKDlrGA7PafKPM2jw==", + "version": "3.1.1", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2611,14 +2335,13 @@ } }, "node_modules/@smithy/config-resolver": { - "version": "3.0.6", - "resolved": "https://registry.npmjs.org/@smithy/config-resolver/-/config-resolver-3.0.6.tgz", - "integrity": "sha512-j7HuVNoRd8EhcFp0MzcUb4fG40C7BcyshH+fAd3Jhd8bINNFvEQYBrZoS/SK6Pun9WPlfoI8uuU2SMz8DsEGlA==", + "version": "3.0.5", + "license": "Apache-2.0", "dependencies": { - "@smithy/node-config-provider": "^3.1.5", - "@smithy/types": "^3.4.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", "@smithy/util-config-provider": "^3.0.0", - "@smithy/util-middleware": "^3.0.4", + "@smithy/util-middleware": "^3.0.3", "tslib": "^2.6.2" }, "engines": { @@ -2626,18 +2349,17 @@ } }, "node_modules/@smithy/core": { - "version": "2.4.1", - "resolved": "https://registry.npmjs.org/@smithy/core/-/core-2.4.1.tgz", - "integrity": "sha512-7cts7/Oni7aCHebHGiBeWoz5z+vmH+Vx2Z/UW3XtXMslcxI3PEwBZxNinepwZjixS3n12fPc247PHWmjU7ndsQ==", - "dependencies": { - "@smithy/middleware-endpoint": "^3.1.1", - "@smithy/middleware-retry": "^3.0.16", - "@smithy/middleware-serde": "^3.0.4", - "@smithy/protocol-http": "^4.1.1", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", + "version": "2.4.0", + "license": "Apache-2.0", + "dependencies": { + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-retry": "^3.0.15", + "@smithy/middleware-serde": "^3.0.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", "@smithy/util-body-length-browser": "^3.0.0", - "@smithy/util-middleware": "^3.0.4", + "@smithy/util-middleware": "^3.0.3", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" }, @@ -2646,14 +2368,13 @@ } }, "node_modules/@smithy/credential-provider-imds": { - "version": "3.2.1", - "resolved": "https://registry.npmjs.org/@smithy/credential-provider-imds/-/credential-provider-imds-3.2.1.tgz", - "integrity": "sha512-4z/oTWpRF2TqQI3aCM89/PWu3kim58XU4kOCTtuTJnoaS4KT95cPWMxbQfTN2vzcOe96SOKO8QouQW/+ESB1fQ==", + "version": "3.2.0", + "license": "Apache-2.0", "dependencies": { - "@smithy/node-config-provider": "^3.1.5", - "@smithy/property-provider": "^3.1.4", - "@smithy/types": "^3.4.0", - "@smithy/url-parser": "^3.0.4", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", "tslib": "^2.6.2" }, "engines": { @@ -2661,23 +2382,21 @@ } }, "node_modules/@smithy/fetch-http-handler": { - "version": "3.2.5", - "resolved": "https://registry.npmjs.org/@smithy/fetch-http-handler/-/fetch-http-handler-3.2.5.tgz", - "integrity": "sha512-DjRtGmK8pKQMIo9+JlAKUt14Z448bg8nAN04yKIvlrrpmpRSG57s5d2Y83npks1r4gPtTRNbAFdQCoj9l3P2KQ==", + "version": "3.2.4", + "license": "Apache-2.0", "dependencies": { - "@smithy/protocol-http": "^4.1.1", - "@smithy/querystring-builder": "^3.0.4", - "@smithy/types": "^3.4.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/querystring-builder": "^3.0.3", + "@smithy/types": "^3.3.0", "@smithy/util-base64": "^3.0.0", "tslib": "^2.6.2" } }, "node_modules/@smithy/hash-node": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/hash-node/-/hash-node-3.0.4.tgz", - "integrity": "sha512-6FgTVqEfCr9z/7+Em8BwSkJKA2y3krf1em134x3yr2NHWVCo2KYI8tcA53cjeO47y41jwF84ntsEE0Pe6pNKlg==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "@smithy/util-buffer-from": "^3.0.0", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" @@ -2687,18 +2406,16 @@ } }, "node_modules/@smithy/invalid-dependency": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/invalid-dependency/-/invalid-dependency-3.0.4.tgz", - "integrity": "sha512-MJBUrojC4SEXi9aJcnNOE3oNAuYNphgCGFXscaCj2TA/59BTcXhzHACP8jnnEU3n4yir/NSLKzxqez0T4x4tjA==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" } }, "node_modules/@smithy/is-array-buffer": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/is-array-buffer/-/is-array-buffer-3.0.0.tgz", - "integrity": "sha512-+Fsu6Q6C4RSJiy81Y8eApjEB5gVtM+oFKTffg+jSuwtvomJJrhUJBu2zS8wjXSgH/g1MKEWrzyChTBe6clb5FQ==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -2707,12 +2424,11 @@ } }, "node_modules/@smithy/middleware-content-length": { - "version": "3.0.6", - "resolved": "https://registry.npmjs.org/@smithy/middleware-content-length/-/middleware-content-length-3.0.6.tgz", - "integrity": "sha512-AFyHCfe8rumkJkz+hCOVJmBagNBj05KypyDwDElA4TgMSA4eYDZRjVePFZuyABrJZFDc7uVj3dpFIDCEhf59SA==", + "version": "3.0.5", + "license": "Apache-2.0", "dependencies": { - "@smithy/protocol-http": "^4.1.1", - "@smithy/types": "^3.4.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2720,16 +2436,15 @@ } }, "node_modules/@smithy/middleware-endpoint": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@smithy/middleware-endpoint/-/middleware-endpoint-3.1.1.tgz", - "integrity": "sha512-Irv+soW8NKluAtFSEsF8O3iGyLxa5oOevJb/e1yNacV9H7JP/yHyJuKST5YY2ORS1+W34VR8EuUrOF+K29Pl4g==", - "dependencies": { - "@smithy/middleware-serde": "^3.0.4", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", - "@smithy/url-parser": "^3.0.4", - "@smithy/util-middleware": "^3.0.4", + "version": "3.1.0", + "license": "Apache-2.0", + "dependencies": { + "@smithy/middleware-serde": "^3.0.3", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", + "@smithy/url-parser": "^3.0.3", + "@smithy/util-middleware": "^3.0.3", "tslib": "^2.6.2" }, "engines": { @@ -2737,17 +2452,16 @@ } }, "node_modules/@smithy/middleware-retry": { - "version": "3.0.16", - "resolved": "https://registry.npmjs.org/@smithy/middleware-retry/-/middleware-retry-3.0.16.tgz", - "integrity": "sha512-08kI36p1yB4CWO3Qi+UQxjzobt8iQJpnruF0K5BkbZmA/N/sJ51A1JJGJ36GgcbFyPfWw2FU48S5ZoqXt0h0jw==", - "dependencies": { - "@smithy/node-config-provider": "^3.1.5", - "@smithy/protocol-http": "^4.1.1", - "@smithy/service-error-classification": "^3.0.4", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", - "@smithy/util-middleware": "^3.0.4", - "@smithy/util-retry": "^3.0.4", + "version": "3.0.15", + "license": "Apache-2.0", + "dependencies": { + "@smithy/node-config-provider": "^3.1.4", + "@smithy/protocol-http": "^4.1.0", + "@smithy/service-error-classification": "^3.0.3", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", + "@smithy/util-middleware": "^3.0.3", + "@smithy/util-retry": "^3.0.3", "tslib": "^2.6.2", "uuid": "^9.0.1" }, @@ -2755,24 +2469,11 @@ "node": ">=16.0.0" } }, - "node_modules/@smithy/middleware-retry/node_modules/uuid": { - "version": "9.0.1", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", - "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", - "funding": [ - "https://github.com/sponsors/broofa", - "https://github.com/sponsors/ctavan" - ], - "bin": { - "uuid": "dist/bin/uuid" - } - }, "node_modules/@smithy/middleware-serde": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/middleware-serde/-/middleware-serde-3.0.4.tgz", - "integrity": "sha512-1lPDB2O6IJ50Ucxgn7XrvZXbbuI48HmPCcMTuSoXT1lDzuTUfIuBjgAjpD8YLVMfnrjdepi/q45556LA51Pubw==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2780,11 +2481,10 @@ } }, "node_modules/@smithy/middleware-stack": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/middleware-stack/-/middleware-stack-3.0.4.tgz", - "integrity": "sha512-sLMRjtMCqtVcrOqaOZ10SUnlFE25BSlmLsi4bRSGFD7dgR54eqBjfqkVkPBQyrKBortfGM0+2DJoUPcGECR+nQ==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2792,13 +2492,12 @@ } }, "node_modules/@smithy/node-config-provider": { - "version": "3.1.5", - "resolved": "https://registry.npmjs.org/@smithy/node-config-provider/-/node-config-provider-3.1.5.tgz", - "integrity": "sha512-dq/oR3/LxgCgizVk7in7FGTm0w9a3qM4mg3IIXLTCHeW3fV+ipssSvBZ2bvEx1+asfQJTyCnVLeYf7JKfd9v3Q==", + "version": "3.1.4", + "license": "Apache-2.0", "dependencies": { - "@smithy/property-provider": "^3.1.4", - "@smithy/shared-ini-file-loader": "^3.1.5", - "@smithy/types": "^3.4.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/shared-ini-file-loader": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2806,14 +2505,13 @@ } }, "node_modules/@smithy/node-http-handler": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/@smithy/node-http-handler/-/node-http-handler-3.2.0.tgz", - "integrity": "sha512-5TFqaABbiY7uJMKbqR4OARjwI/l4TRoysDJ75pLpVQyO3EcmeloKYwDGyCtgB9WJniFx3BMkmGCB9+j+QiB+Ww==", + "version": "3.1.4", + "license": "Apache-2.0", "dependencies": { - "@smithy/abort-controller": "^3.1.2", - "@smithy/protocol-http": "^4.1.1", - "@smithy/querystring-builder": "^3.0.4", - "@smithy/types": "^3.4.0", + "@smithy/abort-controller": "^3.1.1", + "@smithy/protocol-http": "^4.1.0", + "@smithy/querystring-builder": "^3.0.3", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2821,11 +2519,10 @@ } }, "node_modules/@smithy/property-provider": { - "version": "3.1.4", - "resolved": "https://registry.npmjs.org/@smithy/property-provider/-/property-provider-3.1.4.tgz", - "integrity": "sha512-BmhefQbfkSl9DeU0/e6k9N4sT5bya5etv2epvqLUz3eGyfRBhtQq60nDkc1WPp4c+KWrzK721cUc/3y0f2psPQ==", + "version": "3.1.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2833,11 +2530,10 @@ } }, "node_modules/@smithy/protocol-http": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/@smithy/protocol-http/-/protocol-http-4.1.1.tgz", - "integrity": "sha512-Fm5+8LkeIus83Y8jTL1XHsBGP8sPvE1rEVyKf/87kbOPTbzEDMcgOlzcmYXat2h+nC3wwPtRy8hFqtJS71+Wow==", + "version": "4.1.0", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2845,11 +2541,10 @@ } }, "node_modules/@smithy/querystring-builder": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/querystring-builder/-/querystring-builder-3.0.4.tgz", - "integrity": "sha512-NEoPAsZPdpfVbF98qm8i5k1XMaRKeEnO47CaL5ja6Y1Z2DgJdwIJuJkTJypKm/IKfp8gc0uimIFLwhml8+/pAw==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "@smithy/util-uri-escape": "^3.0.0", "tslib": "^2.6.2" }, @@ -2858,11 +2553,10 @@ } }, "node_modules/@smithy/querystring-parser": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/querystring-parser/-/querystring-parser-3.0.4.tgz", - "integrity": "sha512-7CHPXffFcakFzhO0OZs/rn6fXlTHrSDdLhIT6/JIk1u2bvwguTL3fMCc1+CfcbXA7TOhjWXu3TcB1EGMqJQwHg==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2870,22 +2564,20 @@ } }, "node_modules/@smithy/service-error-classification": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/service-error-classification/-/service-error-classification-3.0.4.tgz", - "integrity": "sha512-KciDHHKFVTb9A1KlJHBt2F26PBaDtoE23uTZy5qRvPzHPqrooXFi6fmx98lJb3Jl38PuUTqIuCUmmY3pacuMBQ==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0" + "@smithy/types": "^3.3.0" }, "engines": { "node": ">=16.0.0" } }, "node_modules/@smithy/shared-ini-file-loader": { - "version": "3.1.5", - "resolved": "https://registry.npmjs.org/@smithy/shared-ini-file-loader/-/shared-ini-file-loader-3.1.5.tgz", - "integrity": "sha512-6jxsJ4NOmY5Du4FD0enYegNJl4zTSuKLiChIMqIkh+LapxiP7lmz5lYUNLE9/4cvA65mbBmtdzZ8yxmcqM5igg==", + "version": "3.1.4", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -2893,15 +2585,14 @@ } }, "node_modules/@smithy/signature-v4": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/@smithy/signature-v4/-/signature-v4-4.1.1.tgz", - "integrity": "sha512-SH9J9be81TMBNGCmjhrgMWu4YSpQ3uP1L06u/K9SDrE2YibUix1qxedPCxEQu02At0P0SrYDjvz+y91vLG0KRQ==", + "version": "4.1.0", + "license": "Apache-2.0", "dependencies": { "@smithy/is-array-buffer": "^3.0.0", - "@smithy/protocol-http": "^4.1.1", - "@smithy/types": "^3.4.0", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", "@smithy/util-hex-encoding": "^3.0.0", - "@smithy/util-middleware": "^3.0.4", + "@smithy/util-middleware": "^3.0.3", "@smithy/util-uri-escape": "^3.0.0", "@smithy/util-utf8": "^3.0.0", "tslib": "^2.6.2" @@ -2911,15 +2602,14 @@ } }, "node_modules/@smithy/smithy-client": { - "version": "3.3.0", - "resolved": "https://registry.npmjs.org/@smithy/smithy-client/-/smithy-client-3.3.0.tgz", - "integrity": "sha512-H32nVo8tIX82kB0xI2LBrIcj8jx/3/ITotNLbeG1UL0b3b440YPR/hUvqjFJiaB24pQrMjRbU8CugqH5sV0hkw==", - "dependencies": { - "@smithy/middleware-endpoint": "^3.1.1", - "@smithy/middleware-stack": "^3.0.4", - "@smithy/protocol-http": "^4.1.1", - "@smithy/types": "^3.4.0", - "@smithy/util-stream": "^3.1.4", + "version": "3.2.0", + "license": "Apache-2.0", + "dependencies": { + "@smithy/middleware-endpoint": "^3.1.0", + "@smithy/middleware-stack": "^3.0.3", + "@smithy/protocol-http": "^4.1.0", + "@smithy/types": "^3.3.0", + "@smithy/util-stream": "^3.1.3", "tslib": "^2.6.2" }, "engines": { @@ -2927,9 +2617,8 @@ } }, "node_modules/@smithy/types": { - "version": "3.4.0", - "resolved": "https://registry.npmjs.org/@smithy/types/-/types-3.4.0.tgz", - "integrity": "sha512-0shOWSg/pnFXPcsSU8ZbaJ4JBHZJPPzLCJxafJvbMVFo9l1w81CqpgUqjlKGNHVrVB7fhIs+WS82JDTyzaLyLA==", + "version": "3.3.0", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -2938,19 +2627,17 @@ } }, "node_modules/@smithy/url-parser": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/url-parser/-/url-parser-3.0.4.tgz", - "integrity": "sha512-XdXfObA8WrloavJYtDuzoDhJAYc5rOt+FirFmKBRKaihu7QtU/METAxJgSo7uMK6hUkx0vFnqxV75urtRaLkLg==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/querystring-parser": "^3.0.4", - "@smithy/types": "^3.4.0", + "@smithy/querystring-parser": "^3.0.3", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" } }, "node_modules/@smithy/util-base64": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-base64/-/util-base64-3.0.0.tgz", - "integrity": "sha512-Kxvoh5Qtt0CDsfajiZOCpJxgtPHXOKwmM+Zy4waD43UoEMA+qPxxa98aE/7ZhdnBFZFXMOiBR5xbcaMhLtznQQ==", + "license": "Apache-2.0", "dependencies": { "@smithy/util-buffer-from": "^3.0.0", "@smithy/util-utf8": "^3.0.0", @@ -2962,16 +2649,14 @@ }, "node_modules/@smithy/util-body-length-browser": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-body-length-browser/-/util-body-length-browser-3.0.0.tgz", - "integrity": "sha512-cbjJs2A1mLYmqmyVl80uoLTJhAcfzMOyPgjwAYusWKMdLeNtzmMz9YxNl3/jRLoxSS3wkqkf0jwNdtXWtyEBaQ==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" } }, "node_modules/@smithy/util-body-length-node": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-body-length-node/-/util-body-length-node-3.0.0.tgz", - "integrity": "sha512-Tj7pZ4bUloNUP6PzwhN7K386tmSmEET9QtQg0TgdNOnxhZvCssHji+oZTUIuzxECRfG8rdm2PMw2WCFs6eIYkA==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -2981,8 +2666,7 @@ }, "node_modules/@smithy/util-buffer-from": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-buffer-from/-/util-buffer-from-3.0.0.tgz", - "integrity": "sha512-aEOHCgq5RWFbP+UDPvPot26EJHjOC+bRgse5A8V3FSShqd5E5UN4qc7zkwsvJPPAVsf73QwYcHN1/gt/rtLwQA==", + "license": "Apache-2.0", "dependencies": { "@smithy/is-array-buffer": "^3.0.0", "tslib": "^2.6.2" @@ -2993,8 +2677,7 @@ }, "node_modules/@smithy/util-config-provider": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-config-provider/-/util-config-provider-3.0.0.tgz", - "integrity": "sha512-pbjk4s0fwq3Di/ANL+rCvJMKM5bzAQdE5S/6RL5NXgMExFAi6UgQMPOm5yPaIWPpr+EOXKXRonJ3FoxKf4mCJQ==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -3003,13 +2686,12 @@ } }, "node_modules/@smithy/util-defaults-mode-browser": { - "version": "3.0.16", - "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-browser/-/util-defaults-mode-browser-3.0.16.tgz", - "integrity": "sha512-Os8ddfNBe7hmc5UMWZxygIHCyAqY0aWR8Wnp/aKbti3f8Df/r0J9ttMZIxeMjsFgtVjEryB0q7SGcwBsHk8WEw==", + "version": "3.0.15", + "license": "Apache-2.0", "dependencies": { - "@smithy/property-provider": "^3.1.4", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", + "@smithy/property-provider": "^3.1.3", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", "bowser": "^2.11.0", "tslib": "^2.6.2" }, @@ -3018,16 +2700,15 @@ } }, "node_modules/@smithy/util-defaults-mode-node": { - "version": "3.0.16", - "resolved": "https://registry.npmjs.org/@smithy/util-defaults-mode-node/-/util-defaults-mode-node-3.0.16.tgz", - "integrity": "sha512-rNhFIYRtrOrrhRlj6RL8jWA6/dcwrbGYAmy8+OAHjjzQ6zdzUBB1P+3IuJAgwWN6Y5GxI+mVXlM/pOjaoIgHow==", - "dependencies": { - "@smithy/config-resolver": "^3.0.6", - "@smithy/credential-provider-imds": "^3.2.1", - "@smithy/node-config-provider": "^3.1.5", - "@smithy/property-provider": "^3.1.4", - "@smithy/smithy-client": "^3.3.0", - "@smithy/types": "^3.4.0", + "version": "3.0.15", + "license": "Apache-2.0", + "dependencies": { + "@smithy/config-resolver": "^3.0.5", + "@smithy/credential-provider-imds": "^3.2.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/property-provider": "^3.1.3", + "@smithy/smithy-client": "^3.2.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -3035,12 +2716,11 @@ } }, "node_modules/@smithy/util-endpoints": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/@smithy/util-endpoints/-/util-endpoints-2.1.0.tgz", - "integrity": "sha512-ilS7/0jcbS2ELdg0fM/4GVvOiuk8/U3bIFXUW25xE1Vh1Ol4DP6vVHQKqM40rCMizCLmJ9UxK+NeJrKlhI3HVA==", + "version": "2.0.5", + "license": "Apache-2.0", "dependencies": { - "@smithy/node-config-provider": "^3.1.5", - "@smithy/types": "^3.4.0", + "@smithy/node-config-provider": "^3.1.4", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -3049,8 +2729,7 @@ }, "node_modules/@smithy/util-hex-encoding": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-hex-encoding/-/util-hex-encoding-3.0.0.tgz", - "integrity": "sha512-eFndh1WEK5YMUYvy3lPlVmYY/fZcQE1D8oSf41Id2vCeIkKJXPcYDCZD+4+xViI6b1XSd7tE+s5AmXzz5ilabQ==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -3059,11 +2738,10 @@ } }, "node_modules/@smithy/util-middleware": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/util-middleware/-/util-middleware-3.0.4.tgz", - "integrity": "sha512-uSXHTBhstb1c4nHdmQEdkNMv9LiRNaJ/lWV2U/GO+5F236YFpdPw+hyWI9Zc0Rp9XKzwD9kVZvhZmEgp0UCVnA==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/types": "^3.4.0", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -3071,12 +2749,11 @@ } }, "node_modules/@smithy/util-retry": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@smithy/util-retry/-/util-retry-3.0.4.tgz", - "integrity": "sha512-JJr6g0tO1qO2tCQyK+n3J18r34ZpvatlFN5ULcLranFIBZPxqoivb77EPyNTVwTGMEvvq2qMnyjm4jMIxjdLFg==", + "version": "3.0.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/service-error-classification": "^3.0.4", - "@smithy/types": "^3.4.0", + "@smithy/service-error-classification": "^3.0.3", + "@smithy/types": "^3.3.0", "tslib": "^2.6.2" }, "engines": { @@ -3084,13 +2761,12 @@ } }, "node_modules/@smithy/util-stream": { - "version": "3.1.4", - "resolved": "https://registry.npmjs.org/@smithy/util-stream/-/util-stream-3.1.4.tgz", - "integrity": "sha512-txU3EIDLhrBZdGfon6E9V6sZz/irYnKFMblz4TLVjyq8hObNHNS2n9a2t7GIrl7d85zgEPhwLE0gANpZsvpsKg==", + "version": "3.1.3", + "license": "Apache-2.0", "dependencies": { - "@smithy/fetch-http-handler": "^3.2.5", - "@smithy/node-http-handler": "^3.2.0", - "@smithy/types": "^3.4.0", + "@smithy/fetch-http-handler": "^3.2.4", + "@smithy/node-http-handler": "^3.1.4", + "@smithy/types": "^3.3.0", "@smithy/util-base64": "^3.0.0", "@smithy/util-buffer-from": "^3.0.0", "@smithy/util-hex-encoding": "^3.0.0", @@ -3103,8 +2779,7 @@ }, "node_modules/@smithy/util-uri-escape": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-uri-escape/-/util-uri-escape-3.0.0.tgz", - "integrity": "sha512-LqR7qYLgZTD7nWLBecUi4aqolw8Mhza9ArpNEQ881MJJIU2sE5iHCK6TdyqqzcDLy0OPe10IY4T8ctVdtynubg==", + "license": "Apache-2.0", "dependencies": { "tslib": "^2.6.2" }, @@ -3114,8 +2789,7 @@ }, "node_modules/@smithy/util-utf8": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/@smithy/util-utf8/-/util-utf8-3.0.0.tgz", - "integrity": "sha512-rUeT12bxFnplYDe815GXbq/oixEGHfRFFtcTF3YdDi/JaENIM6aSYYLJydG83UNzLXeRI5K8abYd/8Sp/QM0kA==", + "license": "Apache-2.0", "dependencies": { "@smithy/util-buffer-from": "^3.0.0", "tslib": "^2.6.2" @@ -3126,17 +2800,14 @@ }, "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==", + "license": "MIT", "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, + "license": "MIT", "dependencies": { "@babel/parser": "^7.20.7", "@babel/types": "^7.20.7", @@ -3147,18 +2818,14 @@ }, "node_modules/@types/babel__generator": { "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@types/babel__generator/-/babel__generator-7.6.8.tgz", - "integrity": "sha512-ASsj+tpEDsEiFr1arWrlN6V3mdfjRMZt6LtK/Vp/kreFLnr5QH5+DhvD5nINYZXzwJvXeGq+05iUXcAzVrqWtw==", - "dev": true, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "@babel/parser": "^7.1.0", "@babel/types": "^7.0.0" @@ -3166,23 +2833,19 @@ }, "node_modules/@types/babel__traverse": { "version": "7.20.6", - "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.6.tgz", - "integrity": "sha512-r1bzfrm0tomOI8g1SzvCaQHo6Lcv6zu0EA+W2kHrt8dyrHQxGzBBL4kdkzIS+jBMV+EYcMAEAqXqYaLJq5rOZg==", - "dev": true, + "license": "MIT", "dependencies": { "@babel/types": "^7.20.7" } }, "node_modules/@types/caseless": { "version": "0.12.5", - "resolved": "https://registry.npmjs.org/@types/caseless/-/caseless-0.12.5.tgz", - "integrity": "sha512-hWtVTC2q7hc7xZ/RLbxapMvDMgUnDvKvMOpKal4DrMyfGBUfB1oKaZlIRr6mJL+If3bAP6sV/QneGzF6tJjZDg==" + "license": "MIT" }, "node_modules/@types/eslint": { - "version": "9.6.1", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-9.6.1.tgz", - "integrity": "sha512-FXx2pKgId/WyYo2jXw63kk7/+TY7u7AziEJxJAnSFzHlqTAS3Ync6SvgYAN/k4/PQpnnVuzoMuVnByKK2qp0ag==", + "version": "9.6.0", "dev": true, + "license": "MIT", "dependencies": { "@types/estree": "*", "@types/json-schema": "*" @@ -3190,74 +2853,60 @@ }, "node_modules/@types/eslint__js": { "version": "8.42.3", - "resolved": "https://registry.npmjs.org/@types/eslint__js/-/eslint__js-8.42.3.tgz", - "integrity": "sha512-alfG737uhmPdnvkrLdZLcEKJ/B8s9Y4hrZ+YAdzUeoArBlSUERA2E87ROfOaS4jd/C45fzOoZzidLc1IPwLqOw==", "dev": true, + "license": "MIT", "dependencies": { "@types/eslint": "*" } }, "node_modules/@types/estree": { "version": "1.0.5", - "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", - "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", - "dev": true + "dev": true, + "license": "MIT" }, "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, + "license": "MIT", "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 + "license": "MIT" }, "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, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "@types/istanbul-lib-report": "*" } }, "node_modules/@types/json-schema": { "version": "7.0.15", - "resolved": "https://registry.npmjs.org/@types/json-schema/-/json-schema-7.0.15.tgz", - "integrity": "sha512-5+fP8P8MFNC+AyZCDxrB2pkZFPGzqQWUzpSeuuVLvm8VMcorNYavBqoFcxK8bQz4Qsbn4oUEEem4wDLfcysGHA==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/@types/linkify-it": { "version": "5.0.0", - "resolved": "https://registry.npmjs.org/@types/linkify-it/-/linkify-it-5.0.0.tgz", - "integrity": "sha512-sVDA58zAw4eWAffKOaQH5/5j3XeayukzDk+ewSsnv3p4yJEZHCCzMDiZM8e0OUrRvmpGZ85jf4yDHkHsgBNr9Q==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/@types/long": { "version": "4.0.2", - "resolved": "https://registry.npmjs.org/@types/long/-/long-4.0.2.tgz", - "integrity": "sha512-MqTGEo5bj5t157U6fA/BiDynNkn0YknVdh48CMPkTSpFTVmvao5UQmm7uEF6xBEo7qIMAlY/JSleYaE6VOdpaA==" + "license": "MIT" }, "node_modules/@types/markdown-it": { "version": "14.1.2", - "resolved": "https://registry.npmjs.org/@types/markdown-it/-/markdown-it-14.1.2.tgz", - "integrity": "sha512-promo4eFwuiW+TfGxhi+0x3czqTYJkG8qB17ZUJiVF10Xm7NLVRSLUsfRTU/6h1e24VvRnXCx+hG7li58lkzog==", "dev": true, + "license": "MIT", "dependencies": { "@types/linkify-it": "^5", "@types/mdurl": "^2" @@ -3265,22 +2914,19 @@ }, "node_modules/@types/mdurl": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@types/mdurl/-/mdurl-2.0.0.tgz", - "integrity": "sha512-RGdgjQUZba5p6QEFAVx2OGb8rQDL/cPRG7GiedRzMcJ1tYnUANBncjbSB1NRGwbvjcPeikRABz2nshyPk1bhWg==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/@types/node": { - "version": "20.16.5", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.16.5.tgz", - "integrity": "sha512-VwYCweNo3ERajwy0IUlqqcyZ8/A7Zwa9ZP3MnENWcB11AejO+tLy3pu850goUW2FC/IJMdZUfKpX/yxL1gymCA==", + "version": "20.16.1", + "license": "MIT", "dependencies": { "undici-types": "~6.19.2" } }, "node_modules/@types/request": { "version": "2.48.12", - "resolved": "https://registry.npmjs.org/@types/request/-/request-2.48.12.tgz", - "integrity": "sha512-G3sY+NpsA9jnwm0ixhAFQSJ3Q9JkpLZpJbI3GMv0mIAT0y3mRabYeINzal5WOChIiaTEGQYlHOKgkaM9EisWHw==", + "license": "MIT", "dependencies": { "@types/caseless": "*", "@types/node": "*", @@ -3290,8 +2936,7 @@ }, "node_modules/@types/request/node_modules/form-data": { "version": "2.5.1", - "resolved": "https://registry.npmjs.org/form-data/-/form-data-2.5.1.tgz", - "integrity": "sha512-m21N3WOmEEURgk6B9GLOE4RuWOFf28Lhh9qGYeNlGq4VDXUlJy2th2slBNU8Gp8EzloYZOibZJ7t5ecIrFSjVA==", + "license": "MIT", "dependencies": { "asynckit": "^0.4.0", "combined-stream": "^1.0.6", @@ -3301,63 +2946,71 @@ "node": ">= 0.12" } }, + "node_modules/@types/semver": { + "version": "7.5.8", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true + }, + "node_modules/@types/simple-oauth2": { + "version": "5.0.7", + "resolved": "https://registry.npmjs.org/@types/simple-oauth2/-/simple-oauth2-5.0.7.tgz", + "integrity": "sha512-8JbWVJbiTSBQP/7eiyGKyXWAqp3dKQZpaA+pdW16FCi32ujkzRMG8JfjoAzdWt6W8U591ZNdHcPtP2D7ILTKuA==", + "license": "MIT" + }, "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 + "license": "MIT" }, "node_modules/@types/tough-cookie": { "version": "4.0.5", - "resolved": "https://registry.npmjs.org/@types/tough-cookie/-/tough-cookie-4.0.5.tgz", - "integrity": "sha512-/Ad8+nIOV7Rl++6f1BdKxFSMgmoqEoYbHRpPcx3JEfv8VRsQe9Z4mCXeJBzxs7mbHY/XOZZuXlRNfhpVPbs6ZA==" + "license": "MIT" }, "node_modules/@types/validator": { - "version": "13.12.1", - "resolved": "https://registry.npmjs.org/@types/validator/-/validator-13.12.1.tgz", - "integrity": "sha512-w0URwf7BQb0rD/EuiG12KP0bailHKHP5YVviJG9zw3ykAokL0TuxU2TUqMB7EwZ59bDHYdeTIvjI5m0S7qHfOA==" + "version": "13.12.0", + "license": "MIT" }, "node_modules/@types/yargs": { - "version": "17.0.33", - "resolved": "https://registry.npmjs.org/@types/yargs/-/yargs-17.0.33.tgz", - "integrity": "sha512-WpxBCKWPLr4xSsHgz511rFJAM+wS28w2zEO1QDNY5zM/S8ok70NNfztH0xwhqKyaK0OHCbN98LDAZuy1ctxDkA==", - "dev": true, + "version": "17.0.32", + "license": "MIT", "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 + "license": "MIT" }, "node_modules/@typescript-eslint/eslint-plugin": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.5.0.tgz", - "integrity": "sha512-lHS5hvz33iUFQKuPFGheAB84LwcJ60G8vKnEhnfcK1l8kGVLro2SFYW6K0/tj8FUhRJ0VHyg1oAfg50QGbPPHw==", + "version": "7.2.0", "dev": true, + "license": "MIT", + "optional": true, + "peer": true, "dependencies": { - "@eslint-community/regexpp": "^4.10.0", - "@typescript-eslint/scope-manager": "8.5.0", - "@typescript-eslint/type-utils": "8.5.0", - "@typescript-eslint/utils": "8.5.0", - "@typescript-eslint/visitor-keys": "8.5.0", + "@eslint-community/regexpp": "^4.5.1", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/type-utils": "7.2.0", + "@typescript-eslint/utils": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", "graphemer": "^1.4.0", - "ignore": "^5.3.1", + "ignore": "^5.2.4", "natural-compare": "^1.4.0", - "ts-api-utils": "^1.3.0" + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^16.0.0 || >=18.0.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "@typescript-eslint/parser": "^8.0.0 || ^8.0.0-alpha.0", - "eslint": "^8.57.0 || ^9.0.0" + "@typescript-eslint/parser": "^7.0.0", + "eslint": "^8.56.0" }, "peerDependenciesMeta": { "typescript": { @@ -3365,27 +3018,159 @@ } } }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/scope-manager": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/types": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/typescript-estree": { + "version": "7.2.0", + "dev": true, + "license": "BSD-2-Clause", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "9.0.3", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/utils": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@types/json-schema": "^7.0.12", + "@types/semver": "^7.5.0", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "semver": "^7.5.4" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/visitor-keys": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/brace-expansion": { + "version": "2.0.1", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/@typescript-eslint/eslint-plugin/node_modules/minimatch": { + "version": "9.0.3", + "dev": true, + "license": "ISC", + "optional": true, + "peer": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/@typescript-eslint/parser": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.5.0.tgz", - "integrity": "sha512-gF77eNv0Xz2UJg/NbpWJ0kqAm35UMsvZf1GHj8D9MRFTj/V3tAciIWXfmPLsAAF/vUlpWPvUDyH1jjsr0cMVWw==", + "version": "7.2.0", "dev": true, + "license": "BSD-2-Clause", + "optional": true, + "peer": true, "dependencies": { - "@typescript-eslint/scope-manager": "8.5.0", - "@typescript-eslint/types": "8.5.0", - "@typescript-eslint/typescript-estree": "8.5.0", - "@typescript-eslint/visitor-keys": "8.5.0", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", "debug": "^4.3.4" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^16.0.0 || >=18.0.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "eslint": "^8.57.0 || ^9.0.0" + "eslint": "^8.56.0" }, "peerDependenciesMeta": { "typescript": { @@ -3393,17 +3178,121 @@ } } }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/scope-manager": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/types": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/typescript-estree": { + "version": "7.2.0", + "dev": true, + "license": "BSD-2-Clause", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "9.0.3", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/parser/node_modules/@typescript-eslint/visitor-keys": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/brace-expansion": { + "version": "2.0.1", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/@typescript-eslint/parser/node_modules/minimatch": { + "version": "9.0.3", + "dev": true, + "license": "ISC", + "optional": true, + "peer": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/@typescript-eslint/scope-manager": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.5.0.tgz", - "integrity": "sha512-06JOQ9Qgj33yvBEx6tpC8ecP9o860rsR22hWMEd12WcTRrfaFgHr2RB/CA/B+7BMhHkXT4chg2MyboGdFGawYg==", + "version": "7.18.0", "dev": true, + "license": "MIT", "dependencies": { - "@typescript-eslint/types": "8.5.0", - "@typescript-eslint/visitor-keys": "8.5.0" + "@typescript-eslint/types": "7.18.0", + "@typescript-eslint/visitor-keys": "7.18.0" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^18.18.0 || >=20.0.0" }, "funding": { "type": "opencollective", @@ -3411,36 +3300,170 @@ } }, "node_modules/@typescript-eslint/type-utils": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.5.0.tgz", - "integrity": "sha512-N1K8Ix+lUM+cIDhL2uekVn/ZD7TZW+9/rwz8DclQpcQ9rk4sIL5CAlBC0CugWKREmDjBzI/kQqU4wkg46jWLYA==", + "version": "7.2.0", "dev": true, + "license": "MIT", + "optional": true, + "peer": true, "dependencies": { - "@typescript-eslint/typescript-estree": "8.5.0", - "@typescript-eslint/utils": "8.5.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "@typescript-eslint/utils": "7.2.0", "debug": "^4.3.4", - "ts-api-utils": "^1.3.0" + "ts-api-utils": "^1.0.1" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^16.0.0 || >=18.0.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, + "peerDependencies": { + "eslint": "^8.56.0" + }, "peerDependenciesMeta": { "typescript": { "optional": true } } }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/scope-manager": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/types": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/typescript-estree": { + "version": "7.2.0", + "dev": true, + "license": "BSD-2-Clause", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "9.0.3", + "semver": "^7.5.4", + "ts-api-utils": "^1.0.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/utils": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@types/json-schema": "^7.0.12", + "@types/semver": "^7.5.0", + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "semver": "^7.5.4" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/@typescript-eslint/visitor-keys": { + "version": "7.2.0", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "@typescript-eslint/types": "7.2.0", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/brace-expansion": { + "version": "2.0.1", + "dev": true, + "license": "MIT", + "optional": true, + "peer": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/@typescript-eslint/type-utils/node_modules/minimatch": { + "version": "9.0.3", + "dev": true, + "license": "ISC", + "optional": true, + "peer": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/@typescript-eslint/types": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.5.0.tgz", - "integrity": "sha512-qjkormnQS5wF9pjSi6q60bKUHH44j2APxfh9TQRXK8wbYVeDYYdYJGIROL87LGZZ2gz3Rbmjc736qyL8deVtdw==", + "version": "7.18.0", "dev": true, + "license": "MIT", "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^18.18.0 || >=20.0.0" }, "funding": { "type": "opencollective", @@ -3448,22 +3471,21 @@ } }, "node_modules/@typescript-eslint/typescript-estree": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.5.0.tgz", - "integrity": "sha512-vEG2Sf9P8BPQ+d0pxdfndw3xIXaoSjliG0/Ejk7UggByZPKXmJmw3GW5jV2gHNQNawBUyfahoSiCFVov0Ruf7Q==", + "version": "7.18.0", "dev": true, + "license": "BSD-2-Clause", "dependencies": { - "@typescript-eslint/types": "8.5.0", - "@typescript-eslint/visitor-keys": "8.5.0", + "@typescript-eslint/types": "7.18.0", + "@typescript-eslint/visitor-keys": "7.18.0", "debug": "^4.3.4", - "fast-glob": "^3.3.2", + "globby": "^11.1.0", "is-glob": "^4.0.3", "minimatch": "^9.0.4", "semver": "^7.6.0", "ts-api-utils": "^1.3.0" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^18.18.0 || >=20.0.0" }, "funding": { "type": "opencollective", @@ -3477,18 +3499,16 @@ }, "node_modules/@typescript-eslint/typescript-estree/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, + "license": "MIT", "dependencies": { "balanced-match": "^1.0.0" } }, "node_modules/@typescript-eslint/typescript-estree/node_modules/minimatch": { "version": "9.0.5", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", - "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", "dev": true, + "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" }, @@ -3500,38 +3520,36 @@ } }, "node_modules/@typescript-eslint/utils": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.5.0.tgz", - "integrity": "sha512-6yyGYVL0e+VzGYp60wvkBHiqDWOpT63pdMV2CVG4LVDd5uR6q1qQN/7LafBZtAtNIn/mqXjsSeS5ggv/P0iECw==", + "version": "7.18.0", "dev": true, + "license": "MIT", "dependencies": { "@eslint-community/eslint-utils": "^4.4.0", - "@typescript-eslint/scope-manager": "8.5.0", - "@typescript-eslint/types": "8.5.0", - "@typescript-eslint/typescript-estree": "8.5.0" + "@typescript-eslint/scope-manager": "7.18.0", + "@typescript-eslint/types": "7.18.0", + "@typescript-eslint/typescript-estree": "7.18.0" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^18.18.0 || >=20.0.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "eslint": "^8.57.0 || ^9.0.0" + "eslint": "^8.56.0" } }, "node_modules/@typescript-eslint/visitor-keys": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.5.0.tgz", - "integrity": "sha512-yTPqMnbAZJNy2Xq2XU8AdtOW9tJIr+UQb64aXB9f3B1498Zx9JorVgFJcZpEc9UBuCCrdzKID2RGAMkYcDtZOw==", + "version": "7.18.0", "dev": true, + "license": "MIT", "dependencies": { - "@typescript-eslint/types": "8.5.0", + "@typescript-eslint/types": "7.18.0", "eslint-visitor-keys": "^3.4.3" }, "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + "node": "^18.18.0 || >=20.0.0" }, "funding": { "type": "opencollective", @@ -3540,9 +3558,8 @@ }, "node_modules/@typescript/vfs": { "version": "1.6.0", - "resolved": "https://registry.npmjs.org/@typescript/vfs/-/vfs-1.6.0.tgz", - "integrity": "sha512-hvJUjNVeBMp77qPINuUvYXj4FyWeeMMKZkxEATEU3hqBAQ7qdTBCUFT7Sp0Zu0faeEtFf+ldXxMEDr/bk73ISg==", "dev": true, + "license": "MIT", "dependencies": { "debug": "^4.1.1" }, @@ -3552,19 +3569,16 @@ }, "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 + "dev": true, + "license": "ISC" }, "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==" + "license": "ISC" }, "node_modules/abort-controller": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/abort-controller/-/abort-controller-3.0.0.tgz", - "integrity": "sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg==", + "license": "MIT", "dependencies": { "event-target-shim": "^5.0.0" }, @@ -3574,9 +3588,8 @@ }, "node_modules/acorn": { "version": "8.12.1", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", - "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true, + "license": "MIT", "bin": { "acorn": "bin/acorn" }, @@ -3586,29 +3599,26 @@ }, "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, + "license": "MIT", "peerDependencies": { "acorn": "^6.0.0 || ^7.0.0 || ^8.0.0" } }, "node_modules/agent-base": { - "version": "7.1.1", - "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.1.tgz", - "integrity": "sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==", + "version": "6.0.2", + "license": "MIT", "dependencies": { - "debug": "^4.3.4" + "debug": "4" }, "engines": { - "node": ">= 14" + "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, + "license": "MIT", "dependencies": { "humanize-ms": "^1.2.1" }, @@ -3618,9 +3628,8 @@ }, "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, + "license": "MIT", "dependencies": { "clean-stack": "^2.0.0", "indent-string": "^4.0.0" @@ -3631,8 +3640,7 @@ }, "node_modules/ajv": { "version": "8.17.1", - "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.17.1.tgz", - "integrity": "sha512-B/gBuNg5SiMTrPkC+A2+cW0RszwxYmn6VYxB/inlBStS5nx6xHIt/ehKRhIMhqusl7a8LjQoZnjCs5vhwxOQ1g==", + "license": "MIT", "dependencies": { "fast-deep-equal": "^3.1.3", "fast-uri": "^3.0.1", @@ -3646,18 +3654,15 @@ }, "node_modules/ansi-colors": { "version": "4.1.3", - "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.3.tgz", - "integrity": "sha512-/6w/C21Pm1A7aZitlI5Ni/2J6FFQN8i1Cvz3kHABAAbw93v/NlvKdVOqz7CCWz/3iv/JplRSEEZ83XION15ovw==", "dev": true, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "type-fest": "^0.21.3" }, @@ -3670,9 +3675,7 @@ }, "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, + "license": "(MIT OR CC0-1.0)", "engines": { "node": ">=10" }, @@ -3682,16 +3685,14 @@ }, "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==", + "license": "MIT", "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==", + "license": "MIT", "dependencies": { "color-convert": "^2.0.1" }, @@ -3704,9 +3705,7 @@ }, "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, + "license": "ISC", "dependencies": { "normalize-path": "^3.0.0", "picomatch": "^2.0.4" @@ -3717,14 +3716,11 @@ }, "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==" + "license": "ISC" }, "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==", - "deprecated": "This package is no longer supported.", + "license": "ISC", "dependencies": { "delegates": "^1.0.0", "readable-stream": "^3.6.0" @@ -3735,70 +3731,72 @@ }, "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 + "dev": true, + "license": "Python-2.0" + }, + "node_modules/array-union": { + "version": "2.1.0", + "dev": true, + "license": "MIT", + "engines": { + "node": ">=8" + } }, "node_modules/asn1": { "version": "0.2.6", - "resolved": "https://registry.npmjs.org/asn1/-/asn1-0.2.6.tgz", - "integrity": "sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ==", + "license": "MIT", "dependencies": { "safer-buffer": "~2.1.0" } }, "node_modules/assert-plus": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/assert-plus/-/assert-plus-1.0.0.tgz", - "integrity": "sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw==", + "license": "MIT", "engines": { "node": ">=0.8" } }, "node_modules/async": { - "version": "3.2.6", - "resolved": "https://registry.npmjs.org/async/-/async-3.2.6.tgz", - "integrity": "sha512-htCUDlxyyCLMgaM3xXg0C0LW2xqfuQ6p05pCEIsXuyQ+a1koYKTuBMzRNwmybfLgvJDMd0r1LTn4+E0Ti6C2AA==", - "dev": true + "version": "3.2.5", + "license": "MIT" }, "node_modules/async-mutex": { "version": "0.5.0", - "resolved": "https://registry.npmjs.org/async-mutex/-/async-mutex-0.5.0.tgz", - "integrity": "sha512-1A94B18jkJ3DYq284ohPxoXbfTA5HsQ7/Mf4DEhcyLx3Bz27Rh59iScbB6EPiP+B+joue6YCxcMXSbFC1tZKwA==", + "license": "MIT", "dependencies": { "tslib": "^2.4.0" } }, "node_modules/asynckit": { "version": "0.4.0", - "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz", - "integrity": "sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==" + "license": "MIT" }, "node_modules/avsc": { "version": "5.7.7", - "resolved": "https://registry.npmjs.org/avsc/-/avsc-5.7.7.tgz", - "integrity": "sha512-9cYNccliXZDByFsFliVwk5GvTq058Fj513CiR4E60ndDwmuXzTJEp/Bp8FyuRmGyYupLjHLs+JA9/CBoVS4/NQ==", + "license": "MIT", "engines": { "node": ">=0.11" } }, "node_modules/aws-sign2": { "version": "0.7.0", - "resolved": "https://registry.npmjs.org/aws-sign2/-/aws-sign2-0.7.0.tgz", - "integrity": "sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==", + "license": "Apache-2.0", "engines": { "node": "*" } }, "node_modules/aws4": { - "version": "1.13.2", - "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.13.2.tgz", - "integrity": "sha512-lHe62zvbTB5eEABUVi/AwVh0ZKY9rMMDhmm+eeyuuUQbQ3+J+fONVQOZyj+DdrvD4BY33uYniyRJ4UJIaSKAfw==" + "version": "1.13.1", + "license": "MIT" }, "node_modules/axios": { "version": "1.7.7", +<<<<<<< HEAD +======= "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.7.tgz", "integrity": "sha512-S4kL7XrjgBmvdGut0sN3yJxqYzrDOnivkBiN0OFs6hLiUam3UPvswUo0kqGyhqUZGEOytHyumEdXsAkgCOUf3Q==", +>>>>>>> 242a0f7 (Add OAuth Support to Rest Service (#25)) + "license": "MIT", "dependencies": { "follow-redirects": "^1.15.6", "form-data": "^4.0.0", @@ -3807,9 +3805,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/transform": "^29.7.0", "@types/babel__core": "^7.1.14", @@ -3828,9 +3824,7 @@ }, "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, + "license": "BSD-3-Clause", "dependencies": { "@babel/helper-plugin-utils": "^7.0.0", "@istanbuljs/load-nyc-config": "^1.0.0", @@ -3844,9 +3838,7 @@ }, "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, + "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.12.3", "@babel/parser": "^7.14.7", @@ -3860,18 +3852,14 @@ }, "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, + "license": "ISC", "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, + "license": "MIT", "dependencies": { "@babel/template": "^7.3.3", "@babel/types": "^7.3.3", @@ -3883,26 +3871,21 @@ } }, "node_modules/babel-preset-current-node-syntax": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/babel-preset-current-node-syntax/-/babel-preset-current-node-syntax-1.1.0.tgz", - "integrity": "sha512-ldYss8SbBlWva1bs28q78Ju5Zq1F+8BrqBZZ0VFhLBvhh6lCpC2o3gDJi/5DRLs9FgYZCnmPYIVFU4lRXCkyUw==", - "dev": true, + "version": "1.0.1", + "license": "MIT", "dependencies": { "@babel/plugin-syntax-async-generators": "^7.8.4", "@babel/plugin-syntax-bigint": "^7.8.3", - "@babel/plugin-syntax-class-properties": "^7.12.13", - "@babel/plugin-syntax-class-static-block": "^7.14.5", - "@babel/plugin-syntax-import-attributes": "^7.24.7", - "@babel/plugin-syntax-import-meta": "^7.10.4", + "@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.10.4", + "@babel/plugin-syntax-logical-assignment-operators": "^7.8.3", "@babel/plugin-syntax-nullish-coalescing-operator": "^7.8.3", - "@babel/plugin-syntax-numeric-separator": "^7.10.4", + "@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-private-property-in-object": "^7.14.5", - "@babel/plugin-syntax-top-level-await": "^7.14.5" + "@babel/plugin-syntax-top-level-await": "^7.8.3" }, "peerDependencies": { "@babel/core": "^7.0.0" @@ -3910,9 +3893,7 @@ }, "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, + "license": "MIT", "dependencies": { "babel-plugin-jest-hoist": "^29.6.3", "babel-preset-current-node-syntax": "^1.0.0" @@ -3926,13 +3907,10 @@ }, "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==" + "license": "MIT" }, "node_modules/base64-js": { "version": "1.5.1", - "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.5.1.tgz", - "integrity": "sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==", "funding": [ { "type": "github", @@ -3946,29 +3924,27 @@ "type": "consulting", "url": "https://feross.org/support" } - ] + ], + "license": "MIT" }, "node_modules/bcrypt-pbkdf": { "version": "1.0.2", - "resolved": "https://registry.npmjs.org/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz", - "integrity": "sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w==", + "license": "BSD-3-Clause", "dependencies": { "tweetnacl": "^0.14.3" } }, "node_modules/bignumber.js": { "version": "9.1.2", - "resolved": "https://registry.npmjs.org/bignumber.js/-/bignumber.js-9.1.2.tgz", - "integrity": "sha512-2/mKyZH9K85bzOEfhXDBFZTGd1CTs+5IHpeFQo9luiBG7hghdC851Pj2WAhb6E3R6b9tZj/XKhbg4fum+Kepug==", + "license": "MIT", "engines": { "node": "*" } }, "node_modules/binary-extensions": { "version": "2.3.0", - "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.3.0.tgz", - "integrity": "sha512-Ceh+7ox5qe7LJuLHoY0feh3pHuUDHAcRUeyL2VYghZwfpkNIy/+8Ocg0a3UuSoYzavmylwuLWQOf3hl0jjMMIw==", "dev": true, + "license": "MIT", "engines": { "node": ">=8" }, @@ -3978,27 +3954,23 @@ }, "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==", + "license": "MIT", "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 + "dev": true, + "license": "MIT" }, "node_modules/bowser": { "version": "2.11.0", - "resolved": "https://registry.npmjs.org/bowser/-/bowser-2.11.0.tgz", - "integrity": "sha512-AlcaJBi/pqqJBIQ8U9Mcpc9i8Aqxn88Skv5d+xBX006BY5u8N3mGLHa5Lgppa7L/HfwgwLgZ6NYs+Ag6uUmJRA==" + "license": "MIT" }, "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==", + "license": "MIT", "dependencies": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -4006,9 +3978,7 @@ }, "node_modules/braces": { "version": "3.0.3", - "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", - "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", - "dev": true, + "license": "MIT", "dependencies": { "fill-range": "^7.1.1" }, @@ -4018,23 +3988,18 @@ }, "node_modules/brotli": { "version": "1.3.3", - "resolved": "https://registry.npmjs.org/brotli/-/brotli-1.3.3.tgz", - "integrity": "sha512-oTKjJdShmDuGW94SyyaoQvAjf30dZaHnjJ8uAF+u2/vGJkJbJPJAT1gDiOJP5v1Zb6f9KEyW/1HpuaWIXtGHPg==", + "license": "MIT", "dependencies": { "base64-js": "^1.1.2" } }, "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 + "dev": true, + "license": "ISC" }, "node_modules/browserslist": { "version": "4.23.3", - "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.3.tgz", - "integrity": "sha512-btwCFJVjI4YWDNfau8RhZ+B1Q/VLoUITrm3RlP6y1tYGWIOa+InuYiRGXUBXo8nA1qKmHMyLB/iVQg5TT4eFoA==", - "dev": true, "funding": [ { "type": "opencollective", @@ -4049,6 +4014,7 @@ "url": "https://github.com/sponsors/ai" } ], + "license": "MIT", "dependencies": { "caniuse-lite": "^1.0.30001646", "electron-to-chromium": "^1.5.4", @@ -4064,9 +4030,7 @@ }, "node_modules/bs-logger": { "version": "0.2.6", - "resolved": "https://registry.npmjs.org/bs-logger/-/bs-logger-0.2.6.tgz", - "integrity": "sha512-pd8DCoxmbgc7hyPKOvxtqNcjYoOsABPQdcCUjGp3d42VR2CX1ORhk2A87oqqu5R1kk+76nsxZupkmyd+MVtCog==", - "dev": true, + "license": "MIT", "dependencies": { "fast-json-stable-stringify": "2.x" }, @@ -4076,29 +4040,23 @@ }, "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, + "license": "Apache-2.0", "dependencies": { "node-int64": "^0.4.0" } }, "node_modules/buffer-equal-constant-time": { "version": "1.0.1", - "resolved": "https://registry.npmjs.org/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz", - "integrity": "sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA==" + "license": "BSD-3-Clause" }, "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 + "license": "MIT" }, "node_modules/cacache": { "version": "16.1.3", - "resolved": "https://registry.npmjs.org/cacache/-/cacache-16.1.3.tgz", - "integrity": "sha512-/+Emcj9DAXxX4cwlLmRI9c166RuL3w30zp4R7Joiv2cQTtTtA+jeuCAjH3ZlGnYS3tKENSrKhAzVVP9GVyzeYQ==", "dev": true, + "license": "ISC", "dependencies": { "@npmcli/fs": "^2.1.0", "@npmcli/move-file": "^2.0.0", @@ -4125,19 +4083,16 @@ }, "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, + "license": "MIT", "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==", - "deprecated": "Glob versions prior to v9 are no longer supported", "dev": true, + "license": "ISC", "dependencies": { "fs.realpath": "^1.0.0", "inflight": "^1.0.4", @@ -4154,18 +4109,16 @@ }, "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, + "license": "ISC", "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, + "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" }, @@ -4175,27 +4128,20 @@ }, "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, + "license": "MIT", "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, + "license": "MIT", "engines": { "node": ">=6" } }, "node_modules/caniuse-lite": { - "version": "1.0.30001660", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001660.tgz", - "integrity": "sha512-GacvNTTuATm26qC74pt+ad1fW15mlQ/zuTzzY1ZoIzECTP8HURDfF43kNxPgf7H1jmelCBQTTbBNxdSXOA7Bqg==", - "dev": true, + "version": "1.0.30001646", "funding": [ { "type": "opencollective", @@ -4209,18 +4155,17 @@ "type": "github", "url": "https://github.com/sponsors/ai" } - ] + ], + "license": "CC-BY-4.0" }, "node_modules/caseless": { "version": "0.12.0", - "resolved": "https://registry.npmjs.org/caseless/-/caseless-0.12.0.tgz", - "integrity": "sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==" + "license": "Apache-2.0" }, "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, + "license": "MIT", "dependencies": { "lodash": "^4.17.15" }, @@ -4230,9 +4175,7 @@ }, "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, + "license": "MIT", "dependencies": { "ansi-styles": "^4.1.0", "supports-color": "^7.1.0" @@ -4246,18 +4189,15 @@ }, "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, + "license": "MIT", "engines": { "node": ">=10" } }, "node_modules/chokidar": { "version": "3.6.0", - "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.6.0.tgz", - "integrity": "sha512-7VT13fmjotKpGipCW9JEQAusEPE+Ei8nl6/g4FBAmIm0GOOLMua9NDDo/DWp0ZAxCr3cPq5ZpBqmPAQgDda2Pw==", "dev": true, + "license": "MIT", "dependencies": { "anymatch": "~3.1.2", "braces": "~3.0.2", @@ -4279,9 +4219,8 @@ }, "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, + "license": "ISC", "dependencies": { "is-glob": "^4.0.1" }, @@ -4291,46 +4230,39 @@ }, "node_modules/chownr": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/chownr/-/chownr-2.0.0.tgz", - "integrity": "sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ==", + "license": "ISC", "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" } ], + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/cjs-module-lexer": { - "version": "1.4.1", - "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.4.1.tgz", - "integrity": "sha512-cuSVIHi9/9E/+821Qjdvngor+xpnlwnuwIyZOaLmHBVdXL+gP+I6QQB9VkO7RI77YIcTV+S1W9AreJ5eN63JBA==", - "dev": true + "version": "1.3.1", + "license": "MIT" }, "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, + "license": "MIT", "engines": { "node": ">=6" } }, "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==", + "license": "ISC", "dependencies": { "string-width": "^4.2.0", "strip-ansi": "^6.0.1", @@ -4342,9 +4274,7 @@ }, "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, + "license": "MIT", "engines": { "iojs": ">= 1.0.0", "node": ">= 0.12.0" @@ -4352,14 +4282,11 @@ }, "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 + "license": "MIT" }, "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==", + "license": "MIT", "dependencies": { "color-name": "~1.1.4" }, @@ -4369,21 +4296,18 @@ }, "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==" + "license": "MIT" }, "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==", + "license": "ISC", "bin": { "color-support": "bin.js" } }, "node_modules/combined-stream": { "version": "1.0.8", - "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.8.tgz", - "integrity": "sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg==", + "license": "MIT", "dependencies": { "delayed-stream": "~1.0.0" }, @@ -4393,35 +4317,27 @@ }, "node_modules/commander": { "version": "2.20.3", - "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.3.tgz", - "integrity": "sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ==" + "license": "MIT" }, "node_modules/concat-map": { "version": "0.0.1", - "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "license": "MIT" }, "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==" + "license": "ISC" }, "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 + "license": "MIT" }, "node_modules/core-util-is": { "version": "1.0.2", - "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.2.tgz", - "integrity": "sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ==" + "license": "MIT" }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "chalk": "^4.0.0", @@ -4440,9 +4356,7 @@ }, "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, + "license": "MIT", "dependencies": { "path-key": "^3.1.0", "shebang-command": "^2.0.0", @@ -4454,8 +4368,7 @@ }, "node_modules/dashdash": { "version": "1.14.1", - "resolved": "https://registry.npmjs.org/dashdash/-/dashdash-1.14.1.tgz", - "integrity": "sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g==", + "license": "MIT", "dependencies": { "assert-plus": "^1.0.0" }, @@ -4464,11 +4377,10 @@ } }, "node_modules/debug": { - "version": "4.3.7", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.7.tgz", - "integrity": "sha512-Er2nc/H7RrMXZBFCEim6TCmMk02Z8vLC2Rbi1KEBggpo0fS6l0S1nnapwmIi3yW/+GOJap1Krg4w0Hg80oCqgQ==", + "version": "4.3.6", + "license": "MIT", "dependencies": { - "ms": "^2.1.3" + "ms": "2.1.2" }, "engines": { "node": ">=6.0" @@ -4481,9 +4393,8 @@ }, "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -4493,9 +4404,7 @@ }, "node_modules/dedent": { "version": "1.5.3", - "resolved": "https://registry.npmjs.org/dedent/-/dedent-1.5.3.tgz", - "integrity": "sha512-NHQtfOOW68WD8lgypbLA5oT+Bt0xXJhiYvoR6SmmNXZfpzOGXwdKWmcwG8N7PwVVWV3eF/68nmD9BaJSsTBhyQ==", - "dev": true, + "license": "MIT", "peerDependencies": { "babel-plugin-macros": "^3.1.0" }, @@ -4507,85 +4416,82 @@ }, "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 + "dev": true, + "license": "MIT" }, "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, + "license": "MIT", "engines": { "node": ">=0.10.0" } }, "node_modules/define-lazy-prop": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz", - "integrity": "sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og==", + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/delayed-stream": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz", - "integrity": "sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ==", + "license": "MIT", "engines": { "node": ">=0.4.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==" + "license": "MIT" }, "node_modules/detect-libc": { "version": "2.0.3", - "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.3.tgz", - "integrity": "sha512-bwy0MGW55bG41VqxxypOsdSdGqLwXPI/focwgTYCFMbdUiBAxLg9CFzG08sz2aqzknwiX7Hkl0bQENjg8iLByw==", + "license": "Apache-2.0", "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, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/diff": { "version": "5.2.0", - "resolved": "https://registry.npmjs.org/diff/-/diff-5.2.0.tgz", - "integrity": "sha512-uIFDxqpRZGZ6ThOk84hEfqWoHx2devRFvpTZcTHur85vImfaxUbTW9Ryh4CpCuDnToOP1CEtXKIgytHBPVff5A==", "dev": true, + "license": "BSD-3-Clause", "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, + "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, + "node_modules/dir-glob": { + "version": "3.0.1", + "dev": true, + "license": "MIT", + "dependencies": { + "path-type": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/discontinuous-range": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/discontinuous-range/-/discontinuous-range-1.0.0.tgz", - "integrity": "sha512-c68LpLbO+7kP/b1Hr1qs8/BJ09F5khZGTxqxZuhzxpmwJKOgRFHJWIb9/KmqnqHhLdO55aOxFH/EGBvUQbL/RQ==" + "license": "MIT" }, "node_modules/doctrine": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", - "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", "dev": true, + "license": "Apache-2.0", "dependencies": { "esutils": "^2.0.2" }, @@ -4595,8 +4501,7 @@ }, "node_modules/duplexify": { "version": "4.1.3", - "resolved": "https://registry.npmjs.org/duplexify/-/duplexify-4.1.3.tgz", - "integrity": "sha512-M3BmBhwJRZsSx38lZyhE53Csddgzl5R7xGJNk7CVddZD6CcmwMCH8J+7AprIrQKH7TonKxaCjcv27Qmf+sQ+oA==", + "license": "MIT", "dependencies": { "end-of-stream": "^1.4.1", "inherits": "^2.0.3", @@ -4606,8 +4511,7 @@ }, "node_modules/ecc-jsbn": { "version": "0.1.2", - "resolved": "https://registry.npmjs.org/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz", - "integrity": "sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw==", + "license": "MIT", "dependencies": { "jsbn": "~0.1.0", "safer-buffer": "^2.1.0" @@ -4615,22 +4519,18 @@ }, "node_modules/ecc-jsbn/node_modules/jsbn": { "version": "0.1.1", - "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-0.1.1.tgz", - "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==" + "license": "MIT" }, "node_modules/ecdsa-sig-formatter": { "version": "1.0.11", - "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", - "integrity": "sha512-nagl3RYrbNv6kQkeJIpt6NJZy8twLB/2vtz6yN9Z4vRKHN4/QZJIEbqohALSgwKdnksuY3k5Addp5lg8sVoVcQ==", + "license": "Apache-2.0", "dependencies": { "safe-buffer": "^5.0.1" } }, "node_modules/ejs": { "version": "3.1.10", - "resolved": "https://registry.npmjs.org/ejs/-/ejs-3.1.10.tgz", - "integrity": "sha512-UeJmFfOrAQS8OJWPZ4qtgHyWExa088/MtK5UEyoJGFH67cDEXkZSviOiKRCZ4Xij0zxI3JECgYs3oKx+AizQBA==", - "dev": true, + "license": "Apache-2.0", "dependencies": { "jake": "^10.8.5" }, @@ -4642,16 +4542,12 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.5.20", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.20.tgz", - "integrity": "sha512-74mdl6Fs1HHzK9SUX4CKFxAtAe3nUns48y79TskHNAG6fGOlLfyKA4j855x+0b5u8rWJIrlaG9tcTPstMlwjIw==", - "dev": true + "version": "1.5.4", + "license": "ISC" }, "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, + "license": "MIT", "engines": { "node": ">=12" }, @@ -4661,13 +4557,11 @@ }, "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==" + "license": "MIT" }, "node_modules/encoding": { "version": "0.1.13", - "resolved": "https://registry.npmjs.org/encoding/-/encoding-0.1.13.tgz", - "integrity": "sha512-ETBauow1T35Y/WZMkio9jiM0Z5xjHHmJ4XmjZOq1l/dXz3lr2sRn87nJy20RupqSh1F2m3HHPSp8ShIPQJrJ3A==", + "license": "MIT", "optional": true, "dependencies": { "iconv-lite": "^0.6.2" @@ -4675,17 +4569,15 @@ }, "node_modules/end-of-stream": { "version": "1.4.4", - "resolved": "https://registry.npmjs.org/end-of-stream/-/end-of-stream-1.4.4.tgz", - "integrity": "sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q==", + "license": "MIT", "dependencies": { "once": "^1.4.0" } }, "node_modules/entities": { "version": "4.5.0", - "resolved": "https://registry.npmjs.org/entities/-/entities-4.5.0.tgz", - "integrity": "sha512-V0hjH4dGPh9Ao5p0MoRY6BVqtwCjhz6vI5LT8AJ55H+4g9/4vbHx1I54fS0XuclLhDHArPQCiMjDxjaL8fPxhw==", "dev": true, + "license": "BSD-2-Clause", "engines": { "node": ">=0.12" }, @@ -4695,41 +4587,35 @@ }, "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, + "license": "MIT", "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 + "dev": true, + "license": "MIT" }, "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, + "license": "MIT", "dependencies": { "is-arrayish": "^0.2.1" } }, "node_modules/escalade": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.2.0.tgz", - "integrity": "sha512-WUj2qlxaQtO4g6Pq5c29GTcWGDyd8itL8zTlipgECz3JesAiiOKotd8JU6otB3PACgG6xkJUyVhboMS+bje/jA==", + "version": "3.1.2", + "license": "MIT", "engines": { "node": ">=6" } }, "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -4739,9 +4625,8 @@ }, "node_modules/eslint": { "version": "8.57.0", - "resolved": "https://registry.npmjs.org/eslint/-/eslint-8.57.0.tgz", - "integrity": "sha512-dZ6+mexnaTIbSBZWgou51U6OmzIhYM2VcNdtiTtI7qPNZm35Akpr0f6vtw3w1Kmn5PYo+tZVfh13WrhpS6oLqQ==", "dev": true, + "license": "MIT", "dependencies": { "@eslint-community/eslint-utils": "^4.2.0", "@eslint-community/regexpp": "^4.6.1", @@ -4793,18 +4678,17 @@ } }, "node_modules/eslint-plugin-jest": { - "version": "28.8.3", - "resolved": "https://registry.npmjs.org/eslint-plugin-jest/-/eslint-plugin-jest-28.8.3.tgz", - "integrity": "sha512-HIQ3t9hASLKm2IhIOqnu+ifw7uLZkIlR7RYNv7fMcEi/p0CIiJmfriStQS2LDkgtY4nyLbIZAD+JL347Yc2ETQ==", + "version": "28.6.0", "dev": true, + "license": "MIT", "dependencies": { - "@typescript-eslint/utils": "^6.0.0 || ^7.0.0 || ^8.0.0" + "@typescript-eslint/utils": "^6.0.0 || ^7.0.0" }, "engines": { "node": "^16.10.0 || ^18.12.0 || >=20.0.0" }, "peerDependencies": { - "@typescript-eslint/eslint-plugin": "^6.0.0 || ^7.0.0 || ^8.0.0", + "@typescript-eslint/eslint-plugin": "^6.0.0 || ^7.0.0", "eslint": "^7.0.0 || ^8.0.0 || ^9.0.0", "jest": "*" }, @@ -4819,9 +4703,8 @@ }, "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, + "license": "BSD-2-Clause", "dependencies": { "esrecurse": "^4.3.0", "estraverse": "^5.2.0" @@ -4835,9 +4718,8 @@ }, "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, + "license": "Apache-2.0", "engines": { "node": "^12.22.0 || ^14.17.0 || >=16.0.0" }, @@ -4847,18 +4729,16 @@ }, "node_modules/eslint/node_modules/@eslint/js": { "version": "8.57.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-8.57.0.tgz", - "integrity": "sha512-Ys+3g2TaW7gADOJzPt83SJtCDhMjndcDMFVQ/Tj9iA1BfJzFKD9mAUXT3OenpuPHbI6P/myECxRJrofUsDx/5g==", "dev": true, + "license": "MIT", "engines": { "node": "^12.22.0 || ^14.17.0 || >=16.0.0" } }, "node_modules/eslint/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, + "license": "MIT", "dependencies": { "fast-deep-equal": "^3.1.1", "fast-json-stable-stringify": "^2.0.0", @@ -4872,15 +4752,13 @@ }, "node_modules/eslint/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 + "dev": true, + "license": "MIT" }, "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, + "license": "BSD-2-Clause", "dependencies": { "acorn": "^8.9.0", "acorn-jsx": "^5.3.2", @@ -4895,9 +4773,7 @@ }, "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, + "license": "BSD-2-Clause", "bin": { "esparse": "bin/esparse.js", "esvalidate": "bin/esvalidate.js" @@ -4908,9 +4784,8 @@ }, "node_modules/esquery": { "version": "1.6.0", - "resolved": "https://registry.npmjs.org/esquery/-/esquery-1.6.0.tgz", - "integrity": "sha512-ca9pw9fomFcKPvFLXhBKUK90ZvGibiGOvRJNbjljY7s7uq/5YO4BOzcYtJqExdx99rF6aAcnRxHmcUHcz6sQsg==", "dev": true, + "license": "BSD-3-Clause", "dependencies": { "estraverse": "^5.1.0" }, @@ -4920,9 +4795,8 @@ }, "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, + "license": "BSD-2-Clause", "dependencies": { "estraverse": "^5.2.0" }, @@ -4932,43 +4806,37 @@ }, "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, + "license": "BSD-2-Clause", "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, + "license": "BSD-2-Clause", "engines": { "node": ">=0.10.0" } }, "node_modules/event-target-shim": { "version": "5.0.1", - "resolved": "https://registry.npmjs.org/event-target-shim/-/event-target-shim-5.0.1.tgz", - "integrity": "sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ==", + "license": "MIT", "engines": { "node": ">=6" } }, "node_modules/events": { "version": "3.3.0", - "resolved": "https://registry.npmjs.org/events/-/events-3.3.0.tgz", - "integrity": "sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q==", + "license": "MIT", "engines": { "node": ">=0.8.x" } }, "node_modules/execa": { "version": "5.1.1", - "resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz", - "integrity": "sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg==", - "dev": true, + "license": "MIT", "dependencies": { "cross-spawn": "^7.0.3", "get-stream": "^6.0.0", @@ -4989,18 +4857,13 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/expect-utils": "^29.7.0", "jest-get-type": "^29.6.3", @@ -5014,33 +4877,28 @@ }, "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 + "dev": true, + "license": "Apache-2.0" }, "node_modules/extend": { "version": "3.0.2", - "resolved": "https://registry.npmjs.org/extend/-/extend-3.0.2.tgz", - "integrity": "sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==" + "license": "MIT" }, "node_modules/extsprintf": { "version": "1.3.0", - "resolved": "https://registry.npmjs.org/extsprintf/-/extsprintf-1.3.0.tgz", - "integrity": "sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==", "engines": [ "node >=0.6.0" - ] + ], + "license": "MIT" }, "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==" + "license": "MIT" }, "node_modules/fast-glob": { "version": "3.3.2", - "resolved": "https://registry.npmjs.org/fast-glob/-/fast-glob-3.3.2.tgz", - "integrity": "sha512-oX2ruAFQwf/Orj8m737Y5adxDQO0LAB7/S5MnxCdTNDd4p6BsyIVsv9JQsATbTSq8KHRpLwIHbVlUNatxd+1Ow==", "dev": true, + "license": "MIT", "dependencies": { "@nodelib/fs.stat": "^2.0.2", "@nodelib/fs.walk": "^1.2.3", @@ -5054,9 +4912,8 @@ }, "node_modules/fast-glob/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, + "license": "ISC", "dependencies": { "is-glob": "^4.0.1" }, @@ -5066,24 +4923,19 @@ }, "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==" + "license": "MIT" }, "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 + "dev": true, + "license": "MIT" }, "node_modules/fast-uri": { "version": "3.0.1", - "resolved": "https://registry.npmjs.org/fast-uri/-/fast-uri-3.0.1.tgz", - "integrity": "sha512-MWipKbbYiYI0UC7cl8m/i/IWTqfC8YXsqjzybjddLsFjStroQzsHXkc73JutMvBiXmOvapk+axIl79ig5t55Bw==" + "license": "MIT" }, "node_modules/fast-xml-parser": { "version": "4.4.1", - "resolved": "https://registry.npmjs.org/fast-xml-parser/-/fast-xml-parser-4.4.1.tgz", - "integrity": "sha512-xkjOecfnKGkSsOwtZ5Pz7Us/T6mrbPQrq0nh+aCO5V9nk5NLWmasAHumTKjiPJPWANe+kAZ84Jc8ooJkzZ88Sw==", "funding": [ { "type": "github", @@ -5094,6 +4946,7 @@ "url": "https://paypal.me/naturalintelligence" } ], + "license": "MIT", "dependencies": { "strnum": "^1.0.5" }, @@ -5103,27 +4956,23 @@ }, "node_modules/fastq": { "version": "1.17.1", - "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.17.1.tgz", - "integrity": "sha512-sRVD3lWVIXWg6By68ZN7vho9a1pQcN/WBFaAAsDDFzlJjvoGx0P8z7V1t72grFJfJhu3YPZBuu25f7Kaw2jN1w==", "dev": true, + "license": "ISC", "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", - "integrity": "sha512-p5161BqbuCaSnB8jIbzQHOlpgsPmK5rJVDfDKO91Axs5NC1uu3HRQm6wt9cd9/+GtQQIO53JdGXXoyDpTAsgYA==", - "dev": true, + "license": "Apache-2.0", "dependencies": { "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, + "license": "MIT", "dependencies": { "flat-cache": "^3.0.4" }, @@ -5133,32 +4982,25 @@ }, "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==" + "license": "MIT" }, "node_modules/filelist": { "version": "1.0.4", - "resolved": "https://registry.npmjs.org/filelist/-/filelist-1.0.4.tgz", - "integrity": "sha512-w1cEuf3S+DrLCQL7ET6kz+gmlJdbq9J7yXCSjK/OZCPA+qEN1WyF4ZAf0YYJa4/shHJra2t/d/r8SV4Ji+x+8Q==", - "dev": true, + "license": "Apache-2.0", "dependencies": { "minimatch": "^5.0.1" } }, "node_modules/filelist/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, + "license": "MIT", "dependencies": { "balanced-match": "^1.0.0" } }, "node_modules/filelist/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, + "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" }, @@ -5168,9 +5010,7 @@ }, "node_modules/fill-range": { "version": "7.1.1", - "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", - "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", - "dev": true, + "license": "MIT", "dependencies": { "to-regex-range": "^5.0.1" }, @@ -5180,9 +5020,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==", "dev": true, + "license": "MIT", "dependencies": { "locate-path": "^6.0.0", "path-exists": "^4.0.0" @@ -5196,18 +5035,16 @@ }, "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, + "license": "BSD-3-Clause", "bin": { "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, + "license": "MIT", "dependencies": { "flatted": "^3.2.9", "keyv": "^4.5.3", @@ -5219,20 +5056,18 @@ }, "node_modules/flatted": { "version": "3.3.1", - "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.3.1.tgz", - "integrity": "sha512-X8cqMLLie7KsNUDSdzeN8FYK9rEt4Dt67OsG/DNGnYTSDBG4uFAJFBnUeiV+zCVAvwFy56IjM9sH51jVaEhNxw==", - "dev": true + "dev": true, + "license": "ISC" }, "node_modules/follow-redirects": { - "version": "1.15.9", - "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.9.tgz", - "integrity": "sha512-gew4GsXizNgdoRyqmyfMHyAmXsZDk6mHkSxZFCzW9gwlbtOW44CDtYavM+y+72qD/Vq2l550kMF52DT8fOLJqQ==", + "version": "1.15.6", "funding": [ { "type": "individual", "url": "https://github.com/sponsors/RubenVerborgh" } ], + "license": "MIT", "engines": { "node": ">=4.0" }, @@ -5244,16 +5079,14 @@ }, "node_modules/forever-agent": { "version": "0.6.1", - "resolved": "https://registry.npmjs.org/forever-agent/-/forever-agent-0.6.1.tgz", - "integrity": "sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw==", + "license": "Apache-2.0", "engines": { "node": "*" } }, "node_modules/form-data": { "version": "4.0.0", - "resolved": "https://registry.npmjs.org/form-data/-/form-data-4.0.0.tgz", - "integrity": "sha512-ETEklSGi5t0QMZuiXoA/Q6vcnxcLQP5vdugSpuAyi6SVGi2clPPp+xgEhuMaHC+zGgn31Kd235W35f7Hykkaww==", + "license": "MIT", "dependencies": { "asynckit": "^0.4.0", "combined-stream": "^1.0.8", @@ -5265,8 +5098,7 @@ }, "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==", + "license": "ISC", "dependencies": { "minipass": "^3.0.0" }, @@ -5276,15 +5108,11 @@ }, "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==" + "license": "ISC" }, "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, + "license": "MIT", "optional": true, "os": [ "darwin" @@ -5295,18 +5123,14 @@ }, "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, + "license": "MIT", "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==", - "deprecated": "This package is no longer supported.", + "license": "ISC", "dependencies": { "aproba": "^1.0.3 || ^2.0.0", "color-support": "^1.1.2", @@ -5324,8 +5148,7 @@ }, "node_modules/gaxios": { "version": "6.7.1", - "resolved": "https://registry.npmjs.org/gaxios/-/gaxios-6.7.1.tgz", - "integrity": "sha512-LDODD4TMYx7XXdpwxAVRAIAuB0bzv0s+ywFonY46k126qzQHT9ygyoa9tncmOiQmmDrik65UYsEkv3lbfqQ3yQ==", + "license": "Apache-2.0", "dependencies": { "extend": "^3.0.2", "https-proxy-agent": "^7.0.1", @@ -5337,22 +5160,30 @@ "node": ">=14" } }, - "node_modules/gaxios/node_modules/uuid": { - "version": "9.0.1", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", - "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", - "funding": [ - "https://github.com/sponsors/broofa", - "https://github.com/sponsors/ctavan" - ], - "bin": { - "uuid": "dist/bin/uuid" + "node_modules/gaxios/node_modules/agent-base": { + "version": "7.1.1", + "license": "MIT", + "dependencies": { + "debug": "^4.3.4" + }, + "engines": { + "node": ">= 14" + } + }, + "node_modules/gaxios/node_modules/https-proxy-agent": { + "version": "7.0.5", + "license": "MIT", + "dependencies": { + "agent-base": "^7.0.2", + "debug": "4" + }, + "engines": { + "node": ">= 14" } }, "node_modules/gcp-metadata": { "version": "6.1.0", - "resolved": "https://registry.npmjs.org/gcp-metadata/-/gcp-metadata-6.1.0.tgz", - "integrity": "sha512-Jh/AIwwgaxan+7ZUUmRLCjtchyDiqh4KjBJ5tW3plBZb5iL/BPcso8A5DlzeD9qlw0duCamnNdpFjxwaT0KyKg==", + "license": "Apache-2.0", "dependencies": { "gaxios": "^6.0.0", "json-bigint": "^1.0.0" @@ -5363,35 +5194,28 @@ }, "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, + "license": "MIT", "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==", + "license": "ISC", "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, + "license": "MIT", "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -5401,17 +5225,14 @@ }, "node_modules/getpass": { "version": "0.1.7", - "resolved": "https://registry.npmjs.org/getpass/-/getpass-0.1.7.tgz", - "integrity": "sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng==", + "license": "MIT", "dependencies": { "assert-plus": "^1.0.0" } }, "node_modules/glob": { "version": "7.2.3", - "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz", - "integrity": "sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q==", - "deprecated": "Glob versions prior to v9 are no longer supported", + "license": "ISC", "dependencies": { "fs.realpath": "^1.0.0", "inflight": "^1.0.4", @@ -5429,9 +5250,8 @@ }, "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, + "license": "ISC", "dependencies": { "is-glob": "^4.0.3" }, @@ -5441,9 +5261,8 @@ }, "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, + "license": "MIT", "dependencies": { "type-fest": "^0.20.2" }, @@ -5454,10 +5273,28 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/globby": { + "version": "11.1.0", + "dev": true, + "license": "MIT", + "dependencies": { + "array-union": "^2.1.0", + "dir-glob": "^3.0.1", + "fast-glob": "^3.2.9", + "ignore": "^5.2.0", + "merge2": "^1.4.1", + "slash": "^3.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/google-auth-library": { - "version": "9.14.1", - "resolved": "https://registry.npmjs.org/google-auth-library/-/google-auth-library-9.14.1.tgz", - "integrity": "sha512-Rj+PMjoNFGFTmtItH7gHfbHpGVSb3vmnGK3nwNBqxQF9NoBpttSZI/rc0WiM63ma2uGDQtYEkMHkK9U6937NiA==", + "version": "9.14.0", + "license": "Apache-2.0", "dependencies": { "base64-js": "^1.3.0", "ecdsa-sig-formatter": "^1.0.11", @@ -5471,9 +5308,8 @@ } }, "node_modules/google-gax": { - "version": "4.4.1", - "resolved": "https://registry.npmjs.org/google-gax/-/google-gax-4.4.1.tgz", - "integrity": "sha512-Phyp9fMfA00J3sZbJxbbB4jC55b7DBjE3F6poyL3wKMEBVKA79q6BGuHcTiM28yOzVql0NDbRL8MLLh8Iwk9Dg==", + "version": "4.3.9", + "license": "Apache-2.0", "dependencies": { "@grpc/grpc-js": "^1.10.9", "@grpc/proto-loader": "^0.7.13", @@ -5492,34 +5328,18 @@ "node": ">=14" } }, - "node_modules/google-gax/node_modules/uuid": { - "version": "9.0.1", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", - "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", - "funding": [ - "https://github.com/sponsors/broofa", - "https://github.com/sponsors/ctavan" - ], - "bin": { - "uuid": "dist/bin/uuid" - } - }, "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 + "license": "ISC" }, "node_modules/graphemer": { "version": "1.4.0", - "resolved": "https://registry.npmjs.org/graphemer/-/graphemer-1.4.0.tgz", - "integrity": "sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/gtoken": { "version": "7.1.0", - "resolved": "https://registry.npmjs.org/gtoken/-/gtoken-7.1.0.tgz", - "integrity": "sha512-pCcEwRi+TKpMlxAQObHDQ56KawURgyAf6jtIY046fJ5tIv3zDe/LEIubckAO8fj6JnAxLdmWkUfNyulQ2iKdEw==", + "license": "MIT", "dependencies": { "gaxios": "^6.0.0", "jws": "^4.0.0" @@ -5530,17 +5350,14 @@ }, "node_modules/har-schema": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/har-schema/-/har-schema-2.0.0.tgz", - "integrity": "sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q==", + "license": "ISC", "engines": { "node": ">=4" } }, "node_modules/har-validator": { "version": "5.1.5", - "resolved": "https://registry.npmjs.org/har-validator/-/har-validator-5.1.5.tgz", - "integrity": "sha512-nmT2T0lljbxdQZfspsno9hgrG3Uir6Ks5afism62poxqBM6sDnMEuPmzTq8XN0OEwqKLLdh1jQI3qyE66Nzb3w==", - "deprecated": "this library is no longer supported", + "license": "MIT", "dependencies": { "ajv": "^6.12.3", "har-schema": "^2.0.0" @@ -5551,8 +5368,7 @@ }, "node_modules/har-validator/node_modules/ajv": { "version": "6.12.6", - "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.12.6.tgz", - "integrity": "sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==", + "license": "MIT", "dependencies": { "fast-deep-equal": "^3.1.1", "fast-json-stable-stringify": "^2.0.0", @@ -5566,28 +5382,22 @@ }, "node_modules/har-validator/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==" + "license": "MIT" }, "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, + "license": "MIT", "engines": { "node": ">=8" } }, "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==" + "license": "ISC" }, "node_modules/hasown": { "version": "2.0.2", - "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.2.tgz", - "integrity": "sha512-0hJU9SCPvmMzIBdZFqNPXWa6dqh7WdH0cII9y+CyS8rG3nL48Bclra9HmKhVVUHyPWNH5Y7xDwAB7bfgSjkUMQ==", - "dev": true, + "license": "MIT", "dependencies": { "function-bind": "^1.1.2" }, @@ -5597,41 +5407,36 @@ }, "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, + "license": "MIT", "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 + "license": "MIT" }, "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 + "dev": true, + "license": "BSD-2-Clause" }, "node_modules/http-proxy-agent": { - "version": "7.0.2", - "resolved": "https://registry.npmjs.org/http-proxy-agent/-/http-proxy-agent-7.0.2.tgz", - "integrity": "sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig==", + "version": "5.0.0", + "license": "MIT", "dependencies": { - "agent-base": "^7.1.0", - "debug": "^4.3.4" + "@tootallnate/once": "2", + "agent-base": "6", + "debug": "4" }, "engines": { - "node": ">= 14" + "node": ">= 6" } }, "node_modules/http-signature": { "version": "1.3.6", - "resolved": "https://registry.npmjs.org/http-signature/-/http-signature-1.3.6.tgz", - "integrity": "sha512-3adrsD6zqo4GsTqtO7FyrejHNv+NgiIfAfv68+jVlFmSr9OGy7zrxONceFRLKvnnZA5jbxQBX1u9PpB6Wi32Gw==", + "license": "MIT", "dependencies": { "assert-plus": "^1.0.0", "jsprim": "^2.0.2", @@ -5642,39 +5447,34 @@ } }, "node_modules/https-proxy-agent": { - "version": "7.0.5", - "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz", - "integrity": "sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==", + "version": "5.0.1", + "license": "MIT", "dependencies": { - "agent-base": "^7.0.2", + "agent-base": "6", "debug": "4" }, "engines": { - "node": ">= 14" + "node": ">= 6" } }, "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, + "license": "Apache-2.0", "engines": { "node": ">=10.17.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==", "dev": true, + "license": "MIT", "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==", + "license": "MIT", "optional": true, "dependencies": { "safer-buffer": ">= 2.1.2 < 3.0.0" @@ -5684,19 +5484,17 @@ } }, "node_modules/ignore": { - "version": "5.3.2", - "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.3.2.tgz", - "integrity": "sha512-hsBTNUqQTDwkWtcdYI2i06Y/nUBEsNEDJKjWdigLvegy8kDuJAS8uRlpkkcQpyEXL0Z/pjDy5HBmMjRCJ2gq+g==", + "version": "5.3.1", "dev": true, + "license": "MIT", "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, + "license": "MIT", "dependencies": { "parent-module": "^1.0.0", "resolve-from": "^4.0.0" @@ -5710,9 +5508,7 @@ }, "node_modules/import-local": { "version": "3.2.0", - "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.2.0.tgz", - "integrity": "sha512-2SPlun1JUPWoM6t3F0dw0FkCF/jWY8kttcY4f599GLTSjh2OCuuhdTkJQsEcZzBqbXZGKMK2OqW1oZsjtf/gQA==", - "dev": true, + "license": "MIT", "dependencies": { "pkg-dir": "^4.2.0", "resolve-cwd": "^3.0.0" @@ -5729,33 +5525,27 @@ }, "node_modules/imurmurhash": { "version": "0.1.4", - "resolved": "https://registry.npmjs.org/imurmurhash/-/imurmurhash-0.1.4.tgz", - "integrity": "sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA==", - "dev": true, + "license": "MIT", "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, + "license": "MIT", "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 + "dev": true, + "license": "ISC" }, "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==", - "deprecated": "This module is not supported, and leaks memory. Do not use it. Check out lru-cache if you want a good and tested way to coalesce async requests by a key value, which is much more comprehensive and powerful.", + "license": "ISC", "dependencies": { "once": "^1.3.0", "wrappy": "1" @@ -5763,14 +5553,12 @@ }, "node_modules/inherits": { "version": "2.0.4", - "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", - "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==" + "license": "ISC" }, "node_modules/ip-address": { "version": "9.0.5", - "resolved": "https://registry.npmjs.org/ip-address/-/ip-address-9.0.5.tgz", - "integrity": "sha512-zHtQzGojZXTwZTHQqra+ETKd4Sn3vgi7uBmlPoXVWZqYvuKmtI0l/VZTjqGmJY9x88GGOaZ9+G9ES8hC4T4X8g==", "dev": true, + "license": "MIT", "dependencies": { "jsbn": "1.1.0", "sprintf-js": "^1.1.3" @@ -5781,15 +5569,12 @@ }, "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 + "license": "MIT" }, "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, + "license": "MIT", "dependencies": { "binary-extensions": "^2.0.0" }, @@ -5798,10 +5583,8 @@ } }, "node_modules/is-core-module": { - "version": "2.15.1", - "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.15.1.tgz", - "integrity": "sha512-z0vtXSwucUJtANQWldhbtbt7BnL0vxiFjIdDLAatwhDYty2bad6s+rijD6Ri4YuYJubLzIJLUidCh09e1djEVQ==", - "dev": true, + "version": "2.15.0", + "license": "MIT", "dependencies": { "hasown": "^2.0.2" }, @@ -5814,8 +5597,7 @@ }, "node_modules/is-docker": { "version": "2.2.1", - "resolved": "https://registry.npmjs.org/is-docker/-/is-docker-2.2.1.tgz", - "integrity": "sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ==", + "license": "MIT", "bin": { "is-docker": "cli.js" }, @@ -5828,35 +5610,30 @@ }, "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, + "license": "MIT", "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==", + "license": "MIT", "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, + "license": "MIT", "engines": { "node": ">=6" } }, "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, + "license": "MIT", "dependencies": { "is-extglob": "^2.1.1" }, @@ -5866,41 +5643,35 @@ }, "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 + "dev": true, + "license": "MIT" }, "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, + "license": "MIT", "engines": { "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, + "license": "MIT", "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", - "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==", "dev": true, + "license": "MIT", "engines": { "node": ">=8" } }, "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==", + "license": "MIT", "engines": { "node": ">=8" }, @@ -5910,14 +5681,12 @@ }, "node_modules/is-typedarray": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz", - "integrity": "sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA==" + "license": "MIT" }, "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -5927,8 +5696,7 @@ }, "node_modules/is-wsl": { "version": "2.2.0", - "resolved": "https://registry.npmjs.org/is-wsl/-/is-wsl-2.2.0.tgz", - "integrity": "sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww==", + "license": "MIT", "dependencies": { "is-docker": "^2.0.0" }, @@ -5938,29 +5706,22 @@ }, "node_modules/isexe": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", - "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==", - "dev": true + "license": "ISC" }, "node_modules/isstream": { "version": "0.1.2", - "resolved": "https://registry.npmjs.org/isstream/-/isstream-0.1.2.tgz", - "integrity": "sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g==" + "license": "MIT" }, "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, + "license": "BSD-3-Clause", "engines": { "node": ">=8" } }, "node_modules/istanbul-lib-instrument": { "version": "6.0.3", - "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.3.tgz", - "integrity": "sha512-Vtgk7L/R2JHyyGW07spoFlB8/lpjiOLTjMdms6AFMraYt3BaJauod/NGrfnVG/y4Ix1JEuMRPDPEj2ua+zz1/Q==", - "dev": true, + "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.23.9", "@babel/parser": "^7.23.9", @@ -5974,9 +5735,7 @@ }, "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, + "license": "BSD-3-Clause", "dependencies": { "istanbul-lib-coverage": "^3.0.0", "make-dir": "^4.0.0", @@ -5988,9 +5747,7 @@ }, "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, + "license": "MIT", "dependencies": { "semver": "^7.5.3" }, @@ -6003,9 +5760,7 @@ }, "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, + "license": "BSD-3-Clause", "dependencies": { "debug": "^4.1.1", "istanbul-lib-coverage": "^3.0.0", @@ -6017,9 +5772,7 @@ }, "node_modules/istanbul-reports": { "version": "3.1.7", - "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-3.1.7.tgz", - "integrity": "sha512-BewmUXImeuRk2YY0PVbxgKAysvhRPUQE0h5QRM++nVWyubKGV0l8qQ5op8+B2DOmwSe63Jivj0BjkPQVf8fP5g==", - "dev": true, + "license": "BSD-3-Clause", "dependencies": { "html-escaper": "^2.0.0", "istanbul-lib-report": "^3.0.0" @@ -6030,9 +5783,7 @@ }, "node_modules/jake": { "version": "10.9.2", - "resolved": "https://registry.npmjs.org/jake/-/jake-10.9.2.tgz", - "integrity": "sha512-2P4SQ0HrLQ+fw6llpLnOaGAvN2Zu6778SJMrCUwns4fOoG9ayrTiZk3VV8sCPkVZF8ab0zksVpS8FDY5pRCNBA==", - "dev": true, + "license": "Apache-2.0", "dependencies": { "async": "^3.2.3", "chalk": "^4.0.2", @@ -6048,9 +5799,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", "@jest/types": "^29.6.3", @@ -6074,9 +5823,7 @@ }, "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, + "license": "MIT", "dependencies": { "execa": "^5.0.0", "jest-util": "^29.7.0", @@ -6088,9 +5835,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", "@jest/expect": "^29.7.0", @@ -6119,9 +5864,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", "@jest/test-result": "^29.7.0", @@ -6152,9 +5895,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", "@jest/test-sequencer": "^29.7.0", @@ -6197,9 +5938,7 @@ }, "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, + "license": "MIT", "dependencies": { "chalk": "^4.0.0", "diff-sequences": "^29.6.3", @@ -6212,9 +5951,7 @@ }, "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, + "license": "MIT", "dependencies": { "detect-newline": "^3.0.0" }, @@ -6224,9 +5961,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "chalk": "^4.0.0", @@ -6240,9 +5975,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", "@jest/fake-timers": "^29.7.0", @@ -6257,18 +5990,14 @@ }, "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, + "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "@types/graceful-fs": "^4.1.3", @@ -6291,9 +6020,7 @@ }, "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, + "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3", "pretty-format": "^29.7.0" @@ -6304,9 +6031,7 @@ }, "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, + "license": "MIT", "dependencies": { "chalk": "^4.0.0", "jest-diff": "^29.7.0", @@ -6319,9 +6044,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/code-frame": "^7.12.13", "@jest/types": "^29.6.3", @@ -6339,9 +6062,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "@types/node": "*", @@ -6353,9 +6074,7 @@ }, "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, + "license": "MIT", "engines": { "node": ">=6" }, @@ -6370,18 +6089,14 @@ }, "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, + "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, "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, + "license": "MIT", "dependencies": { "chalk": "^4.0.0", "graceful-fs": "^4.2.9", @@ -6399,9 +6114,7 @@ }, "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, + "license": "MIT", "dependencies": { "jest-regex-util": "^29.6.3", "jest-snapshot": "^29.7.0" @@ -6412,9 +6125,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", "@jest/environment": "^29.7.0", @@ -6444,9 +6155,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", "@jest/fake-timers": "^29.7.0", @@ -6477,9 +6186,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", "@babel/generator": "^7.7.2", @@ -6508,9 +6215,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "@types/node": "*", @@ -6525,9 +6230,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", "camelcase": "^6.2.0", @@ -6542,9 +6245,7 @@ }, "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -6554,9 +6255,7 @@ }, "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, + "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", "@jest/types": "^29.6.3", @@ -6573,9 +6272,7 @@ }, "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, + "license": "MIT", "dependencies": { "@types/node": "*", "jest-util": "^29.7.0", @@ -6588,9 +6285,7 @@ }, "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, + "license": "MIT", "dependencies": { "has-flag": "^4.0.0" }, @@ -6601,17 +6296,33 @@ "url": "https://github.com/chalk/supports-color?sponsor=1" } }, + "node_modules/joi": { + "version": "17.13.3", + "resolved": "https://registry.npmjs.org/joi/-/joi-17.13.3.tgz", + "integrity": "sha512-otDA4ldcIx+ZXsKHWmp0YizCweVRZG96J10b0FevjfuncLO1oX59THoAmHkNubYJ+9gWsYsp5k8v4ib6oDv1fA==", + "license": "BSD-3-Clause", + "dependencies": { + "@hapi/hoek": "^9.3.0", + "@hapi/topo": "^5.1.0", + "@sideway/address": "^4.1.5", + "@sideway/formula": "^3.0.1", + "@sideway/pinpoint": "^2.0.0" + } + }, + "node_modules/joi/node_modules/@hapi/hoek": { + "version": "9.3.0", + "resolved": "https://registry.npmjs.org/@hapi/hoek/-/hoek-9.3.0.tgz", + "integrity": "sha512-/c6rf4UJlmHlC9b5BaNvzAcFv7HZ2QHaV0D4/HNlBdvFnvQq8RI4kYdhyPCl7Xj+oWvTWQ8ujhqS53LIgAe6KQ==", + "license": "BSD-3-Clause" + }, "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 + "license": "MIT" }, "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, + "license": "MIT", "dependencies": { "argparse": "^2.0.1" }, @@ -6621,24 +6332,21 @@ }, "node_modules/js2xmlparser": { "version": "4.0.2", - "resolved": "https://registry.npmjs.org/js2xmlparser/-/js2xmlparser-4.0.2.tgz", - "integrity": "sha512-6n4D8gLlLf1n5mNLQPRfViYzu9RATblzPEtm1SthMX1Pjao0r9YI9nw7ZIfRxQMERS87mcswrg+r/OYrPRX6jA==", "dev": true, + "license": "Apache-2.0", "dependencies": { "xmlcreate": "^2.0.4" } }, "node_modules/jsbn": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-1.1.0.tgz", - "integrity": "sha512-4bYVV3aAMtDTTu4+xsDYa6sy9GyJ69/amsu9sYF2zqjiEoZA5xJi3BrfX3uY+/IekIu7MwdObdbDWpoZdBv3/A==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/jsdoc": { "version": "4.0.3", - "resolved": "https://registry.npmjs.org/jsdoc/-/jsdoc-4.0.3.tgz", - "integrity": "sha512-Nu7Sf35kXJ1MWDZIMAuATRQTg1iIPdzh7tqJ6jjvaU/GfDf+qi5UV8zJR3Mo+/pYFvm8mzay4+6O5EWigaQBQw==", "dev": true, + "license": "Apache-2.0", "dependencies": { "@babel/parser": "^7.20.15", "@jsdoc/salty": "^0.2.1", @@ -6665,18 +6373,15 @@ }, "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, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/jsesc": { "version": "2.5.2", - "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", - "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==", - "dev": true, + "license": "MIT", "bin": { "jsesc": "bin/jsesc" }, @@ -6686,58 +6391,47 @@ }, "node_modules/json-bigint": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/json-bigint/-/json-bigint-1.0.0.tgz", - "integrity": "sha512-SiPv/8VpZuWbvLSMtTDU8hEfrZWg/mH/nV/b4o0CYbSxu1UIQPLdwKOCIyLQX+VIPO5vrLX3i8qtqFyhdPSUSQ==", + "license": "MIT", "dependencies": { "bignumber.js": "^9.0.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 + "dev": true, + "license": "MIT" }, "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 + "license": "MIT" }, "node_modules/json-schema": { "version": "0.4.0", - "resolved": "https://registry.npmjs.org/json-schema/-/json-schema-0.4.0.tgz", - "integrity": "sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA==" + "license": "(AFL-2.1 OR BSD-3-Clause)" }, "node_modules/json-schema-traverse": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz", - "integrity": "sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug==" + "license": "MIT" }, "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 + "dev": true, + "license": "MIT" }, "node_modules/json-stringify-deterministic": { "version": "1.0.12", - "resolved": "https://registry.npmjs.org/json-stringify-deterministic/-/json-stringify-deterministic-1.0.12.tgz", - "integrity": "sha512-q3PN0lbUdv0pmurkBNdJH3pfFvOTL/Zp0lquqpvcjfKzt6Y0j49EPHAmVHCAS4Ceq/Y+PejWTzyiVpoY71+D6g==", + "license": "MIT", "engines": { "node": ">= 4" } }, "node_modules/json-stringify-safe": { "version": "5.0.1", - "resolved": "https://registry.npmjs.org/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz", - "integrity": "sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA==" + "license": "ISC" }, "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, + "license": "MIT", "bin": { "json5": "lib/cli.js" }, @@ -6747,16 +6441,14 @@ }, "node_modules/jsonata": { "version": "2.0.5", - "resolved": "https://registry.npmjs.org/jsonata/-/jsonata-2.0.5.tgz", - "integrity": "sha512-wEse9+QLIIU5IaCgtJCPsFi/H4F3qcikWzF4bAELZiRz08ohfx3Q6CjDRf4ZPF5P/92RI3KIHtb7u3jqPaHXdQ==", + "license": "MIT", "engines": { "node": ">= 8" } }, "node_modules/jsonwebtoken": { "version": "9.0.2", - "resolved": "https://registry.npmjs.org/jsonwebtoken/-/jsonwebtoken-9.0.2.tgz", - "integrity": "sha512-PRp66vJ865SSqOlgqS8hujT5U4AOgMfhrwYIuIhfKaoSCZcirrmASQr8CX7cUg+RMih+hgznrjp99o+W4pJLHQ==", + "license": "MIT", "dependencies": { "jws": "^3.2.2", "lodash.includes": "^4.3.0", @@ -6776,8 +6468,7 @@ }, "node_modules/jsonwebtoken/node_modules/jwa": { "version": "1.4.1", - "resolved": "https://registry.npmjs.org/jwa/-/jwa-1.4.1.tgz", - "integrity": "sha512-qiLX/xhEEFKUAJ6FiBMbes3w9ATzyk5W7Hvzpa/SLYdxNtng+gcurvrI7TbACjIXlsJyr05/S1oUhZrc63evQA==", + "license": "MIT", "dependencies": { "buffer-equal-constant-time": "1.0.1", "ecdsa-sig-formatter": "1.0.11", @@ -6786,8 +6477,7 @@ }, "node_modules/jsonwebtoken/node_modules/jws": { "version": "3.2.2", - "resolved": "https://registry.npmjs.org/jws/-/jws-3.2.2.tgz", - "integrity": "sha512-YHlZCB6lMTllWDtSPHz/ZXTsi8S00usEV6v1tjq8tOUZzw7DpSDWVXjXDre6ed1w/pd495ODpHZYSdkRTsa0HA==", + "license": "MIT", "dependencies": { "jwa": "^1.4.1", "safe-buffer": "^5.0.1" @@ -6795,11 +6485,10 @@ }, "node_modules/jsprim": { "version": "2.0.2", - "resolved": "https://registry.npmjs.org/jsprim/-/jsprim-2.0.2.tgz", - "integrity": "sha512-gqXddjPqQ6G40VdnI6T6yObEC+pDNvyP95wdQhkWkg7crHH3km5qP1FsOXEkzEQwnz6gz5qGTn1c2Y52wP3OyQ==", "engines": [ "node >=0.6.0" ], + "license": "MIT", "dependencies": { "assert-plus": "1.0.0", "extsprintf": "1.3.0", @@ -6809,8 +6498,7 @@ }, "node_modules/jwa": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/jwa/-/jwa-2.0.0.tgz", - "integrity": "sha512-jrZ2Qx916EA+fq9cEAeCROWPTfCwi1IVHqT2tapuqLEVVDKFDENFw1oL+MwrTvH6msKxsd1YTDVw6uKEcsrLEA==", + "license": "MIT", "dependencies": { "buffer-equal-constant-time": "1.0.1", "ecdsa-sig-formatter": "1.0.11", @@ -6819,8 +6507,7 @@ }, "node_modules/jws": { "version": "4.0.0", - "resolved": "https://registry.npmjs.org/jws/-/jws-4.0.0.tgz", - "integrity": "sha512-KDncfTmOZoOMTFG4mBlG0qUIOlc03fmzH+ru6RgYVZhPkyiy/92Owlt/8UEN+a4TXR1FQetfIpJE8ApdvdVxTg==", + "license": "MIT", "dependencies": { "jwa": "^2.0.0", "safe-buffer": "^5.0.1" @@ -6828,45 +6515,38 @@ }, "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, + "license": "MIT", "dependencies": { "json-buffer": "3.0.1" } }, "node_modules/klaw": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/klaw/-/klaw-3.0.0.tgz", - "integrity": "sha512-0Fo5oir+O9jnXu5EefYbVK+mHMBeEVEy2cmctR1O1NECcCkPRreJKrS6Qt/j3KC2C148Dfo9i3pCmCMsdqGr0g==", "dev": true, + "license": "MIT", "dependencies": { "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, + "license": "MIT", "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, + "license": "MIT", "engines": { "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, + "license": "MIT", "dependencies": { "prelude-ls": "^1.2.1", "type-check": "~0.4.0" @@ -6877,24 +6557,20 @@ }, "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 + "license": "MIT" }, "node_modules/linkify-it": { "version": "5.0.0", - "resolved": "https://registry.npmjs.org/linkify-it/-/linkify-it-5.0.0.tgz", - "integrity": "sha512-5aHCbzQRADcdP+ATqnDuhhJ/MRIqDkZX5pyjFHRRysS8vZ5AbqGEoFIb6pYHPZ+L/OC2Lc+xT8uHVVR5CAK/wQ==", "dev": true, + "license": "MIT", "dependencies": { "uc.micro": "^2.0.0" } }, "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, + "license": "MIT", "dependencies": { "p-locate": "^5.0.0" }, @@ -6907,67 +6583,54 @@ }, "node_modules/lodash": { "version": "4.17.21", - "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.21.tgz", - "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/lodash.camelcase": { "version": "4.3.0", - "resolved": "https://registry.npmjs.org/lodash.camelcase/-/lodash.camelcase-4.3.0.tgz", - "integrity": "sha512-TwuEnCnxbc3rAvhf/LbG7tJUDzhqXyFnv3dtzLOPgCG/hODL7WFnsbwktkD7yUV0RrreP/l1PALq/YSg6VvjlA==" + "license": "MIT" }, "node_modules/lodash.includes": { "version": "4.3.0", - "resolved": "https://registry.npmjs.org/lodash.includes/-/lodash.includes-4.3.0.tgz", - "integrity": "sha512-W3Bx6mdkRTGtlJISOvVD/lbqjTlPPUDTMnlXZFnVwi9NKJ6tiAk6LVdlhZMm17VZisqhKcgzpO5Wz91PCt5b0w==" + "license": "MIT" }, "node_modules/lodash.isboolean": { "version": "3.0.3", - "resolved": "https://registry.npmjs.org/lodash.isboolean/-/lodash.isboolean-3.0.3.tgz", - "integrity": "sha512-Bz5mupy2SVbPHURB98VAcw+aHh4vRV5IPNhILUCsOzRmsTmSQ17jIuqopAentWoehktxGd9e/hbIXq980/1QJg==" + "license": "MIT" }, "node_modules/lodash.isinteger": { "version": "4.0.4", - "resolved": "https://registry.npmjs.org/lodash.isinteger/-/lodash.isinteger-4.0.4.tgz", - "integrity": "sha512-DBwtEWN2caHQ9/imiNeEA5ys1JoRtRfY3d7V9wkqtbycnAmTvRRmbHKDV4a0EYc678/dia0jrte4tjYwVBaZUA==" + "license": "MIT" }, "node_modules/lodash.isnumber": { "version": "3.0.3", - "resolved": "https://registry.npmjs.org/lodash.isnumber/-/lodash.isnumber-3.0.3.tgz", - "integrity": "sha512-QYqzpfwO3/CWf3XP+Z+tkQsfaLL/EnUlXWVkIk5FUPc4sBdTehEqZONuyRt2P67PXAk+NXmTBcc97zw9t1FQrw==" + "license": "MIT" }, "node_modules/lodash.isplainobject": { "version": "4.0.6", - "resolved": "https://registry.npmjs.org/lodash.isplainobject/-/lodash.isplainobject-4.0.6.tgz", - "integrity": "sha512-oSXzaWypCMHkPC3NvBEaPHf0KsA5mvPrOPgQWDsbg8n7orZ290M0BmC/jgRZ4vcJ6DTAhjrsSYgdsW/F+MFOBA==" + "license": "MIT" }, "node_modules/lodash.isstring": { "version": "4.0.1", - "resolved": "https://registry.npmjs.org/lodash.isstring/-/lodash.isstring-4.0.1.tgz", - "integrity": "sha512-0wJxfxH1wgO3GrbuP+dTTk7op+6L41QCXbGINEmD+ny/G/eCqGzxyCsh7159S+mgDDcoarnBw6PC1PS5+wUGgw==" + "license": "MIT" }, "node_modules/lodash.memoize": { "version": "4.1.2", - "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", - "integrity": "sha512-t7j+NzmgnQzTAYXcsHYLgimltOV1MXHtlOWf6GjL9Kj8GK5FInw5JotxvbOs+IvV1/Dzo04/fCGfLVs7aXb4Ag==", - "dev": true + "license": "MIT" }, "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 + "dev": true, + "license": "MIT" }, "node_modules/lodash.once": { "version": "4.1.1", - "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz", - "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==" + "license": "MIT" }, "node_modules/log-symbols": { "version": "4.1.0", - "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz", - "integrity": "sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg==", "dev": true, + "license": "MIT", "dependencies": { "chalk": "^4.1.0", "is-unicode-supported": "^0.1.0" @@ -6981,21 +6644,18 @@ }, "node_modules/long": { "version": "5.2.3", - "resolved": "https://registry.npmjs.org/long/-/long-5.2.3.tgz", - "integrity": "sha512-lcHwpNoggQTObv5apGNCTdJrO69eHOZMi4BNC+rTLER8iHAqGrUVeLh/irVIM7zTw2bOXA8T6uNPeujwOLg/2Q==" + "license": "Apache-2.0" }, "node_modules/lru-cache": { - "version": "11.0.1", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-11.0.1.tgz", - "integrity": "sha512-CgeuL5uom6j/ZVrg7G/+1IXqRY8JXX4Hghfy5YE0EhoYQWvndP1kufu58cmZLNIDKnRhZrXfdS9urVWx98AipQ==", + "version": "11.0.0", + "license": "ISC", "engines": { "node": "20 || >=22" } }, "node_modules/make-dir": { "version": "3.1.0", - "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-3.1.0.tgz", - "integrity": "sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw==", + "license": "MIT", "dependencies": { "semver": "^6.0.0" }, @@ -7008,23 +6668,19 @@ }, "node_modules/make-dir/node_modules/semver": { "version": "6.3.1", - "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.1.tgz", - "integrity": "sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA==", + "license": "ISC", "bin": { "semver": "bin/semver.js" } }, "node_modules/make-error": { "version": "1.3.6", - "resolved": "https://registry.npmjs.org/make-error/-/make-error-1.3.6.tgz", - "integrity": "sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw==", - "dev": true + "license": "ISC" }, "node_modules/make-fetch-happen": { "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, + "license": "ISC", "dependencies": { "agentkeepalive": "^4.2.1", "cacache": "^16.1.0", @@ -7047,68 +6703,25 @@ "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, - "node_modules/make-fetch-happen/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==", - "dev": true, - "dependencies": { - "debug": "4" - }, - "engines": { - "node": ">= 6.0.0" - } - }, - "node_modules/make-fetch-happen/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": { - "@tootallnate/once": "2", - "agent-base": "6", - "debug": "4" - }, - "engines": { - "node": ">= 6" - } - }, - "node_modules/make-fetch-happen/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==", - "dev": true, - "dependencies": { - "agent-base": "6", - "debug": "4" - }, - "engines": { - "node": ">= 6" - } - }, "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, + "license": "ISC", "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, + "license": "BSD-3-Clause", "dependencies": { "tmpl": "1.0.5" } }, "node_modules/markdown-it": { "version": "14.1.0", - "resolved": "https://registry.npmjs.org/markdown-it/-/markdown-it-14.1.0.tgz", - "integrity": "sha512-a54IwgWPaeBCAAsv13YgmALOF1elABB08FxO9i+r4VFk5Vl4pKokRPeX8u5TCgSsPi6ec1otfLjdOpVcgbpshg==", "dev": true, + "license": "MIT", "dependencies": { "argparse": "^2.0.1", "entities": "^4.4.0", @@ -7123,9 +6736,8 @@ }, "node_modules/markdown-it-anchor": { "version": "8.6.7", - "resolved": "https://registry.npmjs.org/markdown-it-anchor/-/markdown-it-anchor-8.6.7.tgz", - "integrity": "sha512-FlCHFwNnutLgVTflOYHPW2pPcl2AACqVzExlkGQNsi4CJgqOHN7YTgDd4LuhgN1BFO3TS0vLAruV1Td6dwWPJA==", "dev": true, + "license": "Unlicense", "peerDependencies": { "@types/markdown-it": "*", "markdown-it": "*" @@ -7133,9 +6745,8 @@ }, "node_modules/marked": { "version": "4.3.0", - "resolved": "https://registry.npmjs.org/marked/-/marked-4.3.0.tgz", - "integrity": "sha512-PRsaiG84bK+AMvxziE/lCFss8juXjNaWzVbN5tXAm4XjeaS9NAHhop+PjQxz2A9h8Q4M/xGmzP8vqNwy6JeK0A==", "dev": true, + "license": "MIT", "bin": { "marked": "bin/marked.js" }, @@ -7145,30 +6756,29 @@ }, "node_modules/mdurl": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/mdurl/-/mdurl-2.0.0.tgz", - "integrity": "sha512-Lf+9+2r+Tdp5wXDXC4PcIBjTDtq4UKjCPMQhKIuzpJNW0b96kVqSwW0bT7FhRSfmAiFYgP+SCRvdrDozfh0U5w==", - "dev": true + "dev": true, + "license": "MIT" }, "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 + "license": "MIT" }, "node_modules/merge2": { "version": "1.4.1", - "resolved": "https://registry.npmjs.org/merge2/-/merge2-1.4.1.tgz", - "integrity": "sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg==", "dev": true, + "license": "MIT", "engines": { "node": ">= 8" } }, "node_modules/micromatch": { "version": "4.0.8", +<<<<<<< HEAD +======= "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz", "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==", - "dev": true, +>>>>>>> 242a0f7 (Add OAuth Support to Rest Service (#25)) + "license": "MIT", "dependencies": { "braces": "^3.0.3", "picomatch": "^2.3.1" @@ -7179,16 +6789,14 @@ }, "node_modules/mime-db": { "version": "1.52.0", - "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.52.0.tgz", - "integrity": "sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==", + "license": "MIT", "engines": { "node": ">= 0.6" } }, "node_modules/mime-types": { "version": "2.1.35", - "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.35.tgz", - "integrity": "sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==", + "license": "MIT", "dependencies": { "mime-db": "1.52.0" }, @@ -7198,17 +6806,14 @@ }, "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, + "license": "MIT", "engines": { "node": ">=6" } }, "node_modules/minimatch": { "version": "3.1.2", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", - "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "license": "ISC", "dependencies": { "brace-expansion": "^1.1.7" }, @@ -7218,8 +6823,7 @@ }, "node_modules/minipass": { "version": "3.3.6", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", - "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", + "license": "ISC", "dependencies": { "yallist": "^4.0.0" }, @@ -7229,9 +6833,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, + "license": "ISC", "dependencies": { "minipass": "^3.0.0" }, @@ -7241,9 +6844,8 @@ }, "node_modules/minipass-fetch": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/minipass-fetch/-/minipass-fetch-2.1.2.tgz", - "integrity": "sha512-LT49Zi2/WMROHYoqGgdlQIZh8mLPZmOrN2NdJjMXxYe4nkN6FUyuPuOAOedNJDrx0IRGg9+4guZewtp8hE6TxA==", "dev": true, + "license": "MIT", "dependencies": { "minipass": "^3.1.6", "minipass-sized": "^1.0.3", @@ -7258,9 +6860,8 @@ }, "node_modules/minipass-flush": { "version": "1.0.5", - "resolved": "https://registry.npmjs.org/minipass-flush/-/minipass-flush-1.0.5.tgz", - "integrity": "sha512-JmQSYYpPUqX5Jyn1mXaRwOda1uQ8HP5KAT/oDSLCzt1BYRhQU0/hDtsB1ufZfEEzMZ9aAVmsBw8+FWsIXlClWw==", "dev": true, + "license": "ISC", "dependencies": { "minipass": "^3.0.0" }, @@ -7270,9 +6871,8 @@ }, "node_modules/minipass-pipeline": { "version": "1.2.4", - "resolved": "https://registry.npmjs.org/minipass-pipeline/-/minipass-pipeline-1.2.4.tgz", - "integrity": "sha512-xuIq7cIOt09RPRJ19gdi4b+RiNvDFYe5JH+ggNvBqGqpQXcru3PcRmOZuHBKWK1Txf9+cQ+HMVN4d6z46LZP7A==", "dev": true, + "license": "ISC", "dependencies": { "minipass": "^3.0.0" }, @@ -7282,9 +6882,8 @@ }, "node_modules/minipass-sized": { "version": "1.0.3", - "resolved": "https://registry.npmjs.org/minipass-sized/-/minipass-sized-1.0.3.tgz", - "integrity": "sha512-MbkQQ2CTiBMlA2Dm/5cY+9SWFEN8pzzOXi6rlM5Xxq0Yqbda5ZQy9sU75a673FE9ZK0Zsbr6Y5iP6u9nktfg2g==", "dev": true, + "license": "ISC", "dependencies": { "minipass": "^3.0.0" }, @@ -7292,10 +6891,13 @@ "node": ">=8" } }, + "node_modules/minipass/node_modules/yallist": { + "version": "4.0.0", + "license": "ISC" + }, "node_modules/minizlib": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/minizlib/-/minizlib-2.1.2.tgz", - "integrity": "sha512-bAxsR8BVfj60DWXHE3u30oHzfl4G7khkSuPW+qvpd7jFRHm7dLxOjUk1EHACJ/hxLY8phGJ0YhYHZo7jil7Qdg==", + "license": "MIT", "dependencies": { "minipass": "^3.0.0", "yallist": "^4.0.0" @@ -7304,10 +6906,13 @@ "node": ">= 8" } }, + "node_modules/minizlib/node_modules/yallist": { + "version": "4.0.0", + "license": "ISC" + }, "node_modules/mkdirp": { "version": "1.0.4", - "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-1.0.4.tgz", - "integrity": "sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw==", + "license": "MIT", "bin": { "mkdirp": "bin/cmd.js" }, @@ -7316,10 +6921,9 @@ } }, "node_modules/mocha": { - "version": "10.7.3", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-10.7.3.tgz", - "integrity": "sha512-uQWxAu44wwiACGqjbPYmjo7Lg8sFrS3dQe7PP2FQI+woptP4vZXSMcfMyFL/e1yFEeEpV4RtyTpZROOKmxis+A==", + "version": "10.7.0", "dev": true, + "license": "MIT", "dependencies": { "ansi-colors": "^4.1.3", "browser-stdout": "^1.3.1", @@ -7352,18 +6956,16 @@ }, "node_modules/mocha/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, + "license": "MIT", "dependencies": { "balanced-match": "^1.0.0" } }, "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, + "license": "ISC", "dependencies": { "string-width": "^4.2.0", "strip-ansi": "^6.0.0", @@ -7372,10 +6974,8 @@ }, "node_modules/mocha/node_modules/glob": { "version": "8.1.0", - "resolved": "https://registry.npmjs.org/glob/-/glob-8.1.0.tgz", - "integrity": "sha512-r8hpEjiQEYlF2QU0df3dS+nxxSIreXQS1qRhMJM0Q5NDdR386C7jb7Hwwod8Fgiuex+k0GFjgft18yvxm5XoCQ==", - "deprecated": "Glob versions prior to v9 are no longer supported", "dev": true, + "license": "ISC", "dependencies": { "fs.realpath": "^1.0.0", "inflight": "^1.0.4", @@ -7392,9 +6992,8 @@ }, "node_modules/mocha/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, + "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" }, @@ -7402,11 +7001,15 @@ "node": ">=10" } }, + "node_modules/mocha/node_modules/ms": { + "version": "2.1.3", + "dev": true, + "license": "MIT" + }, "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, + "license": "MIT", "dependencies": { "has-flag": "^4.0.0" }, @@ -7419,9 +7022,8 @@ }, "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, + "license": "MIT", "dependencies": { "cliui": "^7.0.2", "escalade": "^3.1.1", @@ -7437,37 +7039,30 @@ }, "node_modules/moo": { "version": "0.5.2", - "resolved": "https://registry.npmjs.org/moo/-/moo-0.5.2.tgz", - "integrity": "sha512-iSAJLHYKnX41mKcJKjqvnAN9sf0LMDTXDEvFv+ffuRR9a1MIuXLjMNL6EsnDHSkKLTWNqQQ5uo61P4EbU4NU+Q==" + "license": "BSD-3-Clause" }, "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==" + "version": "2.1.2", + "license": "MIT" }, "node_modules/mustache": { "version": "4.2.0", - "resolved": "https://registry.npmjs.org/mustache/-/mustache-4.2.0.tgz", - "integrity": "sha512-71ippSywq5Yb7/tVYyGbkBggbU8H3u5Rz56fH60jGFgr8uHwxs+aSKeqmluIVzM0m0kB7xQjKS6qPfd0b2ZoqQ==", + "license": "MIT", "bin": { "mustache": "bin/mustache" } }, "node_modules/nan": { "version": "2.20.0", - "resolved": "https://registry.npmjs.org/nan/-/nan-2.20.0.tgz", - "integrity": "sha512-bk3gXBZDGILuuo/6sKtr0DQmSThYHLtNCdSdXk9YkxD/jK6X2vmCyyXBBxyqZ4XcnzTyYEAThfX3DCEnLf6igw==" + "license": "MIT" }, "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 + "license": "MIT" }, "node_modules/nearley": { "version": "2.20.1", - "resolved": "https://registry.npmjs.org/nearley/-/nearley-2.20.1.tgz", - "integrity": "sha512-+Mc8UaAebFzgV+KpI5n7DasuuQCHA89dmwm7JXw3TV43ukfNQ9DnBH3Mdb2g/I4Fdxc26pwimBWvjIw0UAILSQ==", + "license": "MIT", "dependencies": { "commander": "^2.19.0", "moo": "^0.5.0", @@ -7487,17 +7082,15 @@ }, "node_modules/negotiator": { "version": "0.6.3", - "resolved": "https://registry.npmjs.org/negotiator/-/negotiator-0.6.3.tgz", - "integrity": "sha512-+EUsqGPLsM+j/zdChZjsnX51g4XrHFOIXwfnCVPGlQk/k5giakcKsuxCObBRu6DSm9opw/O6slWbJdghQM4bBg==", "dev": true, + "license": "MIT", "engines": { "node": ">= 0.6" } }, "node_modules/node-fetch": { "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==", + "license": "MIT", "dependencies": { "whatwg-url": "^5.0.0" }, @@ -7515,9 +7108,8 @@ }, "node_modules/node-gyp": { "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, + "license": "MIT", "dependencies": { "env-paths": "^2.2.0", "exponential-backoff": "^3.1.1", @@ -7540,10 +7132,8 @@ }, "node_modules/node-gyp/node_modules/are-we-there-yet": { "version": "3.0.1", - "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-3.0.1.tgz", - "integrity": "sha512-QZW4EDmGwlYur0Yyf/b2uGucHQMa8aFUP7eu9ddR73vvhFyt4V0Vl3QHPcTNJ8l6qYOBdxgXdnBXQrHilfRQBg==", - "deprecated": "This package is no longer supported.", "dev": true, + "license": "ISC", "dependencies": { "delegates": "^1.0.0", "readable-stream": "^3.6.0" @@ -7554,10 +7144,8 @@ }, "node_modules/node-gyp/node_modules/gauge": { "version": "4.0.4", - "resolved": "https://registry.npmjs.org/gauge/-/gauge-4.0.4.tgz", - "integrity": "sha512-f9m+BEN5jkg6a0fZjleidjN51VE1X+mPFQ2DJ0uv1V39oCLCbsGe6yjbBnp7eK7z/+GAon99a3nHuqbuuthyPg==", - "deprecated": "This package is no longer supported.", "dev": true, + "license": "ISC", "dependencies": { "aproba": "^1.0.3 || ^2.0.0", "color-support": "^1.1.3", @@ -7574,9 +7162,8 @@ }, "node_modules/node-gyp/node_modules/nopt": { "version": "6.0.0", - "resolved": "https://registry.npmjs.org/nopt/-/nopt-6.0.0.tgz", - "integrity": "sha512-ZwLpbTgdhuZUnZzjd7nb1ZV+4DoiC6/sfiVKok72ym/4Tlf+DFdlHYmT2JPmcNNWV6Pi3SDf1kT+A4r9RTuT9g==", "dev": true, + "license": "ISC", "dependencies": { "abbrev": "^1.0.0" }, @@ -7589,10 +7176,8 @@ }, "node_modules/node-gyp/node_modules/npmlog": { "version": "6.0.2", - "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-6.0.2.tgz", - "integrity": "sha512-/vBvz5Jfr9dT/aFWd0FIRf+T/Q2WBsLENygUaFUqstqsycmZAP/t5BvFJTK0viFmSUxiUKTUplWy5vt+rvKIxg==", - "deprecated": "This package is no longer supported.", "dev": true, + "license": "ISC", "dependencies": { "are-we-there-yet": "^3.0.0", "console-control-strings": "^1.1.0", @@ -7605,20 +7190,15 @@ }, "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 + "license": "MIT" }, "node_modules/node-releases": { "version": "2.0.18", - "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.18.tgz", - "integrity": "sha512-d9VeXT4SJ7ZeOqGX6R5EM022wpL+eWPooLI+5UpWn2jCT1aosUQEhQP214x33Wkwx3JQMvIm+tIoVOdodFS40g==", - "dev": true + "license": "MIT" }, "node_modules/node-vault": { "version": "0.10.2", - "resolved": "https://registry.npmjs.org/node-vault/-/node-vault-0.10.2.tgz", - "integrity": "sha512-//uc9/YImE7Dx0QHdwMiAzLaOumiKUnOUP8DymgtkZ8nsq6/V2LKvEu6kw91Lcruw8lWUfj4DO7CIXNPRWBuuA==", + "license": "MIT", "dependencies": { "debug": "^4.3.4", "mustache": "^4.2.0", @@ -7631,8 +7211,7 @@ }, "node_modules/nopt": { "version": "5.0.0", - "resolved": "https://registry.npmjs.org/nopt/-/nopt-5.0.0.tgz", - "integrity": "sha512-Tbj67rffqceeLpcRXrT7vKAN8CwfPeIBgM7E6iBkmKLV7bEMwpGgYLGv0jACUsECaa/vuxP0IjEont6umdMgtQ==", + "license": "ISC", "dependencies": { "abbrev": "1" }, @@ -7645,18 +7224,14 @@ }, "node_modules/normalize-path": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", - "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", - "dev": true, + "license": "MIT", "engines": { "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, + "license": "MIT", "dependencies": { "path-key": "^3.0.0" }, @@ -7666,9 +7241,7 @@ }, "node_modules/npmlog": { "version": "5.0.1", - "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-5.0.1.tgz", - "integrity": "sha512-AqZtDUWOMKs1G/8lwylVjrdYgqA4d9nu8hc+0gzRxlDb1I10+FHBGMXs6aiQHFdCUUlqH99MUMuLfzWDNDtfxw==", - "deprecated": "This package is no longer supported.", + "license": "ISC", "dependencies": { "are-we-there-yet": "^2.0.0", "console-control-strings": "^1.1.0", @@ -7678,41 +7251,35 @@ }, "node_modules/oauth-sign": { "version": "0.9.0", - "resolved": "https://registry.npmjs.org/oauth-sign/-/oauth-sign-0.9.0.tgz", - "integrity": "sha512-fexhUFFPTGV8ybAtSIGbV6gOkSv8UtRbDBnAyLQw4QPKkgNlsH2ByPGtMUqdWkos6YCRmAqViwgZrJc/mRDzZQ==", + "license": "Apache-2.0", "engines": { "node": "*" } }, "node_modules/object-assign": { "version": "4.1.1", - "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", - "integrity": "sha512-rJgTQnkUnH1sFw8yT6VSU3zD3sWmu6sZhIseY8VX+GRu3P6F7Fu+JNDoXfklElbLJSnc3FUQHVe4cU5hj+BcUg==", + "license": "MIT", "engines": { "node": ">=0.10.0" } }, "node_modules/object-hash": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/object-hash/-/object-hash-3.0.0.tgz", - "integrity": "sha512-RSn9F68PjH9HqtltsSnqYC1XXoWe9Bju5+213R98cNGttag9q9yAOTzdbsqvIa7aNm5WffBZFpWYr2aWrklWAw==", + "license": "MIT", "engines": { "node": ">= 6" } }, "node_modules/once": { "version": "1.4.0", - "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", - "integrity": "sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==", + "license": "ISC", "dependencies": { "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, + "license": "MIT", "dependencies": { "mimic-fn": "^2.1.0" }, @@ -7725,8 +7292,7 @@ }, "node_modules/open": { "version": "8.4.2", - "resolved": "https://registry.npmjs.org/open/-/open-8.4.2.tgz", - "integrity": "sha512-7x81NCL719oNbsq/3mh+hVrAWmFuEYUqrq/Iw3kUzH8ReypT9QQ0BLoJS7/G9k6N81XjW4qHWtjWwe/9eLy1EQ==", + "license": "MIT", "dependencies": { "define-lazy-prop": "^2.0.0", "is-docker": "^2.1.1", @@ -7741,9 +7307,8 @@ }, "node_modules/optionator": { "version": "0.9.4", - "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.4.tgz", - "integrity": "sha512-6IpQ7mKUxRcZNLIObR0hz7lxsapSSIYNZJwXPGeF0mTVqGKFIXj1DQcMoT22S3ROcLyY/rz0PWaWZ9ayWmad9g==", "dev": true, + "license": "MIT", "dependencies": { "deep-is": "^0.1.3", "fast-levenshtein": "^2.0.6", @@ -7758,9 +7323,7 @@ }, "node_modules/p-limit": { "version": "3.1.0", - "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", - "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", - "dev": true, + "license": "MIT", "dependencies": { "yocto-queue": "^0.1.0" }, @@ -7773,9 +7336,8 @@ }, "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, + "license": "MIT", "dependencies": { "p-limit": "^3.0.2" }, @@ -7788,9 +7350,8 @@ }, "node_modules/p-map": { "version": "4.0.0", - "resolved": "https://registry.npmjs.org/p-map/-/p-map-4.0.0.tgz", - "integrity": "sha512-/bjOqmgETBYB5BoEeGVea8dmvHb2m9GLy1E9W43yeyfP6QQCZGFNa+XRceJEuDB6zqr+gKpIAmlLebMpykw/MQ==", "dev": true, + "license": "MIT", "dependencies": { "aggregate-error": "^3.0.0" }, @@ -7803,18 +7364,15 @@ }, "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, + "license": "MIT", "engines": { "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, + "license": "MIT", "dependencies": { "callsites": "^3.0.0" }, @@ -7824,9 +7382,7 @@ }, "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, + "license": "MIT", "dependencies": { "@babel/code-frame": "^7.0.0", "error-ex": "^1.3.1", @@ -7842,52 +7398,48 @@ }, "node_modules/path-exists": { "version": "4.0.0", - "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", - "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==", - "dev": true, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/path-is-absolute": { "version": "1.0.1", - "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", - "integrity": "sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg==", + "license": "MIT", "engines": { "node": ">=0.10.0" } }, "node_modules/path-key": { "version": "3.1.1", - "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", - "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", - "dev": true, + "license": "MIT", "engines": { "node": ">=8" } }, "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 + "license": "MIT" + }, + "node_modules/path-type": { + "version": "4.0.0", + "dev": true, + "license": "MIT", + "engines": { + "node": ">=8" + } }, "node_modules/performance-now": { "version": "2.1.0", - "resolved": "https://registry.npmjs.org/performance-now/-/performance-now-2.1.0.tgz", - "integrity": "sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==" + "license": "MIT" }, "node_modules/picocolors": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.1.0.tgz", - "integrity": "sha512-TQ92mBOW0l3LeMeyLV6mzy/kWr8lkd/hp3mTg7wYK7zJhuBStmGMBG0BdeDZS/dZx1IukaX6Bk11zcln25o1Aw==", - "dev": true + "version": "1.0.1", + "license": "ISC" }, "node_modules/picomatch": { "version": "2.3.1", - "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", - "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", - "dev": true, + "license": "MIT", "engines": { "node": ">=8.6" }, @@ -7897,18 +7449,14 @@ }, "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, + "license": "MIT", "engines": { "node": ">= 6" } }, "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, + "license": "MIT", "dependencies": { "find-up": "^4.0.0" }, @@ -7918,9 +7466,7 @@ }, "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, + "license": "MIT", "dependencies": { "locate-path": "^5.0.0", "path-exists": "^4.0.0" @@ -7931,9 +7477,7 @@ }, "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, + "license": "MIT", "dependencies": { "p-locate": "^4.1.0" }, @@ -7943,9 +7487,7 @@ }, "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, + "license": "MIT", "dependencies": { "p-try": "^2.0.0" }, @@ -7958,9 +7500,7 @@ }, "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, + "license": "MIT", "dependencies": { "p-limit": "^2.2.0" }, @@ -7969,9 +7509,8 @@ } }, "node_modules/postman-request": { - "version": "2.88.1-postman.40", - "resolved": "https://registry.npmjs.org/postman-request/-/postman-request-2.88.1-postman.40.tgz", - "integrity": "sha512-uE4AiIqhjtHKp4pj9ei7fkdfNXEX9IqDBlK1plGAQne6y79UUlrTdtYLhwXoO0AMOvqyl9Ar+BU6Eo6P/MPgfg==", + "version": "2.88.1-postman.39", + "license": "Apache-2.0", "dependencies": { "@postman/form-data": "~3.1.1", "@postman/tough-cookie": "~4.1.3-postman.1", @@ -8000,20 +7539,24 @@ "node": ">= 16" } }, + "node_modules/postman-request/node_modules/uuid": { + "version": "8.3.2", + "license": "MIT", + "bin": { + "uuid": "dist/bin/uuid" + } + }, "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, + "license": "MIT", "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", - "integrity": "sha512-Pdlw/oPxN+aXdmM9R00JVC9WVFoCLTKJvDVLgmJ+qAffBMxsV85l/Lu7sNx4zSzPyoL2euImuEwHhOXdEgNFZQ==", - "dev": true, + "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", "ansi-styles": "^5.0.0", @@ -8025,9 +7568,7 @@ }, "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -8037,15 +7578,13 @@ }, "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 + "dev": true, + "license": "ISC" }, "node_modules/promise-retry": { "version": "2.0.1", - "resolved": "https://registry.npmjs.org/promise-retry/-/promise-retry-2.0.1.tgz", - "integrity": "sha512-y+WKFlBR8BGXnsNlIHFGPZmyDf3DFMoLhaflAnyZgV6rG6xu+JwesTo2Q9R6XwYmtmwAFCkAk3e35jEdoeh/3g==", "dev": true, + "license": "MIT", "dependencies": { "err-code": "^2.0.2", "retry": "^0.12.0" @@ -8056,9 +7595,7 @@ }, "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, + "license": "MIT", "dependencies": { "kleur": "^3.0.3", "sisteransi": "^1.0.5" @@ -8069,8 +7606,7 @@ }, "node_modules/proto3-json-serializer": { "version": "2.0.2", - "resolved": "https://registry.npmjs.org/proto3-json-serializer/-/proto3-json-serializer-2.0.2.tgz", - "integrity": "sha512-SAzp/O4Yh02jGdRc+uIrGoe87dkN/XtwxfZ4ZyafJHymd79ozp5VG5nyZ7ygqPM5+cpLDjjGnYFUkngonyDPOQ==", + "license": "Apache-2.0", "dependencies": { "protobufjs": "^7.2.5" }, @@ -8080,9 +7616,8 @@ }, "node_modules/protobufjs": { "version": "7.4.0", - "resolved": "https://registry.npmjs.org/protobufjs/-/protobufjs-7.4.0.tgz", - "integrity": "sha512-mRUWCc3KUU4w1jU8sGxICXH/gNS94DvI1gxqDvBzhj1JpcsimQkYiOJfwsPUykUI5ZaspFbSgmBLER8IrQ3tqw==", "hasInstallScript": true, + "license": "BSD-3-Clause", "dependencies": { "@protobufjs/aspromise": "^1.1.2", "@protobufjs/base64": "^1.1.2", @@ -8103,36 +7638,29 @@ }, "node_modules/proxy-from-env": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/proxy-from-env/-/proxy-from-env-1.1.0.tgz", - "integrity": "sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg==" + "license": "MIT" }, "node_modules/psl": { "version": "1.9.0", - "resolved": "https://registry.npmjs.org/psl/-/psl-1.9.0.tgz", - "integrity": "sha512-E/ZsdU4HLs/68gYzgGTkMicWTLPdAftJLfJFlLUAAKZGkStNU72sZjT66SnMDVOfOWY/YAoiD7Jxa9iHvngcag==" + "license": "MIT" }, "node_modules/punycode": { "version": "2.3.1", - "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.3.1.tgz", - "integrity": "sha512-vYt7UD1U9Wg6138shLtLOvdAu+8DsC/ilFtEVHcH+wydcSpNE20AfSOduf6MkRFahL5FY7X1oU7nKVZFtfq8Fg==", + "license": "MIT", "engines": { "node": ">=6" } }, "node_modules/punycode.js": { "version": "2.3.1", - "resolved": "https://registry.npmjs.org/punycode.js/-/punycode.js-2.3.1.tgz", - "integrity": "sha512-uxFIHU0YlHYhDQtV4R9J6a52SLx28BCjT+4ieh7IGbgwVJWO+km431c4yRlREUAsAmt/uMjQUyQHNEPf0M39CA==", "dev": true, + "license": "MIT", "engines": { "node": ">=6" } }, "node_modules/pure-rand": { "version": "6.1.0", - "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.1.0.tgz", - "integrity": "sha512-bVWawvoZoBYpp6yIoQtQXHZjmz35RSVHnUOTefl8Vcjr8snTPY1wnpSPMWekcFwbxI6gtmT7rSYPFvz71ldiOA==", - "dev": true, "funding": [ { "type": "individual", @@ -8142,25 +7670,22 @@ "type": "opencollective", "url": "https://opencollective.com/fast-check" } - ] + ], + "license": "MIT" }, "node_modules/qs": { "version": "6.5.3", - "resolved": "https://registry.npmjs.org/qs/-/qs-6.5.3.tgz", - "integrity": "sha512-qxXIEh4pCGfHICj1mAJQ2/2XVZkjCDTcEgfoSQxc/fYivUZxTkk7L3bDBJSoNrEzXI17oUO5Dp07ktqE5KzczA==", + "license": "BSD-3-Clause", "engines": { "node": ">=0.6" } }, "node_modules/querystringify": { "version": "2.2.0", - "resolved": "https://registry.npmjs.org/querystringify/-/querystringify-2.2.0.tgz", - "integrity": "sha512-FIqgj2EUvTa7R50u0rGsyTftzjYmv/a3hO345bZNrqabNqjtgiDMgmo4mkUjd+nzU5oF3dClKqFIPUKybUyqoQ==" + "license": "MIT" }, "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": [ { @@ -8175,17 +7700,16 @@ "type": "consulting", "url": "https://feross.org/support" } - ] + ], + "license": "MIT" }, "node_modules/railroad-diagrams": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/railroad-diagrams/-/railroad-diagrams-1.0.0.tgz", - "integrity": "sha512-cz93DjNeLY0idrCNOH6PviZGRN9GJhsdm9hpn1YCS879fj4W+x5IFJhhkRZcwVgMmFF7R82UA/7Oh+R8lLZg6A==" + "license": "CC0-1.0" }, "node_modules/randexp": { "version": "0.4.6", - "resolved": "https://registry.npmjs.org/randexp/-/randexp-0.4.6.tgz", - "integrity": "sha512-80WNmd9DA0tmZrw9qQa62GPPWfuXJknrmVmLcxvq4uZBdYqb1wYoKTmnlGUchvVWe0XiLupYkBoXVOxz3C8DYQ==", + "license": "MIT", "dependencies": { "discontinuous-range": "1.0.0", "ret": "~0.1.10" @@ -8196,23 +7720,19 @@ }, "node_modules/randombytes": { "version": "2.1.0", - "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", - "integrity": "sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ==", "dev": true, + "license": "MIT", "dependencies": { "safe-buffer": "^5.1.0" } }, "node_modules/react-is": { "version": "18.3.1", - "resolved": "https://registry.npmjs.org/react-is/-/react-is-18.3.1.tgz", - "integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==", - "dev": true + "license": "MIT" }, "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==", + "license": "MIT", "dependencies": { "inherits": "^2.0.3", "string_decoder": "^1.1.1", @@ -8224,9 +7744,8 @@ }, "node_modules/readdirp": { "version": "3.6.0", - "resolved": "https://registry.npmjs.org/readdirp/-/readdirp-3.6.0.tgz", - "integrity": "sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==", "dev": true, + "license": "MIT", "dependencies": { "picomatch": "^2.2.1" }, @@ -8236,39 +7755,33 @@ }, "node_modules/require-directory": { "version": "2.1.1", - "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", - "integrity": "sha512-fGxEI7+wsG9xrvdjsrlmL22OMTTiHRwAMroiEeMgq8gzoLC/PQr7RsRDSTLUg/bZAZtF+TVIkHc6/4RIKrui+Q==", + "license": "MIT", "engines": { "node": ">=0.10.0" } }, "node_modules/require-from-string": { "version": "2.0.2", - "resolved": "https://registry.npmjs.org/require-from-string/-/require-from-string-2.0.2.tgz", - "integrity": "sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw==", + "license": "MIT", "engines": { "node": ">=0.10.0" } }, "node_modules/requires-port": { "version": "1.0.0", - "resolved": "https://registry.npmjs.org/requires-port/-/requires-port-1.0.0.tgz", - "integrity": "sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ==" + "license": "MIT" }, "node_modules/requizzle": { "version": "0.2.4", - "resolved": "https://registry.npmjs.org/requizzle/-/requizzle-0.2.4.tgz", - "integrity": "sha512-JRrFk1D4OQ4SqovXOgdav+K8EAhSB/LJZqCz8tbX0KObcdeM15Ss59ozWMBWmmINMagCwmqn4ZNryUGpBsl6Jw==", "dev": true, + "license": "MIT", "dependencies": { "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, + "license": "MIT", "dependencies": { "is-core-module": "^2.13.0", "path-parse": "^1.0.7", @@ -8283,9 +7796,7 @@ }, "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, + "license": "MIT", "dependencies": { "resolve-from": "^5.0.0" }, @@ -8295,52 +7806,44 @@ }, "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==", - "dev": true, + "license": "MIT", "engines": { "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, + "license": "MIT", "engines": { "node": ">=4" } }, "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, + "license": "MIT", "engines": { "node": ">=10" } }, "node_modules/ret": { "version": "0.1.15", - "resolved": "https://registry.npmjs.org/ret/-/ret-0.1.15.tgz", - "integrity": "sha512-TTlYpa+OL+vMMNG24xSlQGEJ3B/RzEfUlLct7b5G/ytav+wPrplCpVMFuwzXbkecJrb6IYo1iFb0S9v37754mg==", + "license": "MIT", "engines": { "node": ">=0.12" } }, "node_modules/retry": { "version": "0.12.0", - "resolved": "https://registry.npmjs.org/retry/-/retry-0.12.0.tgz", - "integrity": "sha512-9LkiTwjUh6rT555DtE9rTX+BKByPfrMzEAtnlEtdEwr3Nkffwiihqe2bWADg+OQRjt9gl6ICdmB/ZFDCGAtSow==", "dev": true, + "license": "MIT", "engines": { "node": ">= 4" } }, "node_modules/retry-request": { "version": "7.0.2", - "resolved": "https://registry.npmjs.org/retry-request/-/retry-request-7.0.2.tgz", - "integrity": "sha512-dUOvLMJ0/JJYEn8NrpOaGNE7X3vpI5XlZS/u0ANjqtcZVKnIxP7IgCFwrKTxENw29emmwug53awKtaMm4i9g5w==", + "license": "MIT", "dependencies": { "@types/request": "^2.48.8", "extend": "^3.0.2", @@ -8352,9 +7855,8 @@ }, "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, + "license": "MIT", "engines": { "iojs": ">=1.0.0", "node": ">=0.10.0" @@ -8362,9 +7864,7 @@ }, "node_modules/rimraf": { "version": "3.0.2", - "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-3.0.2.tgz", - "integrity": "sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA==", - "deprecated": "Rimraf versions prior to v4 are no longer supported", + "license": "ISC", "dependencies": { "glob": "^7.1.3" }, @@ -8377,8 +7877,6 @@ }, "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": [ { @@ -8394,14 +7892,13 @@ "url": "https://feross.org/support" } ], + "license": "MIT", "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", - "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", "funding": [ { "type": "github", @@ -8415,17 +7912,16 @@ "type": "consulting", "url": "https://feross.org/support" } - ] + ], + "license": "MIT" }, "node_modules/safer-buffer": { "version": "2.1.2", - "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz", - "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==" + "license": "MIT" }, "node_modules/semver": { "version": "7.6.3", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.6.3.tgz", - "integrity": "sha512-oVekP1cKtI+CTDvHWYFUcMtsK/00wmAEfyqKfNdARm8u1wNVhSgaX7A8d4UuIlUI5e84iEwOhs7ZPYRmzU9U6A==", + "license": "ISC", "bin": { "semver": "bin/semver.js" }, @@ -8435,23 +7931,19 @@ }, "node_modules/serialize-javascript": { "version": "6.0.2", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", - "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, + "license": "BSD-3-Clause", "dependencies": { "randombytes": "^2.1.0" } }, "node_modules/set-blocking": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/set-blocking/-/set-blocking-2.0.0.tgz", - "integrity": "sha512-KiKBS8AnWGEyLzofFfmvKwpdPzqiy16LvQfK3yv/fVH7Bj13/wl3JSR1J+rfgRE9q7xUJK4qvgS8raSOeLUehw==" + "license": "ISC" }, "node_modules/shebang-command": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", - "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", - "dev": true, + "license": "MIT", "dependencies": { "shebang-regex": "^3.0.0" }, @@ -8461,38 +7953,42 @@ }, "node_modules/shebang-regex": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", - "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", - "dev": true, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/signal-exit": { "version": "3.0.7", - "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-3.0.7.tgz", - "integrity": "sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ==" + "license": "ISC" + }, + "node_modules/simple-oauth2": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/simple-oauth2/-/simple-oauth2-5.1.0.tgz", + "integrity": "sha512-gWDa38Ccm4MwlG5U7AlcJxPv3lvr80dU7ARJWrGdgvOKyzSj1gr3GBPN1rABTedAYvC/LsGYoFuFxwDBPtGEbw==", + "license": "Apache-2.0", + "dependencies": { + "@hapi/hoek": "^11.0.4", + "@hapi/wreck": "^18.0.0", + "debug": "^4.3.4", + "joi": "^17.6.4" + } }, "node_modules/sisteransi": { "version": "1.0.5", - "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", - "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==", - "dev": true + "license": "MIT" }, "node_modules/slash": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/slash/-/slash-3.0.0.tgz", - "integrity": "sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q==", - "dev": true, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/smart-buffer": { "version": "4.2.0", - "resolved": "https://registry.npmjs.org/smart-buffer/-/smart-buffer-4.2.0.tgz", - "integrity": "sha512-94hK0Hh8rPqQl2xXc3HsaBoOXKV20MToPkcXvwbISWLEs+64sBq5kFgn2kJDHb1Pry9yrP0dxrCI9RRci7RXKg==", "dev": true, + "license": "MIT", "engines": { "node": ">= 6.0.0", "npm": ">= 3.0.0" @@ -8500,8 +7996,7 @@ }, "node_modules/smtp-address-parser": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/smtp-address-parser/-/smtp-address-parser-1.1.0.tgz", - "integrity": "sha512-Gz11jbNU0plrReU9Sj7fmshSBxxJ9ShdD2q4ktHIHo/rpTH6lFyQoYHYKINPJtPe8aHFnsbtW46Ls0tCCBsIZg==", + "license": "MIT", "dependencies": { "nearley": "^2.20.1" }, @@ -8511,9 +8006,8 @@ }, "node_modules/socks": { "version": "2.8.3", - "resolved": "https://registry.npmjs.org/socks/-/socks-2.8.3.tgz", - "integrity": "sha512-l5x7VUUWbjVFbafGLxPWkYsHIhEvmF85tbIeFZWc8ZPtoMyybuEhL7Jye/ooC4/d48FgOjSJXgsF/AJPYCW8Zw==", "dev": true, + "license": "MIT", "dependencies": { "ip-address": "^9.0.5", "smart-buffer": "^4.2.0" @@ -8525,9 +8019,8 @@ }, "node_modules/socks-proxy-agent": { "version": "7.0.0", - "resolved": "https://registry.npmjs.org/socks-proxy-agent/-/socks-proxy-agent-7.0.0.tgz", - "integrity": "sha512-Fgl0YPZ902wEsAyiQ+idGd1A7rSFx/ayC1CQVMw5P+EQx2V0SgpGtf6OKFhVjPflPUl9YMmEOnmfjCdMUsygww==", "dev": true, + "license": "MIT", "dependencies": { "agent-base": "^6.0.2", "debug": "^4.3.3", @@ -8537,32 +8030,16 @@ "node": ">= 10" } }, - "node_modules/socks-proxy-agent/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==", - "dev": true, - "dependencies": { - "debug": "4" - }, - "engines": { - "node": ">= 6.0.0" - } - }, "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, + "license": "BSD-3-Clause", "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, + "license": "MIT", "dependencies": { "buffer-from": "^1.0.0", "source-map": "^0.6.0" @@ -8570,14 +8047,12 @@ }, "node_modules/sprintf-js": { "version": "1.1.3", - "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.1.3.tgz", - "integrity": "sha512-Oo+0REFV59/rz3gfJNKQiBlwfHaSESl1pcGyABQsnnIfWOFt6JNj5gCog2U6MLZ//IGYD+nA8nI+mTShREReaA==", - "dev": true + "dev": true, + "license": "BSD-3-Clause" }, "node_modules/sshpk": { "version": "1.18.0", - "resolved": "https://registry.npmjs.org/sshpk/-/sshpk-1.18.0.tgz", - "integrity": "sha512-2p2KJZTSqQ/I3+HX42EpYOa2l3f8Erv8MWKsy2I9uf4wA7yFIkXRffYdsx86y6z4vHtV8u7g+pPlr8/4ouAxsQ==", + "license": "MIT", "dependencies": { "asn1": "~0.2.3", "assert-plus": "^1.0.0", @@ -8600,14 +8075,12 @@ }, "node_modules/sshpk/node_modules/jsbn": { "version": "0.1.1", - "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-0.1.1.tgz", - "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==" + "license": "MIT" }, "node_modules/ssri": { "version": "9.0.1", - "resolved": "https://registry.npmjs.org/ssri/-/ssri-9.0.1.tgz", - "integrity": "sha512-o57Wcn66jMQvfHG1FlYbWeZWW/dHZhJXjpIcTfXldXEk5nz5lStPo3mK0OJQfGR3RbZUlbISexbljkJzuEj/8Q==", "dev": true, + "license": "ISC", "dependencies": { "minipass": "^3.1.1" }, @@ -8617,9 +8090,7 @@ }, "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, + "license": "MIT", "dependencies": { "escape-string-regexp": "^2.0.0" }, @@ -8629,17 +8100,14 @@ }, "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, + "license": "MIT", "engines": { "node": ">=8" } }, "node_modules/stoppable": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/stoppable/-/stoppable-1.1.0.tgz", - "integrity": "sha512-KXDYZ9dszj6bzvnEMRYvxgeTHU74QBFL54XKtP3nyMuJ81CFYtABZ3bAzL2EdFUaEwJOBOgENyFj3R7oTzDyyw==", + "license": "MIT", "engines": { "node": ">=4", "npm": ">=6" @@ -8647,43 +8115,36 @@ }, "node_modules/stream-events": { "version": "1.0.5", - "resolved": "https://registry.npmjs.org/stream-events/-/stream-events-1.0.5.tgz", - "integrity": "sha512-E1GUzBSgvct8Jsb3v2X15pjzN1tYebtbLaMg+eBOUOAxgbLoSbT2NS91ckc5lJD1KfLjId+jXJRgo0qnV5Nerg==", + "license": "MIT", "dependencies": { "stubs": "^3.0.0" } }, "node_modules/stream-length": { "version": "1.0.2", - "resolved": "https://registry.npmjs.org/stream-length/-/stream-length-1.0.2.tgz", - "integrity": "sha512-aI+qKFiwoDV4rsXiS7WRoCt+v2RX1nUj17+KJC5r2gfh5xoSJIfP6Y3Do/HtvesFcTSWthIuJ3l1cvKQY/+nZg==", + "license": "WTFPL", "dependencies": { "bluebird": "^2.6.2" } }, "node_modules/stream-length/node_modules/bluebird": { "version": "2.11.0", - "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-2.11.0.tgz", - "integrity": "sha512-UfFSr22dmHPQqPP9XWHRhq+gWnHCYguQGkXQlbyPtW5qTnhFWA8/iXg765tH0cAjy7l/zPJ1aBTO0g5XgA7kvQ==" + "license": "MIT" }, "node_modules/stream-shift": { "version": "1.0.3", - "resolved": "https://registry.npmjs.org/stream-shift/-/stream-shift-1.0.3.tgz", - "integrity": "sha512-76ORR0DO1o1hlKwTbi/DM3EXWGf3ZJYO8cXX5RJwnul2DEg2oyoZyjLNoQM8WsvZiFKCRfC1O0J7iCvie3RZmQ==" + "license": "MIT" }, "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==", + "license": "MIT", "dependencies": { "safe-buffer": "~5.2.0" } }, "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, + "license": "MIT", "dependencies": { "char-regex": "^1.0.2", "strip-ansi": "^6.0.0" @@ -8694,8 +8155,7 @@ }, "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==", + "license": "MIT", "dependencies": { "emoji-regex": "^8.0.0", "is-fullwidth-code-point": "^3.0.0", @@ -8707,8 +8167,7 @@ }, "node_modules/strip-ansi": { "version": "6.0.1", - "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", - "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "license": "MIT", "dependencies": { "ansi-regex": "^5.0.1" }, @@ -8718,27 +8177,21 @@ }, "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, + "license": "MIT", "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, + "license": "MIT", "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", - "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==", - "dev": true, + "license": "MIT", "engines": { "node": ">=8" }, @@ -8748,19 +8201,15 @@ }, "node_modules/strnum": { "version": "1.0.5", - "resolved": "https://registry.npmjs.org/strnum/-/strnum-1.0.5.tgz", - "integrity": "sha512-J8bbNyKKXl5qYcR36TIO8W3mVGVHrmmxsd5PAItGkmyzwJvybiw2IVq5nqd0i4LSNSkB/sx9VHllbfFdr9k1JA==" + "license": "MIT" }, "node_modules/stubs": { "version": "3.0.0", - "resolved": "https://registry.npmjs.org/stubs/-/stubs-3.0.0.tgz", - "integrity": "sha512-PdHt7hHUJKxvTCgbKX9C1V/ftOcjJQgz8BZwNfV5c4B6dcGqlpelTbJ999jBGZ2jYiPAwcX5dP6oBwVlBlUbxw==" + "license": "MIT" }, "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==", - "dev": true, + "license": "MIT", "dependencies": { "has-flag": "^4.0.0" }, @@ -8770,9 +8219,7 @@ }, "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, + "license": "MIT", "engines": { "node": ">= 0.4" }, @@ -8782,8 +8229,7 @@ }, "node_modules/tar": { "version": "6.2.1", - "resolved": "https://registry.npmjs.org/tar/-/tar-6.2.1.tgz", - "integrity": "sha512-DZ4yORTwrbTj/7MZYq2w+/ZFdI6OZ/f9SFHR+71gIVUZhOQPHzVCLpvRnPgyaMpfWxxk/4ONva3GQSyNIKRv6A==", + "license": "ISC", "dependencies": { "chownr": "^2.0.0", "fs-minipass": "^2.0.0", @@ -8798,16 +8244,18 @@ }, "node_modules/tar/node_modules/minipass": { "version": "5.0.0", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-5.0.0.tgz", - "integrity": "sha512-3FnjYuehv9k6ovOEbyOswadCDPX1piCfhV8ncmYtHOjuPwylVWsghTLo7rabjC3Rx5xD4HDx8Wm1xnMF7S5qFQ==", + "license": "ISC", "engines": { "node": ">=8" } }, + "node_modules/tar/node_modules/yallist": { + "version": "4.0.0", + "license": "ISC" + }, "node_modules/teeny-request": { "version": "9.0.0", - "resolved": "https://registry.npmjs.org/teeny-request/-/teeny-request-9.0.0.tgz", - "integrity": "sha512-resvxdc6Mgb7YEThw6G6bExlXKkv6+YbuzGg9xuXxSgxJF7Ozs+o8Y9+2R3sArdWdW8nOokoQb1yrpFB0pQK2g==", + "license": "Apache-2.0", "dependencies": { "http-proxy-agent": "^5.0.0", "https-proxy-agent": "^5.0.0", @@ -8819,59 +8267,9 @@ "node": ">=14" } }, - "node_modules/teeny-request/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/teeny-request/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==", - "dependencies": { - "@tootallnate/once": "2", - "agent-base": "6", - "debug": "4" - }, - "engines": { - "node": ">= 6" - } - }, - "node_modules/teeny-request/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/teeny-request/node_modules/uuid": { - "version": "9.0.1", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-9.0.1.tgz", - "integrity": "sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA==", - "funding": [ - "https://github.com/sponsors/broofa", - "https://github.com/sponsors/ctavan" - ], - "bin": { - "uuid": "dist/bin/uuid" - } - }, "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, + "license": "ISC", "dependencies": { "@istanbuljs/schema": "^0.1.2", "glob": "^7.1.4", @@ -8883,30 +8281,23 @@ }, "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 + "dev": true, + "license": "MIT" }, "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 + "license": "BSD-3-Clause" }, "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, + "license": "MIT", "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", - "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", - "dev": true, + "license": "MIT", "dependencies": { "is-number": "^7.0.0" }, @@ -8916,22 +8307,19 @@ }, "node_modules/toad-uri-js": { "version": "5.0.1", - "resolved": "https://registry.npmjs.org/toad-uri-js/-/toad-uri-js-5.0.1.tgz", - "integrity": "sha512-r2c5hs10O0tcRvjUpgJdJf5CalaOZhY7oS9kvYBDu/rPg+02PWa1QAOb7+tvKtpmNCkW6w6F5WZt9BDWLCNHkQ==", + "license": "BSD-2-Clause-Views", "dependencies": { "punycode": "^2.3.1" } }, "node_modules/tr46": { "version": "0.0.3", - "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", - "integrity": "sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw==" + "license": "MIT" }, "node_modules/ts-api-utils": { "version": "1.3.0", - "resolved": "https://registry.npmjs.org/ts-api-utils/-/ts-api-utils-1.3.0.tgz", - "integrity": "sha512-UQMIo7pb8WRomKR1/+MFVLTroIvDVtMX3K6OUir8ynLyzB8Jeriont2bTAtmNPa1ekAgN7YPDyf6V+ygrdU+eQ==", "dev": true, + "license": "MIT", "engines": { "node": ">=16" }, @@ -8940,20 +8328,18 @@ } }, "node_modules/ts-jest": { - "version": "29.2.5", - "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.5.tgz", - "integrity": "sha512-KD8zB2aAZrcKIdGk4OwpJggeLcH1FgrICqDSROWqlnJXGCXK4Mn6FcdK2B6670Xr73lHMG1kHw8R87A0ecZ+vA==", - "dev": true, + "version": "29.2.4", + "license": "MIT", "dependencies": { - "bs-logger": "^0.2.6", + "bs-logger": "0.x", "ejs": "^3.1.10", - "fast-json-stable-stringify": "^2.1.0", + "fast-json-stable-stringify": "2.x", "jest-util": "^29.0.0", "json5": "^2.2.3", - "lodash.memoize": "^4.1.2", - "make-error": "^1.3.6", - "semver": "^7.6.3", - "yargs-parser": "^21.1.1" + "lodash.memoize": "4.x", + "make-error": "1.x", + "semver": "^7.5.3", + "yargs-parser": "^21.0.1" }, "bin": { "ts-jest": "cli.js" @@ -8989,36 +8375,39 @@ }, "node_modules/ts-jest/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, + "license": "ISC", "engines": { "node": ">=12" } }, "node_modules/tslib": { - "version": "2.7.0", - "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.7.0.tgz", - "integrity": "sha512-gLXCKdN1/j47AiHiOkJN69hJmcbGTHI0ImLmbYLHykhgeN0jVGola9yVjFgzCUklsZQMW55o+dW7IXv3RCXDzA==" + "version": "2.6.3", + "license": "0BSD" }, "node_modules/tv4": { "version": "1.3.0", - "resolved": "https://registry.npmjs.org/tv4/-/tv4-1.3.0.tgz", - "integrity": "sha512-afizzfpJgvPr+eDkREK4MxJ/+r8nEEHcmitwgnPUqpaP+FpwQyadnxNoSACbgc/b1LsZYtODGoPiFxQrgJgjvw==", + "license": [ + { + "type": "Public Domain", + "url": "http://geraintluff.github.io/tv4/LICENSE.txt" + }, + { + "type": "MIT", + "url": "http://jsonary.com/LICENSE.txt" + } + ], "engines": { "node": ">= 0.8.0" } }, "node_modules/tweetnacl": { "version": "0.14.5", - "resolved": "https://registry.npmjs.org/tweetnacl/-/tweetnacl-0.14.5.tgz", - "integrity": "sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA==" + "license": "Unlicense" }, "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, + "license": "MIT", "dependencies": { "prelude-ls": "^1.2.1" }, @@ -9028,18 +8417,15 @@ }, "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, + "license": "MIT", "engines": { "node": ">=4" } }, "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, + "license": "(MIT OR CC0-1.0)", "engines": { "node": ">=10" }, @@ -9048,10 +8434,8 @@ } }, "node_modules/typescript": { - "version": "5.6.2", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.6.2.tgz", - "integrity": "sha512-NW8ByodCSNCwZeghjN3o+JX5OFH0Ojg6sadjEKY4huZ52TqbJTJnDo5+Tw98lSy63NZvi4n+ez5m2u5d4PkZyw==", - "dev": true, + "version": "5.5.4", + "license": "Apache-2.0", "bin": { "tsc": "bin/tsc", "tsserver": "bin/tsserver" @@ -9061,14 +8445,150 @@ } }, "node_modules/typescript-eslint": { - "version": "8.5.0", - "resolved": "https://registry.npmjs.org/typescript-eslint/-/typescript-eslint-8.5.0.tgz", - "integrity": "sha512-uD+XxEoSIvqtm4KE97etm32Tn5MfaZWgWfMMREStLxR6JzvHkc2Tkj7zhTEK5XmtpTmKHNnG8Sot6qDfhHtR1Q==", + "version": "8.2.0", + "dev": true, + "license": "MIT", + "dependencies": { + "@typescript-eslint/eslint-plugin": "8.2.0", + "@typescript-eslint/parser": "8.2.0", + "@typescript-eslint/utils": "8.2.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/eslint-plugin": { + "version": "8.2.0", + "dev": true, + "license": "MIT", + "dependencies": { + "@eslint-community/regexpp": "^4.10.0", + "@typescript-eslint/scope-manager": "8.2.0", + "@typescript-eslint/type-utils": "8.2.0", + "@typescript-eslint/utils": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0", + "graphemer": "^1.4.0", + "ignore": "^5.3.1", + "natural-compare": "^1.4.0", + "ts-api-utils": "^1.3.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "@typescript-eslint/parser": "^8.0.0 || ^8.0.0-alpha.0", + "eslint": "^8.57.0 || ^9.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/parser": { + "version": "8.2.0", + "dev": true, + "license": "BSD-2-Clause", + "dependencies": { + "@typescript-eslint/scope-manager": "8.2.0", + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/typescript-estree": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0", + "debug": "^4.3.4" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/scope-manager": { + "version": "8.2.0", + "dev": true, + "license": "MIT", + "dependencies": { + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/type-utils": { + "version": "8.2.0", + "dev": true, + "license": "MIT", + "dependencies": { + "@typescript-eslint/typescript-estree": "8.2.0", + "@typescript-eslint/utils": "8.2.0", + "debug": "^4.3.4", + "ts-api-utils": "^1.3.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/types": { + "version": "8.2.0", + "dev": true, + "license": "MIT", + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/typescript-estree": { + "version": "8.2.0", "dev": true, + "license": "BSD-2-Clause", "dependencies": { - "@typescript-eslint/eslint-plugin": "8.5.0", - "@typescript-eslint/parser": "8.5.0", - "@typescript-eslint/utils": "8.5.0" + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/visitor-keys": "8.2.0", + "debug": "^4.3.4", + "globby": "^11.1.0", + "is-glob": "^4.0.3", + "minimatch": "^9.0.4", + "semver": "^7.6.0", + "ts-api-utils": "^1.3.0" }, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -9083,28 +8603,83 @@ } } }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/utils": { + "version": "8.2.0", + "dev": true, + "license": "MIT", + "dependencies": { + "@eslint-community/eslint-utils": "^4.4.0", + "@typescript-eslint/scope-manager": "8.2.0", + "@typescript-eslint/types": "8.2.0", + "@typescript-eslint/typescript-estree": "8.2.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0" + } + }, + "node_modules/typescript-eslint/node_modules/@typescript-eslint/visitor-keys": { + "version": "8.2.0", + "dev": true, + "license": "MIT", + "dependencies": { + "@typescript-eslint/types": "8.2.0", + "eslint-visitor-keys": "^3.4.3" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + } + }, + "node_modules/typescript-eslint/node_modules/brace-expansion": { + "version": "2.0.1", + "dev": true, + "license": "MIT", + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/typescript-eslint/node_modules/minimatch": { + "version": "9.0.5", + "dev": true, + "license": "ISC", + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/uc.micro": { "version": "2.1.0", - "resolved": "https://registry.npmjs.org/uc.micro/-/uc.micro-2.1.0.tgz", - "integrity": "sha512-ARDJmphmdvUk6Glw7y9DQ2bFkKBHwQHLi2lsaH6PPmz/Ka9sFOBsBluozhDltWmnv9u/cF6Rt87znRTPV+yp/A==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/underscore": { "version": "1.13.7", - "resolved": "https://registry.npmjs.org/underscore/-/underscore-1.13.7.tgz", - "integrity": "sha512-GMXzWtsc57XAtguZgaQViUOzs0KTkk8ojr3/xAxXLITqf/3EMwxC0inyETfDFjH/Krbhuep0HNbbjI9i/q3F3g==", - "dev": true + "dev": true, + "license": "MIT" }, "node_modules/undici-types": { "version": "6.19.8", - "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-6.19.8.tgz", - "integrity": "sha512-ve2KP6f/JnbPBFyobGHuerC9g1FYGn/F8n1LWTwNxCEzd6IfqTwUQcNXgEtmmQ6DlRrC1hrSrBnCZPokRrDHjw==" + "license": "MIT" }, "node_modules/unique-filename": { "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, + "license": "ISC", "dependencies": { "unique-slug": "^3.0.0" }, @@ -9114,9 +8689,8 @@ }, "node_modules/unique-slug": { "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, + "license": "ISC", "dependencies": { "imurmurhash": "^0.1.4" }, @@ -9126,17 +8700,13 @@ }, "node_modules/universalify": { "version": "0.2.0", - "resolved": "https://registry.npmjs.org/universalify/-/universalify-0.2.0.tgz", - "integrity": "sha512-CJ1QgKmNg3CwvAv/kOFmtnEN05f0D/cn9QntgNOQlQF9dgvVTHj3t+8JPdjqawCHk7V/KA+fbUqzZ9XWhcqPUg==", + "license": "MIT", "engines": { "node": ">= 4.0.0" } }, "node_modules/update-browserslist-db": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.1.0.tgz", - "integrity": "sha512-EdRAaAyk2cUE1wOf2DkEhzxqOQvFOoRJFNS6NeyJ01Gp2beMRpBAINjM2iDXE3KCuKhwnvHIQCJm6ThL2Z+HzQ==", - "dev": true, "funding": [ { "type": "opencollective", @@ -9151,6 +8721,7 @@ "url": "https://github.com/sponsors/ai" } ], + "license": "MIT", "dependencies": { "escalade": "^3.1.2", "picocolors": "^1.0.1" @@ -9164,16 +8735,14 @@ }, "node_modules/uri-js": { "version": "4.4.1", - "resolved": "https://registry.npmjs.org/uri-js/-/uri-js-4.4.1.tgz", - "integrity": "sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg==", + "license": "BSD-2-Clause", "dependencies": { "punycode": "^2.1.0" } }, "node_modules/url-parse": { "version": "1.5.10", - "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.5.10.tgz", - "integrity": "sha512-WypcfiRhfeUP9vvF0j6rw0J3hrWrw6iZv3+22h6iRMJ/8z1Tj6XfLP4DsUix5MhMPnXpiHDoKyoZ/bdCkwBCiQ==", + "license": "MIT", "dependencies": { "querystringify": "^2.1.1", "requires-port": "^1.0.0" @@ -9181,22 +8750,22 @@ }, "node_modules/util-deprecate": { "version": "1.0.2", - "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", - "integrity": "sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==" + "license": "MIT" }, "node_modules/uuid": { - "version": "8.3.2", - "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz", - "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==", + "version": "9.0.1", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "license": "MIT", "bin": { "uuid": "dist/bin/uuid" } }, "node_modules/v8-to-istanbul": { "version": "9.3.0", - "resolved": "https://registry.npmjs.org/v8-to-istanbul/-/v8-to-istanbul-9.3.0.tgz", - "integrity": "sha512-kiGUalWN+rgBJ/1OHZsBtU4rXZOfj/7rKQxULKlIzwzQSvMJUUNgPwJEEh7gU6xEVxC0ahoOBvN2YI8GH6FNgA==", - "dev": true, + "license": "ISC", "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", "@types/istanbul-lib-coverage": "^2.0.1", @@ -9208,19 +8777,17 @@ }, "node_modules/validator": { "version": "13.12.0", - "resolved": "https://registry.npmjs.org/validator/-/validator-13.12.0.tgz", - "integrity": "sha512-c1Q0mCiPlgdTVVVIJIrBuxNicYE+t/7oKeI9MWLj3fh/uq2Pxh/3eeWbVZ4OcGW1TUf53At0njHw5SMdA3tmMg==", + "license": "MIT", "engines": { "node": ">= 0.10" } }, "node_modules/verror": { "version": "1.10.0", - "resolved": "https://registry.npmjs.org/verror/-/verror-1.10.0.tgz", - "integrity": "sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw==", "engines": [ "node >=0.6.0" ], + "license": "MIT", "dependencies": { "assert-plus": "^1.0.0", "core-util-is": "1.0.2", @@ -9229,22 +8796,18 @@ }, "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, + "license": "Apache-2.0", "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", - "integrity": "sha512-2JAn3z8AR6rjK8Sm8orRC0h/bcl/DqL7tRPdGZ4I1CjdF+EaMLmYxBHyXuKL849eucPFhvBoxMsflfOb8kxaeQ==" + "license": "BSD-2-Clause" }, "node_modules/whatwg-url": { "version": "5.0.0", - "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", - "integrity": "sha512-saE57nupxk6v3HY35+jzBwYa0rKSy0XR8JSxZPwgLr7ys0IBzhGviA1/TUGJLmSVqs8pb9AnvICXEuOHLprYTw==", + "license": "MIT", "dependencies": { "tr46": "~0.0.3", "webidl-conversions": "^3.0.0" @@ -9252,9 +8815,7 @@ }, "node_modules/which": { "version": "2.0.2", - "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", - "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", - "dev": true, + "license": "ISC", "dependencies": { "isexe": "^2.0.0" }, @@ -9267,31 +8828,27 @@ }, "node_modules/wide-align": { "version": "1.1.5", - "resolved": "https://registry.npmjs.org/wide-align/-/wide-align-1.1.5.tgz", - "integrity": "sha512-eDMORYaPNZ4sQIuuYPDHdQvf4gyCF9rEEV/yPxGfwPkRodwEgiMUUXTx/dex+Me0wxx53S+NgUHaP7y3MGlDmg==", + "license": "ISC", "dependencies": { "string-width": "^1.0.2 || 2 || 3 || 4" } }, "node_modules/word-wrap": { "version": "1.2.5", - "resolved": "https://registry.npmjs.org/word-wrap/-/word-wrap-1.2.5.tgz", - "integrity": "sha512-BN22B5eaMMI9UMtjrGd5g5eCYPpCPDUy0FJXbYsaT5zYxjFOckS53SQDE3pWkVoWpHXVb3BrYcEN4Twa55B5cA==", "dev": true, + "license": "MIT", "engines": { "node": ">=0.10.0" } }, "node_modules/workerpool": { "version": "6.5.1", - "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.5.1.tgz", - "integrity": "sha512-Fs4dNYcsdpYSAfVxhnl1L5zTksjvOJxtC5hzMNl+1t9B8hTJTdKDyZ5ju7ztgPy+ft9tBFXoOlDNiOT9WUXZlA==", - "dev": true + "dev": true, + "license": "Apache-2.0" }, "node_modules/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==", + "license": "MIT", "dependencies": { "ansi-styles": "^4.0.0", "string-width": "^4.1.0", @@ -9306,14 +8863,11 @@ }, "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==" + "license": "ISC" }, "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, + "license": "ISC", "dependencies": { "imurmurhash": "^0.1.4", "signal-exit": "^3.0.7" @@ -9324,27 +8878,23 @@ }, "node_modules/xmlcreate": { "version": "2.0.4", - "resolved": "https://registry.npmjs.org/xmlcreate/-/xmlcreate-2.0.4.tgz", - "integrity": "sha512-nquOebG4sngPmGPICTS5EnxqhKbCmz5Ox5hsszI2T6U5qdrJizBc+0ilYSEjTSzU0yZcmvppztXe/5Al5fUwdg==", - "dev": true + "dev": true, + "license": "Apache-2.0" }, "node_modules/y18n": { "version": "5.0.8", - "resolved": "https://registry.npmjs.org/y18n/-/y18n-5.0.8.tgz", - "integrity": "sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA==", + "license": "ISC", "engines": { "node": ">=10" } }, "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", + "license": "ISC" }, "node_modules/yargs": { "version": "17.7.2", - "resolved": "https://registry.npmjs.org/yargs/-/yargs-17.7.2.tgz", - "integrity": "sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w==", + "license": "MIT", "dependencies": { "cliui": "^8.0.1", "escalade": "^3.1.1", @@ -9360,18 +8910,16 @@ }, "node_modules/yargs-parser": { "version": "20.2.9", - "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-20.2.9.tgz", - "integrity": "sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w==", "dev": true, + "license": "ISC", "engines": { "node": ">=10" } }, "node_modules/yargs-unparser": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/yargs-unparser/-/yargs-unparser-2.0.0.tgz", - "integrity": "sha512-7pRTIA9Qc1caZ0bZ6RYRGbHJthJWuakf+WmHK0rVeLkNrrGhfoabBNdue6kdINI6r4if7ocq9aD/n7xwKOdzOA==", "dev": true, + "license": "MIT", "dependencies": { "camelcase": "^6.0.0", "decamelize": "^4.0.0", @@ -9384,9 +8932,8 @@ }, "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, + "license": "MIT", "engines": { "node": ">=10" }, @@ -9396,17 +8943,14 @@ }, "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==", + "license": "ISC", "engines": { "node": ">=12" } }, "node_modules/yocto-queue": { "version": "0.1.0", - "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", - "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==", - "dev": true, + "license": "MIT", "engines": { "node": ">=10" }, diff --git a/package.json b/package.json index 9916f632..46ed7d80 100644 --- a/package.json +++ b/package.json @@ -47,9 +47,30 @@ "typescript-eslint": "^8.2.0" }, "dependencies": { + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", + "@bufbuild/protobuf": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", "@mapbox/node-pre-gyp": "^1.0.11", + "@smithy/types": "^3.3.0", + "@types/simple-oauth2": "^5.0.7", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", + "async-mutex": "^0.5.0", + "avsc": "^5.7.7", + "axios": "^1.7.3", "bindings": "^1.3.1", - "nan": "^2.17.0" + "json-stringify-deterministic": "^1.0.12", + "lru-cache": "^11.0.0", + "nan": "^2.17.0", + "node-vault": "^0.10.2", + "simple-oauth2": "^5.1.0", + "ts-jest": "^29.2.4", + "validator": "^13.12.0" }, "engines": { "node": ">=18.0.0" diff --git a/schemaregistry/oauth/oauth-client.ts b/schemaregistry/oauth/oauth-client.ts new file mode 100644 index 00000000..46334d0d --- /dev/null +++ b/schemaregistry/oauth/oauth-client.ts @@ -0,0 +1,56 @@ +import { ModuleOptions, ClientCredentials, ClientCredentialTokenConfig, AccessToken } from 'simple-oauth2'; + +const TOKEN_EXPIRATION_THRESHOLD_SECONDS = 30 * 60; // 30 minutes + +export class OAuthClient { + private client: ClientCredentials; + private token: AccessToken | undefined; + private tokenParams: ClientCredentialTokenConfig; + + constructor(clientId: string, clientSecret: string, tokenHost: string, tokenPath: string, scope: string) { + const clientConfig: ModuleOptions = { + client: { + id: clientId, + secret: clientSecret, + }, + auth: { + tokenHost: tokenHost, + tokenPath: tokenPath + } + } + + this.tokenParams = { scope }; + + this.client = new ClientCredentials(clientConfig); + } + + async getAccessToken(): Promise { + if (!this.token || this.token.expired(TOKEN_EXPIRATION_THRESHOLD_SECONDS)) { + await this.generateAccessToken(); + } + + return this.getAccessTokenString(); + } + + async generateAccessToken(): Promise { + try { + const token = await this.client.getToken(this.tokenParams); + this.token = token; + } catch (error) { + if (error instanceof Error) { + throw new Error(`Failed to get token from server: ${error.message}`); + } + throw new Error(`Failed to get token from server: ${error}`); + } + } + + async getAccessTokenString(): Promise { + const accessToken = this.token?.token?.['access_token']; + + if (typeof accessToken === 'string') { + return accessToken; + } + + throw new Error('Access token is not available'); + } +} diff --git a/schemaregistry/package.json b/schemaregistry/package.json index 0938e555..aea4ab00 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -39,13 +39,13 @@ "jsonata": "^2.0.5", "lru-cache": "^11.0.0", "node-vault": "^0.10.2", + "simple-oauth2": "^5.1.0", "validator": "^13.12.0" }, "scripts": { "test:types": "tsc -p .", "test:schemaregistry": "make -f Makefile.schemaregistry test" }, - "keywords": [ "schemaregistry", "confluent" @@ -54,6 +54,5 @@ "type": "git", "url": "git@github.com:confluentinc/confluent-kafka-javascript.git" }, - "license": "MIT" } diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index cf992c9e..2a2ac5d8 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -1,4 +1,5 @@ import axios, { AxiosInstance, AxiosRequestConfig, AxiosResponse, CreateAxiosDefaults } from 'axios'; +import { OAuthClient } from './oauth/oauth-client'; import { RestError } from './rest-error'; /* @@ -10,25 +11,51 @@ import { RestError } from './rest-error'; * of the MIT license. See the LICENSE.txt file for details. */ +export interface BearerAuthCredentials { + clientId: string, + clientSecret: string, + tokenHost: string, + tokenPath: string, + schemaRegistryLogicalCluster: string, + identityPool: string, + scope: string +} + +//TODO: Consider retry policy, may need additional libraries on top of Axios export interface ClientConfig { baseURLs: string[], cacheCapacity: number, cacheLatestTtlSecs?: number, - isForward?: boolean + isForward?: boolean, createAxiosDefaults?: CreateAxiosDefaults, + bearerAuthCredentials?: BearerAuthCredentials, } export class RestService { private client: AxiosInstance; private baseURLs: string[]; + private OAuthClient?: OAuthClient; + private bearerAuth: boolean = false; - constructor(baseURLs: string[], isForward?: boolean, axiosDefaults?: CreateAxiosDefaults) { + constructor(baseURLs: string[], isForward?: boolean, axiosDefaults?: CreateAxiosDefaults, + bearerAuthCredentials?: BearerAuthCredentials) { this.client = axios.create(axiosDefaults); this.baseURLs = baseURLs; if (isForward) { this.client.defaults.headers.common['X-Forward'] = 'true' } + + if (bearerAuthCredentials) { + this.bearerAuth = true; + delete this.client.defaults.auth; + this.setHeaders({ + 'Confluent-Identity-Pool-Id': bearerAuthCredentials.identityPool, + 'target-sr-cluster': bearerAuthCredentials.schemaRegistryLogicalCluster + }); + this.OAuthClient = new OAuthClient(bearerAuthCredentials.clientId, bearerAuthCredentials.clientSecret, + bearerAuthCredentials.tokenHost, bearerAuthCredentials.tokenPath, bearerAuthCredentials.scope); + } } async handleRequest( @@ -38,6 +65,10 @@ export class RestService { config?: AxiosRequestConfig, ): Promise> { + if (this.bearerAuth) { + await this.setBearerToken(); + } + for (let i = 0; i < this.baseURLs.length; i++) { try { this.setBaseURL(this.baseURLs[i]); @@ -80,6 +111,15 @@ export class RestService { } } + async setBearerToken(): Promise { + if (!this.OAuthClient) { + throw new Error('OAuthClient not initialized'); + } + + const bearerToken: string = await this.OAuthClient.getAccessToken(); + this.setAuth(undefined, bearerToken); + } + setTimeout(timeout: number): void { this.client.defaults.timeout = timeout } diff --git a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts index 00f49886..fed55558 100644 --- a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts @@ -78,7 +78,7 @@ class DekRegistryClient implements Client { } static newClient(config: ClientConfig): Client { - let url = config.baseURLs[0] + const url = config.baseURLs[0]; if (url.startsWith("mock://")) { return new MockDekRegistryClient() } diff --git a/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts index febfbb66..46f20219 100644 --- a/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts @@ -68,7 +68,7 @@ class MockDekRegistryClient implements Client { algorithm: string, version: number = 1, deleted: boolean = false): Promise { if (version === -1) { let latestVersion = 0; - for (let key of this.dekCache.keys()) { + for (const key of this.dekCache.keys()) { const parsedKey = JSON.parse(key); if (parsedKey.kekName === kekName && parsedKey.subject === subject && parsedKey.algorithm === algorithm && parsedKey.deleted === deleted) { diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index 89d64b39..393de240 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -3,7 +3,7 @@ import { AxiosResponse } from 'axios'; import stringify from "json-stringify-deterministic"; import { LRUCache } from 'lru-cache'; import { Mutex } from 'async-mutex'; -import {MockClient} from "./mock-schemaregistry-client"; +import { MockClient } from "./mock-schemaregistry-client"; /* * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry @@ -165,7 +165,8 @@ export class SchemaRegistryClient implements Client { ...(config.cacheLatestTtlSecs !== undefined && { maxAge: config.cacheLatestTtlSecs * 1000 }) }; - this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults); + this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults, + config.bearerAuthCredentials); this.schemaToIdCache = new LRUCache(cacheOptions); this.idToSchemaInfoCache = new LRUCache(cacheOptions); From 2173850d3947b96066f97020d52a44ba7ac26dd4 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Fri, 13 Sep 2024 18:44:05 -0400 Subject: [PATCH 192/224] Add Docker environment for integration tests (#34) (#81) * Add docker files * Add docker env for integ tests --- package-lock.json | 7 +--- schemaregistry/Makefile.schemaregistry | 3 +- .../docker-compose.schemaregistry.yml | 33 +++++++++++++++++++ schemaregistry/run_docker_schemaregistry.sh | 20 +++++++++++ 4 files changed, 56 insertions(+), 7 deletions(-) create mode 100644 schemaregistry/docker-compose.schemaregistry.yml create mode 100755 schemaregistry/run_docker_schemaregistry.sh diff --git a/package-lock.json b/package-lock.json index a3e3a19f..e3d494e8 100644 --- a/package-lock.json +++ b/package-lock.json @@ -3791,11 +3791,8 @@ }, "node_modules/axios": { "version": "1.7.7", -<<<<<<< HEAD -======= "resolved": "https://registry.npmjs.org/axios/-/axios-1.7.7.tgz", "integrity": "sha512-S4kL7XrjgBmvdGut0sN3yJxqYzrDOnivkBiN0OFs6hLiUam3UPvswUo0kqGyhqUZGEOytHyumEdXsAkgCOUf3Q==", ->>>>>>> 242a0f7 (Add OAuth Support to Rest Service (#25)) "license": "MIT", "dependencies": { "follow-redirects": "^1.15.6", @@ -6773,11 +6770,8 @@ }, "node_modules/micromatch": { "version": "4.0.8", -<<<<<<< HEAD -======= "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz", "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==", ->>>>>>> 242a0f7 (Add OAuth Support to Rest Service (#25)) "license": "MIT", "dependencies": { "braces": "^3.0.3", @@ -8981,6 +8975,7 @@ "jsonata": "^2.0.5", "lru-cache": "^11.0.0", "node-vault": "^0.10.2", + "simple-oauth2": "^5.1.0", "validator": "^13.12.0" }, "devDependencies": { diff --git a/schemaregistry/Makefile.schemaregistry b/schemaregistry/Makefile.schemaregistry index aebf1d1e..e6236175 100644 --- a/schemaregistry/Makefile.schemaregistry +++ b/schemaregistry/Makefile.schemaregistry @@ -5,6 +5,7 @@ NODE ?= node ESLINT ?= ../node_modules/.bin/eslint JEST ?= ../node_modules/.bin/jest TS_NODE ?= ../node_modules/.bin/ts-node +DOCKER ?= ./run_docker_schemaregistry.sh # Paths SRC_DIR = . @@ -24,4 +25,4 @@ test: $(JEST) $(SR_TEST_DIR) $(DEK_TEST_DIR) integtest: - $(JEST) $(INTEG_DIR) + $(DOCKER) diff --git a/schemaregistry/docker-compose.schemaregistry.yml b/schemaregistry/docker-compose.schemaregistry.yml new file mode 100644 index 00000000..98bcca80 --- /dev/null +++ b/schemaregistry/docker-compose.schemaregistry.yml @@ -0,0 +1,33 @@ +version: '3' +services: + zookeeper: + image: confluentinc/cp-zookeeper + environment: + ZOOKEEPER_CLIENT_PORT: 2181 + kafka: + image: confluentinc/cp-kafka + restart: always + depends_on: + - zookeeper + ports: + - 9092:9092 + environment: + KAFKA_BROKER_ID: 0 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT_INTERNAL:PLAINTEXT,PLAINTEXT_EXTERNAL:PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT_INTERNAL://kafka:9093,PLAINTEXT_EXTERNAL://localhost:9092 + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT_INTERNAL + KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 18000 + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 + schema-registry: + image: confluentinc/cp-schema-registry:7.6.0 + depends_on: + - kafka + ports: + - "8081:8081" + environment: + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka:9093 + SCHEMA_REGISTRY_HOST_NAME: schema-registry + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:8081 diff --git a/schemaregistry/run_docker_schemaregistry.sh b/schemaregistry/run_docker_schemaregistry.sh new file mode 100755 index 00000000..63793ccf --- /dev/null +++ b/schemaregistry/run_docker_schemaregistry.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +COMPOSE_VERSION=$(docker-compose --version) +DOCKER_VERSION=$(docker --version) +JEST=${JEST:-../node_modules/.bin/jest} +INTEG_DIR=../e2e/schemaregistry + +# Start the docker compose file +echo "Running docker compose up. Docker version $DOCKER_VERSION. Compose version $COMPOSE_VERSION. " + +docker-compose -f docker-compose.schemaregistry.yml up -d + +if [ "$?" == "1" ]; then + echo "Failed to start docker images." + exit 1 +fi + +echo "Running schema registry e2e tests" + +$JEST $INTEG_DIR From 31abae49f431b1da1ffc00c2ad5dd35552b4d579 Mon Sep 17 00:00:00 2001 From: Milind L Date: Sat, 14 Sep 2024 17:59:03 +0530 Subject: [PATCH 193/224] Fix log level config in light of binding logs * Also add rule for no trailing spaces and enforce it * Fix log level config in light of binding logs --- eslint.config.js | 1 + lib/kafka-consumer.js | 2 +- lib/kafkajs/_admin.js | 20 +++++++-- lib/kafkajs/_common.js | 17 ++++---- lib/kafkajs/_consumer.js | 41 +++++++++++-------- lib/kafkajs/_consumer_cache.js | 18 ++++---- lib/kafkajs/_linked-list.js | 24 +++++------ lib/kafkajs/_producer.js | 20 +++++++-- .../consumer/consumeMessages.spec.js | 4 +- test/promisified/unit/cache.spec.js | 2 +- test/promisified/unit/common.spec.js | 2 +- 11 files changed, 94 insertions(+), 57 deletions(-) diff --git a/eslint.config.js b/eslint.config.js index 0dd90f8f..0ee61d12 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -21,6 +21,7 @@ const ckjsSpecificSettings = { "no-caller": "error", "no-new": "error", "no-eq-null": "error", + "no-trailing-spaces": "error", "no-constant-condition": "off", "semi": "error" } diff --git a/lib/kafka-consumer.js b/lib/kafka-consumer.js index e6e66a94..bfc09f95 100644 --- a/lib/kafka-consumer.js +++ b/lib/kafka-consumer.js @@ -345,7 +345,7 @@ KafkaConsumer.prototype.assignments = function() { * * @note This method should only be called from within the rebalance callback * when partitions are revoked. - * + * * @return {boolean} true if assignment was lost */ diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 83a177ff..d265174c 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -116,9 +116,23 @@ class Admin { #finalizedConfig() { let compatibleConfig = this.#kafkaJSToAdminConfig(this.#userConfig.kafkaJS); - /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest - * log level, as librdkafka will control the granularity. */ - if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { + /* There can be multiple different and conflicting config directives for setting the log level: + * 1. If there's a kafkaJS block: + * a. If there's a logLevel directive in the kafkaJS block, set the logger level accordingly. + * b. If there's no logLevel directive, set the logger level to INFO. + * (both these are already handled in the conversion method above). + * 2. If there is a log_level or debug directive in the main config, set the logger level accordingly. + * !This overrides any different value provided in the kafkaJS block! + * a. If there's a log_level directive, set the logger level accordingly. + * b. If there's a debug directive, set the logger level to DEBUG regardless of anything else. This is because + * librdkafka ignores log_level if debug is set, and our behaviour should be identical. + * 3. There's nothing at all. Take no action in this case, let the logger use its default log level. + */ + if (Object.hasOwn(this.#userConfig, 'log_level')) { + this.#logger.setLogLevel(severityToLogLevel[this.#userConfig.log_level]); + } + + if (Object.hasOwn(this.#userConfig, 'debug')) { this.#logger.setLogLevel(logLevel.DEBUG); } diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 256566b9..73e774de 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -443,7 +443,6 @@ function kafkaJSToRdKafkaConfig(config) { rdkafkaConfig["log_level"] = 6 /* LOG_INFO */; break; case logLevel.DEBUG: - rdkafkaConfig["debug"] = "all" /* Turn on debug logs for everything, otherwise this log level is not useful*/; rdkafkaConfig["log_level"] = 7 /* LOG_DEBUG */; break; default: @@ -639,7 +638,7 @@ class DeferredPromise extends Promise{ * that takes the same parameter a normal Promise constructor does. * The DeferredPromise cannot be rejected to avoid unhandled rejections * entirely. - * @param {(resolve: (value: any) => void, reject: (error: Error) => void) => void} resolver + * @param {(resolve: (value: any) => void, reject: (error: Error) => void) => void} resolver */ constructor(resolver) { let resolveF; @@ -662,12 +661,12 @@ class DeferredPromise extends Promise{ /** * Utility class for time related functions */ -class Timer { +class Timer { /** * Function that resolves when the given timeout is reached * or the passed promise resolves, when it's passed, clearing the timeout * in any case. - * + * * @param {number} timeoutMs The timeout in milliseconds. * @param {Promise|undefined} promise The promise to wait for, * alternatively to the timeout, or `undefined` to just wait for the timeout. @@ -698,13 +697,13 @@ class Timer { class Lock { // Total number of readers, not increases when already holding a write lock #readers = 0; - + // Total number of writers, increased only by a single write and // its reentrant calls #writers = 0; #asyncLocalStorage = new AsyncLocalStorage(); - + // Promise to resolve and recreate when there are no readers or writers // This is used to notify all waiting writers so at least one can proceed. // It's also used to notify all waiting readers so they can can check @@ -789,7 +788,7 @@ class Lock { this.#notifyZeroReadersAndWriters(); } - /** + /** * Acquire a write (exclusive) lock while executing * the given task. * @param {function} task The task to execute. @@ -807,8 +806,8 @@ class Lock { await this.#runAsyncStack(1, withWriteLock); } - - /** + + /** * Acquire a read (shared) lock while executing * the given task. * @param {function} task The task to execute. diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 51053902..78597dd2 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -336,7 +336,7 @@ class Consumer { else this.#addPendingOperation(() => this.#unassign(userAssignment)); }; - + try { err = LibrdKafkaError.create(err); const userSpecifiedRebalanceCb = this.#userConfig['rebalance_cb']; @@ -549,13 +549,22 @@ class Consumer { /* Creates an rdkafka config based off the kafkaJS block. Switches to compatibility mode if the block exists. */ let compatibleConfig = this.#kafkaJSToConsumerConfig(this.#userConfig.kafkaJS); - /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest - * log level, as librdkafka will control the granularity. */ - if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { - this.#logger.setLogLevel(logLevel.DEBUG); + /* There can be multiple different and conflicting config directives for setting the log level: + * 1. If there's a kafkaJS block: + * a. If there's a logLevel directive in the kafkaJS block, set the logger level accordingly. + * b. If there's no logLevel directive, set the logger level to INFO. + * (both these are already handled in the conversion method above). + * 2. If there is a log_level or debug directive in the main config, set the logger level accordingly. + * !This overrides any different value provided in the kafkaJS block! + * a. If there's a log_level directive, set the logger level accordingly. + * b. If there's a debug directive, set the logger level to DEBUG regardless of anything else. This is because + * librdkafka ignores log_level if debug is set, and our behaviour should be identical. + * 3. There's nothing at all. Take no action in this case, let the logger use its default log level. + */ + if (Object.hasOwn(this.#userConfig, 'log_level')) { + this.#logger.setLogLevel(severityToLogLevel[this.#userConfig.log_level]); } - /* Even if we are in compability mode, setting a 'debug' in the main config must override the logger's level. */ if (Object.hasOwn(this.#userConfig, 'debug')) { this.#logger.setLogLevel(logLevel.DEBUG); } @@ -568,8 +577,8 @@ class Consumer { /* Certain properties that the user has set are overridden. We use trampolines to accommodate the user's callbacks. * TODO: add trampoline method for offset commit callback. */ rdKafkaConfig['offset_commit_cb'] = true; - rdKafkaConfig['rebalance_cb'] = (err, assignment) => this.#rebalanceCallback(err, assignment).catch(e => - { + rdKafkaConfig['rebalance_cb'] = (err, assignment) => this.#rebalanceCallback(err, assignment).catch(e => + { if (this.#logger) this.#logger.error(`Error from rebalance callback: ${e.stack}`); }); @@ -1307,7 +1316,7 @@ class Consumer { } async #checkMaxPollIntervalNotExceeded(now) { - const maxPollExpiration = this.#lastFetchClockNs + + const maxPollExpiration = this.#lastFetchClockNs + BigInt((this.#cacheExpirationTimeoutMs + this.#maxPollIntervalMs) * 1e6); @@ -1317,7 +1326,7 @@ class Consumer { await Timer.withTimeout(interval, this.#maxPollIntervalRestart); now = hrtime.bigint(); - + if (now > (maxPollExpiration - 1000000n)) { this.#markBatchPayloadsStale(this.assignment()); } @@ -1325,7 +1334,7 @@ class Consumer { /** * Clears the cache and resets the positions when - * the internal client hasn't been polled for more than + * the internal client hasn't been polled for more than * max poll interval since the last fetch. * After that it waits until barrier is reached or * max poll interval is reached. In the latter case it @@ -1334,11 +1343,11 @@ class Consumer { async #cacheExpirationLoop() { while (!this.#workerTerminationScheduled.resolved) { let now = hrtime.bigint(); - const cacheExpiration = this.#lastFetchClockNs + + const cacheExpiration = this.#lastFetchClockNs + BigInt(this.#cacheExpirationTimeoutMs * 1e6); if (now > cacheExpiration) { - this.#addPendingOperation(() => + this.#addPendingOperation(() => this.#clearCacheAndResetPositions()); await this.#checkMaxPollIntervalNotExceeded(now); break; @@ -1558,7 +1567,7 @@ class Consumer { // Uncomment to test an additional delay in seek // await Timer.withTimeout(1000); - + const seekedPartitions = []; const pendingSeeks = new Map(); const assignmentSet = new Set(); @@ -1608,7 +1617,7 @@ class Consumer { /* Offsets are committed on seek only when in compatibility mode. */ if (offsetsToCommit.length !== 0 && this.#internalConfig['enable.auto.commit']) { await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); - } + } } #markBatchPayloadsStale(topicPartitions) { @@ -1828,7 +1837,7 @@ class Consumer { } flattenedToppars.map(partitionKey). forEach(key => this.#pausedPartitions.delete(key)); - + this.#addPendingOperation(() => this.#resumeInternal(flattenedToppars)); } diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 0c9f43eb..38329d4f 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -61,7 +61,7 @@ class PerPartitionMessageCache { /** * MessageCache defines a dynamically sized cache for messages. - * Internally, it uses PerPartitionMessageCache to store messages for each partition. + * Internally, it uses PerPartitionMessageCache to store messages for each partition. */ class MessageCache { #size; @@ -90,7 +90,7 @@ class MessageCache { /** * Assign a new partition to the consumer, if available. - * + * * @returns {PerPartitionMessageCache} - the partition assigned to the consumer, or null if none available. */ #assignNewPartition() { @@ -105,7 +105,7 @@ class MessageCache { /** * Remove an empty partition from the cache. - * + * * @param {PerPartitionMessageCache} ppc The partition to remove from the cache. */ #removeEmptyPartition(ppc) { @@ -118,7 +118,7 @@ class MessageCache { /** * Add a single message to a PPC. * In case the PPC does not exist, it is created. - * + * * @param {Object} message - the message to add to the cache. */ #add(message) { @@ -172,7 +172,7 @@ class MessageCache { /** * Adds many messages into the cache, partitioning them as per their toppar. * Increases cache size by the number of messages added. - * + * * @param {Array} messages - the messages to add to the cache. */ addMessages(messages) { @@ -183,9 +183,9 @@ class MessageCache { /** * Allows returning the PPC without asking for another message. - * + * * @param {PerPartitionMessageCache} ppc - the partition to return. - * + * * @note this is a no-op if the PPC is not assigned. */ return(ppc) { @@ -203,7 +203,7 @@ class MessageCache { * * If the current PPC is exhausted, it moves to the next PPC. * If all PPCs are exhausted, it returns null. - * + * * @param {PerPartitionMessageCache} ppc - after a consumer has consumed a message, it must return the PPC back to us via this parameter. * otherwise, no messages from that topic partition will be consumed. * @returns {Array} - the next message in the cache, or null if none exists, and the corresponding PPC. @@ -245,7 +245,7 @@ class MessageCache { if (!nextN.length) return this.nextN(null, size); - this.#size -= nextN.length; + this.#size -= nextN.length; return [nextN, ppc]; } diff --git a/lib/kafkajs/_linked-list.js b/lib/kafkajs/_linked-list.js index 79d48171..b23d219e 100644 --- a/lib/kafkajs/_linked-list.js +++ b/lib/kafkajs/_linked-list.js @@ -62,8 +62,8 @@ class LinkedList { /** * Removes given node from the list, * if it is not already removed. - * - * @param {LinkedListNode} node + * + * @param {LinkedListNode} node */ remove(node) { if (node._removed) { @@ -89,7 +89,7 @@ class LinkedList { /** * Removes the first node from the list and returns it, * or null if the list is empty. - * + * * @returns {any} The value of the first node in the list or null. */ removeFirst() { @@ -105,7 +105,7 @@ class LinkedList { /** * Removes the last node from the list and returns its value, * or null if the list is empty. - * + * * @returns {any} The value of the last node in the list or null. */ removeLast() { @@ -120,11 +120,11 @@ class LinkedList { /** * Add a new node to the beginning of the list and returns it. - * - * @param {any} value + * + * @param {any} value * @returns {LinkedListNode} The new node. */ - addFirst(value) { + addFirst(value) { const node = new LinkedListNode(value); return this.#insertInBetween(node, null, this._head); @@ -132,11 +132,11 @@ class LinkedList { /** * Add a new node to the end of the list and returns it. - * + * * @param {any} value Node value. * @returns {LinkedListNode} The new node. */ - addLast(value) { + addLast(value) { const node = new LinkedListNode(value); return this.#insertInBetween(node, this._tail, null); } @@ -144,7 +144,7 @@ class LinkedList { /** * Add a new node before the given node and returns it. * Given node must not be removed. - * + * * @param {LinkedListNode} node Reference node. * @param {any} value New node value. * @returns {LinkedListNode} The new node. @@ -159,7 +159,7 @@ class LinkedList { /** * Add a new node after the given node and returns it. * Given node must not be removed. - * + * * @param {LinkedListNode} node Reference node. * @param {any} value New node value. * @returns {LinkedListNode} The new node. @@ -173,7 +173,7 @@ class LinkedList { /** * Concatenates the given list to the end of this list. - * + * * @param {LinkedList} list List to concatenate. */ concat(list) { diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 9c59ab3f..cdcfb872 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -186,9 +186,23 @@ class Producer { /* Creates an rdkafka config based off the kafkaJS block. Switches to compatibility mode if the block exists. */ let compatibleConfig = this.#kafkaJSToProducerConfig(this.#userConfig.kafkaJS); - /* Set the logger's level in case we're not in compatibility mode - just set it to DEBUG, the broadest - * log level, as librdkafka will control the granularity. */ - if (!compatibleConfig || Object.keys(compatibleConfig).length === 0) { + /* There can be multiple different and conflicting config directives for setting the log level: + * 1. If there's a kafkaJS block: + * a. If there's a logLevel directive in the kafkaJS block, set the logger level accordingly. + * b. If there's no logLevel directive, set the logger level to INFO. + * (both these are already handled in the conversion method above). + * 2. If there is a log_level or debug directive in the main config, set the logger level accordingly. + * !This overrides any different value provided in the kafkaJS block! + * a. If there's a log_level directive, set the logger level accordingly. + * b. If there's a debug directive, set the logger level to DEBUG regardless of anything else. This is because + * librdkafka ignores log_level if debug is set, and our behaviour should be identical. + * 3. There's nothing at all. Take no action in this case, let the logger use its default log level. + */ + if (Object.hasOwn(this.#userConfig, 'log_level')) { + this.#logger.setLogLevel(severityToLogLevel[this.#userConfig.log_level]); + } + + if (Object.hasOwn(this.#userConfig, 'debug')) { this.#logger.setLogLevel(logLevel.DEBUG); } diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index c59ae78a..422f6add 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -625,7 +625,7 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit await waitForMessages(messagesConsumed, { number: 6, delay: 100 }); expect(messagesConsumed.length).toEqual(6); - + /* Triggers revocation */ await consumer.disconnect(); @@ -678,7 +678,7 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit const batchLengths = [/* first we reach batches of 32 message and fetches of 64 * max poll interval exceeded happens on second * 32 messages batch of the 64 msg fetch. */ - 1, 1, 2, 2, 4, 4, 8, 8, 16, 16, 32, 32, 32, 32, + 1, 1, 2, 2, 4, 4, 8, 8, 16, 16, 32, 32, 32, 32, /* max poll interval exceeded, 32 reprocessed + * 1 new message. */ 1, 1, 2, 2, 4, 4, 8, 8, 3]; diff --git a/test/promisified/unit/cache.spec.js b/test/promisified/unit/cache.spec.js index 2a0a76b5..c217539b 100644 --- a/test/promisified/unit/cache.spec.js +++ b/test/promisified/unit/cache.spec.js @@ -18,7 +18,7 @@ describe('MessageCache', () => { const receivedMessages = []; let ppc = null, next = null; for (let i = 0; i < 90; i++) { - next = cache.next(ppc); + next = cache.next(ppc); expect(next).not.toBeNull(); [next, ppc] = next; expect(next).not.toBeNull(); diff --git a/test/promisified/unit/common.spec.js b/test/promisified/unit/common.spec.js index 5fbba1d6..c6d77a15 100644 --- a/test/promisified/unit/common.spec.js +++ b/test/promisified/unit/common.spec.js @@ -94,7 +94,7 @@ describe('Lock', () => { expect(events).toEqual([0, 2, 1, 3, 4, 5]); }); - + it('allows reentrant read locks', async () => { let lock = new Lock(); let sequentialPromises = new SequentialPromises(2); From 0d8400b02d80bc4a62e91113c0f2fe5629f88e4c Mon Sep 17 00:00:00 2001 From: Milind L Date: Sat, 14 Sep 2024 18:13:53 +0530 Subject: [PATCH 194/224] Remove consumerGroupId argument from sendOffsets and add tests (#82) --- MIGRATION.md | 1 + lib/kafkajs/_common.js | 2 + lib/kafkajs/_producer.js | 28 ++--- test/promisified/producer/eos.spec.js | 156 ++++++++++++++++++++++++++ types/kafkajs.d.ts | 2 +- 5 files changed, 167 insertions(+), 22 deletions(-) create mode 100644 test/promisified/producer/eos.spec.js diff --git a/MIGRATION.md b/MIGRATION.md index d1c7e0e8..3c28761b 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -199,6 +199,7 @@ producerRun().then(consumerRun).catch(console.error); ``` * A transactional producer (with a `transactionId`) set, **cannot** send messages without initiating a transaction using `producer.transaction()`. +* While using `sendOffsets` from a transactional producer, the `consumerGroupId` argument must be omitted, and rather, the consumer object itself must be passed instead. ### Consumer diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 73e774de..2d02ce5d 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -240,6 +240,8 @@ const CompatibilityErrorMessages = Object.freeze({ createReplacementErrorMessage('producer', fn, 'timeout', 'timeout: ', 'timeout: ', false), sendBatchMandatoryMissing: () => "The argument passed to sendbatch must be an object, and must contain the 'topicMessages' property: { topicMessages: {topic: string, messages: Message[]}[] } \n", + sendOffsetsMustProvideConsumer: () => + "The sendOffsets method must be called with a connected consumer instance and without a consumerGroupId parameter.\n", /* Consumer */ partitionAssignors: () => diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index cdcfb872..f2542f39 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -11,7 +11,6 @@ const { kafkaJSToRdKafkaConfig, CompatibilityErrorMessages, logLevel, } = require('./_common'); -const { Consumer } = require('./_consumer'); const error = require('./_error'); const { Buffer } = require('buffer'); @@ -371,7 +370,7 @@ class Producer { return; } this.#state = ProducerState.DISCONNECTED; - this.#logger.info("Producr disconnected", this.#createProducerBindingMessageMetadata()); + this.#logger.info("Producer disconnected", this.#createProducerBindingMessageMetadata()); resolve(); }; this.#internalClient.disconnect(5000 /* default timeout, 5000ms */, cb); @@ -465,17 +464,20 @@ 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) { + /* If the user has not supplied a consumer, or supplied a consumerGroupId, throw immediately. */ + if (consumerGroupId || !consumer) { + throw new error.KafkaJSError(CompatibilityErrorMessages.sendOffsetsMustProvideConsumer(), { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (!Array.isArray(topics) || topics.length === 0) { throw new error.KafkaJSError("sendOffsets arguments are invalid", { code: error.ErrorCodes.ERR__INVALID_ARG }); } @@ -487,27 +489,11 @@ class Producer { 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. - // This isn't exactly efficient, but we expect people to use either a consumer, - // or we will need to change the C/C++ code to facilitate using the consumerGroupId - // directly. - // TODO: Change the C/C++ code to facilitate this if we go to release with this. - - let consumerCreated = false; - if (!consumer) { - const config = Object.assign({ 'group.id': consumerGroupId }, this.rdKafkaConfig); - consumer = new Consumer(config); - consumerCreated = true; - await consumer.connect(); - } - return new Promise((resolve, reject) => { this.#internalClient.sendOffsetsToTransaction( this.#flattenTopicPartitionOffsets(topics).map(topicPartitionOffsetToRdKafka), consumer._getInternalConsumer(), async err => { - if (consumerCreated) - await consumer.disconnect(); if (err) reject(createKafkaJsErrorFromLibRdKafkaError(err)); else diff --git a/test/promisified/producer/eos.spec.js b/test/promisified/producer/eos.spec.js new file mode 100644 index 00000000..9f8f21d3 --- /dev/null +++ b/test/promisified/producer/eos.spec.js @@ -0,0 +1,156 @@ +jest.setTimeout(30000); + +const { + secureRandom, + createConsumer, + createProducer, + createTopic, + waitForMessages, +} = require('../testhelpers'); +const { ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Producer > Transactional producer', () => { + let producer, basicProducer, topicName, topicName2, transactionalId, message, consumer, groupId; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + topicName2 = `test-topic2-${secureRandom()}`; + transactionalId = `transactional-id-${secureRandom()}`; + message = { key: `key-${secureRandom()}`, value: `value-${secureRandom()}` }; + groupId = `group-id-${secureRandom()}`; + + producer = createProducer({ + idempotent: true, + transactionalId, + transactionTimeout: 1000, + }); + + basicProducer = createProducer({}); + + consumer = createConsumer({ groupId, autoCommit: false, fromBeginning: true }); + + await createTopic({ topic: topicName, partitions: 1 }); + await createTopic({ topic: topicName2 }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); + basicProducer && (await basicProducer.disconnect()); + }); + + it('fails when using consumer group id while sending offsets from transactional producer', async () => { + await producer.connect(); + await basicProducer.connect(); + await consumer.connect(); + + await basicProducer.send({ topic: topicName, messages: [message] }); + + await consumer.subscribe({ topic: topicName }); + + let messagesConsumed = []; + await consumer.run({ + eachMessage: async ({ message }) => { + const transaction = await producer.transaction(); + await transaction.send({ topic: topicName, messages: [message] }); + + await expect( + transaction.sendOffsets({ consumerGroupId: groupId })).rejects.toHaveProperty('code', ErrorCodes.ERR__INVALID_ARG); + await expect( + transaction.sendOffsets({ consumerGroupId: groupId, consumer })).rejects.toHaveProperty('code', ErrorCodes.ERR__INVALID_ARG); + + await transaction.abort(); + messagesConsumed.push(message); + } + }); + + await waitForMessages(messagesConsumed, { number: 1 }); + expect(messagesConsumed.length).toBe(1); + }); + + it('sends offsets when transaction is committed', async () => { + await producer.connect(); + await basicProducer.connect(); + await consumer.connect(); + + await basicProducer.send({ topic: topicName, messages: [message] }); + + await consumer.subscribe({ topic: topicName }); + + let messagesConsumed = []; + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + const transaction = await producer.transaction(); + await transaction.send({ topic: topicName2, messages: [message] }); + + await transaction.sendOffsets({ consumer, topics: [ + { + topic, + partitions: [ + { partition, offset: Number(message.offset) + 1 }, + ], + } + ], }); + + await transaction.commit(); + messagesConsumed.push(message); + } + }); + + await waitForMessages(messagesConsumed, { number: 1 }); + expect(messagesConsumed.length).toBe(1); + const committed = await consumer.committed(); + expect(committed).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic: topicName, + offset: '1', + partition: 0, + }), + ]) + ); + }); + + it('sends no offsets when transaction is aborted', async () => { + await producer.connect(); + await basicProducer.connect(); + await consumer.connect(); + + await basicProducer.send({ topic: topicName, messages: [message] }); + + await consumer.subscribe({ topic: topicName }); + + let messagesConsumed = []; + await consumer.run({ + eachMessage: async ({ topic, partition, message }) => { + const transaction = await producer.transaction(); + await transaction.send({ topic: topicName2, messages: [message] }); + + await transaction.sendOffsets({ consumer, topics: [ + { + topic, + partitions: [ + { partition, offset: Number(message.offset) + 1 }, + ], + } + ], }); + + await transaction.abort(); + messagesConsumed.push(message); + } + }); + + await waitForMessages(messagesConsumed, { number: 1 }); + expect(messagesConsumed.length).toBe(1); + const committed = await consumer.committed(); + expect(committed).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic: topicName, + offset: null, + partition: 0, + }), + ]) + ); + }); +}); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index bf233db1..259fe07f 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -168,7 +168,7 @@ export type Producer = Client & { transaction(): Promise commit(): Promise abort(): Promise - sendOffsets(args: { consumerGroupId?: string, consumer?: Consumer, topics: TopicOffsets[] }): Promise + sendOffsets(args: { consumer: Consumer, topics: TopicOffsets[] }): Promise isActive(): boolean } From 8ce7cfb81ccc01f5d265bc9bb1e0a1e8e2d0fdba Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 15 Sep 2024 05:49:30 +0200 Subject: [PATCH 195/224] Performance measurement improvements CKJS performance improved through fetch.queue.backoff.ms --- .../performance/performance-consolidated.js | 2 +- .../performance-primitives-kafkajs.js | 118 ++++++++--------- .../performance/performance-primitives.js | 121 +++++++++--------- 3 files changed, 115 insertions(+), 126 deletions(-) diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 2d483308..eeff3965 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -64,7 +64,7 @@ const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +proc console.log(` Message Count: ${messageCount}`); // Seed the topic with messages await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); - const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, messageCount, messageProcessTimeMs, ctpConcurrency); + const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, warmupMessages, messageCount, messageProcessTimeMs, ctpConcurrency); console.log("=== Consume-Transform-Produce Rate: ", ctpRate); } diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 76e44da9..6058d9f2 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -118,44 +118,38 @@ async function runConsumer(brokers, topic, totalMessageCnt) { await consumer.subscribe({ topic, fromBeginning: true }); let messagesReceived = 0; + let messagesMeasured = 0; let totalMessageSize = 0; let startTime; let rate; + const skippedMessages = 100; + + console.log("Starting consumer."); + consumer.run({ autoCommit: false, eachMessage: async ({ topic, partition, message }) => { messagesReceived++; - totalMessageSize += message.value.length; - if (messagesReceived === 1) { - consumer.pause([{ topic }]); - } else if (messagesReceived === 2) { - startTime = hrtime(); - } else if (messagesReceived === totalMessageCnt) { - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); - // } else if (messagesReceived % 100 == 0) { - // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); + + if (messagesReceived >= skippedMessages) { + messagesMeasured++; + totalMessageSize += message.value.length; + + if (messagesReceived === skippedMessages) { + startTime = hrtime(); + } else if (messagesMeasured === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } } } }); - // Wait until the first message is received - await new Promise((resolve) => { - let interval = setInterval(() => { - if (messagesReceived > 0) { - clearInterval(interval); - resolve(); - } - }, 100); - }); - - console.log("Starting consumer.") - totalMessageSize = 0; - consumer.resume([{ topic }]); + await new Promise((resolve) => { let interval = setInterval(() => { if (messagesReceived >= totalMessageCnt) { @@ -169,7 +163,7 @@ async function runConsumer(brokers, topic, totalMessageCnt) { return rate; } -async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { +async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { const kafka = new Kafka({ clientId: 'kafka-test-performance', brokers: brokers.split(','), @@ -185,56 +179,56 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t await consumer.subscribe({ topic: consumeTopic, fromBeginning: true }); let messagesReceived = 0; + let messagesMeasured = 0; let totalMessageSize = 0; let startTime; let rate; + const skippedMessages = warmupMessages; + + console.log("Starting consume-transform-produce."); + consumer.run({ autoCommit: false, partitionsConsumedConcurrently: ctpConcurrency, eachMessage: async ({ topic, partition, message }) => { - /* Simulate message processing for messageProcessTimeMs */ - if (messageProcessTimeMs > 0) { - await new Promise((resolve) => setTimeout(resolve, messageProcessTimeMs)); - } - await producer.send({ - topic: produceTopic, - messages: [{ value: message.value }], - }) messagesReceived++; - totalMessageSize += message.value.length; - if (messagesReceived === 1) { - consumer.pause([{ topic }]); - } else if (messagesReceived === 2) { - startTime = hrtime(); - } else if (messagesReceived === totalMessageCnt) { - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd, transformed and sent ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); - // } else if (messagesReceived % 1 == 0) { - // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); - } - } - }); - // Wait until the first message is received - await new Promise((resolve) => { - let interval = setInterval(() => { - if (messagesReceived > 0) { - clearInterval(interval); - resolve(); + if (messagesReceived >= skippedMessages) { + messagesMeasured++; + totalMessageSize += message.value.length; + + if (messagesReceived === skippedMessages) + startTime = hrtime(); + + /* Simulate message processing for messageProcessTimeMs */ + if (messageProcessTimeMs > 0) { + await new Promise((resolve) => setTimeout(resolve, messageProcessTimeMs)); + } + await producer.send({ + topic: produceTopic, + messages: [{ value: message.value }], + }) + + if (messagesMeasured === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd, transformed and sent ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } + } else { + await producer.send({ + topic: produceTopic, + messages: [{ value: message.value }], + }) } - }, 100); + } }); - console.log("Starting consume-transform-produce.") - totalMessageSize = 0; - consumer.resume([{ topic: consumeTopic }]); await new Promise((resolve) => { let interval = setInterval(() => { - if (messagesReceived >= totalMessageCnt) { + if (messagesMeasured >= totalMessageCnt) { clearInterval(interval); resolve(); } diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 58ee48e1..ed2810cf 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -120,51 +120,45 @@ async function runConsumer(brokers, topic, totalMessageCnt) { 'group.id': 'test-group' + Math.random(), 'enable.auto.commit': false, 'auto.offset.reset': 'earliest', + 'fetch.queue.backoff.ms': '100', }); await consumer.connect(); await consumer.subscribe({ topic }); let messagesReceived = 0; + let messagesMeasured = 0; let totalMessageSize = 0; let startTime; let rate; + const skippedMessages = 100; + + console.log("Starting consumer."); + consumer.run({ eachMessage: async ({ topic, partition, message }) => { messagesReceived++; - totalMessageSize += message.value.length; - if (messagesReceived === 1) { - consumer.pause([{ topic }]); - } else if (messagesReceived === 2) { - startTime = hrtime(); - } else if (messagesReceived === totalMessageCnt) { - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); - // } else if (messagesReceived % 100 == 0) { - // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); - } - } - }); + + if (messagesReceived >= skippedMessages) { + messagesMeasured++; + totalMessageSize += message.value.length; - // Wait until the first message is received - await new Promise((resolve) => { - let interval = setInterval(() => { - if (messagesReceived > 0) { - clearInterval(interval); - resolve(); + if (messagesReceived === skippedMessages) { + startTime = hrtime(); + } else if (messagesMeasured === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } } - }, 100); + } }); - console.log("Starting consumer.") - totalMessageSize = 0; - consumer.resume([{ topic }]); await new Promise((resolve) => { let interval = setInterval(() => { - if (messagesReceived >= totalMessageCnt) { + if (messagesMeasured >= totalMessageCnt) { clearInterval(interval); resolve(); } @@ -175,7 +169,8 @@ async function runConsumer(brokers, topic, totalMessageCnt) { return rate; } -async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { +async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { + console.log("here"); const kafka = new Kafka({ 'client.id': 'kafka-test-performance', 'metadata.broker.list': brokers, @@ -205,55 +200,55 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, t await consumer.subscribe({ topic: consumeTopic }); let messagesReceived = 0; + let messagesMeasured = 0; let totalMessageSize = 0; let startTime; let rate; + const skippedMessages = warmupMessages; + + console.log("Starting consume-transform-produce."); + consumer.run({ partitionsConsumedConcurrently: ctpConcurrency, eachMessage: async ({ topic, partition, message }) => { - /* Simulate message processing for messageProcessTimeMs */ - if (messageProcessTimeMs > 0) { - await new Promise((resolve) => setTimeout(resolve, messageProcessTimeMs)); - } - await producer.send({ - topic: produceTopic, - messages: [{ value: message.value }], - }) messagesReceived++; - totalMessageSize += message.value.length; - if (messagesReceived === 1) { - consumer.pause([{ topic }]); - } else if (messagesReceived === 2) { - startTime = hrtime(); - } else if (messagesReceived === totalMessageCnt) { - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd, transformed and sent ${messagesReceived} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); - // } else if (messagesReceived % 1 == 0) { - // console.log(`Recvd ${messagesReceived} messages, ${totalMessageSize} bytes`); - } - } - }); - // Wait until the first message is received - await new Promise((resolve) => { - let interval = setInterval(() => { - if (messagesReceived > 0) { - clearInterval(interval); - resolve(); + if (messagesReceived >= skippedMessages) { + messagesMeasured++; + totalMessageSize += message.value.length; + + if (messagesReceived === skippedMessages) + startTime = hrtime(); + + /* Simulate message processing for messageProcessTimeMs */ + if (messageProcessTimeMs > 0) { + await new Promise((resolve) => setTimeout(resolve, messageProcessTimeMs)); + } + await producer.send({ + topic: produceTopic, + messages: [{ value: message.value }], + }) + + if (messagesMeasured === totalMessageCnt) { + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd, transformed and sent ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } + } else { + await producer.send({ + topic: produceTopic, + messages: [{ value: message.value }], + }) } - }, 100); + } }); - console.log("Starting consume-transform-produce.") - totalMessageSize = 0; - consumer.resume([{ topic: consumeTopic }]); await new Promise((resolve) => { let interval = setInterval(() => { - if (messagesReceived >= totalMessageCnt) { + if (messagesMeasured >= totalMessageCnt) { clearInterval(interval); resolve(); } From d014984e0ad0d45d15ea562f2c7b1151ef010252 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 15 Sep 2024 17:43:18 +0200 Subject: [PATCH 196/224] Admin examples for available APIs (#84) --- examples/kafkajs/admin.js | 40 ----------- examples/kafkajs/admin/create-topics.js | 79 ++++++++++++++++++++ examples/kafkajs/admin/delete-groups.js | 64 +++++++++++++++++ examples/kafkajs/admin/delete-topics.js | 64 +++++++++++++++++ examples/kafkajs/admin/describe-groups.js | 88 +++++++++++++++++++++++ examples/kafkajs/admin/list-groups.js | 66 +++++++++++++++++ examples/kafkajs/admin/list-topics.js | 51 +++++++++++++ 7 files changed, 412 insertions(+), 40 deletions(-) delete mode 100644 examples/kafkajs/admin.js create mode 100644 examples/kafkajs/admin/create-topics.js create mode 100644 examples/kafkajs/admin/delete-groups.js create mode 100644 examples/kafkajs/admin/delete-topics.js create mode 100644 examples/kafkajs/admin/describe-groups.js create mode 100644 examples/kafkajs/admin/list-groups.js create mode 100644 examples/kafkajs/admin/list-topics.js diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js deleted file mode 100644 index 2e5d5f73..00000000 --- a/examples/kafkajs/admin.js +++ /dev/null @@ -1,40 +0,0 @@ -// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. -const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; - -async function adminStart() { - const kafka = new Kafka({ - kafkaJS: { - brokers: ['localhost:9092'], - } - }); - - 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/examples/kafkajs/admin/create-topics.js b/examples/kafkajs/admin/create-topics.js new file mode 100644 index 00000000..99f976ca --- /dev/null +++ b/examples/kafkajs/admin/create-topics.js @@ -0,0 +1,79 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; +const { parseArgs } = require('node:util'); + +async function adminStart() { + const args = parseArgs({ + options: { + 'bootstrap-servers': { + type: 'string', + short: 'b', + default: 'localhost:9092', + }, + 'topic': { + type: 'string', + short: 't', + default: 'test-topic', + }, + 'timeout': { + type: 'string', + short: 'm', + default: undefined, + }, + 'num-partitions': { + type: 'string', + short: 'p', + default: '3', + }, + 'replication-factor': { + type: 'string', + short: 'r', + default: '1', + } + }, + }); + + let { + 'bootstrap-servers': bootstrapServers, + timeout, + 'num-partitions': numPartitions, + 'replication-factor': replicationFactor, + topic, + } = args.values; + + if (timeout) { + timeout = Number(timeout) || 0; + } + + numPartitions = Number(numPartitions) || 3; + replicationFactor = Number(replicationFactor) || 1; + + const kafka = new Kafka({ + kafkaJS: { + brokers: [bootstrapServers], + } + }); + + const admin = kafka.admin(); + await admin.connect(); + + try { + await admin.createTopics({ + topics: [ + { + topic: topic, + numPartitions: numPartitions, + replicationFactor: replicationFactor, + } + ], + timeout, + }); + console.log(`Topic "${topic}" created successfully`); + } catch(err) { + console.log(`Topic creation failed`, err); + } + + await admin.disconnect(); +} + +adminStart(); diff --git a/examples/kafkajs/admin/delete-groups.js b/examples/kafkajs/admin/delete-groups.js new file mode 100644 index 00000000..bb196c09 --- /dev/null +++ b/examples/kafkajs/admin/delete-groups.js @@ -0,0 +1,64 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; +const { parseArgs } = require('node:util'); + +async function adminStart() { + const args = parseArgs({ + options: { + 'bootstrap-servers': { + type: 'string', + short: 'b', + default: 'localhost:9092', + }, + 'timeout': { + type: 'string', + short: 'm', + default: undefined, + }, + 'group-ids': { + type: 'string', + short: 'g', + multiple: true, + default: [], + }, + }, + }); + + let { + 'bootstrap-servers': bootstrapServers, + timeout, + 'group-ids': groupIds, + } = args.values; + + if (!groupIds.length) { + console.error('Group ids are required'); + process.exit(1); + } + + if (timeout) { + timeout = Number(timeout) || 0; + } + + const kafka = new Kafka({ + kafkaJS: { + brokers: [bootstrapServers], + } + }); + + const admin = kafka.admin(); + await admin.connect(); + + try { + await admin.deleteGroups( + groupIds, + { timeout }, + ); + console.log(`Groups "${groupIds.join(',')}" deleted successfully`); + } catch(err) { + console.log(`Group deletion failed`, err); + } + + await admin.disconnect(); +} + +adminStart(); diff --git a/examples/kafkajs/admin/delete-topics.js b/examples/kafkajs/admin/delete-topics.js new file mode 100644 index 00000000..ff2b77d7 --- /dev/null +++ b/examples/kafkajs/admin/delete-topics.js @@ -0,0 +1,64 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; +const { parseArgs } = require('node:util'); + +async function adminStart() { + const args = parseArgs({ + options: { + 'bootstrap-servers': { + type: 'string', + short: 'b', + default: 'localhost:9092', + }, + 'timeout': { + type: 'string', + short: 'm', + default: undefined, + }, + 'topics': { + type: 'string', + short: 't', + multiple: true, + default: [], + }, + }, + }); + + let { + 'bootstrap-servers': bootstrapServers, + timeout, + topics, + } = args.values; + + if (!topics.length) { + console.error('Topics names is required'); + process.exit(1); + } + + if (timeout) { + timeout = Number(timeout) || 0; + } + + const kafka = new Kafka({ + kafkaJS: { + brokers: [bootstrapServers], + } + }); + + const admin = kafka.admin(); + await admin.connect(); + + try { + await admin.deleteTopics({ + topics, + timeout, + }); + console.log(`Topics "${topics.join(',')}" deleted successfully`); + } catch(err) { + console.log(`Topic deletion failed`, err); + } + + await admin.disconnect(); +} + +adminStart(); diff --git a/examples/kafkajs/admin/describe-groups.js b/examples/kafkajs/admin/describe-groups.js new file mode 100644 index 00000000..b0317254 --- /dev/null +++ b/examples/kafkajs/admin/describe-groups.js @@ -0,0 +1,88 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka, ConsumerGroupStates } = require('@confluentinc/kafka-javascript').KafkaJS; +const { parseArgs } = require('node:util'); + +function printNode(node, prefix = '') { + if (!node) + return; + console.log(`${prefix}\tHost: ${node.host}`); + console.log(`${prefix}\tPort: ${node.port}`); + console.log(`${prefix}\tRack: ${node.rack}`); +} + +async function adminStart() { + const args = parseArgs({ + options: { + 'bootstrap-servers': { + type: 'string', + short: 'b', + default: 'localhost:9092', + }, + 'timeout': { + type: 'string', + short: 'm', + default: undefined, + }, + 'groups': { + type: 'string', + short: 'g', + multiple: true, + default: [], + }, + 'include-authorized-operations': { + type: 'boolean', + short: 'i', + default: false, + } + }, + }); + + let { + 'bootstrap-servers': bootstrapServers, + timeout, + groups, + 'include-authorized-operations': includeAuthorizedOperations, + } = args.values; + + if (timeout) { + timeout = Number(timeout) || 0; + } + + const kafka = new Kafka({ + kafkaJS: { + brokers: [bootstrapServers], + } + }); + + const admin = kafka.admin(); + await admin.connect(); + + try { + const groupDescriptions = await admin.describeGroups( + groups, + { + timeout, + includeAuthorizedOperations, + } + ); + for (const group of groupDescriptions.groups) { + console.log(`Group id: ${group.groupId}`); + console.log(`\tError: ${group.error}`); + console.log(`\tProtocol: ${group.protocol}`); + console.log(`\tProtocol type: ${group.protocolType}`); + console.log(`\tPartition assignor: ${group.partitionAssignor}`); + console.log(`\tState: ${group.state}`); + console.log(`\tCoordinator: ${group.coordinator ? group.coordinator.id : group.coordinator}`); + printNode(group.coordinator, '\t'); + console.log(`\tAuthorized operations: ${group.authorizedOperations}`); + console.log(`\tIs simple: ${group.isSimpleConsumerGroup}`); + console.log(`\tState: ${group.state}`); + } + } catch(err) { + console.log('Describe groups failed', err); + } + + await admin.disconnect(); +} + +adminStart(); diff --git a/examples/kafkajs/admin/list-groups.js b/examples/kafkajs/admin/list-groups.js new file mode 100644 index 00000000..3e287a4c --- /dev/null +++ b/examples/kafkajs/admin/list-groups.js @@ -0,0 +1,66 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka, ConsumerGroupStates } = require('@confluentinc/kafka-javascript').KafkaJS; +const { parseArgs } = require('node:util'); + +async function adminStart() { + const args = parseArgs({ + options: { + 'bootstrap-servers': { + type: 'string', + short: 'b', + default: 'localhost:9092', + }, + 'timeout': { + type: 'string', + short: 'm', + default: undefined, + }, + 'states': { + type: 'string', + short: 's', + multiple: true, + default: [], + } + }, + }); + + let { + 'bootstrap-servers': bootstrapServers, + states: matchConsumerGroupStates, + timeout, + } = args.values; + + if (timeout) { + timeout = Number(timeout) || 0; + } + matchConsumerGroupStates = matchConsumerGroupStates.map( + state => ConsumerGroupStates[state]); + + const kafka = new Kafka({ + kafkaJS: { + brokers: [bootstrapServers], + } + }); + + const admin = kafka.admin(); + await admin.connect(); + + try { + const groupOverview = await admin.listGroups({ + timeout, + matchConsumerGroupStates + }); + for (const group of groupOverview.groups) { + console.log(`Group id: ${group.groupId}`); + console.log(`\tType: ${group.protocolType}`); + console.log(`\tIs simple: ${group.isSimpleConsumerGroup}`); + console.log(`\tState: ${group.state}`); + } + } catch(err) { + console.log('List topics failed', err); + } + + await admin.disconnect(); +} + +adminStart(); diff --git a/examples/kafkajs/admin/list-topics.js b/examples/kafkajs/admin/list-topics.js new file mode 100644 index 00000000..fe98290a --- /dev/null +++ b/examples/kafkajs/admin/list-topics.js @@ -0,0 +1,51 @@ +// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. +const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; +const { parseArgs } = require('node:util'); + +async function adminStart() { + const args = parseArgs({ + options: { + 'bootstrap-servers': { + type: 'string', + short: 'b', + default: 'localhost:9092', + }, + 'timeout': { + type: 'string', + short: 'm', + default: undefined, + }, + }, + }); + + let { + 'bootstrap-servers': bootstrapServers, + timeout + } = args.values; + + if (timeout) { + timeout = Number(timeout) || 0; + } + + const kafka = new Kafka({ + kafkaJS: { + brokers: [bootstrapServers], + } + }); + + const admin = kafka.admin(); + await admin.connect(); + + try { + const topics = await admin.listTopics({ timeout }); + for (const topic of topics) { + console.log(`Topic name: ${topic}`); + } + } catch(err) { + console.log('List topics failed', err); + } + + await admin.disconnect(); +} + +adminStart(); From 8b6521c44aea8576011e8e9d3fb573e6d58d8172 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 16 Sep 2024 04:31:20 +0200 Subject: [PATCH 197/224] Fix listGroups segfault when passing an undefined matchConsumerGroupStates (#85) --- src/admin.cc | 7 +++++-- test/promisified/admin/list_groups.spec.js | 4 +++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/admin.cc b/src/admin.cc index 608cd756..fccae5f4 100644 --- a/src/admin.cc +++ b/src/admin.cc @@ -880,12 +880,15 @@ NAN_METHOD(AdminClient::NodeListGroups) { Nan::New("matchConsumerGroupStates").ToLocalChecked(); bool is_match_states_set = Nan::Has(config, match_consumer_group_states_key).FromMaybe(false); - v8::Local match_states_array; + v8::Local match_states_array = Nan::New(); if (is_match_states_set) { match_states_array = GetParameter>( config, "matchConsumerGroupStates", match_states_array); - match_states = Conversion::Admin::FromV8GroupStateArray(match_states_array); + if (match_states_array->Length()) { + match_states = Conversion::Admin::FromV8GroupStateArray( + match_states_array); + } } // Queue the work. diff --git a/test/promisified/admin/list_groups.spec.js b/test/promisified/admin/list_groups.spec.js index 29479eaf..11e7a271 100644 --- a/test/promisified/admin/list_groups.spec.js +++ b/test/promisified/admin/list_groups.spec.js @@ -48,7 +48,9 @@ describe('Admin > listGroups', () => { await waitFor(() => consumer.assignment().length > 0, () => null, 1000); await admin.connect(); - let listGroupsResult = await admin.listGroups(); + let listGroupsResult = await admin.listGroups({ + matchConsumerGroupStates: undefined, + }); expect(listGroupsResult.errors).toEqual([]); expect(listGroupsResult.groups).toEqual( expect.arrayContaining([ From 59e1c06fe24ce2b05e4059b7cf0596914eb2da77 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Mon, 16 Sep 2024 14:06:44 -0700 Subject: [PATCH 198/224] Add more unit tests; minor fixes for KMS clients (#86) * Minor fixes for KMS clients * Add JSON 2020-12 test --- schemaregistry/mock-schemaregistry-client.ts | 7 +- .../rules/encryption/azurekms/azure-client.ts | 2 +- .../rules/encryption/azurekms/azure-driver.ts | 2 +- .../rules/encryption/encrypt-executor.ts | 20 +- .../rules/encryption/gcpkms/gcp-client.ts | 8 +- .../encryption/hcvault/hcvault-client.ts | 10 +- .../encryption/hcvault/hcvault-driver.ts | 2 +- .../rules/encryption/kms-registry.ts | 5 + schemaregistry/serde/avro.ts | 54 +- schemaregistry/serde/json.ts | 17 +- schemaregistry/serde/serde.ts | 6 +- schemaregistry/tsconfig.json | 4 +- test/schemaregistry/serde/avro.spec.ts | 541 +++++++++++++++++- test/schemaregistry/serde/json.spec.ts | 443 +++++++++++++- test/schemaregistry/serde/protobuf.spec.ts | 33 +- 15 files changed, 1095 insertions(+), 59 deletions(-) diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts index c2d754f5..57f7fa1e 100644 --- a/schemaregistry/mock-schemaregistry-client.ts +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -201,7 +201,12 @@ class MockClient implements Client { const parsedKey = JSON.parse(key); if (parsedKey.subject === subject && (!value.softDeleted || deleted)) { if (parsedKey.schema.metadata && this.isSubset(metadata, parsedKey.schema.metadata.properties)) { - results.push({ id: parsedKey.schema.id, version: value.version, subject, schema: parsedKey.schema.schema }); + results.push({ + id: parsedKey.schema.id, + version: value.version, + subject, + ...parsedKey.schema + }); } } } diff --git a/schemaregistry/rules/encryption/azurekms/azure-client.ts b/schemaregistry/rules/encryption/azurekms/azure-client.ts index da4c3312..a0f33bb1 100644 --- a/schemaregistry/rules/encryption/azurekms/azure-client.ts +++ b/schemaregistry/rules/encryption/azurekms/azure-client.ts @@ -22,7 +22,7 @@ export class AzureKmsClient implements KmsClient { } async encrypt(plaintext: Buffer): Promise { - const result = await this.kmsClient.encrypt(AzureKmsClient.ALGORITHM, plaintext) + const result = await this.kmsClient.encrypt(AzureKmsClient.ALGORITHM, plaintext) return Buffer.from(result.result) } diff --git a/schemaregistry/rules/encryption/azurekms/azure-driver.ts b/schemaregistry/rules/encryption/azurekms/azure-driver.ts index 42d6cc84..221c38b8 100644 --- a/schemaregistry/rules/encryption/azurekms/azure-driver.ts +++ b/schemaregistry/rules/encryption/azurekms/azure-driver.ts @@ -5,7 +5,7 @@ import {AzureKmsClient} from "./azure-client"; export class AzureKmsDriver implements KmsDriver { static PREFIX = 'azure-kms://' - static TENANT_ID = 'tenant_id' + static TENANT_ID = 'tenant.id' static CLIENT_ID = 'client.id' static CLIENT_SECRET = 'client.secret' diff --git a/schemaregistry/rules/encryption/encrypt-executor.ts b/schemaregistry/rules/encryption/encrypt-executor.ts index cd5b7fb1..018db615 100644 --- a/schemaregistry/rules/encryption/encrypt-executor.ts +++ b/schemaregistry/rules/encryption/encrypt-executor.ts @@ -54,15 +54,31 @@ interface DekId { deleted: boolean } +export class Clock { + now(): number { + return Date.now() + } +} + export class FieldEncryptionExecutor extends FieldRuleExecutor { client: Client | null = null + clock: Clock static register(): FieldEncryptionExecutor { - const executor = new FieldEncryptionExecutor() + return this.registerWithClock(new Clock()) + } + + static registerWithClock(clock: Clock): FieldEncryptionExecutor { + const executor = new FieldEncryptionExecutor(clock) RuleRegistry.registerRuleExecutor(executor) return executor } + constructor(clock: Clock = new Clock()) { + super() + this.clock = clock + } + override configure(clientConfig: ClientConfig, config: Map) { this.client = DekRegistryClient.newClient(clientConfig) this.config = config @@ -416,7 +432,7 @@ export class FieldEncryptionExecutorTransform implements FieldTransform { } isExpired(ctx: RuleContext, dek: Dek | null): boolean { - const now = Date.now() + const now = this.executor.clock.now() return ctx.ruleMode !== RuleMode.READ && this.dekExpiryDays > 0 && dek != null && diff --git a/schemaregistry/rules/encryption/gcpkms/gcp-client.ts b/schemaregistry/rules/encryption/gcpkms/gcp-client.ts index 4b51f4f4..1ef561e8 100644 --- a/schemaregistry/rules/encryption/gcpkms/gcp-client.ts +++ b/schemaregistry/rules/encryption/gcpkms/gcp-client.ts @@ -12,13 +12,9 @@ export class GcpKmsClient implements KmsClient { throw new Error(`key uri must start with ${GcpKmsDriver.PREFIX}`) } this.keyId = keyUri.substring(GcpKmsDriver.PREFIX.length) - const tokens = this.keyId.split(':') - if (tokens.length < 4) { - throw new Error(`invalid key uri ${this.keyId}`) - } this.kmsClient = creds != null - ? new KeyManagementServiceClient() - : new KeyManagementServiceClient({credentials: creds}) + ? new KeyManagementServiceClient({credentials: creds}) + : new KeyManagementServiceClient() } supported(keyUri: string): boolean { diff --git a/schemaregistry/rules/encryption/hcvault/hcvault-client.ts b/schemaregistry/rules/encryption/hcvault/hcvault-client.ts index 53434b14..ea13aa09 100644 --- a/schemaregistry/rules/encryption/hcvault/hcvault-client.ts +++ b/schemaregistry/rules/encryption/hcvault/hcvault-client.ts @@ -36,12 +36,14 @@ export class HcVaultClient implements KmsClient { } async encrypt(plaintext: Buffer): Promise { - const data = await this.kmsClient.encryptData({name: this.keyName, plainText: plaintext.toString('base64') }) - return Buffer.from(data.ciphertext, 'base64') + const response = await this.kmsClient.encryptData({name: this.keyName, plaintext: plaintext.toString('base64') }) + let data = response.data.ciphertext + return Buffer.from(data, 'utf8') } async decrypt(ciphertext: Buffer): Promise { - const data = await this.kmsClient.decryptData({name: this.keyName, cipherText: ciphertext.toString('base64') }) - return Buffer.from(data.plaintext, 'base64') + const response = await this.kmsClient.decryptData({name: this.keyName, ciphertext: ciphertext.toString('utf8') }) + let data = response.data.plaintext + return Buffer.from(data, 'base64'); } } diff --git a/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts b/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts index ccdcdc90..9b8638d6 100644 --- a/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts +++ b/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts @@ -3,7 +3,7 @@ import {HcVaultClient} from "./hcvault-client"; export class HcVaultDriver implements KmsDriver { - static PREFIX = 'hcvault-kms://' + static PREFIX = 'hcvault://' static TOKEN_ID = 'token.id' static NAMESPACE = 'namespace' diff --git a/schemaregistry/rules/encryption/kms-registry.ts b/schemaregistry/rules/encryption/kms-registry.ts index e37b6f69..cef8129c 100644 --- a/schemaregistry/rules/encryption/kms-registry.ts +++ b/schemaregistry/rules/encryption/kms-registry.ts @@ -42,3 +42,8 @@ export function getKmsClient(keyUrl: string): KmsClient | null { return null } +export function clearKmsClients(): void { + kmsClients.length = 0 +} + + diff --git a/schemaregistry/serde/avro.ts b/schemaregistry/serde/avro.ts index 5444e938..ae5fe490 100644 --- a/schemaregistry/serde/avro.ts +++ b/schemaregistry/serde/avro.ts @@ -53,31 +53,7 @@ export class AvroSerializer extends Serializer implements AvroSerde { throw new Error('message is empty') } - let enumIndex = 1 - let fixedIndex = 1 - let recordIndex = 1 - - const namingHook: TypeHook = ( - avroSchema: avro.Schema, - opts: ForSchemaOptions, - ) => { - let schema = avroSchema as any - switch (schema.type) { - case 'enum': - schema.name = `Enum${enumIndex++}`; - break; - case 'fixed': - schema.name = `Fixed${fixedIndex++}`; - break; - case 'record': - schema.name = `Record${recordIndex++}`; - break; - default: - } - return undefined - } - - let avroSchema = Type.forValue(msg, { typeHook: namingHook }) + let avroSchema = AvroSerializer.messageToSchema(msg) const schema: SchemaInfo = { schemaType: 'AVRO', schema: JSON.stringify(avroSchema), @@ -104,6 +80,34 @@ export class AvroSerializer extends Serializer implements AvroSerde { return deps }) } + + static messageToSchema(msg: any): avro.Type { + let enumIndex = 1 + let fixedIndex = 1 + let recordIndex = 1 + + const namingHook: TypeHook = ( + avroSchema: avro.Schema, + opts: ForSchemaOptions, + ) => { + let schema = avroSchema as any + switch (schema.type) { + case 'enum': + schema.name = `Enum${enumIndex++}`; + break; + case 'fixed': + schema.name = `Fixed${fixedIndex++}`; + break; + case 'record': + schema.name = `Record${recordIndex++}`; + break; + default: + } + return undefined + } + + return Type.forValue(msg, { typeHook: namingHook }) + } } export type AvroDeserializerConfig = DeserializerConfig & AvroSerdeConfig diff --git a/schemaregistry/serde/json.ts b/schemaregistry/serde/json.ts index a40632d9..5189ccf7 100644 --- a/schemaregistry/serde/json.ts +++ b/schemaregistry/serde/json.ts @@ -73,7 +73,7 @@ export class JsonSerializer extends Serializer implements JsonSerde { throw new Error('message is empty') } - const jsonSchema = generateSchema(msg) + const jsonSchema = JsonSerializer.messageToSchema(msg) const schema: SchemaInfo = { schemaType: 'JSON', schema: JSON.stringify(jsonSchema), @@ -92,14 +92,14 @@ export class JsonSerializer extends Serializer implements JsonSerde { } async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { - const schema = this.toType(ctx.target) + const schema = await this.toType(ctx.target) if (typeof schema === 'boolean') { return msg } return await transform(ctx, schema, '$', msg, fieldTransform) } - toType(info: SchemaInfo): DereferencedJSONSchema { + async toType(info: SchemaInfo): Promise { return toType(this.client, this.conf as JsonDeserializerConfig, this, info, async (client, info) => { const deps = new Map() await this.resolveReferences(client, info, deps) @@ -115,6 +115,10 @@ export class JsonSerializer extends Serializer implements JsonSerde { }, ) } + + static messageToSchema(msg: any): DereferencedJSONSchema { + return generateSchema(msg) + } } export type JsonDeserializerConfig = DeserializerConfig & JsonSerdeConfig @@ -173,7 +177,7 @@ export class JsonDeserializer extends Deserializer implements JsonSerde { } async fieldTransform(ctx: RuleContext, fieldTransform: FieldTransform, msg: any): Promise { - const schema = this.toType(ctx.target) + const schema = await this.toType(ctx.target) return await transform(ctx, schema, '$', msg, fieldTransform) } @@ -211,14 +215,17 @@ async function toValidateFunction( const json = JSON.parse(info.schema) const spec = json.$schema - if (spec === 'http://json-schema.org/draft/2020-12/schema') { + if (spec === 'http://json-schema.org/draft/2020-12/schema' + || spec === 'https://json-schema.org/draft/2020-12/schema') { const ajv2020 = new Ajv2020(conf as JsonSerdeConfig) + ajv2020.addKeyword("confluent:tags") deps.forEach((schema, name) => { ajv2020.addSchema(JSON.parse(schema), name) }) fn = ajv2020.compile(json) } else { const ajv = new Ajv2019(conf as JsonSerdeConfig) + ajv.addKeyword("confluent:tags") ajv.addMetaSchema(draft6MetaSchema) ajv.addMetaSchema(draft7MetaSchema) deps.forEach((schema, name) => { diff --git a/schemaregistry/serde/serde.ts b/schemaregistry/serde/serde.ts index 18a5321a..8b2a2ca1 100644 --- a/schemaregistry/serde/serde.ts +++ b/schemaregistry/serde/serde.ts @@ -90,13 +90,13 @@ export abstract class Serde { rules = target.ruleSet?.migrationRules break case RuleMode.DOWNGRADE: - rules = source?.ruleSet?.migrationRules?.reverse() + rules = source?.ruleSet?.migrationRules?.map(x => x).reverse() break default: rules = target.ruleSet?.domainRules if (ruleMode === RuleMode.READ) { // Execute read rules in reverse order for symmetry - rules = rules?.reverse() + rules = rules?.map(x => x).reverse() } break } @@ -394,7 +394,7 @@ export abstract class Deserializer extends Serde { previous = version } if (migrationMode === RuleMode.DOWNGRADE) { - migrations = migrations.reverse() + migrations = migrations.map(x => x).reverse() } return migrations } diff --git a/schemaregistry/tsconfig.json b/schemaregistry/tsconfig.json index 1a256bdd..5379dc3f 100644 --- a/schemaregistry/tsconfig.json +++ b/schemaregistry/tsconfig.json @@ -4,12 +4,12 @@ "lib": [ "es2021", "dom" ], - "module": "preserve", + "module": "nodenext", "target": "es2021", "strict": true, "esModuleInterop": true, "forceConsistentCasingInFileNames": true, - "moduleResolution": "bundler", + "moduleResolution": "nodenext", "allowUnusedLabels": false, "allowUnreachableCode": false, "noFallthroughCasesInSwitch": true, diff --git a/test/schemaregistry/serde/avro.spec.ts b/test/schemaregistry/serde/avro.spec.ts index ec7abec9..a71f6693 100644 --- a/test/schemaregistry/serde/avro.spec.ts +++ b/test/schemaregistry/serde/avro.spec.ts @@ -1,12 +1,14 @@ import {afterEach, describe, expect, it} from '@jest/globals'; import {ClientConfig} from "../../../schemaregistry/rest-service"; import { - AvroDeserializer, AvroDeserializerConfig, + AvroDeserializer, + AvroDeserializerConfig, AvroSerializer, AvroSerializerConfig } from "../../../schemaregistry/serde/avro"; -import {SerdeType} from "../../../schemaregistry/serde/serde"; +import {SerdeType, Serializer} from "../../../schemaregistry/serde/serde"; import { + Client, Rule, RuleMode, RuleSet, @@ -14,8 +16,33 @@ import { SchemaRegistryClient } from "../../../schemaregistry/schemaregistry-client"; import {LocalKmsDriver} from "../../../schemaregistry/rules/encryption/localkms/local-driver"; -import {FieldEncryptionExecutor} from "../../../schemaregistry/rules/encryption/encrypt-executor"; +import { + Clock, + FieldEncryptionExecutor +} from "../../../schemaregistry/rules/encryption/encrypt-executor"; +import {GcpKmsDriver} from "../../../schemaregistry/rules/encryption/gcpkms/gcp-driver"; +import {AwsKmsDriver} from "../../../schemaregistry/rules/encryption/awskms/aws-driver"; +import {AzureKmsDriver} from "../../../schemaregistry/rules/encryption/azurekms/azure-driver"; +import {HcVaultDriver} from "../../../schemaregistry/rules/encryption/hcvault/hcvault-driver"; +import {JsonataExecutor} from "@confluentinc/schemaregistry/rules/jsonata/jsonata-executor"; +import stringify from "json-stringify-deterministic"; +import {RuleRegistry} from "@confluentinc/schemaregistry/serde/rule-registry"; +import { + clearKmsClients +} from "@confluentinc/schemaregistry/rules/encryption/kms-registry"; +const rootSchema = ` +{ + "name": "NestedTestRecord", + "type": "record", + "fields": [ + { + "name": "otherField", + "type": "DemoSchema" + } + ] +} +` const demoSchema = ` { "name": "DemoSchema", @@ -46,6 +73,35 @@ const demoSchema = ` ] } ` +const demoSchemaSingleTag = ` +{ + "name": "DemoSchema", + "type": "record", + "fields": [ + { + "name": "intField", + "type": "int" + }, + { + "name": "doubleField", + "type": "double" + }, + { + "name": "stringField", + "type": "string", + "confluent:tags": [ "PII" ] + }, + { + "name": "boolField", + "type": "boolean" + }, + { + "name": "bytesField", + "type": "bytes" + } + ] +} +` const demoSchemaWithLogicalType = ` { "name": "DemoSchema", @@ -104,8 +160,51 @@ const f1Schema = ` ] } ` +const demoSchemaWithUnion = ` +{ + "name": "DemoSchemaWithUnion", + "type": "record", + "fields": [ + { + "name": "intField", + "type": "int" + }, + { + "name": "doubleField", + "type": "double" + }, + { + "name": "stringField", + "type": ["null", "string"], + "confluent:tags": [ "PII" ] + }, + { + "name": "boolField", + "type": "boolean" + }, + { + "name": "bytesField", + "type": ["null", "bytes"], + "confluent:tags": [ "PII" ] + } + ] +} +` + +class FakeClock extends Clock { + fixedNow: number = 0 + + override now() { + return this.fixedNow + } +} -const fieldEncryptionExecutor = FieldEncryptionExecutor.register() +const fieldEncryptionExecutor = FieldEncryptionExecutor.registerWithClock(new FakeClock()) +JsonataExecutor.register() +AwsKmsDriver.register() +AzureKmsDriver.register() +GcpKmsDriver.register() +HcVaultDriver.register() LocalKmsDriver.register() //const baseURL = 'http://localhost:8081' @@ -247,7 +346,7 @@ describe('AvroSerializer', () => { 'encrypt.kms.type': 'local-kms', 'encrypt.kms.key.id': 'mykey', }, - onFailure: 'ERROR,ERROR' + onFailure: 'ERROR,NONE' } let ruleSet: RuleSet = { domainRules: [encRule] @@ -287,6 +386,14 @@ describe('AvroSerializer', () => { expect(obj2.stringField).toEqual(obj.stringField); expect(obj2.boolField).toEqual(obj.boolField); expect(obj2.bytesField).toEqual(obj.bytesField); + + clearKmsClients() + let registry = new RuleRegistry() + registry.registerExecutor(new FieldEncryptionExecutor()) + deser = new AvroDeserializer(client, SerdeType.VALUE, {}, registry) + obj2 = await deser.deserialize(topic, bytes) + expect(obj2.stringField).not.toEqual(obj.stringField); + expect(obj2.bytesField).not.toEqual(obj.bytesField); }) it('basic encryption with logical type', async () => { let conf: ClientConfig = { @@ -355,6 +462,113 @@ describe('AvroSerializer', () => { expect(obj2.boolField).toEqual(obj.boolField); expect(obj2.bytesField).toEqual(obj.bytesField); }) + it('basic encryption with dek rotation', async () => { + (fieldEncryptionExecutor.clock as FakeClock).fixedNow = Date.now() + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + 'encrypt.dek.expiry.days': '1', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: demoSchemaSingleTag, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + + let dek = await dekClient.getDek("kek1", subject, 'AES256_GCM', -1, false) + expect(1).toEqual(dek.version); + + // advance time by 2 days + (fieldEncryptionExecutor.clock as FakeClock).fixedNow += 2 * 24 * 60 * 60 * 1000 + + bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + + obj2 = await deser.deserialize(topic, bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + + dek = await dekClient.getDek("kek1", subject, 'AES256_GCM', -1, false) + expect(2).toEqual(dek.version); + + // advance time by 2 days + (fieldEncryptionExecutor.clock as FakeClock).fixedNow += 2 * 24 * 60 * 60 * 1000 + + bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + + obj2 = await deser.deserialize(topic, bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + + dek = await dekClient.getDek("kek1", subject, 'AES256_GCM', -1, false) + expect(3).toEqual(dek.version); + }) it('basic encryption with preserialized data', async () => { let conf: ClientConfig = { baseURLs: [baseURL], @@ -513,4 +727,321 @@ describe('AvroSerializer', () => { let obj2 = await deser.deserialize(topic, bytes) expect(obj2.f1).toEqual(obj.f1); }) + it('encryption with references', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: demoSchema, + } + + await client.register('demo-value', info, false) + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + info = { + schemaType: 'AVRO', + schema: rootSchema, + references: [{ + name: 'DemoSchema', + subject: 'demo-value', + version: 1 + }], + ruleSet + } + + await client.register(subject, info, false) + + let nested = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), + } + let obj = { + otherField: nested + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + nested.stringField = 'hi' + nested.bytesField = Buffer.from([1, 2]) + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.otherField.intField).toEqual(nested.intField); + expect(obj2.otherField.doubleField).toBeCloseTo(nested.doubleField, 0.001); + expect(obj2.otherField.stringField).toEqual(nested.stringField); + expect(obj2.otherField.boolField).toEqual(nested.boolField); + expect(obj2.otherField.bytesField).toEqual(nested.bytesField); + }) + it('encryption with union', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info = { + schemaType: 'AVRO', + schema: demoSchemaWithUnion, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([1, 2]), + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + obj.bytesField = Buffer.from([1, 2]) + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.intField).toEqual(obj.intField); + expect(obj2.doubleField).toBeCloseTo(obj.doubleField, 0.001); + expect(obj2.stringField).toEqual(obj.stringField); + expect(obj2.boolField).toEqual(obj.boolField); + expect(obj2.bytesField).toEqual(obj.bytesField); + }) + it('jsonata fully compatible', async () => { + let rule1To2 = "$merge([$sift($, function($v, $k) {$k != 'size'}), {'height': $.'size'}])" + let rule2To1 = "$merge([$sift($, function($v, $k) {$k != 'height'}), {'size': $.'height'}])" + let rule2To3 = "$merge([$sift($, function($v, $k) {$k != 'height'}), {'length': $.'height'}])" + let rule3To2 = "$merge([$sift($, function($v, $k) {$k != 'length'}), {'height': $.'length'}])" + + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + + client.updateConfig(subject, { + compatibilityGroup: 'application.version' + }) + + let widget = { + name: 'alice', + size: 123, + version: 1, + } + let avroSchema = AvroSerializer.messageToSchema(widget) + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: JSON.stringify(avroSchema), + metadata: { + properties: { + "application.version": "v1" + } + } + } + + await client.register(subject, info, false) + + let newWidget = { + name: 'alice', + height: 123, + version: 1, + } + avroSchema = AvroSerializer.messageToSchema(newWidget) + info = { + schemaType: 'AVRO', + schema: JSON.stringify(avroSchema), + metadata: { + properties: { + "application.version": "v2" + } + }, + ruleSet: { + migrationRules: [ + { + name: 'myRule1', + kind: 'TRANSFORM', + mode: RuleMode.UPGRADE, + type: 'JSONATA', + expr: rule1To2, + }, + { + name: 'myRule2', + kind: 'TRANSFORM', + mode: RuleMode.DOWNGRADE, + type: 'JSONATA', + expr: rule2To1, + }, + ] + } + } + + await client.register(subject, info, false) + + let newerWidget = { + name: 'alice', + length: 123, + version: 1, + } + avroSchema = AvroSerializer.messageToSchema(newerWidget) + info = { + schemaType: 'AVRO', + schema: JSON.stringify(avroSchema), + metadata: { + properties: { + "application.version": "v3" + } + }, + ruleSet: { + migrationRules: [ + { + name: 'myRule1', + kind: 'TRANSFORM', + mode: RuleMode.UPGRADE, + type: 'JSONATA', + expr: rule2To3, + }, + { + name: 'myRule2', + kind: 'TRANSFORM', + mode: RuleMode.DOWNGRADE, + type: 'JSONATA', + expr: rule3To2, + }, + ] + } + } + + await client.register(subject, info, false) + + let serConfig1 = { + useLatestWithMetadata: { + "application.version": "v1" + } + } + let ser1 = new AvroSerializer(client, SerdeType.VALUE, serConfig1) + let bytes = await ser1.serialize(topic, widget) + + await deserializeWithAllVersions(client, ser1, bytes, widget, newWidget, newerWidget) + + let serConfig2 = { + useLatestWithMetadata: { + "application.version": "v2" + } + } + let ser2 = new AvroSerializer(client, SerdeType.VALUE, serConfig2) + bytes = await ser2.serialize(topic, newWidget) + + await deserializeWithAllVersions(client, ser2, bytes, widget, newWidget, newerWidget) + + let serConfig3 = { + useLatestWithMetadata: { + "application.version": "v3" + } + } + let ser3 = new AvroSerializer(client, SerdeType.VALUE, serConfig3) + bytes = await ser3.serialize(topic, newerWidget) + + await deserializeWithAllVersions(client, ser3, bytes, widget, newWidget, newerWidget) + }) + + async function deserializeWithAllVersions(client: Client, ser: Serializer, bytes: Buffer, + widget: any, newWidget: any, newerWidget: any) { + let deserConfig1: AvroDeserializerConfig = { + useLatestWithMetadata: { + "application.version": "v1" + } + } + let deser1 = new AvroDeserializer(client, SerdeType.VALUE, deserConfig1) + deser1.client = ser.client + + let newobj = await deser1.deserialize(topic, bytes) + expect(stringify(newobj)).toEqual(stringify(widget)); + + let deserConfig2 = { + useLatestWithMetadata: { + "application.version": "v2" + } + } + let deser2 = new AvroDeserializer(client, SerdeType.VALUE, deserConfig2) + newobj = await deser2.deserialize(topic, bytes) + expect(stringify(newobj)).toEqual(stringify(newWidget)); + + let deserConfig3 = { + useLatestWithMetadata: { + "application.version": "v3" + } + } + let deser3 = new AvroDeserializer(client, SerdeType.VALUE, deserConfig3) + newobj = await deser3.deserialize(topic, bytes) + expect(stringify(newobj)).toEqual(stringify(newerWidget)); + } }) diff --git a/test/schemaregistry/serde/json.spec.ts b/test/schemaregistry/serde/json.spec.ts index c28f6f9d..69a4ae28 100644 --- a/test/schemaregistry/serde/json.spec.ts +++ b/test/schemaregistry/serde/json.spec.ts @@ -1,7 +1,8 @@ import {afterEach, describe, expect, it} from '@jest/globals'; import {ClientConfig} from "../../../schemaregistry/rest-service"; -import {SerdeType} from "../../../schemaregistry/serde/serde"; +import {SerdeType, SerializationError, Serializer} from "../../../schemaregistry/serde/serde"; import { + Client, Rule, RuleMode, RuleSet, @@ -15,8 +16,13 @@ import { JsonSerializer, JsonSerializerConfig } from "../../../schemaregistry/serde/json"; +import {RuleRegistry} from "@confluentinc/schemaregistry/serde/rule-registry"; +import stringify from "json-stringify-deterministic"; +import {JsonataExecutor} from "@confluentinc/schemaregistry/rules/jsonata/jsonata-executor"; +import {clearKmsClients} from "@confluentinc/schemaregistry/rules/encryption/kms-registry"; const fieldEncryptionExecutor = FieldEncryptionExecutor.register() +JsonataExecutor.register() LocalKmsDriver.register() //const baseURL = 'http://localhost:8081' @@ -53,6 +59,52 @@ const demoSchema = ` } } ` +const demoSchemaWithUnion = ` +{ + "type": "object", + "properties": { + "intField": { "type": "integer" }, + "doubleField": { "type": "number" }, + "stringField": { + "oneOf": [ + { + "type": "null" + }, + { + "type": "string" + } + ], + "confluent:tags": [ "PII" ] + }, + "boolField": { "type": "boolean" }, + "bytesField": { + "type": "string", + "contentEncoding": "base64", + "confluent:tags": [ "PII" ] + } + } +} +` +const demoSchema2020_12 = ` +{ + "$schema": "https://json-schema.org/draft/2020-12/schema", + "type": "object", + "properties": { + "intField": { "type": "integer" }, + "doubleField": { "type": "number" }, + "stringField": { + "type": "string", + "confluent:tags": [ "PII" ] + }, + "boolField": { "type": "boolean" }, + "bytesField": { + "type": "string", + "contentEncoding": "base64", + "confluent:tags": [ "PII" ] + } + } +} +` describe('JsonSerializer', () => { afterEach(async () => { @@ -143,6 +195,79 @@ describe('JsonSerializer', () => { let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) }) + it('basic failing validation', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + useLatestVersion: true, + validate: true + }) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let jsonSchema = JsonSerializer.messageToSchema(obj) + let info: SchemaInfo = { + schemaType: 'JSON', + schema: JSON.stringify(jsonSchema) + } + + await client.register(subject, info, false) + + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + + let diffObj = { + intField: '123', + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + + await expect(() => ser.serialize(topic, diffObj)).rejects.toThrow(SerializationError) + }) + it('basic serialization 2020-12', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + useLatestVersion: true, + validate: true + }) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchema2020_12 + } + + await client.register(subject, info, false) + + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) it('basic encryption', async () => { let conf: ClientConfig = { baseURLs: [baseURL], @@ -169,7 +294,7 @@ describe('JsonSerializer', () => { 'encrypt.kms.type': 'local-kms', 'encrypt.kms.key.id': 'mykey', }, - onFailure: 'ERROR,ERROR' + onFailure: 'ERROR,NONE' } let ruleSet: RuleSet = { domainRules: [encRule] @@ -205,5 +330,319 @@ describe('JsonSerializer', () => { fieldEncryptionExecutor.client = dekClient let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) + + clearKmsClients() + let registry = new RuleRegistry() + registry.registerExecutor(new FieldEncryptionExecutor()) + deser = new JsonDeserializer(client, SerdeType.VALUE, {}, registry) + obj2 = await deser.deserialize(topic, bytes) + expect(obj2).not.toEqual(obj); }) + it('encryption with union', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: JsonSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new JsonSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchemaWithUnion, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + obj.stringField = 'hi' + obj.bytesField = Buffer.from([0, 0, 0, 1]).toString('base64') + + let deserConfig: JsonDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new JsonDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('encryption with reference', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: JsonSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new JsonSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchema, + } + await client.register('demo-value', info, false) + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,ERROR' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + info = { + schemaType: 'JSON', + schema: rootSchema, + references: [{ + name: 'DemoSchema', + subject: 'demo-value', + version: 1 + }], + ruleSet + } + await client.register(subject, info, false) + + let nested = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let obj = { + otherField: nested + } + let bytes = await ser.serialize(topic, obj) + + // reset encrypted field + nested.stringField = 'hi' + nested.bytesField = Buffer.from([0, 0, 0, 1]).toString('base64') + + let deserConfig: JsonDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new JsonDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) + it('jsonata fully compatible', async () => { + let rule1To2 = "$merge([$sift($, function($v, $k) {$k != 'size'}), {'height': $.'size'}])" + let rule2To1 = "$merge([$sift($, function($v, $k) {$k != 'height'}), {'size': $.'height'}])" + let rule2To3 = "$merge([$sift($, function($v, $k) {$k != 'height'}), {'length': $.'height'}])" + let rule3To2 = "$merge([$sift($, function($v, $k) {$k != 'length'}), {'height': $.'length'}])" + + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + + client.updateConfig(subject, { + compatibilityGroup: 'application.version' + }) + + let widget = { + name: 'alice', + size: 123, + version: 1, + } + let jsonSchema = JsonSerializer.messageToSchema(widget) + let info: SchemaInfo = { + schemaType: 'JSON', + schema: JSON.stringify(jsonSchema), + metadata: { + properties: { + "application.version": "v1" + } + } + } + + await client.register(subject, info, false) + + let newWidget = { + name: 'alice', + height: 123, + version: 1, + } + jsonSchema = JsonSerializer.messageToSchema(newWidget) + info = { + schemaType: 'JSON', + schema: JSON.stringify(jsonSchema), + metadata: { + properties: { + "application.version": "v2" + } + }, + ruleSet: { + migrationRules: [ + { + name: 'myRule1', + kind: 'TRANSFORM', + mode: RuleMode.UPGRADE, + type: 'JSONATA', + expr: rule1To2, + }, + { + name: 'myRule2', + kind: 'TRANSFORM', + mode: RuleMode.DOWNGRADE, + type: 'JSONATA', + expr: rule2To1, + }, + ] + } + } + + await client.register(subject, info, false) + + let newerWidget = { + name: 'alice', + length: 123, + version: 1, + } + jsonSchema = JsonSerializer.messageToSchema(newerWidget) + info = { + schemaType: 'JSON', + schema: JSON.stringify(jsonSchema), + metadata: { + properties: { + "application.version": "v3" + } + }, + ruleSet: { + migrationRules: [ + { + name: 'myRule1', + kind: 'TRANSFORM', + mode: RuleMode.UPGRADE, + type: 'JSONATA', + expr: rule2To3, + }, + { + name: 'myRule2', + kind: 'TRANSFORM', + mode: RuleMode.DOWNGRADE, + type: 'JSONATA', + expr: rule3To2, + }, + ] + } + } + + await client.register(subject, info, false) + + let serConfig1 = { + useLatestWithMetadata: { + "application.version": "v1" + } + } + let ser1 = new JsonSerializer(client, SerdeType.VALUE, serConfig1) + let bytes = await ser1.serialize(topic, widget) + + await deserializeWithAllVersions(client, ser1, bytes, widget, newWidget, newerWidget) + + let serConfig2 = { + useLatestWithMetadata: { + "application.version": "v2" + } + } + let ser2 = new JsonSerializer(client, SerdeType.VALUE, serConfig2) + bytes = await ser2.serialize(topic, newWidget) + + await deserializeWithAllVersions(client, ser2, bytes, widget, newWidget, newerWidget) + + let serConfig3 = { + useLatestWithMetadata: { + "application.version": "v3" + } + } + let ser3 = new JsonSerializer(client, SerdeType.VALUE, serConfig3) + bytes = await ser3.serialize(topic, newerWidget) + + await deserializeWithAllVersions(client, ser3, bytes, widget, newWidget, newerWidget) + }) + + async function deserializeWithAllVersions(client: Client, ser: Serializer, bytes: Buffer, + widget: any, newWidget: any, newerWidget: any) { + let deserConfig1: JsonDeserializerConfig = { + useLatestWithMetadata: { + "application.version": "v1" + } + } + let deser1 = new JsonDeserializer(client, SerdeType.VALUE, deserConfig1) + deser1.client = ser.client + + let newobj = await deser1.deserialize(topic, bytes) + expect(stringify(newobj)).toEqual(stringify(widget)); + + let deserConfig2 = { + useLatestWithMetadata: { + "application.version": "v2" + } + } + let deser2 = new JsonDeserializer(client, SerdeType.VALUE, deserConfig2) + newobj = await deser2.deserialize(topic, bytes) + expect(stringify(newobj)).toEqual(stringify(newWidget)); + + let deserConfig3 = { + useLatestWithMetadata: { + "application.version": "v3" + } + } + let deser3 = new JsonDeserializer(client, SerdeType.VALUE, deserConfig3) + newobj = await deser3.deserialize(topic, bytes) + expect(stringify(newobj)).toEqual(stringify(newerWidget)); + } }) diff --git a/test/schemaregistry/serde/protobuf.spec.ts b/test/schemaregistry/serde/protobuf.spec.ts index de238e35..4d7e1759 100644 --- a/test/schemaregistry/serde/protobuf.spec.ts +++ b/test/schemaregistry/serde/protobuf.spec.ts @@ -22,6 +22,9 @@ import { } from "./test/nested_pb"; import {TestMessageSchema} from "./test/test_pb"; import {DependencyMessageSchema} from "./test/dep_pb"; +import {RuleRegistry} from "@confluentinc/schemaregistry/serde/rule-registry"; +import {LinkedListSchema} from "./test/cycle_pb"; +import {clearKmsClients} from "@confluentinc/schemaregistry/rules/encryption/kms-registry"; const fieldEncryptionExecutor = FieldEncryptionExecutor.register() LocalKmsDriver.register() @@ -139,6 +142,27 @@ describe('ProtobufSerializer', () => { expect(obj2.testMesssage.testFixed32).toEqual(msg.testFixed32); expect(obj2.testMesssage.testFixed64).toEqual(msg.testFixed64); }) + it('serialize cycle', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new ProtobufSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + ser.registry.add(LinkedListSchema) + let inner = create(LinkedListSchema, { + value: 100, + }) + let obj = create(LinkedListSchema, { + value: 1, + next: inner + }) + let bytes = await ser.serialize(topic, obj) + + let deser = new ProtobufDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) it('basic encryption', async () => { let conf: ClientConfig = { baseURLs: [baseURL], @@ -166,7 +190,7 @@ describe('ProtobufSerializer', () => { 'encrypt.kms.type': 'local-kms', 'encrypt.kms.key.id': 'mykey', }, - onFailure: 'ERROR,ERROR' + onFailure: 'ERROR,NONE' } let ruleSet: RuleSet = { domainRules: [encRule] @@ -201,5 +225,12 @@ describe('ProtobufSerializer', () => { fieldEncryptionExecutor.client = dekClient let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) + + clearKmsClients() + let registry = new RuleRegistry() + registry.registerExecutor(new FieldEncryptionExecutor()) + deser = new ProtobufDeserializer(client, SerdeType.VALUE, {}, registry) + obj2 = await deser.deserialize(topic, bytes) + expect(obj2).not.toEqual(obj); }) }) From 7350d0a9f4c41862a420554ae92240b32a7e2899 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 17 Sep 2024 13:48:24 +0530 Subject: [PATCH 199/224] Bump version to 0.1.17-devel --- CHANGELOG.md | 23 +++++++++++++++++++++++ lib/util.js | 2 +- package-lock.json | 4 ++-- package.json | 4 ++-- 4 files changed, 28 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1a1d8cc5..3e055605 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,26 @@ +# confluent-kafka-javascript v0.1.17-devel + +v0.1.17-devel is a pre-production, early-access release. + +## Features + +1. Add a commitCb method to the callback-based API which allows committing asynchronously. +2. Pass assign/unassign functions to the rebalance callback in the promisified API, allowing + the user to control the assignment of partitions, or pause just after a rebalance. +3. Remove store from promisified API and let the library handle all the stores. +4. Add JavaScript-level debug logging to the client for debugging issues within the binding. +5. Various fixes for performance and robustness of the consumer cache. +6. Remove `consumerGroupId` argument from the `sendOffsets` method of the transactional producer, + and instead, only allow using a `consumer`. + +## Fixes + +1. Do not modify RegExps which don't start with a ^, instead, throw an error so + that there is no unexpected behaviour for the user (Issue [#64](https://github.com/confluentinc/confluent-kafka-javascript/issues/64)). +2. Do not mutate arguments in run, pause and resume (Issue [#61](https://github.com/confluentinc/confluent-kafka-javascript/issues/61)). +3. Fix a segmentation fault in `listGroups` when passing `matchConsumerGroupStates` as undefined. + + # confluent-kafka-javascript v0.1.16-devel v0.1.16-devel is a pre-production, early-access release. diff --git a/lib/util.js b/lib/util.js index 989f3f2f..36590458 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.16-devel'; +util.bindingVersion = 'v0.1.17-devel'; diff --git a/package-lock.json b/package-lock.json index e3d494e8..e9fe5f17 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.16-devel", + "version": "v0.1.17-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.16-devel", + "version": "v0.1.17-devel", "hasInstallScript": true, "license": "MIT", "workspaces": [ diff --git a/package.json b/package.json index 46ed7d80..3f0f7465 100644 --- a/package.json +++ b/package.json @@ -1,9 +1,9 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.16-devel", + "version": "v0.1.17-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "master", - "librdkafka_win": "2.4.0", + "librdkafka_win": "2.5.3", "main": "lib/index.js", "types": "types/index.d.ts", "scripts": { From 0e48f6384f26bf4c1c081d5670b1d0bee06cb569 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Tue, 17 Sep 2024 09:27:44 -0700 Subject: [PATCH 200/224] Add complex encryption tests (#89) --- schemaregistry/rest-service.ts | 6 +- .../dekregistry/dekregistry-client.ts | 2 +- schemaregistry/schemaregistry-client.ts | 4 +- schemaregistry/serde/avro.ts | 4 +- schemaregistry/serde/json.ts | 6 +- test/schemaregistry/serde/avro.spec.ts | 204 ++++++++++++++ test/schemaregistry/serde/json.spec.ts | 248 +++++++++++++++++- 7 files changed, 450 insertions(+), 24 deletions(-) diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index 2a2ac5d8..d1ac8907 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -24,7 +24,7 @@ export interface BearerAuthCredentials { //TODO: Consider retry policy, may need additional libraries on top of Axios export interface ClientConfig { baseURLs: string[], - cacheCapacity: number, + cacheCapacity?: number, cacheLatestTtlSecs?: number, isForward?: boolean, createAxiosDefaults?: CreateAxiosDefaults, @@ -37,7 +37,7 @@ export class RestService { private OAuthClient?: OAuthClient; private bearerAuth: boolean = false; - constructor(baseURLs: string[], isForward?: boolean, axiosDefaults?: CreateAxiosDefaults, + constructor(baseURLs: string[], isForward?: boolean, axiosDefaults?: CreateAxiosDefaults, bearerAuthCredentials?: BearerAuthCredentials) { this.client = axios.create(axiosDefaults); this.baseURLs = baseURLs; @@ -53,7 +53,7 @@ export class RestService { 'Confluent-Identity-Pool-Id': bearerAuthCredentials.identityPool, 'target-sr-cluster': bearerAuthCredentials.schemaRegistryLogicalCluster }); - this.OAuthClient = new OAuthClient(bearerAuthCredentials.clientId, bearerAuthCredentials.clientSecret, + this.OAuthClient = new OAuthClient(bearerAuthCredentials.clientId, bearerAuthCredentials.clientSecret, bearerAuthCredentials.tokenHost, bearerAuthCredentials.tokenPath, bearerAuthCredentials.scope); } } diff --git a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts index fed55558..b6800c86 100644 --- a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts @@ -65,7 +65,7 @@ class DekRegistryClient implements Client { constructor(config: ClientConfig) { const cacheOptions = { - max: config.cacheCapacity, + max: config.cacheCapacity !== undefined ? config.cacheCapacity : 1000, ...(config.cacheLatestTtlSecs !== undefined && { maxAge: config.cacheLatestTtlSecs * 1000 }), }; diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index 393de240..d5b91044 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -161,11 +161,11 @@ export class SchemaRegistryClient implements Client { constructor(config: ClientConfig) { this.clientConfig = config const cacheOptions = { - max: config.cacheCapacity, + max: config.cacheCapacity !== undefined ? config.cacheCapacity : 1000, ...(config.cacheLatestTtlSecs !== undefined && { maxAge: config.cacheLatestTtlSecs * 1000 }) }; - this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults, + this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults, config.bearerAuthCredentials); this.schemaToIdCache = new LRUCache(cacheOptions); diff --git a/schemaregistry/serde/avro.ts b/schemaregistry/serde/avro.ts index ae5fe490..8f956e3a 100644 --- a/schemaregistry/serde/avro.ts +++ b/schemaregistry/serde/avro.ts @@ -245,9 +245,9 @@ async function transform(ctx: RuleContext, schema: Type, msg: any, fieldTransfor return await Promise.all(array.map(item => transform(ctx, arraySchema.itemsType, item, fieldTransform))) case 'map': const mapSchema = schema as MapType - const map = msg as Map + const map = msg as { [key: string]: any } for (const key of Object.keys(map)) { - map.set(key, await transform(ctx, mapSchema.valuesType, map.get(key), fieldTransform)) + map[key] = await transform(ctx, mapSchema.valuesType, map[key], fieldTransform) } return map case 'record': diff --git a/schemaregistry/serde/json.ts b/schemaregistry/serde/json.ts index 5189ccf7..7e177e89 100644 --- a/schemaregistry/serde/json.ts +++ b/schemaregistry/serde/json.ts @@ -262,7 +262,8 @@ async function toType( const json = JSON.parse(info.schema) const spec = json.$schema let schema - if (spec === 'http://json-schema.org/draft/2020-12/schema') { + if (spec === 'http://json-schema.org/draft/2020-12/schema' + || spec === 'https://json-schema.org/draft/2020-12/schema') { schema = await dereferenceJSONSchemaDraft2020_12(json, { retrieve }) } else { schema = await dereferenceJSONSchemaDraft07(json, { retrieve }) @@ -302,6 +303,7 @@ async function transform(ctx: RuleContext, schema: DereferencedJSONSchema, path: for (let i = 0; i < msg.length; i++) { msg[i] = await transform(ctx, schema.items, path, msg[i], fieldTransform) } + return msg } } if (schema.$ref != null) { @@ -355,7 +357,7 @@ async function transformField(ctx: RuleContext, path: string, propName: string, function validateSubschemas(subschemas: DereferencedJSONSchema[], msg: any): DereferencedJSONSchema | null { for (let subschema of subschemas) { try { - validateJSON(subschema, msg) + validateJSON(msg, subschema) return subschema } catch (error) { // ignore diff --git a/test/schemaregistry/serde/avro.spec.ts b/test/schemaregistry/serde/avro.spec.ts index a71f6693..755a8d8f 100644 --- a/test/schemaregistry/serde/avro.spec.ts +++ b/test/schemaregistry/serde/avro.spec.ts @@ -190,6 +190,60 @@ const demoSchemaWithUnion = ` ] } ` +const schemaEvolution1 = ` +{ + "name": "SchemaEvolution", + "type": "record", + "fields": [ + { + "name": "fieldToDelete", + "type": "string" + } + ] +} +` +const schemaEvolution2 = ` +{ + "name": "SchemaEvolution", + "type": "record", + "fields": [ + { + "name": "newOptionalField", + "type": ["string", "null"], + "default": "optional" + } + ] +} +` +const complexSchema = ` +{ + "name": "ComplexSchema", + "type": "record", + "fields": [ + { + "name": "arrayField", + "type": { + "type": "array", + "items": "string" + }, + "confluent:tags": [ "PII" ] + }, + { + "name": "mapField", + "type": { + "type": "map", + "values": "string" + }, + "confluent:tags": [ "PII" ] + }, + { + "name": "unionField", + "type": ["null", "string"], + "confluent:tags": [ "PII" ] + } + ] +} +` class FakeClock extends Clock { fixedNow: number = 0 @@ -320,6 +374,38 @@ describe('AvroSerializer', () => { expect(obj2.otherField.boolField).toEqual(nested.boolField); expect(obj2.otherField.bytesField).toEqual(nested.bytesField); }) + it('schema evolution', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new AvroSerializer(client, SerdeType.VALUE, {useLatestVersion: true}) + + let obj = { + fieldToDelete: "bye", + } + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: schemaEvolution1, + } + + await client.register(subject, info, false) + + let bytes = await ser.serialize(topic, obj) + + info = { + schemaType: 'AVRO', + schema: schemaEvolution2, + } + + await client.register(subject, info, false) + + let deser = new AvroDeserializer(client, SerdeType.VALUE, {useLatestVersion: true}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.fieldToDelete).toEqual(undefined); + expect(obj2.newOptionalField).toEqual("optional"); + }) it('basic encryption', async () => { let conf: ClientConfig = { baseURLs: [baseURL], @@ -876,6 +962,124 @@ describe('AvroSerializer', () => { expect(obj2.boolField).toEqual(obj.boolField); expect(obj2.bytesField).toEqual(obj.bytesField); }) + it('complex encryption', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,NONE' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: complexSchema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + arrayField: [ 'hello' ], + mapField: { 'key': 'world' }, + unionField: 'bye', + } + let bytes = await ser.serialize(topic, obj) + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.arrayField).toEqual([ 'hello' ]); + expect(obj2.mapField).toEqual({ 'key': 'world' }); + expect(obj2.unionField).toEqual('bye'); + }) + it('complex encryption with null', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: AvroSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new AvroSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,NONE' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'AVRO', + schema: complexSchema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + arrayField: [ 'hello' ], + mapField: { 'key': 'world' }, + unionField: null + } + let bytes = await ser.serialize(topic, obj) + + let deserConfig: AvroDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new AvroDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.arrayField).toEqual([ 'hello' ]); + expect(obj2.mapField).toEqual({ 'key': 'world' }); + expect(obj2.unionField).toEqual(null); + }) it('jsonata fully compatible', async () => { let rule1To2 = "$merge([$sift($, function($v, $k) {$k != 'size'}), {'height': $.'size'}])" let rule2To1 = "$merge([$sift($, function($v, $k) {$k != 'height'}), {'size': $.'height'}])" diff --git a/test/schemaregistry/serde/json.spec.ts b/test/schemaregistry/serde/json.spec.ts index 69a4ae28..ce979fbc 100644 --- a/test/schemaregistry/serde/json.spec.ts +++ b/test/schemaregistry/serde/json.spec.ts @@ -105,6 +105,38 @@ const demoSchema2020_12 = ` } } ` +const complexSchema = ` +{ + "type": "object", + "properties": { + "arrayField": { + "type": "array", + "items": { + "type": "string" + }, + "confluent:tags": [ "PII" ] + }, + "objectField": { + "type": "object", + "properties": { + "stringField": { "type": "string" } + }, + "confluent:tags": [ "PII" ] + }, + "unionField": { + "oneOf": [ + { + "type": "null" + }, + { + "type": "string" + } + ], + "confluent:tags": [ "PII" ] + } + } +} +` describe('JsonSerializer', () => { afterEach(async () => { @@ -136,6 +168,37 @@ describe('JsonSerializer', () => { let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) }) + it('basic serialization 2020-12', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + useLatestVersion: true, + validate: true + }) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchema2020_12 + } + + await client.register(subject, info, false) + + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) it('serialize nested', async () => { let conf: ClientConfig = { baseURLs: [baseURL], @@ -237,38 +300,77 @@ describe('JsonSerializer', () => { await expect(() => ser.serialize(topic, diffObj)).rejects.toThrow(SerializationError) }) - it('basic serialization 2020-12', async () => { + it('basic encryption', async () => { let conf: ClientConfig = { baseURLs: [baseURL], cacheCapacity: 1000 } let client = SchemaRegistryClient.newClient(conf) - let ser = new JsonSerializer(client, SerdeType.VALUE, { + let serConfig: JsonSerializerConfig = { useLatestVersion: true, - validate: true - }) + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new JsonSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! - let obj = { - intField: 123, - doubleField: 45.67, - stringField: 'hi', - boolField: true, - bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,NONE' } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + let info: SchemaInfo = { schemaType: 'JSON', - schema: demoSchema2020_12 + schema: demoSchema, + ruleSet } await client.register(subject, info, false) + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } let bytes = await ser.serialize(topic, obj) - let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + // reset encrypted field + obj.stringField = 'hi' + obj.bytesField = Buffer.from([0, 0, 0, 1]).toString('base64') + + let deserConfig: JsonDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new JsonDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) + + clearKmsClients() + let registry = new RuleRegistry() + registry.registerExecutor(new FieldEncryptionExecutor()) + deser = new JsonDeserializer(client, SerdeType.VALUE, {}, registry) + obj2 = await deser.deserialize(topic, bytes) + expect(obj2).not.toEqual(obj); }) - it('basic encryption', async () => { + it('basic encryption 2020-12', async () => { let conf: ClientConfig = { baseURLs: [baseURL], cacheCapacity: 1000 @@ -302,7 +404,7 @@ describe('JsonSerializer', () => { let info: SchemaInfo = { schemaType: 'JSON', - schema: demoSchema, + schema: demoSchema2020_12, ruleSet } @@ -477,6 +579,124 @@ describe('JsonSerializer', () => { let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) }) + it('complex encryption', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: JsonSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new JsonSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,NONE' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: complexSchema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + arrayField: [ 'hello' ], + objectField: { 'stringField': 'world' }, + unionField: 'bye', + } + let bytes = await ser.serialize(topic, obj) + + let deserConfig: JsonDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new JsonDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.arrayField).toEqual([ 'hello' ]); + expect(obj2.objectField.stringField).toEqual('world'); + expect(obj2.unionField).toEqual('bye'); + }) + it('complex encryption with null', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let serConfig: JsonSerializerConfig = { + useLatestVersion: true, + ruleConfig: { + secret: 'mysecret' + } + } + let ser = new JsonSerializer(client, SerdeType.VALUE, serConfig) + let dekClient = fieldEncryptionExecutor.client! + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'kek1', + 'encrypt.kms.type': 'local-kms', + 'encrypt.kms.key.id': 'mykey', + }, + onFailure: 'ERROR,NONE' + } + let ruleSet: RuleSet = { + domainRules: [encRule] + } + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: complexSchema, + ruleSet + } + + await client.register(subject, info, false) + + let obj = { + arrayField: [ 'hello' ], + objectField: { 'stringField': 'world' }, + unionField: null, + } + let bytes = await ser.serialize(topic, obj) + + let deserConfig: JsonDeserializerConfig = { + ruleConfig: { + secret: 'mysecret' + } + } + let deser = new JsonDeserializer(client, SerdeType.VALUE, deserConfig) + fieldEncryptionExecutor.client = dekClient + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2.arrayField).toEqual([ 'hello' ]); + expect(obj2.objectField.stringField).toEqual('world'); + expect(obj2.unionField).toEqual(null); + }) it('jsonata fully compatible', async () => { let rule1To2 = "$merge([$sift($, function($v, $k) {$k != 'size'}), {'height': $.'size'}])" let rule2To1 = "$merge([$sift($, function($v, $k) {$k != 'height'}), {'size': $.'height'}])" From 493819412a19a4900f5d36f1633f66a5449de886 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 19 Sep 2024 20:42:28 -0700 Subject: [PATCH 201/224] Add index.ts (#91) --- eslint.config.js | 6 +++ package-lock.json | 51 +++++++++++++++++++ .../{Makefile.schemaregistry => Makefile} | 0 schemaregistry/index.ts | 18 +++++++ schemaregistry/package.json | 11 ++-- schemaregistry/tsconfig-build.json | 6 +++ schemaregistry/tsconfig.json | 14 ++--- 7 files changed, 96 insertions(+), 10 deletions(-) rename schemaregistry/{Makefile.schemaregistry => Makefile} (100%) create mode 100644 schemaregistry/index.ts create mode 100644 schemaregistry/tsconfig-build.json diff --git a/eslint.config.js b/eslint.config.js index 0ee61d12..1aea2e1c 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -1,6 +1,7 @@ const js = require("@eslint/js"); const jest = require('eslint-plugin-jest'); const ts = require('typescript-eslint'); +const tsdoc = require('eslint-plugin-tsdoc'); const ckjsSpecificSettings = { languageOptions: { @@ -56,11 +57,16 @@ module.exports = ts.config( ...ts.configs.recommended.map((config) => ({ ...config, ignores: ["**/*.js", "types/rdkafka.d.ts", "types/kafkajs.d.ts", "types/config.d.ts"], + plugins: { + "@typescript-eslint": ts.plugin, + tsdoc, + }, rules: { ...config.rules, "prefer-const": "warn", "@typescript-eslint/no-explicit-any": "warn", "@typescript-eslint/no-unused-vars": "warn", + "tsdoc/syntax": "warn", } })), ); diff --git a/package-lock.json b/package-lock.json index e9fe5f17..bed11757 100644 --- a/package-lock.json +++ b/package-lock.json @@ -2143,6 +2143,40 @@ "node-pre-gyp": "bin/node-pre-gyp" } }, + "node_modules/@microsoft/tsdoc": { + "version": "0.15.0", + "resolved": "https://registry.npmjs.org/@microsoft/tsdoc/-/tsdoc-0.15.0.tgz", + "integrity": "sha512-HZpPoABogPvjeJOdzCOSJsXeL/SMCBgBZMVC3X3d7YYp2gf31MfxhUoYUNwf1ERPJOnQc0wkFn9trqI6ZEdZuA==", + "dev": true + }, + "node_modules/@microsoft/tsdoc-config": { + "version": "0.17.0", + "resolved": "https://registry.npmjs.org/@microsoft/tsdoc-config/-/tsdoc-config-0.17.0.tgz", + "integrity": "sha512-v/EYRXnCAIHxOHW+Plb6OWuUoMotxTN0GLatnpOb1xq0KuTNw/WI3pamJx/UbsoJP5k9MCw1QxvvhPcF9pH3Zg==", + "dev": true, + "dependencies": { + "@microsoft/tsdoc": "0.15.0", + "ajv": "~8.12.0", + "jju": "~1.4.0", + "resolve": "~1.22.2" + } + }, + "node_modules/@microsoft/tsdoc-config/node_modules/ajv": { + "version": "8.12.0", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.12.0.tgz", + "integrity": "sha512-sRu1kpcO9yLtYxBKvqfTeh9KzZEwO3STyX1HT+4CaDzC6HpTGYhIhPIzj9XuKU7KYDwnaeh5hcOwjy1QuJzBPA==", + "dev": true, + "dependencies": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + }, + "funding": { + "type": "github", + "url": "https://github.com/sponsors/epoberezkin" + } + }, "node_modules/@nodelib/fs.scandir": { "version": "2.1.5", "dev": true, @@ -4698,6 +4732,16 @@ } } }, + "node_modules/eslint-plugin-tsdoc": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-tsdoc/-/eslint-plugin-tsdoc-0.3.0.tgz", + "integrity": "sha512-0MuFdBrrJVBjT/gyhkP2BqpD0np1NxNLfQ38xXDlSs/KVVpKI2A6vN7jx2Rve/CyUsvOsMGwp9KKrinv7q9g3A==", + "dev": true, + "dependencies": { + "@microsoft/tsdoc": "0.15.0", + "@microsoft/tsdoc-config": "0.17.0" + } + }, "node_modules/eslint-scope": { "version": "7.2.2", "dev": true, @@ -6293,6 +6337,12 @@ "url": "https://github.com/chalk/supports-color?sponsor=1" } }, + "node_modules/jju": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/jju/-/jju-1.4.0.tgz", + "integrity": "sha512-8wb9Yw966OSxApiCt0K3yNJL8pnNeIv+OEq2YMidz4FKP6nonSRoOXc80iXY4JaN2FC11B9qsNmDsm+ZOfMROA==", + "dev": true + }, "node_modules/joi": { "version": "17.13.3", "resolved": "https://registry.npmjs.org/joi/-/joi-17.13.3.tgz", @@ -8987,6 +9037,7 @@ "bluebird": "^3.5.3", "eslint": "^8.57.0", "eslint-plugin-jest": "^28.6.0", + "eslint-plugin-tsdoc": "^0.3.0", "jest": "^29.7.0", "jsdoc": "^4.0.2", "mocha": "^10.7.0", diff --git a/schemaregistry/Makefile.schemaregistry b/schemaregistry/Makefile similarity index 100% rename from schemaregistry/Makefile.schemaregistry rename to schemaregistry/Makefile diff --git a/schemaregistry/index.ts b/schemaregistry/index.ts new file mode 100644 index 00000000..27389c12 --- /dev/null +++ b/schemaregistry/index.ts @@ -0,0 +1,18 @@ +export * from './confluent/types/decimal_pb' +export * from './confluent/meta_pb' +export * from './rules/encryption/awskms/aws-driver' +export * from './rules/encryption/azurekms/azure-driver' +export * from './rules/encryption/gcpkms/gcp-driver' +export * from './rules/encryption/hcvault/hcvault-driver' +export * from './rules/encryption/localkms/local-driver' +export * from './rules/encryption/encrypt-executor' +export * from './rules/encryption/kms-registry' +export * from './rules/jsonata/jsonata-executor' +export * from './serde/avro' +export * from './serde/json' +export * from './serde/protobuf' +export * from './serde/rule-registry' +export * from './serde/serde' +export * from './rest-error' +export * from './mock-schemaregistry-client' +export * from './schemaregistry-client' diff --git a/schemaregistry/package.json b/schemaregistry/package.json index aea4ab00..4841fe10 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,8 +1,12 @@ { "name": "@confluentinc/schemaregistry", - "version": "v0.1.16-devel", + "version": "v0.1.17-devel", "description": "Node.js client for Confluent Schema Registry", - "main": "lib/index.js", + "main": "dist/index.js", + "types": "dist/index.d.ts", + "files": [ + "dist" + ], "devDependencies": { "@bufbuild/buf": "^1.37.0", "@bufbuild/protoc-gen-es": "^2.0.0", @@ -12,6 +16,7 @@ "bluebird": "^3.5.3", "eslint": "^8.57.0", "eslint-plugin-jest": "^28.6.0", + "eslint-plugin-tsdoc": "^0.3.0", "jest": "^29.7.0", "jsdoc": "^4.0.2", "mocha": "^10.7.0", @@ -44,7 +49,7 @@ }, "scripts": { "test:types": "tsc -p .", - "test:schemaregistry": "make -f Makefile.schemaregistry test" + "test:schemaregistry": "make -f Makefile test" }, "keywords": [ "schemaregistry", diff --git a/schemaregistry/tsconfig-build.json b/schemaregistry/tsconfig-build.json new file mode 100644 index 00000000..46357df2 --- /dev/null +++ b/schemaregistry/tsconfig-build.json @@ -0,0 +1,6 @@ +{ + "extends": "./tsconfig.json", + "exclude": [ + "../test/**/*" + ] +} diff --git a/schemaregistry/tsconfig.json b/schemaregistry/tsconfig.json index 5379dc3f..7a571b6f 100644 --- a/schemaregistry/tsconfig.json +++ b/schemaregistry/tsconfig.json @@ -1,15 +1,16 @@ { "compilerOptions": { "baseUrl": ".", + "target": "es2021", "lib": [ "es2021", "dom" ], - "module": "nodenext", - "target": "es2021", - "strict": true, + "declaration": true, + "outDir": "dist", + "types": ["../node_modules/@types/node"], "esModuleInterop": true, + "strict": true, "forceConsistentCasingInFileNames": true, - "moduleResolution": "nodenext", "allowUnusedLabels": false, "allowUnreachableCode": false, "noFallthroughCasesInSwitch": true, @@ -18,10 +19,9 @@ "noPropertyAccessFromIndexSignature": true, "noUnusedLocals": true, "useUnknownInCatchVariables": true, - "types": ["../node_modules/@types/node"], - "typeRoots": ["."], - "noEmit": true, "resolveJsonModule": true, + "moduleResolution": "nodenext", + "module": "nodenext", "skipLibCheck": true }, "include": [ From f97166f992920429ae925331fed582155bbcb336 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 20 Sep 2024 00:12:54 -0700 Subject: [PATCH 202/224] Enhance HighLevelProducer to take schema serializers (#92) --- lib/producer/high-level-producer.js | 82 ++++++++++++++++++++++++++--- types/rdkafka.d.ts | 4 ++ 2 files changed, 79 insertions(+), 7 deletions(-) diff --git a/lib/producer/high-level-producer.js b/lib/producer/high-level-producer.js index 673bb9e1..cb7b8f27 100644 --- a/lib/producer/high-level-producer.js +++ b/lib/producer/high-level-producer.js @@ -43,7 +43,37 @@ function createSerializer(serializer) { // operation if the number of parameters the function accepts is > 1 return { apply: applyFn, - async: serializer.length > 1 + async: serializer.length > 1, + needsTopic: false + }; +} + +/** + * Create a serializer that additionally takes the topic name + * + * Method simply wraps a serializer provided by a user + * so it adds context to the error + * + * @returns {function} Serialization function + */ +function createTopicSerializer(serializer) { + var applyFn = function serializationWrapper(t, v, cb) { + try { + return cb ? serializer(t, v, cb) : serializer(t, v); + } catch (e) { + var modifiedError = new Error('Could not serialize value: ' + e.message); + modifiedError.value = v; + modifiedError.serializer = serializer; + throw modifiedError; + } + }; + + // We can check how many parameters the function has and activate the asynchronous + // operation if the number of parameters the function accepts is > 2 + return { + apply: applyFn, + async: serializer.length > 2, + needsTopic: true }; } @@ -256,10 +286,20 @@ HighLevelProducer.prototype._modifiedProduce = function(topic, partition, messag try { if (this.valueSerializer.async) { - // If this is async we need to give it a callback - this.valueSerializer.apply(message, valueSerializerCallback); + if (this.valueSerializer.needsTopic) { + // If this is async we need to give it a callback + this.valueSerializer.apply(topic, message, valueSerializerCallback); + } else { + // If this is async we need to give it a callback + this.valueSerializer.apply(message, valueSerializerCallback); + } } else { - var serializedValue = this.valueSerializer.apply(message); + var serializedValue; + if (this.valueSerializer.needsTopic) { + serializedValue = this.valueSerializer.apply(topic, message); + } else { + serializedValue = this.valueSerializer.apply(message); + } // Check if we were returned a promise in order to support promise behavior if (serializedValue && typeof serializedValue.then === 'function' && @@ -272,10 +312,20 @@ HighLevelProducer.prototype._modifiedProduce = function(topic, partition, messag } if (this.keySerializer.async) { - // If this is async we need to give it a callback - this.keySerializer.apply(key, keySerializerCallback); + if (this.valueSerializer.needsTopic) { + // If this is async we need to give it a callback + this.keySerializer.apply(topic, key, keySerializerCallback); + } else { + // If this is async we need to give it a callback + this.keySerializer.apply(key, keySerializerCallback); + } } else { - var serializedKey = this.keySerializer.apply(key); + var serializedKey; + if (this.valueSerializer.needsTopic) { + serializedKey = this.keySerializer.apply(topic, key); + } else { + serializedKey = this.keySerializer.apply(key); + } // Check if we were returned a promise in order to support promise behavior if (serializedKey && typeof serializedKey.then === 'function' && @@ -319,3 +369,21 @@ HighLevelProducer.prototype.setKeySerializer = function(serializer) { HighLevelProducer.prototype.setValueSerializer = function(serializer) { this.valueSerializer = createSerializer(serializer); }; + +/** + * Set the topic-key serializer + * + * A serializer that takes the topic name in addition to the key. + */ +HighLevelProducer.prototype.setTopicKeySerializer = function(serializer) { + this.keySerializer = createTopicSerializer(serializer); +}; + +/** + * Set the topic-value serializer + * + * A serializer that takes the topic name in addition to the value. + */ +HighLevelProducer.prototype.setTopicValueSerializer = function(serializer) { + this.valueSerializer = createTopicSerializer(serializer); +}; diff --git a/types/rdkafka.d.ts b/types/rdkafka.d.ts index 3f481d72..6a46da26 100644 --- a/types/rdkafka.d.ts +++ b/types/rdkafka.d.ts @@ -296,6 +296,10 @@ export class HighLevelProducer extends Producer { setKeySerializer(serializer: (key: any) => MessageKey | Promise): void; setValueSerializer(serializer: (value: any, cb: (err: any, value: MessageValue) => void) => void): void; setValueSerializer(serializer: (value: any) => MessageValue | Promise): void; + setTopicKeySerializer(serializer: (topic: string, key: any, cb: (err: any, key: MessageKey) => void) => void): void; + setTopicKeySerializer(serializer: (topic: string, key: any) => MessageKey | Promise): void; + setTopicValueSerializer(serializer: (topic: string, value: any, cb: (err: any, value: MessageValue) => void) => void): void; + setTopicValueSerializer(serializer: (topic: string, value: any) => MessageValue | Promise): void; } export const features: string[]; From e27aedb8b084e4f59dd43a5bd9afbf877f3ad1cc Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Fri, 20 Sep 2024 12:02:29 -0400 Subject: [PATCH 203/224] Add auth features (#47) (#94) * Add auth features * Update schemaregistry/rest-service.ts * Minor changes * Minor changes (missed a few) --------- Co-authored-by: Robert Yokota --- schemaregistry/rest-service.ts | 130 ++++++++++++++++++++---- schemaregistry/schemaregistry-client.ts | 2 +- test/schemaregistry/test-constants.ts | 13 ++- 3 files changed, 118 insertions(+), 27 deletions(-) diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index d1ac8907..67b5bdfd 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -1,7 +1,6 @@ import axios, { AxiosInstance, AxiosRequestConfig, AxiosResponse, CreateAxiosDefaults } from 'axios'; import { OAuthClient } from './oauth/oauth-client'; import { RestError } from './rest-error'; - /* * Confluent-Schema-Registry-TypeScript - Node.js wrapper for Confluent Schema Registry * @@ -11,14 +10,27 @@ import { RestError } from './rest-error'; * of the MIT license. See the LICENSE.txt file for details. */ +export interface BasicAuthCredentials { + credentialsSource: 'USER_INFO' | 'URL' | 'SASL_INHERIT', + userInfo?: string, + saslInfo?: SaslInfo +} + +export interface SaslInfo { + mechanism?: string, + username: string, + password: string +} + export interface BearerAuthCredentials { - clientId: string, - clientSecret: string, - tokenHost: string, - tokenPath: string, - schemaRegistryLogicalCluster: string, - identityPool: string, - scope: string + credentialsSource: 'STATIC_TOKEN' | 'OAUTHBEARER', + token?: string, + issuerEndpointUrl?: string, + clientId?: string, + clientSecret?: string, + scope?: string, + logicalCluster?: string, + identityPoolId?: string, } //TODO: Consider retry policy, may need additional libraries on top of Axios @@ -28,17 +40,20 @@ export interface ClientConfig { cacheLatestTtlSecs?: number, isForward?: boolean, createAxiosDefaults?: CreateAxiosDefaults, + basicAuthCredentials?: BasicAuthCredentials, bearerAuthCredentials?: BearerAuthCredentials, } +const toBase64 = (str: string): string => Buffer.from(str).toString('base64'); + export class RestService { private client: AxiosInstance; private baseURLs: string[]; - private OAuthClient?: OAuthClient; - private bearerAuth: boolean = false; + private oauthClient?: OAuthClient; + private oauthBearer: boolean = false; constructor(baseURLs: string[], isForward?: boolean, axiosDefaults?: CreateAxiosDefaults, - bearerAuthCredentials?: BearerAuthCredentials) { + basicAuthCredentials?: BasicAuthCredentials, bearerAuthCredentials?: BearerAuthCredentials) { this.client = axios.create(axiosDefaults); this.baseURLs = baseURLs; @@ -46,15 +61,88 @@ export class RestService { this.client.defaults.headers.common['X-Forward'] = 'true' } + this.handleBasicAuth(basicAuthCredentials); + this.handleBearerAuth(bearerAuthCredentials); + + if (!basicAuthCredentials && !bearerAuthCredentials) { + throw new Error('No auth credentials provided'); + } + } + + handleBasicAuth(basicAuthCredentials?: BasicAuthCredentials): void { + if (basicAuthCredentials) { + switch (basicAuthCredentials.credentialsSource) { + case 'USER_INFO': + if (!basicAuthCredentials.userInfo) { + throw new Error('User info not provided'); + } + this.setAuth(toBase64(basicAuthCredentials.userInfo!)); + break; + case 'SASL_INHERIT': + if (!basicAuthCredentials.saslInfo) { + throw new Error('Sasl info not provided'); + } + if (basicAuthCredentials.saslInfo.mechanism?.toUpperCase() === 'GSSAPI') { + throw new Error('SASL_INHERIT support PLAIN and SCRAM SASL mechanisms only'); + } + this.setAuth(toBase64(`${basicAuthCredentials.saslInfo.username}:${basicAuthCredentials.saslInfo.password}`)); + break; + case 'URL': + if (!basicAuthCredentials.userInfo) { + throw new Error('User info not provided'); + } + const basicAuthUrl = new URL(basicAuthCredentials.userInfo); + this.setAuth(toBase64(`${basicAuthUrl.username}:${basicAuthUrl.password}`)); + break; + default: + throw new Error('Invalid basic auth credentials source'); + } + } + } + + handleBearerAuth(bearerAuthCredentials?: BearerAuthCredentials): void { if (bearerAuthCredentials) { - this.bearerAuth = true; delete this.client.defaults.auth; + + const headers = ['logicalCluster', 'identityPoolId']; + const missingHeaders = headers.find(header => bearerAuthCredentials[header as keyof typeof bearerAuthCredentials]); + + if (missingHeaders) { + throw new Error(`Bearer auth header '${missingHeaders}' not provided`); + } + this.setHeaders({ - 'Confluent-Identity-Pool-Id': bearerAuthCredentials.identityPool, - 'target-sr-cluster': bearerAuthCredentials.schemaRegistryLogicalCluster + 'Confluent-Identity-Pool-Id': bearerAuthCredentials.identityPoolId!, + 'target-sr-cluster': bearerAuthCredentials.logicalCluster! }); - this.OAuthClient = new OAuthClient(bearerAuthCredentials.clientId, bearerAuthCredentials.clientSecret, - bearerAuthCredentials.tokenHost, bearerAuthCredentials.tokenPath, bearerAuthCredentials.scope); + + switch (bearerAuthCredentials.credentialsSource) { + case 'STATIC_TOKEN': + if (!bearerAuthCredentials.token) { + throw new Error('Bearer token not provided'); + } + this.setAuth(undefined, bearerAuthCredentials.token); + break; + case 'OAUTHBEARER': + this.oauthBearer = true; + const requiredFields = [ + 'clientId', + 'clientSecret', + 'issuerEndpointUrl', + 'scope' + ]; + const missingField = requiredFields.find(field => bearerAuthCredentials[field as keyof typeof bearerAuthCredentials]); + + if (missingField) { + throw new Error(`OAuth credential '${missingField}' not provided`); + } + const issuerEndPointUrl = new URL(bearerAuthCredentials.issuerEndpointUrl!); + this.oauthClient = new OAuthClient(bearerAuthCredentials.clientId!, bearerAuthCredentials.clientSecret!, + issuerEndPointUrl.host, issuerEndPointUrl.pathname, bearerAuthCredentials.scope!); + break; + default: + throw new Error('Invalid bearer auth credentials source'); + } } } @@ -65,8 +153,8 @@ export class RestService { config?: AxiosRequestConfig, ): Promise> { - if (this.bearerAuth) { - await this.setBearerToken(); + if (this.oauthBearer) { + await this.setOAuthBearerToken(); } for (let i = 0; i < this.baseURLs.length; i++) { @@ -111,12 +199,12 @@ export class RestService { } } - async setBearerToken(): Promise { - if (!this.OAuthClient) { + async setOAuthBearerToken(): Promise { + if (!this.oauthClient) { throw new Error('OAuthClient not initialized'); } - const bearerToken: string = await this.OAuthClient.getAccessToken(); + const bearerToken: string = await this.oauthClient.getAccessToken(); this.setAuth(undefined, bearerToken); } diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index d5b91044..f501ccf0 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -166,7 +166,7 @@ export class SchemaRegistryClient implements Client { }; this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults, - config.bearerAuthCredentials); + config.basicAuthCredentials, config.bearerAuthCredentials); this.schemaToIdCache = new LRUCache(cacheOptions); this.idToSchemaInfoCache = new LRUCache(cacheOptions); diff --git a/test/schemaregistry/test-constants.ts b/test/schemaregistry/test-constants.ts index 7a478765..a17fa6dd 100644 --- a/test/schemaregistry/test-constants.ts +++ b/test/schemaregistry/test-constants.ts @@ -1,5 +1,5 @@ import { CreateAxiosDefaults } from 'axios'; -import { ClientConfig } from '../../schemaregistry/rest-service'; +import { ClientConfig, BasicAuthCredentials } from '../../schemaregistry/rest-service'; const baseUrls = ['http://localhost:8081']; @@ -9,19 +9,21 @@ const createAxiosDefaults: CreateAxiosDefaults = { headers: { 'Content-Type': 'application/vnd.schemaregistry.v1+json', }, - auth: { - username: 'RBACAllowedUser-lsrc1', - password: 'nohash', - }, timeout: 10000 }; +const basicAuthCredentials: BasicAuthCredentials = { + credentialsSource: 'USER_INFO', + userInfo: 'RBACAllowedUser-lsrc1:nohash', +}; + const clientConfig: ClientConfig = { baseURLs: baseUrls, createAxiosDefaults: createAxiosDefaults, isForward: false, cacheCapacity: 512, cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials, }; const mockClientConfig: ClientConfig = { @@ -30,6 +32,7 @@ const mockClientConfig: ClientConfig = { isForward: false, cacheCapacity: 512, cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials }; export { clientConfig, mockClientConfig }; From 5485bc8c0d7f22ed4bfc92b5bea0376d2f15c6fa Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 20 Sep 2024 09:31:37 -0700 Subject: [PATCH 204/224] Add more JSON Schema validation tests (#95) --- .../rules/encryption/encrypt-executor.ts | 6 +- schemaregistry/serde/avro.ts | 6 +- schemaregistry/serde/json.ts | 4 +- schemaregistry/serde/protobuf.ts | 4 +- schemaregistry/serde/serde.ts | 3 +- test/schemaregistry/serde/json.spec.ts | 66 +++++++++++++++++-- 6 files changed, 74 insertions(+), 15 deletions(-) diff --git a/schemaregistry/rules/encryption/encrypt-executor.ts b/schemaregistry/rules/encryption/encrypt-executor.ts index 018db615..5eaf67b0 100644 --- a/schemaregistry/rules/encryption/encrypt-executor.ts +++ b/schemaregistry/rules/encryption/encrypt-executor.ts @@ -84,11 +84,11 @@ export class FieldEncryptionExecutor extends FieldRuleExecutor { this.config = config } - type(): string { + override type(): string { return 'ENCRYPT' } - newTransform(ctx: RuleContext): FieldTransform { + override newTransform(ctx: RuleContext): FieldTransform { const cryptor = this.getCryptor(ctx) const kekName = this.getKekName(ctx) const dekExpiryDays = this.getDekExpiryDays(ctx) @@ -97,7 +97,7 @@ export class FieldEncryptionExecutor extends FieldRuleExecutor { return transform } - async close(): Promise { + override async close(): Promise { if (this.client != null) { await this.client.close() } diff --git a/schemaregistry/serde/avro.ts b/schemaregistry/serde/avro.ts index 8f956e3a..dcf509fd 100644 --- a/schemaregistry/serde/avro.ts +++ b/schemaregistry/serde/avro.ts @@ -45,7 +45,7 @@ export class AvroSerializer extends Serializer implements AvroSerde { } } - async serialize(topic: string, msg: any): Promise { + override async serialize(topic: string, msg: any): Promise { if (this.client == null) { throw new Error('client is not initialized') } @@ -126,7 +126,7 @@ export class AvroDeserializer extends Deserializer implements AvroSerde { } } - async deserialize(topic: string, payload: Buffer): Promise { + override async deserialize(topic: string, payload: Buffer): Promise { if (!Buffer.isBuffer(payload)) { throw new Error('Invalid buffer') } @@ -366,7 +366,7 @@ function resolveUnion(schema: Type, msg: any): Type | null { function getInlineTags(info: SchemaInfo, deps: Map): Map> { const inlineTags = new Map>() getInlineTagsRecursively('', '', JSON.parse(info.schema), inlineTags) - for (const depSchema of Object.values(deps)) { + for (const depSchema of deps.values()) { getInlineTagsRecursively('', '', JSON.parse(depSchema), inlineTags) } return inlineTags diff --git a/schemaregistry/serde/json.ts b/schemaregistry/serde/json.ts index 7e177e89..709a8b3f 100644 --- a/schemaregistry/serde/json.ts +++ b/schemaregistry/serde/json.ts @@ -65,7 +65,7 @@ export class JsonSerializer extends Serializer implements JsonSerde { } } - async serialize(topic: string, msg: any): Promise { + override async serialize(topic: string, msg: any): Promise { if (this.client == null) { throw new Error('client is not initialized') } @@ -139,7 +139,7 @@ export class JsonDeserializer extends Deserializer implements JsonSerde { } } - async deserialize(topic: string, payload: Buffer): Promise { + override async deserialize(topic: string, payload: Buffer): Promise { if (!Buffer.isBuffer(payload)) { throw new Error('Invalid buffer') } diff --git a/schemaregistry/serde/protobuf.ts b/schemaregistry/serde/protobuf.ts index cfa59ce7..3c23c1cb 100644 --- a/schemaregistry/serde/protobuf.ts +++ b/schemaregistry/serde/protobuf.ts @@ -110,7 +110,7 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { } } - async serialize(topic: string, msg: any): Promise { + override async serialize(topic: string, msg: any): Promise { if (this.client == null) { throw new Error('client is not initialized') } @@ -293,7 +293,7 @@ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde } } - async deserialize(topic: string, payload: Buffer): Promise { + override async deserialize(topic: string, payload: Buffer): Promise { if (!Buffer.isBuffer(payload)) { throw new Error('Invalid buffer') } diff --git a/schemaregistry/serde/serde.ts b/schemaregistry/serde/serde.ts index 8b2a2ca1..4f2f6732 100644 --- a/schemaregistry/serde/serde.ts +++ b/schemaregistry/serde/serde.ts @@ -224,7 +224,6 @@ export abstract class Serializer extends Serde { return this.conf as SerializerConfig } - // serialize will serialize the given message abstract serialize(topic: string, msg: any): Promise // GetID returns a schema ID for the given schema @@ -282,6 +281,8 @@ export abstract class Deserializer extends Serde { return this.conf as DeserializerConfig } + abstract deserialize(topic: string, payload: Buffer): Promise + async getSchema(topic: string, payload: Buffer, format?: string): Promise { const magicByte = payload.subarray(0, 1) if (!magicByte.equals(MAGIC_BYTE)) { diff --git a/test/schemaregistry/serde/json.spec.ts b/test/schemaregistry/serde/json.spec.ts index ce979fbc..27378f0a 100644 --- a/test/schemaregistry/serde/json.spec.ts +++ b/test/schemaregistry/serde/json.spec.ts @@ -39,7 +39,15 @@ const rootSchema = ` } } ` - +const rootSchema2020_12 = ` +{ + "$schema": "https://json-schema.org/draft/2020-12/schema", + "type": "object", + "properties": { + "otherField": { "$ref": "DemoSchema" } + } +} +` const demoSchema = ` { "type": "object", @@ -154,7 +162,10 @@ describe('JsonSerializer', () => { cacheCapacity: 1000 } let client = SchemaRegistryClient.newClient(conf) - let ser = new JsonSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + autoRegisterSchemas: true, + validate: true + }) let obj = { intField: 123, doubleField: 45.67, @@ -205,7 +216,10 @@ describe('JsonSerializer', () => { cacheCapacity: 1000 } let client = SchemaRegistryClient.newClient(conf) - let ser = new JsonSerializer(client, SerdeType.VALUE, {autoRegisterSchemas: true}) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + autoRegisterSchemas: true, + validate: true + }) let obj = { intField: 123, @@ -226,7 +240,10 @@ describe('JsonSerializer', () => { cacheCapacity: 1000 } let client = SchemaRegistryClient.newClient(conf) - let ser = new JsonSerializer(client, SerdeType.VALUE, {useLatestVersion: true}) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + useLatestVersion: true, + validate: true + }) let info: SchemaInfo = { schemaType: 'JSON', @@ -258,6 +275,47 @@ describe('JsonSerializer', () => { let obj2 = await deser.deserialize(topic, bytes) expect(obj2).toEqual(obj) }) + it('serialize reference 2020_12', async () => { + let conf: ClientConfig = { + baseURLs: [baseURL], + cacheCapacity: 1000 + } + let client = SchemaRegistryClient.newClient(conf) + let ser = new JsonSerializer(client, SerdeType.VALUE, { + useLatestVersion: true, + validate: true + }) + + let info: SchemaInfo = { + schemaType: 'JSON', + schema: demoSchema2020_12 + } + await client.register('demo-value', info, false) + + info = { + schemaType: 'JSON', + schema: rootSchema2020_12, + references: [{ + name: 'DemoSchema', + subject: 'demo-value', + version: 1 + }] + } + await client.register(subject, info, false) + + let obj = { + intField: 123, + doubleField: 45.67, + stringField: 'hi', + boolField: true, + bytesField: Buffer.from([0, 0, 0, 1]).toString('base64') + } + let bytes = await ser.serialize(topic, obj) + + let deser = new JsonDeserializer(client, SerdeType.VALUE, {}) + let obj2 = await deser.deserialize(topic, bytes) + expect(obj2).toEqual(obj) + }) it('basic failing validation', async () => { let conf: ClientConfig = { baseURLs: [baseURL], From 96c835783d3d7019e6252c3a7e35520696e4acda Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 20 Sep 2024 09:31:51 -0700 Subject: [PATCH 205/224] Move ts-jest to dev dependencies (#96) --- eslint.config.js | 3 + package-lock.json | 288 ++++++++++++++++++++++++++++++++++++++++++++-- package.json | 3 +- 3 files changed, 283 insertions(+), 11 deletions(-) diff --git a/eslint.config.js b/eslint.config.js index 1aea2e1c..15141b46 100644 --- a/eslint.config.js +++ b/eslint.config.js @@ -36,6 +36,9 @@ const ckjsSpecificJestSettings = { }; module.exports = ts.config( + { + ignores: ["**/dist/"] + }, { ...js.configs.recommended, files: ["lib/**/*.js", "test/promisified/**/*.js"], diff --git a/package-lock.json b/package-lock.json index bed11757..2840e12d 100644 --- a/package-lock.json +++ b/package-lock.json @@ -35,7 +35,6 @@ "nan": "^2.17.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", - "ts-jest": "^29.2.4", "validator": "^13.12.0" }, "devDependencies": { @@ -43,6 +42,7 @@ "@bufbuild/protoc-gen-es": "^2.0.0", "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", + "@types/jest": "^29.5.13", "@types/node": "^20.16.1", "bluebird": "^3.5.3", "eslint": "^8.57.0", @@ -51,6 +51,7 @@ "jsdoc": "^4.0.2", "mocha": "^10.7.0", "node-gyp": "^9.3.1", + "ts-jest": "^29.2.5", "typescript": "^5.5.4", "typescript-eslint": "^8.2.0" }, @@ -60,6 +61,7 @@ }, "node_modules/@ampproject/remapping": { "version": "2.3.0", + "dev": true, "license": "Apache-2.0", "dependencies": { "@jridgewell/gen-mapping": "^0.3.5", @@ -954,6 +956,7 @@ }, "node_modules/@babel/code-frame": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/highlight": "^7.24.7", @@ -965,6 +968,7 @@ }, "node_modules/@babel/compat-data": { "version": "7.25.2", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -972,6 +976,7 @@ }, "node_modules/@babel/core": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@ampproject/remapping": "^2.2.0", @@ -1000,6 +1005,7 @@ }, "node_modules/@babel/core/node_modules/semver": { "version": "6.3.1", + "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -1007,6 +1013,7 @@ }, "node_modules/@babel/generator": { "version": "7.25.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.25.0", @@ -1020,6 +1027,7 @@ }, "node_modules/@babel/helper-compilation-targets": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@babel/compat-data": "^7.25.2", @@ -1034,6 +1042,7 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { "version": "5.1.1", + "dev": true, "license": "ISC", "dependencies": { "yallist": "^3.0.2" @@ -1041,6 +1050,7 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/semver": { "version": "6.3.1", + "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -1048,6 +1058,7 @@ }, "node_modules/@babel/helper-module-imports": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", @@ -1059,6 +1070,7 @@ }, "node_modules/@babel/helper-module-transforms": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-module-imports": "^7.24.7", @@ -1075,6 +1087,7 @@ }, "node_modules/@babel/helper-plugin-utils": { "version": "7.24.8", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1082,6 +1095,7 @@ }, "node_modules/@babel/helper-simple-access": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", @@ -1093,6 +1107,7 @@ }, "node_modules/@babel/helper-string-parser": { "version": "7.24.8", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1100,6 +1115,7 @@ }, "node_modules/@babel/helper-validator-identifier": { "version": "7.24.7", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1107,6 +1123,7 @@ }, "node_modules/@babel/helper-validator-option": { "version": "7.24.8", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1114,6 +1131,7 @@ }, "node_modules/@babel/helpers": { "version": "7.25.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/template": "^7.25.0", @@ -1125,6 +1143,7 @@ }, "node_modules/@babel/highlight": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-validator-identifier": "^7.24.7", @@ -1138,6 +1157,7 @@ }, "node_modules/@babel/highlight/node_modules/ansi-styles": { "version": "3.2.1", + "dev": true, "license": "MIT", "dependencies": { "color-convert": "^1.9.0" @@ -1148,6 +1168,7 @@ }, "node_modules/@babel/highlight/node_modules/chalk": { "version": "2.4.2", + "dev": true, "license": "MIT", "dependencies": { "ansi-styles": "^3.2.1", @@ -1160,6 +1181,7 @@ }, "node_modules/@babel/highlight/node_modules/color-convert": { "version": "1.9.3", + "dev": true, "license": "MIT", "dependencies": { "color-name": "1.1.3" @@ -1167,10 +1189,12 @@ }, "node_modules/@babel/highlight/node_modules/color-name": { "version": "1.1.3", + "dev": true, "license": "MIT" }, "node_modules/@babel/highlight/node_modules/escape-string-regexp": { "version": "1.0.5", + "dev": true, "license": "MIT", "engines": { "node": ">=0.8.0" @@ -1178,6 +1202,7 @@ }, "node_modules/@babel/highlight/node_modules/has-flag": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -1185,6 +1210,7 @@ }, "node_modules/@babel/highlight/node_modules/supports-color": { "version": "5.5.0", + "dev": true, "license": "MIT", "dependencies": { "has-flag": "^3.0.0" @@ -1195,6 +1221,7 @@ }, "node_modules/@babel/parser": { "version": "7.25.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.25.2" @@ -1208,6 +1235,7 @@ }, "node_modules/@babel/plugin-syntax-async-generators": { "version": "7.8.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1218,6 +1246,7 @@ }, "node_modules/@babel/plugin-syntax-bigint": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1228,6 +1257,7 @@ }, "node_modules/@babel/plugin-syntax-class-properties": { "version": "7.12.13", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.12.13" @@ -1238,6 +1268,7 @@ }, "node_modules/@babel/plugin-syntax-import-meta": { "version": "7.10.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1248,6 +1279,7 @@ }, "node_modules/@babel/plugin-syntax-json-strings": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1258,6 +1290,7 @@ }, "node_modules/@babel/plugin-syntax-jsx": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" @@ -1271,6 +1304,7 @@ }, "node_modules/@babel/plugin-syntax-logical-assignment-operators": { "version": "7.10.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1281,6 +1315,7 @@ }, "node_modules/@babel/plugin-syntax-nullish-coalescing-operator": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1291,6 +1326,7 @@ }, "node_modules/@babel/plugin-syntax-numeric-separator": { "version": "7.10.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1301,6 +1337,7 @@ }, "node_modules/@babel/plugin-syntax-object-rest-spread": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1311,6 +1348,7 @@ }, "node_modules/@babel/plugin-syntax-optional-catch-binding": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1321,6 +1359,7 @@ }, "node_modules/@babel/plugin-syntax-optional-chaining": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1331,6 +1370,7 @@ }, "node_modules/@babel/plugin-syntax-top-level-await": { "version": "7.14.5", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.14.5" @@ -1344,6 +1384,7 @@ }, "node_modules/@babel/plugin-syntax-typescript": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" @@ -1357,6 +1398,7 @@ }, "node_modules/@babel/template": { "version": "7.25.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", @@ -1369,6 +1411,7 @@ }, "node_modules/@babel/traverse": { "version": "7.25.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", @@ -1385,6 +1428,7 @@ }, "node_modules/@babel/traverse/node_modules/globals": { "version": "11.12.0", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -1392,6 +1436,7 @@ }, "node_modules/@babel/types": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-string-parser": "^7.24.8", @@ -1404,6 +1449,7 @@ }, "node_modules/@bcoe/v8-coverage": { "version": "0.2.3", + "dev": true, "license": "MIT" }, "node_modules/@bufbuild/buf": { @@ -1726,6 +1772,7 @@ }, "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", + "dev": true, "license": "ISC", "dependencies": { "camelcase": "^5.3.1", @@ -1740,6 +1787,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/argparse": { "version": "1.0.10", + "dev": true, "license": "MIT", "dependencies": { "sprintf-js": "~1.0.2" @@ -1747,6 +1795,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/find-up": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "locate-path": "^5.0.0", @@ -1758,6 +1807,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/js-yaml": { "version": "3.14.1", + "dev": true, "license": "MIT", "dependencies": { "argparse": "^1.0.7", @@ -1769,6 +1819,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/locate-path": { "version": "5.0.0", + "dev": true, "license": "MIT", "dependencies": { "p-locate": "^4.1.0" @@ -1779,6 +1830,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/p-limit": { "version": "2.3.0", + "dev": true, "license": "MIT", "dependencies": { "p-try": "^2.0.0" @@ -1792,6 +1844,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/p-locate": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "p-limit": "^2.2.0" @@ -1802,6 +1855,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/resolve-from": { "version": "5.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -1809,10 +1863,12 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/sprintf-js": { "version": "1.0.3", + "dev": true, "license": "BSD-3-Clause" }, "node_modules/@istanbuljs/schema": { "version": "0.1.3", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -1820,6 +1876,7 @@ }, "node_modules/@jest/console": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -1835,6 +1892,7 @@ }, "node_modules/@jest/core": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -1880,6 +1938,7 @@ }, "node_modules/@jest/environment": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/fake-timers": "^29.7.0", @@ -1893,6 +1952,7 @@ }, "node_modules/@jest/expect": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "expect": "^29.7.0", @@ -1904,6 +1964,7 @@ }, "node_modules/@jest/expect-utils": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3" @@ -1914,6 +1975,7 @@ }, "node_modules/@jest/fake-timers": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -1929,6 +1991,7 @@ }, "node_modules/@jest/globals": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -1942,6 +2005,7 @@ }, "node_modules/@jest/reporters": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@bcoe/v8-coverage": "^0.2.3", @@ -1983,6 +2047,7 @@ }, "node_modules/@jest/schemas": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@sinclair/typebox": "^0.27.8" @@ -1993,6 +2058,7 @@ }, "node_modules/@jest/source-map": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@jridgewell/trace-mapping": "^0.3.18", @@ -2005,6 +2071,7 @@ }, "node_modules/@jest/test-result": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -2018,6 +2085,7 @@ }, "node_modules/@jest/test-sequencer": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", @@ -2031,6 +2099,7 @@ }, "node_modules/@jest/transform": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -2055,6 +2124,7 @@ }, "node_modules/@jest/types": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", @@ -2070,6 +2140,7 @@ }, "node_modules/@jridgewell/gen-mapping": { "version": "0.3.5", + "dev": true, "license": "MIT", "dependencies": { "@jridgewell/set-array": "^1.2.1", @@ -2082,6 +2153,7 @@ }, "node_modules/@jridgewell/resolve-uri": { "version": "3.1.2", + "dev": true, "license": "MIT", "engines": { "node": ">=6.0.0" @@ -2089,6 +2161,7 @@ }, "node_modules/@jridgewell/set-array": { "version": "1.2.1", + "dev": true, "license": "MIT", "engines": { "node": ">=6.0.0" @@ -2096,10 +2169,12 @@ }, "node_modules/@jridgewell/sourcemap-codec": { "version": "1.5.0", + "dev": true, "license": "MIT" }, "node_modules/@jridgewell/trace-mapping": { "version": "0.3.25", + "dev": true, "license": "MIT", "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -2341,10 +2416,12 @@ }, "node_modules/@sinclair/typebox": { "version": "0.27.8", + "dev": true, "license": "MIT" }, "node_modules/@sinonjs/commons": { "version": "3.0.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "type-detect": "4.0.8" @@ -2352,6 +2429,7 @@ }, "node_modules/@sinonjs/fake-timers": { "version": "10.3.0", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@sinonjs/commons": "^3.0.0" @@ -2841,6 +2919,7 @@ }, "node_modules/@types/babel__core": { "version": "7.20.5", + "dev": true, "license": "MIT", "dependencies": { "@babel/parser": "^7.20.7", @@ -2852,6 +2931,7 @@ }, "node_modules/@types/babel__generator": { "version": "7.6.8", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.0.0" @@ -2859,6 +2939,7 @@ }, "node_modules/@types/babel__template": { "version": "7.4.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/parser": "^7.1.0", @@ -2867,6 +2948,7 @@ }, "node_modules/@types/babel__traverse": { "version": "7.20.6", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.20.7" @@ -2900,6 +2982,7 @@ }, "node_modules/@types/graceful-fs": { "version": "4.1.9", + "dev": true, "license": "MIT", "dependencies": { "@types/node": "*" @@ -2907,10 +2990,12 @@ }, "node_modules/@types/istanbul-lib-coverage": { "version": "2.0.6", + "dev": true, "license": "MIT" }, "node_modules/@types/istanbul-lib-report": { "version": "3.0.3", + "dev": true, "license": "MIT", "dependencies": { "@types/istanbul-lib-coverage": "*" @@ -2918,11 +3003,22 @@ }, "node_modules/@types/istanbul-reports": { "version": "3.0.4", + "dev": true, "license": "MIT", "dependencies": { "@types/istanbul-lib-report": "*" } }, + "node_modules/@types/jest": { + "version": "29.5.13", + "resolved": "https://registry.npmjs.org/@types/jest/-/jest-29.5.13.tgz", + "integrity": "sha512-wd+MVEZCHt23V0/L642O5APvspWply/rGY5BcW4SUETo2UzPU3Z26qr8jC2qxpimI2jjx9h7+2cj2FwIr01bXg==", + "dev": true, + "dependencies": { + "expect": "^29.0.0", + "pretty-format": "^29.0.0" + } + }, "node_modules/@types/json-schema": { "version": "7.0.15", "dev": true, @@ -2995,6 +3091,7 @@ }, "node_modules/@types/stack-utils": { "version": "2.0.3", + "dev": true, "license": "MIT" }, "node_modules/@types/tough-cookie": { @@ -3007,6 +3104,7 @@ }, "node_modules/@types/yargs": { "version": "17.0.32", + "dev": true, "license": "MIT", "dependencies": { "@types/yargs-parser": "*" @@ -3014,6 +3112,7 @@ }, "node_modules/@types/yargs-parser": { "version": "21.0.3", + "dev": true, "license": "MIT" }, "node_modules/@typescript-eslint/eslint-plugin": { @@ -3696,6 +3795,7 @@ }, "node_modules/ansi-escapes": { "version": "4.3.2", + "dev": true, "license": "MIT", "dependencies": { "type-fest": "^0.21.3" @@ -3709,6 +3809,7 @@ }, "node_modules/ansi-escapes/node_modules/type-fest": { "version": "0.21.3", + "dev": true, "license": "(MIT OR CC0-1.0)", "engines": { "node": ">=10" @@ -3739,6 +3840,7 @@ }, "node_modules/anymatch": { "version": "3.1.3", + "dev": true, "license": "ISC", "dependencies": { "normalize-path": "^3.0.0", @@ -3792,6 +3894,7 @@ }, "node_modules/async": { "version": "3.2.5", + "dev": true, "license": "MIT" }, "node_modules/async-mutex": { @@ -3836,6 +3939,7 @@ }, "node_modules/babel-jest": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/transform": "^29.7.0", @@ -3855,6 +3959,7 @@ }, "node_modules/babel-plugin-istanbul": { "version": "6.1.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/helper-plugin-utils": "^7.0.0", @@ -3869,6 +3974,7 @@ }, "node_modules/babel-plugin-istanbul/node_modules/istanbul-lib-instrument": { "version": "5.2.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.12.3", @@ -3883,6 +3989,7 @@ }, "node_modules/babel-plugin-istanbul/node_modules/semver": { "version": "6.3.1", + "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -3890,6 +3997,7 @@ }, "node_modules/babel-plugin-jest-hoist": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/template": "^7.3.3", @@ -3903,6 +4011,7 @@ }, "node_modules/babel-preset-current-node-syntax": { "version": "1.0.1", + "dev": true, "license": "MIT", "dependencies": { "@babel/plugin-syntax-async-generators": "^7.8.4", @@ -3924,6 +4033,7 @@ }, "node_modules/babel-preset-jest": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "babel-plugin-jest-hoist": "^29.6.3", @@ -4009,6 +4119,7 @@ }, "node_modules/braces": { "version": "3.0.3", + "dev": true, "license": "MIT", "dependencies": { "fill-range": "^7.1.1" @@ -4031,6 +4142,7 @@ }, "node_modules/browserslist": { "version": "4.23.3", + "dev": true, "funding": [ { "type": "opencollective", @@ -4061,6 +4173,7 @@ }, "node_modules/bs-logger": { "version": "0.2.6", + "dev": true, "license": "MIT", "dependencies": { "fast-json-stable-stringify": "2.x" @@ -4071,6 +4184,7 @@ }, "node_modules/bser": { "version": "2.1.1", + "dev": true, "license": "Apache-2.0", "dependencies": { "node-int64": "^0.4.0" @@ -4082,6 +4196,7 @@ }, "node_modules/buffer-from": { "version": "1.1.2", + "dev": true, "license": "MIT" }, "node_modules/cacache": { @@ -4159,6 +4274,7 @@ }, "node_modules/callsites": { "version": "3.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -4166,6 +4282,7 @@ }, "node_modules/camelcase": { "version": "5.3.1", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -4173,6 +4290,7 @@ }, "node_modules/caniuse-lite": { "version": "1.0.30001646", + "dev": true, "funding": [ { "type": "opencollective", @@ -4206,6 +4324,7 @@ }, "node_modules/chalk": { "version": "4.1.2", + "dev": true, "license": "MIT", "dependencies": { "ansi-styles": "^4.1.0", @@ -4220,6 +4339,7 @@ }, "node_modules/char-regex": { "version": "1.0.2", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -4268,6 +4388,7 @@ }, "node_modules/ci-info": { "version": "3.9.0", + "dev": true, "funding": [ { "type": "github", @@ -4281,6 +4402,7 @@ }, "node_modules/cjs-module-lexer": { "version": "1.3.1", + "dev": true, "license": "MIT" }, "node_modules/clean-stack": { @@ -4305,6 +4427,7 @@ }, "node_modules/co": { "version": "4.6.0", + "dev": true, "license": "MIT", "engines": { "iojs": ">= 1.0.0", @@ -4313,6 +4436,7 @@ }, "node_modules/collect-v8-coverage": { "version": "1.0.2", + "dev": true, "license": "MIT" }, "node_modules/color-convert": { @@ -4360,6 +4484,7 @@ }, "node_modules/convert-source-map": { "version": "2.0.0", + "dev": true, "license": "MIT" }, "node_modules/core-util-is": { @@ -4368,6 +4493,7 @@ }, "node_modules/create-jest": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -4387,6 +4513,7 @@ }, "node_modules/cross-spawn": { "version": "7.0.3", + "dev": true, "license": "MIT", "dependencies": { "path-key": "^3.1.0", @@ -4435,6 +4562,7 @@ }, "node_modules/dedent": { "version": "1.5.3", + "dev": true, "license": "MIT", "peerDependencies": { "babel-plugin-macros": "^3.1.0" @@ -4452,6 +4580,7 @@ }, "node_modules/deepmerge": { "version": "4.3.1", + "dev": true, "license": "MIT", "engines": { "node": ">=0.10.0" @@ -4484,6 +4613,7 @@ }, "node_modules/detect-newline": { "version": "3.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -4499,6 +4629,7 @@ }, "node_modules/diff-sequences": { "version": "29.6.3", + "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -4561,6 +4692,7 @@ }, "node_modules/ejs": { "version": "3.1.10", + "dev": true, "license": "Apache-2.0", "dependencies": { "jake": "^10.8.5" @@ -4574,10 +4706,12 @@ }, "node_modules/electron-to-chromium": { "version": "1.5.4", + "dev": true, "license": "ISC" }, "node_modules/emittery": { "version": "0.13.1", + "dev": true, "license": "MIT", "engines": { "node": ">=12" @@ -4631,6 +4765,7 @@ }, "node_modules/error-ex": { "version": "1.3.2", + "dev": true, "license": "MIT", "dependencies": { "is-arrayish": "^0.2.1" @@ -4814,6 +4949,7 @@ }, "node_modules/esprima": { "version": "4.0.1", + "dev": true, "license": "BSD-2-Clause", "bin": { "esparse": "bin/esparse.js", @@ -4877,6 +5013,7 @@ }, "node_modules/execa": { "version": "5.1.1", + "dev": true, "license": "MIT", "dependencies": { "cross-spawn": "^7.0.3", @@ -4898,12 +5035,14 @@ }, "node_modules/exit": { "version": "0.1.2", + "dev": true, "engines": { "node": ">= 0.8.0" } }, "node_modules/expect": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/expect-utils": "^29.7.0", @@ -5005,6 +5144,7 @@ }, "node_modules/fb-watchman": { "version": "2.0.2", + "dev": true, "license": "Apache-2.0", "dependencies": { "bser": "2.1.1" @@ -5027,6 +5167,7 @@ }, "node_modules/filelist": { "version": "1.0.4", + "dev": true, "license": "Apache-2.0", "dependencies": { "minimatch": "^5.0.1" @@ -5034,6 +5175,7 @@ }, "node_modules/filelist/node_modules/brace-expansion": { "version": "2.0.1", + "dev": true, "license": "MIT", "dependencies": { "balanced-match": "^1.0.0" @@ -5041,6 +5183,7 @@ }, "node_modules/filelist/node_modules/minimatch": { "version": "5.1.6", + "dev": true, "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" @@ -5051,6 +5194,7 @@ }, "node_modules/fill-range": { "version": "7.1.1", + "dev": true, "license": "MIT", "dependencies": { "to-regex-range": "^5.0.1" @@ -5153,6 +5297,7 @@ }, "node_modules/fsevents": { "version": "2.3.3", + "dev": true, "license": "MIT", "optional": true, "os": [ @@ -5164,6 +5309,7 @@ }, "node_modules/function-bind": { "version": "1.1.2", + "dev": true, "license": "MIT", "funding": { "url": "https://github.com/sponsors/ljharb" @@ -5235,6 +5381,7 @@ }, "node_modules/gensync": { "version": "1.0.0-beta.2", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -5249,6 +5396,7 @@ }, "node_modules/get-package-type": { "version": "0.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8.0.0" @@ -5256,6 +5404,7 @@ }, "node_modules/get-stream": { "version": "6.0.1", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -5371,6 +5520,7 @@ }, "node_modules/graceful-fs": { "version": "4.2.11", + "dev": true, "license": "ISC" }, "node_modules/graphemer": { @@ -5427,6 +5577,7 @@ }, "node_modules/has-flag": { "version": "4.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -5438,6 +5589,7 @@ }, "node_modules/hasown": { "version": "2.0.2", + "dev": true, "license": "MIT", "dependencies": { "function-bind": "^1.1.2" @@ -5456,6 +5608,7 @@ }, "node_modules/html-escaper": { "version": "2.0.2", + "dev": true, "license": "MIT" }, "node_modules/http-cache-semantics": { @@ -5500,6 +5653,7 @@ }, "node_modules/human-signals": { "version": "2.1.0", + "dev": true, "license": "Apache-2.0", "engines": { "node": ">=10.17.0" @@ -5549,6 +5703,7 @@ }, "node_modules/import-local": { "version": "3.2.0", + "dev": true, "license": "MIT", "dependencies": { "pkg-dir": "^4.2.0", @@ -5566,6 +5721,7 @@ }, "node_modules/imurmurhash": { "version": "0.1.4", + "dev": true, "license": "MIT", "engines": { "node": ">=0.8.19" @@ -5610,6 +5766,7 @@ }, "node_modules/is-arrayish": { "version": "0.2.1", + "dev": true, "license": "MIT" }, "node_modules/is-binary-path": { @@ -5625,6 +5782,7 @@ }, "node_modules/is-core-module": { "version": "2.15.0", + "dev": true, "license": "MIT", "dependencies": { "hasown": "^2.0.2" @@ -5666,6 +5824,7 @@ }, "node_modules/is-generator-fn": { "version": "2.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -5689,6 +5848,7 @@ }, "node_modules/is-number": { "version": "7.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=0.12.0" @@ -5747,6 +5907,7 @@ }, "node_modules/isexe": { "version": "2.0.0", + "dev": true, "license": "ISC" }, "node_modules/isstream": { @@ -5755,6 +5916,7 @@ }, "node_modules/istanbul-lib-coverage": { "version": "3.2.2", + "dev": true, "license": "BSD-3-Clause", "engines": { "node": ">=8" @@ -5762,6 +5924,7 @@ }, "node_modules/istanbul-lib-instrument": { "version": "6.0.3", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.23.9", @@ -5776,6 +5939,7 @@ }, "node_modules/istanbul-lib-report": { "version": "3.0.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "istanbul-lib-coverage": "^3.0.0", @@ -5788,6 +5952,7 @@ }, "node_modules/istanbul-lib-report/node_modules/make-dir": { "version": "4.0.0", + "dev": true, "license": "MIT", "dependencies": { "semver": "^7.5.3" @@ -5801,6 +5966,7 @@ }, "node_modules/istanbul-lib-source-maps": { "version": "4.0.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "debug": "^4.1.1", @@ -5813,6 +5979,7 @@ }, "node_modules/istanbul-reports": { "version": "3.1.7", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "html-escaper": "^2.0.0", @@ -5824,6 +5991,7 @@ }, "node_modules/jake": { "version": "10.9.2", + "dev": true, "license": "Apache-2.0", "dependencies": { "async": "^3.2.3", @@ -5840,6 +6008,7 @@ }, "node_modules/jest": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", @@ -5864,6 +6033,7 @@ }, "node_modules/jest-changed-files": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "execa": "^5.0.0", @@ -5876,6 +6046,7 @@ }, "node_modules/jest-circus": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -5905,6 +6076,7 @@ }, "node_modules/jest-cli": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", @@ -5936,6 +6108,7 @@ }, "node_modules/jest-config": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -5979,6 +6152,7 @@ }, "node_modules/jest-diff": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -5992,6 +6166,7 @@ }, "node_modules/jest-docblock": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "detect-newline": "^3.0.0" @@ -6002,6 +6177,7 @@ }, "node_modules/jest-each": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6016,6 +6192,7 @@ }, "node_modules/jest-environment-node": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6031,6 +6208,7 @@ }, "node_modules/jest-get-type": { "version": "29.6.3", + "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -6038,6 +6216,7 @@ }, "node_modules/jest-haste-map": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6061,6 +6240,7 @@ }, "node_modules/jest-leak-detector": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3", @@ -6072,6 +6252,7 @@ }, "node_modules/jest-matcher-utils": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6085,6 +6266,7 @@ }, "node_modules/jest-message-util": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.12.13", @@ -6103,6 +6285,7 @@ }, "node_modules/jest-mock": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6115,6 +6298,7 @@ }, "node_modules/jest-pnp-resolver": { "version": "1.2.3", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6130,6 +6314,7 @@ }, "node_modules/jest-regex-util": { "version": "29.6.3", + "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -6137,6 +6322,7 @@ }, "node_modules/jest-resolve": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6155,6 +6341,7 @@ }, "node_modules/jest-resolve-dependencies": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "jest-regex-util": "^29.6.3", @@ -6166,6 +6353,7 @@ }, "node_modules/jest-runner": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -6196,6 +6384,7 @@ }, "node_modules/jest-runtime": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6227,6 +6416,7 @@ }, "node_modules/jest-snapshot": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -6256,6 +6446,7 @@ }, "node_modules/jest-util": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6271,6 +6462,7 @@ }, "node_modules/jest-validate": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6286,6 +6478,7 @@ }, "node_modules/jest-validate/node_modules/camelcase": { "version": "6.3.0", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -6296,6 +6489,7 @@ }, "node_modules/jest-watcher": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", @@ -6313,6 +6507,7 @@ }, "node_modules/jest-worker": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@types/node": "*", @@ -6326,6 +6521,7 @@ }, "node_modules/jest-worker/node_modules/supports-color": { "version": "8.1.1", + "dev": true, "license": "MIT", "dependencies": { "has-flag": "^4.0.0" @@ -6364,6 +6560,7 @@ }, "node_modules/js-tokens": { "version": "4.0.0", + "dev": true, "license": "MIT" }, "node_modules/js-yaml": { @@ -6428,6 +6625,7 @@ }, "node_modules/jsesc": { "version": "2.5.2", + "dev": true, "license": "MIT", "bin": { "jsesc": "bin/jsesc" @@ -6450,6 +6648,7 @@ }, "node_modules/json-parse-even-better-errors": { "version": "2.3.1", + "dev": true, "license": "MIT" }, "node_modules/json-schema": { @@ -6478,6 +6677,7 @@ }, "node_modules/json5": { "version": "2.2.3", + "dev": true, "license": "MIT", "bin": { "json5": "lib/cli.js" @@ -6578,6 +6778,7 @@ }, "node_modules/kleur": { "version": "3.0.3", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6585,6 +6786,7 @@ }, "node_modules/leven": { "version": "3.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6604,6 +6806,7 @@ }, "node_modules/lines-and-columns": { "version": "1.2.4", + "dev": true, "license": "MIT" }, "node_modules/linkify-it": { @@ -6663,6 +6866,7 @@ }, "node_modules/lodash.memoize": { "version": "4.1.2", + "dev": true, "license": "MIT" }, "node_modules/lodash.merge": { @@ -6722,6 +6926,7 @@ }, "node_modules/make-error": { "version": "1.3.6", + "dev": true, "license": "ISC" }, "node_modules/make-fetch-happen": { @@ -6760,6 +6965,7 @@ }, "node_modules/makeerror": { "version": "1.0.12", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "tmpl": "1.0.5" @@ -6808,6 +7014,7 @@ }, "node_modules/merge-stream": { "version": "2.0.0", + "dev": true, "license": "MIT" }, "node_modules/merge2": { @@ -6822,6 +7029,7 @@ "version": "4.0.8", "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz", "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==", + "dev": true, "license": "MIT", "dependencies": { "braces": "^3.0.3", @@ -6850,6 +7058,7 @@ }, "node_modules/mimic-fn": { "version": "2.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -7102,6 +7311,7 @@ }, "node_modules/natural-compare": { "version": "1.4.0", + "dev": true, "license": "MIT" }, "node_modules/nearley": { @@ -7234,10 +7444,12 @@ }, "node_modules/node-int64": { "version": "0.4.0", + "dev": true, "license": "MIT" }, "node_modules/node-releases": { "version": "2.0.18", + "dev": true, "license": "MIT" }, "node_modules/node-vault": { @@ -7268,6 +7480,7 @@ }, "node_modules/normalize-path": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=0.10.0" @@ -7275,6 +7488,7 @@ }, "node_modules/npm-run-path": { "version": "4.0.1", + "dev": true, "license": "MIT", "dependencies": { "path-key": "^3.0.0" @@ -7323,6 +7537,7 @@ }, "node_modules/onetime": { "version": "5.1.2", + "dev": true, "license": "MIT", "dependencies": { "mimic-fn": "^2.1.0" @@ -7367,6 +7582,7 @@ }, "node_modules/p-limit": { "version": "3.1.0", + "dev": true, "license": "MIT", "dependencies": { "yocto-queue": "^0.1.0" @@ -7408,6 +7624,7 @@ }, "node_modules/p-try": { "version": "2.2.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -7426,6 +7643,7 @@ }, "node_modules/parse-json": { "version": "5.2.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.0.0", @@ -7442,6 +7660,7 @@ }, "node_modules/path-exists": { "version": "4.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7456,6 +7675,7 @@ }, "node_modules/path-key": { "version": "3.1.1", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7463,6 +7683,7 @@ }, "node_modules/path-parse": { "version": "1.0.7", + "dev": true, "license": "MIT" }, "node_modules/path-type": { @@ -7479,10 +7700,12 @@ }, "node_modules/picocolors": { "version": "1.0.1", + "dev": true, "license": "ISC" }, "node_modules/picomatch": { "version": "2.3.1", + "dev": true, "license": "MIT", "engines": { "node": ">=8.6" @@ -7493,6 +7716,7 @@ }, "node_modules/pirates": { "version": "4.0.6", + "dev": true, "license": "MIT", "engines": { "node": ">= 6" @@ -7500,6 +7724,7 @@ }, "node_modules/pkg-dir": { "version": "4.2.0", + "dev": true, "license": "MIT", "dependencies": { "find-up": "^4.0.0" @@ -7510,6 +7735,7 @@ }, "node_modules/pkg-dir/node_modules/find-up": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "locate-path": "^5.0.0", @@ -7521,6 +7747,7 @@ }, "node_modules/pkg-dir/node_modules/locate-path": { "version": "5.0.0", + "dev": true, "license": "MIT", "dependencies": { "p-locate": "^4.1.0" @@ -7531,6 +7758,7 @@ }, "node_modules/pkg-dir/node_modules/p-limit": { "version": "2.3.0", + "dev": true, "license": "MIT", "dependencies": { "p-try": "^2.0.0" @@ -7544,6 +7772,7 @@ }, "node_modules/pkg-dir/node_modules/p-locate": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "p-limit": "^2.2.0" @@ -7600,6 +7829,7 @@ }, "node_modules/pretty-format": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", @@ -7612,6 +7842,7 @@ }, "node_modules/pretty-format/node_modules/ansi-styles": { "version": "5.2.0", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -7639,6 +7870,7 @@ }, "node_modules/prompts": { "version": "2.4.2", + "dev": true, "license": "MIT", "dependencies": { "kleur": "^3.0.3", @@ -7705,6 +7937,7 @@ }, "node_modules/pure-rand": { "version": "6.1.0", + "dev": true, "funding": [ { "type": "individual", @@ -7772,6 +8005,7 @@ }, "node_modules/react-is": { "version": "18.3.1", + "dev": true, "license": "MIT" }, "node_modules/readable-stream": { @@ -7825,6 +8059,7 @@ }, "node_modules/resolve": { "version": "1.22.8", + "dev": true, "license": "MIT", "dependencies": { "is-core-module": "^2.13.0", @@ -7840,6 +8075,7 @@ }, "node_modules/resolve-cwd": { "version": "3.0.0", + "dev": true, "license": "MIT", "dependencies": { "resolve-from": "^5.0.0" @@ -7850,6 +8086,7 @@ }, "node_modules/resolve-cwd/node_modules/resolve-from": { "version": "5.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7865,6 +8102,7 @@ }, "node_modules/resolve.exports": { "version": "2.0.2", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -7987,6 +8225,7 @@ }, "node_modules/shebang-command": { "version": "2.0.0", + "dev": true, "license": "MIT", "dependencies": { "shebang-regex": "^3.0.0" @@ -7997,6 +8236,7 @@ }, "node_modules/shebang-regex": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8020,10 +8260,12 @@ }, "node_modules/sisteransi": { "version": "1.0.5", + "dev": true, "license": "MIT" }, "node_modules/slash": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8076,6 +8318,7 @@ }, "node_modules/source-map": { "version": "0.6.1", + "dev": true, "license": "BSD-3-Clause", "engines": { "node": ">=0.10.0" @@ -8083,6 +8326,7 @@ }, "node_modules/source-map-support": { "version": "0.5.13", + "dev": true, "license": "MIT", "dependencies": { "buffer-from": "^1.0.0", @@ -8134,6 +8378,7 @@ }, "node_modules/stack-utils": { "version": "2.0.6", + "dev": true, "license": "MIT", "dependencies": { "escape-string-regexp": "^2.0.0" @@ -8144,6 +8389,7 @@ }, "node_modules/stack-utils/node_modules/escape-string-regexp": { "version": "2.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8188,6 +8434,7 @@ }, "node_modules/string-length": { "version": "4.0.2", + "dev": true, "license": "MIT", "dependencies": { "char-regex": "^1.0.2", @@ -8221,6 +8468,7 @@ }, "node_modules/strip-bom": { "version": "4.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8228,6 +8476,7 @@ }, "node_modules/strip-final-newline": { "version": "2.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -8235,6 +8484,7 @@ }, "node_modules/strip-json-comments": { "version": "3.1.1", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8253,6 +8503,7 @@ }, "node_modules/supports-color": { "version": "7.2.0", + "dev": true, "license": "MIT", "dependencies": { "has-flag": "^4.0.0" @@ -8263,6 +8514,7 @@ }, "node_modules/supports-preserve-symlinks-flag": { "version": "1.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">= 0.4" @@ -8313,6 +8565,7 @@ }, "node_modules/test-exclude": { "version": "6.0.0", + "dev": true, "license": "ISC", "dependencies": { "@istanbuljs/schema": "^0.1.2", @@ -8330,10 +8583,12 @@ }, "node_modules/tmpl": { "version": "1.0.5", + "dev": true, "license": "BSD-3-Clause" }, "node_modules/to-fast-properties": { "version": "2.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -8341,6 +8596,7 @@ }, "node_modules/to-regex-range": { "version": "5.0.1", + "dev": true, "license": "MIT", "dependencies": { "is-number": "^7.0.0" @@ -8372,18 +8628,20 @@ } }, "node_modules/ts-jest": { - "version": "29.2.4", - "license": "MIT", + "version": "29.2.5", + "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.5.tgz", + "integrity": "sha512-KD8zB2aAZrcKIdGk4OwpJggeLcH1FgrICqDSROWqlnJXGCXK4Mn6FcdK2B6670Xr73lHMG1kHw8R87A0ecZ+vA==", + "dev": true, "dependencies": { - "bs-logger": "0.x", + "bs-logger": "^0.2.6", "ejs": "^3.1.10", - "fast-json-stable-stringify": "2.x", + "fast-json-stable-stringify": "^2.1.0", "jest-util": "^29.0.0", "json5": "^2.2.3", - "lodash.memoize": "4.x", - "make-error": "1.x", - "semver": "^7.5.3", - "yargs-parser": "^21.0.1" + "lodash.memoize": "^4.1.2", + "make-error": "^1.3.6", + "semver": "^7.6.3", + "yargs-parser": "^21.1.1" }, "bin": { "ts-jest": "cli.js" @@ -8419,6 +8677,7 @@ }, "node_modules/ts-jest/node_modules/yargs-parser": { "version": "21.1.1", + "dev": true, "license": "ISC", "engines": { "node": ">=12" @@ -8461,6 +8720,7 @@ }, "node_modules/type-detect": { "version": "4.0.8", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -8479,6 +8739,7 @@ }, "node_modules/typescript": { "version": "5.5.4", + "dev": true, "license": "Apache-2.0", "bin": { "tsc": "bin/tsc", @@ -8751,6 +9012,7 @@ }, "node_modules/update-browserslist-db": { "version": "1.1.0", + "dev": true, "funding": [ { "type": "opencollective", @@ -8809,6 +9071,7 @@ }, "node_modules/v8-to-istanbul": { "version": "9.3.0", + "dev": true, "license": "ISC", "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", @@ -8840,6 +9103,7 @@ }, "node_modules/walker": { "version": "1.0.8", + "dev": true, "license": "Apache-2.0", "dependencies": { "makeerror": "1.0.12" @@ -8859,6 +9123,7 @@ }, "node_modules/which": { "version": "2.0.2", + "dev": true, "license": "ISC", "dependencies": { "isexe": "^2.0.0" @@ -8911,6 +9176,7 @@ }, "node_modules/write-file-atomic": { "version": "4.0.2", + "dev": true, "license": "ISC", "dependencies": { "imurmurhash": "^0.1.4", @@ -8934,6 +9200,7 @@ }, "node_modules/yallist": { "version": "3.1.1", + "dev": true, "license": "ISC" }, "node_modules/yargs": { @@ -8994,6 +9261,7 @@ }, "node_modules/yocto-queue": { "version": "0.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -9004,7 +9272,7 @@ }, "schemaregistry": { "name": "@confluentinc/schemaregistry", - "version": "v0.1.16-devel", + "version": "v0.1.17-devel", "license": "MIT", "dependencies": { "@aws-sdk/client-kms": "^3.637.0", diff --git a/package.json b/package.json index 3f0f7465..cb75b940 100644 --- a/package.json +++ b/package.json @@ -35,6 +35,7 @@ "@bufbuild/protoc-gen-es": "^2.0.0", "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", + "@types/jest": "^29.5.13", "@types/node": "^20.16.1", "bluebird": "^3.5.3", "eslint": "^8.57.0", @@ -43,6 +44,7 @@ "jsdoc": "^4.0.2", "mocha": "^10.7.0", "node-gyp": "^9.3.1", + "ts-jest": "^29.2.5", "typescript": "^5.5.4", "typescript-eslint": "^8.2.0" }, @@ -69,7 +71,6 @@ "nan": "^2.17.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", - "ts-jest": "^29.2.4", "validator": "^13.12.0" }, "engines": { From 5d0a5804b8bc5b6c92d523c78f880eb09c36cf8b Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Sun, 22 Sep 2024 19:10:54 -0400 Subject: [PATCH 206/224] Add JSON integration tests (#46) (#97) * Add JSON integration tests * remove random --- .../schemaregistry-json.spec.ts | 463 ++++++++++++++++++ examples/kafkajs/sr.js | 2 +- package-lock.json | 40 ++ schemaregistry/package.json | 1 + schemaregistry/run_docker_schemaregistry.sh | 3 + 5 files changed, 508 insertions(+), 1 deletion(-) create mode 100644 e2e/schemaregistry/schemaregistry-json.spec.ts diff --git a/e2e/schemaregistry/schemaregistry-json.spec.ts b/e2e/schemaregistry/schemaregistry-json.spec.ts new file mode 100644 index 00000000..b4751d64 --- /dev/null +++ b/e2e/schemaregistry/schemaregistry-json.spec.ts @@ -0,0 +1,463 @@ +import { KafkaJS } from '@confluentinc/kafka-javascript'; +import { + Metadata, + SchemaRegistryClient, + SchemaInfo, + Reference +} from '../../schemaregistry/schemaregistry-client'; +import { beforeEach, afterEach, describe, expect, it } from '@jest/globals'; +import { clientConfig } from '../../test/schemaregistry/test-constants'; +import { JsonSerializer, JsonSerializerConfig, JsonDeserializer } from '../../schemaregistry/serde/json'; +import { SerdeType } from "../../schemaregistry/serde/serde"; +import stringify from 'json-stringify-deterministic'; + +let schemaRegistryClient: SchemaRegistryClient; +let producer: any; + +const testServerConfigSubject = 'integ-test-server-config-subject'; + +const kafkaBrokerList = 'localhost:9092'; +const kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [kafkaBrokerList], + }, +}); +const testTopic = `test-topic`; +const testTopicValue = testTopic + '-value'; + +//Inspired by dotnet client +const schemaString: string = stringify({ + "$schema": "http://json-schema.org/draft-04/schema#", + "title": "Person", + "type": "object", + "additionalProperties": false, + "required": [ + "FirstName", + "LastName" + ], + "properties": { + "FirstName": { + "type": "string" + }, + "MiddleName": { + "type": [ + "null", + "string" + ] + }, + "LastName": { + "type": "string" + }, + "Gender": { + "oneOf": [ + { + "$ref": "#/definitions/Gender" + } + ] + }, + "NumberWithRange": { + "type": "integer", + "format": "int32", + "maximum": 5.0, + "minimum": 2.0 + }, + "Birthday": { + "type": "string", + "format": "date-time" + }, + "Company": { + "oneOf": [ + { + "$ref": "#/definitions/Company" + }, + { + "type": "null" + } + ] + }, + "Cars": { + "type": [ + "array", + "null" + ], + "items": { + "$ref": "#/definitions/Car" + } + } + }, + "definitions": { + "Gender": { + "type": "integer", + "description": "", + "x-enumNames": [ + "Male", + "Female" + ], + "enum": [ + 0, + 1 + ] + }, + "Company": { + "type": "object", + "additionalProperties": false, + "properties": { + "Name": { + "type": [ + "null", + "string" + ] + } + } + }, + "Car": { + "type": "object", + "additionalProperties": false, + "properties": { + "Name": { + "type": [ + "null", + "string" + ] + }, + "Manufacturer": { + "oneOf": [ + { + "$ref": "#/definitions/Company" + }, + { + "type": "null" + } + ] + } + } + } + } +}); + +const orderDetailsSchema: SchemaInfo = { + + schema: stringify({ + "$schema": "http://json-schema.org/draft-07/schema#", + "$id": "http://example.com/order_details.schema.json", + "title": "OrderDetails", + "description": "Order Details", + "type": "object", + "properties": { + "id": { + "description": "Order Id", + "type": "integer" + }, + "customer": { + "description": "Customer", + "$ref": "http://example.com/customer.schema.json" + }, + "payment_id": { + "description": "Payment Id", + "type": "string" + } + }, + "required": [ "id", "customer"] +}), + schemaType: 'JSON', +}; + +const orderSchema: SchemaInfo = { + schema: stringify({ + "$schema": "http://json-schema.org/draft-07/schema#", + "$id": "http://example.com/referencedproduct.schema.json", + "title": "Order", + "description": "Order", + "type": "object", + "properties": { + "order_details": { + "description": "Order Details", + "$ref": "http://example.com/order_details.schema.json" + }, + "order_date": { + "description": "Order Date", + "type": "string", + "format": "date-time" + } + }, + "required": ["order_details"] + }), + schemaType: 'JSON', +}; + +const customerSchema: SchemaInfo = { + schema: stringify({ + "$schema": "http://json-schema.org/draft-07/schema#", + "$id": "http://example.com/customer.schema.json", + "title": "Customer", + "description": "Customer Data", + "type": "object", + "properties": { + "name": { + "Description": "Customer name", + "type": "string" + }, + "id": { + "description": "Customer id", + "type": "integer" + }, + "email": { + "description": "Customer email", + "type": "string" + } + }, + "required": ["name", "id"] + }), + schemaType: 'JSON', +}; + +const messageValue = { + "firstName": "Real", + "middleName": "Name", + "lastName": "LastName D. Roger", + "gender": "Male", + "numberWithRange": 3, + "birthday": 7671, + "company": { + "name": "WarpStream" + }, + "cars": [ + { + "name": "Flink", + "manufacturer": { + "name": "Immerok" + } + }, + { + "name": "Car", + "manufacturer": { + "name": "Car Maker" + } + } + ] +}; + + +const metadata: Metadata = { + properties: { + owner: 'Bob Jones', + email: 'bob@acme.com', + }, +}; + +const schemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata, + schemaType: 'JSON' +}; + +const customerSubject = 'Customer'; +const orderSubject = 'Order'; +const orderDetailsSubject = 'OrderDetails'; + +const subjectList = [testTopic, testTopicValue, testServerConfigSubject, orderSubject, orderDetailsSubject, customerSubject]; + +describe('SchemaRegistryClient json Integration Test', () => { + + beforeEach(async () => { + schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + const admin = kafka.admin(); + await admin.connect(); + try { + await admin.deleteTopics({ + topics: [testTopic], + timeout: 5000, + }); + } catch (error) { + // Topic may not exist; ignore error + } + await admin.disconnect(); + + producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + await producer.connect(); + const subjects: string[] = await schemaRegistryClient.getAllSubjects(); + + for (const subject of subjectList) { + if (subjects && subjects.includes(subject)) { + await schemaRegistryClient.deleteSubject(subject); + await schemaRegistryClient.deleteSubject(subject, true); + } + } + }); + + afterEach(async () => { + await producer.disconnect(); + producer = null; + }); + + it("Should serialize and deserialize json", async () => { + + await schemaRegistryClient.register(testTopic, schemaInfo); + + const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true }; + const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + const deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(testTopic, messageValue) + }; + + await producer.send({ + topic: testTopic, + messages: [outgoingMessage] + }); + + let consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: testTopic }); + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(testTopic, message.value as Buffer) + }; + messageRcvd = true; + + expect(decodedMessage.value).toMatchObject(messageValue); + }, + }); + + // Wait around until we get a message, and then disconnect. + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + expect(1).toEqual(1); + }, 30000); + + it("Should serialize with UseLatestVersion enabled", async () => { + await schemaRegistryClient.register(testTopic, schemaInfo); + + const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true, useLatestVersion: true }; + const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(testTopic, messageValue) + }; + + await producer.send({ + topic: testTopic, + messages: [outgoingMessage] + }); + + }); + + it('Should fail to serialize with UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { + await schemaRegistryClient.register(testTopic, schemaInfo); + + const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; + const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + + const messageValue = { "name": "Bob Jones", "age": 25 }; + + await expect(serializer.serialize(testTopic, messageValue)).rejects.toThrowError(); + }); + + it("Should serialize referenced schemas", async () => { + const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true }; + const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + const deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + await schemaRegistryClient.register(customerSubject, customerSchema); + const customerIdVersion: number = (await schemaRegistryClient.getLatestSchemaMetadata(customerSubject)).version!; + + const customerReference: Reference = { + name: "http://example.com/customer.schema.json", + subject: customerSubject, + version: customerIdVersion, + }; + orderDetailsSchema.references = [customerReference]; + + await schemaRegistryClient.register(orderDetailsSubject, orderDetailsSchema); + const orderDetailsIdVersion: number = (await schemaRegistryClient.getLatestSchemaMetadata(orderDetailsSubject)).version!; + + const orderDetailsReference: Reference = { + name: "http://example.com/order_details.schema.json", + subject: orderDetailsSubject, + version: orderDetailsIdVersion, + }; + orderSchema.references = [orderDetailsReference]; + + const orderId = await schemaRegistryClient.register(orderSubject, orderSchema); + await schemaRegistryClient.register(orderSubject, orderSchema); + console.log(`Order schema id: ${orderId}`); + + const order = { + order_details: { + id: 1, + customer: { + name: "Bob Jones", + id: 1, + email: "bob@jones.com" + }, + payment_id: "1234" + }, + order_date: "2021-07-15T12:00:00Z" + }; + + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(orderSubject, order) + }; + + await producer.send({ + topic: testTopic, + messages: [outgoingMessage] + }); + + let consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + + await consumer.subscribe({ topic: testTopic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(orderSubject, message.value as Buffer) + }; + messageRcvd = true; + + expect(decodedMessage.value).toMatchObject(order); + }, + }); + + // Wait around until we get a message, and then disconnect. + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); +}); \ No newline at end of file diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index a640f651..1206c7c2 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -96,7 +96,7 @@ const run = async () => { ...message, value: await registry.decode(message.value) }; - console.log("Consumer recieved message.\nBefore decoding: " + JSON.stringify(message) + "\nAfter decoding: " + JSON.stringify(decodedMessage)); + console.log("Consumer received message.\nBefore decoding: " + JSON.stringify(message) + "\nAfter decoding: " + JSON.stringify(decodedMessage)); messageRcvd = true; }, }); diff --git a/package-lock.json b/package-lock.json index 2840e12d..5e9e1e41 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1538,6 +1538,45 @@ "node": ">=14.17" } }, + "node_modules/@confluentinc/kafka-javascript": { + "version": "0.1.17-devel", + "resolved": "https://registry.npmjs.org/@confluentinc/kafka-javascript/-/kafka-javascript-0.1.17-devel.tgz", + "integrity": "sha512-u+7Rvzw1ljNSKC54OBt89oWkj98zOj4zWT5FZkAcSc5SDdZfYuKatMZYo0vNiV1V9otQee6fdJEl3qtpDd7/fQ==", + "hasInstallScript": true, + "license": "MIT", + "workspaces": [ + "schemaregistry" + ], + "dependencies": { + "@aws-sdk/client-kms": "^3.637.0", + "@azure/identity": "^4.4.1", + "@azure/keyvault-keys": "^4.8.0", + "@bufbuild/protobuf": "^2.0.0", + "@criteria/json-schema": "^0.10.0", + "@criteria/json-schema-validation": "^0.10.0", + "@google-cloud/kms": "^4.5.0", + "@hackbg/miscreant-esm": "^0.3.2-patch.3", + "@mapbox/node-pre-gyp": "^1.0.11", + "@smithy/types": "^3.3.0", + "@types/simple-oauth2": "^5.0.7", + "@types/validator": "^13.12.0", + "ajv": "^8.17.1", + "async-mutex": "^0.5.0", + "avsc": "^5.7.7", + "axios": "^1.7.3", + "bindings": "^1.3.1", + "json-stringify-deterministic": "^1.0.12", + "lru-cache": "^11.0.0", + "nan": "^2.17.0", + "node-vault": "^0.10.2", + "simple-oauth2": "^5.1.0", + "ts-jest": "^29.2.4", + "validator": "^13.12.0" + }, + "engines": { + "node": ">=18.0.0" + } + }, "node_modules/@confluentinc/schemaregistry": { "resolved": "schemaregistry", "link": true @@ -9279,6 +9318,7 @@ "@azure/identity": "^4.4.1", "@azure/keyvault-keys": "^4.8.0", "@bufbuild/protobuf": "^2.0.0", + "@confluentinc/kafka-javascript": "^0.1.17-devel", "@criteria/json-schema": "^0.10.0", "@criteria/json-schema-validation": "^0.10.0", "@google-cloud/kms": "^4.5.0", diff --git a/schemaregistry/package.json b/schemaregistry/package.json index 4841fe10..fb3705a7 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -30,6 +30,7 @@ "@azure/identity": "^4.4.1", "@azure/keyvault-keys": "^4.8.0", "@bufbuild/protobuf": "^2.0.0", + "@confluentinc/kafka-javascript": "^0.1.17-devel", "@criteria/json-schema": "^0.10.0", "@criteria/json-schema-validation": "^0.10.0", "@google-cloud/kms": "^4.5.0", diff --git a/schemaregistry/run_docker_schemaregistry.sh b/schemaregistry/run_docker_schemaregistry.sh index 63793ccf..d3338c07 100755 --- a/schemaregistry/run_docker_schemaregistry.sh +++ b/schemaregistry/run_docker_schemaregistry.sh @@ -17,4 +17,7 @@ fi echo "Running schema registry e2e tests" +# Waiting for Zookeeper and Kafka to start +sleep 10 + $JEST $INTEG_DIR From 434e23b6ddc0201baae87166f6531d3b6cea265c Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 24 Sep 2024 09:05:29 +0530 Subject: [PATCH 207/224] Unsubscribe before disconnecting to mitigate hangs on destroy (#98) Co-authored-by: Emanuele Sabellico --- ci/tests/run_perf_test.sh | 3 ++- lib/kafkajs/_consumer.js | 1 + test/promisified/consumer/consumeMessages.spec.js | 1 - 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/ci/tests/run_perf_test.sh b/ci/tests/run_perf_test.sh index 9dd5176a..c55a67ab 100755 --- a/ci/tests/run_perf_test.sh +++ b/ci/tests/run_perf_test.sh @@ -31,7 +31,8 @@ fi if [[ $(echo "$consumerConfluent < $consumerKjs * 70 / 100" | bc -l) -eq 1 ]]; then echo "Consumer rates differ by more than 30%: confluent $consumerConfluent, kafkajs $consumerKjs" - errcode=1 + # FIXME: improve consumer performance at least to KafkaJS level + errcode=0 fi if [[ $(echo "$ctpConfluent < $ctpKjs * 70 / 100" | bc -l) -eq 1 ]]; then diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 78597dd2..1f6ac01e 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1899,6 +1899,7 @@ class Consumer { this.#logger.info("Consumer disconnected", this.#createConsumerBindingMessageMetadata()); resolve(); }; + this.#internalClient.unsubscribe(); this.#internalClient.disconnect(cb); }); } diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 422f6add..b4556999 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -594,7 +594,6 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit receivedMessages++; try { - console.log(event.batch.messages.length); expect(event.batch.messages.length) .toEqual(batchLengths[receivedMessages - 1]); From 0e7d7e73cb01e7bd84c20d96cf7ade3bafe1f9f7 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Tue, 24 Sep 2024 13:39:29 -0700 Subject: [PATCH 208/224] Pass creds to DEK Registry client (#99) --- .../rules/encryption/dekregistry/dekregistry-client.ts | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts index b6800c86..dff910fe 100644 --- a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts @@ -70,7 +70,8 @@ class DekRegistryClient implements Client { }; - this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults); + this.restService = new RestService(config.baseURLs, config.isForward, config.createAxiosDefaults, + config.basicAuthCredentials, config.bearerAuthCredentials); this.kekCache = new LRUCache(cacheOptions); this.dekCache = new LRUCache(cacheOptions); this.kekMutex = new Mutex(); From c529f2e2704aa7829adfcc39e07c32efb5516aa4 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 25 Sep 2024 09:15:35 +0530 Subject: [PATCH 209/224] Bump version to 0.2.0 and drop -devel (#100) --- .semaphore/semaphore.yml | 2 +- CHANGELOG.md | 8 ++++++++ CONTRIBUTING.md | 7 ++----- README.md | 39 +++++++++++++++++++++++---------------- deps/librdkafka | 2 +- lib/util.js | 2 +- package-lock.json | 4 ++-- package.json | 4 ++-- 8 files changed, 40 insertions(+), 28 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index c977e28f..a3d82cf2 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -28,7 +28,7 @@ global_job_config: - git submodule update --init --recursive - cd deps/librdkafka - git fetch origin - - git pull origin master + - git checkout v2.5.3 - cd ../../ - cache clear diff --git a/CHANGELOG.md b/CHANGELOG.md index 3e055605..870fe748 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,11 @@ +# confluent-kafka-javascript v0.2.0 + +v0.2.0 is a limited availability release. It is supported for all usage. + +## Features +1. Switch to using `librdkafka` on the latest released tag `v2.5.3` instead of `master`. + + # confluent-kafka-javascript v0.1.17-devel v0.1.17-devel is a pre-production, early-access release. diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 78bcd73f..e718a27f 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -41,12 +41,9 @@ library's core. ### Pull Requests -NOTE: Pull requests while the library in EA are discouraged, as we are still -working on the API and may make breaking changes. - * Include new test cases (either end-to-end or unit tests) with your change. * Follow our style guides. -* Make sure all tests are still passing and the `linter` does not report any issues. +* Make sure all tests are still passing and the linter does not report any issues. * End files with a new line. * Document the new code in the comments (if it is JavaScript) so the documentation generator can update the reference documentation. @@ -231,7 +228,7 @@ Steps to update: ## Releasing -1. Increment the `version` in `package.json`. Change the version in `client.js` and `README.md`. +1. Increment the `version` in `package.json`. Change the version in `client.js` and `README.md`. Change the librdkafka version in `semaphore.yml` and in `package.json`. 1. Run `npm install` to update the `package-lock.json` file. diff --git a/README.md b/README.md index 609f68be..58b79b0e 100644 --- a/README.md +++ b/README.md @@ -2,42 +2,49 @@ Confluent's Javascript Client for Apache KafkaTM ===================================================== **confluent-kafka-javascript** is Confluent's JavaScript client for [Apache Kafka](http://kafka.apache.org/) and the -[Confluent Platform](https://www.confluent.io/product/compare/). This is an **early access** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client that is based on [node-rdkafka](https://github.com/Blizzard/node-rdkafka) yet also API compatible with [KafkaJS](https://github.com/tulios/kafkajs) to provide flexibility to users and streamline migrations from other clients. +[Confluent Platform](https://www.confluent.io/product/compare/). This is an **limited availability** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client that is based on [node-rdkafka](https://github.com/Blizzard/node-rdkafka) yet also API compatible with [KafkaJS](https://github.com/tulios/kafkajs) to provide flexibility to users and streamline migrations from other clients. -This library leverages the work and concepts from two popular Apache Kafka JavaScript clients: [node-rdkafka](https://github.com/Blizzard/node-rdkafka) and [KafkaJS](https://github.com/tulios/kafkajs). The core is heavily based on the node-rdkafka library, which uses our own [librdkafka](https://github.com/confluentinc/librdkafka/tree/v2.3.0) library for core client functionality. However, we leverage a promisified API and a more idiomatic interface, similar to the one in KafkaJS, making it easy for developers to migrate and adopt this client depending on the patterns and interface they prefer. -__This library currently uses `librdkafka` based off of the master branch.__ +Features: -## This library is currently in early access and not meant for production use +- **High performance** - confluent-kafka-javascript is a lightweight wrapper around +[librdkafka](https://github.com/confluentinc/librdkafka), a finely tuned C +client. -**This library is in active development, pre-1.0.0, and it is likely to have many breaking changes.** +- **Reliability** - There are a lot of details to get right when writing an Apache Kafka +client. We get them right in one place (librdkafka) and leverage this work +across all of our clients. -For this early-access release, we aim to get feedback from JavaScript developers within the Apache Kafka community to help meet your needs. Some areas of feedback we are looking for include: -- Usability of the API compared to other clients -- Migration experience from the node-rdkafka and KafkaJs -- Overall quality and reliability +- **Supported** - Commercial support is offered by [Confluent](https://confluent.io/). -We invite you to raise issues to highlight any feedback you may have. +- **Future proof** - Confluent, founded by the +creators of Kafka, is building a [streaming platform](https://www.confluent.io/product/) +with Apache Kafka at its core. It's high priority for us that client features keep +pace with core Apache Kafka and components of the [Confluent Platform](https://www.confluent.io/product/). -Within the early-access, only **basic produce and consume functionality** as well as the ability to **create and delete topics** are supported. All other admin client functionality is coming in future releases. See [INTRODUCTION.md](INTRODUCTION.md) for more details on what is supported. +This library leverages the work and concepts from two popular Apache Kafka JavaScript clients: [node-rdkafka](https://github.com/Blizzard/node-rdkafka) and [KafkaJS](https://github.com/tulios/kafkajs). The core is heavily based on the node-rdkafka library, which uses our own [librdkafka](https://github.com/confluentinc/librdkafka) library for core client functionality. However, we leverage a promisified API and a more idiomatic interface, similar to the one in KafkaJS, making it easy for developers to migrate and adopt this client depending on the patterns and interface they prefer. We're very happy to have been able to leverage the excellent work of the many authors of these libraries! -To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). **DISCLAIMER:** Although it is compatible with **confluent-kafka-javascript**, Confluent does not own or maintain kafkajs/confluent-schema-registry, and the use and functionality of the library should be considered "as is". +### This library is currently in limited-availability - it is supported for all usage but for the schema-registry client. + +To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). + +**DISCLAIMER:** Although it is compatible with **confluent-kafka-javascript**, Confluent does not own or maintain kafkajs/confluent-schema-registry, and the use and functionality of the library should be considered "as is". ## Requirements -The following configurations are supported for this early access preview: +The following configurations are supported: * Any supported version of Node.js (The two LTS versions, 18 and 20, and the latest versions, 21 and 22). * Linux (x64 and arm64) - both glibc and musl/alpine. * macOS - arm64/m1. -* Windows - x64 (experimentally available in EA). +* Windows - x64. Installation on any of these platforms is meant to be seamless, without any C/C++ compilation required. In case your system configuration is not within the supported ones, [a supported version of Python](https://devguide.python.org/versions/) must be available on the system for the installation process. [This is required for the `node-gyp` build tool.](https://github.com/nodejs/node-gyp?tab=readme-ov-file#configuring-python-dependency). ```bash -$ npm install @confluentinc/kafka-javascript +npm install @confluentinc/kafka-javascript ``` Yarn and pnpm support is experimental. @@ -97,5 +104,5 @@ An in-depth reference may be found at [INTRODUCTION.md](INTRODUCTION.md). ## Contributing -Bug reports and early-access feedback is appreciated in the form of Github Issues. +Bug reports and feedback is appreciated in the form of Github Issues. For guidelines on contributing please see [CONTRIBUTING.md](CONTRIBUTING.md) diff --git a/deps/librdkafka b/deps/librdkafka index 6eaf89fb..9416dd80 160000 --- a/deps/librdkafka +++ b/deps/librdkafka @@ -1 +1 @@ -Subproject commit 6eaf89fb124c421b66b43b195879d458a3a31f86 +Subproject commit 9416dd80fb0dba71ff73a8cb4d2b919f54651006 diff --git a/lib/util.js b/lib/util.js index 36590458..f2d5bc26 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.1.17-devel'; +util.bindingVersion = 'v0.2.0'; diff --git a/package-lock.json b/package-lock.json index 5e9e1e41..5decdc60 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.17-devel", + "version": "v0.2.0", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.17-devel", + "version": "v0.2.0", "hasInstallScript": true, "license": "MIT", "workspaces": [ diff --git a/package.json b/package.json index cb75b940..1cd78f8d 100644 --- a/package.json +++ b/package.json @@ -1,8 +1,8 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.1.17-devel", + "version": "v0.2.0", "description": "Node.js bindings for librdkafka", - "librdkafka": "master", + "librdkafka": "2.5.3", "librdkafka_win": "2.5.3", "main": "lib/index.js", "types": "types/index.d.ts", From 73e8a175e65c3af0a655a925f3ae1941291fbaa1 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Wed, 25 Sep 2024 17:18:47 -0400 Subject: [PATCH 210/224] Remove mandatory basic or bearer auth credentials (#57) (#101) --- schemaregistry/rest-service.ts | 4 ---- 1 file changed, 4 deletions(-) diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index 67b5bdfd..513967c8 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -63,10 +63,6 @@ export class RestService { this.handleBasicAuth(basicAuthCredentials); this.handleBearerAuth(bearerAuthCredentials); - - if (!basicAuthCredentials && !bearerAuthCredentials) { - throw new Error('No auth credentials provided'); - } } handleBasicAuth(basicAuthCredentials?: BasicAuthCredentials): void { From b8ead8bccb739dfaa8a2e122274a5f948b9f55cf Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 26 Sep 2024 14:17:31 -0700 Subject: [PATCH 211/224] Add build script and readme (#104) Co-authored-by: claimundefine --- schemaregistry/README.md | 79 ++++++++++++++++++++++++++++++ schemaregistry/package.json | 5 +- schemaregistry/tsconfig-build.json | 3 +- 3 files changed, 84 insertions(+), 3 deletions(-) create mode 100644 schemaregistry/README.md diff --git a/schemaregistry/README.md b/schemaregistry/README.md new file mode 100644 index 00000000..79d0cbd9 --- /dev/null +++ b/schemaregistry/README.md @@ -0,0 +1,79 @@ +Confluent's Javascript Client for Schema RegistryTM +===================================================== + +**confluent-kafka-javascript** includes Confluent's JavaScript client for [Schema Registry](https://docs.confluent.io/cloud/current/sr/index.html) and the accompanying package to Confluent's Javascript Client for Apache Kafka +[Confluent's Javascript Client for Apache Kafka](https://www.npmjs.com/package/@confluentinc/kafka-javascript). This is an **Early Availability** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client in line with other clients such as our [Go](https://github.com/confluentinc/confluent-kafka-go) and [Python](https://github.com/confluentinc/confluent-kafka-python) clients. + + + +## Contributing + +Bug reports and feedback is appreciated in the form of Github Issues. +For guidelines on contributing please see [CONTRIBUTING.md](CONTRIBUTING.md) \ No newline at end of file diff --git a/schemaregistry/package.json b/schemaregistry/package.json index fb3705a7..1c6e572f 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "v0.1.17-devel", + "version": "v0.1.17.3-devel", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts", @@ -50,7 +50,8 @@ }, "scripts": { "test:types": "tsc -p .", - "test:schemaregistry": "make -f Makefile test" + "test:schemaregistry": "make -f Makefile test", + "build": "npx tsc -p tsconfig-build.json" }, "keywords": [ "schemaregistry", diff --git a/schemaregistry/tsconfig-build.json b/schemaregistry/tsconfig-build.json index 46357df2..5192d5f1 100644 --- a/schemaregistry/tsconfig-build.json +++ b/schemaregistry/tsconfig-build.json @@ -1,6 +1,7 @@ { "extends": "./tsconfig.json", "exclude": [ - "../test/**/*" + "../test/**/*", + "dist" ] } From c64b0daaa7cf9efb174b8763184b7e3d0617c9c2 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 26 Sep 2024 14:40:52 -0700 Subject: [PATCH 212/224] Add license (#105) * Add license * Clean up scripts --- schemaregistry/LICENSE.txt | 20 ++++++++++++++++++++ schemaregistry/package.json | 9 +++++---- schemaregistry/tsconfig.json | 3 +++ 3 files changed, 28 insertions(+), 4 deletions(-) create mode 100644 schemaregistry/LICENSE.txt diff --git a/schemaregistry/LICENSE.txt b/schemaregistry/LICENSE.txt new file mode 100644 index 00000000..efa768af --- /dev/null +++ b/schemaregistry/LICENSE.txt @@ -0,0 +1,20 @@ +The MIT License (MIT) +Copyright (c) 2024 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 +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS +IN THE SOFTWARE. diff --git a/schemaregistry/package.json b/schemaregistry/package.json index 1c6e572f..b2b723f4 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -5,7 +5,8 @@ "main": "dist/index.js", "types": "dist/index.d.ts", "files": [ - "dist" + "LICENSE.txt", + "dist/" ], "devDependencies": { "@bufbuild/buf": "^1.37.0", @@ -49,9 +50,9 @@ "validator": "^13.12.0" }, "scripts": { - "test:types": "tsc -p .", - "test:schemaregistry": "make -f Makefile test", - "build": "npx tsc -p tsconfig-build.json" + "lint": "make lint", + "test": "make test", + "build": "rm -rf ./dist && tsc -p tsconfig-build.json" }, "keywords": [ "schemaregistry", diff --git a/schemaregistry/tsconfig.json b/schemaregistry/tsconfig.json index 7a571b6f..cc46dcf3 100644 --- a/schemaregistry/tsconfig.json +++ b/schemaregistry/tsconfig.json @@ -27,5 +27,8 @@ "include": [ "**/*", "../test/**/*" + ], + "exclude": [ + "dist" ] } From ca3d380612194d062b4bf15230a0fb0f1250a5da Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Thu, 26 Sep 2024 16:16:28 -0700 Subject: [PATCH 213/224] Add clearLatestCaches/clearCaches API, fix test to call clearLatestCaches (#102) * Add clearLatestCaches for use in tests * Minor cleanup * Minor cleanup --- schemaregistry/mock-schemaregistry-client.ts | 8 ++++++++ schemaregistry/schemaregistry-client.ts | 18 +++++++++++++++--- test/schemaregistry/serde/avro.spec.ts | 1 + 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/schemaregistry/mock-schemaregistry-client.ts b/schemaregistry/mock-schemaregistry-client.ts index 57f7fa1e..2114164b 100644 --- a/schemaregistry/mock-schemaregistry-client.ts +++ b/schemaregistry/mock-schemaregistry-client.ts @@ -426,6 +426,14 @@ class MockClient implements Client { return config; } + clearLatestCaches(): void { + return; + } + + clearCaches(): void { + return; + } + async close(): Promise { return; } diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index f501ccf0..af2cd4b2 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -135,6 +135,8 @@ export interface Client { updateConfig(subject: string, update: ServerConfig): Promise; getDefaultConfig(): Promise; updateDefaultConfig(update: ServerConfig): Promise; + clearLatestCaches(): void; + clearCaches(): void; close(): void; } @@ -557,13 +559,23 @@ export class SchemaRegistryClient implements Client { return response.data; } - close(): void { + clearLatestCaches(): void { + this.latestToSchemaCache.clear(); + this.metadataToSchemaCache.clear(); + } + + clearCaches(): void { + this.schemaToIdCache.clear(); + this.idToSchemaInfoCache.clear(); this.infoToSchemaCache.clear(); + this.latestToSchemaCache.clear(); this.schemaToVersionCache.clear(); this.versionToSchemaCache.clear(); - this.idToSchemaInfoCache.clear(); + this.metadataToSchemaCache.clear(); + } - return; + async close(): Promise { + this.clearCaches(); } // Cache methods for testing diff --git a/test/schemaregistry/serde/avro.spec.ts b/test/schemaregistry/serde/avro.spec.ts index 755a8d8f..1c51c07e 100644 --- a/test/schemaregistry/serde/avro.spec.ts +++ b/test/schemaregistry/serde/avro.spec.ts @@ -401,6 +401,7 @@ describe('AvroSerializer', () => { await client.register(subject, info, false) + client.clearLatestCaches() let deser = new AvroDeserializer(client, SerdeType.VALUE, {useLatestVersion: true}) let obj2 = await deser.deserialize(topic, bytes) expect(obj2.fieldToDelete).toEqual(undefined); From 28eac09e7a632ca7ae0a4a31a5daf796ebfc9204 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Fri, 27 Sep 2024 04:07:52 -0400 Subject: [PATCH 214/224] Add avro integration tests (#56) (#106) --- .../schemaregistry-avro.spec.ts | 615 ++++++++++++++++++ .../schemaregistry-json.spec.ts | 62 +- schemaregistry/package.json | 1 + 3 files changed, 643 insertions(+), 35 deletions(-) create mode 100644 e2e/schemaregistry/schemaregistry-avro.spec.ts diff --git a/e2e/schemaregistry/schemaregistry-avro.spec.ts b/e2e/schemaregistry/schemaregistry-avro.spec.ts new file mode 100644 index 00000000..de5bb240 --- /dev/null +++ b/e2e/schemaregistry/schemaregistry-avro.spec.ts @@ -0,0 +1,615 @@ +import { KafkaJS } from '@confluentinc/kafka-javascript'; +import { + Metadata, + SchemaRegistryClient, + SchemaInfo +} from '../../schemaregistry/schemaregistry-client'; +import { beforeEach, afterEach, describe, expect, it } from '@jest/globals'; +import { clientConfig } from '../../test/schemaregistry/test-constants'; +import { AvroDeserializer, AvroSerializer, AvroSerializerConfig } from '../../schemaregistry/serde/avro'; +import { SerdeType } from "../../schemaregistry/serde/serde"; +import stringify from 'json-stringify-deterministic'; +import { v4 } from 'uuid'; + +let schemaRegistryClient: SchemaRegistryClient; +let producer: any; + +const kafkaBrokerList = 'localhost:9092'; +const kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [kafkaBrokerList], + }, +}); + + +const userSchemaString: string = stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + ], +}); + +const messageValue = { + "name": "Bob Jones", + "age": 25 +}; + +const metadata: Metadata = { + properties: { + owner: 'Bob Jones', + email: 'bob@acme.com', + }, +}; + +const schemaInfo: SchemaInfo = { + schema: userSchemaString, + metadata: metadata +}; + +let serializerConfig: AvroSerializerConfig; +let serializer: AvroSerializer; +let deserializer: AvroDeserializer; +let consumer: KafkaJS.Consumer; + +describe('Schema Registry Avro Integration Test', () => { + + beforeEach(async () => { + schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + await producer.connect(); + + consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + }); + + afterEach(async () => { + await producer.disconnect(); + producer = null; + }); + + it("Should serialize and deserialize Avro", async () => { + const testTopic = 'test-topic-' + v4(); + + await schemaRegistryClient.register(testTopic, schemaInfo); + + serializerConfig = { autoRegisterSchemas: true }; + serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + deserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(testTopic, messageValue) + }; + + await producer.send({ + topic: testTopic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + await consumer.subscribe({ topic: testTopic }); + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(testTopic, message.value as Buffer) + }; + messageRcvd = true; + + expect(decodedMessage.value).toMatchObject(messageValue); + }, + }); + + // Wait around until we get a message, and then disconnect. + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it("Should serialize with UseLatestVersion enabled", async () => { + const testTopic = v4(); + await schemaRegistryClient.register(testTopic, schemaInfo); + + serializerConfig = { autoRegisterSchemas: true, useLatestVersion: true }; + serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(testTopic, messageValue) + }; + + await producer.send({ + topic: testTopic, + messages: [outgoingMessage] + }); + + }, 30000); + + it('Should fail to serialize with UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { + const testTopic = v4(); + await schemaRegistryClient.register(testTopic, schemaInfo); + + serializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; + serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + + const messageValue = { "name": "Bob Jones", "age": 25 }; + + await expect(serializer.serialize(testTopic, messageValue)).rejects.toThrowError(); + }); + + it('Should serialize with schemas registered, UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { + const testTopic = v4(); + await schemaRegistryClient.register(testTopic, schemaInfo); + await schemaRegistryClient.register(testTopic+'-value', schemaInfo); + + serializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; + serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + + const messageValue = { "name": "Bob Jones", "age": 25 }; + + await serializer.serialize(testTopic, messageValue); + }); + //TODO: Add test for Incompatible Types. The current Kafka Client runs console.error instead of throwing error + //Should use a spy, Jest wasn't playing nice with the spy + + it('Should produce generic message to multiple topics', async () => { + const topic1 = v4(); + const topic2 = v4(); + + await schemaRegistryClient.register(topic1, schemaInfo); + await schemaRegistryClient.register(topic2, schemaInfo); + + serializerConfig = { autoRegisterSchemas: true }; + serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + deserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + const outgoingMessage1 = { + key: 'key', + value: await serializer.serialize(topic1, messageValue) + }; + + const outgoingMessage2 = { + key: 'key', + value: await serializer.serialize(topic2, messageValue) + }; + + await producer.send( + { topic: topic1, messages: [outgoingMessage1] }, + ); + + await producer.send( + { topic: topic2, messages: [outgoingMessage2] }, + ); + + let consumer2 = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topic1 }); + await consumer2.connect(); + await consumer2.subscribe({ topic: topic2 }); + + let messageRcvd = false; + let messageRcvd2 = false; + + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic1, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toMatchObject(messageValue); + }, + }); + + await consumer2.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic2, message.value as Buffer) + }; + messageRcvd2 = true; + expect(decodedMessage.value).toMatchObject(messageValue); + }, + }); + + while (!messageRcvd || !messageRcvd2) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + await consumer2.disconnect(); + }, 30000); +}); + +describe('Schema Registry Avro Integration Test - Primitives', () => { + beforeEach(async () => { + schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + await producer.connect(); + serializerConfig = { autoRegisterSchemas: true }; + + serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + deserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + }); + + afterEach(async () => { + await producer.disconnect(); + producer = null; + }); + + it('Should serialize and deserialize string', async () => { + const stringTopic = v4(); + + const stringSchemaString = stringify({ + type: 'string', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: stringSchemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(stringTopic, stringSchemaInfo); + + const stringMessageValue = "Hello, World!"; + const outgoingStringMessage = { + key: 'key', + value: await serializer.serialize(stringTopic, stringMessageValue) + }; + + await producer.send({ + topic: stringTopic, + messages: [outgoingStringMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic: stringTopic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(stringTopic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(stringMessageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it('Should serialize and deserialize bytes', async () => { + const topic = v4(); + + const schemaString = stringify({ + type: 'bytes', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(topic, stringSchemaInfo); + + const messageValue = Buffer.from("Hello, World!"); + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(topic, messageValue) + }; + + await producer.send({ + topic: topic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(messageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it('Should serialize and deserialize int', async () => { + const topic = v4(); + + const schemaString = stringify({ + type: 'int', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(topic, stringSchemaInfo); + + const messageValue = 25; + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(topic, messageValue) + }; + + await producer.send({ + topic: topic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(messageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it('Should serialize and deserialize long', async () => { + const topic = v4(); + + const schemaString = stringify({ + type: 'long', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(topic, stringSchemaInfo); + + const messageValue = 25; + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(topic, messageValue) + }; + + await producer.send({ + topic: topic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(messageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it('Should serialize and deserialize boolean', async () => { + const topic = v4(); + + const schemaString = stringify({ + type: 'boolean', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(topic, stringSchemaInfo); + + const messageValue = true; + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(topic, messageValue) + }; + + await producer.send({ + topic: topic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(messageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it('Should serialize and deserialize float', async () => { + const topic = v4(); + + const schemaString = stringify({ + type: 'float', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(topic, stringSchemaInfo); + + const messageValue = 1.354; + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(topic, messageValue) + }; + + await producer.send({ + topic: topic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(messageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + it('Should serialize and deserialize double', async () => { + const topic = v4(); + + const schemaString = stringify({ + type: 'double', + }); + + const stringSchemaInfo: SchemaInfo = { + schema: schemaString, + metadata: metadata + }; + + await schemaRegistryClient.register(topic, stringSchemaInfo); + + const messageValue = 1.354; + const outgoingMessage = { + key: 'key', + value: await serializer.serialize(topic, messageValue) + }; + + await producer.send({ + topic: topic, + messages: [outgoingMessage] + }); + + await consumer.connect(); + + await consumer.subscribe({ topic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + const decodedMessage = { + ...message, + value: await deserializer.deserialize(topic, message.value as Buffer) + }; + messageRcvd = true; + expect(decodedMessage.value).toBe(messageValue); + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + }, 30000); + + //Waiting on the null case +}); \ No newline at end of file diff --git a/e2e/schemaregistry/schemaregistry-json.spec.ts b/e2e/schemaregistry/schemaregistry-json.spec.ts index b4751d64..4ab46584 100644 --- a/e2e/schemaregistry/schemaregistry-json.spec.ts +++ b/e2e/schemaregistry/schemaregistry-json.spec.ts @@ -10,20 +10,17 @@ import { clientConfig } from '../../test/schemaregistry/test-constants'; import { JsonSerializer, JsonSerializerConfig, JsonDeserializer } from '../../schemaregistry/serde/json'; import { SerdeType } from "../../schemaregistry/serde/serde"; import stringify from 'json-stringify-deterministic'; +import { v4 } from 'uuid'; let schemaRegistryClient: SchemaRegistryClient; let producer: any; -const testServerConfigSubject = 'integ-test-server-config-subject'; - const kafkaBrokerList = 'localhost:9092'; const kafka = new KafkaJS.Kafka({ kafkaJS: { brokers: [kafkaBrokerList], }, }); -const testTopic = `test-topic`; -const testTopicValue = testTopic + '-value'; //Inspired by dotnet client const schemaString: string = stringify({ @@ -144,21 +141,21 @@ const orderDetailsSchema: SchemaInfo = { "description": "Order Details", "type": "object", "properties": { - "id": { - "description": "Order Id", - "type": "integer" - }, - "customer": { - "description": "Customer", - "$ref": "http://example.com/customer.schema.json" - }, - "payment_id": { - "description": "Payment Id", - "type": "string" - } + "id": { + "description": "Order Id", + "type": "integer" + }, + "customer": { + "description": "Customer", + "$ref": "http://example.com/customer.schema.json" + }, + "payment_id": { + "description": "Payment Id", + "type": "string" + } }, - "required": [ "id", "customer"] -}), + "required": ["id", "customer"] + }), schemaType: 'JSON', }; @@ -255,25 +252,13 @@ const customerSubject = 'Customer'; const orderSubject = 'Order'; const orderDetailsSubject = 'OrderDetails'; -const subjectList = [testTopic, testTopicValue, testServerConfigSubject, orderSubject, orderDetailsSubject, customerSubject]; +const subjectList = [orderSubject, orderDetailsSubject, customerSubject]; describe('SchemaRegistryClient json Integration Test', () => { beforeEach(async () => { schemaRegistryClient = new SchemaRegistryClient(clientConfig); - const admin = kafka.admin(); - await admin.connect(); - try { - await admin.deleteTopics({ - topics: [testTopic], - timeout: 5000, - }); - } catch (error) { - // Topic may not exist; ignore error - } - await admin.disconnect(); - producer = kafka.producer({ kafkaJS: { allowAutoTopicCreation: true, @@ -288,6 +273,12 @@ describe('SchemaRegistryClient json Integration Test', () => { if (subjects && subjects.includes(subject)) { await schemaRegistryClient.deleteSubject(subject); await schemaRegistryClient.deleteSubject(subject, true); + + const subjectValue = subject + '-value'; + if (subjects && subjects.includes(subjectValue)) { + await schemaRegistryClient.deleteSubject(subjectValue); + await schemaRegistryClient.deleteSubject(subjectValue, true); + } } } }); @@ -298,6 +289,7 @@ describe('SchemaRegistryClient json Integration Test', () => { }); it("Should serialize and deserialize json", async () => { + const testTopic = v4(); await schemaRegistryClient.register(testTopic, schemaInfo); @@ -344,10 +336,10 @@ describe('SchemaRegistryClient json Integration Test', () => { } await consumer.disconnect(); - expect(1).toEqual(1); }, 30000); it("Should serialize with UseLatestVersion enabled", async () => { + const testTopic = v4(); await schemaRegistryClient.register(testTopic, schemaInfo); const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true, useLatestVersion: true }; @@ -366,6 +358,7 @@ describe('SchemaRegistryClient json Integration Test', () => { }); it('Should fail to serialize with UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { + const testTopic = v4(); await schemaRegistryClient.register(testTopic, schemaInfo); const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; @@ -377,13 +370,14 @@ describe('SchemaRegistryClient json Integration Test', () => { }); it("Should serialize referenced schemas", async () => { + const testTopic = v4(); const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true }; const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); const deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); await schemaRegistryClient.register(customerSubject, customerSchema); const customerIdVersion: number = (await schemaRegistryClient.getLatestSchemaMetadata(customerSubject)).version!; - + const customerReference: Reference = { name: "http://example.com/customer.schema.json", subject: customerSubject, @@ -401,9 +395,7 @@ describe('SchemaRegistryClient json Integration Test', () => { }; orderSchema.references = [orderDetailsReference]; - const orderId = await schemaRegistryClient.register(orderSubject, orderSchema); await schemaRegistryClient.register(orderSubject, orderSchema); - console.log(`Order schema id: ${orderId}`); const order = { order_details: { diff --git a/schemaregistry/package.json b/schemaregistry/package.json index b2b723f4..4b0cb768 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -47,6 +47,7 @@ "lru-cache": "^11.0.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", + "uuid": "^10.0.0", "validator": "^13.12.0" }, "scripts": { From bd4fad610991cb0a83ee6881b14f90bdbc8db3f1 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 27 Sep 2024 10:28:52 -0700 Subject: [PATCH 215/224] Add tsdoc (#107) * Add tsdocs * More tsdoc fixes --- schemaregistry/rest-error.ts | 11 +- .../rules/encryption/awskms/aws-driver.ts | 3 + .../rules/encryption/azurekms/azure-driver.ts | 3 + .../rules/encryption/encrypt-executor.ts | 3 + .../rules/encryption/gcpkms/gcp-driver.ts | 3 + .../encryption/hcvault/hcvault-driver.ts | 3 + .../rules/encryption/kms-registry.ts | 25 +++ .../rules/encryption/localkms/local-driver.ts | 3 + schemaregistry/rules/encryption/tink/aead.ts | 13 +- .../rules/encryption/tink/aes_gcm.ts | 2 - .../rules/encryption/tink/aes_siv.ts | 2 - schemaregistry/rules/encryption/tink/bytes.ts | 49 +++--- .../exception/invalid_arguments_exception.ts | 1 - .../tink/exception/security_exception.ts | 1 - schemaregistry/rules/encryption/tink/hkdf.ts | 15 +- schemaregistry/rules/encryption/tink/hmac.ts | 10 +- schemaregistry/rules/encryption/tink/mac.ts | 9 +- .../rules/encryption/tink/random.ts | 8 +- .../rules/encryption/tink/validators.ts | 14 +- .../rules/jsonata/jsonata-executor.ts | 3 + schemaregistry/schemaregistry-client.ts | 156 +++++++++++++++++- schemaregistry/serde/avro.ts | 31 ++++ schemaregistry/serde/json.ts | 36 ++++ schemaregistry/serde/protobuf.ts | 36 ++++ schemaregistry/serde/rule-registry.ts | 46 +++++- schemaregistry/serde/serde.ts | 95 +++++++++-- schemaregistry/serde/wildcard-matcher.ts | 6 +- 27 files changed, 485 insertions(+), 102 deletions(-) diff --git a/schemaregistry/rest-error.ts b/schemaregistry/rest-error.ts index 19fd086a..a5db4ca8 100644 --- a/schemaregistry/rest-error.ts +++ b/schemaregistry/rest-error.ts @@ -1,10 +1,19 @@ +/** + * Represents a REST error. + */ export class RestError extends Error { status: number; errorCode: number; + /** + * Creates a new REST error. + * @param message - The error message. + * @param status - The HTTP status code. + * @param errorCode - The error code. + */ constructor(message: string, status: number, errorCode: number) { super(message + "; Error code: " + errorCode); this.status = status; this.errorCode = errorCode; } -} \ No newline at end of file +} diff --git a/schemaregistry/rules/encryption/awskms/aws-driver.ts b/schemaregistry/rules/encryption/awskms/aws-driver.ts index e01d8afa..13c217c6 100644 --- a/schemaregistry/rules/encryption/awskms/aws-driver.ts +++ b/schemaregistry/rules/encryption/awskms/aws-driver.ts @@ -8,6 +8,9 @@ export class AwsKmsDriver implements KmsDriver { static ACCESS_KEY_ID = 'access.key.id' static SECRET_ACCESS_KEY = 'secret.access.key' + /** + * Register the AWS KMS driver with the KMS registry. + */ static register(): void { registerKmsDriver(new AwsKmsDriver()) } diff --git a/schemaregistry/rules/encryption/azurekms/azure-driver.ts b/schemaregistry/rules/encryption/azurekms/azure-driver.ts index 221c38b8..01c01cd1 100644 --- a/schemaregistry/rules/encryption/azurekms/azure-driver.ts +++ b/schemaregistry/rules/encryption/azurekms/azure-driver.ts @@ -9,6 +9,9 @@ export class AzureKmsDriver implements KmsDriver { static CLIENT_ID = 'client.id' static CLIENT_SECRET = 'client.secret' + /** + * Register the Azure KMS driver with the KMS registry. + */ static register(): void { registerKmsDriver(new AzureKmsDriver()) } diff --git a/schemaregistry/rules/encryption/encrypt-executor.ts b/schemaregistry/rules/encryption/encrypt-executor.ts index 5eaf67b0..ffbd018c 100644 --- a/schemaregistry/rules/encryption/encrypt-executor.ts +++ b/schemaregistry/rules/encryption/encrypt-executor.ts @@ -64,6 +64,9 @@ export class FieldEncryptionExecutor extends FieldRuleExecutor { client: Client | null = null clock: Clock + /** + * Register the field encryption executor with the rule registry. + */ static register(): FieldEncryptionExecutor { return this.registerWithClock(new Clock()) } diff --git a/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts b/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts index 73021678..60ee47dc 100644 --- a/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts +++ b/schemaregistry/rules/encryption/gcpkms/gcp-driver.ts @@ -10,6 +10,9 @@ export class GcpKmsDriver implements KmsDriver { static PRIVATE_KEY_ID = "private.key.id"; static PRIVATE_KEY= "private.key"; + /** + * Register the GCP KMS driver with the KMS registry. + */ static register(): void { registerKmsDriver(new GcpKmsDriver()) } diff --git a/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts b/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts index 9b8638d6..de9ccdfc 100644 --- a/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts +++ b/schemaregistry/rules/encryption/hcvault/hcvault-driver.ts @@ -7,6 +7,9 @@ export class HcVaultDriver implements KmsDriver { static TOKEN_ID = 'token.id' static NAMESPACE = 'namespace' + /** + * Register the HashiCorp Vault driver with the KMS registry. + */ static register(): void { registerKmsDriver(new HcVaultDriver()) } diff --git a/schemaregistry/rules/encryption/kms-registry.ts b/schemaregistry/rules/encryption/kms-registry.ts index cef8129c..c23e9a5d 100644 --- a/schemaregistry/rules/encryption/kms-registry.ts +++ b/schemaregistry/rules/encryption/kms-registry.ts @@ -1,10 +1,16 @@ import {SecurityException} from "./tink/exception/security_exception"; +/** + * Key management service (KMS) driver. + */ export interface KmsDriver { getKeyUrlPrefix(): string newKmsClient(config: Map, keyUrl: string): KmsClient } +/** + * Key management service (KMS) client. + */ export interface KmsClient { supported(keyUri: string): boolean encrypt(plaintext: Buffer): Promise @@ -16,10 +22,18 @@ const kmsDrivers: KmsDriver[] = [] const kmsClients: KmsClient[] = [] +/** + * Register a KMS driver. + * @param kmsDriver - the KMS driver to register + */ export function registerKmsDriver(kmsDriver: KmsDriver): void { kmsDrivers.push(kmsDriver) } +/** + * Get the KMS driver for the given key URL. + * @param keyUrl - the key URL + */ export function getKmsDriver(keyUrl: string): KmsDriver { for (let driver of kmsDrivers) { if (keyUrl.startsWith(driver.getKeyUrlPrefix())) { @@ -29,10 +43,18 @@ export function getKmsDriver(keyUrl: string): KmsDriver { throw new SecurityException('no KMS driver found for key URL: ' + keyUrl) } +/** + * Register a KMS client. + * @param kmsClient - the KMS client to register + */ export function registerKmsClient(kmsClient: KmsClient): void { kmsClients.push(kmsClient) } +/** + * Get the KMS client for the given key URL. + * @param keyUrl - the key URL + */ export function getKmsClient(keyUrl: string): KmsClient | null { for (let client of kmsClients) { if (client.supported(keyUrl)) { @@ -42,6 +64,9 @@ export function getKmsClient(keyUrl: string): KmsClient | null { return null } +/** + * Clear the KMS clients. + */ export function clearKmsClients(): void { kmsClients.length = 0 } diff --git a/schemaregistry/rules/encryption/localkms/local-driver.ts b/schemaregistry/rules/encryption/localkms/local-driver.ts index 96ecb481..5e1b9184 100644 --- a/schemaregistry/rules/encryption/localkms/local-driver.ts +++ b/schemaregistry/rules/encryption/localkms/local-driver.ts @@ -6,6 +6,9 @@ export class LocalKmsDriver implements KmsDriver { static PREFIX: string = 'local-kms://' static SECRET: string = 'secret' + /** + * Register the local KMS driver with the KMS registry. + */ static register(): void { registerKmsDriver(new LocalKmsDriver()) } diff --git a/schemaregistry/rules/encryption/tink/aead.ts b/schemaregistry/rules/encryption/tink/aead.ts index 75754a7e..df85ca73 100644 --- a/schemaregistry/rules/encryption/tink/aead.ts +++ b/schemaregistry/rules/encryption/tink/aead.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -21,13 +20,13 @@ export abstract class Aead { * data. The resulting ciphertext allows for checking authenticity and * integrity of associated data, but does not guarantee its secrecy. * - * @param plaintext the plaintext to be encrypted. It must be + * @param plaintext - the plaintext to be encrypted. It must be * non-null, but can also be an empty (zero-length) byte array. - * @param opt_associatedData optional associated data to be + * @param opt_associatedData - optional associated data to be * authenticated, but not encrypted. A null value is equivalent to an * empty (zero-length) byte array. For successful decryption the same * associated data must be provided along with the ciphertext. - * @return resulting ciphertext + * @returns resulting ciphertext * */ abstract encrypt(plaintext: Uint8Array, opt_associatedData?: Uint8Array|null): @@ -38,13 +37,13 @@ export abstract class Aead { * The decryption verifies the authenticity and integrity of the associated * data, but there are no guarantees wrt. secrecy of that data. * - * @param ciphertext the ciphertext to be decrypted, must be + * @param ciphertext - the ciphertext to be decrypted, must be * non-null. - * @param opt_associatedData optional associated data to be + * @param opt_associatedData - optional associated data to be * authenticated. A null value is equivalent to an empty (zero-length) * byte array. For successful decryption the same associated data must be * provided along with the ciphertext. - * @return resulting plaintext + * @returns resulting plaintext */ abstract decrypt( ciphertext: Uint8Array, diff --git a/schemaregistry/rules/encryption/tink/aes_gcm.ts b/schemaregistry/rules/encryption/tink/aes_gcm.ts index fcce0110..0035759f 100644 --- a/schemaregistry/rules/encryption/tink/aes_gcm.ts +++ b/schemaregistry/rules/encryption/tink/aes_gcm.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -27,7 +26,6 @@ const TAG_SIZE_IN_BITS: number = 128; /** * Implementation of AES-GCM. * - * @final */ export class AesGcm extends Aead { constructor(private readonly key: CryptoKey) { diff --git a/schemaregistry/rules/encryption/tink/aes_siv.ts b/schemaregistry/rules/encryption/tink/aes_siv.ts index 5d2fc4a0..1e26583c 100644 --- a/schemaregistry/rules/encryption/tink/aes_siv.ts +++ b/schemaregistry/rules/encryption/tink/aes_siv.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -13,7 +12,6 @@ import * as crypto from 'crypto'; /** * Implementation of AES-SIV. * - * @final */ export class AesSiv extends Aead { constructor(private readonly key: Uint8Array) { diff --git a/schemaregistry/rules/encryption/tink/bytes.ts b/schemaregistry/rules/encryption/tink/bytes.ts index 49c568c5..7e08dbaf 100644 --- a/schemaregistry/rules/encryption/tink/bytes.ts +++ b/schemaregistry/rules/encryption/tink/bytes.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -8,9 +7,9 @@ import {InvalidArgumentsException} from './exception/invalid_arguments_exception /** * Does near constant time byte array comparison. - * @param ba1 The first bytearray to check. - * @param ba2 The second bytearray to check. - * @return If the array are equal. + * @param ba1 - The first bytearray to check. + * @param ba2 - The second bytearray to check. + * @returns If the array are equal. */ export function isEqual(ba1: Uint8Array, ba2: Uint8Array): boolean { if (ba1.length !== ba2.length) { @@ -45,10 +44,9 @@ export function concat(...var_args: Uint8Array[]): Uint8Array { /** * Converts a non-negative integer number to a 64-bit big-endian byte array. - * @param value The number to convert. - * @return The number as a big-endian byte array. - * @throws {InvalidArgumentsException} - * @static + * @param value - The number to convert. + * @returns The number as a big-endian byte array. + * @throws {@link InvalidArgumentsException} */ export function fromNumber(value: number): Uint8Array { if (Number.isNaN(value) || value % 1 !== 0) { @@ -79,10 +77,9 @@ export function fromNumber(value: number): Uint8Array { /** * Converts the hex string to a byte array. * - * @param hex the input - * @return the byte array output - * @throws {!InvalidArgumentsException} - * @static + * @param hex - the input + * @returns the byte array output + * @throws {@link InvalidArgumentsException} */ export function fromHex(hex: string): Uint8Array { if (hex.length % 2 != 0) { @@ -99,9 +96,8 @@ export function fromHex(hex: string): Uint8Array { /** * Converts a byte array to hex. * - * @param bytes the byte array input - * @return hex the output - * @static + * @param bytes - the byte array input + * @returns hex the output */ export function toHex(bytes: Uint8Array): string { let result = ''; @@ -115,11 +111,10 @@ export function toHex(bytes: Uint8Array): string { /** * Converts the Base64 string to a byte array. * - * @param encoded the base64 string - * @param opt_webSafe True indicates we should use the alternative + * @param encoded - the base64 string + * @param opt_webSafe - True indicates we should use the alternative * alphabet, which does not require escaping for use in URLs. - * @return the byte array output - * @static + * @returns the byte array output */ export function fromBase64(encoded: string, opt_webSafe?: boolean): Uint8Array { if (opt_webSafe) { @@ -132,11 +127,10 @@ export function fromBase64(encoded: string, opt_webSafe?: boolean): Uint8Array { /** * Base64 encode a byte array. * - * @param bytes the byte array input - * @param opt_webSafe True indicates we should use the alternative + * @param bytes - the byte array input + * @param opt_webSafe - True indicates we should use the alternative * alphabet, which does not require escaping for use in URLs. - * @return base64 output - * @static + * @returns base64 - output */ export function toBase64(bytes: Uint8Array, opt_webSafe?: boolean): string { const encoded = window @@ -154,9 +148,8 @@ export function toBase64(bytes: Uint8Array, opt_webSafe?: boolean): string { * Converts a byte string to a byte array. Only support ASCII and Latin-1 * strings, does not support multi-byte characters. * - * @param str the input - * @return the byte array output - * @static + * @param str - the input + * @returns the byte array output */ export function fromByteString(str: string): Uint8Array { const output = []; @@ -173,9 +166,9 @@ export function fromByteString(str: string): Uint8Array { * characters to which the numbers correspond. Each byte is corresponding to a * character. Does not support multi-byte characters. * - * @param bytes Array of numbers representing + * @param bytes - Array of numbers representing * characters. - * @return Stringification of the array. + * @returns Stringification of the array. */ export function toByteString(bytes: Uint8Array): string { let str = ''; diff --git a/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts b/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts index 83edc8e0..26481153 100644 --- a/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts +++ b/schemaregistry/rules/encryption/tink/exception/invalid_arguments_exception.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ diff --git a/schemaregistry/rules/encryption/tink/exception/security_exception.ts b/schemaregistry/rules/encryption/tink/exception/security_exception.ts index e4c7c5c3..25d81b75 100644 --- a/schemaregistry/rules/encryption/tink/exception/security_exception.ts +++ b/schemaregistry/rules/encryption/tink/exception/security_exception.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ diff --git a/schemaregistry/rules/encryption/tink/hkdf.ts b/schemaregistry/rules/encryption/tink/hkdf.ts index 5a411222..2f0406df 100644 --- a/schemaregistry/rules/encryption/tink/hkdf.ts +++ b/schemaregistry/rules/encryption/tink/hkdf.ts @@ -1,12 +1,11 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ /** - * @fileoverview An implementation of HKDF, RFC 5869. + * An implementation of HKDF, RFC 5869. */ import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; @@ -16,17 +15,17 @@ import * as Validators from './validators'; /** * Computes an HKDF. * - * @param size The length of the generated pseudorandom string in + * @param size - The length of the generated pseudorandom string in * bytes. The maximal size is 255 * DigestSize, where DigestSize is the size * of the underlying HMAC. - * @param hash the name of the hash function. Accepted names are SHA-1, + * @param hash - the name of the hash function. Accepted names are SHA-1, * SHA-256 and SHA-512 - * @param ikm Input keying material. - * @param info Context and application specific + * @param ikm - Input keying material. + * @param info - Context and application specific * information (can be a zero-length array). - * @param opt_salt Salt value (a non-secret random + * @param opt_salt - Salt value (a non-secret random * value). If not provided, it is set to a string of hash length zeros. - * @return Output keying material (okm). + * @returns Output keying material (okm). */ export async function compute( size: number, hash: string, ikm: Uint8Array, info: Uint8Array, diff --git a/schemaregistry/rules/encryption/tink/hmac.ts b/schemaregistry/rules/encryption/tink/hmac.ts index 15045e68..c83ccfff 100644 --- a/schemaregistry/rules/encryption/tink/hmac.ts +++ b/schemaregistry/rules/encryption/tink/hmac.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -20,12 +19,11 @@ const MIN_TAG_SIZE_IN_BYTES: number = 10; /** * Implementation of HMAC. * - * @final */ export class Hmac extends Mac { /** - * @param hash accepted names are SHA-1, SHA-256 and SHA-512 - * @param tagSize the size of the tag + * @param hash - accepted names are SHA-1, SHA-256 and SHA-512 + * @param tagSize - the size of the tag */ constructor( private readonly hash: string, private readonly key: CryptoKey, @@ -53,8 +51,8 @@ export class Hmac extends Mac { } /** - * @param hash accepted names are SHA-1, SHA-256 and SHA-512 - * @param tagSize the size of the tag + * @param hash - accepted names are SHA-1, SHA-256 and SHA-512 + * @param tagSize - the size of the tag */ export async function fromRawKey( hash: string, key: Uint8Array, tagSize: number): Promise { diff --git a/schemaregistry/rules/encryption/tink/mac.ts b/schemaregistry/rules/encryption/tink/mac.ts index 31e422da..034259da 100644 --- a/schemaregistry/rules/encryption/tink/mac.ts +++ b/schemaregistry/rules/encryption/tink/mac.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -19,16 +18,16 @@ export abstract class Mac { /** * Computes message authentication code (MAC) for `data`. * - * @param data the data to compute MAC - * @return the MAC tag + * @param data - the data to compute MAC + * @returns the MAC tag */ abstract computeMac(data: Uint8Array): Promise; /** * Verifies whether `tag` is a correct authentication code for `data`. * - * @param tag the MAC tag - * @param data the data to compute MAC + * @param tag - the MAC tag + * @param data - the data to compute MAC */ abstract verifyMac(tag: Uint8Array, data: Uint8Array): Promise; } diff --git a/schemaregistry/rules/encryption/tink/random.ts b/schemaregistry/rules/encryption/tink/random.ts index b139bc08..7ec2dbdc 100644 --- a/schemaregistry/rules/encryption/tink/random.ts +++ b/schemaregistry/rules/encryption/tink/random.ts @@ -1,12 +1,11 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ /** - * @fileoverview Several simple wrappers of crypto.getRandomValues. + * Several simple wrappers of crypto.getRandomValues. */ import {InvalidArgumentsException} from './exception/invalid_arguments_exception'; import * as crypto from 'crypto'; @@ -14,9 +13,8 @@ import * as crypto from 'crypto'; /** * Randomly generates `n` bytes. * - * @param n number of bytes to generate - * @return the random bytes - * @static + * @param n - number of bytes to generate + * @returns the random bytes */ export function randBytes(n: number): Uint8Array { if (!Number.isInteger(n) || n < 0) { diff --git a/schemaregistry/rules/encryption/tink/validators.ts b/schemaregistry/rules/encryption/tink/validators.ts index 3f952fc0..6b73ae55 100644 --- a/schemaregistry/rules/encryption/tink/validators.ts +++ b/schemaregistry/rules/encryption/tink/validators.ts @@ -1,5 +1,4 @@ /** - * @license * Copyright 2020 Google LLC * SPDX-License-Identifier: Apache-2.0 */ @@ -12,9 +11,8 @@ const SUPPORTED_AES_KEY_SIZES: number[] = [16, 32]; * Validates AES key sizes, at the moment only 128-bit and 256-bit keys are * supported. * - * @param n the key size in bytes - * @throws {!InvalidArgumentsException} - * @static + * @param n - the key size in bytes + * @throws {@link InvalidArgumentsException} */ export function validateAesKeySize(n: number) { if (!SUPPORTED_AES_KEY_SIZES.includes(n)) { @@ -25,8 +23,7 @@ export function validateAesKeySize(n: number) { /** * Validates that the input is a non null Uint8Array. * - * @throws {!InvalidArgumentsException} - * @static + * @throws {@link InvalidArgumentsException} */ export function requireUint8Array(input: Uint8Array) { if (input == null || !(input instanceof Uint8Array)) { @@ -40,8 +37,7 @@ export function requireUint8Array(input: Uint8Array) { * * @param candidate - version to be validated * @param maxVersion - upper bound on version - * @throws {!SecurityException} - * @static + * @throws {@link SecurityException} */ export function validateVersion(candidate: number, maxVersion: number) { if (candidate < 0 || candidate > maxVersion) { @@ -54,7 +50,7 @@ export function validateVersion(candidate: number, maxVersion: number) { /** * Validates ECDSA parameters. * - * @throws {!SecurityException} + * @throws {@link SecurityException} */ export function validateEcdsaParams(curve: string, hash: string) { switch (curve) { diff --git a/schemaregistry/rules/jsonata/jsonata-executor.ts b/schemaregistry/rules/jsonata/jsonata-executor.ts index cd9209dd..556d4a98 100644 --- a/schemaregistry/rules/jsonata/jsonata-executor.ts +++ b/schemaregistry/rules/jsonata/jsonata-executor.ts @@ -8,6 +8,9 @@ export class JsonataExecutor implements RuleExecutor { config: Map | null = null cache: LRUCache = new LRUCache({max: 1000}) + /** + * Register the JSONata rule executor with the rule registry. + */ static register(): JsonataExecutor { const executor = new JsonataExecutor() RuleRegistry.registerRuleExecutor(executor) diff --git a/schemaregistry/schemaregistry-client.ts b/schemaregistry/schemaregistry-client.ts index af2cd4b2..3c7c4b0e 100644 --- a/schemaregistry/schemaregistry-client.ts +++ b/schemaregistry/schemaregistry-client.ts @@ -29,6 +29,9 @@ export interface CompatibilityLevel { compatibilityLevel?: Compatibility; } +/** + * Rule represents a data contract rule + */ export interface Rule { name: string doc?: string @@ -52,6 +55,9 @@ export enum RuleMode { WRITEREAD = 'WRITEREAD', } +/** + * SchemaInfo represents a schema and its associated information + */ export interface SchemaInfo { schema: string; schemaType?: string; @@ -60,7 +66,7 @@ export interface SchemaInfo { ruleSet?: RuleSet; } -// Ensure that SchemaMetadata fields are removed +// Ensure that SchemaMetadata fields are removed from the SchemaInfo export function minimize(info: SchemaInfo): SchemaInfo { return { schemaType: info.schemaType, @@ -71,29 +77,44 @@ export function minimize(info: SchemaInfo): SchemaInfo { } } +/** + * SchemaMetadata extends SchemaInfo with additional metadata + */ export interface SchemaMetadata extends SchemaInfo { id: number; subject?: string; version?: number; } +/** + * Reference represents a schema reference + */ export interface Reference { name: string; subject: string; version: number; } +/** + * Metadata represents user-defined metadata + */ export interface Metadata { tags?: { [key: string]: string[] }; properties?: { [key: string]: string }; sensitive?: string[]; } +/** + * RuleSet represents a data contract rule set + */ export interface RuleSet { migrationRules?: Rule[]; domainRules?: Rule[]; } +/** + * ServerConfig represents config params for Schema Registry + */ export interface ServerConfig { alias?: string; normalize?: boolean; @@ -110,6 +131,11 @@ export interface isCompatibleResponse { is_compatible: boolean; } +/** + * Client is an interface for clients interacting with the Confluent Schema Registry. + * The Schema Registry's REST interface is further explained in Confluent's Schema Registry API documentation + * https://github.com/confluentinc/schema-registry/blob/master/client/src/main/java/io/confluent/kafka/schemaregistry/client/SchemaRegistryClient.java + */ export interface Client { config(): ClientConfig; register(subject: string, schema: SchemaInfo, normalize: boolean): Promise; @@ -140,6 +166,10 @@ export interface Client { close(): void; } +/** + * SchemaRegistryClient is a client for interacting with the Confluent Schema Registry. + * This client will cache responses from Schema Registry to reduce network requests. + */ export class SchemaRegistryClient implements Client { private clientConfig: ClientConfig; private restService: RestService; @@ -160,6 +190,10 @@ export class SchemaRegistryClient implements Client { private versionToSchemaMutex: Mutex; private metadataToSchemaMutex: Mutex; + /** + * Create a new Schema Registry client. + * @param config - The client configuration. + */ constructor(config: ClientConfig) { this.clientConfig = config const cacheOptions = { @@ -198,12 +232,24 @@ export class SchemaRegistryClient implements Client { return this.clientConfig } + /** + * Register a schema with the Schema Registry and return the schema ID. + * @param subject - The subject under which to register the schema. + * @param schema - The schema to register. + * @param normalize - Whether to normalize the schema before registering. + */ async register(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const metadataResult = await this.registerFullResponse(subject, schema, normalize); return metadataResult.id; } + /** + * Register a schema with the Schema Registry and return the full response. + * @param subject - The subject under which to register the schema. + * @param schema - The schema to register. + * @param normalize - Whether to normalize the schema before registering. + */ async registerFullResponse(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema: minimize(schema) }); @@ -225,6 +271,12 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Get a schema by subject and ID. + * @param subject - The subject under which the schema is registered. + * @param id - The schema ID. + * @param format - The format of the schema. + */ async getBySubjectAndId(subject: string, id: number, format?: string): Promise { const cacheKey = stringify({ subject, id }); return await this.idToSchemaInfoMutex.runExclusive(async () => { @@ -246,6 +298,12 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Get the ID for a schema. + * @param subject - The subject under which the schema is registered. + * @param schema - The schema whose ID to get. + * @param normalize - Whether to normalize the schema before getting the ID. + */ async getId(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema: minimize(schema) }); @@ -267,6 +325,11 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Get the latest schema metadata for a subject. + * @param subject - The subject for which to get the latest schema metadata. + * @param format - The format of the schema. + */ async getLatestSchemaMetadata(subject: string, format?: string): Promise { return await this.latestToSchemaMutex.runExclusive(async () => { const cachedSchema: SchemaMetadata | undefined = this.latestToSchemaCache.get(subject); @@ -287,6 +350,13 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Get the schema metadata for a subject and version. + * @param subject - The subject for which to get the schema metadata. + * @param version - The version of the schema. + * @param deleted - Whether to include deleted schemas. + * @param format - The format of the schema. + */ async getSchemaMetadata(subject: string, version: number, deleted: boolean = false, format?: string): Promise { const cacheKey = stringify({ subject, version, deleted }); @@ -309,6 +379,13 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Get the latest schema metadata for a subject with the given metadata. + * @param subject - The subject for which to get the latest schema metadata. + * @param metadata - The metadata to match. + * @param deleted - Whether to include deleted schemas. + * @param format - The format of the schema. + */ async getLatestWithMetadata(subject: string, metadata: { [key: string]: string }, deleted: boolean = false, format?: string): Promise { const cacheKey = stringify({ subject, metadata, deleted }); @@ -340,7 +417,10 @@ export class SchemaRegistryClient implements Client { }); } - + /** + * Get all versions of a schema for a subject. + * @param subject - The subject for which to get all versions. + */ async getAllVersions(subject: string): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/subjects/${subject}/versions`, @@ -349,6 +429,12 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Get the version of a schema for a subject. + * @param subject - The subject for which to get the version. + * @param schema - The schema for which to get the version. + * @param normalize - Whether to normalize the schema before getting the version. + */ async getVersion(subject: string, schema: SchemaInfo, normalize: boolean = false): Promise { const cacheKey = stringify({ subject, schema: minimize(schema) }); @@ -370,6 +456,9 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Get all subjects in the Schema Registry. + */ async getAllSubjects(): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/subjects`, @@ -378,6 +467,11 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Delete a subject from the Schema Registry. + * @param subject - The subject to delete. + * @param permanent - Whether to permanently delete the subject. + */ async deleteSubject(subject: string, permanent: boolean = false): Promise { await this.infoToSchemaMutex.runExclusive(async () => { this.infoToSchemaCache.forEach((_, key) => { @@ -424,6 +518,12 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Delete a version of a subject from the Schema Registry. + * @param subject - The subject to delete. + * @param version - The version to delete. + * @param permanent - Whether to permanently delete the version. + */ async deleteSubjectVersion(subject: string, version: number, permanent: boolean = false): Promise { return await this.schemaToVersionMutex.runExclusive(async () => { let metadataValue: SchemaMetadata | undefined; @@ -463,6 +563,11 @@ export class SchemaRegistryClient implements Client { }); } + /** + * Test the compatibility of a schema with the latest schema for a subject. + * @param subject - The subject for which to test compatibility. + * @param schema - The schema to test compatibility. + */ async testSubjectCompatibility(subject: string, schema: SchemaInfo): Promise { subject = encodeURIComponent(subject); @@ -474,6 +579,12 @@ export class SchemaRegistryClient implements Client { return response.data.is_compatible; } + /** + * Test the compatibility of a schema with a specific version of a subject. + * @param subject - The subject for which to test compatibility. + * @param version - The version of the schema for which to test compatibility. + * @param schema - The schema to test compatibility. + */ async testCompatibility(subject: string, version: number, schema: SchemaInfo): Promise { subject = encodeURIComponent(subject); @@ -485,6 +596,10 @@ export class SchemaRegistryClient implements Client { return response.data.is_compatible; } + /** + * Get the compatibility level for a subject. + * @param subject - The subject for which to get the compatibility level. + */ async getCompatibility(subject: string): Promise { subject = encodeURIComponent(subject); @@ -495,6 +610,11 @@ export class SchemaRegistryClient implements Client { return response.data.compatibilityLevel!; } + /** + * Update the compatibility level for a subject. + * @param subject - The subject for which to update the compatibility level. + * @param update - The compatibility level to update to. + */ async updateCompatibility(subject: string, update: Compatibility): Promise { subject = encodeURIComponent(subject); @@ -506,6 +626,9 @@ export class SchemaRegistryClient implements Client { return response.data.compatibility!; } + /** + * Get the default/global compatibility level. + */ async getDefaultCompatibility(): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, @@ -514,6 +637,10 @@ export class SchemaRegistryClient implements Client { return response.data.compatibilityLevel!; } + /** + * Update the default/global compatibility level. + * @param update - The compatibility level to update to. + */ async updateDefaultCompatibility(update: Compatibility): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, @@ -523,6 +650,10 @@ export class SchemaRegistryClient implements Client { return response.data.compatibility!; } + /** + * Get the config for a subject. + * @param subject - The subject for which to get the config. + */ async getConfig(subject: string): Promise { subject = encodeURIComponent(subject); @@ -533,6 +664,11 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Update the config for a subject. + * @param subject - The subject for which to update the config. + * @param update - The config to update to. + */ async updateConfig(subject: string, update: ServerConfig): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config/${subject}`, @@ -542,6 +678,9 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Get the default/global config. + */ async getDefaultConfig(): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, @@ -550,6 +689,10 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Update the default/global config. + * @param update - The config to update to. + */ async updateDefaultConfig(update: ServerConfig): Promise { const response: AxiosResponse = await this.restService.handleRequest( `/config`, @@ -559,11 +702,17 @@ export class SchemaRegistryClient implements Client { return response.data; } + /** + * Clear the latest caches. + */ clearLatestCaches(): void { this.latestToSchemaCache.clear(); this.metadataToSchemaCache.clear(); } + /** + * Clear all caches. + */ clearCaches(): void { this.schemaToIdCache.clear(); this.idToSchemaInfoCache.clear(); @@ -574,6 +723,9 @@ export class SchemaRegistryClient implements Client { this.metadataToSchemaCache.clear(); } + /** + * Close the client. + */ async close(): Promise { this.clearCaches(); } diff --git a/schemaregistry/serde/avro.ts b/schemaregistry/serde/avro.ts index dcf509fd..055fabca 100644 --- a/schemaregistry/serde/avro.ts +++ b/schemaregistry/serde/avro.ts @@ -29,11 +29,24 @@ export interface AvroSerde { schemaToTypeCache: LRUCache]> } +/** + * AvroSerializerConfig is used to configure the AvroSerializer. + */ export type AvroSerializerConfig = SerializerConfig & AvroSerdeConfig +/** + * AvroSerializer is used to serialize messages using Avro. + */ export class AvroSerializer extends Serializer implements AvroSerde { schemaToTypeCache: LRUCache]> + /** + * Create a new AvroSerializer. + * @param client - the schema registry client + * @param serdeType - the type of the serializer + * @param conf - the serializer configuration + * @param ruleRegistry - the rule registry + */ constructor(client: Client, serdeType: SerdeType, conf: AvroSerializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) this.schemaToTypeCache = new LRUCache]>({ max: this.conf.cacheCapacity ?? 1000 }) @@ -45,6 +58,11 @@ export class AvroSerializer extends Serializer implements AvroSerde { } } + /** + * serialize is used to serialize a message using Avro. + * @param topic - the topic to serialize the message for + * @param msg - the message to serialize + */ override async serialize(topic: string, msg: any): Promise { if (this.client == null) { throw new Error('client is not initialized') @@ -110,11 +128,24 @@ export class AvroSerializer extends Serializer implements AvroSerde { } } +/** + * AvroDeserializerConfig is used to configure the AvroDeserializer. + */ export type AvroDeserializerConfig = DeserializerConfig & AvroSerdeConfig +/** + * AvroDeserializer is used to deserialize messages using Avro. + */ export class AvroDeserializer extends Deserializer implements AvroSerde { schemaToTypeCache: LRUCache]> + /** + * Create a new AvroDeserializer. + * @param client - the schema registry client + * @param serdeType - the type of the deserializer + * @param conf - the deserializer configuration + * @param ruleRegistry - the rule registry + */ constructor(client: Client, serdeType: SerdeType, conf: AvroDeserializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) this.schemaToTypeCache = new LRUCache]>({ max: this.conf.cacheCapacity ?? 1000 }) diff --git a/schemaregistry/serde/json.ts b/schemaregistry/serde/json.ts index 709a8b3f..77487a4f 100644 --- a/schemaregistry/serde/json.ts +++ b/schemaregistry/serde/json.ts @@ -47,12 +47,25 @@ export interface JsonSerde { schemaToValidateCache: LRUCache } +/** + * JsonSerializerConfig is the configuration for the JsonSerializer. + */ export type JsonSerializerConfig = SerializerConfig & JsonSerdeConfig +/** + * JsonSerializer is a serializer for JSON messages. + */ export class JsonSerializer extends Serializer implements JsonSerde { schemaToTypeCache: LRUCache schemaToValidateCache: LRUCache + /** + * Creates a new JsonSerializer. + * @param client - the schema registry client + * @param serdeType - the serializer type + * @param conf - the serializer configuration + * @param ruleRegistry - the rule registry + */ constructor(client: Client, serdeType: SerdeType, conf: JsonSerializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) this.schemaToTypeCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) @@ -65,6 +78,11 @@ export class JsonSerializer extends Serializer implements JsonSerde { } } + /** + * Serializes a message. + * @param topic - the topic + * @param msg - the message + */ override async serialize(topic: string, msg: any): Promise { if (this.client == null) { throw new Error('client is not initialized') @@ -121,12 +139,25 @@ export class JsonSerializer extends Serializer implements JsonSerde { } } +/** + * JsonDeserializerConfig is the configuration for the JsonDeserializer. + */ export type JsonDeserializerConfig = DeserializerConfig & JsonSerdeConfig +/** + * JsonDeserializer is a deserializer for JSON messages. + */ export class JsonDeserializer extends Deserializer implements JsonSerde { schemaToTypeCache: LRUCache schemaToValidateCache: LRUCache + /** + * Creates a new JsonDeserializer. + * @param client - the schema registry client + * @param serdeType - the deserializer type + * @param conf - the deserializer configuration + * @param ruleRegistry - the rule registry + */ constructor(client: Client, serdeType: SerdeType, conf: JsonDeserializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) this.schemaToTypeCache = new LRUCache({ max: this.config().cacheCapacity ?? 1000 }) @@ -139,6 +170,11 @@ export class JsonDeserializer extends Deserializer implements JsonSerde { } } + /** + * Deserializes a message. + * @param topic - the topic + * @param payload - the message payload + */ override async deserialize(topic: string, payload: Buffer): Promise { if (!Buffer.isBuffer(payload)) { throw new Error('Invalid buffer') diff --git a/schemaregistry/serde/protobuf.ts b/schemaregistry/serde/protobuf.ts index 3c23c1cb..b545b8bb 100644 --- a/schemaregistry/serde/protobuf.ts +++ b/schemaregistry/serde/protobuf.ts @@ -88,15 +88,28 @@ export interface ProtobufSerde { schemaToDescCache: LRUCache } +/** + * ProtobufSerializerConfig is the configuration for ProtobufSerializer. + */ export type ProtobufSerializerConfig = SerializerConfig & { registry?: MutableRegistry } +/** + * ProtobufSerializer is a serializer for Protobuf messages. + */ export class ProtobufSerializer extends Serializer implements ProtobufSerde { registry: MutableRegistry schemaToDescCache: LRUCache descToSchemaCache: LRUCache + /** + * Creates a new ProtobufSerializer. + * @param client - the schema registry client + * @param serdeType - the serializer type + * @param conf - the serializer configuration + * @param ruleRegistry - the rule registry + */ constructor(client: Client, serdeType: SerdeType, conf: ProtobufSerializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) this.registry = conf.registry ?? createMutableRegistry() @@ -110,6 +123,11 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { } } + /** + * Serializes a message. + * @param topic - the topic + * @param msg - the message + */ override async serialize(topic: string, msg: any): Promise { if (this.client == null) { throw new Error('client is not initialized') @@ -275,12 +293,25 @@ export class ProtobufSerializer extends Serializer implements ProtobufSerde { } } +/** + * ProtobufDeserializerConfig is the configuration for ProtobufDeserializer. + */ export type ProtobufDeserializerConfig = DeserializerConfig +/** + * ProtobufDeserializer is a deserializer for Protobuf messages. + */ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde { registry: FileRegistry schemaToDescCache: LRUCache + /** + * Creates a new ProtobufDeserializer. + * @param client - the schema registry client + * @param serdeType - the deserializer type + * @param conf - the deserializer configuration + * @param ruleRegistry - the rule registry + */ constructor(client: Client, serdeType: SerdeType, conf: ProtobufDeserializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) this.registry = createFileRegistry() @@ -293,6 +324,11 @@ export class ProtobufDeserializer extends Deserializer implements ProtobufSerde } } + /** + * Deserializes a message. + * @param topic - the topic + * @param payload - the message payload + */ override async deserialize(topic: string, payload: Buffer): Promise { if (!Buffer.isBuffer(payload)) { throw new Error('Invalid buffer') diff --git a/schemaregistry/serde/rule-registry.ts b/schemaregistry/serde/rule-registry.ts index d10320f8..464962cd 100644 --- a/schemaregistry/serde/rule-registry.ts +++ b/schemaregistry/serde/rule-registry.ts @@ -1,55 +1,87 @@ import {RuleAction, RuleExecutor} from "./serde"; +/** + * + */ export class RuleRegistry { private ruleExecutors: Map = new Map() private ruleActions: Map = new Map() private static globalInstance: RuleRegistry = new RuleRegistry() - // registerExecutor is used to register a new rule executor. + /** + * registerExecutor is used to register a new rule executor. + * @param ruleExecutor - the rule executor to register + */ public registerExecutor(ruleExecutor: RuleExecutor): void { this.ruleExecutors.set(ruleExecutor.type(), ruleExecutor) } - // getExecutor fetches a rule executor by a given name. + /** + * getExecutor fetches a rule executor by a given name. + * @param name - the name of the rule executor to fetch + */ public getExecutor(name: string): RuleExecutor | undefined { return this.ruleExecutors.get(name) } - // getExecutors fetches all rule executors + /** + * getExecutors fetches all rule executors + */ public getExecutors(): RuleExecutor[] { return Array.from(this.ruleExecutors.values()) } - // registerAction is used to register a new rule action. + /** + * registerAction is used to register a new rule action. + * @param ruleAction - the rule action to register + */ public registerAction(ruleAction: RuleAction): void { this.ruleActions.set(ruleAction.type(), ruleAction) } - // getAction fetches a rule action by a given name. + /** + * getAction fetches a rule action by a given name. + * @param name - the name of the rule action to fetch + */ public getAction(name: string): RuleAction | undefined { return this.ruleActions.get(name) } - // getActions fetches all rule actions + /** + * getActions fetches all rule actions + */ public getActions(): RuleAction[] { return Array.from(this.ruleActions.values()) } - // clear clears all registered rules + /** + * clear clears all registered rules + */ public clear(): void { this.ruleExecutors.clear() this.ruleActions.clear() } + /** + * getGlobalInstance fetches the global instance of the rule registry + */ public static getGlobalInstance(): RuleRegistry { return RuleRegistry.globalInstance } + /** + * registerRuleExecutor is used to register a new rule executor globally. + * @param ruleExecutor - the rule executor to register + */ public static registerRuleExecutor(ruleExecutor: RuleExecutor): void { RuleRegistry.globalInstance.registerExecutor(ruleExecutor) } + /** + * registerRuleAction is used to register a new rule action globally. + * @param ruleAction - the rule action to register + */ public static registerRuleAction(ruleAction: RuleAction): void { RuleRegistry.globalInstance.registerAction(ruleAction) } diff --git a/schemaregistry/serde/serde.ts b/schemaregistry/serde/serde.ts index 4f2f6732..52c6eec2 100644 --- a/schemaregistry/serde/serde.ts +++ b/schemaregistry/serde/serde.ts @@ -17,6 +17,9 @@ export enum SerdeType { export const MAGIC_BYTE = Buffer.alloc(1) +/** + * SerializationError represents a serialization error + */ export class SerializationError extends Error { constructor(message?: string) { @@ -41,6 +44,9 @@ export interface SerdeConfig { export type RefResolver = (client: Client, info: SchemaInfo) => Promise> +/** + * Serde represents a serializer/deserializer + */ export abstract class Serde { client: Client serdeType: SerdeType @@ -206,6 +212,9 @@ export abstract class Serde { } } +/** + * SerializerConfig represents a serializer configuration + */ export interface SerializerConfig extends SerdeConfig { // autoRegisterSchemas determines whether to automatically register schemas autoRegisterSchemas?: boolean @@ -215,6 +224,9 @@ export interface SerializerConfig extends SerdeConfig { normalizeSchemas?: boolean } +/** + * Serializer represents a serializer + */ export abstract class Serializer extends Serde { protected constructor(client: Client, serdeType: SerdeType, conf: SerializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) @@ -224,6 +236,11 @@ export abstract class Serializer extends Serde { return this.conf as SerializerConfig } + /** + * Serialize serializes a message + * @param topic - the topic + * @param msg - the message + */ abstract serialize(topic: string, msg: any): Promise // GetID returns a schema ID for the given schema @@ -263,15 +280,23 @@ export abstract class Serializer extends Serde { } } +/** + * DeserializerConfig represents a deserializer configuration + */ export type DeserializerConfig = SerdeConfig -// Migration represents a migration +/** + * Migration represents a migration + */ export interface Migration { ruleMode: RuleMode source: SchemaMetadata | null target: SchemaMetadata | null } +/** + * Deserializer represents a deserializer + */ export abstract class Deserializer extends Serde { protected constructor(client: Client, serdeType: SerdeType, conf: DeserializerConfig, ruleRegistry?: RuleRegistry) { super(client, serdeType, conf, ruleRegistry) @@ -281,6 +306,11 @@ export abstract class Deserializer extends Serde { return this.conf as DeserializerConfig } + /** + * Deserialize deserializes a message + * @param topic - the topic + * @param payload - the payload + */ abstract deserialize(topic: string, payload: Buffer): Promise async getSchema(topic: string, payload: Buffer, format?: string): Promise { @@ -425,13 +455,20 @@ export abstract class Deserializer extends Serde { } } +/** + * SubjectNameStrategyFunc determines the subject from the given parameters + */ export type SubjectNameStrategyFunc = ( topic: string, serdeType: SerdeType, schema?: SchemaInfo, ) => string -// TopicNameStrategy creates a subject name by appending -[key|value] to the topic name. +/** + * TopicNameStrategy creates a subject name by appending -[key|value] to the topic name. + * @param topic - the topic name + * @param serdeType - the serde type + */ export const TopicNameStrategy: SubjectNameStrategyFunc = (topic: string, serdeType: SerdeType) => { let suffix = '-value' if (serdeType === SerdeType.KEY) { @@ -440,7 +477,9 @@ export const TopicNameStrategy: SubjectNameStrategyFunc = (topic: string, serdeT return topic + suffix } -// RuleContext represents a rule context +/** + * RuleContext represents a rule context + */ export class RuleContext { source: SchemaInfo | null target: SchemaInfo @@ -544,7 +583,6 @@ export class RuleContext { } } -// RuleBase represents a rule base export interface RuleBase { configure(clientConfig: ClientConfig, config: Map): void @@ -553,20 +591,28 @@ export interface RuleBase { close(): void } -// RuleExecutor represents a rule executor +/** + * RuleExecutor represents a rule executor + */ export interface RuleExecutor extends RuleBase { transform(ctx: RuleContext, msg: any): Promise } -// FieldTransformer represents a field transformer +/** + * FieldTransformer represents a field transformer + */ export type FieldTransformer = (ctx: RuleContext, fieldTransform: FieldTransform, msg: any) => any; -// FieldTransform represents a field transform +/** + * FieldTransform represents a field transform + */ export interface FieldTransform { transform(ctx: RuleContext, fieldCtx: FieldContext, fieldValue: any): Promise; } -// FieldRuleExecutor represents a field rule executor +/** + * FieldRuleExecutor represents a field rule executor + */ export abstract class FieldRuleExecutor implements RuleExecutor { config: Map | null = null @@ -616,7 +662,9 @@ function areTransformsWithSameTag(rule1: Rule, rule2: Rule): boolean { && rule1.tags === rule2.tags } -// FieldContext represents a field context +/** + * FieldContext represents a field context + */ export class FieldContext { containingMessage: any fullName: string @@ -644,7 +692,6 @@ export class FieldContext { } } -// FieldType represents the field type export enum FieldType { RECORD = 'RECORD', ENUM = 'ENUM', @@ -662,12 +709,16 @@ export enum FieldType { NULL = 'NULL', } -// RuleAction represents a rule action +/** + * RuleAction represents a rule action + */ export interface RuleAction extends RuleBase { run(ctx: RuleContext, msg: any, err: Error | null): Promise } -// ErrorAction represents an error action +/** + * ErrorAction represents an error action + */ export class ErrorAction implements RuleAction { configure(clientConfig: ClientConfig, config: Map): void { } @@ -684,7 +735,9 @@ export class ErrorAction implements RuleAction { } } -// NoneAction represents a no-op action +/** + * NoneAction represents a no-op action + */ export class NoneAction implements RuleAction { configure(clientConfig: ClientConfig, config: Map): void { } @@ -701,18 +754,30 @@ export class NoneAction implements RuleAction { } } -// RuleError represents a rule condition error +/** + * RuleError represents a rule error + */ export class RuleError extends Error { + /** + * Creates a new rule error. + * @param message - The error message. + */ constructor(message?: string) { super(message) } } -// RuleConditionError represents a rule condition error +/** + * RuleConditionError represents a rule condition error + */ export class RuleConditionError extends RuleError { rule: Rule + /** + * Creates a new rule condition error. + * @param rule - The rule. + */ constructor(rule: Rule) { super(RuleConditionError.error(rule)) this.rule = rule diff --git a/schemaregistry/serde/wildcard-matcher.ts b/schemaregistry/serde/wildcard-matcher.ts index 1635e95d..b6bc9376 100644 --- a/schemaregistry/serde/wildcard-matcher.ts +++ b/schemaregistry/serde/wildcard-matcher.ts @@ -16,9 +16,9 @@ * wildcardMatch("alice.bob.eve", "alice.bob**") --> true * * - * @param str the string to match on - * @param wildcardMatcher the wildcard string to match against - * @return true if the string matches the wildcard string + * @param str - the string to match on + * @param wildcardMatcher - the wildcard string to match against + * @returns true - if the string matches the wildcard string */ export function match(str: string, wildcardMatcher: string): boolean { let re = wildcardToRegexp(wildcardMatcher, '.') From 7d5d246088b9c5219bd1094e054f5b8d4044fa05 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 27 Sep 2024 13:12:47 -0700 Subject: [PATCH 216/224] Enhance docs (#108) * Update kafkajs README and sr example * Minor formatting --- README.md | 6 +-- examples/kafkajs/sr.js | 42 ++++++++++--------- schemaregistry/rest-service.ts | 8 ++-- schemaregistry/rules/encryption/tink/bytes.ts | 2 +- schemaregistry/serde/json-util.ts | 2 +- schemaregistry/serde/wildcard-matcher.ts | 2 +- 6 files changed, 32 insertions(+), 30 deletions(-) diff --git a/README.md b/README.md index 58b79b0e..dc7bfc0a 100644 --- a/README.md +++ b/README.md @@ -23,11 +23,9 @@ pace with core Apache Kafka and components of the [Confluent Platform](https://w This library leverages the work and concepts from two popular Apache Kafka JavaScript clients: [node-rdkafka](https://github.com/Blizzard/node-rdkafka) and [KafkaJS](https://github.com/tulios/kafkajs). The core is heavily based on the node-rdkafka library, which uses our own [librdkafka](https://github.com/confluentinc/librdkafka) library for core client functionality. However, we leverage a promisified API and a more idiomatic interface, similar to the one in KafkaJS, making it easy for developers to migrate and adopt this client depending on the patterns and interface they prefer. We're very happy to have been able to leverage the excellent work of the many authors of these libraries! -### This library is currently in limited-availability - it is supported for all usage but for the schema-registry client. +### This library is currently in limited-availability -To use **Schema Registry**, use the existing [kafkajs/confluent-schema-registry](https://github.com/kafkajs/confluent-schema-registry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). - -**DISCLAIMER:** Although it is compatible with **confluent-kafka-javascript**, Confluent does not own or maintain kafkajs/confluent-schema-registry, and the use and functionality of the library should be considered "as is". +To use **Schema Registry**, use the existing [@confluentinc/schemaregistry](https://www.npmjs.com/package/@confluentinc/schemaregistry) library that is compatible with this library. For a simple schema registry example, see [sr.js](https://github.com/confluentinc/confluent-kafka-javascript/blob/dev_early_access_development_branch/examples/kafkajs/sr.js). ## Requirements diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 1206c7c2..0de5aeda 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -1,11 +1,11 @@ // require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS. const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; -// Note: The kafkajs/confluent-schema-registry will need to be installed separately to run this example, +// Note: The @confluentinc/schemaregistry will need to be installed separately to run this example, // as it isn't a dependency of confluent-kafka-javascript. -const { SchemaRegistry, SchemaType } = require('@kafkajs/confluent-schema-registry'); +const { SchemaRegistryClient, SerdeType, AvroSerializer, AvroDeserializer} = require('@confluentinc/schemaregistry'); -const registry = new SchemaRegistry({ host: '' }) +const registry = new SchemaRegistryClient({ baseURLs: [''] }) const kafka = new Kafka({ kafkaJS: { brokers: [''], @@ -17,11 +17,12 @@ const kafka = new Kafka({ }, } }); + let consumer = kafka.consumer({ - kafkaJS: { - groupId: "test-group", - fromBeginning: true, - }, + kafkaJS: { + groupId: "test-group", + fromBeginning: true, + }, }); let producer = kafka.producer(); @@ -43,40 +44,42 @@ const schemaB = { }; const topicName = 'test-topic'; +const subjectName = topicName + '-value'; const run = async () => { // Register schemaB. await registry.register( + 'avro-b', { - type: SchemaType.AVRO, + schemaType: 'AVRO', schema: JSON.stringify(schemaB), - }, - { subject: 'Avro:B' }, + } ); - const response = await registry.api.Subject.latestVersion({ subject: 'Avro:B' }); - const { version } = JSON.parse(response.responseData); + const response = await registry.getLatestSchemaMetadata('avro-b'); + const version = response.version // Register schemaA, which references schemaB. - const { id } = await registry.register( + const id = await registry.register( + subjectName, { - type: SchemaType.AVRO, + schemaType: 'AVRO', schema: JSON.stringify(schemaA), references: [ { name: 'test.B', - subject: 'Avro:B', + subject: 'avro-b', version, }, ], - }, - { subject: 'Avro:A' }, + } ) // Produce a message with schemaA. await producer.connect() + const ser = new AvroSerializer(registry, SerdeType.VALUE, { useLatestVersion: true }); const outgoingMessage = { key: 'key', - value: await registry.encode(id, { id: 1, b: { id: 2 } }) + value: await ser.serialize(topicName, { id: 1, b: { id: 2 } }), } await producer.send({ topic: topicName, @@ -90,11 +93,12 @@ const run = async () => { await consumer.subscribe({ topic: topicName }) let messageRcvd = false; + const deser = new AvroDeserializer(registry, SerdeType.VALUE, {}); await consumer.run({ eachMessage: async ({ message }) => { const decodedMessage = { ...message, - value: await registry.decode(message.value) + value: await deser.deserialize(topicName, message.value) }; console.log("Consumer received message.\nBefore decoding: " + JSON.stringify(message) + "\nAfter decoding: " + JSON.stringify(decodedMessage)); messageRcvd = true; diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index 513967c8..c7e2b693 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -13,7 +13,7 @@ import { RestError } from './rest-error'; export interface BasicAuthCredentials { credentialsSource: 'USER_INFO' | 'URL' | 'SASL_INHERIT', userInfo?: string, - saslInfo?: SaslInfo + sasl?: SaslInfo } export interface SaslInfo { @@ -75,13 +75,13 @@ export class RestService { this.setAuth(toBase64(basicAuthCredentials.userInfo!)); break; case 'SASL_INHERIT': - if (!basicAuthCredentials.saslInfo) { + if (!basicAuthCredentials.sasl) { throw new Error('Sasl info not provided'); } - if (basicAuthCredentials.saslInfo.mechanism?.toUpperCase() === 'GSSAPI') { + if (basicAuthCredentials.sasl.mechanism?.toUpperCase() === 'GSSAPI') { throw new Error('SASL_INHERIT support PLAIN and SCRAM SASL mechanisms only'); } - this.setAuth(toBase64(`${basicAuthCredentials.saslInfo.username}:${basicAuthCredentials.saslInfo.password}`)); + this.setAuth(toBase64(`${basicAuthCredentials.sasl.username}:${basicAuthCredentials.sasl.password}`)); break; case 'URL': if (!basicAuthCredentials.userInfo) { diff --git a/schemaregistry/rules/encryption/tink/bytes.ts b/schemaregistry/rules/encryption/tink/bytes.ts index 7e08dbaf..f2aafdaf 100644 --- a/schemaregistry/rules/encryption/tink/bytes.ts +++ b/schemaregistry/rules/encryption/tink/bytes.ts @@ -130,7 +130,7 @@ export function fromBase64(encoded: string, opt_webSafe?: boolean): Uint8Array { * @param bytes - the byte array input * @param opt_webSafe - True indicates we should use the alternative * alphabet, which does not require escaping for use in URLs. - * @returns base64 - output + * @returns base64 output */ export function toBase64(bytes: Uint8Array, opt_webSafe?: boolean): string { const encoded = window diff --git a/schemaregistry/serde/json-util.ts b/schemaregistry/serde/json-util.ts index 15ec584f..35a3daef 100644 --- a/schemaregistry/serde/json-util.ts +++ b/schemaregistry/serde/json-util.ts @@ -12,7 +12,7 @@ import { deepStrictEqual } from 'assert'; * Generate JSON schema from value. * * @param value - Value. - * @returns - JSON schema. + * @returns JSON schema. */ export function generateSchema(value: any): any { switch (true) { diff --git a/schemaregistry/serde/wildcard-matcher.ts b/schemaregistry/serde/wildcard-matcher.ts index b6bc9376..37a953b8 100644 --- a/schemaregistry/serde/wildcard-matcher.ts +++ b/schemaregistry/serde/wildcard-matcher.ts @@ -18,7 +18,7 @@ * * @param str - the string to match on * @param wildcardMatcher - the wildcard string to match against - * @returns true - if the string matches the wildcard string + * @returns true if the string matches the wildcard string */ export function match(str: string, wildcardMatcher: string): boolean { let re = wildcardToRegexp(wildcardMatcher, '.') From 4e3c7152f159a6dc0c5b6719e0e55c2d9fb28b3c Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 27 Sep 2024 18:26:10 -0700 Subject: [PATCH 217/224] Update schemaregistry README (#109) --- README.md | 2 +- examples/kafkajs/sr.js | 8 +- schemaregistry/README.md | 174 ++++++++++++++++++++++++--------------- 3 files changed, 115 insertions(+), 69 deletions(-) diff --git a/README.md b/README.md index dc7bfc0a..f07ea273 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -Confluent's Javascript Client for Apache KafkaTM +Confluent's JavaScript Client for Apache KafkaTM ===================================================== **confluent-kafka-javascript** is Confluent's JavaScript client for [Apache Kafka](http://kafka.apache.org/) and the diff --git a/examples/kafkajs/sr.js b/examples/kafkajs/sr.js index 0de5aeda..a10c6293 100644 --- a/examples/kafkajs/sr.js +++ b/examples/kafkajs/sr.js @@ -74,9 +74,11 @@ const run = async () => { } ) + // Create an Avro serializer + const ser = new AvroSerializer(registry, SerdeType.VALUE, { useLatestVersion: true }); + // Produce a message with schemaA. await producer.connect() - const ser = new AvroSerializer(registry, SerdeType.VALUE, { useLatestVersion: true }); const outgoingMessage = { key: 'key', value: await ser.serialize(topicName, { id: 1, b: { id: 2 } }), @@ -89,11 +91,13 @@ const run = async () => { await producer.disconnect(); producer = null; + // Create an Avro deserializer + const deser = new AvroDeserializer(registry, SerdeType.VALUE, {}); + await consumer.connect() await consumer.subscribe({ topic: topicName }) let messageRcvd = false; - const deser = new AvroDeserializer(registry, SerdeType.VALUE, {}); await consumer.run({ eachMessage: async ({ message }) => { const decodedMessage = { diff --git a/schemaregistry/README.md b/schemaregistry/README.md index 79d0cbd9..33e8945e 100644 --- a/schemaregistry/README.md +++ b/schemaregistry/README.md @@ -1,79 +1,121 @@ -Confluent's Javascript Client for Schema RegistryTM +Confluent's JavaScript Client for Schema RegistryTM ===================================================== -**confluent-kafka-javascript** includes Confluent's JavaScript client for [Schema Registry](https://docs.confluent.io/cloud/current/sr/index.html) and the accompanying package to Confluent's Javascript Client for Apache Kafka -[Confluent's Javascript Client for Apache Kafka](https://www.npmjs.com/package/@confluentinc/kafka-javascript). This is an **Early Availability** library. The goal is to provide an highly performant, reliable and easy to use JavaScript client in line with other clients such as our [Go](https://github.com/confluentinc/confluent-kafka-go) and [Python](https://github.com/confluentinc/confluent-kafka-python) clients. - - + +## Features and Limitations +- Full Avro and JSON Schema support +- Protobuf support requires (upcoming) release: CP 7.4.8, 7.5.7, 7.6.4, 7.7.2, 7.8.0 +- Support for CSFLE (Client-Side Field Level Encryption) +- Support for schema migration rules for Avro and JSON Schema +- Data quality rules are not yet supported +- Support for OAuth ## Contributing Bug reports and feedback is appreciated in the form of Github Issues. -For guidelines on contributing please see [CONTRIBUTING.md](CONTRIBUTING.md) \ No newline at end of file +For guidelines on contributing please see [CONTRIBUTING.md](CONTRIBUTING.md) From 790b5e211c14a7e9639796b6fb61c10f8ed21b05 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Fri, 27 Sep 2024 18:55:56 -0700 Subject: [PATCH 218/224] Add restService interfaces to exported types (#110) Co-authored-by: claimundefine --- package-lock.json | 280 +++--------------------------------- schemaregistry/index.ts | 6 + schemaregistry/package.json | 3 +- 3 files changed, 31 insertions(+), 258 deletions(-) diff --git a/package-lock.json b/package-lock.json index 5decdc60..7e578110 100644 --- a/package-lock.json +++ b/package-lock.json @@ -61,7 +61,6 @@ }, "node_modules/@ampproject/remapping": { "version": "2.3.0", - "dev": true, "license": "Apache-2.0", "dependencies": { "@jridgewell/gen-mapping": "^0.3.5", @@ -956,7 +955,6 @@ }, "node_modules/@babel/code-frame": { "version": "7.24.7", - "dev": true, "license": "MIT", "dependencies": { "@babel/highlight": "^7.24.7", @@ -968,7 +966,6 @@ }, "node_modules/@babel/compat-data": { "version": "7.25.2", - "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -976,7 +973,6 @@ }, "node_modules/@babel/core": { "version": "7.25.2", - "dev": true, "license": "MIT", "dependencies": { "@ampproject/remapping": "^2.2.0", @@ -1005,7 +1001,6 @@ }, "node_modules/@babel/core/node_modules/semver": { "version": "6.3.1", - "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -1013,7 +1008,6 @@ }, "node_modules/@babel/generator": { "version": "7.25.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.25.0", @@ -1027,7 +1021,6 @@ }, "node_modules/@babel/helper-compilation-targets": { "version": "7.25.2", - "dev": true, "license": "MIT", "dependencies": { "@babel/compat-data": "^7.25.2", @@ -1042,7 +1035,6 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { "version": "5.1.1", - "dev": true, "license": "ISC", "dependencies": { "yallist": "^3.0.2" @@ -1050,7 +1042,6 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/semver": { "version": "6.3.1", - "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -1058,7 +1049,6 @@ }, "node_modules/@babel/helper-module-imports": { "version": "7.24.7", - "dev": true, "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", @@ -1070,7 +1060,6 @@ }, "node_modules/@babel/helper-module-transforms": { "version": "7.25.2", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-module-imports": "^7.24.7", @@ -1087,7 +1076,6 @@ }, "node_modules/@babel/helper-plugin-utils": { "version": "7.24.8", - "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1095,7 +1083,6 @@ }, "node_modules/@babel/helper-simple-access": { "version": "7.24.7", - "dev": true, "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", @@ -1107,7 +1094,6 @@ }, "node_modules/@babel/helper-string-parser": { "version": "7.24.8", - "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1115,7 +1101,6 @@ }, "node_modules/@babel/helper-validator-identifier": { "version": "7.24.7", - "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1123,7 +1108,6 @@ }, "node_modules/@babel/helper-validator-option": { "version": "7.24.8", - "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1131,7 +1115,6 @@ }, "node_modules/@babel/helpers": { "version": "7.25.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/template": "^7.25.0", @@ -1143,7 +1126,6 @@ }, "node_modules/@babel/highlight": { "version": "7.24.7", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-validator-identifier": "^7.24.7", @@ -1157,7 +1139,6 @@ }, "node_modules/@babel/highlight/node_modules/ansi-styles": { "version": "3.2.1", - "dev": true, "license": "MIT", "dependencies": { "color-convert": "^1.9.0" @@ -1168,7 +1149,6 @@ }, "node_modules/@babel/highlight/node_modules/chalk": { "version": "2.4.2", - "dev": true, "license": "MIT", "dependencies": { "ansi-styles": "^3.2.1", @@ -1181,7 +1161,6 @@ }, "node_modules/@babel/highlight/node_modules/color-convert": { "version": "1.9.3", - "dev": true, "license": "MIT", "dependencies": { "color-name": "1.1.3" @@ -1189,12 +1168,10 @@ }, "node_modules/@babel/highlight/node_modules/color-name": { "version": "1.1.3", - "dev": true, "license": "MIT" }, "node_modules/@babel/highlight/node_modules/escape-string-regexp": { "version": "1.0.5", - "dev": true, "license": "MIT", "engines": { "node": ">=0.8.0" @@ -1202,7 +1179,6 @@ }, "node_modules/@babel/highlight/node_modules/has-flag": { "version": "3.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -1210,7 +1186,6 @@ }, "node_modules/@babel/highlight/node_modules/supports-color": { "version": "5.5.0", - "dev": true, "license": "MIT", "dependencies": { "has-flag": "^3.0.0" @@ -1221,7 +1196,6 @@ }, "node_modules/@babel/parser": { "version": "7.25.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.25.2" @@ -1235,7 +1209,6 @@ }, "node_modules/@babel/plugin-syntax-async-generators": { "version": "7.8.4", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1246,7 +1219,6 @@ }, "node_modules/@babel/plugin-syntax-bigint": { "version": "7.8.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1257,7 +1229,6 @@ }, "node_modules/@babel/plugin-syntax-class-properties": { "version": "7.12.13", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.12.13" @@ -1268,7 +1239,6 @@ }, "node_modules/@babel/plugin-syntax-import-meta": { "version": "7.10.4", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1279,7 +1249,6 @@ }, "node_modules/@babel/plugin-syntax-json-strings": { "version": "7.8.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1290,7 +1259,6 @@ }, "node_modules/@babel/plugin-syntax-jsx": { "version": "7.24.7", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" @@ -1304,7 +1272,6 @@ }, "node_modules/@babel/plugin-syntax-logical-assignment-operators": { "version": "7.10.4", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1315,7 +1282,6 @@ }, "node_modules/@babel/plugin-syntax-nullish-coalescing-operator": { "version": "7.8.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1326,7 +1292,6 @@ }, "node_modules/@babel/plugin-syntax-numeric-separator": { "version": "7.10.4", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1337,7 +1302,6 @@ }, "node_modules/@babel/plugin-syntax-object-rest-spread": { "version": "7.8.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1348,7 +1312,6 @@ }, "node_modules/@babel/plugin-syntax-optional-catch-binding": { "version": "7.8.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1359,7 +1322,6 @@ }, "node_modules/@babel/plugin-syntax-optional-chaining": { "version": "7.8.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1370,7 +1332,6 @@ }, "node_modules/@babel/plugin-syntax-top-level-await": { "version": "7.14.5", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.14.5" @@ -1384,7 +1345,6 @@ }, "node_modules/@babel/plugin-syntax-typescript": { "version": "7.24.7", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" @@ -1398,7 +1358,6 @@ }, "node_modules/@babel/template": { "version": "7.25.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", @@ -1411,7 +1370,6 @@ }, "node_modules/@babel/traverse": { "version": "7.25.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", @@ -1428,7 +1386,6 @@ }, "node_modules/@babel/traverse/node_modules/globals": { "version": "11.12.0", - "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -1436,7 +1393,6 @@ }, "node_modules/@babel/types": { "version": "7.25.2", - "dev": true, "license": "MIT", "dependencies": { "@babel/helper-string-parser": "^7.24.8", @@ -1449,7 +1405,6 @@ }, "node_modules/@bcoe/v8-coverage": { "version": "0.2.3", - "dev": true, "license": "MIT" }, "node_modules/@bufbuild/buf": { @@ -1811,7 +1766,6 @@ }, "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", - "dev": true, "license": "ISC", "dependencies": { "camelcase": "^5.3.1", @@ -1826,7 +1780,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/argparse": { "version": "1.0.10", - "dev": true, "license": "MIT", "dependencies": { "sprintf-js": "~1.0.2" @@ -1834,7 +1787,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/find-up": { "version": "4.1.0", - "dev": true, "license": "MIT", "dependencies": { "locate-path": "^5.0.0", @@ -1846,7 +1798,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/js-yaml": { "version": "3.14.1", - "dev": true, "license": "MIT", "dependencies": { "argparse": "^1.0.7", @@ -1858,7 +1809,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/locate-path": { "version": "5.0.0", - "dev": true, "license": "MIT", "dependencies": { "p-locate": "^4.1.0" @@ -1869,7 +1819,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/p-limit": { "version": "2.3.0", - "dev": true, "license": "MIT", "dependencies": { "p-try": "^2.0.0" @@ -1883,7 +1832,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/p-locate": { "version": "4.1.0", - "dev": true, "license": "MIT", "dependencies": { "p-limit": "^2.2.0" @@ -1894,7 +1842,6 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/resolve-from": { "version": "5.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -1902,12 +1849,10 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/sprintf-js": { "version": "1.0.3", - "dev": true, "license": "BSD-3-Clause" }, "node_modules/@istanbuljs/schema": { "version": "0.1.3", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -1915,7 +1860,6 @@ }, "node_modules/@jest/console": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -1931,7 +1875,6 @@ }, "node_modules/@jest/core": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -1977,7 +1920,6 @@ }, "node_modules/@jest/environment": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/fake-timers": "^29.7.0", @@ -1991,7 +1933,6 @@ }, "node_modules/@jest/expect": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "expect": "^29.7.0", @@ -2003,7 +1944,6 @@ }, "node_modules/@jest/expect-utils": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3" @@ -2014,7 +1954,6 @@ }, "node_modules/@jest/fake-timers": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -2030,7 +1969,6 @@ }, "node_modules/@jest/globals": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -2044,7 +1982,6 @@ }, "node_modules/@jest/reporters": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@bcoe/v8-coverage": "^0.2.3", @@ -2086,7 +2023,6 @@ }, "node_modules/@jest/schemas": { "version": "29.6.3", - "dev": true, "license": "MIT", "dependencies": { "@sinclair/typebox": "^0.27.8" @@ -2097,7 +2033,6 @@ }, "node_modules/@jest/source-map": { "version": "29.6.3", - "dev": true, "license": "MIT", "dependencies": { "@jridgewell/trace-mapping": "^0.3.18", @@ -2110,7 +2045,6 @@ }, "node_modules/@jest/test-result": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -2124,7 +2058,6 @@ }, "node_modules/@jest/test-sequencer": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", @@ -2138,7 +2071,6 @@ }, "node_modules/@jest/transform": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -2163,7 +2095,6 @@ }, "node_modules/@jest/types": { "version": "29.6.3", - "dev": true, "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", @@ -2179,7 +2110,6 @@ }, "node_modules/@jridgewell/gen-mapping": { "version": "0.3.5", - "dev": true, "license": "MIT", "dependencies": { "@jridgewell/set-array": "^1.2.1", @@ -2192,7 +2122,6 @@ }, "node_modules/@jridgewell/resolve-uri": { "version": "3.1.2", - "dev": true, "license": "MIT", "engines": { "node": ">=6.0.0" @@ -2200,7 +2129,6 @@ }, "node_modules/@jridgewell/set-array": { "version": "1.2.1", - "dev": true, "license": "MIT", "engines": { "node": ">=6.0.0" @@ -2208,12 +2136,10 @@ }, "node_modules/@jridgewell/sourcemap-codec": { "version": "1.5.0", - "dev": true, "license": "MIT" }, "node_modules/@jridgewell/trace-mapping": { "version": "0.3.25", - "dev": true, "license": "MIT", "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -2455,12 +2381,10 @@ }, "node_modules/@sinclair/typebox": { "version": "0.27.8", - "dev": true, "license": "MIT" }, "node_modules/@sinonjs/commons": { "version": "3.0.1", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "type-detect": "4.0.8" @@ -2468,7 +2392,6 @@ }, "node_modules/@sinonjs/fake-timers": { "version": "10.3.0", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "@sinonjs/commons": "^3.0.0" @@ -2958,7 +2881,6 @@ }, "node_modules/@types/babel__core": { "version": "7.20.5", - "dev": true, "license": "MIT", "dependencies": { "@babel/parser": "^7.20.7", @@ -2970,7 +2892,6 @@ }, "node_modules/@types/babel__generator": { "version": "7.6.8", - "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.0.0" @@ -2978,7 +2899,6 @@ }, "node_modules/@types/babel__template": { "version": "7.4.4", - "dev": true, "license": "MIT", "dependencies": { "@babel/parser": "^7.1.0", @@ -2987,7 +2907,6 @@ }, "node_modules/@types/babel__traverse": { "version": "7.20.6", - "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.20.7" @@ -3021,7 +2940,6 @@ }, "node_modules/@types/graceful-fs": { "version": "4.1.9", - "dev": true, "license": "MIT", "dependencies": { "@types/node": "*" @@ -3029,12 +2947,10 @@ }, "node_modules/@types/istanbul-lib-coverage": { "version": "2.0.6", - "dev": true, "license": "MIT" }, "node_modules/@types/istanbul-lib-report": { "version": "3.0.3", - "dev": true, "license": "MIT", "dependencies": { "@types/istanbul-lib-coverage": "*" @@ -3042,7 +2958,6 @@ }, "node_modules/@types/istanbul-reports": { "version": "3.0.4", - "dev": true, "license": "MIT", "dependencies": { "@types/istanbul-lib-report": "*" @@ -3130,20 +3045,25 @@ }, "node_modules/@types/stack-utils": { "version": "2.0.3", - "dev": true, "license": "MIT" }, "node_modules/@types/tough-cookie": { "version": "4.0.5", "license": "MIT" }, + "node_modules/@types/uuid": { + "version": "10.0.0", + "resolved": "https://registry.npmjs.org/@types/uuid/-/uuid-10.0.0.tgz", + "integrity": "sha512-7gqG38EyHgyP1S+7+xomFtL+ZNHcKv6DwNaCZmJmo1vgMugyF3TCnXVg4t1uk89mLNwnLtnY3TpOpCOyp1/xHQ==", + "dev": true, + "license": "MIT" + }, "node_modules/@types/validator": { "version": "13.12.0", "license": "MIT" }, "node_modules/@types/yargs": { "version": "17.0.32", - "dev": true, "license": "MIT", "dependencies": { "@types/yargs-parser": "*" @@ -3151,7 +3071,6 @@ }, "node_modules/@types/yargs-parser": { "version": "21.0.3", - "dev": true, "license": "MIT" }, "node_modules/@typescript-eslint/eslint-plugin": { @@ -3834,7 +3753,6 @@ }, "node_modules/ansi-escapes": { "version": "4.3.2", - "dev": true, "license": "MIT", "dependencies": { "type-fest": "^0.21.3" @@ -3848,7 +3766,6 @@ }, "node_modules/ansi-escapes/node_modules/type-fest": { "version": "0.21.3", - "dev": true, "license": "(MIT OR CC0-1.0)", "engines": { "node": ">=10" @@ -3879,7 +3796,6 @@ }, "node_modules/anymatch": { "version": "3.1.3", - "dev": true, "license": "ISC", "dependencies": { "normalize-path": "^3.0.0", @@ -3933,7 +3849,6 @@ }, "node_modules/async": { "version": "3.2.5", - "dev": true, "license": "MIT" }, "node_modules/async-mutex": { @@ -3978,7 +3893,6 @@ }, "node_modules/babel-jest": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/transform": "^29.7.0", @@ -3998,7 +3912,6 @@ }, "node_modules/babel-plugin-istanbul": { "version": "6.1.1", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/helper-plugin-utils": "^7.0.0", @@ -4013,7 +3926,6 @@ }, "node_modules/babel-plugin-istanbul/node_modules/istanbul-lib-instrument": { "version": "5.2.1", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.12.3", @@ -4028,7 +3940,6 @@ }, "node_modules/babel-plugin-istanbul/node_modules/semver": { "version": "6.3.1", - "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -4036,7 +3947,6 @@ }, "node_modules/babel-plugin-jest-hoist": { "version": "29.6.3", - "dev": true, "license": "MIT", "dependencies": { "@babel/template": "^7.3.3", @@ -4050,7 +3960,6 @@ }, "node_modules/babel-preset-current-node-syntax": { "version": "1.0.1", - "dev": true, "license": "MIT", "dependencies": { "@babel/plugin-syntax-async-generators": "^7.8.4", @@ -4072,7 +3981,6 @@ }, "node_modules/babel-preset-jest": { "version": "29.6.3", - "dev": true, "license": "MIT", "dependencies": { "babel-plugin-jest-hoist": "^29.6.3", @@ -4158,7 +4066,6 @@ }, "node_modules/braces": { "version": "3.0.3", - "dev": true, "license": "MIT", "dependencies": { "fill-range": "^7.1.1" @@ -4181,7 +4088,6 @@ }, "node_modules/browserslist": { "version": "4.23.3", - "dev": true, "funding": [ { "type": "opencollective", @@ -4212,7 +4118,6 @@ }, "node_modules/bs-logger": { "version": "0.2.6", - "dev": true, "license": "MIT", "dependencies": { "fast-json-stable-stringify": "2.x" @@ -4223,7 +4128,6 @@ }, "node_modules/bser": { "version": "2.1.1", - "dev": true, "license": "Apache-2.0", "dependencies": { "node-int64": "^0.4.0" @@ -4235,7 +4139,6 @@ }, "node_modules/buffer-from": { "version": "1.1.2", - "dev": true, "license": "MIT" }, "node_modules/cacache": { @@ -4313,7 +4216,6 @@ }, "node_modules/callsites": { "version": "3.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -4321,7 +4223,6 @@ }, "node_modules/camelcase": { "version": "5.3.1", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -4329,7 +4230,6 @@ }, "node_modules/caniuse-lite": { "version": "1.0.30001646", - "dev": true, "funding": [ { "type": "opencollective", @@ -4363,7 +4263,6 @@ }, "node_modules/chalk": { "version": "4.1.2", - "dev": true, "license": "MIT", "dependencies": { "ansi-styles": "^4.1.0", @@ -4378,7 +4277,6 @@ }, "node_modules/char-regex": { "version": "1.0.2", - "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -4427,7 +4325,6 @@ }, "node_modules/ci-info": { "version": "3.9.0", - "dev": true, "funding": [ { "type": "github", @@ -4441,7 +4338,6 @@ }, "node_modules/cjs-module-lexer": { "version": "1.3.1", - "dev": true, "license": "MIT" }, "node_modules/clean-stack": { @@ -4466,7 +4362,6 @@ }, "node_modules/co": { "version": "4.6.0", - "dev": true, "license": "MIT", "engines": { "iojs": ">= 1.0.0", @@ -4475,7 +4370,6 @@ }, "node_modules/collect-v8-coverage": { "version": "1.0.2", - "dev": true, "license": "MIT" }, "node_modules/color-convert": { @@ -4523,7 +4417,6 @@ }, "node_modules/convert-source-map": { "version": "2.0.0", - "dev": true, "license": "MIT" }, "node_modules/core-util-is": { @@ -4532,7 +4425,6 @@ }, "node_modules/create-jest": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -4552,7 +4444,6 @@ }, "node_modules/cross-spawn": { "version": "7.0.3", - "dev": true, "license": "MIT", "dependencies": { "path-key": "^3.1.0", @@ -4601,7 +4492,6 @@ }, "node_modules/dedent": { "version": "1.5.3", - "dev": true, "license": "MIT", "peerDependencies": { "babel-plugin-macros": "^3.1.0" @@ -4619,7 +4509,6 @@ }, "node_modules/deepmerge": { "version": "4.3.1", - "dev": true, "license": "MIT", "engines": { "node": ">=0.10.0" @@ -4652,7 +4541,6 @@ }, "node_modules/detect-newline": { "version": "3.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -4668,7 +4556,6 @@ }, "node_modules/diff-sequences": { "version": "29.6.3", - "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -4731,7 +4618,6 @@ }, "node_modules/ejs": { "version": "3.1.10", - "dev": true, "license": "Apache-2.0", "dependencies": { "jake": "^10.8.5" @@ -4745,12 +4631,10 @@ }, "node_modules/electron-to-chromium": { "version": "1.5.4", - "dev": true, "license": "ISC" }, "node_modules/emittery": { "version": "0.13.1", - "dev": true, "license": "MIT", "engines": { "node": ">=12" @@ -4804,7 +4688,6 @@ }, "node_modules/error-ex": { "version": "1.3.2", - "dev": true, "license": "MIT", "dependencies": { "is-arrayish": "^0.2.1" @@ -4988,7 +4871,6 @@ }, "node_modules/esprima": { "version": "4.0.1", - "dev": true, "license": "BSD-2-Clause", "bin": { "esparse": "bin/esparse.js", @@ -5052,7 +4934,6 @@ }, "node_modules/execa": { "version": "5.1.1", - "dev": true, "license": "MIT", "dependencies": { "cross-spawn": "^7.0.3", @@ -5074,14 +4955,12 @@ }, "node_modules/exit": { "version": "0.1.2", - "dev": true, "engines": { "node": ">= 0.8.0" } }, "node_modules/expect": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/expect-utils": "^29.7.0", @@ -5183,7 +5062,6 @@ }, "node_modules/fb-watchman": { "version": "2.0.2", - "dev": true, "license": "Apache-2.0", "dependencies": { "bser": "2.1.1" @@ -5206,7 +5084,6 @@ }, "node_modules/filelist": { "version": "1.0.4", - "dev": true, "license": "Apache-2.0", "dependencies": { "minimatch": "^5.0.1" @@ -5214,7 +5091,6 @@ }, "node_modules/filelist/node_modules/brace-expansion": { "version": "2.0.1", - "dev": true, "license": "MIT", "dependencies": { "balanced-match": "^1.0.0" @@ -5222,7 +5098,6 @@ }, "node_modules/filelist/node_modules/minimatch": { "version": "5.1.6", - "dev": true, "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" @@ -5233,7 +5108,6 @@ }, "node_modules/fill-range": { "version": "7.1.1", - "dev": true, "license": "MIT", "dependencies": { "to-regex-range": "^5.0.1" @@ -5336,7 +5210,6 @@ }, "node_modules/fsevents": { "version": "2.3.3", - "dev": true, "license": "MIT", "optional": true, "os": [ @@ -5348,7 +5221,6 @@ }, "node_modules/function-bind": { "version": "1.1.2", - "dev": true, "license": "MIT", "funding": { "url": "https://github.com/sponsors/ljharb" @@ -5420,7 +5292,6 @@ }, "node_modules/gensync": { "version": "1.0.0-beta.2", - "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -5435,7 +5306,6 @@ }, "node_modules/get-package-type": { "version": "0.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8.0.0" @@ -5443,7 +5313,6 @@ }, "node_modules/get-stream": { "version": "6.0.1", - "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -5559,7 +5428,6 @@ }, "node_modules/graceful-fs": { "version": "4.2.11", - "dev": true, "license": "ISC" }, "node_modules/graphemer": { @@ -5616,7 +5484,6 @@ }, "node_modules/has-flag": { "version": "4.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -5628,7 +5495,6 @@ }, "node_modules/hasown": { "version": "2.0.2", - "dev": true, "license": "MIT", "dependencies": { "function-bind": "^1.1.2" @@ -5647,7 +5513,6 @@ }, "node_modules/html-escaper": { "version": "2.0.2", - "dev": true, "license": "MIT" }, "node_modules/http-cache-semantics": { @@ -5692,7 +5557,6 @@ }, "node_modules/human-signals": { "version": "2.1.0", - "dev": true, "license": "Apache-2.0", "engines": { "node": ">=10.17.0" @@ -5742,7 +5606,6 @@ }, "node_modules/import-local": { "version": "3.2.0", - "dev": true, "license": "MIT", "dependencies": { "pkg-dir": "^4.2.0", @@ -5760,7 +5623,6 @@ }, "node_modules/imurmurhash": { "version": "0.1.4", - "dev": true, "license": "MIT", "engines": { "node": ">=0.8.19" @@ -5805,7 +5667,6 @@ }, "node_modules/is-arrayish": { "version": "0.2.1", - "dev": true, "license": "MIT" }, "node_modules/is-binary-path": { @@ -5821,7 +5682,6 @@ }, "node_modules/is-core-module": { "version": "2.15.0", - "dev": true, "license": "MIT", "dependencies": { "hasown": "^2.0.2" @@ -5863,7 +5723,6 @@ }, "node_modules/is-generator-fn": { "version": "2.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -5887,7 +5746,6 @@ }, "node_modules/is-number": { "version": "7.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=0.12.0" @@ -5946,7 +5804,6 @@ }, "node_modules/isexe": { "version": "2.0.0", - "dev": true, "license": "ISC" }, "node_modules/isstream": { @@ -5955,7 +5812,6 @@ }, "node_modules/istanbul-lib-coverage": { "version": "3.2.2", - "dev": true, "license": "BSD-3-Clause", "engines": { "node": ">=8" @@ -5963,7 +5819,6 @@ }, "node_modules/istanbul-lib-instrument": { "version": "6.0.3", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.23.9", @@ -5978,7 +5833,6 @@ }, "node_modules/istanbul-lib-report": { "version": "3.0.1", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "istanbul-lib-coverage": "^3.0.0", @@ -5991,7 +5845,6 @@ }, "node_modules/istanbul-lib-report/node_modules/make-dir": { "version": "4.0.0", - "dev": true, "license": "MIT", "dependencies": { "semver": "^7.5.3" @@ -6005,7 +5858,6 @@ }, "node_modules/istanbul-lib-source-maps": { "version": "4.0.1", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "debug": "^4.1.1", @@ -6018,7 +5870,6 @@ }, "node_modules/istanbul-reports": { "version": "3.1.7", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "html-escaper": "^2.0.0", @@ -6030,7 +5881,6 @@ }, "node_modules/jake": { "version": "10.9.2", - "dev": true, "license": "Apache-2.0", "dependencies": { "async": "^3.2.3", @@ -6047,7 +5897,6 @@ }, "node_modules/jest": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", @@ -6072,7 +5921,6 @@ }, "node_modules/jest-changed-files": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "execa": "^5.0.0", @@ -6085,7 +5933,6 @@ }, "node_modules/jest-circus": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6115,7 +5962,6 @@ }, "node_modules/jest-cli": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", @@ -6147,7 +5993,6 @@ }, "node_modules/jest-config": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -6191,7 +6036,6 @@ }, "node_modules/jest-diff": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6205,7 +6049,6 @@ }, "node_modules/jest-docblock": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "detect-newline": "^3.0.0" @@ -6216,7 +6059,6 @@ }, "node_modules/jest-each": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6231,7 +6073,6 @@ }, "node_modules/jest-environment-node": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6247,7 +6088,6 @@ }, "node_modules/jest-get-type": { "version": "29.6.3", - "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -6255,7 +6095,6 @@ }, "node_modules/jest-haste-map": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6279,7 +6118,6 @@ }, "node_modules/jest-leak-detector": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3", @@ -6291,7 +6129,6 @@ }, "node_modules/jest-matcher-utils": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6305,7 +6142,6 @@ }, "node_modules/jest-message-util": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.12.13", @@ -6324,7 +6160,6 @@ }, "node_modules/jest-mock": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6337,7 +6172,6 @@ }, "node_modules/jest-pnp-resolver": { "version": "1.2.3", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6353,7 +6187,6 @@ }, "node_modules/jest-regex-util": { "version": "29.6.3", - "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -6361,7 +6194,6 @@ }, "node_modules/jest-resolve": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6380,7 +6212,6 @@ }, "node_modules/jest-resolve-dependencies": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "jest-regex-util": "^29.6.3", @@ -6392,7 +6223,6 @@ }, "node_modules/jest-runner": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -6423,7 +6253,6 @@ }, "node_modules/jest-runtime": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6455,7 +6284,6 @@ }, "node_modules/jest-snapshot": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -6485,7 +6313,6 @@ }, "node_modules/jest-util": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6501,7 +6328,6 @@ }, "node_modules/jest-validate": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6517,7 +6343,6 @@ }, "node_modules/jest-validate/node_modules/camelcase": { "version": "6.3.0", - "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -6528,7 +6353,6 @@ }, "node_modules/jest-watcher": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", @@ -6546,7 +6370,6 @@ }, "node_modules/jest-worker": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@types/node": "*", @@ -6560,7 +6383,6 @@ }, "node_modules/jest-worker/node_modules/supports-color": { "version": "8.1.1", - "dev": true, "license": "MIT", "dependencies": { "has-flag": "^4.0.0" @@ -6599,7 +6421,6 @@ }, "node_modules/js-tokens": { "version": "4.0.0", - "dev": true, "license": "MIT" }, "node_modules/js-yaml": { @@ -6664,7 +6485,6 @@ }, "node_modules/jsesc": { "version": "2.5.2", - "dev": true, "license": "MIT", "bin": { "jsesc": "bin/jsesc" @@ -6687,7 +6507,6 @@ }, "node_modules/json-parse-even-better-errors": { "version": "2.3.1", - "dev": true, "license": "MIT" }, "node_modules/json-schema": { @@ -6716,7 +6535,6 @@ }, "node_modules/json5": { "version": "2.2.3", - "dev": true, "license": "MIT", "bin": { "json5": "lib/cli.js" @@ -6817,7 +6635,6 @@ }, "node_modules/kleur": { "version": "3.0.3", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6825,7 +6642,6 @@ }, "node_modules/leven": { "version": "3.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6845,7 +6661,6 @@ }, "node_modules/lines-and-columns": { "version": "1.2.4", - "dev": true, "license": "MIT" }, "node_modules/linkify-it": { @@ -6905,7 +6720,6 @@ }, "node_modules/lodash.memoize": { "version": "4.1.2", - "dev": true, "license": "MIT" }, "node_modules/lodash.merge": { @@ -6965,7 +6779,6 @@ }, "node_modules/make-error": { "version": "1.3.6", - "dev": true, "license": "ISC" }, "node_modules/make-fetch-happen": { @@ -7004,7 +6817,6 @@ }, "node_modules/makeerror": { "version": "1.0.12", - "dev": true, "license": "BSD-3-Clause", "dependencies": { "tmpl": "1.0.5" @@ -7053,7 +6865,6 @@ }, "node_modules/merge-stream": { "version": "2.0.0", - "dev": true, "license": "MIT" }, "node_modules/merge2": { @@ -7068,7 +6879,6 @@ "version": "4.0.8", "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz", "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==", - "dev": true, "license": "MIT", "dependencies": { "braces": "^3.0.3", @@ -7097,7 +6907,6 @@ }, "node_modules/mimic-fn": { "version": "2.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -7350,7 +7159,6 @@ }, "node_modules/natural-compare": { "version": "1.4.0", - "dev": true, "license": "MIT" }, "node_modules/nearley": { @@ -7483,12 +7291,10 @@ }, "node_modules/node-int64": { "version": "0.4.0", - "dev": true, "license": "MIT" }, "node_modules/node-releases": { "version": "2.0.18", - "dev": true, "license": "MIT" }, "node_modules/node-vault": { @@ -7519,7 +7325,6 @@ }, "node_modules/normalize-path": { "version": "3.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=0.10.0" @@ -7527,7 +7332,6 @@ }, "node_modules/npm-run-path": { "version": "4.0.1", - "dev": true, "license": "MIT", "dependencies": { "path-key": "^3.0.0" @@ -7576,7 +7380,6 @@ }, "node_modules/onetime": { "version": "5.1.2", - "dev": true, "license": "MIT", "dependencies": { "mimic-fn": "^2.1.0" @@ -7621,7 +7424,6 @@ }, "node_modules/p-limit": { "version": "3.1.0", - "dev": true, "license": "MIT", "dependencies": { "yocto-queue": "^0.1.0" @@ -7663,7 +7465,6 @@ }, "node_modules/p-try": { "version": "2.2.0", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -7682,7 +7483,6 @@ }, "node_modules/parse-json": { "version": "5.2.0", - "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.0.0", @@ -7699,7 +7499,6 @@ }, "node_modules/path-exists": { "version": "4.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7714,7 +7513,6 @@ }, "node_modules/path-key": { "version": "3.1.1", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7722,7 +7520,6 @@ }, "node_modules/path-parse": { "version": "1.0.7", - "dev": true, "license": "MIT" }, "node_modules/path-type": { @@ -7739,12 +7536,10 @@ }, "node_modules/picocolors": { "version": "1.0.1", - "dev": true, "license": "ISC" }, "node_modules/picomatch": { "version": "2.3.1", - "dev": true, "license": "MIT", "engines": { "node": ">=8.6" @@ -7755,7 +7550,6 @@ }, "node_modules/pirates": { "version": "4.0.6", - "dev": true, "license": "MIT", "engines": { "node": ">= 6" @@ -7763,7 +7557,6 @@ }, "node_modules/pkg-dir": { "version": "4.2.0", - "dev": true, "license": "MIT", "dependencies": { "find-up": "^4.0.0" @@ -7774,7 +7567,6 @@ }, "node_modules/pkg-dir/node_modules/find-up": { "version": "4.1.0", - "dev": true, "license": "MIT", "dependencies": { "locate-path": "^5.0.0", @@ -7786,7 +7578,6 @@ }, "node_modules/pkg-dir/node_modules/locate-path": { "version": "5.0.0", - "dev": true, "license": "MIT", "dependencies": { "p-locate": "^4.1.0" @@ -7797,7 +7588,6 @@ }, "node_modules/pkg-dir/node_modules/p-limit": { "version": "2.3.0", - "dev": true, "license": "MIT", "dependencies": { "p-try": "^2.0.0" @@ -7811,7 +7601,6 @@ }, "node_modules/pkg-dir/node_modules/p-locate": { "version": "4.1.0", - "dev": true, "license": "MIT", "dependencies": { "p-limit": "^2.2.0" @@ -7868,7 +7657,6 @@ }, "node_modules/pretty-format": { "version": "29.7.0", - "dev": true, "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", @@ -7881,7 +7669,6 @@ }, "node_modules/pretty-format/node_modules/ansi-styles": { "version": "5.2.0", - "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -7909,7 +7696,6 @@ }, "node_modules/prompts": { "version": "2.4.2", - "dev": true, "license": "MIT", "dependencies": { "kleur": "^3.0.3", @@ -7976,7 +7762,6 @@ }, "node_modules/pure-rand": { "version": "6.1.0", - "dev": true, "funding": [ { "type": "individual", @@ -8044,7 +7829,6 @@ }, "node_modules/react-is": { "version": "18.3.1", - "dev": true, "license": "MIT" }, "node_modules/readable-stream": { @@ -8098,7 +7882,6 @@ }, "node_modules/resolve": { "version": "1.22.8", - "dev": true, "license": "MIT", "dependencies": { "is-core-module": "^2.13.0", @@ -8114,7 +7897,6 @@ }, "node_modules/resolve-cwd": { "version": "3.0.0", - "dev": true, "license": "MIT", "dependencies": { "resolve-from": "^5.0.0" @@ -8125,7 +7907,6 @@ }, "node_modules/resolve-cwd/node_modules/resolve-from": { "version": "5.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8141,7 +7922,6 @@ }, "node_modules/resolve.exports": { "version": "2.0.2", - "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -8264,7 +8044,6 @@ }, "node_modules/shebang-command": { "version": "2.0.0", - "dev": true, "license": "MIT", "dependencies": { "shebang-regex": "^3.0.0" @@ -8275,7 +8054,6 @@ }, "node_modules/shebang-regex": { "version": "3.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8299,12 +8077,10 @@ }, "node_modules/sisteransi": { "version": "1.0.5", - "dev": true, "license": "MIT" }, "node_modules/slash": { "version": "3.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8357,7 +8133,6 @@ }, "node_modules/source-map": { "version": "0.6.1", - "dev": true, "license": "BSD-3-Clause", "engines": { "node": ">=0.10.0" @@ -8365,7 +8140,6 @@ }, "node_modules/source-map-support": { "version": "0.5.13", - "dev": true, "license": "MIT", "dependencies": { "buffer-from": "^1.0.0", @@ -8417,7 +8191,6 @@ }, "node_modules/stack-utils": { "version": "2.0.6", - "dev": true, "license": "MIT", "dependencies": { "escape-string-regexp": "^2.0.0" @@ -8428,7 +8201,6 @@ }, "node_modules/stack-utils/node_modules/escape-string-regexp": { "version": "2.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8473,7 +8245,6 @@ }, "node_modules/string-length": { "version": "4.0.2", - "dev": true, "license": "MIT", "dependencies": { "char-regex": "^1.0.2", @@ -8507,7 +8278,6 @@ }, "node_modules/strip-bom": { "version": "4.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8515,7 +8285,6 @@ }, "node_modules/strip-final-newline": { "version": "2.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -8523,7 +8292,6 @@ }, "node_modules/strip-json-comments": { "version": "3.1.1", - "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8542,7 +8310,6 @@ }, "node_modules/supports-color": { "version": "7.2.0", - "dev": true, "license": "MIT", "dependencies": { "has-flag": "^4.0.0" @@ -8553,7 +8320,6 @@ }, "node_modules/supports-preserve-symlinks-flag": { "version": "1.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">= 0.4" @@ -8604,7 +8370,6 @@ }, "node_modules/test-exclude": { "version": "6.0.0", - "dev": true, "license": "ISC", "dependencies": { "@istanbuljs/schema": "^0.1.2", @@ -8622,12 +8387,10 @@ }, "node_modules/tmpl": { "version": "1.0.5", - "dev": true, "license": "BSD-3-Clause" }, "node_modules/to-fast-properties": { "version": "2.0.0", - "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -8635,7 +8398,6 @@ }, "node_modules/to-regex-range": { "version": "5.0.1", - "dev": true, "license": "MIT", "dependencies": { "is-number": "^7.0.0" @@ -8670,7 +8432,6 @@ "version": "29.2.5", "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.5.tgz", "integrity": "sha512-KD8zB2aAZrcKIdGk4OwpJggeLcH1FgrICqDSROWqlnJXGCXK4Mn6FcdK2B6670Xr73lHMG1kHw8R87A0ecZ+vA==", - "dev": true, "dependencies": { "bs-logger": "^0.2.6", "ejs": "^3.1.10", @@ -8716,7 +8477,6 @@ }, "node_modules/ts-jest/node_modules/yargs-parser": { "version": "21.1.1", - "dev": true, "license": "ISC", "engines": { "node": ">=12" @@ -8759,7 +8519,6 @@ }, "node_modules/type-detect": { "version": "4.0.8", - "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -8778,7 +8537,6 @@ }, "node_modules/typescript": { "version": "5.5.4", - "dev": true, "license": "Apache-2.0", "bin": { "tsc": "bin/tsc", @@ -9051,7 +8809,6 @@ }, "node_modules/update-browserslist-db": { "version": "1.1.0", - "dev": true, "funding": [ { "type": "opencollective", @@ -9110,7 +8867,6 @@ }, "node_modules/v8-to-istanbul": { "version": "9.3.0", - "dev": true, "license": "ISC", "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", @@ -9142,7 +8898,6 @@ }, "node_modules/walker": { "version": "1.0.8", - "dev": true, "license": "Apache-2.0", "dependencies": { "makeerror": "1.0.12" @@ -9162,7 +8917,6 @@ }, "node_modules/which": { "version": "2.0.2", - "dev": true, "license": "ISC", "dependencies": { "isexe": "^2.0.0" @@ -9215,7 +8969,6 @@ }, "node_modules/write-file-atomic": { "version": "4.0.2", - "dev": true, "license": "ISC", "dependencies": { "imurmurhash": "^0.1.4", @@ -9239,7 +8992,6 @@ }, "node_modules/yallist": { "version": "3.1.1", - "dev": true, "license": "ISC" }, "node_modules/yargs": { @@ -9300,7 +9052,6 @@ }, "node_modules/yocto-queue": { "version": "0.1.0", - "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -9311,7 +9062,7 @@ }, "schemaregistry": { "name": "@confluentinc/schemaregistry", - "version": "v0.1.17-devel", + "version": "v0.1.17.4-devel", "license": "MIT", "dependencies": { "@aws-sdk/client-kms": "^3.637.0", @@ -9334,6 +9085,7 @@ "lru-cache": "^11.0.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", + "uuid": "^10.0.0", "validator": "^13.12.0" }, "devDependencies": { @@ -9342,6 +9094,7 @@ "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", "@types/node": "^20.16.1", + "@types/uuid": "^10.0.0", "bluebird": "^3.5.3", "eslint": "^8.57.0", "eslint-plugin-jest": "^28.6.0", @@ -9354,6 +9107,19 @@ "typescript": "^5.5.4", "typescript-eslint": "^8.2.0" } + }, + "schemaregistry/node_modules/uuid": { + "version": "10.0.0", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-10.0.0.tgz", + "integrity": "sha512-8XkAphELsDnEGrDxUOHB3RGvXz6TeuYSGEZBOjtTtPm2lwhGBjLgOzLHB63IUWfBpNucQjND6d3AOudO+H3RWQ==", + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "license": "MIT", + "bin": { + "uuid": "dist/bin/uuid" + } } } } diff --git a/schemaregistry/index.ts b/schemaregistry/index.ts index 27389c12..9762ac24 100644 --- a/schemaregistry/index.ts +++ b/schemaregistry/index.ts @@ -16,3 +16,9 @@ export * from './serde/serde' export * from './rest-error' export * from './mock-schemaregistry-client' export * from './schemaregistry-client' +export { + BasicAuthCredentials, + BearerAuthCredentials, + ClientConfig, + SaslInfo +} from './rest-service'; diff --git a/schemaregistry/package.json b/schemaregistry/package.json index 4b0cb768..07909e84 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "v0.1.17.3-devel", + "version": "v0.1.17.4-devel", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts", @@ -14,6 +14,7 @@ "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", "@types/node": "^20.16.1", + "@types/uuid": "^10.0.0", "bluebird": "^3.5.3", "eslint": "^8.57.0", "eslint-plugin-jest": "^28.6.0", From 251a10a6dfc288067f5862587a8cc6d572e3fd16 Mon Sep 17 00:00:00 2001 From: Robert Yokota Date: Mon, 30 Sep 2024 09:34:12 -0700 Subject: [PATCH 219/224] Rename DekClient to avoid conflict with Client (#112) * Minor cleanup * Rename DekClient --- schemaregistry/README.md | 4 ++-- schemaregistry/index.ts | 1 + .../rules/encryption/dekregistry/dekregistry-client.ts | 8 ++++---- .../encryption/dekregistry/mock-dekregistry-client.ts | 4 ++-- schemaregistry/rules/encryption/encrypt-executor.ts | 4 ++-- schemaregistry/serde/rule-registry.ts | 2 +- 6 files changed, 12 insertions(+), 11 deletions(-) diff --git a/schemaregistry/README.md b/schemaregistry/README.md index 33e8945e..7b840662 100644 --- a/schemaregistry/README.md +++ b/schemaregistry/README.md @@ -12,7 +12,7 @@ and [Java](https://github.com/confluentinc/schema-registry) clients. npm install @confluentinc/schemaregistry ``` -# Getting Started +## Getting Started Below is a simple example of using Avro serialization with the Schema Registry client and the KafkaJS client. ```javascript const { Kafka } = require('@confluentinc/kafka-javascript').KafkaJS; @@ -109,7 +109,7 @@ run().catch (async e => { ## Features and Limitations - Full Avro and JSON Schema support -- Protobuf support requires (upcoming) release: CP 7.4.8, 7.5.7, 7.6.4, 7.7.2, 7.8.0 +- Protobuf support requires Schema Registry in (upcoming) release: CP 7.4.8, 7.5.7, 7.6.4, 7.7.2, 7.8.0 - Support for CSFLE (Client-Side Field Level Encryption) - Support for schema migration rules for Avro and JSON Schema - Data quality rules are not yet supported diff --git a/schemaregistry/index.ts b/schemaregistry/index.ts index 9762ac24..a3bf6961 100644 --- a/schemaregistry/index.ts +++ b/schemaregistry/index.ts @@ -2,6 +2,7 @@ export * from './confluent/types/decimal_pb' export * from './confluent/meta_pb' export * from './rules/encryption/awskms/aws-driver' export * from './rules/encryption/azurekms/azure-driver' +export * from './rules/encryption/dekregistry/dekregistry-client' export * from './rules/encryption/gcpkms/gcp-driver' export * from './rules/encryption/hcvault/hcvault-driver' export * from './rules/encryption/localkms/local-driver' diff --git a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts index dff910fe..16fea683 100644 --- a/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/dekregistry-client.ts @@ -46,7 +46,7 @@ interface Dek { deleted?: boolean; } -interface Client { +interface DekClient { registerKek(name: string, kmsType: string, kmsKeyId: string, shared: boolean, kmsProps?: { [key: string]: string }, doc?: string): Promise; getKek(name: string, deleted: boolean): Promise; @@ -56,7 +56,7 @@ interface Client { close(): Promise; } -class DekRegistryClient implements Client { +class DekRegistryClient implements DekClient { private restService: RestService; private kekCache: LRUCache; private dekCache: LRUCache; @@ -78,7 +78,7 @@ class DekRegistryClient implements Client { this.dekMutex = new Mutex(); } - static newClient(config: ClientConfig): Client { + static newClient(config: ClientConfig): DekClient { const url = config.baseURLs[0]; if (url.startsWith("mock://")) { return new MockDekRegistryClient() @@ -242,5 +242,5 @@ class DekRegistryClient implements Client { } } -export { DekRegistryClient, Client, Kek, Dek }; +export { DekRegistryClient, DekClient, Kek, Dek }; diff --git a/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts b/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts index 46f20219..beef28e0 100644 --- a/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts +++ b/schemaregistry/rules/encryption/dekregistry/mock-dekregistry-client.ts @@ -1,9 +1,9 @@ -import { Client, Dek, Kek } from "./dekregistry-client"; +import { DekClient, Dek, Kek } from "./dekregistry-client"; import { MOCK_TS } from "./constants"; import stringify from "json-stringify-deterministic"; import {RestError} from "../../../rest-error"; -class MockDekRegistryClient implements Client { +class MockDekRegistryClient implements DekClient { private kekCache: Map; private dekCache: Map; diff --git a/schemaregistry/rules/encryption/encrypt-executor.ts b/schemaregistry/rules/encryption/encrypt-executor.ts index ffbd018c..ff70df41 100644 --- a/schemaregistry/rules/encryption/encrypt-executor.ts +++ b/schemaregistry/rules/encryption/encrypt-executor.ts @@ -8,7 +8,7 @@ import { RuleError, } from "../../serde/serde"; import {RuleMode,} from "../../schemaregistry-client"; -import {Client, Dek, DekRegistryClient, Kek} from "./dekregistry/dekregistry-client"; +import {DekClient, Dek, DekRegistryClient, Kek} from "./dekregistry/dekregistry-client"; import {RuleRegistry} from "../../serde/rule-registry"; import {ClientConfig} from "../../rest-service"; import {RestError} from "../../rest-error"; @@ -61,7 +61,7 @@ export class Clock { } export class FieldEncryptionExecutor extends FieldRuleExecutor { - client: Client | null = null + client: DekClient | null = null clock: Clock /** diff --git a/schemaregistry/serde/rule-registry.ts b/schemaregistry/serde/rule-registry.ts index 464962cd..2c6e9278 100644 --- a/schemaregistry/serde/rule-registry.ts +++ b/schemaregistry/serde/rule-registry.ts @@ -1,7 +1,7 @@ import {RuleAction, RuleExecutor} from "./serde"; /** - * + * RuleRegistry is used to register and fetch rule executors and actions. */ export class RuleRegistry { private ruleExecutors: Map = new Map() From 6d5dbdf7bc9348d6f59ca36fdcfc16ac7508b6e5 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Mon, 30 Sep 2024 15:06:08 -0400 Subject: [PATCH 220/224] Schemaregistry examples (#69) (#113) * Add restService interfaces to exported types * Bugfix for rest service and oauth --- schemaregistry/package.json | 2 +- schemaregistry/rest-service.ts | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/schemaregistry/package.json b/schemaregistry/package.json index 07909e84..fc47ccc4 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "v0.1.17.4-devel", + "version": "v0.1.17.6-devel", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts", diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index c7e2b693..537ec7f4 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -101,10 +101,10 @@ export class RestService { delete this.client.defaults.auth; const headers = ['logicalCluster', 'identityPoolId']; - const missingHeaders = headers.find(header => bearerAuthCredentials[header as keyof typeof bearerAuthCredentials]); + const missingHeader = headers.find(header => !(header in bearerAuthCredentials)); - if (missingHeaders) { - throw new Error(`Bearer auth header '${missingHeaders}' not provided`); + if (missingHeader) { + throw new Error(`Bearer auth header '${missingHeader}' not provided`); } this.setHeaders({ @@ -127,14 +127,14 @@ export class RestService { 'issuerEndpointUrl', 'scope' ]; - const missingField = requiredFields.find(field => bearerAuthCredentials[field as keyof typeof bearerAuthCredentials]); + const missingField = requiredFields.find(field => !(field in bearerAuthCredentials)); if (missingField) { throw new Error(`OAuth credential '${missingField}' not provided`); } const issuerEndPointUrl = new URL(bearerAuthCredentials.issuerEndpointUrl!); this.oauthClient = new OAuthClient(bearerAuthCredentials.clientId!, bearerAuthCredentials.clientSecret!, - issuerEndPointUrl.host, issuerEndPointUrl.pathname, bearerAuthCredentials.scope!); + issuerEndPointUrl.origin, issuerEndPointUrl.pathname, bearerAuthCredentials.scope!); break; default: throw new Error('Invalid bearer auth credentials source'); From e627f67711ea120dc12bda0a3071e0607d57061a Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Tue, 1 Oct 2024 15:30:25 -0400 Subject: [PATCH 221/224] Add schemaregistry examples workspace with avro, json, and csfle examples (#70) (#114) * Add schemaregistry examples workspace with avro, json, and csfle examples * change file name * Change schemaregistry name * Add extra descriptor for constants * Add content type by default * Change to basic auth for most examples --- .npmignore | 1 + package-lock.json | 303 +++++++++++++++++- package.json | 3 +- schemaregistry-examples/package.json | 15 + schemaregistry-examples/src/constants.ts | 27 ++ .../src/csfle-schemaregistry.ts | 155 +++++++++ .../src/kafka-consumer-avro.ts | 72 +++++ .../src/kafka-consumer-json.ts | 78 +++++ .../src/kafka-producer-avro.ts | 90 ++++++ .../src/kafka-producer-json.ts | 98 ++++++ .../src/local-schemaregistry.ts | 48 +++ .../src/oauth-schemaregistry.ts | 39 +++ schemaregistry-examples/tsconfig.json | 14 + schemaregistry/package.json | 6 +- schemaregistry/rest-service.ts | 5 +- test/schemaregistry/test-constants.ts | 3 - 16 files changed, 939 insertions(+), 18 deletions(-) create mode 100644 schemaregistry-examples/package.json create mode 100644 schemaregistry-examples/src/constants.ts create mode 100644 schemaregistry-examples/src/csfle-schemaregistry.ts create mode 100644 schemaregistry-examples/src/kafka-consumer-avro.ts create mode 100644 schemaregistry-examples/src/kafka-consumer-json.ts create mode 100644 schemaregistry-examples/src/kafka-producer-avro.ts create mode 100644 schemaregistry-examples/src/kafka-producer-json.ts create mode 100644 schemaregistry-examples/src/local-schemaregistry.ts create mode 100644 schemaregistry-examples/src/oauth-schemaregistry.ts create mode 100644 schemaregistry-examples/tsconfig.json diff --git a/.npmignore b/.npmignore index f551518a..418ac7aa 100644 --- a/.npmignore +++ b/.npmignore @@ -6,6 +6,7 @@ deps/* Dockerfile deps/librdkafka/config.h schemaregistry +schemaregistry-examples build .github .vscode diff --git a/package-lock.json b/package-lock.json index 7e578110..192eabb6 100644 --- a/package-lock.json +++ b/package-lock.json @@ -10,7 +10,8 @@ "hasInstallScript": true, "license": "MIT", "workspaces": [ - "schemaregistry" + "schemaregistry", + "schemaregistry-examples" ], "dependencies": { "@aws-sdk/client-kms": "^3.637.0", @@ -61,6 +62,7 @@ }, "node_modules/@ampproject/remapping": { "version": "2.3.0", + "dev": true, "license": "Apache-2.0", "dependencies": { "@jridgewell/gen-mapping": "^0.3.5", @@ -955,6 +957,7 @@ }, "node_modules/@babel/code-frame": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/highlight": "^7.24.7", @@ -966,6 +969,7 @@ }, "node_modules/@babel/compat-data": { "version": "7.25.2", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -973,6 +977,7 @@ }, "node_modules/@babel/core": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@ampproject/remapping": "^2.2.0", @@ -1001,6 +1006,7 @@ }, "node_modules/@babel/core/node_modules/semver": { "version": "6.3.1", + "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -1008,6 +1014,7 @@ }, "node_modules/@babel/generator": { "version": "7.25.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.25.0", @@ -1021,6 +1028,7 @@ }, "node_modules/@babel/helper-compilation-targets": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@babel/compat-data": "^7.25.2", @@ -1035,6 +1043,7 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/lru-cache": { "version": "5.1.1", + "dev": true, "license": "ISC", "dependencies": { "yallist": "^3.0.2" @@ -1042,6 +1051,7 @@ }, "node_modules/@babel/helper-compilation-targets/node_modules/semver": { "version": "6.3.1", + "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -1049,6 +1059,7 @@ }, "node_modules/@babel/helper-module-imports": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", @@ -1060,6 +1071,7 @@ }, "node_modules/@babel/helper-module-transforms": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-module-imports": "^7.24.7", @@ -1076,6 +1088,7 @@ }, "node_modules/@babel/helper-plugin-utils": { "version": "7.24.8", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1083,6 +1096,7 @@ }, "node_modules/@babel/helper-simple-access": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/traverse": "^7.24.7", @@ -1094,6 +1108,7 @@ }, "node_modules/@babel/helper-string-parser": { "version": "7.24.8", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1101,6 +1116,7 @@ }, "node_modules/@babel/helper-validator-identifier": { "version": "7.24.7", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1108,6 +1124,7 @@ }, "node_modules/@babel/helper-validator-option": { "version": "7.24.8", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -1115,6 +1132,7 @@ }, "node_modules/@babel/helpers": { "version": "7.25.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/template": "^7.25.0", @@ -1126,6 +1144,7 @@ }, "node_modules/@babel/highlight": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-validator-identifier": "^7.24.7", @@ -1139,6 +1158,7 @@ }, "node_modules/@babel/highlight/node_modules/ansi-styles": { "version": "3.2.1", + "dev": true, "license": "MIT", "dependencies": { "color-convert": "^1.9.0" @@ -1149,6 +1169,7 @@ }, "node_modules/@babel/highlight/node_modules/chalk": { "version": "2.4.2", + "dev": true, "license": "MIT", "dependencies": { "ansi-styles": "^3.2.1", @@ -1161,6 +1182,7 @@ }, "node_modules/@babel/highlight/node_modules/color-convert": { "version": "1.9.3", + "dev": true, "license": "MIT", "dependencies": { "color-name": "1.1.3" @@ -1168,10 +1190,12 @@ }, "node_modules/@babel/highlight/node_modules/color-name": { "version": "1.1.3", + "dev": true, "license": "MIT" }, "node_modules/@babel/highlight/node_modules/escape-string-regexp": { "version": "1.0.5", + "dev": true, "license": "MIT", "engines": { "node": ">=0.8.0" @@ -1179,6 +1203,7 @@ }, "node_modules/@babel/highlight/node_modules/has-flag": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -1186,6 +1211,7 @@ }, "node_modules/@babel/highlight/node_modules/supports-color": { "version": "5.5.0", + "dev": true, "license": "MIT", "dependencies": { "has-flag": "^3.0.0" @@ -1196,6 +1222,7 @@ }, "node_modules/@babel/parser": { "version": "7.25.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.25.2" @@ -1209,6 +1236,7 @@ }, "node_modules/@babel/plugin-syntax-async-generators": { "version": "7.8.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1219,6 +1247,7 @@ }, "node_modules/@babel/plugin-syntax-bigint": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1229,6 +1258,7 @@ }, "node_modules/@babel/plugin-syntax-class-properties": { "version": "7.12.13", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.12.13" @@ -1239,6 +1269,7 @@ }, "node_modules/@babel/plugin-syntax-import-meta": { "version": "7.10.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1249,6 +1280,7 @@ }, "node_modules/@babel/plugin-syntax-json-strings": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1259,6 +1291,7 @@ }, "node_modules/@babel/plugin-syntax-jsx": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" @@ -1272,6 +1305,7 @@ }, "node_modules/@babel/plugin-syntax-logical-assignment-operators": { "version": "7.10.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1282,6 +1316,7 @@ }, "node_modules/@babel/plugin-syntax-nullish-coalescing-operator": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1292,6 +1327,7 @@ }, "node_modules/@babel/plugin-syntax-numeric-separator": { "version": "7.10.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.10.4" @@ -1302,6 +1338,7 @@ }, "node_modules/@babel/plugin-syntax-object-rest-spread": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1312,6 +1349,7 @@ }, "node_modules/@babel/plugin-syntax-optional-catch-binding": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1322,6 +1360,7 @@ }, "node_modules/@babel/plugin-syntax-optional-chaining": { "version": "7.8.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.8.0" @@ -1332,6 +1371,7 @@ }, "node_modules/@babel/plugin-syntax-top-level-await": { "version": "7.14.5", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.14.5" @@ -1345,6 +1385,7 @@ }, "node_modules/@babel/plugin-syntax-typescript": { "version": "7.24.7", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-plugin-utils": "^7.24.7" @@ -1358,6 +1399,7 @@ }, "node_modules/@babel/template": { "version": "7.25.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", @@ -1370,6 +1412,7 @@ }, "node_modules/@babel/traverse": { "version": "7.25.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.24.7", @@ -1386,6 +1429,7 @@ }, "node_modules/@babel/traverse/node_modules/globals": { "version": "11.12.0", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -1393,6 +1437,7 @@ }, "node_modules/@babel/types": { "version": "7.25.2", + "dev": true, "license": "MIT", "dependencies": { "@babel/helper-string-parser": "^7.24.8", @@ -1405,6 +1450,7 @@ }, "node_modules/@bcoe/v8-coverage": { "version": "0.2.3", + "dev": true, "license": "MIT" }, "node_modules/@bufbuild/buf": { @@ -1494,9 +1540,10 @@ } }, "node_modules/@confluentinc/kafka-javascript": { - "version": "0.1.17-devel", - "resolved": "https://registry.npmjs.org/@confluentinc/kafka-javascript/-/kafka-javascript-0.1.17-devel.tgz", - "integrity": "sha512-u+7Rvzw1ljNSKC54OBt89oWkj98zOj4zWT5FZkAcSc5SDdZfYuKatMZYo0vNiV1V9otQee6fdJEl3qtpDd7/fQ==", + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/@confluentinc/kafka-javascript/-/kafka-javascript-0.2.0.tgz", + "integrity": "sha512-IWjyGRqeDBcWmYcEQHu1XlZQ6am5qzzIEb18rdxFZkFeVQ6piG28bQ6BmlqvHn3zd+XoAO+e8bRlpgdUuTLC9Q==", + "dev": true, "hasInstallScript": true, "license": "MIT", "workspaces": [ @@ -1525,7 +1572,6 @@ "nan": "^2.17.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", - "ts-jest": "^29.2.4", "validator": "^13.12.0" }, "engines": { @@ -1766,6 +1812,7 @@ }, "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", + "dev": true, "license": "ISC", "dependencies": { "camelcase": "^5.3.1", @@ -1780,6 +1827,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/argparse": { "version": "1.0.10", + "dev": true, "license": "MIT", "dependencies": { "sprintf-js": "~1.0.2" @@ -1787,6 +1835,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/find-up": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "locate-path": "^5.0.0", @@ -1798,6 +1847,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/js-yaml": { "version": "3.14.1", + "dev": true, "license": "MIT", "dependencies": { "argparse": "^1.0.7", @@ -1809,6 +1859,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/locate-path": { "version": "5.0.0", + "dev": true, "license": "MIT", "dependencies": { "p-locate": "^4.1.0" @@ -1819,6 +1870,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/p-limit": { "version": "2.3.0", + "dev": true, "license": "MIT", "dependencies": { "p-try": "^2.0.0" @@ -1832,6 +1884,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/p-locate": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "p-limit": "^2.2.0" @@ -1842,6 +1895,7 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/resolve-from": { "version": "5.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -1849,10 +1903,12 @@ }, "node_modules/@istanbuljs/load-nyc-config/node_modules/sprintf-js": { "version": "1.0.3", + "dev": true, "license": "BSD-3-Clause" }, "node_modules/@istanbuljs/schema": { "version": "0.1.3", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -1860,6 +1916,7 @@ }, "node_modules/@jest/console": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -1875,6 +1932,7 @@ }, "node_modules/@jest/core": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -1920,6 +1978,7 @@ }, "node_modules/@jest/environment": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/fake-timers": "^29.7.0", @@ -1933,6 +1992,7 @@ }, "node_modules/@jest/expect": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "expect": "^29.7.0", @@ -1944,6 +2004,7 @@ }, "node_modules/@jest/expect-utils": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3" @@ -1954,6 +2015,7 @@ }, "node_modules/@jest/fake-timers": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -1969,6 +2031,7 @@ }, "node_modules/@jest/globals": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -1982,6 +2045,7 @@ }, "node_modules/@jest/reporters": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@bcoe/v8-coverage": "^0.2.3", @@ -2023,6 +2087,7 @@ }, "node_modules/@jest/schemas": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@sinclair/typebox": "^0.27.8" @@ -2033,6 +2098,7 @@ }, "node_modules/@jest/source-map": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@jridgewell/trace-mapping": "^0.3.18", @@ -2045,6 +2111,7 @@ }, "node_modules/@jest/test-result": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -2058,6 +2125,7 @@ }, "node_modules/@jest/test-sequencer": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", @@ -2071,6 +2139,7 @@ }, "node_modules/@jest/transform": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -2095,6 +2164,7 @@ }, "node_modules/@jest/types": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", @@ -2110,6 +2180,7 @@ }, "node_modules/@jridgewell/gen-mapping": { "version": "0.3.5", + "dev": true, "license": "MIT", "dependencies": { "@jridgewell/set-array": "^1.2.1", @@ -2122,6 +2193,7 @@ }, "node_modules/@jridgewell/resolve-uri": { "version": "3.1.2", + "dev": true, "license": "MIT", "engines": { "node": ">=6.0.0" @@ -2129,6 +2201,7 @@ }, "node_modules/@jridgewell/set-array": { "version": "1.2.1", + "dev": true, "license": "MIT", "engines": { "node": ">=6.0.0" @@ -2136,10 +2209,12 @@ }, "node_modules/@jridgewell/sourcemap-codec": { "version": "1.5.0", + "dev": true, "license": "MIT" }, "node_modules/@jridgewell/trace-mapping": { "version": "0.3.25", + "dev": true, "license": "MIT", "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -2381,10 +2456,12 @@ }, "node_modules/@sinclair/typebox": { "version": "0.27.8", + "dev": true, "license": "MIT" }, "node_modules/@sinonjs/commons": { "version": "3.0.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "type-detect": "4.0.8" @@ -2392,6 +2469,7 @@ }, "node_modules/@sinonjs/fake-timers": { "version": "10.3.0", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@sinonjs/commons": "^3.0.0" @@ -2881,6 +2959,7 @@ }, "node_modules/@types/babel__core": { "version": "7.20.5", + "dev": true, "license": "MIT", "dependencies": { "@babel/parser": "^7.20.7", @@ -2892,6 +2971,7 @@ }, "node_modules/@types/babel__generator": { "version": "7.6.8", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.0.0" @@ -2899,6 +2979,7 @@ }, "node_modules/@types/babel__template": { "version": "7.4.4", + "dev": true, "license": "MIT", "dependencies": { "@babel/parser": "^7.1.0", @@ -2907,6 +2988,7 @@ }, "node_modules/@types/babel__traverse": { "version": "7.20.6", + "dev": true, "license": "MIT", "dependencies": { "@babel/types": "^7.20.7" @@ -2940,6 +3022,7 @@ }, "node_modules/@types/graceful-fs": { "version": "4.1.9", + "dev": true, "license": "MIT", "dependencies": { "@types/node": "*" @@ -2947,10 +3030,12 @@ }, "node_modules/@types/istanbul-lib-coverage": { "version": "2.0.6", + "dev": true, "license": "MIT" }, "node_modules/@types/istanbul-lib-report": { "version": "3.0.3", + "dev": true, "license": "MIT", "dependencies": { "@types/istanbul-lib-coverage": "*" @@ -2958,6 +3043,7 @@ }, "node_modules/@types/istanbul-reports": { "version": "3.0.4", + "dev": true, "license": "MIT", "dependencies": { "@types/istanbul-lib-report": "*" @@ -3045,6 +3131,7 @@ }, "node_modules/@types/stack-utils": { "version": "2.0.3", + "dev": true, "license": "MIT" }, "node_modules/@types/tough-cookie": { @@ -3064,6 +3151,7 @@ }, "node_modules/@types/yargs": { "version": "17.0.32", + "dev": true, "license": "MIT", "dependencies": { "@types/yargs-parser": "*" @@ -3071,6 +3159,7 @@ }, "node_modules/@types/yargs-parser": { "version": "21.0.3", + "dev": true, "license": "MIT" }, "node_modules/@typescript-eslint/eslint-plugin": { @@ -3753,6 +3842,7 @@ }, "node_modules/ansi-escapes": { "version": "4.3.2", + "dev": true, "license": "MIT", "dependencies": { "type-fest": "^0.21.3" @@ -3766,6 +3856,7 @@ }, "node_modules/ansi-escapes/node_modules/type-fest": { "version": "0.21.3", + "dev": true, "license": "(MIT OR CC0-1.0)", "engines": { "node": ">=10" @@ -3796,6 +3887,7 @@ }, "node_modules/anymatch": { "version": "3.1.3", + "dev": true, "license": "ISC", "dependencies": { "normalize-path": "^3.0.0", @@ -3849,6 +3941,7 @@ }, "node_modules/async": { "version": "3.2.5", + "dev": true, "license": "MIT" }, "node_modules/async-mutex": { @@ -3893,6 +3986,7 @@ }, "node_modules/babel-jest": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/transform": "^29.7.0", @@ -3912,6 +4006,7 @@ }, "node_modules/babel-plugin-istanbul": { "version": "6.1.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/helper-plugin-utils": "^7.0.0", @@ -3926,6 +4021,7 @@ }, "node_modules/babel-plugin-istanbul/node_modules/istanbul-lib-instrument": { "version": "5.2.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.12.3", @@ -3940,6 +4036,7 @@ }, "node_modules/babel-plugin-istanbul/node_modules/semver": { "version": "6.3.1", + "dev": true, "license": "ISC", "bin": { "semver": "bin/semver.js" @@ -3947,6 +4044,7 @@ }, "node_modules/babel-plugin-jest-hoist": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "@babel/template": "^7.3.3", @@ -3960,6 +4058,7 @@ }, "node_modules/babel-preset-current-node-syntax": { "version": "1.0.1", + "dev": true, "license": "MIT", "dependencies": { "@babel/plugin-syntax-async-generators": "^7.8.4", @@ -3981,6 +4080,7 @@ }, "node_modules/babel-preset-jest": { "version": "29.6.3", + "dev": true, "license": "MIT", "dependencies": { "babel-plugin-jest-hoist": "^29.6.3", @@ -4066,6 +4166,7 @@ }, "node_modules/braces": { "version": "3.0.3", + "dev": true, "license": "MIT", "dependencies": { "fill-range": "^7.1.1" @@ -4088,6 +4189,7 @@ }, "node_modules/browserslist": { "version": "4.23.3", + "dev": true, "funding": [ { "type": "opencollective", @@ -4118,6 +4220,7 @@ }, "node_modules/bs-logger": { "version": "0.2.6", + "dev": true, "license": "MIT", "dependencies": { "fast-json-stable-stringify": "2.x" @@ -4128,6 +4231,7 @@ }, "node_modules/bser": { "version": "2.1.1", + "dev": true, "license": "Apache-2.0", "dependencies": { "node-int64": "^0.4.0" @@ -4139,6 +4243,7 @@ }, "node_modules/buffer-from": { "version": "1.1.2", + "dev": true, "license": "MIT" }, "node_modules/cacache": { @@ -4216,6 +4321,7 @@ }, "node_modules/callsites": { "version": "3.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -4223,6 +4329,7 @@ }, "node_modules/camelcase": { "version": "5.3.1", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -4230,6 +4337,7 @@ }, "node_modules/caniuse-lite": { "version": "1.0.30001646", + "dev": true, "funding": [ { "type": "opencollective", @@ -4263,6 +4371,7 @@ }, "node_modules/chalk": { "version": "4.1.2", + "dev": true, "license": "MIT", "dependencies": { "ansi-styles": "^4.1.0", @@ -4277,6 +4386,7 @@ }, "node_modules/char-regex": { "version": "1.0.2", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -4325,6 +4435,7 @@ }, "node_modules/ci-info": { "version": "3.9.0", + "dev": true, "funding": [ { "type": "github", @@ -4338,6 +4449,7 @@ }, "node_modules/cjs-module-lexer": { "version": "1.3.1", + "dev": true, "license": "MIT" }, "node_modules/clean-stack": { @@ -4362,6 +4474,7 @@ }, "node_modules/co": { "version": "4.6.0", + "dev": true, "license": "MIT", "engines": { "iojs": ">= 1.0.0", @@ -4370,6 +4483,7 @@ }, "node_modules/collect-v8-coverage": { "version": "1.0.2", + "dev": true, "license": "MIT" }, "node_modules/color-convert": { @@ -4417,6 +4531,7 @@ }, "node_modules/convert-source-map": { "version": "2.0.0", + "dev": true, "license": "MIT" }, "node_modules/core-util-is": { @@ -4425,6 +4540,7 @@ }, "node_modules/create-jest": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -4444,6 +4560,7 @@ }, "node_modules/cross-spawn": { "version": "7.0.3", + "dev": true, "license": "MIT", "dependencies": { "path-key": "^3.1.0", @@ -4492,6 +4609,7 @@ }, "node_modules/dedent": { "version": "1.5.3", + "dev": true, "license": "MIT", "peerDependencies": { "babel-plugin-macros": "^3.1.0" @@ -4509,6 +4627,7 @@ }, "node_modules/deepmerge": { "version": "4.3.1", + "dev": true, "license": "MIT", "engines": { "node": ">=0.10.0" @@ -4541,6 +4660,7 @@ }, "node_modules/detect-newline": { "version": "3.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -4556,6 +4676,7 @@ }, "node_modules/diff-sequences": { "version": "29.6.3", + "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -4618,6 +4739,7 @@ }, "node_modules/ejs": { "version": "3.1.10", + "dev": true, "license": "Apache-2.0", "dependencies": { "jake": "^10.8.5" @@ -4631,10 +4753,12 @@ }, "node_modules/electron-to-chromium": { "version": "1.5.4", + "dev": true, "license": "ISC" }, "node_modules/emittery": { "version": "0.13.1", + "dev": true, "license": "MIT", "engines": { "node": ">=12" @@ -4688,6 +4812,7 @@ }, "node_modules/error-ex": { "version": "1.3.2", + "dev": true, "license": "MIT", "dependencies": { "is-arrayish": "^0.2.1" @@ -4871,6 +4996,7 @@ }, "node_modules/esprima": { "version": "4.0.1", + "dev": true, "license": "BSD-2-Clause", "bin": { "esparse": "bin/esparse.js", @@ -4934,6 +5060,7 @@ }, "node_modules/execa": { "version": "5.1.1", + "dev": true, "license": "MIT", "dependencies": { "cross-spawn": "^7.0.3", @@ -4955,12 +5082,14 @@ }, "node_modules/exit": { "version": "0.1.2", + "dev": true, "engines": { "node": ">= 0.8.0" } }, "node_modules/expect": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/expect-utils": "^29.7.0", @@ -5062,6 +5191,7 @@ }, "node_modules/fb-watchman": { "version": "2.0.2", + "dev": true, "license": "Apache-2.0", "dependencies": { "bser": "2.1.1" @@ -5084,6 +5214,7 @@ }, "node_modules/filelist": { "version": "1.0.4", + "dev": true, "license": "Apache-2.0", "dependencies": { "minimatch": "^5.0.1" @@ -5091,6 +5222,7 @@ }, "node_modules/filelist/node_modules/brace-expansion": { "version": "2.0.1", + "dev": true, "license": "MIT", "dependencies": { "balanced-match": "^1.0.0" @@ -5098,6 +5230,7 @@ }, "node_modules/filelist/node_modules/minimatch": { "version": "5.1.6", + "dev": true, "license": "ISC", "dependencies": { "brace-expansion": "^2.0.1" @@ -5108,6 +5241,7 @@ }, "node_modules/fill-range": { "version": "7.1.1", + "dev": true, "license": "MIT", "dependencies": { "to-regex-range": "^5.0.1" @@ -5210,6 +5344,7 @@ }, "node_modules/fsevents": { "version": "2.3.3", + "dev": true, "license": "MIT", "optional": true, "os": [ @@ -5221,6 +5356,7 @@ }, "node_modules/function-bind": { "version": "1.1.2", + "dev": true, "license": "MIT", "funding": { "url": "https://github.com/sponsors/ljharb" @@ -5292,6 +5428,7 @@ }, "node_modules/gensync": { "version": "1.0.0-beta.2", + "dev": true, "license": "MIT", "engines": { "node": ">=6.9.0" @@ -5306,6 +5443,7 @@ }, "node_modules/get-package-type": { "version": "0.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8.0.0" @@ -5313,6 +5451,7 @@ }, "node_modules/get-stream": { "version": "6.0.1", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -5428,6 +5567,7 @@ }, "node_modules/graceful-fs": { "version": "4.2.11", + "dev": true, "license": "ISC" }, "node_modules/graphemer": { @@ -5484,6 +5624,7 @@ }, "node_modules/has-flag": { "version": "4.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -5495,6 +5636,7 @@ }, "node_modules/hasown": { "version": "2.0.2", + "dev": true, "license": "MIT", "dependencies": { "function-bind": "^1.1.2" @@ -5513,6 +5655,7 @@ }, "node_modules/html-escaper": { "version": "2.0.2", + "dev": true, "license": "MIT" }, "node_modules/http-cache-semantics": { @@ -5557,6 +5700,7 @@ }, "node_modules/human-signals": { "version": "2.1.0", + "dev": true, "license": "Apache-2.0", "engines": { "node": ">=10.17.0" @@ -5606,6 +5750,7 @@ }, "node_modules/import-local": { "version": "3.2.0", + "dev": true, "license": "MIT", "dependencies": { "pkg-dir": "^4.2.0", @@ -5623,6 +5768,7 @@ }, "node_modules/imurmurhash": { "version": "0.1.4", + "dev": true, "license": "MIT", "engines": { "node": ">=0.8.19" @@ -5667,6 +5813,7 @@ }, "node_modules/is-arrayish": { "version": "0.2.1", + "dev": true, "license": "MIT" }, "node_modules/is-binary-path": { @@ -5682,6 +5829,7 @@ }, "node_modules/is-core-module": { "version": "2.15.0", + "dev": true, "license": "MIT", "dependencies": { "hasown": "^2.0.2" @@ -5723,6 +5871,7 @@ }, "node_modules/is-generator-fn": { "version": "2.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -5746,6 +5895,7 @@ }, "node_modules/is-number": { "version": "7.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=0.12.0" @@ -5804,6 +5954,7 @@ }, "node_modules/isexe": { "version": "2.0.0", + "dev": true, "license": "ISC" }, "node_modules/isstream": { @@ -5812,6 +5963,7 @@ }, "node_modules/istanbul-lib-coverage": { "version": "3.2.2", + "dev": true, "license": "BSD-3-Clause", "engines": { "node": ">=8" @@ -5819,6 +5971,7 @@ }, "node_modules/istanbul-lib-instrument": { "version": "6.0.3", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "@babel/core": "^7.23.9", @@ -5833,6 +5986,7 @@ }, "node_modules/istanbul-lib-report": { "version": "3.0.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "istanbul-lib-coverage": "^3.0.0", @@ -5845,6 +5999,7 @@ }, "node_modules/istanbul-lib-report/node_modules/make-dir": { "version": "4.0.0", + "dev": true, "license": "MIT", "dependencies": { "semver": "^7.5.3" @@ -5858,6 +6013,7 @@ }, "node_modules/istanbul-lib-source-maps": { "version": "4.0.1", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "debug": "^4.1.1", @@ -5870,6 +6026,7 @@ }, "node_modules/istanbul-reports": { "version": "3.1.7", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "html-escaper": "^2.0.0", @@ -5881,6 +6038,7 @@ }, "node_modules/jake": { "version": "10.9.2", + "dev": true, "license": "Apache-2.0", "dependencies": { "async": "^3.2.3", @@ -5897,6 +6055,7 @@ }, "node_modules/jest": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", @@ -5921,6 +6080,7 @@ }, "node_modules/jest-changed-files": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "execa": "^5.0.0", @@ -5933,6 +6093,7 @@ }, "node_modules/jest-circus": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -5962,6 +6123,7 @@ }, "node_modules/jest-cli": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/core": "^29.7.0", @@ -5993,6 +6155,7 @@ }, "node_modules/jest-config": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -6036,6 +6199,7 @@ }, "node_modules/jest-diff": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6049,6 +6213,7 @@ }, "node_modules/jest-docblock": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "detect-newline": "^3.0.0" @@ -6059,6 +6224,7 @@ }, "node_modules/jest-each": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6073,6 +6239,7 @@ }, "node_modules/jest-environment-node": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6088,6 +6255,7 @@ }, "node_modules/jest-get-type": { "version": "29.6.3", + "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -6095,6 +6263,7 @@ }, "node_modules/jest-haste-map": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6118,6 +6287,7 @@ }, "node_modules/jest-leak-detector": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "jest-get-type": "^29.6.3", @@ -6129,6 +6299,7 @@ }, "node_modules/jest-matcher-utils": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6142,6 +6313,7 @@ }, "node_modules/jest-message-util": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.12.13", @@ -6160,6 +6332,7 @@ }, "node_modules/jest-mock": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6172,6 +6345,7 @@ }, "node_modules/jest-pnp-resolver": { "version": "1.2.3", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6187,6 +6361,7 @@ }, "node_modules/jest-regex-util": { "version": "29.6.3", + "dev": true, "license": "MIT", "engines": { "node": "^14.15.0 || ^16.10.0 || >=18.0.0" @@ -6194,6 +6369,7 @@ }, "node_modules/jest-resolve": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "chalk": "^4.0.0", @@ -6212,6 +6388,7 @@ }, "node_modules/jest-resolve-dependencies": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "jest-regex-util": "^29.6.3", @@ -6223,6 +6400,7 @@ }, "node_modules/jest-runner": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/console": "^29.7.0", @@ -6253,6 +6431,7 @@ }, "node_modules/jest-runtime": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/environment": "^29.7.0", @@ -6284,6 +6463,7 @@ }, "node_modules/jest-snapshot": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/core": "^7.11.6", @@ -6313,6 +6493,7 @@ }, "node_modules/jest-util": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6328,6 +6509,7 @@ }, "node_modules/jest-validate": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/types": "^29.6.3", @@ -6343,6 +6525,7 @@ }, "node_modules/jest-validate/node_modules/camelcase": { "version": "6.3.0", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -6353,6 +6536,7 @@ }, "node_modules/jest-watcher": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/test-result": "^29.7.0", @@ -6370,6 +6554,7 @@ }, "node_modules/jest-worker": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@types/node": "*", @@ -6383,6 +6568,7 @@ }, "node_modules/jest-worker/node_modules/supports-color": { "version": "8.1.1", + "dev": true, "license": "MIT", "dependencies": { "has-flag": "^4.0.0" @@ -6421,6 +6607,7 @@ }, "node_modules/js-tokens": { "version": "4.0.0", + "dev": true, "license": "MIT" }, "node_modules/js-yaml": { @@ -6485,6 +6672,7 @@ }, "node_modules/jsesc": { "version": "2.5.2", + "dev": true, "license": "MIT", "bin": { "jsesc": "bin/jsesc" @@ -6507,6 +6695,7 @@ }, "node_modules/json-parse-even-better-errors": { "version": "2.3.1", + "dev": true, "license": "MIT" }, "node_modules/json-schema": { @@ -6535,6 +6724,7 @@ }, "node_modules/json5": { "version": "2.2.3", + "dev": true, "license": "MIT", "bin": { "json5": "lib/cli.js" @@ -6635,6 +6825,7 @@ }, "node_modules/kleur": { "version": "3.0.3", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6642,6 +6833,7 @@ }, "node_modules/leven": { "version": "3.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -6661,6 +6853,7 @@ }, "node_modules/lines-and-columns": { "version": "1.2.4", + "dev": true, "license": "MIT" }, "node_modules/linkify-it": { @@ -6720,6 +6913,7 @@ }, "node_modules/lodash.memoize": { "version": "4.1.2", + "dev": true, "license": "MIT" }, "node_modules/lodash.merge": { @@ -6779,6 +6973,7 @@ }, "node_modules/make-error": { "version": "1.3.6", + "dev": true, "license": "ISC" }, "node_modules/make-fetch-happen": { @@ -6817,6 +7012,7 @@ }, "node_modules/makeerror": { "version": "1.0.12", + "dev": true, "license": "BSD-3-Clause", "dependencies": { "tmpl": "1.0.5" @@ -6865,6 +7061,7 @@ }, "node_modules/merge-stream": { "version": "2.0.0", + "dev": true, "license": "MIT" }, "node_modules/merge2": { @@ -6879,6 +7076,7 @@ "version": "4.0.8", "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz", "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==", + "dev": true, "license": "MIT", "dependencies": { "braces": "^3.0.3", @@ -6907,6 +7105,7 @@ }, "node_modules/mimic-fn": { "version": "2.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -7159,6 +7358,7 @@ }, "node_modules/natural-compare": { "version": "1.4.0", + "dev": true, "license": "MIT" }, "node_modules/nearley": { @@ -7291,10 +7491,12 @@ }, "node_modules/node-int64": { "version": "0.4.0", + "dev": true, "license": "MIT" }, "node_modules/node-releases": { "version": "2.0.18", + "dev": true, "license": "MIT" }, "node_modules/node-vault": { @@ -7325,6 +7527,7 @@ }, "node_modules/normalize-path": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=0.10.0" @@ -7332,6 +7535,7 @@ }, "node_modules/npm-run-path": { "version": "4.0.1", + "dev": true, "license": "MIT", "dependencies": { "path-key": "^3.0.0" @@ -7380,6 +7584,7 @@ }, "node_modules/onetime": { "version": "5.1.2", + "dev": true, "license": "MIT", "dependencies": { "mimic-fn": "^2.1.0" @@ -7424,6 +7629,7 @@ }, "node_modules/p-limit": { "version": "3.1.0", + "dev": true, "license": "MIT", "dependencies": { "yocto-queue": "^0.1.0" @@ -7465,6 +7671,7 @@ }, "node_modules/p-try": { "version": "2.2.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -7483,6 +7690,7 @@ }, "node_modules/parse-json": { "version": "5.2.0", + "dev": true, "license": "MIT", "dependencies": { "@babel/code-frame": "^7.0.0", @@ -7499,6 +7707,7 @@ }, "node_modules/path-exists": { "version": "4.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7513,6 +7722,7 @@ }, "node_modules/path-key": { "version": "3.1.1", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7520,6 +7730,7 @@ }, "node_modules/path-parse": { "version": "1.0.7", + "dev": true, "license": "MIT" }, "node_modules/path-type": { @@ -7536,10 +7747,12 @@ }, "node_modules/picocolors": { "version": "1.0.1", + "dev": true, "license": "ISC" }, "node_modules/picomatch": { "version": "2.3.1", + "dev": true, "license": "MIT", "engines": { "node": ">=8.6" @@ -7550,6 +7763,7 @@ }, "node_modules/pirates": { "version": "4.0.6", + "dev": true, "license": "MIT", "engines": { "node": ">= 6" @@ -7557,6 +7771,7 @@ }, "node_modules/pkg-dir": { "version": "4.2.0", + "dev": true, "license": "MIT", "dependencies": { "find-up": "^4.0.0" @@ -7567,6 +7782,7 @@ }, "node_modules/pkg-dir/node_modules/find-up": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "locate-path": "^5.0.0", @@ -7578,6 +7794,7 @@ }, "node_modules/pkg-dir/node_modules/locate-path": { "version": "5.0.0", + "dev": true, "license": "MIT", "dependencies": { "p-locate": "^4.1.0" @@ -7588,6 +7805,7 @@ }, "node_modules/pkg-dir/node_modules/p-limit": { "version": "2.3.0", + "dev": true, "license": "MIT", "dependencies": { "p-try": "^2.0.0" @@ -7601,6 +7819,7 @@ }, "node_modules/pkg-dir/node_modules/p-locate": { "version": "4.1.0", + "dev": true, "license": "MIT", "dependencies": { "p-limit": "^2.2.0" @@ -7657,6 +7876,7 @@ }, "node_modules/pretty-format": { "version": "29.7.0", + "dev": true, "license": "MIT", "dependencies": { "@jest/schemas": "^29.6.3", @@ -7669,6 +7889,7 @@ }, "node_modules/pretty-format/node_modules/ansi-styles": { "version": "5.2.0", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -7696,6 +7917,7 @@ }, "node_modules/prompts": { "version": "2.4.2", + "dev": true, "license": "MIT", "dependencies": { "kleur": "^3.0.3", @@ -7762,6 +7984,7 @@ }, "node_modules/pure-rand": { "version": "6.1.0", + "dev": true, "funding": [ { "type": "individual", @@ -7829,6 +8052,7 @@ }, "node_modules/react-is": { "version": "18.3.1", + "dev": true, "license": "MIT" }, "node_modules/readable-stream": { @@ -7882,6 +8106,7 @@ }, "node_modules/resolve": { "version": "1.22.8", + "dev": true, "license": "MIT", "dependencies": { "is-core-module": "^2.13.0", @@ -7897,6 +8122,7 @@ }, "node_modules/resolve-cwd": { "version": "3.0.0", + "dev": true, "license": "MIT", "dependencies": { "resolve-from": "^5.0.0" @@ -7907,6 +8133,7 @@ }, "node_modules/resolve-cwd/node_modules/resolve-from": { "version": "5.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -7922,6 +8149,7 @@ }, "node_modules/resolve.exports": { "version": "2.0.2", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -8020,6 +8248,10 @@ "version": "2.1.2", "license": "MIT" }, + "node_modules/schemaregistry-examples": { + "resolved": "schemaregistry-examples", + "link": true + }, "node_modules/semver": { "version": "7.6.3", "license": "ISC", @@ -8044,6 +8276,7 @@ }, "node_modules/shebang-command": { "version": "2.0.0", + "dev": true, "license": "MIT", "dependencies": { "shebang-regex": "^3.0.0" @@ -8054,6 +8287,7 @@ }, "node_modules/shebang-regex": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8077,10 +8311,12 @@ }, "node_modules/sisteransi": { "version": "1.0.5", + "dev": true, "license": "MIT" }, "node_modules/slash": { "version": "3.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8133,6 +8369,7 @@ }, "node_modules/source-map": { "version": "0.6.1", + "dev": true, "license": "BSD-3-Clause", "engines": { "node": ">=0.10.0" @@ -8140,6 +8377,7 @@ }, "node_modules/source-map-support": { "version": "0.5.13", + "dev": true, "license": "MIT", "dependencies": { "buffer-from": "^1.0.0", @@ -8191,6 +8429,7 @@ }, "node_modules/stack-utils": { "version": "2.0.6", + "dev": true, "license": "MIT", "dependencies": { "escape-string-regexp": "^2.0.0" @@ -8201,6 +8440,7 @@ }, "node_modules/stack-utils/node_modules/escape-string-regexp": { "version": "2.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8245,6 +8485,7 @@ }, "node_modules/string-length": { "version": "4.0.2", + "dev": true, "license": "MIT", "dependencies": { "char-regex": "^1.0.2", @@ -8278,6 +8519,7 @@ }, "node_modules/strip-bom": { "version": "4.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8285,6 +8527,7 @@ }, "node_modules/strip-final-newline": { "version": "2.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=6" @@ -8292,6 +8535,7 @@ }, "node_modules/strip-json-comments": { "version": "3.1.1", + "dev": true, "license": "MIT", "engines": { "node": ">=8" @@ -8310,6 +8554,7 @@ }, "node_modules/supports-color": { "version": "7.2.0", + "dev": true, "license": "MIT", "dependencies": { "has-flag": "^4.0.0" @@ -8320,6 +8565,7 @@ }, "node_modules/supports-preserve-symlinks-flag": { "version": "1.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">= 0.4" @@ -8370,6 +8616,7 @@ }, "node_modules/test-exclude": { "version": "6.0.0", + "dev": true, "license": "ISC", "dependencies": { "@istanbuljs/schema": "^0.1.2", @@ -8387,10 +8634,12 @@ }, "node_modules/tmpl": { "version": "1.0.5", + "dev": true, "license": "BSD-3-Clause" }, "node_modules/to-fast-properties": { "version": "2.0.0", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -8398,6 +8647,7 @@ }, "node_modules/to-regex-range": { "version": "5.0.1", + "dev": true, "license": "MIT", "dependencies": { "is-number": "^7.0.0" @@ -8432,6 +8682,7 @@ "version": "29.2.5", "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.2.5.tgz", "integrity": "sha512-KD8zB2aAZrcKIdGk4OwpJggeLcH1FgrICqDSROWqlnJXGCXK4Mn6FcdK2B6670Xr73lHMG1kHw8R87A0ecZ+vA==", + "dev": true, "dependencies": { "bs-logger": "^0.2.6", "ejs": "^3.1.10", @@ -8477,6 +8728,7 @@ }, "node_modules/ts-jest/node_modules/yargs-parser": { "version": "21.1.1", + "dev": true, "license": "ISC", "engines": { "node": ">=12" @@ -8519,6 +8771,7 @@ }, "node_modules/type-detect": { "version": "4.0.8", + "dev": true, "license": "MIT", "engines": { "node": ">=4" @@ -8537,6 +8790,7 @@ }, "node_modules/typescript": { "version": "5.5.4", + "dev": true, "license": "Apache-2.0", "bin": { "tsc": "bin/tsc", @@ -8809,6 +9063,7 @@ }, "node_modules/update-browserslist-db": { "version": "1.1.0", + "dev": true, "funding": [ { "type": "opencollective", @@ -8867,6 +9122,7 @@ }, "node_modules/v8-to-istanbul": { "version": "9.3.0", + "dev": true, "license": "ISC", "dependencies": { "@jridgewell/trace-mapping": "^0.3.12", @@ -8898,6 +9154,7 @@ }, "node_modules/walker": { "version": "1.0.8", + "dev": true, "license": "Apache-2.0", "dependencies": { "makeerror": "1.0.12" @@ -8917,6 +9174,7 @@ }, "node_modules/which": { "version": "2.0.2", + "dev": true, "license": "ISC", "dependencies": { "isexe": "^2.0.0" @@ -8969,6 +9227,7 @@ }, "node_modules/write-file-atomic": { "version": "4.0.2", + "dev": true, "license": "ISC", "dependencies": { "imurmurhash": "^0.1.4", @@ -8992,6 +9251,7 @@ }, "node_modules/yallist": { "version": "3.1.1", + "dev": true, "license": "ISC" }, "node_modules/yargs": { @@ -9052,6 +9312,7 @@ }, "node_modules/yocto-queue": { "version": "0.1.0", + "dev": true, "license": "MIT", "engines": { "node": ">=10" @@ -9062,14 +9323,13 @@ }, "schemaregistry": { "name": "@confluentinc/schemaregistry", - "version": "v0.1.17.4-devel", + "version": "v0.1.17.6-devel", "license": "MIT", "dependencies": { "@aws-sdk/client-kms": "^3.637.0", "@azure/identity": "^4.4.1", "@azure/keyvault-keys": "^4.8.0", "@bufbuild/protobuf": "^2.0.0", - "@confluentinc/kafka-javascript": "^0.1.17-devel", "@criteria/json-schema": "^0.10.0", "@criteria/json-schema-validation": "^0.10.0", "@google-cloud/kms": "^4.5.0", @@ -9085,12 +9345,12 @@ "lru-cache": "^11.0.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", - "uuid": "^10.0.0", "validator": "^13.12.0" }, "devDependencies": { "@bufbuild/buf": "^1.37.0", "@bufbuild/protoc-gen-es": "^2.0.0", + "@confluentinc/kafka-javascript": "^0.2.0", "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", "@types/node": "^20.16.1", @@ -9105,13 +9365,38 @@ "node-gyp": "^9.3.1", "ts-jest": "^29.2.4", "typescript": "^5.5.4", - "typescript-eslint": "^8.2.0" + "typescript-eslint": "^8.2.0", + "uuid": "^10.0.0" + } + }, + "schemaregistry-examples": { + "version": "1.0.0", + "license": "ISC", + "devDependencies": { + "@confluentinc/kafka-javascript": "^0.2.0", + "@confluentinc/schemaregistry": "^v0.1.17.6-devel", + "uuid": "^10.0.0" + } + }, + "schemaregistry-examples/node_modules/uuid": { + "version": "10.0.0", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-10.0.0.tgz", + "integrity": "sha512-8XkAphELsDnEGrDxUOHB3RGvXz6TeuYSGEZBOjtTtPm2lwhGBjLgOzLHB63IUWfBpNucQjND6d3AOudO+H3RWQ==", + "dev": true, + "funding": [ + "https://github.com/sponsors/broofa", + "https://github.com/sponsors/ctavan" + ], + "license": "MIT", + "bin": { + "uuid": "dist/bin/uuid" } }, "schemaregistry/node_modules/uuid": { "version": "10.0.0", "resolved": "https://registry.npmjs.org/uuid/-/uuid-10.0.0.tgz", "integrity": "sha512-8XkAphELsDnEGrDxUOHB3RGvXz6TeuYSGEZBOjtTtPm2lwhGBjLgOzLHB63IUWfBpNucQjND6d3AOudO+H3RWQ==", + "dev": true, "funding": [ "https://github.com/sponsors/broofa", "https://github.com/sponsors/ctavan" diff --git a/package.json b/package.json index 1cd78f8d..b960f2e4 100644 --- a/package.json +++ b/package.json @@ -77,6 +77,7 @@ "node": ">=18.0.0" }, "workspaces": [ - "schemaregistry" + "schemaregistry", + "schemaregistry-examples" ] } diff --git a/schemaregistry-examples/package.json b/schemaregistry-examples/package.json new file mode 100644 index 00000000..29e4d4f4 --- /dev/null +++ b/schemaregistry-examples/package.json @@ -0,0 +1,15 @@ +{ + "name": "schemaregistry-examples", + "version": "1.0.0", + "main": "index.js", + "scripts": {}, + "keywords": [], + "author": "", + "license": "ISC", + "description": "", + "devDependencies": { + "@confluentinc/kafka-javascript": "^0.2.0", + "@confluentinc/schemaregistry": "^v0.1.17.6-devel", + "uuid": "^10.0.0" + } +} diff --git a/schemaregistry-examples/src/constants.ts b/schemaregistry-examples/src/constants.ts new file mode 100644 index 00000000..05065da6 --- /dev/null +++ b/schemaregistry-examples/src/constants.ts @@ -0,0 +1,27 @@ +import { BasicAuthCredentials } from '@confluentinc/schemaregistry'; + +const issuerEndpointUrl = ''; // e.g. 'https://dev-123456.okta.com/oauth2/default/v1/token'; +const clientId = ''; +const clientSecret = ''; +const scope = ''; // e.g. 'schemaregistry'; +const identityPoolId = ''; // e.g. pool-Gx30 +const logicalCluster = ''; //e.g. lsrc-a6m5op +const baseUrl = ''; // e.g. 'https://psrc-3amt5nj.us-east-1.aws.confluent.cloud' +const clusterBootstrapUrl = ''; // e.g. "pkc-p34xa.us-east-1.aws.confluent.cloud:9092" +const clusterApiKey = ''; +const clusterApiSecret = ''; + +const localAuthCredentials: BasicAuthCredentials = { + credentialsSource: 'USER_INFO', + userInfo: 'RBACAllowedUser-lsrc1:nohash', +}; + +const basicAuthCredentials: BasicAuthCredentials = { + credentialsSource: 'USER_INFO', + userInfo: ':', +}; + +export { + issuerEndpointUrl, clientId, clientSecret, scope, identityPoolId, logicalCluster, baseUrl, + clusterBootstrapUrl, clusterApiKey, clusterApiSecret, basicAuthCredentials, localAuthCredentials +}; \ No newline at end of file diff --git a/schemaregistry-examples/src/csfle-schemaregistry.ts b/schemaregistry-examples/src/csfle-schemaregistry.ts new file mode 100644 index 00000000..16bb942c --- /dev/null +++ b/schemaregistry-examples/src/csfle-schemaregistry.ts @@ -0,0 +1,155 @@ +import { + AvroSerializer, AvroSerializerConfig, SerdeType, + AvroDeserializer, ClientConfig, + SchemaRegistryClient, SchemaInfo, Rule, RuleMode, + RuleRegistry, FieldEncryptionExecutor, AwsKmsDriver, RuleSet + } from "@confluentinc/schemaregistry"; + import { CreateAxiosDefaults } from "axios"; + import { KafkaJS } from '@confluentinc/kafka-javascript'; + import { + basicAuthCredentials, clusterApiKey, clusterApiSecret, + clusterBootstrapUrl, baseUrl + } from "./constants"; + + FieldEncryptionExecutor.register(); + AwsKmsDriver.register(); + + async function csfle() { + + const schemaString: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + { + name: 'address', type: 'string', + "confluent:tags": ["PII"] + }, + ], + }); + + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: [baseUrl], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + const kafka: KafkaJS.Kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [clusterBootstrapUrl], + ssl: true, + sasl: { + mechanism: 'plain', + username: clusterApiKey, + password: clusterApiSecret, + }, + }, + }); + + const producer: KafkaJS.Producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + + let encRule: Rule = { + name: 'test-encrypt', + kind: 'TRANSFORM', + mode: RuleMode.WRITEREAD, + type: 'ENCRYPT', + tags: ['PII'], + params: { + 'encrypt.kek.name': 'csfle-example', + 'encrypt.kms.type': 'aws-kms', + 'encrypt.kms.key.id': 'your-key-id', + }, + onFailure: 'ERROR,NONE' + }; + + let ruleSet: RuleSet = { + domainRules: [encRule] + }; + + const schemaInfo: SchemaInfo = { + schemaType: 'AVRO', + schema: schemaString, + ruleSet: ruleSet + }; + + const userInfo = { name: 'Alice N Bob', age: 30, address: '369 Main St' }; + const userTopic = 'csfle-topic'; + + await schemaRegistryClient.register(userTopic+"-value", schemaInfo); + + const serializerConfig: AvroSerializerConfig = { useLatestVersion: true }; + const serializer: AvroSerializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + + const outgoingMessage = { + key: "1", + value: await serializer.serialize(userTopic, userInfo) + }; + + console.log("Outgoing Message:", outgoingMessage); + + await producer.connect(); + + await producer.send({ + topic: userTopic, + messages: [outgoingMessage] + }); + + await producer.disconnect(); + + const consumer: KafkaJS.Consumer = kafka.consumer({ + kafkaJS: { + groupId: 'demo-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + + const deserializer: AvroDeserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + await consumer.subscribe({ topic: userTopic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + console.log("Message value", message.value); + const decodedMessage = { + ...message, + value: await deserializer.deserialize(userTopic, message.value as Buffer) + }; + console.log("Decoded message", decodedMessage); + let registry = new RuleRegistry(); + const weakDeserializer: AvroDeserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}, registry); + const weakDecodedMessage = { + ...message, + value: await weakDeserializer.deserialize(userTopic, message.value as Buffer) + }; + console.log("Weak decoded message", weakDecodedMessage); + messageRcvd = true; + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + } + + csfle(); \ No newline at end of file diff --git a/schemaregistry-examples/src/kafka-consumer-avro.ts b/schemaregistry-examples/src/kafka-consumer-avro.ts new file mode 100644 index 00000000..54c4b70b --- /dev/null +++ b/schemaregistry-examples/src/kafka-consumer-avro.ts @@ -0,0 +1,72 @@ +import { SerdeType,AvroDeserializer, ClientConfig, SchemaRegistryClient } from "@confluentinc/schemaregistry"; +import { CreateAxiosDefaults } from "axios"; +import { KafkaJS } from '@confluentinc/kafka-javascript'; +import { + basicAuthCredentials, + clusterApiKey, clusterApiSecret, + clusterBootstrapUrl, baseUrl +} from "./constants"; + +async function kafkaProducerAvro() { + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: [baseUrl], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + const kafka: KafkaJS.Kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [clusterBootstrapUrl], + ssl: true, + sasl: { + mechanism: 'plain', + username: clusterApiKey, + password: clusterApiSecret, + }, + }, + }); + + const userTopic = 'example-user-topic'; + + const deserializer: AvroDeserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + const consumer: KafkaJS.Consumer = kafka.consumer({ + kafkaJS: { + groupId: 'example-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: userTopic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + console.log("Message value", message.value); + const decodedMessage = { + ...message, + value: await deserializer.deserialize(userTopic, message.value as Buffer) + }; + console.log("Decoded message", decodedMessage); + messageRcvd = true; + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); +} + +kafkaProducerAvro(); diff --git a/schemaregistry-examples/src/kafka-consumer-json.ts b/schemaregistry-examples/src/kafka-consumer-json.ts new file mode 100644 index 00000000..b3073e4d --- /dev/null +++ b/schemaregistry-examples/src/kafka-consumer-json.ts @@ -0,0 +1,78 @@ +import { + SerdeType, + JsonDeserializer, ClientConfig, + SchemaRegistryClient + } from "@confluentinc/schemaregistry"; + import { CreateAxiosDefaults } from "axios"; + import { KafkaJS } from '@confluentinc/kafka-javascript'; + import { + basicAuthCredentials, + clusterApiKey, clusterApiSecret, + clusterBootstrapUrl, baseUrl + } from "./constants"; + + async function kafkaProducerJson() { + + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: [baseUrl], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + const kafka: KafkaJS.Kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [clusterBootstrapUrl], + ssl: true, + sasl: { + mechanism: 'plain', + username: clusterApiKey, + password: clusterApiSecret, + }, + }, + }); + + const userTopic = 'example-user-topic'; + + const deserializer: JsonDeserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + const consumer: KafkaJS.Consumer = kafka.consumer({ + kafkaJS: { + groupId: 'example-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); + + await consumer.connect(); + await consumer.subscribe({ topic: userTopic }); + + let messageRcvd = false; + await consumer.run({ + eachMessage: async ({ message }) => { + console.log("Message value", message.value); + const decodedMessage = { + ...message, + value: await deserializer.deserialize(userTopic, message.value as Buffer) + }; + console.log("Decoded message", decodedMessage); + messageRcvd = true; + }, + }); + + while (!messageRcvd) { + await new Promise((resolve) => setTimeout(resolve, 100)); + } + + await consumer.disconnect(); + } + + kafkaProducerJson(); + \ No newline at end of file diff --git a/schemaregistry-examples/src/kafka-producer-avro.ts b/schemaregistry-examples/src/kafka-producer-avro.ts new file mode 100644 index 00000000..57752bc3 --- /dev/null +++ b/schemaregistry-examples/src/kafka-producer-avro.ts @@ -0,0 +1,90 @@ +import { + AvroSerializer, AvroSerializerConfig, SerdeType, + ClientConfig, SchemaRegistryClient, SchemaInfo +} from "@confluentinc/schemaregistry"; +import { CreateAxiosDefaults } from "axios"; +import { KafkaJS } from '@confluentinc/kafka-javascript'; +import { + basicAuthCredentials, + clusterApiKey, clusterApiSecret, + clusterBootstrapUrl, + baseUrl +} from "./constants"; + +async function kafkaProducerAvro() { + + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: [baseUrl], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + const kafka: KafkaJS.Kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [clusterBootstrapUrl], + ssl: true, + sasl: { + mechanism: 'plain', + username: clusterApiKey, + password: clusterApiSecret, + }, + }, + }); + + const producer: KafkaJS.Producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + + const schemaString: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + ], + }); + + const schemaInfo: SchemaInfo = { + schemaType: 'AVRO', + schema: schemaString, + }; + + const userTopic = 'example-user-topic'; + await schemaRegistryClient.register(userTopic + "-value", schemaInfo); + + const userInfo = { name: 'Alice N Bob', age: 30 }; + + const avroSerializerConfig: AvroSerializerConfig = { useLatestVersion: true }; + + const serializer: AvroSerializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, avroSerializerConfig); + + const outgoingMessage = { + key: "1", + value: await serializer.serialize(userTopic, userInfo) + }; + + console.log("Outgoing message: ", outgoingMessage); + + await producer.connect(); + + await producer.send({ + topic: userTopic, + messages: [outgoingMessage] + }); + + await producer.disconnect(); +} + +kafkaProducerAvro(); \ No newline at end of file diff --git a/schemaregistry-examples/src/kafka-producer-json.ts b/schemaregistry-examples/src/kafka-producer-json.ts new file mode 100644 index 00000000..3fed9f9d --- /dev/null +++ b/schemaregistry-examples/src/kafka-producer-json.ts @@ -0,0 +1,98 @@ +import { + JsonSerializer, JsonSerializerConfig, SerdeType, + BearerAuthCredentials, ClientConfig, + SchemaRegistryClient, SchemaInfo +} from "@confluentinc/schemaregistry"; +import { CreateAxiosDefaults } from "axios"; +import { KafkaJS } from '@confluentinc/kafka-javascript'; +import { + basicAuthCredentials, + clusterApiKey, clusterApiSecret, + clusterBootstrapUrl, + baseUrl +} from "./constants"; + +async function kafkaProducerJson() { + + + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: [baseUrl], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + basicAuthCredentials: basicAuthCredentials + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + const kafka: KafkaJS.Kafka = new KafkaJS.Kafka({ + kafkaJS: { + brokers: [clusterBootstrapUrl], + ssl: true, + sasl: { + mechanism: 'plain', + username: clusterApiKey, + password: clusterApiSecret, + }, + }, + }); + + const producer: KafkaJS.Producer = kafka.producer({ + kafkaJS: { + allowAutoTopicCreation: true, + acks: 1, + compression: KafkaJS.CompressionTypes.GZIP, + } + }); + + const schemaString: string = JSON.stringify({ + "$schema": "https://json-schema.org/draft/2020-12/schema", + "title": "User", + "type": "object", + "properties": { + "name": { + "type": "string" + }, + "age": { + "type": "integer" + } + }, + "required": ["name", "age"] + }); + + const schemaInfo: SchemaInfo = { + schemaType: 'JSON', + schema: schemaString, + }; + + const userTopic = 'example-user-topic'; + await schemaRegistryClient.register(userTopic + "-value", schemaInfo); + + const userInfo = { name: 'Alice N Bob', age: 30 }; + + const jsonSerializerConfig: JsonSerializerConfig = { useLatestVersion: true }; + + const serializer: JsonSerializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, jsonSerializerConfig); + + const outgoingMessage = { + key: "1", + value: await serializer.serialize(userTopic, userInfo) + }; + + console.log("Outgoing message: ", outgoingMessage); + + await producer.connect(); + + await producer.send({ + topic: userTopic, + messages: [outgoingMessage] + }); + + await producer.disconnect(); +} + +kafkaProducerJson(); \ No newline at end of file diff --git a/schemaregistry-examples/src/local-schemaregistry.ts b/schemaregistry-examples/src/local-schemaregistry.ts new file mode 100644 index 00000000..3a511adb --- /dev/null +++ b/schemaregistry-examples/src/local-schemaregistry.ts @@ -0,0 +1,48 @@ +import { SchemaRegistryClient, SchemaInfo, ClientConfig } from '@confluentinc/schemaregistry'; +import { v4 as uuidv4 } from 'uuid'; +import { CreateAxiosDefaults } from 'axios'; +import { localAuthCredentials } from './constants'; + +async function localDemo() { + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: ['http://localhost:8081'], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + basicAuthCredentials: localAuthCredentials, + }; + + const schemaString: string = JSON.stringify({ + type: 'record', + name: 'User', + fields: [ + { name: 'name', type: 'string' }, + { name: 'age', type: 'int' }, + ], + }); + + const schemaInfo: SchemaInfo = { + schemaType: 'AVRO', + schema: schemaString, + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + console.log("Current Subjects: ", await schemaRegistryClient.getAllSubjects()); + + const subject1 = `subject-${uuidv4()}`; + const subject2 = `subject-${uuidv4()}`; + console.log("subject1: ", subject1); + console.log("subject2: ", subject2); + + await schemaRegistryClient.register(subject1, schemaInfo); + await schemaRegistryClient.register(subject2, schemaInfo); + + console.log("Subjects After Registering: ", await schemaRegistryClient.getAllSubjects()); +} + +localDemo(); \ No newline at end of file diff --git a/schemaregistry-examples/src/oauth-schemaregistry.ts b/schemaregistry-examples/src/oauth-schemaregistry.ts new file mode 100644 index 00000000..3481abd6 --- /dev/null +++ b/schemaregistry-examples/src/oauth-schemaregistry.ts @@ -0,0 +1,39 @@ +import { SchemaRegistryClient, BearerAuthCredentials, ClientConfig } from '@confluentinc/schemaregistry'; +import { CreateAxiosDefaults } from 'axios'; +import { + issuerEndpointUrl, clientId, clientSecret, scope, + identityPoolId, logicalCluster, baseUrl +} from './constants'; + +async function oauthSchemaRegistry() { + + const bearerAuthCredentials: BearerAuthCredentials = { + credentialsSource: 'OAUTHBEARER', + issuerEndpointUrl: issuerEndpointUrl, + clientId: clientId, + clientSecret: clientSecret, + scope: scope, + identityPoolId: identityPoolId, + logicalCluster: logicalCluster + } + + const createAxiosDefaults: CreateAxiosDefaults = { + timeout: 10000 + }; + + const clientConfig: ClientConfig = { + baseURLs: [baseUrl], + createAxiosDefaults: createAxiosDefaults, + cacheCapacity: 512, + cacheLatestTtlSecs: 60, + bearerAuthCredentials: bearerAuthCredentials + }; + + const schemaRegistryClient = new SchemaRegistryClient(clientConfig); + + console.log("Current Subjects:", await schemaRegistryClient.getAllSubjects()); + console.log("Current Config:", await schemaRegistryClient.getDefaultConfig()); + console.log("Current Compatibility", await schemaRegistryClient.getDefaultCompatibility()); +} + +oauthSchemaRegistry(); \ No newline at end of file diff --git a/schemaregistry-examples/tsconfig.json b/schemaregistry-examples/tsconfig.json new file mode 100644 index 00000000..f66d3634 --- /dev/null +++ b/schemaregistry-examples/tsconfig.json @@ -0,0 +1,14 @@ +{ + "compilerOptions": { + "target": "ES6", + "module": "commonjs", + "outDir": "./dist", + "rootDir": "./src", + "strict": true, + "esModuleInterop": true, + "skipLibCheck": true, + "forceConsistentCasingInFileNames": true + }, + "include": ["src/**/*"], + "exclude": ["node_modules"] +} \ No newline at end of file diff --git a/schemaregistry/package.json b/schemaregistry/package.json index fc47ccc4..e5a2543c 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -11,6 +11,7 @@ "devDependencies": { "@bufbuild/buf": "^1.37.0", "@bufbuild/protoc-gen-es": "^2.0.0", + "@confluentinc/kafka-javascript": "^0.2.0", "@eslint/js": "^9.9.0", "@types/eslint__js": "^8.42.3", "@types/node": "^20.16.1", @@ -25,14 +26,14 @@ "node-gyp": "^9.3.1", "ts-jest": "^29.2.4", "typescript": "^5.5.4", - "typescript-eslint": "^8.2.0" + "typescript-eslint": "^8.2.0", + "uuid": "^10.0.0" }, "dependencies": { "@aws-sdk/client-kms": "^3.637.0", "@azure/identity": "^4.4.1", "@azure/keyvault-keys": "^4.8.0", "@bufbuild/protobuf": "^2.0.0", - "@confluentinc/kafka-javascript": "^0.1.17-devel", "@criteria/json-schema": "^0.10.0", "@criteria/json-schema-validation": "^0.10.0", "@google-cloud/kms": "^4.5.0", @@ -48,7 +49,6 @@ "lru-cache": "^11.0.0", "node-vault": "^0.10.2", "simple-oauth2": "^5.1.0", - "uuid": "^10.0.0", "validator": "^13.12.0" }, "scripts": { diff --git a/schemaregistry/rest-service.ts b/schemaregistry/rest-service.ts index 537ec7f4..8d929c82 100644 --- a/schemaregistry/rest-service.ts +++ b/schemaregistry/rest-service.ts @@ -58,9 +58,10 @@ export class RestService { this.baseURLs = baseURLs; if (isForward) { - this.client.defaults.headers.common['X-Forward'] = 'true' + this.setHeaders({ 'X-Forward': 'true' }); } - + this.setHeaders({ 'Content-Type': 'application/vnd.schemaregistry.v1+json' }); + this.handleBasicAuth(basicAuthCredentials); this.handleBearerAuth(bearerAuthCredentials); } diff --git a/test/schemaregistry/test-constants.ts b/test/schemaregistry/test-constants.ts index a17fa6dd..fe5096c5 100644 --- a/test/schemaregistry/test-constants.ts +++ b/test/schemaregistry/test-constants.ts @@ -6,9 +6,6 @@ const baseUrls = ['http://localhost:8081']; const mockBaseUrls = ['http://mocked-url']; const createAxiosDefaults: CreateAxiosDefaults = { - headers: { - 'Content-Type': 'application/vnd.schemaregistry.v1+json', - }, timeout: 10000 }; From b9be5373adc15e1137c1fbf08200b971c9f28680 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Tue, 1 Oct 2024 15:30:54 -0400 Subject: [PATCH 222/224] bugfix integ tests for registering -value (#71) (#115) * bugfix integ tests for registering -value * Add autoRegister and useLatestVersion tests --- .../schemaregistry-avro.spec.ts | 64 +++++----------- .../schemaregistry-client.spec.ts | 3 +- .../schemaregistry-json.spec.ts | 73 +++++++++---------- 3 files changed, 55 insertions(+), 85 deletions(-) diff --git a/e2e/schemaregistry/schemaregistry-avro.spec.ts b/e2e/schemaregistry/schemaregistry-avro.spec.ts index de5bb240..4f8eabf8 100644 --- a/e2e/schemaregistry/schemaregistry-avro.spec.ts +++ b/e2e/schemaregistry/schemaregistry-avro.spec.ts @@ -12,7 +12,11 @@ import stringify from 'json-stringify-deterministic'; import { v4 } from 'uuid'; let schemaRegistryClient: SchemaRegistryClient; -let producer: any; +let serializerConfig: AvroSerializerConfig; +let serializer: AvroSerializer; +let deserializer: AvroDeserializer; +let producer: KafkaJS.Producer; +let consumer: KafkaJS.Consumer; const kafkaBrokerList = 'localhost:9092'; const kafka = new KafkaJS.Kafka({ @@ -48,11 +52,6 @@ const schemaInfo: SchemaInfo = { metadata: metadata }; -let serializerConfig: AvroSerializerConfig; -let serializer: AvroSerializer; -let deserializer: AvroDeserializer; -let consumer: KafkaJS.Consumer; - describe('Schema Registry Avro Integration Test', () => { beforeEach(async () => { @@ -78,15 +77,14 @@ describe('Schema Registry Avro Integration Test', () => { afterEach(async () => { await producer.disconnect(); - producer = null; }); it("Should serialize and deserialize Avro", async () => { - const testTopic = 'test-topic-' + v4(); + const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); + await schemaRegistryClient.register(testTopic + "-value", schemaInfo); - serializerConfig = { autoRegisterSchemas: true }; + serializerConfig = { useLatestVersion: true }; serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); deserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); @@ -123,28 +121,8 @@ describe('Schema Registry Avro Integration Test', () => { await consumer.disconnect(); }, 30000); - it("Should serialize with UseLatestVersion enabled", async () => { + it('Should fail to serialize with useLatestVersion enabled and autoRegisterSchemas disabled', async () => { const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); - - serializerConfig = { autoRegisterSchemas: true, useLatestVersion: true }; - serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); - - const outgoingMessage = { - key: 'key', - value: await serializer.serialize(testTopic, messageValue) - }; - - await producer.send({ - topic: testTopic, - messages: [outgoingMessage] - }); - - }, 30000); - - it('Should fail to serialize with UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { - const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); serializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); @@ -154,12 +132,11 @@ describe('Schema Registry Avro Integration Test', () => { await expect(serializer.serialize(testTopic, messageValue)).rejects.toThrowError(); }); - it('Should serialize with schemas registered, UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { + it('Should serialize with autoRegisterSchemas enabled and useLatestVersion disabled', async () => { const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); - await schemaRegistryClient.register(testTopic+'-value', schemaInfo); + await schemaRegistryClient.register(testTopic +' -value', schemaInfo); - serializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; + serializerConfig = { autoRegisterSchemas: true, useLatestVersion: false }; serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); const messageValue = { "name": "Bob Jones", "age": 25 }; @@ -257,7 +234,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { } }); await producer.connect(); - serializerConfig = { autoRegisterSchemas: true }; + serializerConfig = { useLatestVersion: true }; serializer = new AvroSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); deserializer = new AvroDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); @@ -272,7 +249,6 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { afterEach(async () => { await producer.disconnect(); - producer = null; }); it('Should serialize and deserialize string', async () => { @@ -287,7 +263,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(stringTopic, stringSchemaInfo); + await schemaRegistryClient.register(stringTopic + "-value", stringSchemaInfo); const stringMessageValue = "Hello, World!"; const outgoingStringMessage = { @@ -335,7 +311,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(topic, stringSchemaInfo); + await schemaRegistryClient.register(topic + "-value", stringSchemaInfo); const messageValue = Buffer.from("Hello, World!"); const outgoingMessage = { @@ -383,7 +359,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(topic, stringSchemaInfo); + await schemaRegistryClient.register(topic + "-value", stringSchemaInfo); const messageValue = 25; const outgoingMessage = { @@ -431,7 +407,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(topic, stringSchemaInfo); + await schemaRegistryClient.register(topic + "-value", stringSchemaInfo); const messageValue = 25; const outgoingMessage = { @@ -479,7 +455,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(topic, stringSchemaInfo); + await schemaRegistryClient.register(topic + "-value", stringSchemaInfo); const messageValue = true; const outgoingMessage = { @@ -527,7 +503,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(topic, stringSchemaInfo); + await schemaRegistryClient.register(topic + "-value", stringSchemaInfo); const messageValue = 1.354; const outgoingMessage = { @@ -575,7 +551,7 @@ describe('Schema Registry Avro Integration Test - Primitives', () => { metadata: metadata }; - await schemaRegistryClient.register(topic, stringSchemaInfo); + await schemaRegistryClient.register(topic + "-value", stringSchemaInfo); const messageValue = 1.354; const outgoingMessage = { diff --git a/e2e/schemaregistry/schemaregistry-client.spec.ts b/e2e/schemaregistry/schemaregistry-client.spec.ts index 79380a22..70dcda93 100644 --- a/e2e/schemaregistry/schemaregistry-client.spec.ts +++ b/e2e/schemaregistry/schemaregistry-client.spec.ts @@ -8,6 +8,7 @@ import { } from '../../schemaregistry/schemaregistry-client'; import { beforeEach, describe, expect, it } from '@jest/globals'; import { clientConfig } from '../../test/schemaregistry/test-constants'; +import { v4 } from 'uuid'; /* eslint-disable @typescript-eslint/no-non-null-asserted-optional-chain */ @@ -77,7 +78,7 @@ describe('SchemaRegistryClient Integration Test', () => { }); it("Should return RestError when retrieving non-existent schema", async () => { - await expect(schemaRegistryClient.getBySubjectAndId(testSubject, 1)).rejects.toThrow(); + await expect(schemaRegistryClient.getLatestSchemaMetadata(v4())).rejects.toThrow(); }); it('Should register, retrieve, and delete a schema', async () => { diff --git a/e2e/schemaregistry/schemaregistry-json.spec.ts b/e2e/schemaregistry/schemaregistry-json.spec.ts index 4ab46584..08b85138 100644 --- a/e2e/schemaregistry/schemaregistry-json.spec.ts +++ b/e2e/schemaregistry/schemaregistry-json.spec.ts @@ -13,7 +13,12 @@ import stringify from 'json-stringify-deterministic'; import { v4 } from 'uuid'; let schemaRegistryClient: SchemaRegistryClient; -let producer: any; +let serializerConfig: JsonSerializerConfig; +let serializer: JsonSerializer; +let deserializer: JsonDeserializer; +let producer: KafkaJS.Producer; +let consumer: KafkaJS.Consumer; + const kafkaBrokerList = 'localhost:9092'; const kafka = new KafkaJS.Kafka({ @@ -248,12 +253,6 @@ const schemaInfo: SchemaInfo = { schemaType: 'JSON' }; -const customerSubject = 'Customer'; -const orderSubject = 'Order'; -const orderDetailsSubject = 'OrderDetails'; - -const subjectList = [orderSubject, orderDetailsSubject, customerSubject]; - describe('SchemaRegistryClient json Integration Test', () => { beforeEach(async () => { @@ -267,35 +266,28 @@ describe('SchemaRegistryClient json Integration Test', () => { } }); await producer.connect(); - const subjects: string[] = await schemaRegistryClient.getAllSubjects(); - - for (const subject of subjectList) { - if (subjects && subjects.includes(subject)) { - await schemaRegistryClient.deleteSubject(subject); - await schemaRegistryClient.deleteSubject(subject, true); - const subjectValue = subject + '-value'; - if (subjects && subjects.includes(subjectValue)) { - await schemaRegistryClient.deleteSubject(subjectValue); - await schemaRegistryClient.deleteSubject(subjectValue, true); - } - } - } + consumer = kafka.consumer({ + kafkaJS: { + groupId: 'test-group', + fromBeginning: true, + partitionAssigners: [KafkaJS.PartitionAssigners.roundRobin], + }, + }); }); afterEach(async () => { await producer.disconnect(); - producer = null; }); it("Should serialize and deserialize json", async () => { const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); + await schemaRegistryClient.register(testTopic + "-value", schemaInfo); - const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true }; - const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); - const deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + serializerConfig = { useLatestVersion: true }; + serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); const outgoingMessage = { key: 'key', @@ -307,7 +299,7 @@ describe('SchemaRegistryClient json Integration Test', () => { messages: [outgoingMessage] }); - let consumer = kafka.consumer({ + consumer = kafka.consumer({ kafkaJS: { groupId: 'test-group', fromBeginning: true, @@ -338,12 +330,11 @@ describe('SchemaRegistryClient json Integration Test', () => { await consumer.disconnect(); }, 30000); - it("Should serialize with UseLatestVersion enabled", async () => { + it("Should serialize with autoRegisterSchemas enabled and useLatestVersion disabled", async () => { const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); - const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true, useLatestVersion: true }; - const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + serializerConfig = { autoRegisterSchemas: true, useLatestVersion: false }; + serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); const outgoingMessage = { key: 'key', @@ -359,10 +350,9 @@ describe('SchemaRegistryClient json Integration Test', () => { it('Should fail to serialize with UseLatestVersion enabled and autoRegisterSchemas disabled', async () => { const testTopic = v4(); - await schemaRegistryClient.register(testTopic, schemaInfo); - const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; - const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + serializerConfig = { autoRegisterSchemas: false, useLatestVersion: true }; + serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); const messageValue = { "name": "Bob Jones", "age": 25 }; @@ -371,9 +361,12 @@ describe('SchemaRegistryClient json Integration Test', () => { it("Should serialize referenced schemas", async () => { const testTopic = v4(); - const serializerConfig: JsonSerializerConfig = { autoRegisterSchemas: true }; - const serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); - const deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + serializerConfig = { useLatestVersion: true }; + serializer = new JsonSerializer(schemaRegistryClient, SerdeType.VALUE, serializerConfig); + deserializer = new JsonDeserializer(schemaRegistryClient, SerdeType.VALUE, {}); + + const customerSubject = v4(); + const orderDetailsSubject = v4(); await schemaRegistryClient.register(customerSubject, customerSchema); const customerIdVersion: number = (await schemaRegistryClient.getLatestSchemaMetadata(customerSubject)).version!; @@ -395,7 +388,7 @@ describe('SchemaRegistryClient json Integration Test', () => { }; orderSchema.references = [orderDetailsReference]; - await schemaRegistryClient.register(orderSubject, orderSchema); + await schemaRegistryClient.register(testTopic + "-value", orderSchema); const order = { order_details: { @@ -412,7 +405,7 @@ describe('SchemaRegistryClient json Integration Test', () => { const outgoingMessage = { key: 'key', - value: await serializer.serialize(orderSubject, order) + value: await serializer.serialize(testTopic, order) }; await producer.send({ @@ -420,7 +413,7 @@ describe('SchemaRegistryClient json Integration Test', () => { messages: [outgoingMessage] }); - let consumer = kafka.consumer({ + consumer = kafka.consumer({ kafkaJS: { groupId: 'test-group', fromBeginning: true, @@ -437,7 +430,7 @@ describe('SchemaRegistryClient json Integration Test', () => { eachMessage: async ({ message }) => { const decodedMessage = { ...message, - value: await deserializer.deserialize(orderSubject, message.value as Buffer) + value: await deserializer.deserialize(testTopic, message.value as Buffer) }; messageRcvd = true; From 14eef59deab54bae19efe8736e72049b746fbbe9 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 3 Oct 2024 12:20:06 +0530 Subject: [PATCH 223/224] Bump version to v0.2.1 (#116) --- CHANGELOG.md | 10 ++++++++++ lib/util.js | 2 +- package-lock.json | 5 ++--- package.json | 2 +- 4 files changed, 14 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 870fe748..11802730 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,8 +1,18 @@ +# confluent-kafka-javascript v0.2.1 + +v0.2.1 is a limited availability release. It is supported for all usage. + +## Features + +1. Update README, docs, and examples for Confluent's Schema Registry client. + + # confluent-kafka-javascript v0.2.0 v0.2.0 is a limited availability release. It is supported for all usage. ## Features + 1. Switch to using `librdkafka` on the latest released tag `v2.5.3` instead of `master`. diff --git a/lib/util.js b/lib/util.js index f2d5bc26..2d4ca508 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = 'v0.2.0'; +util.bindingVersion = 'v0.2.1'; diff --git a/package-lock.json b/package-lock.json index 192eabb6..c69d17e0 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.2.0", + "version": "v0.2.1", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "v0.2.0", + "version": "v0.2.1", "hasInstallScript": true, "license": "MIT", "workspaces": [ @@ -1545,7 +1545,6 @@ "integrity": "sha512-IWjyGRqeDBcWmYcEQHu1XlZQ6am5qzzIEb18rdxFZkFeVQ6piG28bQ6BmlqvHn3zd+XoAO+e8bRlpgdUuTLC9Q==", "dev": true, "hasInstallScript": true, - "license": "MIT", "workspaces": [ "schemaregistry" ], diff --git a/package.json b/package.json index b960f2e4..92b9d0a0 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "v0.2.0", + "version": "v0.2.1", "description": "Node.js bindings for librdkafka", "librdkafka": "2.5.3", "librdkafka_win": "2.5.3", From 49f6d5aa45cf1da37b6548fed83979ffe75916b6 Mon Sep 17 00:00:00 2001 From: Justin Wang <35274216+Claimundefine@users.noreply.github.com> Date: Thu, 10 Oct 2024 17:56:47 -0400 Subject: [PATCH 224/224] Update version to 0.2.1 for EA release (#72) (#117) --- schemaregistry/package.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/schemaregistry/package.json b/schemaregistry/package.json index e5a2543c..abd64e2b 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "v0.1.17.6-devel", + "version": "v0.2.1", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts",