From b789352fffb19b63bbe811a952fff84599608853 Mon Sep 17 00:00:00 2001 From: Robert Shepherd Date: Mon, 17 Aug 2020 17:01:01 +0800 Subject: [PATCH] Update to remove old kafka modules (#154) * Removed kafka-node and node-rdkafka to only support kafkaJS. Updated tests to reflect * Cleaned up old config * Converted to typescript. Added types where missing. * Fixed tests, such as ensuring message sending order * Upgraded wurstmeister/kafka docker to 2.11-1.1.1. Updated stop script to remove volumes to ensure we can run clean. Updated CHANGELOG * Removed debug code. Removed empty methods and updated Analytics to set client as abstract so the consumer/producer is forced to set the client * Added typescript eslint * Fixed error linting errors * Added missing new line Co-authored-by: Robert Shepherd --- .eslintrc.js | 12 +- .gitignore | 3 + .npmignore | 4 +- CHANGELOG.md | 6 + README.md | 57 +- index.d.ts | 382 ----- index.js | 1 - kafka-setup/docker-compose.yml | 2 +- kafka-setup/stop.sh | 2 +- lib/Sinek.js | 35 - lib/connect/Consumer.js | 122 -- lib/connect/Producer.js | 195 --- lib/connect/README.md | 3 - lib/kafka/Drainer.js | 469 ------ lib/kafka/Kafka.js | 430 ------ lib/kafka/PartitionDrainer.js | 539 ------- lib/kafka/PartitionQueue.js | 167 --- lib/kafka/Publisher.js | 515 ------- lib/librdkafka/NConsumer.js | 1266 ----------------- lib/librdkafka/NProducer.js | 711 --------- lib/librdkafka/README.md | 3 - lib/shared/index.js | 5 - package.json | 30 +- src/index.ts | 1 + src/lib/Sinek.ts | 4 + src/lib/interfaces.ts | 277 ++++ .../lib/kafkajs/JSConsumer.ts | 437 +++--- .../lib/kafkajs/JSProducer.ts | 410 +++--- src/lib/kafkajs/index.ts | 2 + .../lib/shared/Analytics.ts | 141 +- .../lib/shared/CompressionTypes.ts | 6 +- .../Health.js => src/lib/shared/Health.ts | 76 +- .../Metadata.js => src/lib/shared/Metadata.ts | 82 +- src/lib/shared/index.ts | 6 + test/config.js | 88 -- test/config.ts | 44 + test/int/{Health.test.js => Health.test.ts} | 41 +- test/int/JSSinek.test.js | 102 -- test/int/JSSinek.test.ts | 104 ++ test/int/NSinek.test.js | 127 -- tsconfig.dist.json | 14 + tsconfig.json | 21 + yarn.lock | 645 +++------ 43 files changed, 1377 insertions(+), 6210 deletions(-) delete mode 100644 index.d.ts delete mode 100644 index.js delete mode 100644 lib/Sinek.js delete mode 100644 lib/connect/Consumer.js delete mode 100644 lib/connect/Producer.js delete mode 100644 lib/connect/README.md delete mode 100644 lib/kafka/Drainer.js delete mode 100644 lib/kafka/Kafka.js delete mode 100644 lib/kafka/PartitionDrainer.js delete mode 100644 lib/kafka/PartitionQueue.js delete mode 100644 lib/kafka/Publisher.js delete mode 100644 lib/librdkafka/NConsumer.js delete mode 100644 lib/librdkafka/NProducer.js delete mode 100644 lib/librdkafka/README.md delete mode 100644 lib/shared/index.js create mode 100644 src/index.ts create mode 100644 src/lib/Sinek.ts create mode 100644 src/lib/interfaces.ts rename lib/kafkajs/JSConsumer.js => src/lib/kafkajs/JSConsumer.ts (64%) rename lib/kafkajs/JSProducer.js => src/lib/kafkajs/JSProducer.ts (62%) create mode 100644 src/lib/kafkajs/index.ts rename lib/shared/Analytics.js => src/lib/shared/Analytics.ts (66%) rename lib/shared/CompressionTypes.js => src/lib/shared/CompressionTypes.ts (71%) rename lib/shared/Health.js => src/lib/shared/Health.ts (77%) rename lib/shared/Metadata.js => src/lib/shared/Metadata.ts (58%) create mode 100644 src/lib/shared/index.ts delete mode 100644 test/config.js create mode 100644 test/config.ts rename test/int/{Health.test.js => Health.test.ts} (80%) delete mode 100644 test/int/JSSinek.test.js create mode 100644 test/int/JSSinek.test.ts delete mode 100644 test/int/NSinek.test.js create mode 100644 tsconfig.dist.json create mode 100644 tsconfig.json diff --git a/.eslintrc.js b/.eslintrc.js index 29194d4..f0c8757 100644 --- a/.eslintrc.js +++ b/.eslintrc.js @@ -4,10 +4,14 @@ module.exports = { "node": true, "mocha": true }, - "parserOptions": { - "sourceType": "module", - "ecmaVersion": 2017 - }, + "parser": '@typescript-eslint/parser', + "plugins": [ + '@typescript-eslint', + ], + "extends": [ + 'eslint:recommended', + 'plugin:@typescript-eslint/recommended', + ], "rules": { "indent": [ "error", diff --git a/.gitignore b/.gitignore index e35139e..ab7f666 100644 --- a/.gitignore +++ b/.gitignore @@ -40,3 +40,6 @@ logs .vscode certs + +# generated files +dist diff --git a/.npmignore b/.npmignore index 7f7c70d..432c052 100644 --- a/.npmignore +++ b/.npmignore @@ -10,4 +10,6 @@ test/ .eslintrc.js .gitignore .jshintrc -.travis.yml \ No newline at end of file +.travis.yml +src +.github/ \ No newline at end of file diff --git a/CHANGELOG.md b/CHANGELOG.md index 4fb8be2..d088469 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,11 @@ # sinek CHANGELOG +## 2020-08-10, Version 10.0.0 +** **BREAKING removed deprecated client versions +* **BREAKING** removed node-rdkafka +* KafkaJS and primary connection to kafka +* Convert to typescript. + ## 2020-04-19, Version 9.1.0 * support for message headers in NProducer diff --git a/README.md b/README.md index 5165fb5..8214416 100644 --- a/README.md +++ b/README.md @@ -34,46 +34,43 @@ npm install --save sinek ## Usage -### Usage - Native Client (based on node-rdkafka) +### Usage - JS Client (based on kafka.js) -#### Please Note: +```javascript +const { + JSConsumer, + JSProducer +} = require("sinek"); -You will have to manually install `node-rdkafka` alongside sinek. -(This requires a Node.js version between 9 and 12 and will not work with Node.js >= 13, last tested with 12.16.1) +const jsProducerConfig = { + clientId: "my-app", + brokers: ["kafka1:9092"] +} -On Mac OS High Sierra / Mojave: -`CPPFLAGS=-I/usr/local/opt/openssl/include LDFLAGS=-L/usr/local/opt/openssl/lib yarn add --frozen-lockfile node-rdkafka@2.7.4` +(async () => { -Otherwise: -`yarn add --frozen-lockfile node-rdkafka@2.7.4` + const topic = "my-topic"; -(Please also note: Doing this with npm does not work, it will remove your deps, `npm i -g yarn`) + const producer = new JSProducer(jsProducerConfig); + const consumer = new JSConsumer(topic, jsConsumerConfig); -```javascript -const { - NConsumer, - NProducer -} = require("sinek"); -``` + producer.on("error", error => console.error(error)); + consumer.on("error", error => console.error(error)); -* [Native Client (NConsumer & NProducer)](docs/native.md) + await consumer.connect(); -### Usage - JS Client (based on kafka.js) + // consume from a topic. + consumer.consume(async (messages) => { + messages.forEach((message) => { + console.log(message); + }) + }); -```javascript -const { - JSConsumer, - JSProducer -} = require("sinek"); -``` - -### Usage - Old JS Client (based on kafka-node) + // Produce messages to a topic. + await producer.connect(); + producer.send(topic, "a message") +})().catch(console.error); -```javascript -const { - Consumer, - Producer -} = require("sinek"); ``` # Further Docs diff --git a/index.d.ts b/index.d.ts deleted file mode 100644 index 595613e..0000000 --- a/index.d.ts +++ /dev/null @@ -1,382 +0,0 @@ -// declare type Buffer = any; - -declare module "sinek" { - - export interface KafkaHealthConfig { - thresholds?: { - consumer?: { - errors?: number; - lag?: number; - stallLag?: number; - minMessages?: number; - }; - producer?: { - errors?: number; - minMessages?: number; - }; - }; - } - - export interface NCommonKafkaOptions { - "builtin.features"?: string; - "client.id"?: string; - "metadata.broker.list": string; - "message.max.bytes"?: number; - "message.copy.max.bytes"?: number; - "receive.message.max.bytes"?: number; - "max.in.flight.requests.per.connection"?: number; - "metadata.request.timeout.ms"?: number; - "topic.metadata.refresh.interval.ms"?: number; - "metadata.max.age.ms"?: number; - "topic.metadata.refresh.fast.interval.ms"?: number; - "topic.metadata.refresh.fast.cnt"?: number; - "topic.metadata.refresh.sparse"?: boolean; - "topic.blacklist"?: string; - "debug"?: string; - "socket.timeout.ms"?: number; - "socket.blocking.max.ms"?: number; - "socket.send.buffer.bytes"?: number; - "socket.receive.buffer.bytes"?: number; - "socket.keepalive.enable"?: boolean; - "socket.nagle.disable"?: boolean; - "socket.max.fails"?: number; - "broker.address.ttl"?: number; - "broker.address.family"?: "any" | "v4" | "v6"; - "reconnect.backoff.jitter.ms"?: number; - "statistics.interval.ms"?: number; - "enabled_events"?: number; - "log_level"?: number; - "log.queue"?: boolean; - "log.thread.name"?: boolean; - "log.connection.close"?: boolean; - "internal.termination.signal"?: number; - "api.version.request"?: boolean; - "api.version.fallback.ms"?: number; - "broker.version.fallback"?: string; - "security.protocol"?: "plaintext" | "ssl" | "sasl_plaintext" | "sasl_ssl"; - "ssl.cipher.suites"?: string; - "ssl.key.location"?: string; - "ssl.key.password"?: string; - "ssl.certificate.location"?: string; - "ssl.ca.location"?: string; - "ssl.crl.location"?: string; - "sasl.mechanisms"?: string; - "sasl.kerberos.service.name"?: string; - "sasl.kerberos.principal"?: string; - "sasl.kerberos.kinit.cmd"?: string; - "sasl.kerberos.keytab"?: string; - "sasl.kerberos.min.time.before.relogin"?: number; - "sasl.username"?: string; - "sasl.password"?: string; - "partition.assignment.strategy"?: string; - "session.timeout.ms"?: number; - "heartbeat.interval.ms"?: number; - "group.protocol.type"?: string; - "coordinator.query.interval.ms"?: number; - "group.id"?: string; - "event_cb"?: boolean; - "dr_cb"?: boolean; - } - - export interface NConsumerKafkaOptions extends NCommonKafkaOptions { - "group.id": string; - "enable.auto.commit"?: boolean; - "auto.commit.interval.ms"?: number; - "enable.auto.offset.store"?: boolean; - "queued.min.messages"?: number; - "queued.max.messages.kbytes"?: number; - "fetch.wait.max.ms"?: number; - "fetch.message.max.bytes"?: number; - "fetch.min.bytes"?: number; - "fetch.error.backoff.ms"?: number; - "offset.store.method"?: "none" | "file" | "broker"; - "enable.partition.eof"?: boolean; - "check.crcs"?: boolean; - } - - export interface NProducerKafkaOptions extends NCommonKafkaOptions { - "queue.buffering.max.messages"?: number; - "queue.buffering.max.kbytes"?: number; - "queue.buffering.max.ms"?: number; - "message.send.max.retries"?: number; - "retry.backoff.ms"?: number; - "compression.codec"?: "none" | "gzip" | "snappy" | "lz4"; - "batch.num.messages"?: number; - "delivery.report.only.error"?: boolean; - } - - export interface KafkaConsumerConfig { - kafkaHost?: string; - groupId?: string; - workerPerPartition?: number; - options?: { - sessionTimeout?: number; - protocol?: [string]; - fromOffset?: string; - fetchMaxBytes?: number; - fetchMinBytes?: number; - fetchMaxWaitMs?: number; - heartbeatInterval?: number; - retryMinTimeout?: number; - autoCommit?: boolean; - autoCommitIntervalMs?: number; - requireAcks?: number; - ackTimeoutMs?: number; - }; - health?: KafkaHealthConfig; - tconf?: { - "auto.commit.enable"?: boolean; - "auto.commit.interval.ms"?: number; - "auto.offset.reset"?: "smallest" | "earliest" | "beginning" | "largest" | "latest" | "end" | "error"; - "offset.store.path"?: string; - "offset.store.sync.interval.ms"?: number; - "offset.store.method"?: "file" | "broker"; - "consume.callback.max.messages"?: number; - }; - noptions?: NConsumerKafkaOptions; - logger?: KafkaLogger; - } - - export interface KafkaProducerConfig { - kafkaHost?: string; - clientName?: string; - workerPerPartition?: number; - options?: { - sessionTimeout?: number; - protocol?: [string]; - fromOffset?: string; - fetchMaxBytes?: number; - fetchMinBytes?: number; - fetchMaxWaitMs?: number; - heartbeatInterval?: number; - retryMinTimeout?: number; - requireAcks?: number; - ackTimeoutMs?: number; - partitionerType?: number; - }; - health?: KafkaHealthConfig; - tconf?: { - "request.required.acks"?: number; - "request.timeout.ms"?: number; - "message.timeout.ms"?: number; - "produce.offset.report"?: boolean; - }; - noptions?: NProducerKafkaOptions; - logger?: KafkaLogger; - } - - export interface KafkaMessage { - topic: string; - partition: number; - offset: number; - key: Buffer | string; - value: Buffer | string | any; - size: number; - timestamp: number; - } - - export interface SortedMessageBatch { - [topic: string]: { - [partition: string]: KafkaMessage[]; - }; - } - - export interface BatchConfig { - batchSize?: number; - commitEveryNBatch?: number; - concurrency?: number; - commitSync?: boolean; - noBatchCommits?: boolean; - manualBatching?: boolean; - sortedManualBatch?: boolean; - } - - export interface ConsumerStats { - totalIncoming: number; - lastMessage: number; - receivedFirstMsg: boolean; - totalProcessed: number; - lastProcessed: number; - queueSize: null; - isPaused: boolean; - drainStats: null; - omittingQueue: boolean; - autoComitting: boolean; - consumedSinceCommit: number; - batch: { - current: number; - committs: number; - total: number; - config: BatchConfig; - currentEmptyFetches: number; - avgProcessingTime: number; - }; - lag: any; - totalErrors: number; - } - - export interface LagStatus { - topic: string; - partition: number; - lowDistance: number; - highDistance: number; - detail: { - lowOffset: number; - highOffset: number; - comittedOffset: number; - }; - } - - export interface ProducerStats { - totalPublished: number; - last: number; - isPaused: boolean; - totalErrors: number; - } - - export interface MessageReturn { - key: string; - partition: number; - offset?: number | null; - } - - export class NConsumer { - constructor(topic: Array | string, config: KafkaConsumerConfig); - on(eventName: "message", callback: (message: KafkaMessage) => any): void; - on(eventName: "error", callback: (error: any) => any): void; - on(eventName: "ready", callback: () => any): void; - on(eventName: "analytics", callback: (analytics: object) => void): void; - on(eventName: "batch", callback: (batch: Array) => void): void; - on(eventName: "first-drain-message", callback: () => void): void; - connect(asStream?: boolean, opts?: {asString?: boolean, asJSON?: boolean}): Promise; - - consume(syncEvent?: (message: KafkaMessage | KafkaMessage[] | SortedMessageBatch, callback: (error?: any) => void) => void, - asString?: boolean, asJSON?: boolean, options?: BatchConfig): Promise; - - pause(topics: Array): void; - resume(topics: Array): void; - getStats(): ConsumerStats; - close(commit?: boolean): object; - enableAnalytics(options: object): void; - haltAnalytics(): void; - addSubscriptions(topics: Array): Array; - adjustSubscription(topics: Array): Array; - commit(async: boolean): boolean; - commitMessage(async: boolean, message: KafkaMessage): boolean; - commitOffsetHard(topic: string, partition: number, offset: number, async: boolean): boolean; - commitLocalOffsetsForTopic(topic: string): any; - getOffsetForTopicPartition(topic: string, partition: number, timeout: number): Promise; - getComittedOffsets(timeout: number): Promise>; - getAssignedPartitions(): Array; - static findPartitionOffset(topic: string, partition: number, offsets: Array): object; - getLagStatus(noCache: boolean): Promise>; - getAnalytics(): object; - checkHealth(): object; - getTopicMetadata(topic: string, timeout?: number): Promise; - getMetadata(timeout?: number): Promise; - getTopicList(): Promise>; - } - - export class NProducer { - constructor(config: KafkaProducerConfig, _?: null, defaultPartitionCount?: number | "auto") - on(eventName: "error", callback: (error: any) => any): void; - on(eventName: "ready", callback: () => any): void; - connect(): Promise; - - send(topicName: string, message: string | Buffer, _partition?: number, _key?: string | Buffer, - _partitionKey?: string, _opaqueKey?: string): Promise; - - buffer(topic: string, identifier: string | undefined, payload: object, partition?: number, - version?: number, partitionKey?: string): Promise; - - bufferFormat(topic: string, identifier: string | undefined, payload: object, version?: number, - compressionType?: number, partitionKey?: string): Promise; - - bufferFormatPublish(topic: string, identifier: string | undefined, _payload: object, version?: number, - _?: null, partitionKey?: string, partition?: number): Promise; - - bufferFormatUpdate(topic: string, identifier: string | undefined, _payload: object, version?: number, - _?: null, partitionKey?: string, partition?: number): Promise; - - bufferFormatUnpublish(topic: string, identifier: string | undefined, _payload: object, version?: number, - _?: null, partitionKey?: string, partition?: number): Promise; - - pause(): void; - resume(): void; - getStats(): ProducerStats; - refreshMetadata(topics: Array): void; - close(): object; - enableAnalytics(options: object): void; - haltAnalytics(): void; - getAnalytics(): object; - checkHealth(): object; - getTopicMetadata(topic: string, timeout?: number): Promise; - getMetadata(timeout?: number): Promise; - getTopicList(): Promise>; - getPartitionCountOfTopic(topic: string): Promise; - getStoredPartitionCounts(): object; - tombstone(topic: string, key: string | Buffer, _partition?: number | null): Promise; - } - - export class Consumer { - constructor(topic: String, config: KafkaConsumerConfig); - on(eventName: "message", callback: (message: object) => any): void; - on(eventName: "error", callback: (error: any) => any): void; - connect(backpressure?: boolean): Promise; - consume(syncEvent?: object): Promise; - consumeOnce(syncEvent?: object, drainThreshold?: number, timeout?: number): object; - pause(): void; - resume(): void; - getStats(): object; - close(commit?: boolean): object; - } - - export class Producer { - constructor(config: KafkaProducerConfig, topic: Array, defaultPartitionCount: number); - on(eventName: "error", callback: (error: any) => any): void; - connect(): Promise; - send(topic: string, message: string | string[]): Promise; - buffer(topic: string, identifier?: string, payload?: object, compressionType?: number): Promise; - bufferFormat(topic: string, identifier?: string, payload?: object, version?: number, compressionType?: number, partitionKey?: string): Promise; - bufferFormatPublish(topic: string, identifier?: string, payload?: object, version?: number, compressionType?: number, partitionKey?: string): Promise; - bufferFormatUpdate(topic: string, identifier?: string, payload?: object, version?: number, compressionType?: number, partitionKey?: string): Promise; - bufferFormatUnpublish(topic: string, identifier?: string, payload?: object, version?: number, compressionType?: number, partitionKey?: string): Promise; - pause(): void; - resume(): void; - getStats(): object; - refreshMetadata(topics: Array): void; - close(): object; - } - - export class Kafka { - constructor(conString: String, logger: object, connectDirectlyToBroker: boolean) - } - - export class Drainer { - constructor(consumer: object, asyncLimit: number, autoJsonParsing: boolean, omitQueue: boolean, commitOnDrain: boolean) - } - - export class Publisher { - constructor(producer: object, partitionCount: number, autoFlushBuffer: number, flushPeriod: number) - } - - export class PartitionDrainer { - constructor(consumer: object, asyncLimit: number, commitOnDrain: boolean, autoJsonParsing: boolean) - } - - export class PartitionQueue { - constructor(partition: object, drainEvent: object, drainer: object, asyncLimit: number, queueDrain: object) - } - - export interface MessageType { - key: String; - value: String; - } - - export interface KafkaLogger { - debug(message: string): void; - info(message: string): void; - warn(message: string, error?: Error): void; - error(error: string | Error): void; - } -} diff --git a/index.js b/index.js deleted file mode 100644 index eb7aee2..0000000 --- a/index.js +++ /dev/null @@ -1 +0,0 @@ -module.exports = require("./lib/Sinek.js"); \ No newline at end of file diff --git a/kafka-setup/docker-compose.yml b/kafka-setup/docker-compose.yml index 231251f..661f148 100644 --- a/kafka-setup/docker-compose.yml +++ b/kafka-setup/docker-compose.yml @@ -10,7 +10,7 @@ services: - ./server-jaas.conf:/etc/kafka/server-jaas.conf kafka: - image: wurstmeister/kafka:0.10.2.1 + image: wurstmeister/kafka:2.11-1.1.1 ports: - "9092:9092" - "9093:9093" diff --git a/kafka-setup/stop.sh b/kafka-setup/stop.sh index 02b6732..0a8e303 100755 --- a/kafka-setup/stop.sh +++ b/kafka-setup/stop.sh @@ -1,3 +1,3 @@ #!/usr/bin/env sh BASEDIR=$(git rev-parse --show-toplevel) -docker-compose --file ${BASEDIR}/kafka-setup/docker-compose.yml down +docker-compose --file ${BASEDIR}/kafka-setup/docker-compose.yml rm -vfs diff --git a/lib/Sinek.js b/lib/Sinek.js deleted file mode 100644 index baa04f2..0000000 --- a/lib/Sinek.js +++ /dev/null @@ -1,35 +0,0 @@ -"use strict"; - -const util = require("util"); - -const Kafka = require("./kafka/Kafka.js"); -const Drainer = require("./kafka/Drainer.js"); -const Publisher = require("./kafka/Publisher.js"); -const PartitionDrainer = require("./kafka/PartitionDrainer.js"); - -const Consumer = require("./connect/Consumer.js"); -const Producer = require("./connect/Producer.js"); - -const NConsumer = require("./librdkafka/NConsumer.js"); -const NProducer = require("./librdkafka/NProducer.js"); - -const JSConsumer = require("./kafkajs/JSConsumer.js"); -const JSProducer = require("./kafkajs/JSProducer.js"); - -const Health = require("./shared/Health.js"); -const Analytics = require("./shared/Analytics.js"); - -module.exports = { - Kafka: util.deprecate(Kafka, "Kafka is deprecated, please use 'NConsumer' if possible."), - Drainer: util.deprecate(Drainer, "Drainer is deprecated, please use 'NConsumer' if possible."), - PartitionDrainer: util.deprecate(PartitionDrainer, "PartitionDrainer is deprecated, please use 'NConsumer' if possible."), - Publisher: util.deprecate(Publisher, "Publisher is deprecated, please use 'NProducer' if possible."), - Consumer: util.deprecate(Consumer, "Consumer is deprecated, please use (noptions) 'NConsumer' if possible."), - Producer: util.deprecate(Producer, "Producer is deprecated, please use (noptions) 'NProducer' if possible."), - NConsumer, - NProducer, - JSConsumer, - JSProducer, - Health, - Analytics -}; diff --git a/lib/connect/Consumer.js b/lib/connect/Consumer.js deleted file mode 100644 index 488c746..0000000 --- a/lib/connect/Consumer.js +++ /dev/null @@ -1,122 +0,0 @@ -"use strict"; - -const Promise = require("bluebird"); -const EventEmitter = require("events"); - -const Kafka = require("./../kafka/Kafka.js"); -const Drainer = require("./../kafka/Drainer.js"); - -class Consumer extends EventEmitter { - - constructor(topic, config = { options: {} }) { - super(); - - this.topic = topic; - this.config = config; - - this.kafkaConsumerClient = null; - this.consumer = null; - } - - connect(backpressure = false) { - return new Promise(resolve => { - - const { zkConStr, kafkaHost, logger, groupId, workerPerPartition, options } = this.config; - - let conStr = null; - - if(typeof kafkaHost === "string"){ - conStr = kafkaHost; - } - - if(typeof zkConStr === "string"){ - conStr = zkConStr; - } - - if(conStr === null){ - throw new Error("One of the following: zkConStr or kafkaHost must be defined."); - } - - this.kafkaConsumerClient = new Kafka(conStr, logger, conStr === kafkaHost); - - this.kafkaConsumerClient.on("ready", () => resolve()); - this.kafkaConsumerClient.on("error", error => super.emit("error", error)); - - this.kafkaConsumerClient.becomeConsumer([this.topic], groupId, options); - - const commitManually = !options.autoCommit; - this.consumer = new Drainer(this.kafkaConsumerClient, workerPerPartition, false, !backpressure, commitManually); - }); - } - - consume(syncEvent = null) { - return new Promise(resolve => { - - this.consumer.drain((message, done) => { - - super.emit("message", message); - - if (!syncEvent) { - return done(); - } - - syncEvent(message, () => { - - /* ### sync event callback does not handle errors ### */ - - done(); - }); - }); - - this.consumer.once("first-drain-message", () => resolve()); - }); - } - - consumeOnce(syncEvent = null, drainThreshold = 10000, timeout = 0) { - return this.consumer.drainOnce((message, done) => { - - super.emit("message", message); - - if (!syncEvent) { - return done(); - } - - syncEvent(message, () => { - - /* ### sync event callback does not handle errors ### */ - - done(); - }); - - }, drainThreshold, timeout); - } - - pause() { - - if (this.consumer) { - this.consumer.pause(); - } - } - - resume() { - - if (this.consumer) { - this.consumer.resume(); - } - } - - getStats() { - return this.consumer ? this.consumer.getStats() : {}; - } - - close(commit = false) { - - if (this.consumer) { - const result = this.consumer.close(commit); - this.consumer = null; - return result; - } - } -} - -module.exports = Consumer; diff --git a/lib/connect/Producer.js b/lib/connect/Producer.js deleted file mode 100644 index 7b6d53d..0000000 --- a/lib/connect/Producer.js +++ /dev/null @@ -1,195 +0,0 @@ -"use strict"; - -const EventEmitter = require("events"); -const Promise = require("bluebird"); -const uuid = require("uuid"); - -const Kafka = require("./../kafka/Kafka.js"); -const Publisher = require("./../kafka/Publisher.js"); - -class Producer extends EventEmitter { - - constructor(config, topic = [], defaultPartitionCount = 1) { - super(); - - this.targetTopics = Array.isArray(topic) ? topic : [topic]; - this.config = config; - - this.kafkaProducerClient = null; - this.producer = null; - this.defaultPartitionCount = defaultPartitionCount; - } - - connect() { - return new Promise(resolve => { - - const { zkConStr, kafkaHost, logger, clientName, options } = this.config; - - let conStr = null; - - if(typeof kafkaHost === "string"){ - conStr = kafkaHost; - } - - if(typeof zkConStr === "string"){ - conStr = zkConStr; - } - - if(conStr === null){ - throw new Error("One of the following: zkConStr or kafkaHost must be defined."); - } - - this.kafkaProducerClient = new Kafka(conStr, logger, conStr === kafkaHost); - - this.kafkaProducerClient.on("ready", () => resolve()); - this.kafkaProducerClient.on("error", error => super.emit("error", error)); - - this.kafkaProducerClient.becomeProducer(this.targetTopics, clientName, options); - this.producer = new Publisher(this.kafkaProducerClient, this.defaultPartitionCount); - }); - } - - send(topic, message) { - - if (!this.producer) { - return Promise.reject("producer is not yet setup."); - } - - let partition = -1; - if (this.defaultPartitionCount < 2) { - partition = 0; - } else { - partition = Producer._getRandomIntInclusive(0, this.defaultPartitionCount - 1); - } - - return this.producer.send(topic, - Array.isArray(message) ? message : [message], - null, - partition, - 0 - ); - } - - buffer(topic, identifier, payload, compressionType = 0) { - - if (!this.producer) { - return Promise.reject("producer is not yet setup."); - } - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - return this.producer.appendBuffer(topic, identifier, payload, compressionType); - } - - bufferFormat(topic, identifier, payload, version = 1, compressionType = 0, partitionKey = null) { - - if (!this.producer) { - return Promise.reject("producer is not yet setup."); - } - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - return this.producer.bufferPublishMessage(topic, identifier, payload, version, compressionType, partitionKey); - } - - bufferFormatPublish(topic, identifier, payload, version = 1, compressionType = 0, partitionKey = null) { - - if (!this.producer) { - return Promise.reject("producer is not yet setup."); - } - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - return this.producer.bufferPublishMessage(topic, identifier, payload, version, compressionType, partitionKey); - } - - bufferFormatUpdate(topic, identifier, payload, version = 1, compressionType = 0, partitionKey = null) { - - if (!this.producer) { - return Promise.reject("producer is not yet setup."); - } - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - return this.producer.bufferUpdateMessage(topic, identifier, payload, version, compressionType, partitionKey); - } - - bufferFormatUnpublish(topic, identifier, payload, version = 1, compressionType = 0, partitionKey = null) { - - if (!this.producer) { - return Promise.reject("producer is not yet setup."); - } - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - return this.producer.bufferUnpublishMessage(topic, identifier, payload, version, compressionType, partitionKey); - } - - pause() { - - if (this.producer) { - this.producer.pause(); - } - } - - resume() { - - if (this.producer) { - this.producer.resume(); - } - } - - getStats() { - return this.producer ? this.producer.getStats() : {}; - } - - refreshMetadata(topics = []) { - return this.producer.refreshMetadata(topics); - } - - close() { - - if (this.producer) { - this.producer.close(); - this.producer = null; - } - } - - static _getRandomIntInclusive(min, max) { - min = Math.ceil(min); - max = Math.floor(max); - return Math.floor(Math.random() * (max - min + 1)) + min; - } -} - -module.exports = Producer; diff --git a/lib/connect/README.md b/lib/connect/README.md deleted file mode 100644 index c19921a..0000000 --- a/lib/connect/README.md +++ /dev/null @@ -1,3 +0,0 @@ -# JS Client Docs - -Have moved [here](../../docs/connect.md). \ No newline at end of file diff --git a/lib/kafka/Drainer.js b/lib/kafka/Drainer.js deleted file mode 100644 index 4bacbc6..0000000 --- a/lib/kafka/Drainer.js +++ /dev/null @@ -1,469 +0,0 @@ -"use strict"; - -const Kafka = require("./Kafka.js"); -const async = require("async"); -const Promise = require("bluebird"); - -const DEFAULT_DRAIN_INTV = 3000; -const NOOP = () => { }; - -class Drainer { - - constructor(consumer = null, asyncLimit = 1, autoJsonParsing = true, omitQueue = false, commitOnDrain = false) { - - if (!consumer || !(consumer instanceof Kafka) || - !consumer.isConsumer) { - throw new Error("Consumer is not a valid Sinek Kafka(Consumer)"); - } - - if (omitQueue && commitOnDrain) { - throw new Error("Cannot run drain commits when queue is omitted. Please either run: " + - " a manual committer with backpressure OR an auto-commiter without backpressure."); - } - - this.consumer = consumer; - this.raw = consumer.consumer; - - this.asyncLimit = asyncLimit; - this.commitOnDrain = commitOnDrain; - this.autoJsonParsing = autoJsonParsing; - this.omitQueue = omitQueue; - - this._drainEvent = null; - this._q = null; - - this._lastProcessed = Date.now(); - this._lastReceived = Date.now(); - - this._totalIncomingMessages = 0; - this._totalProcessedMessages = 0; - - this._messagesSinceLastDrain = 0; - this._receivedFirst = false; - this._drainStart = null; - - this._stats = {}; - - this._lastMessageHandlerRef = null; - - this.DRAIN_INTV = DEFAULT_DRAIN_INTV; - } - - _getLogger() { - return this.consumer._getLogger(); - } - - /** - * stops any active drain process - * closes the consumer and its client - */ - close() { - - if (this._lastMessageHandlerRef) { - this.raw.removeListener("message", this._lastMessageHandlerRef); - this._lastMessageHandlerRef = null; - } else { - this._getLogger().warn("message handler ref not present during close, could not remove listener."); - } - - this._getLogger().info("[Drainer] closed."); - return this.consumer.close(); - } - - /** - * returns a few insights - * @returns {{totalIncoming: number, last: (number|*), isPaused: *}} - */ - getStats() { - return { - totalIncoming: this._totalIncomingMessages, - lastMessage: this._lastReceived, - - receivedFirstMsg: this._receivedFirst, - - totalProcessed: this._totalProcessedMessages, - lastProcessed: this._lastProcessed, - - queueSize: this._q ? this._q.length() : null, - - isPaused: this.consumer && this.consumer.isConsumer ? this.isPaused() : null, - - drainStats: this._stats, - omittingQueue: this.omitQueue - }; - } - - /** - * resets all offsets and starts from being - * also un-pauses consumer if necessary - * @param topics - * @returns {Promise.} - */ - resetConsumer() { - return Promise.reject("resetConsumer has been removed, due to supporting bad kafka consumer behaviour."); - } - - /** - * resets all offsets and makes sure the consumer is paused - * @param topics - * @returns {Promise.} - */ - resetOffset() { - return Promise.reject("resetOffset has been removed, due to supporting bad kafka consumer behaviour."); - } - - /** - * main reg. function, pass it a function to receive messages - * under flow control - * @param drainEvent - */ - drain(drainEvent = null) { - - if (!drainEvent || typeof drainEvent !== "function") { - throw new Error("drainEvent must be a valid function"); - } - - if (this._drainEvent) { - throw new Error("a drain process is currently active."); - } - - //reset - this._lastProcessed = Date.now(); - this._lastReceived = Date.now(); - this._stats = {}; - - this._messagesSinceLastDrain = this._totalIncomingMessages; - this._drainEvent = drainEvent; - this._startToReceiveMessages(); - - if (this.isPaused()) { - this.resume(); - } - } - - /** - * main req. function, pass it a function to receive messages - * under flow control, until they are stall for a certain amount - * of time (e.g. when all messages on the queue are consumed) - * returns a Promise - * @param drainEvent - * @param drainThreshold - * @param timeout - */ - drainOnce(drainEvent = null, drainThreshold = 10000, timeout = 0) { - return new Promise((resolve, reject) => { - - if (!drainEvent || typeof drainEvent !== "function") { - return reject("drainEvent must be a valid function"); - } - - if (this._drainEvent) { - return reject("a drain process is currently active."); - } - - if (timeout !== 0 && timeout < this.DRAIN_INTV) { - return reject(`timeout must be either 0 or > ${this.DRAIN_INTV}.`); - } - - if (timeout !== 0 && timeout <= drainThreshold) { - return reject(`timeout ${timeout} must be greater than the drainThreshold ${drainThreshold}.`); - } - - let t = null; - let intv = null; - - intv = setInterval(() => { - - const spanProcessed = Date.now() - this._lastProcessed; - const spanReceived = Date.now() - this._lastReceived; - - this._getLogger().debug("drainOnce interval running, current span-rec: " + - `${spanReceived} / span-proc: ${spanProcessed} ms.`); - - //set stats - this._countStats("intv-cycle"); - this._stats["last-proc-since"] = spanProcessed; - this._stats["last-rec-since"] = spanReceived; - - //choose the smaller span - const span = spanProcessed < spanReceived ? spanProcessed : spanReceived; - - if (span >= drainThreshold) { - this._getLogger().info(`drainOnce span ${span} hit threshold ${drainThreshold}.`); - clearInterval(intv); - clearTimeout(t); - this.stopDrain(); - resolve(this._totalIncomingMessages - this._messagesSinceLastDrain); - } - }, this.DRAIN_INTV); - - if (timeout !== 0) { - this._getLogger().info(`drainOnce timeout active: ${timeout} ms.`); - t = setTimeout(() => { - this._getLogger().warn(`drainOnce timeout hit after ${timeout} ms.`); - clearInterval(intv); - this.stopDrain(); - reject("drainOnce ran into timeout."); - }, timeout); - } - - //start the drain process - this.drain(drainEvent); - }); - } - - /** - * stops any active drain process - */ - stopDrain() { - - if (!this._drainEvent) { - throw new Error("there is no drain active."); - } - - //reset - if (this._lastMessageHandlerRef) { - this.raw.removeListener("message", this._lastMessageHandlerRef); - this._lastMessageHandlerRef = null; - } else { - this._getLogger().warn("message handler ref not present during close, could not remove listener."); - } - - this._drainEvent = null; - this._q = null; - this._receivedFirst = false; - - const duration = (Date.now() - this._drainStart) / 1000; - this._getLogger().info(`[Drainer] stopped drain process, had been open for ${duration} seconds.`); - } - - /** - * removes kafka topics (if broker allows this action) - * @param topics - */ - removeTopics(topics = []) { - return new Promise((resolve, reject) => { - this._getLogger().info(`deleting topics ${JSON.stringify(topics)}.`); - this.raw.client.removeTopicMetadata(topics, (err, data) => { - - if (err) { - return reject(err); - } - - resolve(data); - }); - }); - } - - pause() { - - if (!this.isPaused()) { - this._countStats("paused"); - } - - return this.consumer.pause(); - } - - resume() { - - if (this.isPaused()) { - this._countStats("resumed"); - } - - return this.consumer.resume(); - } - - isPaused() { - return this.consumer.isPaused(); - } - - _startToReceiveMessages() { - - if (!this.omitQueue) { - this._startToReceiveMessagesThroughQueue(); - } else { - this._startToReceiveMessagesWithoutQueue(); - } - } - - _startToReceiveMessagesThroughQueue() { - - this._q = async.queue((msg, done) => { - if (this._drainEvent) { - setImmediate(() => this._drainEvent(msg, err => { - this._lastProcessed = Date.now(); - this._totalProcessedMessages++; - done(err); - })); - } else { - this._getLogger().debug("drainEvent not present, message is dropped."); - } - }, this.asyncLimit); - - this._q.drain = () => { - - if (!this.commitOnDrain) { - return this.resume(); - } - - //commit state first, before resuming - this._getLogger().debug("committing manually, reason: drain event."); - this._commit().then(() => { - this._getLogger().debug("committed successfully, resuming."); - this.resume(); - }).catch(error => { - this._getLogger().error("failed to commit offsets, resuming anyway after: " + error); - this.resume(); - }); - }; - - this._q.error(err => { - if (err) { - this._countStats("msg-process-fail"); - this._getLogger().warn("error was passed back to consumer queue, dropping it silently: " + JSON.stringify(err)); - } - }); - - this._lastMessageHandlerRef = this._onMessageForQueue.bind(this); - this.raw.on("message", this._lastMessageHandlerRef); - this._getLogger().info("[Drainer] started drain process."); - this._drainStart = Date.now(); - } - - _commit() { - return this.consumer.commitCurrentOffsets(); - } - - _startToReceiveMessagesWithoutQueue() { - - this._lastMessageHandlerRef = this._onMessageNoQueue.bind(this); - this.raw.on("message", this._lastMessageHandlerRef); - this._getLogger().info("[Drainer] started drain process."); - this._drainStart = Date.now(); - } - - /** - * with backpressure - * @param {*} message - */ - _onMessageForQueue(message) { - - this._getLogger().debug("received kafka message => length: " + (message.value && message.value.length) + ", offset: " + - message.offset + ", partition: " + message.partition + ", on topic: " + message.topic); - - if (this.autoJsonParsing) { - try { - message.value = JSON.parse(message.value); - } catch (e) { - this._countStats("msg-parse-fail"); - return this.emit("error", "failed to json parse message value: " + message); - } - - if (!message.value) { - this._countStats("msg-empty"); - return this.emit("error", "message value is empty: " + message); - } - } - - this._q.push(message); - //error handling happens directly on the queue object initialisation - - this.pause(); - - this._totalIncomingMessages++; - this._lastReceived = Date.now(); - - if (!this._receivedFirst) { - this._receivedFirst = true; - this._getLogger().info("consumer received first message."); - this.emit("first-drain-message", message); - } - } - - /** - * no backpressure - * @param {*} message - */ - _onMessageNoQueue(message) { - - this._getLogger().debug("received kafka message => length: " + (message.value && message.value.length) + ", offset: " + - message.offset + ", partition: " + message.partition + ", on topic: " + message.topic); - - if (this.autoJsonParsing) { - try { - message.value = JSON.parse(message.value); - } catch (e) { - this._countStats("msg-parse-fail"); - return this.emit("error", "failed to json parse message value: " + message); - } - - if (!message.value) { - this._countStats("msg-empty"); - return this.emit("error", "message value is empty: " + message); - } - } - - this._totalIncomingMessages++; - this._lastReceived = Date.now(); - - if (this._drainEvent) { - this._drainEvent(message, NOOP); - this._lastProcessed = Date.now(); - this._totalProcessedMessages++; - } - - if (!this._receivedFirst) { - this._receivedFirst = true; - this._getLogger().info("consumer received first message."); - this.emit("first-drain-message", message); - } - } - - _countStats(key) { - - if (!this._stats) { - return; - } - - if (!this._stats[key]) { - this._stats[key] = 1; - return; - } - - this._stats[key]++; - } - - /** - * consumer proxy - * @param args - */ - on(...args) { - this.consumer.on(...args); - } - - /** - * consumer proxy - * @param args - */ - once(...args) { - this.consumer.once(...args); - } - - /** - * consumer proxy - * @param args - */ - removeListener(...args) { - this.consumer.removeListener(...args); - } - - /** - * consumer proxy - * @param args - */ - emit(...args) { - this.consumer.emit(...args); - } -} - -module.exports = Drainer; diff --git a/lib/kafka/Kafka.js b/lib/kafka/Kafka.js deleted file mode 100644 index 7c829c0..0000000 --- a/lib/kafka/Kafka.js +++ /dev/null @@ -1,430 +0,0 @@ -"use strict"; - -const EventEmitter = require("events"); -const { ConsumerGroup, Offset, Client, KafkaClient, HighLevelProducer } = require("kafka-node"); -const Promise = require("bluebird"); -const debug = require("debug"); - -const NOOPL = { - debug: debug("sinek:debug"), - info: debug("sinek:info"), - warn: debug("sinek:warn"), - error: debug("sinek:error") -}; - -const DEFAULT_RETRY_OPTIONS = { - retries: 1000, // overwritten by forever - factor: 3, - minTimeout: 1000, // 1 sec - maxTimeout: 30000, // 30 secs - randomize: true, - forever: true, - unref: false -}; - -class Kafka extends EventEmitter { - - constructor(conString, logger = null, connectDirectlyToBroker = false) { - - super(); - - this.conString = conString; - this.connectDirectlyToBroker = connectDirectlyToBroker; - this.client = null; - - //consumer - this.consumer = null; - this.offset = null; - this.isConsumer = false; - this._autoCommitEnabled = null; - this._isManual = false; - - //producer - this.isProducer = false; - this.producer = null; - this.targetTopics = []; - - this._logger = logger; - this._producerReadyFired = false; - } - - getPartitions(topic) { - return new Promise((resolve, reject) => { - - if (!this.client) { - return reject("client is not defined yet, cannot create offset to gather partitions."); - } - - const offset = new Offset(this.client); - offset.fetchEarliestOffsets([topic], (err, data) => { - - if (err || !data[topic]) { - return reject("failed to get offsets of topic: " + topic + "; " + err); - } - - resolve(Object.keys(data[topic]).map(key => key)); - }); - }); - } - - getEarliestOffsets(topic) { - return new Promise((resolve, reject) => { - - if (!this.client) { - return reject("client is not defined yet, cannot create offset to reset."); - } - - const offset = new Offset(this.client); - offset.fetchEarliestOffsets([topic], (err, data) => { - - if (err || !data[topic]) { - return reject("failed to get offsets of topic: " + topic + "; " + err); - } - - resolve(data[topic]); - }); - }); - } - - getOffsets(topic) { - return new Promise((resolve, reject) => { - - if (!this.client) { - return reject("client is not defined yet, cannot create offset to reset."); - } - - const offset = new Offset(this.client); - offset.fetchLatestOffsets([topic], (err, data) => { - - if (err || !data[topic]) { - return reject("failed to get offsets of topic: " + topic + "; " + err); - } - - resolve(data[topic]); - }); - }); - } - - getTopics() { - return this.targetTopics; - } - - hardOffsetReset() { - return Promise.reject("hardOffsetReset has been removed, as it was supporting bad kafka consumer behaviour."); - } - - _getLogger() { - - if (this._logger) { - return this._logger; - } - - return NOOPL; - } - - setConsumerOffset(topic = "t", partition = 0, offset = 0) { - this._getLogger().debug("adjusting offset for topic: " + topic + " on partition: " + partition + " to " + offset); - this.consumer.setOffset(topic, partition, offset); - } - - commitCurrentOffsets() { - return new Promise((resolve, reject) => { - this.consumer.commit((err, data) => { - - if (err) { - return reject(err); - } - - resolve(data); - }); - }); - } - - becomeManualConsumer(topics, groupId, options, dontListenForSIGINT) { - this._isManual = true; - return this.becomeConsumer(topics, groupId, options, dontListenForSIGINT, false); - } - - becomeConsumer(topics = ["t"], groupId = "kafka-node-group", _options = {}, dontListenForSIGINT = false, autoCommit = true) { - - if (!Array.isArray(topics) || topics.length <= 0) { - throw new Error("becomeConsumer requires a valid topics array, with at least a single topic."); - } - - if (this.isConsumer) { - throw new Error("this kafka instance has already been initialised as consumer."); - } - - if (this.isProducer) { - throw new Error("this kafka instance has already been initialised as producer."); - } - - if (!groupId) { - throw new Error("missing groupId or consumer configuration."); - } - - const options = { - host: this.connectDirectlyToBroker ? undefined : this.conString, - kafkaHost: this.connectDirectlyToBroker ? this.conString : undefined, - //zk: undefined, - //batch: undefined, - ssl: false, - groupId: groupId, - sessionTimeout: 30000, - protocol: ["roundrobin"], - fromOffset: "earliest", // latest - migrateHLC: false, - migrateRolling: false, - fetchMaxBytes: 1024 * 100, - fetchMinBytes: 1, - fetchMaxWaitMs: 100, - autoCommit: autoCommit, - autoCommitIntervalMs: 5000, - connectRetryOptions: this.connectDirectlyToBroker ? DEFAULT_RETRY_OPTIONS : undefined, - encoding: "buffer", - keyEncoding: "buffer" - }; - - //overwrite default options - _options = _options || {}; - Object.keys(_options).forEach(key => options[key] = _options[key]); - - this._autoCommitEnabled = options.autoCommit; - - this.consumer = new ConsumerGroup(options, topics); - this.client = this.consumer.client; - this.isConsumer = true; - this.pause(); - - this.targetTopics = topics; - this._getLogger().info("starting ConsumerGroup for topic: " + JSON.stringify(topics)); - - this._attachConsumerListeners(dontListenForSIGINT); - } - - becomeProducer(targetTopics = ["t"], clientId = "kafka-node-client", _options = {}) { - - if (this.isConsumer) { - throw new Error("this kafka instance has already been initialised as consumer."); - } - - if (this.isProducer) { - throw new Error("this kafka instance has already been initialised as producer."); - } - - const options = { - requireAcks: 1, - ackTimeoutMs: 100, - partitionerType: 3 - }; - - //overwrite default options - _options = _options || {}; - Object.keys(_options).forEach(key => options[key] = _options[key]); - - this.client = null; - if (this.connectDirectlyToBroker) { - - const kafkaOptions = { - kafkaHost: this.conString, - ssl: !!_options.sslOptions, - sslOptions: _options.sslOptions, - connectTimeout: 1000, - requestTimeout: 30000, - autoConnect: _options.autoConnect || true, - connectRetryOptions: DEFAULT_RETRY_OPTIONS - }; - - this.client = new KafkaClient(kafkaOptions); - } else { - this.client = new Client(this.conString, clientId, {}, _options.sslOptions || {}); - } - - this.producer = new HighLevelProducer(this.client, _options); - this.isProducer = true; - - this._getLogger().info("starting Producer."); - this.targetTopics = targetTopics; - this._attachProducerListeners(); - } - - _attachProducerListeners() { - - this.client.on("connect", () => { - this._getLogger().info("producer is connected."); - }); - - this.producer.on("ready", () => { - - this._getLogger().debug("producer ready fired."); - if (this._producerReadyFired) { - return; - } - - this._producerReadyFired = true; - this._getLogger().info("producer is ready."); - - //prevents key-partition errors - this.refreshMetadata(this.targetTopics).then(() => { - this.emit("ready"); - }); - }); - - this.producer.on("error", error => { - //dont log these, they emit very often - this.emit("error", error); - }); - } - - _attachConsumerListeners(dontListenForSIGINT = false, commitOnSIGINT = false) { - - this.consumer.once("connect", () => { - this._getLogger().info("consumer is connected / ready."); - this.emit("connect"); - this.emit("ready"); - }); - - //do not listen for "message" here - - this.consumer.on("error", error => { - //dont log these, they emit very often - this.emit("error", error); - }); - - this.consumer.on("offsetOutOfRange", error => { - //dont log these, they emit very often - this.emit("error", error); - }); - - //prevents re-balance errors - if (!dontListenForSIGINT) { - process.on("SIGINT", () => { - if (this.consumer) { - this.consumer.close(commitOnSIGINT, () => { - process.exit(); - }); - } - }); - } - } - - _resetConsumer() { - this.isConsumer = false; - this.client = null; - this.consumer = null; - } - - _resetProducer() { - this.isProducer = false; - this.client = null; - this.producer = null; - this._producerReadyFired = false; - } - - _closeConsumer(commit) { - return new Promise((resolve, reject) => { - - this._getLogger().info("trying to close consumer."); - - if (!this.consumer) { - return reject("consumer is null"); - } - - if (!commit) { - - this.consumer.close(() => { - this._resetConsumer(); - resolve(); - }); - - return; - } - - this._getLogger().info("trying to commit kafka consumer before close."); - - this.consumer.commit((err, data) => { - - if (err) { - return reject(err); - } - - this.consumer.close(() => { - this._resetConsumer(); - resolve(data); - }); - }); - }); - } - - _closeProducer() { - return new Promise((resolve, reject) => { - - this._getLogger().info("trying to close producer."); - - if (!this.producer) { - return reject("producer is null"); - } - - this.producer.close(() => { - this._resetProducer(); - resolve(true); - }); - }); - } - - refreshMetadata(topics = []) { - - if (!topics || topics.length <= 0) { - return Promise.resolve(); - } - - return new Promise(resolve => { - this.client.refreshMetadata(topics, () => { - this._getLogger().info("meta-data refreshed."); - resolve(); - }); - }); - } - - isPaused() { - - if (this.isConsumer) { - return this.consumer.paused; - } - - return false; - } - - pause() { - - if (this.isConsumer) { - return this.consumer.pause(); - } - - return false; - } - - resume() { - - if (this.isConsumer) { - return this.consumer.resume(); - } - - return false; - } - - close(commit = false) { - - if (this.isConsumer) { - return this._closeConsumer(commit); - } - - if (this.isProducer) { - return this._closeProducer(); - } - - return null; - } -} - -module.exports = Kafka; diff --git a/lib/kafka/PartitionDrainer.js b/lib/kafka/PartitionDrainer.js deleted file mode 100644 index 580f938..0000000 --- a/lib/kafka/PartitionDrainer.js +++ /dev/null @@ -1,539 +0,0 @@ -"use strict"; - -const Kafka = require("./Kafka.js"); -const Promise = require("bluebird"); - -const PartitionQueue = require("./PartitionQueue.js"); - -const DEFAULT_DRAIN_INTV = 3000; - -class PartitionDrainer { - - constructor(consumer = null, asyncLimit = 1, commitOnDrain = false, autoJsonParsing = true) { - - if (!consumer || !(consumer instanceof Kafka) || - !consumer.isConsumer) { - throw new Error("consumer is not a valid Sinek Kafka(Consumer)"); - } - - this.consumer = consumer; - this.raw = consumer.consumer; - - this.asyncLimit = asyncLimit; - this.commitOnDrain = commitOnDrain; - this.autoJsonParsing = autoJsonParsing; - - this._queueMap = null; - this._drainMap = {}; - - this._totalIncomingMessages = 0; - this._incomingSinceLastDrain = 0; - - this._lastReceived = Date.now(); - this._receivedFirst = false; - this._drainStart = null; - - this._lastMessageHandlerRef = null; - - this._stats = {}; - - this.DRAIN_INTV = DEFAULT_DRAIN_INTV; - - this._drainTargetTopic = null; - this.disablePauseResume = false; - } - - _getLogger() { - return this.consumer._getLogger(); - } - - /** - * gets all partitions of the given topic - * and builds a map of async.queues (PartionQueue) - * with a single queue for each partition - * they will all call the same queue-drain callback to control the message flow - * and they will all call the same drain-event callback to process messages - * queues also expose their own stats - * @param topic - * @param drainEvent - * @param asyncLimit - * @returns {*} - * @private - */ - _buildOffsetMap(topic, drainEvent, asyncLimit = 1) { - - if (typeof topic !== "string") { - return Promise.reject("offset map can only be build for a single topic."); - } - - if (typeof drainEvent !== "function") { - return Promise.reject("drainEvent must be a valid function."); - } - - if (this.consumer.getTopics().indexOf(topic) === -1) { - return Promise.reject(topic + " is not a supported topic, it has to be set during becomeConsumer()."); - } - - return this.consumer.getPartitions(topic).then(partitions => { - - if (!partitions || partitions.length <= 0) { - return Promise.reject(`partitions request for topic ${topic} returned empty.`); - } - - const queueMap = {}; - const drainMap = {}; - - partitions.forEach(partition => { - //build a parition queue for each partition - queueMap[partition] = new PartitionQueue(partition, drainEvent, this, asyncLimit, - this._onPartitionQueueDrain.bind(this)).build(); - }); - - partitions.forEach(partition => { - //drain map is build to check if all queues have been drained - drainMap[partition] = false; - }); - - return { - queueMap, - drainMap - }; - }); - } - - /** - * partiton queue drain callback that makes sure to resume the consumer if - * all queues have drained - * @param partition - * @param offset - * @private - */ - _onPartitionQueueDrain(partition) { - - if (typeof this._drainMap[partition] === "undefined") { - this._getLogger().warn(`partition queue drain called but ${partition} is not a present key.`); - return; - } - - this._drainMap[partition] = true; - - //this queue drained, lets commit the latest offset - if (this.commitOnDrain && (this.consumer._isManual || !this.consumer._autoCommitEnabled)) { - - if (this.consumer._autoCommitEnabled) { - throw new Error("you have started a consumer with auto commit enabled, but requested partition drainer" + - "to run commits manually for you - both cannot work at the same time."); - } - - //running setConsumerOffset while commit manually is a bad idea - //message offset is already hold in the client, only committing is needed - //this.consumer.setConsumerOffset(this._drainTargetTopic, partition, offset); - } - - if (Object.keys(this._drainMap).map(key => this._drainMap[key]).filter(v => !v).length) { - this._getLogger().debug("not all partition queues have drained yet."); - } else { - this._getLogger().debug("all partition queues have drained."); - - // reset drain map - Object.keys(this._drainMap).forEach(key => { - this._drainMap[key] = false; - }); - - if (!this.commitOnDrain) { - if (!this.disablePauseResume) { - this.resume(); - } - return; //do not execute commit logic^ - } - - //resume consumer, which will cause new message to be pushed into the queues - //but make sure to commit current offsets first - this.consumer.commitCurrentOffsets().then(() => { - if (!this.disablePauseResume) { - this.resume(); - } - }).catch(e => { - this._getLogger().error(`failed to commit offsets after all partitions have been drained. ${e}.`); - if (!this.disablePauseResume) { - this.resume(); - } - }); - } - } - - _resetQueueMaps() { - - this._getLogger().info("resetting queue maps."); - - if (this._queueMap) { - Object.keys(this._queueMap).forEach(key => { - this._queueMap[key].close(); - }); - this._queueMap = null; - this._drainMap = {}; - } - } - - /** - * stops any active drain process - * closes the consumer and its client - */ - close() { - - if (this._lastMessageHandlerRef) { - this.raw.removeListener("message", this._lastMessageHandlerRef); - this._lastMessageHandlerRef = null; - } else { - this._getLogger().warn("message handler ref not present during close, could not remove listener."); - } - - this._resetQueueMaps(); - - this._getLogger().info("[Drainer] closed."); - return this.consumer.close(); - } - - /** - * returns a few insights - * @returns {{totalIncoming: number, last: (number|*), isPaused: *}} - */ - getStats() { - return { - totalIncoming: this._totalIncomingMessages, - lastMessage: this._lastReceived, - - receivedFirstMsg: this._receivedFirst, - - isPaused: this.consumer && this.consumer.isConsumer ? this.isPaused() : null, - - drainStats: this._stats, - partitions: this._queueMap ? Object.keys(this._queueMap).length : null, - queues: this._queueMap ? Object.keys(this._queueMap).map(key => this._queueMap[key].getStats()) : null - }; - } - - /** - * resets all offsets and starts from being - * also un-pauses consumer if necessary - * @param topics - * @returns {Promise.} - */ - resetConsumer() { - return Promise.reject("resetConsumer has been removed, due to supporting bad kafka consumer behaviour."); - } - - /** - * resets all offsets and makes sure the consumer is paused - * @param topics - * @returns {Promise.} - */ - resetOffset() { - return Promise.reject("resetOffset has been removed, due to supporting bad kafka consumer behaviour."); - } - - /** - * main reg. function, pass it a function to receive messages - * under flow control, returns a promise - * @param topic - * @param drainEvent - */ - drain(topic = "t", drainEvent = null) { - - this._drainTargetTopic = topic; - - if (!drainEvent || typeof drainEvent !== "function") { - throw new Error("drainEvent must be a valid function"); - } - - if (this._drainEvent) { - throw new Error("a drain process is currently active."); - } - - this._incomingSinceLastDrain = this._totalIncomingMessages; - this._drainEvent = drainEvent; - this._lastReceived = Date.now(); - this._stats = {}; - - return this._buildOffsetMap(topic, drainEvent, this.asyncLimit).then(maps => { - - this._queueMap = maps.queueMap; - this._drainMap = maps.drainMap; - - this._startToReceiveMessages(); - - if (this.isPaused()) { - this.resume(); - } - }); - } - - _getEarliestProcessedOnQueues() { - - //error prevention - if (!this._queueMap) { - return this._lastReceived; - } - - let earliest = this._queueMap[Object.keys(this._queueMap)[0]].getLastProcessed(); - let ne = null; - Object.keys(this._queueMap).forEach(key => { - ne = this._queueMap[key].getLastProcessed(); - if (ne < earliest) { - earliest = ne; - } - }); - - return earliest; - } - - /** - * main req. function, pass it a function to receive messages - * under flow control, until they are stall for a certain amount - * of time (e.g. when all messages on the queue are consumed) - * returns a Promise - * @param topic - * @param drainEvent - * @param drainThreshold - * @param timeout - */ - drainOnce(topic = "t", drainEvent = null, drainThreshold = 10000, timeout = 0) { - return new Promise((resolve, reject) => { - - if (!drainEvent || typeof drainEvent !== "function") { - return reject("drainEvent must be a valid function"); - } - - if (this._drainEvent) { - return reject("a drain process is currently active."); - } - - if (timeout !== 0 && timeout < this.DRAIN_INTV) { - return reject(`timeout must be either 0 or > ${this.DRAIN_INTV}.`); - } - - if (timeout !== 0 && timeout <= drainThreshold) { - return reject(`timeout ${timeout} must be greater than the drainThreshold ${drainThreshold}.`); - } - - let t = null; - let intv = null; - - intv = setInterval(() => { - - const spanProcessed = Date.now() - this._getEarliestProcessedOnQueues(); - const spanReceived = Date.now() - this._lastReceived; - - this._getLogger().debug("drainOnce interval running, current span-rec: " + - `${spanReceived} / span-proc: ${spanProcessed} ms.`); - - //set stats - this._countStats("intv-cycle"); - this._stats["last-proc-since"] = spanProcessed; - this._stats["last-rec-since"] = spanReceived; - - //choose the smaller span - const span = spanProcessed < spanReceived ? spanProcessed : spanReceived; - - if (span >= drainThreshold) { - this._getLogger().info(`drainOnce span ${span} hit threshold ${drainThreshold}.`); - clearInterval(intv); - clearTimeout(t); - this.stopDrain(); - resolve(this._totalIncomingMessages - this._incomingSinceLastDrain); - } - }, this.DRAIN_INTV); - - if (timeout !== 0) { - this._getLogger().info(`drainOnce timeout active: ${timeout} ms.`); - t = setTimeout(() => { - this._getLogger().warn(`drainOnce timeout hit after ${timeout} ms.`); - clearInterval(intv); - this.stopDrain(); - reject("drainOnce ran into timeout."); - }, timeout); - } - - //start the drain process - this.drain(topic, drainEvent).then(() => { - this._getLogger().info("drain process of drainOnce has started."); - }).catch(e => { - reject(`failed to start drain process of drainOnce, because: ${e}.`); - }); - }); - } - - /** - * stops any active drain process - */ - stopDrain() { - - if (!this._drainEvent) { - throw new Error("there is no drain active."); - } - - this._drainTargetTopic = null; - - //reset - if (this._lastMessageHandlerRef) { - this.raw.removeListener("message", this._lastMessageHandlerRef); - this._lastMessageHandlerRef = null; - } else { - this._getLogger().warn("message handler ref not present during close, could not remove listener."); - } - - this._drainEvent = null; - this._receivedFirst = false; - - this._resetQueueMaps(); - - const duration = (Date.now() - this._drainStart) / 1000; - this._getLogger().info(`[Drainer] stopped drain process, had been open for ${duration} seconds.`); - } - - /** - * removes kafka topics (if broker allows this action) - * @param topics - */ - removeTopics(topics = []) { - return new Promise((resolve, reject) => { - this._getLogger().info(`deleting topics ${JSON.stringify(topics)}.`); - this.raw.client.removeTopicMetadata(topics, (err, data) => { - - if (err) { - return reject(err); - } - - resolve(data); - }); - }); - } - - pause() { - - if (!this.isPaused()) { - this._countStats("paused"); - } - - return this.consumer.pause(); - } - - resume() { - - if (this.isPaused()) { - this._countStats("resumed"); - } - - return this.consumer.resume(); - } - - isPaused() { - return this.consumer.isPaused(); - } - - _startToReceiveMessages() { - this._lastMessageHandlerRef = this._onMessage.bind(this); - this.raw.on("message", this._lastMessageHandlerRef); - this._getLogger().info("[Drainer] started drain process."); - this._drainStart = Date.now(); - } - - _onMessage(message) { - - this._getLogger().debug("received kafka message => length: " + message.value.length + ", offset: " + - message.offset + ", partition: " + message.partition + ", on topic: " + message.topic); - - if (this.autoJsonParsing) { - try { - message.value = JSON.parse(message.value); - } catch (e) { - this._countStats("msg-parse-fail"); - return this.emit("error", "failed to json parse message value: " + message); - } - - if (!message.value) { - this._countStats("msg-empty"); - return this.emit("error", "message value is empty: " + message); - } - } - - //we only want to drain messages that belong to our topic - if (message.topic === this._drainTargetTopic) { - - //identify queue for this topic - if (!this._queueMap) { - this._countStats("queue-map-missing"); - this._getLogger().warn("received message, but queue map is missing."); - } else if (!this._queueMap[message.partition]) { - this._countStats("queue-partition-missing"); - this._getLogger().warn("received message, but queue partition is missing for partition: " + message.partition); - } else { - //and push message into queue - this._queueMap[message.partition].push(message); - } - } else { - this._getLogger().warn(`receiving messages from other topic ${message.topic} only expecting to receive from ${this._drainTargetTopic} for this instance.`); - } - - if (!this.disablePauseResume) { - this.pause(); - } - - this._totalIncomingMessages++; - this._lastReceived = Date.now(); - - if (!this._receivedFirst) { - this._receivedFirst = true; - this._getLogger().info("consumer received first message."); - this.emit("first-drain-message", message); - } - } - - _countStats(key) { - - if (!this._stats) { - return; - } - - if (!this._stats[key]) { - this._stats[key] = 1; - return; - } - - this._stats[key]++; - } - - /** - * consumer proxy - * @param args - */ - on(...args) { - this.consumer.on(...args); - } - - /** - * consumer proxy - * @param args - */ - once(...args) { - this.consumer.once(...args); - } - - /** - * consumer proxy - * @param args - */ - removeListener(...args) { - this.consumer.removeListener(...args); - } - - /** - * consumer proxy - * @param args - */ - emit(...args) { - this.consumer.emit(...args); - } -} - -module.exports = PartitionDrainer; diff --git a/lib/kafka/PartitionQueue.js b/lib/kafka/PartitionQueue.js deleted file mode 100644 index 469a3df..0000000 --- a/lib/kafka/PartitionQueue.js +++ /dev/null @@ -1,167 +0,0 @@ -"use strict"; - -const async = require("async"); - -class PartitionQueue { - - constructor(partition, drainEvent, drainer, asyncLimit = 1, queueDrain = null) { - - if (typeof drainEvent !== "function") { - throw new Error("drainEvent must be a function."); - } - - if (typeof queueDrain !== "function") { - throw new Error("queueDrain must be a function."); - } - - this.partition = partition; - this._onQueueDrain = queueDrain; - this._drainEvent = drainEvent; - this._drainer = drainer; - this.asyncLimit = asyncLimit; - this._q = null; - - this._lastProcessed = Date.now(); - this._lastPushed = Date.now(); - - this._totalPushed = 0; - this._totalProcessedMessages = 0; - this._totalMessageProcessFails = 0; - this._lastOffset = -1; - - this._drainCheckIntv = null; - } - - _getLogger() { - return this._drainer._getLogger(); - } - - _emitDrain() { - if (this._onQueueDrain) { - process.nextTick(() => { //await potential writing of lastOffset - this._onQueueDrain(this.partition, this._lastOffset); - }); - } - } - - _runDrainCheckIntv(ms = 500, drainSpan = 2500) { - - if (this._drainCheckIntv) { - throw new Error("drain check interval already active for partition queue."); - } - - this._drainCheckIntv = setInterval(() => { - - if (!this._q) { - return; - } - - //queue size is greater than 0, will emit own drain event soon anyway - if (this._q.length() > 0) { - return; - } - - if (Date.now() - this._lastPushed > drainSpan) { - this._getLogger().debug(`partition ${this.partition} received no messages, flushing queue anyway.`); - this._emitDrain(); - } - }, ms); - } - - _closeDrainCheckIntv() { - if (this._drainCheckIntv) { - clearInterval(this._drainCheckIntv); - } - } - - push(message) { - if (this._q) { - this._totalPushed++; - this._lastPushed = Date.now(); - this._q.push(message); - } - } - - getLastProcessed() { - return this._lastProcessed; - } - - getStats() { - return { - partition: this.partition, - lastProcessed: this._lastProcessed, - totalProcessed: this._totalProcessedMessages, - totalProcessFails: this._totalMessageProcessFails, - queueSize: this._q ? this._q.length() : null, - workers: this.asyncLimit, - lastPushed: this._lastPushed, - totalPushed: this._totalPushed, - lastProcessedOffset: this._lastOffset - }; - } - - build() { - - if (this._q) { - throw new Error("this queue has already been build."); - } - - this._q = async.queue((msg, done) => { - if (this._drainEvent) { - setImmediate(() => this._drainEvent(msg, err => { - - try { - if (typeof msg.offset === "undefined") { - if (!err) { - err = new Error("missing offset on message: " + JSON.stringify(msg)); - } - this._getLogger().error("missing offset on message: " + JSON.stringify(msg)); - } else { - this._lastOffset = msg.offset; - } - } catch (e) { - if (!err) { - err = new Error("failed to parse message offset: " + e); - } - this._getLogger().error("failed to parse message offset: " + e); - } - - this._lastProcessed = Date.now(); - this._totalProcessedMessages++; - done(err); - })); - } else { - this._getLogger().debug("drainEvent not present, message is dropped."); - } - }, this.asyncLimit); - - this._q.drain = () => { - this._emitDrain(); - }; - - this._q.error(err => { - if (err) { - this._totalMessageProcessFails++; - this._getLogger().warn("error was passed back to consumer queue, dropping it silently: " + JSON.stringify(err)); - } - }); - - this._getLogger().info(`partition queue has been build for partition: ${this.partition}.`); - this._runDrainCheckIntv(); - return this; //chain? - } - - close() { - - this._closeDrainCheckIntv(); - - if (this._q) { - this._q.kill(); - this._q = null; - } - - this._getLogger().info("queue closed."); - } -} - -module.exports = PartitionQueue; diff --git a/lib/kafka/Publisher.js b/lib/kafka/Publisher.js deleted file mode 100644 index 95decea..0000000 --- a/lib/kafka/Publisher.js +++ /dev/null @@ -1,515 +0,0 @@ -"use strict"; - -const Kafka = require("./Kafka.js"); -const Promise = require("bluebird"); -const murmur = require("murmurhash").v3; -const uuid = require("uuid"); -const { KeyedMessage } = require("kafka-node"); - -const { CompressionTypes } = require("./../shared/index.js"); - -const MESSAGE_TYPES = { - PUBLISH: "-published", - UNPUBLISH: "-unpublished", - UPDATE: "-updated" -}; - -class Publisher { - - constructor(producer = null, partitionCount = 1, autoFlushBuffer = 0, flushPeriod = 100) { - - if (!producer || !(producer instanceof Kafka) || !producer.isProducer) { - throw new Error("producer is not a valid Sinek Kafka(Producer)"); - } - - this.producer = producer; - this.raw = producer.producer; - this.partitionCount = partitionCount; - - this._lastProcessed = Date.now(); - this._totalSentMessages = 0; - - this._paused = false; - - this._buffer = {}; - this._flushIntv = null; - - this.CompressionTypes = CompressionTypes; - - - this.autoFlushBuffer = autoFlushBuffer; - this.flushPeriod = flushPeriod; - - this._bufferDisabled = false; - this.disableBuffer(); - } - - /** - * default behaviour - */ - disableBuffer() { - this._getLogger().info("[Publisher] buffer disabled."); - this._stopAutoBufferFlushInterval(); - this._bufferDisabled = true; - } - - /** - * BETA - */ - enableBuffer() { - - this._getLogger().info("[Publisher] buffer enabled."); - - if (this.autoFlushBuffer > 0) { - this.setAutoFlushBuffer(this.autoFlushBuffer, this.flushPeriod); - } - } - - setAutoFlushBuffer(minBufferSize = 0, period = 100) { - - if (typeof minBufferSize !== "number" || minBufferSize < 0) { - throw new Error("minBufferSize must be a number and higher or equal to 0."); - } - - if (typeof period !== "number" || period < 5 || period > 60000) { - throw new Error("period must be a number and > 5 and < 60000."); - } - - this._getLogger().info(`[Publisher] Adjusting auto flush buffer size: ${minBufferSize} and period: ${period}.`); - this._runAutoBufferFlushInterval(minBufferSize, period); - } - - stopAutoFlushBuffer() { - this._stopAutoBufferFlushInterval(); - } - - _runAutoBufferFlushInterval(minSize, ms) { - this._flushIntv = setInterval(() => { - - Promise.all(Object - .keys(this._buffer) - .filter(k => this._buffer[k].length >= minSize) - .map(topic => this.flushBuffer(topic))) - .then(() => { - this._getLogger().debug("[Publisher] flushed buffer."); - }, e => { - this._getLogger().error(`[Publisher] failed to flush buffer: ${e}.`); - }); - }, ms); - } - - _stopAutoBufferFlushInterval() { - - if (this._flushIntv) { - this._getLogger().debug("[Publisher] stopping auto-buffer flush interval."); - clearInterval(this._flushIntv); - } - } - - _getLogger() { - return this.producer._getLogger(); - } - - /** - * closes the publisher (and the underlying producer/client) - */ - close() { - this._getLogger().info("[Publisher] closed."); - this._stopAutoBufferFlushInterval(); - return this.producer.close(); - } - - /** - * returns a few insights - * @returns {{totalPublished: (number|*), last: (number|*), isPaused: *}} - */ - getStats() { - return { - totalPublished: this._totalSentMessages, - last: this._lastProcessed, - isPaused: this.producer && this.producer.isProducer ? this.isPaused() : null - }; - } - - /** - * uses the partition count to identify - * a partition in range using a hashed representation - * of the key's string value - * @param key - * @param partitionCount - * @returns {Promise} - */ - getPartitionForKey(key, partitionCount = 0) { - - if (typeof key !== "string") { - return Promise.reject("key must be a valid string"); - } - - if (partitionCount === 0) { - partitionCount = this.partitionCount; - } - - return Promise.resolve(murmur(key) % partitionCount); - } - - getRandomPartition(partitionCount = 0) { - return this.getPartitionForKey(uuid.v4(), partitionCount); - } - - /** - * create topics (be aware that this requires - * certain settings in your broker to be active) - * @param topics - */ - createTopics(topics = ["t"]) { - return new Promise((resolve, reject) => { - this._getLogger().info(`[Publisher] creating topics ${JSON.stringify(topics)}.`); - this.raw.createTopics(topics, true, (err, data) => { - - if (err) { - return reject(err); - } - - resolve(data); - }); - }); - } - - /** - * returns a default message type object - * @returns {{topic: string, messages: Array, key: null, partition: number, attributes: number}} - */ - static getKafkaBaseMessage() { - return { - topic: "", - messages: [], - key: null, - partition: 0, - attributes: 0 - }; - } - - /** - * returns a kafka producer payload ready to be sent - * identifies partition of message by using identifier - * @param topic - * @param identifier - * @param object - * @param compressionType - * @param {string | null} partitionKey base string for partition determination - * @returns {*} - */ - getKeyedPayload(topic = "t", identifier = "", object = {}, compressionType = 0, partitionKey = null) { - - if (!this.CompressionTypes.isValid(compressionType)) { - return Promise.reject("compressionType is not valid checkout publisher.CompressionTypes."); - } - - partitionKey = typeof partitionKey === "string" ? partitionKey : identifier; - - return this.getPartitionForKey(partitionKey).then(partition => { - return { - topic, - partition, - messages: new KeyedMessage(identifier, JSON.stringify(object)), - attributes: compressionType - }; - }); - } - - /** - * easy access to compliant kafka topic api - * this will create a store a message payload describing a "CREATE" event - * @param topic - * @param identifier - * @param object - * @param version - * @param compressionType - * @param {string | null} partitionKey base string for partition determination - * @returns {*} - */ - bufferPublishMessage(topic, identifier, object, version = 1, compressionType = 0, partitionKey = null) { - - if (typeof identifier !== "string") { - return Promise.reject("expecting identifier to be of type string."); - } - - if (typeof object !== "object") { - return Promise.reject("expecting object to be of type object."); - } - - if (!object.id) { - object.id = identifier; - } - - if (typeof object.version === "undefined") { - object.version = version; - } - - return this.appendBuffer(topic, identifier, { - payload: object, - key: identifier, - id: uuid.v4(), - time: (new Date()).toISOString(), - type: topic + MESSAGE_TYPES.PUBLISH - }, compressionType, partitionKey); - } - - /** - * easy access to compliant kafka topic api - * this will create a store a message payload describing a "DELETE" event - * @param topic - * @param identifier - * @param object - * @param version - * @param compressionType - * @param partitionKey - * @returns {*} - */ - bufferUnpublishMessage(topic, identifier, object = {}, version = 1, compressionType = 0, partitionKey = null) { - - if (typeof identifier !== "string") { - return Promise.reject("expecting identifier to be of type string."); - } - - if (typeof object !== "object") { - return Promise.reject("expecting object to be of type object."); - } - - if (!object.id) { - object.id = identifier; - } - - if (typeof object.version === "undefined") { - object.version = version; - } - - return this.appendBuffer(topic, identifier, { - payload: object, - key: identifier, - id: uuid.v4(), - time: (new Date()).toISOString(), - type: topic + MESSAGE_TYPES.UNPUBLISH - }, compressionType, partitionKey); - } - - /** - * easy access to compliant kafka topic api - * this will create a store a message payload describing an "UPDATE" event - * @param topic - * @param identifier - * @param object - * @param version - * @param compressionType - * @param partitionKey - * @returns {*} - */ - bufferUpdateMessage(topic, identifier, object, version = 1, compressionType = 0, partitionKey = null) { - - if (typeof identifier !== "string") { - return Promise.reject("expecting identifier to be of type string."); - } - - if (typeof object !== "object") { - return Promise.reject("expecting object to be of type object."); - } - - if (!object.id) { - object.id = identifier; - } - - if (typeof object.version === "undefined") { - object.version = version; - } - - return this.appendBuffer(topic, identifier, { - payload: object, - key: identifier, - id: uuid.v4(), - time: (new Date()).toISOString(), - type: topic + MESSAGE_TYPES.UPDATE - }, compressionType, partitionKey); - } - - /** - * build a buffer per topic for message payloads - * if autoBufferFlush is > 0 flushBuffer might be called - * @param topic - * @param identifier - * @param object - * @param compressionType - * @param {string | null} partitionKey base string for partition determination - * @returns {Promise.} - */ - appendBuffer(topic, identifier, object, compressionType = 0, partitionKey = null) { - - return this.getKeyedPayload(topic, identifier, object, compressionType, partitionKey).then(payload => { - - //if buffer is disbaled, this message will be send instantly - if (this._bufferDisabled) { - return this.batch([payload]); - } - - if (!this._buffer[topic]) { - this._buffer[topic] = []; - } - - this._buffer[topic].push(payload); - }); - } - - /** - * send all message payloads in buffer for a topic - * in a single batch request - * @param topic - * @param skipBlock - * @returns {*} - */ - flushBuffer(topic) { - - if (!this._buffer[topic]) { - return Promise.reject(`topic ${topic} has no buffer, you should call appendBuffer() first.`); - } - - const batch = this._buffer[topic]; - this._buffer[topic] = []; - - return this.batch(batch); - } - - /** - * appends and sends the message payloads in the buffer - * (you can also use this so send a single message immediately) - * @param topic - * @param identifier - * @param object - * @param compressionType - * @returns {Promise.} - */ - appendAndFlushBuffer(topic, identifier, object, compressionType = 0) { - return this.appendBuffer(topic, identifier, object, compressionType).then(() => { - return this.flushBuffer(topic); - }); - } - - /** - * most versatile function to produce a message on a topic(s) - * you can send multiple messages at once (but keep them to the same topic!) - * if you need full flexibility on payload (message definition) basis - * you should use .batch([]) - * @param topic - * @param messages - * @param partitionKey - * @param partition - * @param compressionType - * @returns {*} - */ - send(topic = "t", messages = [], partitionKey = null, partition = null, compressionType = 0) { - - if (!this.CompressionTypes.isValid(compressionType)) { - return Promise.reject("compressionType is not valid checkout publisher.CompressionTypes."); - } - - const payload = { - topic, - messages, - attributes: compressionType - }; - - if (partitionKey !== null) { - payload.key = partitionKey; - } - - if (partition !== null) { - payload.partition = partition; - } - - return this.batch([payload]); - } - - /** - * leaves full flexibility when sending different message definitions (e.g. mulitple topics) - * at once use with care: https://www.npmjs.com/package/kafka-node#sendpayloads-cb - * @param payloads - * @returns {Promise.<{}>} - */ - batch(payloads) { - - if (this._paused) { - return Promise.resolve({}); - } - - return new Promise((resolve, reject) => { - this.raw.send(payloads, (err, data) => { - - if (err) { - return reject(err); - } - - //update stats - this._lastProcessed = Date.now(); - payloads.forEach(p => { - if (p && p.messages) { - if (Array.isArray(p.messages)) { - this._totalSentMessages += p.messages.length; - } else { - this._totalSentMessages++; - } - } - }); - - resolve(data); - }); - }); - } - - pause() { - this._paused = true; - } - - resume() { - this._paused = false; - } - - isPaused() { - return this._paused; - } - - refreshMetadata(topics = []) { - return this.producer.refreshMetadata(topics); - } - - /** - * producer proxy - * @param args - */ - on(...args) { - this.producer.on(...args); - } - - /** - * producer proxy - * @param args - */ - once(...args) { - this.producer.once(...args); - } - - /** - * producer proxy - * @param args - */ - removeListener(...args) { - this.producer.removeListener(...args); - } - - /** - * producer proxy - * @param args - */ - emit(...args) { - this.producer.emit(...args); - } -} - -module.exports = Publisher; diff --git a/lib/librdkafka/NConsumer.js b/lib/librdkafka/NConsumer.js deleted file mode 100644 index 58bebf4..0000000 --- a/lib/librdkafka/NConsumer.js +++ /dev/null @@ -1,1266 +0,0 @@ -"use strict"; - -const Promise = require("bluebird"); -const EventEmitter = require("events"); -const debug = require("debug"); -const async = require("async"); - -const { ConsumerAnalytics } = require("./../shared/Analytics.js"); -const { ConsumerHealth } = require("./../shared/Health.js"); -const Metadata = require("./../shared/Metadata.js"); - -//@OPTIONAL -let BlizzKafka = null; - -const SINGLE_CONSUME_GRACE_TIME_MS = 100; -const GRACE_TIME_FACTOR = 1.5; -const MAX_EMPTY_FETCH_COUNT = 26; // 3,9 seconds -const FETCH_ERROR_GRACE_TIME_MS = 1500; -const ASYNC_COMMIT_REQ_TIME_MS = 250; -const MESSAGE_CHARSET = "utf8"; - -const DEFAULT_LOGGER = { - debug: debug("sinek:nconsumer:debug"), - info: debug("sinek:nconsumer:info"), - warn: debug("sinek:nconsumer:warn"), - error: debug("sinek:nconsumer:error") -}; - -/** - * native consumer wrapper for node-librdkafka - * @extends EventEmitter - */ -class NConsumer extends EventEmitter { - - /** - * creates a new consumer instance - * @param {string|Array} topics - topic or topics to subscribe to - * @param {object} config - configuration object - */ - constructor(topics, config = { options: {}, health: {} }) { - super(); - - if (!config) { - throw new Error("You are missing a config object."); - } - - if (!config.logger || typeof config.logger !== "object") { - config.logger = DEFAULT_LOGGER; - } - - try { - if (!BlizzKafka) { - BlizzKafka = require("node-rdkafka"); - } - } catch (error) { - config.logger.error(error); - throw new Error("You have to install node-rdkafka to use NConsumer. " + error.message); - } - - if (!config.options) { - config.options = {}; - } - - this.topics = Array.isArray(topics) ? topics : [topics]; - this.config = config; - this._health = new ConsumerHealth(this, this.config.health || {}); - - this.consumer = null; - this._resume = true; - this._inClosing = false; - this._firstMessageConsumed = false; - this._totalIncomingMessages = 0; - this._lastReceived = null; - this._totalProcessedMessages = 0; - this._lastProcessed = null; - this._stream = null; - this._asStream = null; - this._isAutoCommitting = null; - this._batchCount = 0; - this._batchCommitts = 0; - this._totalBatches = 0; - this._batchConfig = null; - this._analyticsIntv = null; - this._lagCheckIntv = null; - this._lagCache = null; - this._analyticsOptions = null; - this._analytics = null; - this._lastLagStatus = null; - this._consumedSinceCommit = 0; - this._emptyFetches = 0; - this._avgBatchProcessingTime = 0; - this._extCommitCallback = null; - - this._errors = 0; - super.on("error", () => this._errors++); - } - - /** - * @throws - * starts analytics tasks - * @param {object} options - analytic options - * @returns {Promise} resolves after as soon as analytics are available - */ - enableAnalytics(options = {}) { - - if (this._analyticsIntv || this._lagCheckIntv) { - throw new Error("analytics intervals are already running."); - } - - let { - analyticsInterval, - lagFetchInterval - } = options; - this._analyticsOptions = options; - - analyticsInterval = analyticsInterval || 1000 * 150; // 150 sec - lagFetchInterval = lagFetchInterval || 1000 * 60 * 5; // 5 minutes - - this._analyticsIntv = setInterval(this._runAnalytics.bind(this), analyticsInterval); - this._lagCheckIntv = setInterval(this._runLagCheck.bind(this), lagFetchInterval); - - // Make analytics available immediately - return this._runAnalytics() - .then(() => this._runLagCheck()); - } - - /** - * halts all analytics tasks - */ - haltAnalytics() { - - if (this._analyticsIntv) { - clearInterval(this._analyticsIntv); - } - - if (this._lagCheckIntv) { - clearInterval(this._lagCheckIntv); - } - } - - /** - * connect to broker - * @param {boolean} asStream - optional, if client should be started in streaming mode - * @param {object} opts - optional, options asString, asJSON (booleans) - * @returns {Promise.<*>} - */ - connect(asStream = false, opts = {}) { - - let { zkConStr, kafkaHost, logger, groupId, options, noptions, tconf } = this.config; - const { autoCommit } = options; - const { asString = false, asJSON = false } = opts; - - let conStr = null; - - if (typeof kafkaHost === "string") { - conStr = kafkaHost; - } - - if (typeof zkConStr === "string") { - conStr = zkConStr; - } - - if (conStr === null && !noptions) { - return Promise.reject(new Error("One of the following: zkConStr or kafkaHost must be defined.")); - } - - if (conStr === zkConStr) { - return Promise.reject(new Error("NProducer does not support zookeeper connection.")); - } - - const config = { - "metadata.broker.list": conStr, - "group.id": typeof groupId === "string" ? groupId : "", - "enable.auto.commit": false, // default in librdkafka is true - what makes this dangerous for our batching logic(s) - }; - - const overwriteConfig = { - "offset_commit_cb": this._onOffsetCommit.bind(this) - }; - - if (noptions && noptions["offset_commit_cb"]) { - if (typeof noptions["offset_commit_cb"] !== "function") { - return Promise.reject(new Error("offset_commit_cb must be a function.")); - } - this._extCommitCallback = noptions["offset_commit_cb"]; - } - - noptions = noptions || {}; - noptions = Object.assign({}, config, noptions, overwriteConfig); - logger.debug(noptions); - this._isAutoCommitting = noptions["enable.auto.commit"]; - - tconf = tconf || undefined; - logger.debug(tconf); - - this._asStream = asStream; - - if (asStream) { - return this._connectAsStream(logger, noptions, tconf, { asString, asJSON }); - } - - return this._connectInFlow(logger, noptions, tconf); - } - - /** - * @private - * event handler for async offset committs - * @param {Error} error - * @param {Array} partitions - */ - _onOffsetCommit(error, partitions) { - - if (this._extCommitCallback) { - try { - this._extCommitCallback(error, partitions); - } catch (error) { - super.emit("error", error); - } - } - - if (error) { - return this.config.logger.warn("commit request failed with an error: " + JSON.stringify(error)); - } - - this.config.logger.debug(partitions); - } - - /** - * @private - * connects in flow mode mode - * @param {object} logger - * @param {object} noptions - * @param {object} tconf - * @returns {Promise.<*>} - */ - _connectInFlow(logger, noptions, tconf = {}) { - return new Promise((resolve, reject) => { - - this.consumer = new BlizzKafka.KafkaConsumer(noptions, tconf); - - this.consumer.on("event.log", log => { - logger.debug(log.message); - }); - - this.consumer.on("event.error", error => { - super.emit("error", error); - }); - - this.consumer.on("error", error => { - super.emit("error", error); - }); - - this.consumer.on("disconnected", () => { - if (this._inClosing) { - this._reset(); - } - logger.warn("Disconnected."); - //auto-reconnect --> handled by consumer.consume(); - }); - - this.consumer.on("ready", () => { - logger.info(`Native consumer (flow) ready v. ${BlizzKafka.librdkafkaVersion}, e. ${BlizzKafka.features.join(", ")}.`); - super.emit("ready"); - }); - - logger.debug("Connecting.."); - this.consumer.connect(null, (error, metadata) => { - - if (error) { - super.emit("error", error); - return reject(error); - } - - logger.debug(metadata); - resolve(); - }); - }); - } - - /** - * @private - * connects in streaming mode - * @param {object} logger - * @param {object} noptions - * @param {object} tconf - * @param {object} opts - * @returns {Promise.<*>} - */ - _connectAsStream(logger, noptions, tconf = {}, opts = {}) { - return new Promise(resolve => { - - const { asString = false, asJSON = false } = opts; - - const topics = this.topics; - if (topics && topics.length) { - this.config.logger.info(`Subscribing to topics: ${topics.join(", ")}.`); - } else { - this.config.logger.info("Not subscribing to any topics initially."); - } - - const stream = BlizzKafka.KafkaConsumer.createReadStream(noptions, tconf, { - topics, - waitInterval: 1, - objectMode: true - }); - - this._stream = stream; - this.consumer = stream.consumer; - - stream.on("error", error => { - - //bugfix-hack - if (this.consumer && this.consumer._isDisconnecting) { - return; - } - - super.emit("error", error); - }); - - stream.on("data", message => { - - this.config.logger.debug(message); - - this._totalIncomingMessages++; - this._lastReceived = Date.now(); - message.value = this._convertMessageValue(message.value, asString, asJSON); - - this._convertHeadersValue(message.headers); - - if (!this._firstMessageConsumed) { - this._firstMessageConsumed = true; - super.emit("first-drain-message", message); - } - - super.emit("message", message); - }); - - this.consumer.on("event.log", log => { - logger.debug(log.message); - }); - - this.consumer.on("event.error", error => { - - //bugfix-hack - if (this.consumer && this.consumer._isDisconnecting) { - return; - } - - super.emit("error", error); - }); - - this.consumer.on("disconnected", () => { - if (this._inClosing) { - this._reset(); - } - logger.warn("Disconnected."); - //auto-reconnect --> handled by stream - }); - - this.consumer.on("ready", () => { - logger.info(`Native consumer (stream) ready v. ${BlizzKafka.librdkafkaVersion}, e. ${BlizzKafka.features.join(", ")}.`); - super.emit("ready"); - resolve(); - }); - - logger.debug("Connecting.."); - }); - } - - /** - * @private - * runs (and calls itself) until it has successfully - * read a certain size of messages from the broker - * @param {number} batchSize - * @returns {boolean} - */ - _singleConsumeRecursive(batchSize = 1) { - - if (!this._resume || !this.consumer || !this.consumer.consume) { - return false; - } - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Consuming recursively.. ${batchSize}`); - } - - this.consumer.consume(batchSize, (error, messages) => { - - if (error || !messages.length) { - - //always ensure longer wait on consume error - if (error) { - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Consumed recursively with error ${error.message}`); - } - - super.emit("error", error); - return setTimeout(this._singleConsumeRecursive.bind(this), FETCH_ERROR_GRACE_TIME_MS); - } - - //retry asap - this._emptyFetches++; - let graceTime = (this.config.options.consumeGraceMs || SINGLE_CONSUME_GRACE_TIME_MS) * GRACE_TIME_FACTOR; - graceTime = graceTime * (this._emptyFetches > MAX_EMPTY_FETCH_COUNT ? MAX_EMPTY_FETCH_COUNT : this._emptyFetches); - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Consumed recursively for empty batch, backing off ${graceTime} ms after ${this._emptyFetches} attempts.`); - } - - setTimeout(this._singleConsumeRecursive.bind(this), graceTime); - - } else { - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Consumed recursively with success ${messages.length}`); - } - - this._emptyFetches = 0; //reset - super.emit("batch", messages); - } - return true; - }); - } - - /** - * @private - * converts message value according to booleans - * @param {Buffer} _value - * @param {boolean} asString - * @param {boolean} asJSON - * @returns {Buffer|string|object} - */ - _convertMessageValue(_value, asString = true, asJSON = false) { - if (!_value) { - return _value; - } - - let value = _value; - - if (!asString && !asJSON) { - return value; - } - - if (asString || asJSON) { - value = value.toString(MESSAGE_CHARSET); - } - - if (asJSON) { - try { - value = JSON.parse(value); - } catch (error) { - this.config.logger.warn(`Failed to parse message value as json: ${error.message}, ${value}`); - } - } - - return value; - } - - /** - * @private - * converts the header values to string - * @param {Array} headers - * @returns {void} - */ - _convertHeadersValue(headers) { - if (!headers || headers.length === 0) { - return; - } - - headers.forEach(item => { - Object.keys(item).forEach(key => { - item[key] = item[key] && item[key].toString(MESSAGE_CHARSET); - }); - }); - } - - /** - * subscribe and start to consume, should be called only once after connection is successfull - * options object supports the following fields: - * batchSize amount of messages that is max. fetched per round - * commitEveryNBatch amount of messages that should be processed before committing - * concurrency the concurrency of the execution per batch - * commitSync if the commit action should be blocking or non-blocking - * noBatchCommits defaults to false, if set to true, no commits will be made for batches - * - * @param {function} syncEvent - callback (receives messages and callback as params) - * @param {string} asString - optional, if message value should be decoded to utf8 - * @param {boolean} asJSON - optional, if message value should be json deserialised - * @param {object} options - optional object containing options for 1:n mode: - * @returns {Promise.<*>} - */ - consume(syncEvent = null, asString = true, asJSON = false, options = {}) { - - let { - batchSize, - commitEveryNBatch, - concurrency, - commitSync, - noBatchCommits, - manualBatching, - sortedManualBatch, - } = options; - - batchSize = batchSize || 1; - commitEveryNBatch = commitEveryNBatch || 1; - concurrency = concurrency || 1; - commitSync = typeof commitSync === "undefined" ? true : commitSync; //default is true - noBatchCommits = typeof noBatchCommits === "undefined" ? false : noBatchCommits; //default is false - manualBatching = typeof manualBatching === "undefined" ? false : manualBatching; //default is false - sortedManualBatch = typeof sortedManualBatch === "undefined" ? false : sortedManualBatch; //default is false - - if (!this.consumer) { - return Promise.reject(new Error("You must call and await .connect() before trying to consume messages.")); - } - - if (syncEvent && this._asStream) { - return Promise.reject(new Error("Usage of syncEvent is not permitted in streaming mode.")); - } - - if (this._asStream) { - return Promise.reject(new Error("Calling .consume() is not required in streaming mode.")); - } - - if (sortedManualBatch && !manualBatching) { - return Promise.reject(new Error("manualBatching batch option must be enabled, if you enable sortedManualBatch batch option.")); - } - - return new Promise((resolve, reject) => { - - //if a sync event is present, we only consume a single message - //await its callback and commit, if its not present, we just consume - //asap, convert and emit the message event - if (!syncEvent) { - - this.consumer.on("data", message => { - - this.config.logger.debug(message); - - this._totalIncomingMessages++; - this._lastReceived = Date.now(); - message.value = this._convertMessageValue(message.value, asString, asJSON); - - this._convertHeadersValue(message.headers); - - if (!this._firstMessageConsumed) { - this._firstMessageConsumed = true; - super.emit("first-drain-message", message); - resolve(); //resolves on first message - } - - super.emit("message", message); - }); - - } else { - - if (this._isAutoCommitting !== null && typeof this._isAutoCommitting !== "undefined") { - this.config.logger.warn("enable.auto.commit has no effect in 1:n consume-mode, set to null or undefined to remove this message." + - "You can pass 'noBatchCommits' as true via options to .consume(), if you want to commit manually."); - } - - if (this._isAutoCommitting) { - return reject(new Error("Please disable enable.auto.commit when using 1:n consume-mode.")); - } - - this.config.logger.info("running in", `1:${batchSize}`, "mode"); - this._batchConfig = options; //store for stats - - if (manualBatching) { - - this.config.logger.info("Batching manually.."); - - super.on("batch", (messages) => { - - const startBPT = Date.now(); - this._totalIncomingMessages += messages.length; - this._lastReceived = Date.now(); - - const mappedMessages = messages.map((message) => { - - this.config.logger.debug(message); - message.value = this._convertMessageValue(message.value, asString, asJSON); - - this._convertHeadersValue(message.headers); - - if (!this._firstMessageConsumed) { - this._firstMessageConsumed = true; - super.emit("first-drain-message", message); - resolve(); //resolves on first message - } - - super.emit("message", message); - return message; - }); - - const sortedBatch = {}; - if (sortedManualBatch) { - mappedMessages.forEach((mappedMessage) => { - - if (typeof sortedBatch[mappedMessage.topic] === "undefined") { - sortedBatch[mappedMessage.topic] = {}; - } - - if (typeof sortedBatch[mappedMessage.topic][mappedMessage.partition] === "undefined") { - sortedBatch[mappedMessage.topic][mappedMessage.partition] = []; - } - - sortedBatch[mappedMessage.topic][mappedMessage.partition].push(mappedMessage); - }); - } - - //execute sync event and wrap callback (in this mode the sync event recieves all messages as batch) - syncEvent(!sortedManualBatch ? mappedMessages : sortedBatch, (__error) => { - - /* ### sync event callback does not handle errors ### */ - if (__error && this.config && this.config.logger && this.config.logger.warn) { - this.config.logger.warn("Please dont pass errors to sinek consume callback", __error); - } - - this._totalProcessedMessages += mappedMessages.length; - this._lastProcessed = Date.now(); - - //when all messages from the batch are processed - - this._avgBatchProcessingTime = (this._avgBatchProcessingTime + (Date.now() - startBPT)) / 2; - this._consumedSinceCommit += mappedMessages.length; - this._totalBatches++; - this._batchCount++; - - //check whether we have to commit first - //we do not commit after a certain amount of batches - //we commit whenever the exepected amount of batches in messages is rconsoleeached - //we also do not commit, if noBatchCommits is true - if (noBatchCommits === true || this._consumedSinceCommit < (batchSize * commitEveryNBatch)) { - return this._singleConsumeRecursive(batchSize); - } - - this.config.logger.debug("committing after", this._batchCount, "batches, messages: " + this._consumedSinceCommit); - super.emit("commit", this._consumedSinceCommit); - this._batchCount = 0; - this._batchCommitts++; - this._consumedSinceCommit = 0; - - //commit last state (of all offsets) - if (commitSync) { - try { - this.consumer.commitSync(); - } catch (error) { - super.emit("error", error); - } - this._singleConsumeRecursive(batchSize); //recall instant - } else { - this.consumer.commit(); - setTimeout(() => { - this._singleConsumeRecursive(batchSize); //recall with some grace time - }, ASYNC_COMMIT_REQ_TIME_MS); - } - }); - }); - - } else { // TODO: refactor this monstrousity - - this.config.logger.info("Batching automatically.."); - - //we do not listen to "data" here - //we have to grab the whole batch that is delivered via consume(count) - super.on("batch", messages => { - - const startBPT = Date.now(); - this._totalIncomingMessages += messages.length; - this._lastReceived = Date.now(); - - async.eachLimit(messages, concurrency, (message, _callback) => { - - this.config.logger.debug(message); - - message.value = this._convertMessageValue(message.value, asString, asJSON); - - this._convertHeadersValue(message.headers); - - if (!this._firstMessageConsumed) { - this._firstMessageConsumed = true; - super.emit("first-drain-message", message); - resolve(); //resolves on first message - } - - super.emit("message", message); - - //execute sync event and wrap callback - syncEvent(message, (__error) => { - - /* ### sync event callback does not handle errors ### */ - if (__error && this.config && this.config.logger && this.config.logger.warn) { - this.config.logger.warn("Please dont pass errors to sinek consume callback", __error); - } - - this._totalProcessedMessages++; - this._lastProcessed = Date.now(); - _callback(); //return async cb - }); - - }, () => { - //when all messages from the batch are processed - - this._avgBatchProcessingTime = (this._avgBatchProcessingTime + (Date.now() - startBPT)) / 2; - this._consumedSinceCommit += messages.length; - this._totalBatches++; - this._batchCount++; - - //check whether we have to commit first - //we do not commit after a certain amount of batches - //we commit whenever the exepected amount of batches in messages is rconsoleeached - //we also do not commit, if noBatchCommits is true - if (noBatchCommits === true || this._consumedSinceCommit < (batchSize * commitEveryNBatch)) { - return this._singleConsumeRecursive(batchSize); - } - - this.config.logger.debug("committing after", this._batchCount, "batches, messages: " + this._consumedSinceCommit); - super.emit("commit", this._consumedSinceCommit); - this._batchCount = 0; - this._batchCommitts++; - this._consumedSinceCommit = 0; - - //commit last state (of all offsets) - if (commitSync) { - try { - this.consumer.commitSync(); - } catch (error) { - super.emit("error", error); - } - this._singleConsumeRecursive(batchSize); //recall instant - } else { - this.consumer.commit(); - setTimeout(() => { - this._singleConsumeRecursive(batchSize); //recall with some grace time - }, ASYNC_COMMIT_REQ_TIME_MS); - } - }); //EOF everyLimit - }); //EOF super.on("batch") - } //EOF else !manualBatching - } //EOF else !syncEvent - - const topics = this.topics; - if (topics && topics.length) { - this.config.logger.info(`Subscribing to topics: ${topics.join(", ")}.`); - this.consumer.subscribe(topics); - } else { - this.config.logger.info("Not subscribing to any topics initially."); - } - - if (!syncEvent) { - //if sync event is not present, consume as fast as possible - this.consumer.consume(); - } else { - //if sync event is present, we have to keep single messages coming - this._singleConsumeRecursive(batchSize); - } - }); - } - - /** - * Subscribe to additional topics - * this function is synchronous - * (yet the action is a long running native function) - * @param {Array} topics - topics array - * @returns {Array} - */ - addSubscriptions(topics = []) { - - const subscribedTo = this.consumer.subscription(); - if (subscribedTo && !!subscribedTo.length) { - return this.adjustSubscription(topics.concat(subscribedTo)); - } - - return this.adjustSubscription(topics); - } - - /** - * Change subscription and re-subscribe. - * this function is synchronous - * (yet the action is a long running native function) - * @param {string|Array} topics - topics string or array - * @returns {Array} - */ - adjustSubscription(topics = []) { - - if (!Array.isArray(topics)) { - topics = [topics]; - } - - if (!topics.length) { - this.config.logger.info("Unsubscribing from all topics."); - this.consumer.unsubscribe(); - this.topics = []; - return []; - } - - const subscribedTo = this.consumer.subscription(); - if (subscribedTo && !!subscribedTo.length) { - this.config.logger.info("Unsubscribing current topics."); - this.consumer.unsubscribe(); - } - - this.config.logger.info("Subscribing to new topics."); - this.consumer.subscribe(topics); - this.topics = topics; //update member field - return this.topics; - } - - /** - * commit all stored offsets - * @param {boolean} async - optional, if commit should be async (default is false) - * @returns {boolean} - */ - commit(async = false) { - - if (!this.consumer) { - return false; - } - - if (async) { - this.consumer.commit(); - return true; - } - - try { - this.consumer.commitSync(); - return true; - } catch (error) { - super.emit("error", error); - return false; - } - } - - /** - * commit given message - * @param {Boolean} async - optional, if commit should be async (default is false) - * @param {{topic, partition, offset}} message - * @returns {boolean} - */ - commitMessage(async = false, message) { - - if (!this.consumer) { - return false; - } - - if (async) { - this.consumer.commitMessage(message); - return true; - } - - try { - this.consumer.commitMessageSync(message); - return true; - } catch (error) { - super.emit("error", error); - return false; - } - } - - /** - * fakes a message with an offset - 1 for the given - * topic and partition and makes an attempt to commit it - * @param {string} topic - * @param {number} partition - * @param {number} offset - * @param {boolean} async - */ - commitOffsetHard(topic, partition, offset, async = false) { - - const fakeMessage = { - topic, - partition, - offset: offset - 1 // node-rd-kafka commitMessage => offset + 1 - }; - - return this.commitMessage(async, fakeMessage); - } - - /** - * commits all local stored offsets for assigned partitions of a topic - * @param {string} topic - */ - async commitLocalOffsetsForTopic(topic) { - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Committing local offsets for topic ${topic}`); - } - - const assignedPartitionsOfTopic = - this.getAssignedPartitions() - .filter((topicPartition) => topicPartition.topic === topic); - - const currentLocalOffsets = this.consumer.position(assignedPartitionsOfTopic) - .filter((topicPartition) => typeof topicPartition.offset !== "undefined"); - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Committing local offsets for topic ${topic} as`, currentLocalOffsets); - } - - return this.consumer.commit(currentLocalOffsets); - } - - /** - * pause the consumer for specific topics (partitions) - * @param {Array.<{}>} topicPartitions - * @throws {LibrdKafkaError} - */ - pause(topicPartitions = []) { - if (this.consumer) { - return this.consumer.pause(topicPartitions); - } - } - - /** - * resume the consumer for specific topic (partitions) - * @param {Array.<{}>} topicPartitions - * @throws {LibrdKafkaError} - */ - resume(topicPartitions = []) { - if (this.consumer) { - return this.consumer.resume(topicPartitions); - } - } - - /** - * returns consumer statistics - * @returns {object} - */ - getStats() { - return { - totalIncoming: this._totalIncomingMessages, - lastMessage: this._lastReceived, - receivedFirstMsg: this._firstMessageConsumed, - totalProcessed: this._totalProcessedMessages, - lastProcessed: this._lastProcessed, - queueSize: null, - isPaused: false, - drainStats: null, - omittingQueue: true, - autoComitting: this._isAutoCommitting, - consumedSinceCommit: this._consumedSinceCommit, - batch: { - current: this._batchCount, - committs: this._batchCommitts, - total: this._totalBatches, - config: this._batchConfig, - currentEmptyFetches: this._emptyFetches, - avgProcessingTime: this._avgBatchProcessingTime - }, - lag: this._lagCache, //read from private cache - totalErrors: this._errors - }; - } - - /** - * @private - * resets internal values - */ - _reset() { - this._firstMessageConsumed = false; - this._resume = true; - this._inClosing = false; - this._totalIncomingMessages = 0; - this._lastReceived = null; - this._totalProcessedMessages = 0; - this._lastProcessed = null; - this._stream = null; - this._asStream = null; - this._batchCount = 0; - this._batchCommitts = 0; - this._totalBatches = 0; - this._batchConfig = null; - this._lagCache = null; - this._analytics = null; - this._lastLagStatus = null; - this._consumedSinceCommit = 0; - this._emptyFetches = 0; - this._avgBatchProcessingTime = 0; - this._errors = 0; - this._extCommitCallback = null; - } - - /** - * closes connection if open - * @param {boolean} commit - if last offsets should be commited before closing connection - */ - close(commit = false) { - - this.haltAnalytics(); - - if (this.consumer) { - this._inClosing = true; - this._resume = false; //abort any running recursive consumption - if (!commit) { - this.consumer.disconnect(); - //this.consumer = null; - } else { - this.consumer.commit(); - this.config.logger.info("Committing on close."); - process.nextTick(() => { - this.consumer.disconnect(); - //this.consumer = null; - }); - } - } - } - - /** - * gets the lowest and highest offset that is available - * for a given kafka topic - * @param {string} topic - name of the kafka topic - * @param {number} partition - optional, default is 0 - * @param {number} timeout - optional, default is 2500 - * @returns {Promise.} - */ - getOffsetForTopicPartition(topic, partition = 0, timeout = 2500) { - - if (!this.consumer) { - return Promise.reject(new Error("Consumer not yet connected.")); - } - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Fetching offsets for topic partition ${topic} ${partition}.`); - } - - return new Promise((resolve, reject) => { - this.consumer.queryWatermarkOffsets(topic, partition, timeout, (error, offsets) => { - - if (error) { - return reject(error); - } - - resolve(offsets); - }); - }); - } - - /** - * gets all comitted offsets - * @param {number} timeout - optional, default is 2500 - * @returns {Promise.} - */ - getComittedOffsets(timeout = 2500) { - - if (!this.consumer) { - return Promise.resolve([]); - } - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Fetching committed offsets ${timeout}`); - } - - return new Promise((resolve, reject) => { - this.consumer.committed(timeout, (error, partitions) => { - - if (error) { - return reject(error); - } - - resolve(partitions); - }); - }); - } - - /** - * gets all topic-partitions which are assigned to this consumer - * @returns {Array} - */ - getAssignedPartitions() { - try { - return this.consumer.assignments(); - } catch (error) { - super.emit("error", error); - return []; - } - } - - /** - * @static - * return the offset that has been comitted for a given topic and partition - * @param {string} topic - topic name - * @param {number} partition - partition - * @param {Array} offsets - commit offsets from getComittedOffsets() - */ - static findPartitionOffset(topic, partition, offsets) { - - for (let i = 0; i < offsets.length; i++) { - if (offsets[i].topic === topic && offsets[i].partition === partition) { - return offsets[i].offset; - } - } - - throw new Error(`no offset found for ${topic}:${partition} in comitted offsets.`); - } - - /** - * compares the local commit offset status with the remote broker - * status for the topic partitions, for all assigned partitions of - * the consumer - * @param {boolean} noCache - when analytics are enabled the results can be taken from cache - * @returns {Promise.} - */ - async getLagStatus(noCache = false) { - - if (!this.consumer) { - return []; - } - - //if allowed serve from cache - if (!noCache && this._lagCache && this._lagCache.status) { - return this._lagCache.status; - } - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Getting lag status ${noCache}`); - } - - const startT = Date.now(); - const assigned = this.getAssignedPartitions(); - const comitted = await this.getComittedOffsets(); - - const status = await Promise.all(assigned.map(async topicPartition => { - try { - const brokerState = await this.getOffsetForTopicPartition(topicPartition.topic, topicPartition.partition); - const comittedOffset = NConsumer.findPartitionOffset(topicPartition.topic, topicPartition.partition, comitted); - return { - topic: topicPartition.topic, - partition: topicPartition.partition, - lowDistance: comittedOffset - brokerState.lowOffset, - highDistance: brokerState.highOffset - comittedOffset, - detail: { - lowOffset: brokerState.lowOffset, - highOffset: brokerState.highOffset, - comittedOffset - } - }; - } catch (error) { - return { - topic: topicPartition.topic, - partition: topicPartition.partition, - error - }; - } - })); - - const duration = Date.now() - startT; - this.config.logger.info(`fetching and comparing lag status took: ${duration} ms.`); - - //store cached version - if (status && Array.isArray(status)) { - - //keep last version - if (this._lagCache && this._lagCache.status) { - this._lastLagStatus = Object.assign({}, this._lagCache); - } - - //cache new version - this._lagCache = { - status, - at: startT, - took: Date.now() - startT - }; - } - - return status; - } - - /** - * called in interval - * @private - */ - _runAnalytics() { - - if (!this._analytics) { - this._analytics = new ConsumerAnalytics(this, this._analyticsOptions || {}, this.config.logger); - } - - return this._analytics.run() - .then(res => super.emit("analytics", res)) - .catch(error => super.emit("error", error)); - } - - /** - * returns the last computed analytics results - * @throws - * @returns {object} - */ - getAnalytics() { - - if (!this._analytics) { - super.emit("error", new Error("You have not enabled analytics on this consumer instance.")); - return {}; - } - - return this._analytics.getLastResult(); - } - - /** - * called in interval - * @private - */ - _runLagCheck() { - return this.getLagStatus(true).catch(() => { }); - } - - /** - * runs a health check and returns object with status and message - * @returns {Promise.} - */ - checkHealth() { - return this._health.check(); - } - - /** - * resolve the metadata information for a give topic - * will create topic if it doesnt exist - * @param {string} topic - name of the topic to query metadata for - * @param {number} timeout - optional, default is 2500 - * @returns {Promise.} - */ - getTopicMetadata(topic, timeout = 2500) { - return new Promise((resolve, reject) => { - - if (!this.consumer) { - return reject(new Error("You must call and await .connect() before trying to get metadata.")); - } - - if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Fetching topic metadata ${topic}`); - } - - this.consumer.getMetadata({ - topic, - timeout - }, (error, raw) => { - - if (error) { - return reject(error); - } - - resolve(new Metadata(raw)); - }); - }); - } - - /** - * @alias getTopicMetadata - * @param {number} timeout - optional, default is 2500 - * @returns {Promise.} - */ - getMetadata(timeout = 2500) { - return this.getTopicMetadata(null, timeout); - } - - /** - * returns a list of available kafka topics on the connected brokers - * @param {number} timeout - */ - async getTopicList(timeout = 2500) { - const metadata = await this.getMetadata(timeout); - return metadata.asTopicList(); - } -} - -module.exports = NConsumer; diff --git a/lib/librdkafka/NProducer.js b/lib/librdkafka/NProducer.js deleted file mode 100644 index 1eebacb..0000000 --- a/lib/librdkafka/NProducer.js +++ /dev/null @@ -1,711 +0,0 @@ -"use strict"; - -const EventEmitter = require("events"); -const Promise = require("bluebird"); -const uuid = require("uuid"); -const murmur = require("murmurhash"); -const debug = require("debug"); -const murmur2Partitioner = require("murmur2-partitioner"); - -const Metadata = require("./../shared/Metadata.js"); - -const { - ProducerAnalytics -} = require("./../shared/Analytics.js"); - -const { - ProducerHealth -} = require("./../shared/Health.js"); - -//@OPTIONAL -let BlizzKafka = null; - -const MESSAGE_TYPES = { - PUBLISH: "-published", - UNPUBLISH: "-unpublished", - UPDATE: "-updated" -}; - -const MAX_PART_AGE_MS = 1e3 * 60 * 5; //5 minutes -const MAX_PART_STORE_SIZE = 1e4; -const DEFAULT_MURMURHASH_VERSION = "3"; - -const DEFAULT_LOGGER = { - debug: debug("sinek:nproducer:debug"), - info: debug("sinek:nproducer:info"), - warn: debug("sinek:nproducer:warn"), - error: debug("sinek:nproducer:error") -}; - -/** - * native producer wrapper for node-librdkafka - * @extends EventEmitter - */ -class NProducer extends EventEmitter { - - /** - * creates a new producer instance - * @param {object} config - configuration object - * @param {*} _ - ignore this param (api compatability) - * @param {number|string} defaultPartitionCount - amount of default partitions for the topics to produce to - */ - constructor(config = { options: {}, health: {} }, _, defaultPartitionCount = 1) { - super(); - - if (!config) { - throw new Error("You are missing a config object."); - } - - if (!config.logger || typeof config.logger !== "object") { - config.logger = DEFAULT_LOGGER; - } - - try { - if (!BlizzKafka) { - BlizzKafka = require("node-rdkafka"); - } - } catch (error) { - config.logger.error(error); - throw new Error("You have to install node-rdkafka to use NProducer. " + error.message); - } - - if (!config.options) { - config.options = {}; - } - - this.config = config; - this._health = new ProducerHealth(this, this.config.health || {}); - - this.paused = false; - this.producer = null; - this._producerPollIntv = null; - this.defaultPartitionCount = defaultPartitionCount; - this._partitionCounts = {}; - this._inClosing = false; - this._totalSentMessages = 0; - this._lastProcessed = null; - this._analyticsOptions = null; - this._analyticsIntv = null; - this._analytics = null; - - this._murmurHashVersion = this.config.options.murmurHashVersion || DEFAULT_MURMURHASH_VERSION; - this.config.logger.info(`using murmur ${this._murmurHashVersion} partitioner.`); - - switch (this._murmurHashVersion) { - case "2": - this._murmur = (key, partitionCount) => murmur2Partitioner.partition(key, partitionCount); - break; - - case "3": - this._murmur = (key, partitionCount) => murmur.v3(key) % partitionCount; - break; - - default: - throw new Error(`${this._murmurHashVersion} is not a supported murmur hash version. Choose '2' or '3'.`); - } - - this._errors = 0; - super.on("error", () => this._errors++); - } - - /** - * @throws - * starts analytics tasks - * @param {object} options - analytic options - */ - enableAnalytics(options = {}) { - - if (this._analyticsIntv) { - throw new Error("analytics intervals are already running."); - } - - let { - analyticsInterval - } = options; - this._analyticsOptions = options; - - analyticsInterval = analyticsInterval || 1000 * 150; // 150 sec - - this._analyticsIntv = setInterval(this._runAnalytics.bind(this), analyticsInterval); - } - - /** - * halts all analytics tasks - */ - haltAnalytics() { - - if (this._analyticsIntv) { - clearInterval(this._analyticsIntv); - } - } - - /** - * connects to the broker - * @returns {Promise.<*>} - */ - connect() { - return new Promise((resolve, reject) => { - - let { - zkConStr, - kafkaHost, - logger, - options, - noptions, - tconf - } = this.config; - - const { - pollIntervalMs - } = options; - - let conStr = null; - - if (typeof kafkaHost === "string") { - conStr = kafkaHost; - } - - if (typeof zkConStr === "string") { - conStr = zkConStr; - } - - if (conStr === null && !noptions) { - return reject(new Error("One of the following: zkConStr or kafkaHost must be defined.")); - } - - if (conStr === zkConStr) { - return reject(new Error("NProducer does not support zookeeper connection.")); - } - - const config = { - "metadata.broker.list": conStr, - "dr_cb": true - }; - - noptions = noptions || {}; - noptions = Object.assign({}, config, noptions); - logger.debug(noptions); - - tconf = tconf ? tconf : { - "request.required.acks": 1 - }; - logger.debug(tconf); - - this.producer = new BlizzKafka.HighLevelProducer(noptions, tconf); - - this.producer.on("event.log", log => { - logger.debug(log.message); - }); - - this.producer.on("event.error", error => { - super.emit("error", error); - }); - - this.producer.on("error", error => { - super.emit("error", error); - }); - - this.producer.on("delivery-report", (error, report) => { - if (error) { - logger.error("DeliveryReport-Error: " + error.message); - } else { - logger.debug("DeliveryReport: " + JSON.stringify(report)); - } - }); - - /* ### this stuff is important, for the HighLevelProducer to work ### */ - - this.producer.setPollInterval(pollIntervalMs || 100); - - this.producer.setKeySerializer((value) => { - return Promise.resolve(value); - }); - - this.producer.setValueSerializer((value) => { - return value; - }); - - /* ### EOF STUFF ### */ - - this.producer.on("disconnected", () => { - if (this._inClosing) { - this._reset(); - } - logger.warn("Disconnected."); - //auto-reconnect??? -> handled by producer.poll() - }); - - this.producer.on("ready", () => { - logger.info(`Native producer ready v. ${BlizzKafka.librdkafkaVersion}, e. ${BlizzKafka.features.join(", ")}.`); - super.emit("ready"); - }); - - logger.debug("Connecting.."); - this.producer.connect(null, (error, metadata) => { - - if (error) { - super.emit("error", error); - return reject(error); - } - - logger.debug(metadata); - resolve(); - }); - }); - } - - /** - * returns a partition for a key - * @private - * @param {string} - message key - * @param {number} - partition count of topic, if 0 defaultPartitionCount is used - * @returns {string} - deterministic partition value for key - */ - _getPartitionForKey(key, partitionCount = 1) { - - if (typeof key !== "string") { - throw new Error("key must be a string."); - } - - if (typeof partitionCount !== "number") { - throw new Error("partitionCount must be number."); - } - - return this._murmur(key, partitionCount); - } - - /** - * @async - * produces a kafka message to a certain topic - * @param {string} topicName - name of the topic to produce to - * @param {object|string|null} message - value object for the message - * @param {number} _partition - optional partition to produce to - * @param {string} _key - optional message key - * @param {string} _partitionKey - optional key to evaluate partition for this message - * @param {array} _headers - optional array containing custom key value pairs that provide message metadata - * @returns {Promise.} - */ - async send(topicName, message, _partition = null, _key = null, _partitionKey = null, _headers = null) { - - if (!this.producer) { - throw new Error("You must call and await .connect() before trying to produce messages."); - } - - if (this.paused) { - throw new Error("producer is paused."); - } - - if (typeof message === "undefined" || !(typeof message === "string" || Buffer.isBuffer(message) || message === null)) { - throw new Error("message must be a string, an instance of Buffer or null."); - } - - const key = _key ? _key : uuid.v4(); - if (message !== null) { - message = Buffer.isBuffer(message) ? message : Buffer.from(message); - } - - let maxPartitions = 0; - //find correct max partition count - if (this.defaultPartitionCount === "auto" && typeof _partition !== "number") { //manual check to improve performance - maxPartitions = await this.getPartitionCountOfTopic(topicName); - if (maxPartitions === -1) { - throw new Error("defaultPartition set to 'auto', but was not able to resolve partition count for topic" + - topicName + ", please make sure the topic exists before starting the producer in auto mode."); - } - } else { - maxPartitions = this.defaultPartitionCount; - } - - let partition = 0; - //find correct partition for this key - if (maxPartitions >= 2 && typeof _partition !== "number") { //manual check to improve performance - partition = this._getPartitionForKey(_partitionKey ? _partitionKey : key, maxPartitions); - } - - //if _partition (manual) is set, it always overwrites a selected partition - partition = typeof _partition === "number" ? _partition : partition; - - this.config.logger.debug(JSON.stringify({ - topicName, - partition, - key - })); - - const producedAt = Date.now(); - - this._lastProcessed = producedAt; - this._totalSentMessages++; - - return new Promise((resolve, reject) => { - this.producer.produce(topicName, partition, message, key, producedAt, _headers, (error, offset) => { - - if (error) { - return reject(error); - } - - resolve({ - key, - partition, - offset - }); - }); - }); - } - - /** - * @async - * produces a formatted message to a topic - * @param {string} topic - topic to produce to - * @param {string} identifier - identifier of message (is the key) - * @param {object} payload - object (part of message value) - * @param {number} partition - optional partition to produce to - * @param {number} version - optional version of the message value - * @param {string} partitionKey - optional key to evaluate partition for this message - * @param {array} headers - optional array containing custom key value pairs that provide message metadata - * @returns {Promise.} - */ - async buffer(topic, identifier, payload, partition = null, version = null, partitionKey = null, headers = null) { - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - if (typeof payload !== "object") { - throw new Error("expecting payload to be of type object."); - } - - if (typeof payload.id === "undefined") { - payload.id = identifier; - } - - if (version && typeof payload.version === "undefined") { - payload.version = version; - } - - return await this.send(topic, JSON.stringify(payload), partition, identifier, partitionKey, headers); - } - - /** - * @async - * @private - * produces a specially formatted message to a topic - * @param {string} topic - topic to produce to - * @param {string} identifier - identifier of message (is the key) - * @param {object} _payload - object message value payload - * @param {number} version - optional version (default is 1) - * @param {*} _ -ignoreable, here for api compatibility - * @param {string} partitionKey - optional key to deterministcally detect partition - * @param {number} partition - optional partition (overwrites partitionKey) - * @param {string} messageType - optional messageType (for the formatted message value) - * @param {array} headers - optional object containing custom key value pairs that provide message metadata - * @returns {Promise.} - */ - async _sendBufferFormat(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null, messageType = "", headers = null) { - - if (typeof identifier === "undefined") { - identifier = uuid.v4(); - } - - if (typeof identifier !== "string") { - identifier = identifier + ""; - } - - if (typeof _payload !== "object") { - throw new Error("expecting payload to be of type object."); - } - - if (typeof _payload.id === "undefined") { - _payload.id = identifier; - } - - if (version && typeof _payload.version === "undefined") { - _payload.version = version; - } - - const payload = { - payload: _payload, - key: identifier, - id: uuid.v4(), - time: (new Date()).toISOString(), - type: topic + messageType - }; - - return await this.send(topic, JSON.stringify(payload), partition, identifier, partitionKey, headers); - } - - /** - * an alias for bufferFormatPublish() - * @alias bufferFormatPublish - */ - bufferFormat(topic, identifier, payload, version = 1, compressionType = 0, partitionKey = null, headers = null) { - return this.bufferFormatPublish(topic, identifier, payload, version, compressionType, partitionKey, headers); - } - - /** - * produces a specially formatted message to a topic, with type "publish" - * @param {string} topic - topic to produce to - * @param {string} identifier - identifier of message (is the key) - * @param {object} _payload - object message value payload - * @param {number} version - optional version (default is 1) - * @param {*} _ -ignoreable, here for api compatibility - * @param {string} partitionKey - optional key to deterministcally detect partition - * @param {number} partition - optional partition (overwrites partitionKey) - * @param {array} headers - optional object containing custom key value pairs that provide message metadata - * @returns {Promise.} - */ - bufferFormatPublish(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null, headers = null) { - return this._sendBufferFormat(topic, identifier, _payload, version, _, partitionKey, partition, MESSAGE_TYPES.PUBLISH, headers); - } - - /** - * produces a specially formatted message to a topic, with type "update" - * @param {string} topic - topic to produce to - * @param {string} identifier - identifier of message (is the key) - * @param {object} _payload - object message value payload - * @param {number} version - optional version (default is 1) - * @param {*} _ -ignoreable, here for api compatibility - * @param {string} partitionKey - optional key to deterministcally detect partition - * @param {number} partition - optional partition (overwrites partitionKey) - * @param {array} headers - optional object containing custom key value pairs that provide message metadata - * @returns {Promise.} - */ - bufferFormatUpdate(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null, headers = null) { - return this._sendBufferFormat(topic, identifier, _payload, version, _, partitionKey, partition, MESSAGE_TYPES.UPDATE, headers); - } - - /** - * produces a specially formatted message to a topic, with type "unpublish" - * @param {string} topic - topic to produce to - * @param {string} identifier - identifier of message (is the key) - * @param {object} _payload - object message value payload - * @param {number} version - optional version (default is 1) - * @param {*} _ -ignoreable, here for api compatibility - * @param {string} partitionKey - optional key to deterministcally detect partition - * @param {number} partition - optional partition (overwrites partitionKey) - * @param {array} headers - optional object containing custom key value pairs that provide message metadata - * @returns {Promise.} - */ - bufferFormatUnpublish(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null, headers = null) { - return this._sendBufferFormat(topic, identifier, _payload, version, _, partitionKey, partition, MESSAGE_TYPES.UNPUBLISH, headers); - } - - /** - * produces a tombstone (null payload with -1 size) message - * on a key compacted topic/partition this will delete all occurances of the key - * @param {string} topic - name of the topic - * @param {string} key - key - * @param {number|null} _partition - optional partition - */ - tombstone(topic, key, _partition = null) { - - if (!key) { - return Promise.reject(new Error("Tombstone messages only work on a key compacted topic, please provide a key.")); - } - - return this.send(topic, null, _partition, key, null, null); - } - - /** - * pauses production (sends will not be queued) - */ - pause() { - this.paused = true; - } - - /** - * resumes production - */ - resume() { - this.paused = false; - } - - /** - * returns producer statistics - * @returns {object} - */ - getStats() { - return { - totalPublished: this._totalSentMessages, - last: this._lastProcessed, - isPaused: this.paused, - totalErrors: this._errors - }; - } - - /** - * @deprecated - */ - refreshMetadata() { - throw new Error("refreshMetadata not implemented for nproducer."); - } - - /** - * resolve the metadata information for a give topic - * will create topic if it doesnt exist - * @param {string} topic - name of the topic to query metadata for - * @param {number} timeout - optional, default is 2500 - * @returns {Promise.} - */ - getTopicMetadata(topic, timeout = 2500) { - return new Promise((resolve, reject) => { - - if (!this.producer) { - return reject(new Error("You must call and await .connect() before trying to get metadata.")); - } - - this.producer.getMetadata({ - topic, - timeout - }, (error, raw) => { - - if (error) { - return reject(error); - } - - resolve(new Metadata(raw)); - }); - }); - } - - /** - * @alias getTopicMetadata - * @param {number} timeout - optional, default is 2500 - * @returns {Promise.} - */ - getMetadata(timeout = 2500) { - return this.getTopicMetadata(null, timeout); - } - - /** - * returns a list of available kafka topics on the connected brokers - * @param {number} timeout - */ - async getTopicList(timeout = 2500) { - const metadata = await this.getMetadata(timeout); - return metadata.asTopicList(); - } - - /** - * @async - * gets the partition count of the topic from the brokers metadata - * keeps a local cache to speed up future requests - * resolves to -1 if an error occures - * @param {string} topic - name of topic - * @returns {Promise.} - */ - async getPartitionCountOfTopic(topic) { - - if (!this.producer) { - throw new Error("You must call and await .connect() before trying to get metadata."); - } - - //prevent long running leaks.. - if (Object.keys(this._partitionCounts).length > MAX_PART_STORE_SIZE) { - this._partitionCounts = {}; - } - - const now = Date.now(); - if (!this._partitionCounts[topic] || this._partitionCounts[topic].requested + MAX_PART_AGE_MS < now) { - - let count = -1; - try { - const metadata = await this.getMetadata(); //prevent creation of topic, if it does not exist - count = metadata.getPartitionCountOfTopic(topic); - } catch (error) { - super.emit("error", new Error(`Failed to get metadata for topic ${topic}, because: ${error.message}.`)); - return -1; - } - - this._partitionCounts[topic] = { - requested: now, - count - }; - - return count; - } - - return this._partitionCounts[topic].count; - } - - /** - * gets the local partition count cache - * @returns {object} - */ - getStoredPartitionCounts() { - return this._partitionCounts; - } - - /** - * @private - * resets internal values - */ - _reset() { - this._lastProcessed = null; - this._totalSentMessages = 0; - this.paused = false; - this._inClosing = false; - this._partitionCounts = {}; - this._analytics = null; - this._errors = 0; - } - - /** - * closes connection if open - * stops poll interval if open - */ - close() { - - this.haltAnalytics(); - - if (this.producer) { - this._inClosing = true; - clearInterval(this._producerPollIntv); - this.producer.disconnect(); - //this.producer = null; - } - } - - /** - * called in interval - * @private - */ - _runAnalytics() { - - if (!this._analytics) { - this._analytics = new ProducerAnalytics(this, this._analyticsOptions || {}, this.config.logger); - } - - this._analytics.run() - .then(res => super.emit("analytics", res)) - .catch(error => super.emit("error", error)); - } - - /** - * returns the last computed analytics results - * @throws - * @returns {object} - */ - getAnalytics() { - - if (!this._analytics) { - super.emit("error", new Error("You have not enabled analytics on this consumer instance.")); - return {}; - } - - return this._analytics.getLastResult(); - } - - /** - * runs a health check and returns object with status and message - * @returns {Promise.} - */ - checkHealth() { - return this._health.check(); - } -} - -module.exports = NProducer; diff --git a/lib/librdkafka/README.md b/lib/librdkafka/README.md deleted file mode 100644 index 51ab3bc..0000000 --- a/lib/librdkafka/README.md +++ /dev/null @@ -1,3 +0,0 @@ -# Native Client Docs - -Have moved [here](../../docs/native.md). \ No newline at end of file diff --git a/lib/shared/index.js b/lib/shared/index.js deleted file mode 100644 index c88d3d6..0000000 --- a/lib/shared/index.js +++ /dev/null @@ -1,5 +0,0 @@ -"use strict"; - -const CompressionTypes = require("./CompressionTypes.js"); - -module.exports = { CompressionTypes }; diff --git a/package.json b/package.json index d1d7faf..c2a1a5a 100644 --- a/package.json +++ b/package.json @@ -2,20 +2,21 @@ "name": "sinek", "version": "9.1.0", "description": "Node.js kafka client, consumer, producer polite out of the box", - "main": "index.js", - "typings": "index.d.ts", + "main": "build/dist/index.js", + "types": "build/dist/index.d.ts", "engines": { "node": ">=8.11.3" }, "scripts": { - "lint": "eslint .", + "lint": "eslint src/. --ext .ts", "fix": "eslint --fix .", "kafka:start": "./kafka-setup/start.sh", "kafka:stop": "./kafka-setup/stop.sh", "kafka:logs": "docker-compose --file ./kafka-setup/docker-compose.yml logs -f", "kafka:console": "./kafka-setup/kafka-console.sh", "test": "_mocha --recursive --timeout 32500 --exit -R spec test/int", - "yarn:openssl": "LDFLAGS='-L/usr/local/opt/openssl/lib' CPPFLAGS='-I/usr/local/opt/openssl/include' yarn" + "yarn:openssl": "LDFLAGS='-L/usr/local/opt/openssl/lib' CPPFLAGS='-I/usr/local/opt/openssl/include' yarn", + "prepublishOnly": "tsc -p tsconfig.dist.json" }, "repository": { "type": "git", @@ -53,7 +54,6 @@ "async": "~3.2.0", "bluebird": "~3.7.2", "debug": "~4.1.1", - "kafka-node": "~5.0.0", "kafkajs": "1.12.0", "lodash.merge": "~4.6.2", "murmur2-partitioner": "~1.0.0", @@ -61,11 +61,29 @@ "uuid": "~7.0.3" }, "devDependencies": { + "@types/mocha": "^8.0.1", + "@types/node": "^14.0.27", + "@typescript-eslint/eslint-plugin": "^3.9.0", + "@typescript-eslint/parser": "^3.9.0", "eslint": "~6.8.0", "expect.js": "~0.3.1", "express": "~4.17.1", "istanbul": "~0.4.5", "mocha": "~7.1.1", - "sinon": "~9.0.2" + "sinon": "~9.0.2", + "ts-node": "^8.10.2", + "typescript": "^3.9.7" + }, + "mocha": { + "extension": [ + "ts" + ], + "exit": true, + "timeout": 32500, + "recursive": true, + "require": [ + "ts-node/register", + "source-map-support/register" + ] } } diff --git a/src/index.ts b/src/index.ts new file mode 100644 index 0000000..9c4e15b --- /dev/null +++ b/src/index.ts @@ -0,0 +1 @@ +export * from "./lib/Sinek"; diff --git a/src/lib/Sinek.ts b/src/lib/Sinek.ts new file mode 100644 index 0000000..deda282 --- /dev/null +++ b/src/lib/Sinek.ts @@ -0,0 +1,4 @@ +export * from "./kafkajs/JSConsumer"; +export * from "./kafkajs/JSProducer"; +export * from "./shared/Health"; +export * from "./shared/Analytics"; diff --git a/src/lib/interfaces.ts b/src/lib/interfaces.ts new file mode 100644 index 0000000..5b02f1a --- /dev/null +++ b/src/lib/interfaces.ts @@ -0,0 +1,277 @@ +import { CompressionTypes } from "kafkajs"; + +export interface KafkaHealthConfig { + thresholds: { + consumer: { + errors: number; + lag: number; + stallLag: number; + minMessages: number; + }; + producer: { + errors: number; + minMessages: number; + }; + }; +} + +export interface NCommonKafkaOptions { + "builtin.features"?: string; + "client.id"?: string; + "metadata.broker.list": string; + "message.max.bytes"?: number; + "message.copy.max.bytes"?: number; + "receive.message.max.bytes"?: number; + "max.in.flight.requests.per.connection"?: number; + "metadata.request.timeout.ms"?: number; + "topic.metadata.refresh.interval.ms"?: number; + "metadata.max.age.ms"?: number; + "topic.metadata.refresh.fast.interval.ms"?: number; + "topic.metadata.refresh.fast.cnt"?: number; + "topic.metadata.refresh.sparse"?: boolean; + "topic.blacklist"?: string; + "debug"?: string; + "socket.timeout.ms"?: number; + "socket.blocking.max.ms"?: number; + "socket.send.buffer.bytes"?: number; + "socket.receive.buffer.bytes"?: number; + "socket.keepalive.enable"?: boolean; + "socket.nagle.disable"?: boolean; + "socket.max.fails"?: number; + "broker.address.ttl"?: number; + "broker.address.family"?: "any" | "v4" | "v6"; + "reconnect.backoff.jitter.ms"?: number; + "statistics.interval.ms"?: number; + "enabled_events"?: number; + "log_level"?: number; + "log.queue"?: boolean; + "log.thread.name"?: boolean; + "log.connection.close"?: boolean; + "internal.termination.signal"?: number; + "api.version.request"?: boolean; + "api.version.fallback.ms"?: number; + "broker.version.fallback"?: string; + "security.protocol"?: "plaintext" | "ssl" | "sasl_plaintext" | "sasl_ssl"; + "ssl.cipher.suites"?: string; + "ssl.key.location"?: string; + "ssl.key.password"?: string; + "ssl.certificate.location"?: string; + "ssl.ca.location"?: string; + "ssl.crl.location"?: string; + "sasl.mechanisms"?: string; + "sasl.kerberos.service.name"?: string; + "sasl.kerberos.principal"?: string; + "sasl.kerberos.kinit.cmd"?: string; + "sasl.kerberos.keytab"?: string; + "sasl.kerberos.min.time.before.relogin"?: number; + "sasl.username"?: string; + "sasl.password"?: string; + "partition.assignment.strategy"?: string; + "session.timeout.ms"?: number; + "heartbeat.interval.ms"?: number; + "group.protocol.type"?: string; + "coordinator.query.interval.ms"?: number; + "group.id"?: string; + "event_cb"?: boolean; + "dr_cb"?: boolean; +} + +export interface NConsumerKafkaOptions extends NCommonKafkaOptions { + "group.id": string; + "enable.auto.commit"?: boolean; + "auto.commit.interval.ms"?: number; + "enable.auto.offset.store"?: boolean; + "queued.min.messages"?: number; + "queued.max.messages.kbytes"?: number; + "fetch.wait.max.ms"?: number; + "fetch.message.max.bytes"?: number; + "fetch.min.bytes"?: number; + "fetch.error.backoff.ms"?: number; + "offset.store.method"?: "none" | "file" | "broker"; + "enable.partition.eof"?: boolean; + "check.crcs"?: boolean; +} + +export interface NProducerKafkaOptions extends NCommonKafkaOptions { + "queue.buffering.max.messages"?: number; + "queue.buffering.max.kbytes"?: number; + "queue.buffering.max.ms"?: number; + "message.send.max.retries"?: number; + "retry.backoff.ms"?: number; + "compression.codec"?: CompressionTypes; + "batch.num.messages"?: number; + "delivery.report.only.error"?: boolean; +} + +export interface KafkaConsumerConfig { + kafkaHost?: string; + groupId?: string; + workerPerPartition?: number; + options?: { + sessionTimeout?: number; + protocol?: [string]; + fromOffset?: string; + fetchMaxBytes?: number; + fetchMinBytes?: number; + fetchMaxWaitMs?: number; + heartbeatInterval?: number; + retryMinTimeout?: number; + autoCommit?: boolean; + autoCommitIntervalMs?: number; + requireAcks?: number; + ackTimeoutMs?: number; + }; + health?: KafkaHealthConfig; + tconf?: { + "auto.commit.enable"?: boolean; + "auto.commit.interval.ms"?: number; + "auto.offset.reset"?: "smallest" | "earliest" | "beginning" | "largest" | "latest" | "end" | "error"; + "offset.store.path"?: string; + "offset.store.sync.interval.ms"?: number; + "offset.store.method"?: "file" | "broker"; + "consume.callback.max.messages"?: number; + }; + noptions?: NConsumerKafkaOptions; + logger?: KafkaLogger; +} + +export interface KafkaProducerConfig { + kafkaHost?: string; + clientName?: string; + workerPerPartition?: number; + options?: { + sessionTimeout?: number; + protocol?: [string]; + fromOffset?: string; + fetchMaxBytes?: number; + fetchMinBytes?: number; + fetchMaxWaitMs?: number; + heartbeatInterval?: number; + retryMinTimeout?: number; + requireAcks?: number; + ackTimeoutMs?: number; + partitionerType?: number; + murmurHashVersion?: string; + }; + health?: KafkaHealthConfig; + tconf?: { + "request.required.acks"?: number; + "request.timeout.ms"?: number; + "message.timeout.ms"?: number; + "produce.offset.report"?: boolean; + }; + noptions?: NProducerKafkaOptions; + logger?: KafkaLogger; +} + +// Setting noptions to required. +export interface JSKafkaProducerConfig extends KafkaProducerConfig { + noptions: NProducerKafkaOptions +} + +export interface JSKafkaConsumerConfig extends KafkaConsumerConfig { + noptions: NConsumerKafkaOptions +} + +export interface KafkaMessage { + topic: string; + partition: number; + offset: number; + key: Buffer | string; + value: Buffer | string | any; + size: number; + timestamp: number; +} + +export interface SortedMessageBatch { + [topic: string]: { + [partition: string]: KafkaMessage[]; + }; +} + +export interface BatchConfig { + batchSize?: number; + commitEveryNBatch?: number; + concurrency?: number; + commitSync?: boolean; + noBatchCommits?: boolean; + manualBatching?: boolean; + sortedManualBatch?: boolean; +} + +export interface ConsumerStats { + totalIncoming: number; + lastMessage: number; + receivedFirstMsg: boolean; + totalProcessed: number; + lastProcessed: number; + queueSize: null; + isPaused: boolean; + omittingQueue: boolean; + autoComitting: boolean; + consumedSinceCommit: number; + batch: { + current: number; + committs: number; + total: number; + config: BatchConfig; + currentEmptyFetches: number; + avgProcessingTime: number; + }; + lag: any; + totalErrors: number; + drainStats: Record|null; +} + +export interface LagStatus { + topic: string; + partition: number; + lowDistance: number; + highDistance: number; + detail: { + lowOffset: number; + highOffset: number; + comittedOffset: number; + }; +} + +export interface ProducerStats { + totalPublished: number; + last: number; + isPaused: boolean; + totalErrors: number; +} + +export interface MessageReturn { + key: string; + partition: number; + offset?: number | null; + value: string; +} + +export interface MessageProduce { + id: string; + version: number; +} + +export interface KafkaLogger { + debug(message: string): void; + info(message: string): void; + warn(message: string, error?: Error): void; + error(error: string | Error): void; +} + +export interface AnalyticsLagChange { + timelyDifference: number; + fetchPerformance: number; + newLags: Record; + changedLags: Record; + resolvedLags: { + [key: string]: Record; + }, + stallLags: Record; +} + +export interface AnalyticsConfig { + analyticsInterval: number; +} diff --git a/lib/kafkajs/JSConsumer.js b/src/lib/kafkajs/JSConsumer.ts similarity index 64% rename from lib/kafkajs/JSConsumer.js rename to src/lib/kafkajs/JSConsumer.ts index 01d1c2d..7ef45c8 100644 --- a/lib/kafkajs/JSConsumer.js +++ b/src/lib/kafkajs/JSConsumer.ts @@ -1,35 +1,92 @@ -"use strict"; +import { Promise } from "bluebird"; +import Debug from "debug"; +import { Kafka, Admin, Consumer, SASLMechanism, KafkaMessage } from "kafkajs"; +import fs from "fs"; +import { EventEmitter } from "events"; +import { BatchConfig, LagStatus, JSKafkaConsumerConfig, KafkaLogger, ConsumerStats, AnalyticsConfig } from "../interfaces"; +import { ConsumerAnalytics, ConsumerHealth, Metadata, Check, ConsumerRunResult } from "../shared"; -const Promise = require("bluebird"); -const EventEmitter = require("events"); -const debug = require("debug"); -const { Kafka } = require("kafkajs"); +const MESSAGE_CHARSET = "utf8"; -const { ConsumerAnalytics } = require("./../shared/Analytics.js"); -const { ConsumerHealth } = require("./../shared/Health.js"); -const Metadata = require("./../shared/Metadata.js"); +export interface FormattedKafkaMessage extends Omit { + value: Buffer | string | Record; +} -const MESSAGE_CHARSET = "utf8"; +export interface ComittedOffsets { + partition: number; + offset: string; + metadata: string | null; + topic: string; +} const DEFAULT_LOGGER = { - debug: debug("sinek:jsconsumer:debug"), - info: debug("sinek:jsconsumer:info"), - warn: debug("sinek:jsconsumer:warn"), - error: debug("sinek:jsconsumer:error") + debug: Debug("sinek:jsconsumer:debug"), + info: Debug("sinek:jsconsumer:info"), + warn: Debug("sinek:jsconsumer:warn"), + error: Debug("sinek:jsconsumer:error") +}; + +type Lag = { + status: LagStatus[], + at: number, + took: number +} + +type ConsumeCallback = ((messages, callback) => void) | null; + +const defaultLag = { + status: [], + at: 0, + took: 0, }; /** * wrapper around kafkajs that immitates nconsumer * @extends EventEmitter */ -class JSConsumer extends EventEmitter { +export class JSConsumer extends EventEmitter { + + kafkaClient: Kafka; + topics: string[]; + config: JSKafkaConsumerConfig; + asString = true; + asJSON = false; + asStream = false; + consumer: Consumer | undefined; + + private _firstMessageConsumed = false; + private _totalIncomingMessages = 0; + private _lastReceived = 0; + private _totalProcessedMessages = 0; + private _lastProcessed = 0; + private _isAutoCommitting = false; + private _batchCount = 0; + private _batchCommitts = 0; + private _batchConfig: BatchConfig = {}; + private _totalBatches = 0; + + private _lastLagStatus: Lag = defaultLag; + private _lagCache: Lag = defaultLag; + + private _analyticsOptions: AnalyticsConfig | null = null; + _analytics: ConsumerAnalytics | undefined; + private _consumedSinceCommit = 0; + private _emptyFetches = 0; + private _avgBatchProcessingTime = 0; + private _extCommitCallback: ((e: Error, partitions: any[]) => void) | undefined; + + private _errors = 0; + private _groupId = ""; + private _adminClient: Admin; + private _health: ConsumerHealth; + private _inClosing = false; /** * creates a new consumer instance * @param {string|Array} topics - topic or topics to subscribe to * @param {object} config - configuration object */ - constructor(topics, config = { options: {}, health: {} }) { + constructor(topics: string | string[], config: JSKafkaConsumerConfig) { super(); if (!config) { @@ -40,36 +97,39 @@ class JSConsumer extends EventEmitter { config.logger = DEFAULT_LOGGER; } - if (!config.options) { - config.options = {}; - } - - if (!config.noptions) { - config.noptions = {}; - } + const { + "metadata.broker.list": brokerList, + "client.id": clientId, + "security.protocol": securityProtocol, + "ssl.ca.location": sslCALocation, + "ssl.certificate.location": sslCertLocation, + "ssl.key.location": sslKeyLocation, + "ssl.key.password": sslKeyPassword, + "sasl.mechanisms": mechanism, + "sasl.username": username, + "sasl.password": password, + } = config.noptions; - const brokers = config.kafkaHost || - (config.noptions["metadata.broker.list"] && config.noptions["metadata.broker.list"].split(",")); - const clientId = config.noptions["client.id"]; + const brokers = brokerList.split(","); if (!brokers || !clientId) { throw new Error("You are missing a broker or group configs"); } - if (config.noptions["security.protocol"]) { + if (securityProtocol) { this.kafkaClient = new Kafka({ brokers, clientId, ssl: { - ca: [fs.readFileSync(config.noptions["ssl.ca.location"], "utf-8")], - cert: fs.readFileSync(config.noptions["ssl.certificate.location"], "utf-8"), - key: fs.readFileSync(config.noptions["ssl.key.location"], "utf-8"), - passphrase: config.noptions["ssl.key.password"], + ca: [fs.readFileSync(sslCALocation as string, "utf-8")], + cert: fs.readFileSync(sslCertLocation as string, "utf-8"), + key: fs.readFileSync(sslKeyLocation as string, "utf-8"), + passphrase: sslKeyPassword, }, sasl: { - mechanism: config.noptions["sasl.mechanisms"], - username: config.noptions["sasl.username"], - password: config.noptions["sasl.password"], + mechanism: mechanism as SASLMechanism, + username: username as string, + password: password as string, }, }); } else { @@ -79,50 +139,25 @@ class JSConsumer extends EventEmitter { this._adminClient = this.kafkaClient.admin(); this.topics = Array.isArray(topics) ? topics : [topics]; this.config = config; - this._health = new ConsumerHealth(this, this.config.health || {}); - - this.consumer = null; - this._inClosing = false; - this._firstMessageConsumed = false; - this._totalIncomingMessages = 0; - this._lastReceived = null; - this._totalProcessedMessages = 0; - this._lastProcessed = null; - this._isAutoCommitting = null; - this._batchCount = 0; - this._batchCommitts = 0; - this._totalBatches = 0; - this._lagCache = null; - this._analyticsOptions = null; - this._analytics = null; - this._lastLagStatus = null; - this._consumedSinceCommit = 0; - this._emptyFetches = 0; - this._avgBatchProcessingTime = 0; - this._extCommitCallback = null; - this._uncommitedOffsets = null; - this._asString = true; - this._asJSON = false; - - this._errors = 0; + this._health = new ConsumerHealth(this, this.config.health); - super.on("error", () => { - this._errors++ + this.on("error", () => { + this._errors++; }); - super.on("batch", (messages, { resolveOffset, syncEvent }) => { + this.on("batch", (messages, { resolveOffset, syncEvent }) => { const startBPT = Date.now(); this._totalIncomingMessages += messages.length; this._lastReceived = Date.now(); - const messageOffstes = []; + const messageOffsets: any[] = []; const mappedMessages = messages.map((message) => { - this.config.logger.debug(message); + this.config.logger!.debug(message); message.value = this._convertMessageValue(message.value, this.asString, this.asJSON); - super.emit("message", message); - messageOffstes.push(message.offset); + this.emit("message", message); + messageOffsets.push(message.offset); return message; }); @@ -137,11 +172,11 @@ class JSConsumer extends EventEmitter { this._bumpVariableOfBatch(startBPT, mappedMessages.length); try { - messageOffstes.forEach((offset) => { + messageOffsets.forEach((offset) => { resolveOffset(offset); }); } catch (error) { - super.emit("error", error); + this.emit("error", error); } }); }); @@ -153,34 +188,17 @@ class JSConsumer extends EventEmitter { * @param {object} opts - optional, options asString, asJSON (booleans) * @returns {Promise.<*>} */ - connect(asStream = false, opts = {}) { + connect(asStream = false): Promise { if (asStream) { return Promise.reject(new Error("JSConsumer does not support streaming mode.")); } - let { zkConStr, kafkaHost, logger, groupId, options, noptions, tconf } = this.config; - - let conStr = null; - - if (typeof kafkaHost === "string") { - conStr = kafkaHost; - } - - if (typeof zkConStr === "string") { - conStr = zkConStr; - } - - if (conStr === null && !noptions) { - return Promise.reject(new Error("One of the following: zkConStr or kafkaHost must be defined.")); - } - - if (conStr === zkConStr) { - return Promise.reject(new Error("NProducer does not support zookeeper connection.")); - } + const { logger, groupId } = this.config; + let { noptions, tconf } = this.config; const config = { - "metadata.broker.list": conStr, + "broker.list": null, "group.id": typeof groupId === "string" ? groupId : "", "enable.auto.commit": false, // default in librdkafka is true - what makes this dangerous for our batching logic(s) }; @@ -196,13 +214,14 @@ class JSConsumer extends EventEmitter { this._extCommitCallback = noptions["offset_commit_cb"]; } - noptions = noptions || {}; noptions = Object.assign({}, config, noptions, overwriteConfig); - logger.debug(noptions); - this._isAutoCommitting = noptions["enable.auto.commit"]; + + logger!.debug(JSON.stringify(noptions)); + + this._isAutoCommitting = noptions["enable.auto.commit"] || false; tconf = tconf || undefined; - logger.debug(tconf); + logger!.debug(JSON.stringify(tconf)); this._groupId = noptions["group.id"]; @@ -210,7 +229,7 @@ class JSConsumer extends EventEmitter { return Promise.reject(new Error("Group need to be configured on noptions['groupId.id']")); } - return this._connectInFlow(logger); + return this._connectInFlow(logger as KafkaLogger); } /** @@ -219,21 +238,21 @@ class JSConsumer extends EventEmitter { * @param {Error} error * @param {Array} partitions */ - _onOffsetCommit(error, partitions) { + _onOffsetCommit(error: Error, partitions: any[]): void { if (this._extCommitCallback) { try { this._extCommitCallback(error, partitions); } catch (error) { - super.emit("error", error); + this.emit("error", error); } } if (error) { - return this.config.logger.warn("commit request failed with an error: " + JSON.stringify(error)); + return this.config.logger!.warn("commit request failed with an error: " + JSON.stringify(error)); } - this.config.logger.debug(partitions); + this.config.logger!.debug(JSON.stringify(partitions)); } /** @@ -244,15 +263,15 @@ class JSConsumer extends EventEmitter { * @param {object} tconf * @returns {Promise.<*>} */ - _connectInFlow(logger) { + _connectInFlow(logger: KafkaLogger): Promise { - return new Promise(async (resolve, reject) => { + return new Promise(( resolve, reject ) => { this.consumer = this.kafkaClient.consumer({ groupId: this._groupId }); const { CONNECT, CRASH, DISCONNECT } = this.consumer.events; this.consumer.on(CRASH, error => { - super.emit("error", error); + this.emit("error", error); }); this.consumer.on(DISCONNECT, () => { @@ -265,22 +284,20 @@ class JSConsumer extends EventEmitter { this.consumer.on(CONNECT, payload => { logger.info(`KafkaJS consumer (flow) ready with group. Info: ${payload}.`); - super.emit("ready"); + this.emit("ready"); }); logger.debug("Connecting.."); try { - await Promise.all([ + Promise.all([ this.consumer.connect(), this._adminClient.connect(), - ]); + ]).then(resolve); } catch (error) { - super.emit("error", error); + this.emit("error", error); return reject(error); } - - resolve(); }); } @@ -290,7 +307,7 @@ class JSConsumer extends EventEmitter { * read a certain size of messages from the broker * @returns {boolean} */ - _consumerRun(syncEvent) { + _consumerRun(syncEvent: ConsumeCallback): Promise { if (!this.resume || !this.consumer) { return false; @@ -308,10 +325,12 @@ class JSConsumer extends EventEmitter { if (!isRunning() || isStale()) { if (this.config && this.config.logger && this.config.logger.debug) { - this.config.logger.debug(`Consumed recursively with error ${error.message}`); + // @todo - not sure where error comes from? + // this.config.logger.debug(`Consumed recursively with error ${error.message}`); + this.config.logger.debug(`Consumed recursively with error ${messages}`); } - super.emit("error", error); + this.emit("error", Error); } //retry asap @@ -323,8 +342,8 @@ class JSConsumer extends EventEmitter { } this._emptyFetches = 0; //reset - this._uncommitedOffsets = await uncommittedOffsets(); - super.emit("batch", batch.messages, { resolveOffset, syncEvent }); + await uncommittedOffsets(); + this.emit("batch", batch.messages, { resolveOffset, syncEvent }); } await heartbeat(); } @@ -339,33 +358,37 @@ class JSConsumer extends EventEmitter { * @param {boolean} asJSON * @returns {Buffer|string|object} */ - _convertMessageValue(_value, asString = true, asJSON = false) { + _convertMessageValue( + _value: Buffer, + asString = true, + asJSON = false + ): Buffer | string| Record { if (!_value) { return _value; } - let value = _value; - if (!asString && !asJSON) { - return value; + return _value; } + + let value; if (asString || asJSON) { - value = value.toString(MESSAGE_CHARSET); + value = _value.toString(MESSAGE_CHARSET); } if (asJSON) { try { value = JSON.parse(value); } catch (error) { - this.config.logger.warn(`Failed to parse message value as json: ${error.message}, ${value}`); + this.config.logger!.warn(`Failed to parse message value as json: ${error.message}, ${value}`); } } return value; } - _bumpVariableOfBatch(startBPT, batchLength) { + _bumpVariableOfBatch(startBPT: number, batchLength: number): void { this._totalProcessedMessages += batchLength; this._lastProcessed = Date.now(); @@ -376,19 +399,17 @@ class JSConsumer extends EventEmitter { this._totalBatches++; this._batchCount++; - this.config.logger.debug("committing after", this._batchCount, "batches, messages: " + this._consumedSinceCommit); - super.emit("commit", this._consumedSinceCommit); + this.config.logger!.debug(`committing after ${this._batchCount}, batches, messages: ${this._consumedSinceCommit}`); + this.emit("commit", this._consumedSinceCommit); this._batchCount = 0; this._batchCommitts++; this._consumedSinceCommit = 0; } - async _consumeHandler(syncEvent, { - manualBatching, - }) { + async _consumeHandler(syncEvent: ConsumeCallback, { manualBatching }: { manualBatching: boolean }): Promise { if (this._isAutoCommitting !== null && typeof this._isAutoCommitting !== "undefined") { - this.config.logger.warn("enable.auto.commit has no effect in 1:n consume-mode, set to null or undefined to remove this message." + + this.config.logger!.warn("enable.auto.commit has no effect in 1:n consume-mode, set to null or undefined to remove this message." + "You can pass 'noBatchCommits' as true via options to .consume(), if you want to commit manually."); } @@ -397,10 +418,10 @@ class JSConsumer extends EventEmitter { } if (!manualBatching) { - this.config.logger.warn("The consumer only allow manual batching for now"); + this.config.logger!.warn("The consumer only allow manual batching for now"); } - this.config.logger.info("Batching manually.."); + this.config.logger!.info("Batching manually.."); this._consumerRun(syncEvent); } @@ -419,7 +440,7 @@ class JSConsumer extends EventEmitter { * @param {object} options - optional object containing options for 1:n mode: * @returns {Promise.<*>} */ - consume(syncEvent = null, asString = true, asJSON = false, options = {}) { + consume(syncEvent: ConsumeCallback = null, asString = true, asJSON = false, options: BatchConfig): Promise { let { batchSize, @@ -439,18 +460,28 @@ class JSConsumer extends EventEmitter { manualBatching = typeof manualBatching === "undefined" ? true : manualBatching; //default is true sortedManualBatch = typeof sortedManualBatch === "undefined" ? false : sortedManualBatch; //default is false - this._asString = asString; - this._asJSON = asJSON; + this._batchConfig = { + batchSize, + commitEveryNBatch, + concurrency, + commitSync, + noBatchCommits, + manualBatching, + sortedManualBatch + } as BatchConfig; + + this.asString = asString; + this.asJSON = asJSON; if (!this.consumer) { return Promise.reject(new Error("You must call and await .connect() before trying to consume messages.")); } - if (syncEvent && this._asStream) { + if (syncEvent && this.asStream) { return Promise.reject(new Error("Usage of syncEvent is not permitted in streaming mode.")); } - if (this._asStream) { + if (this.asStream) { return Promise.reject(new Error("Calling .consume() is not required in streaming mode.")); } @@ -465,30 +496,32 @@ class JSConsumer extends EventEmitter { const topics = this.topics; if (topics && topics.length) { - this.config.logger.info(`Subscribing to topics: ${topics.join(", ")}.`); + this.config.logger!.info(`Subscribing to topics: ${topics.join(", ")}.`); topics.forEach(async (topic) => { - await this.consumer.subscribe({ topic }); + await this.consumer!.subscribe({ topic }); }); } else { - this.config.logger.info("Not subscribing to any topics initially."); + this.config.logger!.info("Not subscribing to any topics initially."); } if (!syncEvent) { return this.consumer.run({ eachMessage: async ({ message }) => { + const m: FormattedKafkaMessage = message; - this.config.logger.debug(message); + this.config.logger!.debug(JSON.stringify(message)); this._totalIncomingMessages++; this._lastReceived = Date.now(); - message.value = this._convertMessageValue(message.value, asString, asJSON); + + m.value = this._convertMessageValue(message.value, asString, asJSON); if (!this._firstMessageConsumed) { this._firstMessageConsumed = true; - super.emit("first-drain-message", message); + this.emit("first-drain-message", m); } - super.emit("message", message); + this.emit("message", m); } }); } @@ -503,7 +536,7 @@ class JSConsumer extends EventEmitter { * @param {Array.<{}>} topicPartitions * @throws {LibrdKafkaError} */ - pause(topicPartitions = []) { + pause(topicPartitions = []): void { if (this.consumer) { return this.consumer.pause(topicPartitions); } @@ -514,7 +547,7 @@ class JSConsumer extends EventEmitter { * @param {Array.<{}>} topicPartitions * @throws {LibrdKafkaError} */ - resume(topicPartitions = []) { + resume(topicPartitions = []): void { if (this.consumer) { return this.consumer.resume(topicPartitions); } @@ -522,9 +555,10 @@ class JSConsumer extends EventEmitter { /** * returns consumer statistics + * @todo - update type for consumer stats. * @returns {object} */ - getStats() { + getStats(): ConsumerStats { return { totalIncoming: this._totalIncomingMessages, lastMessage: this._lastReceived, @@ -542,7 +576,8 @@ class JSConsumer extends EventEmitter { committs: this._batchCommitts, total: this._totalBatches, currentEmptyFetches: this._emptyFetches, - avgProcessingTime: this._avgBatchProcessingTime + avgProcessingTime: this._avgBatchProcessingTime, + config: this._batchConfig, }, lag: this._lagCache, //read from private cache totalErrors: this._errors @@ -553,31 +588,30 @@ class JSConsumer extends EventEmitter { * @private * resets internal values */ - _reset() { + _reset(): void { this._firstMessageConsumed = false; this._inClosing = false; this._totalIncomingMessages = 0; - this._lastReceived = null; + this._lastReceived = 0; this._totalProcessedMessages = 0; - this._lastProcessed = null; - this._asStream = null; + this._lastProcessed = 0; + this.asStream = false; this._batchCount = 0; this._batchCommitts = 0; this._totalBatches = 0; - this._lagCache = null; - this._analytics = null; - this._lastLagStatus = null; + this._lagCache = defaultLag; + this._analytics = undefined; this._consumedSinceCommit = 0; this._emptyFetches = 0; this._avgBatchProcessingTime = 0; this._errors = 0; - this._extCommitCallback = null; + this._extCommitCallback = undefined; } /** * closes connection if open */ - async close() { + async close(): Promise { if (this.consumer) { this._inClosing = true; @@ -594,10 +628,9 @@ class JSConsumer extends EventEmitter { * for a given kafka topic * @param {string} topic - name of the kafka topic * @param {number} partition - optional, default is 0 - * @param {number} timeout - optional, default is 2500 * @returns {Promise.} */ - async getOffsetForTopicPartition(topic, partition = 0, timeout = 2500) { + async getOffsetForTopicPartition(topic: string, partition = 0): Promise { if (!this.consumer) { return Promise.reject(new Error("Consumer not yet connected.")); @@ -617,7 +650,7 @@ class JSConsumer extends EventEmitter { * @param {number} timeout - optional, default is 2500 * @returns {Promise.} */ - async getComittedOffsets(timeout = 2500) { + async getComittedOffsets(timeout = 2500): Promise { if (!this.consumer) { return []; @@ -627,7 +660,7 @@ class JSConsumer extends EventEmitter { this.config.logger.debug(`Fetching committed offsets ${timeout}`); } - return [].concat.apply([], + return [].concat([], await Promise.all( this.topics.map(async (topic) => { @@ -637,10 +670,7 @@ class JSConsumer extends EventEmitter { topic, }); - return offsets.map((offsetInfo) => { - offsetInfo.topic = topic; - return offsetInfo; - }); + return offsets.map((offsetInfo) => ({...offsetInfo, topic})); }) ) ); @@ -650,11 +680,11 @@ class JSConsumer extends EventEmitter { * gets all topic-partitions which are assigned to this consumer * @returns {Array} */ - async getAssignedPartitions() { + async getAssignedPartitions(): Promise<[]> { try { return (await this.getComittedOffsets()); } catch (error) { - super.emit("error", error); + this.emit("error", error); return []; } } @@ -666,7 +696,7 @@ class JSConsumer extends EventEmitter { * @param {number} partition - partition * @param {Array} offsets - commit offsets from getComittedOffsets() */ - static findPartitionOffset(topic, partition, offsets) { + static findPartitionOffset(topic: string, partition: number, offsets: ComittedOffsets[]): string { for (let i = 0; i < offsets.length; i++) { if (offsets[i].topic === topic && offsets[i].partition === partition) { @@ -684,14 +714,14 @@ class JSConsumer extends EventEmitter { * @param {boolean} noCache - when analytics are enabled the results can be taken from cache * @returns {Promise.} */ - async getLagStatus(noCache = false) { + async getLagStatus(noCache = false): Promise { if (!this.consumer) { return []; } //if allowed serve from cache - if (!noCache && this._lagCache && this._lagCache.status) { + if (!noCache && this._lagCache && this._lagCache.status!) { return this._lagCache.status; } @@ -706,16 +736,17 @@ class JSConsumer extends EventEmitter { const status = await Promise.all(assigned.map(async topicPartition => { try { const brokerState = await this.getOffsetForTopicPartition(topicPartition.topic, topicPartition.partition); - const comittedOffset = NConsumer.findPartitionOffset(topicPartition.topic, topicPartition.partition, comitted); + // const comittedOffset = NConsumer.findPartitionOffset(topicPartition.topic, topicPartition.partition, comitted); + // const topicOffset = await (await this._adminClient.fetchTopicOffsets(topicPartition.topic)).pop(); + // const comittedOffset = topicOffset.offset; return { topic: topicPartition.topic, partition: topicPartition.partition, - lowDistance: comittedOffset - brokerState.lowOffset, - highDistance: brokerState.highOffset - comittedOffset, + lowDistance: comitted - brokerState.lowOffset, + highDistance: brokerState.highOffset - comitted, detail: { lowOffset: brokerState.lowOffset, highOffset: brokerState.highOffset, - comittedOffset } }; } catch (error) { @@ -728,7 +759,7 @@ class JSConsumer extends EventEmitter { })); const duration = Date.now() - startT; - this.config.logger.info(`fetching and comparing lag status took: ${duration} ms.`); + this.config.logger!.info(`fetching and comparing lag status took: ${duration} ms.`); //store cached version if (status && Array.isArray(status)) { @@ -753,15 +784,15 @@ class JSConsumer extends EventEmitter { * called in interval * @private */ - _runAnalytics() { + _runAnalytics(): Promise { if (!this._analytics) { - this._analytics = new ConsumerAnalytics(this, this._analyticsOptions || {}, this.config.logger); + this._analytics = new ConsumerAnalytics(this, this._analyticsOptions, this.config.logger as KafkaLogger); } return this._analytics.run() - .then(res => super.emit("analytics", res)) - .catch(error => super.emit("error", error)); + .then(res => this.emit("analytics", res)) + .catch(error => this.emit("error", error)); } /** @@ -769,11 +800,11 @@ class JSConsumer extends EventEmitter { * @throws * @returns {object} */ - getAnalytics() { + getAnalytics(): ConsumerRunResult | null { if (!this._analytics) { - super.emit("error", new Error("You have not enabled analytics on this consumer instance.")); - return {}; + this.emit("error", new Error("You have not enabled analytics on this consumer instance.")); + return null; } return this._analytics.getLastResult(); @@ -783,15 +814,15 @@ class JSConsumer extends EventEmitter { * called in interval * @private */ - _runLagCheck() { - return this.getLagStatus(true).catch(() => { }); + _runLagCheck(): LagStatus[] { + return this.getLagStatus(true).catch(error => this.emit("error", error)); } /** * runs a health check and returns object with status and message * @returns {Promise.} */ - checkHealth() { + checkHealth(): Promise { return this._health.check(); } @@ -799,11 +830,10 @@ class JSConsumer extends EventEmitter { * resolve the metadata information for a give topic * will create topic if it doesnt exist * @param {string} topic - name of the topic to query metadata for - * @param {number} timeout - optional, default is 2500 * @returns {Promise.} */ - getTopicMetadata(topic, timeout = 2500) { - return new Promise((resolve, reject) => { + getTopicMetadata(topic: string): Promise { + return new Promise(( resolve, reject ) => { if (!this.consumer) { return reject(new Error("You must call and await .connect() before trying to get metadata.")); @@ -815,15 +845,9 @@ class JSConsumer extends EventEmitter { this._adminClient.fetchTopicMetadata({ topics: [topic], - timeout - }, (error, raw) => { - - if (error) { - return reject(error); - } - - resolve(new Metadata(raw[0])); - }); + }) + .then((raw) => resolve(new Metadata(raw[0]))) + .catch((e) => reject(e)); }); } @@ -832,18 +856,33 @@ class JSConsumer extends EventEmitter { * @param {number} timeout - optional, default is 2500 * @returns {Promise.} */ - getMetadata(timeout = 2500) { - return this.getTopicMetadata(null, timeout); + getMetadata(): Promise { + return this.getTopicMetadata(""); } /** * returns a list of available kafka topics on the connected brokers - * @param {number} timeout */ - async getTopicList(timeout = 2500) { - const metadata = await this.getMetadata(timeout); + async getTopicList(): Promise { + const metadata: Metadata = await this.getMetadata(); return metadata.asTopicList(); } -} -module.exports = JSConsumer; + /** + * Gets the last lag status + * + * @returns {Lag} + */ + getLastLagStatus(): Lag { + return this._lastLagStatus; + } + + /** + * Gets the lag cache + * + * @returns {Lag} + */ + getLagCache(): Lag { + return this._lagCache; + } +} diff --git a/lib/kafkajs/JSProducer.js b/src/lib/kafkajs/JSProducer.ts similarity index 62% rename from lib/kafkajs/JSProducer.js rename to src/lib/kafkajs/JSProducer.ts index 951ec3e..4f7e499 100644 --- a/lib/kafkajs/JSProducer.js +++ b/src/lib/kafkajs/JSProducer.ts @@ -1,22 +1,13 @@ -"use strict"; - -const EventEmitter = require("events"); -const Promise = require("bluebird"); -const uuid = require("uuid"); -const murmur = require("murmurhash"); -const debug = require("debug"); -const murmur2Partitioner = require("murmur2-partitioner"); -const { Kafka } = require("kafkajs"); - -const Metadata = require("./../shared/Metadata.js"); - -const { - ProducerAnalytics -} = require("./../shared/Analytics.js"); - -const { - ProducerHealth -} = require("./../shared/Health.js"); +import { Promise } from "bluebird"; +import Debug from "debug"; +import { EventEmitter } from "events"; +import { v4 as uuidv4} from "uuid"; +import { murmur } from "murmurhash"; +import { murmur2Partitioner } from "murmur2-partitioner"; +import { Kafka, SASLMechanism, Admin, Producer, RecordMetadata, CompressionTypes } from "kafkajs"; +import { Metadata, ProducerAnalytics, ProducerHealth, Check, ProducerRunResult, defaultAnalyticsInterval } from "../shared"; +import { MessageReturn, JSKafkaProducerConfig, ProducerStats, AnalyticsConfig, KafkaLogger } from "../interfaces"; +import fs from "fs"; const MESSAGE_TYPES = { PUBLISH: "-published", @@ -29,25 +20,47 @@ const MAX_PART_STORE_SIZE = 1e4; const DEFAULT_MURMURHASH_VERSION = "3"; const DEFAULT_LOGGER = { - debug: debug("sinek:jsproducer:debug"), - info: debug("sinek:jsproducer:info"), - warn: debug("sinek:jsproducer:warn"), - error: debug("sinek:jsproducer:error") + debug: Debug("sinek:jsproducer:debug"), + info: Debug("sinek:jsproducer:info"), + warn: Debug("sinek:jsproducer:warn"), + error: Debug("sinek:jsproducer:error") }; /** * native producer wrapper for node-librdkafka * @extends EventEmitter */ -class JSProducer extends EventEmitter { +export class JSProducer extends EventEmitter { + + kafkaClient: Kafka; + config: JSKafkaProducerConfig; + + paused = false; + producer: Producer | undefined; + + private _health: ProducerHealth; + private _adminClient: Admin; + private _producerPollIntv = 0; + private _partitionCounts = {}; + private _inClosing = false; + private _totalSentMessages = 0; + private _lastProcessed = 0; + private _analyticsOptions: AnalyticsConfig | null = null; + private _analyticsIntv: NodeJS.Timeout | null = null; + _analytics: ProducerAnalytics | undefined; + private _murmurHashVersion: string = DEFAULT_MURMURHASH_VERSION; + private _murmur; + private _errors = 0; + + defaultPartitionCount = 1; /** * creates a new producer instance * @param {object} config - configuration object * @param {*} _ - ignore this param (api compatability) - * @param {number|string} defaultPartitionCount - amount of default partitions for the topics to produce to + * @param {number} defaultPartitionCount - amount of default partitions for the topics to produce to */ - constructor(config = { options: {}, health: {} }, _, defaultPartitionCount = 1) { + constructor(config: JSKafkaProducerConfig, defaultPartitionCount = 1) { super(); if (!config) { @@ -62,32 +75,39 @@ class JSProducer extends EventEmitter { config.options = {}; } - if (!config.noptions) { - config.noptions = {}; - } + const { + "metadata.broker.list": brokerList, + "client.id": clientId, + "security.protocol": securityProtocol, + "ssl.ca.location": sslCALocation, + "ssl.certificate.location": sslCertLocation, + "ssl.key.location": sslKeyLocation, + "ssl.key.password": sslKeyPassword, + "sasl.mechanisms": mechanism, + "sasl.username": username, + "sasl.password": password, + } = config.noptions; - const brokers = config.kafkaHost || - (config.noptions["metadata.broker.list"] && config.noptions["metadata.broker.list"].split(",")); - const clientId = config.noptions["client.id"]; + const brokers = brokerList.split(","); if (!brokers || !clientId) { throw new Error("You are missing a broker or group configs"); } - if (config.noptions["security.protocol"]) { + if (securityProtocol) { this.kafkaClient = new Kafka({ brokers, clientId, ssl: { - ca: [fs.readFileSync(config.noptions["ssl.ca.location"], "utf-8")], - cert: fs.readFileSync(config.noptions["ssl.certificate.location"], "utf-8"), - key: fs.readFileSync(config.noptions["ssl.key.location"], "utf-8"), - passphrase: config.noptions["ssl.key.password"], + ca: [fs.readFileSync(sslCALocation as string, "utf-8")], + cert: fs.readFileSync(sslCertLocation as string, "utf-8"), + key: fs.readFileSync(sslKeyLocation as string, "utf-8"), + passphrase: sslKeyPassword, }, sasl: { - mechanism: config.noptions["sasl.mechanisms"], - username: config.noptions["sasl.username"], - password: config.noptions["sasl.password"], + mechanism: mechanism as SASLMechanism, + username: username as string, + password: password as string, }, }); } else { @@ -95,39 +115,28 @@ class JSProducer extends EventEmitter { } this.config = config; - this._health = new ProducerHealth(this, this.config.health || {}); + this._health = new ProducerHealth(this, this.config.health); this._adminClient = this.kafkaClient.admin(); - this.paused = false; - this.producer = null; - this._producerPollIntv = null; - this.defaultPartitionCount = defaultPartitionCount; - this._partitionCounts = {}; - this._inClosing = false; - this._totalSentMessages = 0; - this._lastProcessed = null; - this._analyticsOptions = null; - this._analyticsIntv = null; - this._analytics = null; + this._murmurHashVersion = this.config.options!.murmurHashVersion || DEFAULT_MURMURHASH_VERSION; + this.config.logger!.info(`using murmur ${this._murmurHashVersion} partitioner.`); - this._murmurHashVersion = this.config.options.murmurHashVersion || DEFAULT_MURMURHASH_VERSION; - this.config.logger.info(`using murmur ${this._murmurHashVersion} partitioner.`); + this.defaultPartitionCount = defaultPartitionCount; switch (this._murmurHashVersion) { - case "2": - this._murmur = (key, partitionCount) => murmur2Partitioner.partition(key, partitionCount); - break; + case "2": + this._murmur = (key, partitionCount) => murmur2Partitioner.partition(key, partitionCount); + break; - case "3": - this._murmur = (key, partitionCount) => murmur.v3(key) % partitionCount; - break; + case "3": + this._murmur = (key, partitionCount) => murmur.v3(key) % partitionCount; + break; - default: - throw new Error(`${this._murmurHashVersion} is not a supported murmur hash version. Choose '2' or '3'.`); + default: + throw new Error(`${this._murmurHashVersion} is not a supported murmur hash version. Choose '2' or '3'.`); } - this._errors = 0; - super.on("error", () => this._errors++); + this.on("error", () => this._errors++); } /** @@ -135,18 +144,16 @@ class JSProducer extends EventEmitter { * starts analytics tasks * @param {object} options - analytic options */ - enableAnalytics(options = {}) { + enableAnalytics(options: { analyticsInterval: number } = {analyticsInterval: defaultAnalyticsInterval}): void { if (this._analyticsIntv) { throw new Error("analytics intervals are already running."); } - let { - analyticsInterval - } = options; + let { analyticsInterval } = options; this._analyticsOptions = options; - analyticsInterval = analyticsInterval || 1000 * 150; // 150 sec + analyticsInterval = analyticsInterval || defaultAnalyticsInterval; // 150 sec this._analyticsIntv = setInterval(this._runAnalytics.bind(this), analyticsInterval); } @@ -154,7 +161,7 @@ class JSProducer extends EventEmitter { /** * halts all analytics tasks */ - haltAnalytics() { + haltAnalytics(): void { if (this._analyticsIntv) { clearInterval(this._analyticsIntv); @@ -165,38 +172,19 @@ class JSProducer extends EventEmitter { * connects to the broker * @returns {Promise.<*>} */ - connect() { - return new Promise(async (resolve, reject) => { - - let { - zkConStr, - kafkaHost, - logger, - options, - noptions, - tconf - } = this.config; - - const { - pollIntervalMs - } = options; + connect(): Promise { + return new Promise((resolve, reject) => { + const { kafkaHost, logger } = this.config; + let { noptions, tconf } = this.config; - let conStr = null; + let conStr: string | null = null; if (typeof kafkaHost === "string") { conStr = kafkaHost; } - if (typeof zkConStr === "string") { - conStr = zkConStr; - } - if (conStr === null && !noptions) { - return reject(new Error("One of the following: zkConStr or kafkaHost must be defined.")); - } - - if (conStr === zkConStr) { - return reject(new Error("NProducer does not support zookeeper connection.")); + return reject(new Error("KafkaHost must be defined.")); } const config = { @@ -204,52 +192,49 @@ class JSProducer extends EventEmitter { "dr_cb": true }; - noptions = noptions || {}; noptions = Object.assign({}, config, noptions); - logger.debug(noptions); + logger!.debug(JSON.stringify(noptions)); tconf = tconf ? tconf : { "request.required.acks": 1 }; - logger.debug(tconf); + + logger!.debug(JSON.stringify(tconf)); this.producer = this.kafkaClient.producer(); const { CONNECT, DISCONNECT, REQUEST_TIMEOUT } = this.producer.events; this.producer.on(REQUEST_TIMEOUT, details => { - super.emit("error", new Error(`Request Timed out. Info ${JSON.stringify(details)}`)); + this.emit("error", new Error(`Request Timed out. Info ${JSON.stringify(details)}`)); }); /* ### EOF STUFF ### */ - this.producer.on(DISCONNECT, () => { if (this._inClosing) { this._reset(); } - logger.warn("Disconnected."); + logger!.warn("Disconnected."); //auto-reconnect??? -> handled by producer.poll() }); this.producer.on(CONNECT, () => { - logger.info(`KafkaJS producer is ready.`); - super.emit("ready"); + logger!.info("KafkaJS producer is ready."); + this.emit("ready"); }); - logger.debug("Connecting.."); + logger!.debug("Connecting.."); try { - await Promise.all([ + Promise.all([ this.producer.connect(), this._adminClient.connect(), - ]); + ]).then(resolve); } catch (error) { - super.emit("error", error); + this.emit("error", error); return reject(error); } - - resolve(); }); } @@ -260,7 +245,7 @@ class JSProducer extends EventEmitter { * @param {number} - partition count of topic, if 0 defaultPartitionCount is used * @returns {string} - deterministic partition value for key */ - _getPartitionForKey(key, partitionCount = 1) { + _getPartitionForKey(key: string, partitionCount = 1): number { if (typeof key !== "string") { throw new Error("key must be a string."); @@ -283,7 +268,13 @@ class JSProducer extends EventEmitter { * @param {string} _partitionKey - optional key to evaluate partition for this message * @returns {Promise.} */ - async send(topicName, message, _partition = null, _key = null, _partitionKey = null) { + async send( + topicName: string, + message: Record | string | null | Buffer, + _partition: number | null = null, + _key: string | null = null, + _partitionKey: string | null = null + ): Promise { /* these are not supported in the HighLevelProducer of node-rdkafka @@ -303,14 +294,16 @@ class JSProducer extends EventEmitter { throw new Error("message must be a string, an instance of Buffer or null."); } - const key = _key ? _key : uuid.v4(); + const key = _key ? _key : uuidv4(); + let convertedMessage: Buffer; + if (message !== null) { - message = Buffer.isBuffer(message) ? message : Buffer.from(message); + convertedMessage = Buffer.isBuffer(message) ? message : Buffer.from(message); } let maxPartitions = 0; //find correct max partition count - if (this.defaultPartitionCount === "auto" && typeof _partition !== "number") { //manual check to improve performance + if (typeof _partition !== "number") { //manual check to improve performance maxPartitions = await this.getPartitionCountOfTopic(topicName); if (maxPartitions === -1) { throw new Error("defaultPartition set to 'auto', but was not able to resolve partition count for topic" + @@ -329,7 +322,7 @@ class JSProducer extends EventEmitter { //if _partition (manual) is set, it always overwrites a selected partition partition = typeof _partition === "number" ? _partition : partition; - this.config.logger.debug(JSON.stringify({ + this.config.logger!.debug(JSON.stringify({ topicName, partition, key @@ -339,29 +332,34 @@ class JSProducer extends EventEmitter { this._lastProcessed = producedAt; this._totalSentMessages++; - const acks = this.config - && this.config.tconf && - this.config.tconf["request.required.acks"] || 1; + const timestamp = producedAt.toString(); + const acks = this.config && this.config.tconf && this.config.tconf["request.required.acks"] || 1; + const compression = (this.config.noptions) + ? this.config.noptions["compression.codec"] + : CompressionTypes.None; - return new Promise(async (resolve, reject) => { - - this.producer.send({ + return new Promise((resolve, reject) => { + this.producer!.send({ topic: topicName, acks, - messages: [ - { key: key, value: message, partition, timestamp: producedAt }, - ], - }) - .then((metadata) => { - resolve({ - key, + compression, + messages: [{ + key, + value: convertedMessage, partition, - offset: metadata.offset, - }); + timestamp + }], }) - .catch((error) => { - reject(error); - }); + .then((metadata: RecordMetadata[] ) => { + resolve({ + key, + partition, + offset: metadata[0].offset, + }); + }) + .catch((error) => { + reject(error); + }); }); } @@ -377,10 +375,17 @@ class JSProducer extends EventEmitter { * @param {string} partitionKey - optional key to evaluate partition for this message * @returns {Promise.} */ - async buffer(topic, identifier, payload, partition = null, version = null, partitionKey = null) { + async buffer( + topic: string, + identifier: string, + payload: Record, + partition: number | null = null, + version: number | null = null, + partitionKey: string | null = null + ): Promise { if (typeof identifier === "undefined") { - identifier = uuid.v4(); + identifier = uuidv4(); } if (typeof identifier !== "string") { @@ -416,10 +421,19 @@ class JSProducer extends EventEmitter { * @param {string} messageType - optional messageType (for the formatted message value) * @returns {Promise.} */ - async _sendBufferFormat(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null, messageType = "") { + async _sendBufferFormat( + topic: string, + identifier: string, + _payload: Record, + version = 1, + _: null | number, + partitionKey: string | null = null, + partition: number | null = null, + messageType = "" + ): Promise { if (typeof identifier === "undefined") { - identifier = uuid.v4(); + identifier = uuidv4(); } if (typeof identifier !== "string") { @@ -441,7 +455,7 @@ class JSProducer extends EventEmitter { const payload = { payload: _payload, key: identifier, - id: uuid.v4(), + id: uuidv4(), time: (new Date()).toISOString(), type: topic + messageType }; @@ -453,7 +467,14 @@ class JSProducer extends EventEmitter { * an alias for bufferFormatPublish() * @alias bufferFormatPublish */ - bufferFormat(topic, identifier, payload, version = 1, compressionType = 0, partitionKey = null) { + bufferFormat( + topic: string, + identifier: string, + payload: Record, + version = 1, + compressionType = 0, + partitionKey: string | null = null + ): Promise { return this.bufferFormatPublish(topic, identifier, payload, version, compressionType, partitionKey); } @@ -468,7 +489,15 @@ class JSProducer extends EventEmitter { * @param {number} partition - optional partition (overwrites partitionKey) * @returns {Promise.} */ - bufferFormatPublish(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null) { + bufferFormatPublish( + topic: string, + identifier: string, + _payload: Record, + version = 1, + _: null | number, + partitionKey: string | null = null, + partition: number | null = null + ): Promise { return this._sendBufferFormat(topic, identifier, _payload, version, _, partitionKey, partition, MESSAGE_TYPES.PUBLISH); } @@ -483,7 +512,15 @@ class JSProducer extends EventEmitter { * @param {number} partition - optional partition (overwrites partitionKey) * @returns {Promise.} */ - bufferFormatUpdate(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null) { + bufferFormatUpdate( + topic: string, + identifier: string, + _payload: Record, + version = 1, + _: null | number, + partitionKey: string | null = null, + partition: number | null = null + ): Promise { return this._sendBufferFormat(topic, identifier, _payload, version, _, partitionKey, partition, MESSAGE_TYPES.UPDATE); } @@ -498,7 +535,15 @@ class JSProducer extends EventEmitter { * @param {number} partition - optional partition (overwrites partitionKey) * @returns {Promise.} */ - bufferFormatUnpublish(topic, identifier, _payload, version = 1, _, partitionKey = null, partition = null) { + bufferFormatUnpublish( + topic: string, + identifier: string, + _payload: Record, + version = 1, + _: null | number, + partitionKey: string | null = null, + partition: number | null = null + ): Promise { return this._sendBufferFormat(topic, identifier, _payload, version, _, partitionKey, partition, MESSAGE_TYPES.UNPUBLISH); } @@ -509,34 +554,39 @@ class JSProducer extends EventEmitter { * @param {string} key - key * @param {number|null} _partition - optional partition */ - tombstone(topic, key, _partition = null) { + tombstone( + topic: string, + key: string, + _partition: number | null = null + ): Promise { if (!key) { return Promise.reject(new Error("Tombstone messages only work on a key compacted topic, please provide a key.")); } - return this.send(topic, null, _partition, key, null, null); + return this.send(topic, null, _partition, key, null); } /** * pauses production (sends will not be queued) */ - pause() { + pause(): void { this.paused = true; } /** * resumes production */ - resume() { + resume(): void { this.paused = false; } /** * returns producer statistics + * * @todo - update type for producer stats. * @returns {object} */ - getStats() { + getStats(): ProducerStats { return { totalPublished: this._totalSentMessages, last: this._lastProcessed, @@ -548,7 +598,7 @@ class JSProducer extends EventEmitter { /** * @deprecated */ - refreshMetadata() { + refreshMetadata(): void { throw new Error("refreshMetadata not implemented for nproducer."); } @@ -559,42 +609,38 @@ class JSProducer extends EventEmitter { * @param {number} timeout - optional, default is 2500 * @returns {Promise.} */ - getTopicMetadata(topic, timeout = 2500) { + getTopicMetadata(topic: string): Promise { return new Promise((resolve, reject) => { if (!this.producer) { return reject(new Error("You must call and await .connect() before trying to get metadata.")); } - this._adminClient.fetchTopicMetadata({ - topics: [topic], - timeout - }, (error, raw) => { + const topics = (topic === "") + ? [] + : [topic]; - if (error) { - return reject(error); - } - - resolve(new Metadata(raw[0])); - }); + this._adminClient.fetchTopicMetadata({ + topics, + }).then((raw) => { + resolve(new Metadata(raw)); + }).catch((e) => reject(e)); }); } /** * @alias getTopicMetadata - * @param {number} timeout - optional, default is 2500 * @returns {Promise.} */ - getMetadata(timeout = 2500) { - return this.getTopicMetadata(null, timeout); + getMetadata(): Promise { + return this.getTopicMetadata(""); } /** * returns a list of available kafka topics on the connected brokers - * @param {number} timeout */ - async getTopicList(timeout = 2500) { - const metadata = await this.getMetadata(timeout); + async getTopicList(): Promise { + const metadata: Metadata = await this.getMetadata(); return metadata.asTopicList(); } @@ -606,7 +652,7 @@ class JSProducer extends EventEmitter { * @param {string} topic - name of topic * @returns {Promise.} */ - async getPartitionCountOfTopic(topic) { + async getPartitionCountOfTopic(topic: string): Promise { if (!this.producer) { throw new Error("You must call and await .connect() before trying to get metadata."); @@ -625,7 +671,7 @@ class JSProducer extends EventEmitter { const metadata = await this.getMetadata(); //prevent creation of topic, if it does not exist count = metadata.getPartitionCountOfTopic(topic); } catch (error) { - super.emit("error", new Error(`Failed to get metadata for topic ${topic}, because: ${error.message}.`)); + this.emit("error", new Error(`Failed to get metadata for topic ${topic}, because: ${error}.`)); return -1; } @@ -644,7 +690,7 @@ class JSProducer extends EventEmitter { * gets the local partition count cache * @returns {object} */ - getStoredPartitionCounts() { + getStoredPartitionCounts(): Record { return this._partitionCounts; } @@ -652,13 +698,13 @@ class JSProducer extends EventEmitter { * @private * resets internal values */ - _reset() { - this._lastProcessed = null; + private _reset() { + this._lastProcessed = 0; this._totalSentMessages = 0; this.paused = false; this._inClosing = false; this._partitionCounts = {}; - this._analytics = null; + this._analytics = undefined; this._errors = 0; } @@ -666,7 +712,7 @@ class JSProducer extends EventEmitter { * closes connection if open * stops poll interval if open */ - async close() { + async close(): Promise { this.haltAnalytics(); @@ -691,15 +737,15 @@ class JSProducer extends EventEmitter { * called in interval * @private */ - _runAnalytics() { + private _runAnalytics(): void { if (!this._analytics) { - this._analytics = new ProducerAnalytics(this, this._analyticsOptions || {}, this.config.logger); + this._analytics = new ProducerAnalytics(this, this._analyticsOptions, this.config.logger as KafkaLogger); } this._analytics.run() - .then(res => super.emit("analytics", res)) - .catch(error => super.emit("error", error)); + .then(res => this.emit("analytics", res)) + .catch(error => this.emit("error", error)); } /** @@ -707,11 +753,11 @@ class JSProducer extends EventEmitter { * @throws * @returns {object} */ - getAnalytics() { + getAnalytics(): ProducerRunResult|null { if (!this._analytics) { - super.emit("error", new Error("You have not enabled analytics on this consumer instance.")); - return {}; + this.emit("error", new Error("You have not enabled analytics on this consumer instance.")); + return null; } return this._analytics.getLastResult(); @@ -719,11 +765,9 @@ class JSProducer extends EventEmitter { /** * runs a health check and returns object with status and message - * @returns {Promise.} + * @returns {Promise.} */ - checkHealth() { + checkHealth(): Promise { return this._health.check(); } } - -module.exports = JSProducer; diff --git a/src/lib/kafkajs/index.ts b/src/lib/kafkajs/index.ts new file mode 100644 index 0000000..4d309d4 --- /dev/null +++ b/src/lib/kafkajs/index.ts @@ -0,0 +1,2 @@ +export * from "./JSConsumer"; +export * from "./JSProducer"; diff --git a/lib/shared/Analytics.js b/src/lib/shared/Analytics.ts similarity index 66% rename from lib/shared/Analytics.js rename to src/lib/shared/Analytics.ts index ff3efa1..a04d1b1 100644 --- a/lib/shared/Analytics.js +++ b/src/lib/shared/Analytics.ts @@ -1,24 +1,59 @@ -"use strict"; +import { LagStatus, AnalyticsLagChange, KafkaLogger, AnalyticsConfig, ConsumerStats, ProducerStats } from "../interfaces"; +import { JSConsumer, JSProducer } from "../kafkajs"; const INTERESTING_DISTANCE = 10; +export const defaultAnalyticsInterval = 1000 * 150; + +interface RunResult { + generatedAt: number; + interval: number; + errors: number | null; +} + +export interface ConsumerRunResult extends RunResult { + lagChange: AnalyticsLagChange; + largestLag: { + topic: string; + partition: number; + lowDistance: number; + highDistance: number; + detail: { + lowOffset: number, + highOffset: number, + comittedOffset: number + } + }; + consumed: number | null; +} + +export interface ProducerRunResult extends RunResult { + produced: number | null; + interval: number; +} /** * parent analytics class */ -class Analytics { +abstract class Analytics { + + abstract client: JSConsumer | JSProducer; + config: AnalyticsConfig | null = null; + logger: KafkaLogger; + + _lastErrors = 0; + _consumedCount = 0; + + abstract _lastRes: RunResult | null = null; + _producedCount = 0; /** * creates a new instance - * @param {NConsumer|NProducer} client * @param {object} config * @param {object} logger */ - constructor(client, config, logger) { - this.client = client; + constructor(config: AnalyticsConfig | null = null, logger: KafkaLogger) { this.config = config; this.logger = logger; - - this._lastErrors = 0; } /** @@ -27,7 +62,7 @@ class Analytics { * @param {object} stats - getStats() client result * @returns {number} */ - _errorsInInterval(stats) { + _errorsInInterval(stats): number { const diff = (stats.totalErrors || 0) - this._lastErrors; this._lastErrors = stats.totalErrors || 0; return diff; @@ -37,7 +72,7 @@ class Analytics { * @static * @param {Array} offsets */ - static statusArrayToKeyedObject(offsets = []) { + static statusArrayToKeyedObject(offsets: LagStatus[] = []) { const obj = {}; @@ -56,24 +91,28 @@ class Analytics { return obj; } + + abstract run(); } /** * outsourced analytics for nconsumers */ -class ConsumerAnalytics extends Analytics { +export class ConsumerAnalytics extends Analytics { + + _lastRes: ConsumerRunResult | null = null; + + client: JSConsumer; /** * creates a new instance - * @param {NConsumer} nconsumer + * @param {NConsumer|NProducer} client * @param {object} config * @param {object} logger */ - constructor(nconsumer, config, logger) { - super(nconsumer, config, logger); - - this._lastRes = null; - this._consumedCount = 0; + constructor(client: JSConsumer, config: AnalyticsConfig | null = null, logger: KafkaLogger) { + super(config, logger); + this.client = client; // consumer or producer. } /** @@ -81,11 +120,11 @@ class ConsumerAnalytics extends Analytics { * @private * @returns {Promise.} */ - async _checkLagChanges() { + async _checkLagChanges(): Promise { - const last = this.client._lastLagStatus; + const last = this.client.getLastLagStatus(); await this.client.getLagStatus(); //await potential refresh - const newest = this.client._lagCache; + const newest = this.client.getLagCache(); if (!last || !newest) { return { @@ -174,13 +213,13 @@ class ConsumerAnalytics extends Analytics { * @private * @returns {object} */ - _identifyLargestLag() { + _identifyLargestLag(): { highDistance?: number, error?: string } { let lag = { highDistance: -1 }; - const newest = this.client._lagCache; + const newest = this.client.getLagCache(); if (!newest) { return { @@ -203,7 +242,7 @@ class ConsumerAnalytics extends Analytics { * @param {object} stats - getStats() client result * @returns {number} */ - _consumed(stats) { + _consumed(stats: ConsumerStats): number { const diff = (stats.totalIncoming || 0) - this._consumedCount; this._consumedCount = stats.totalIncoming || 0; return diff; @@ -214,25 +253,29 @@ class ConsumerAnalytics extends Analytics { * called in interval * @returns {object} */ - async run() { + async run(): Promise { const res = { generatedAt: Date.now(), - interval: this.config.analyticsInterval + interval: (this.config) ? this.config.analyticsInterval : defaultAnalyticsInterval, + lagChange: {}, + largestLag: {}, + consumed: 0, + errors: 0, }; try { res.lagChange = await this._checkLagChanges(); } catch (error) { this.logger.error(`Failed to calculate lag changes ${error.message}.`); - res.lagChange = null; + // res.lagChange = null; } try { res.largestLag = this._identifyLargestLag(); } catch (error) { this.logger.error(`Failed to calculate largest lag ${error.message}.`); - res.largestLag = null; + // res.largestLag = null; } const stats = this.client.getStats(); @@ -241,26 +284,26 @@ class ConsumerAnalytics extends Analytics { res.consumed = this._consumed(stats); } catch (error) { this.logger.error(`Failed to get consumed count ${error.message}.`); - res.consumed = null; + // res.consumed = null; } try { res.errors = this._errorsInInterval(stats); } catch (error) { this.logger.error(`Failed to get error count ${error.message}.`); - res.errors = null; + // res.errors = null; } - this.logger.debug(res); - this._lastRes = res; - return res; + this.logger.debug(JSON.stringify(res)); + this._lastRes = res as ConsumerRunResult; + return res as ConsumerRunResult; } /** * returns the last result of run() * @returns {object} */ - getLastResult() { + getLastResult(): ConsumerRunResult | null { return this._lastRes; } } @@ -268,19 +311,20 @@ class ConsumerAnalytics extends Analytics { /** * outsourced analytics for nproducers */ -class ProducerAnalytics extends Analytics { +export class ProducerAnalytics extends Analytics { + + _lastRes: ProducerRunResult | null = null; + + client: JSProducer; /** * creates a new instance - * @param {NProducer} nproducer * @param {object} config * @param {object} logger */ - constructor(nproducer, config, logger) { - super(nproducer, config, logger); - - this._lastRes = null; - this._producedCount = 0; + constructor(client: JSProducer, config: AnalyticsConfig | null = null, logger: KafkaLogger) { + super(config, logger); + this.client = client; // consumer or producer. } /** @@ -289,7 +333,7 @@ class ProducerAnalytics extends Analytics { * @param {object} stats - getStats() client result * @returns {number} */ - _produced(stats) { + _produced(stats: ProducerStats): number { const diff = (stats.totalPublished || 0) - this._producedCount; this._producedCount = stats.totalPublished || 0; return diff; @@ -299,11 +343,13 @@ class ProducerAnalytics extends Analytics { * called in interval * @returns {object} */ - async run() { + async run(): Promise { - const res = { + const res: ProducerRunResult = { generatedAt: Date.now(), - interval: this.config.analyticsInterval + interval: (this.config) ? this.config.analyticsInterval : defaultAnalyticsInterval, + produced: 0, + errors: null, }; const stats = this.client.getStats(); @@ -322,7 +368,7 @@ class ProducerAnalytics extends Analytics { res.errors = null; } - this.logger.debug(res); + this.logger.debug(JSON.stringify(res)); this._lastRes = res; return res; } @@ -331,12 +377,7 @@ class ProducerAnalytics extends Analytics { * returns the last result of run() * @returns {object} */ - getLastResult() { + getLastResult(): ProducerRunResult | null { return this._lastRes; } } - -module.exports = { - ConsumerAnalytics, - ProducerAnalytics -}; diff --git a/lib/shared/CompressionTypes.js b/src/lib/shared/CompressionTypes.ts similarity index 71% rename from lib/shared/CompressionTypes.js rename to src/lib/shared/CompressionTypes.ts index 4fbd6c8..a061319 100644 --- a/lib/shared/CompressionTypes.js +++ b/src/lib/shared/CompressionTypes.ts @@ -4,6 +4,10 @@ const TYPES = [0, 1, 2]; class CompressionTypes { + public NONE: number; + public GZIP: number; + public SNAPPY: number; + constructor() { this.NONE = 0; this.GZIP = 1; @@ -20,4 +24,4 @@ class CompressionTypes { } } -module.exports = new CompressionTypes(); +export default new CompressionTypes(); diff --git a/lib/shared/Health.js b/src/lib/shared/Health.ts similarity index 77% rename from lib/shared/Health.js rename to src/lib/shared/Health.ts index 370ad74..bf73a08 100644 --- a/lib/shared/Health.js +++ b/src/lib/shared/Health.ts @@ -1,6 +1,6 @@ -"use strict"; - -const merge = require("lodash.merge"); +import merge from "lodash.merge"; +import { KafkaHealthConfig } from "../interfaces"; +import { JSProducer, JSConsumer } from "../kafkajs"; const defaultConfig = { thresholds: { @@ -17,7 +17,7 @@ const defaultConfig = { } }; -const STATES = { +export const STATES = { DIS_ANALYTICS: -4, NO_ANALYTICS: -3, UNKNOWN: -2, @@ -33,20 +33,24 @@ const MESSAGES = { NO_ANALYTICS: "Analytics have not yet run, checks will be available after first run.", UNKNOWN: "State is unknown.", UNCONNECTED: "The client is not connected.", - HEALTHY: "No problems detected, client is healthy." + HEALTHY: "No problems detected, client is healthy.", + ERRORS: "There was an error." }; /** * little pojso class around the check object */ -class Check { +export class Check { + + status: number; + messages: string[]; /** * creates a new instance * @param {number} status - status code * @param {Array|string} message - message/s, pass an empty array to initialise clean */ - constructor(status = STATES.HEALTHY, message = MESSAGES.HEALTHY) { + constructor(status = STATES.HEALTHY, message: string | string[] = MESSAGES.HEALTHY) { this.status = status; this.messages = Array.isArray(message) ? message : [message]; } @@ -56,7 +60,7 @@ class Check { * @param {number} status - new status code * @returns {boolean} */ - changeStatus(status = STATES.UNKNOWN) { + changeStatus(status: number = STATES.UNKNOWN): boolean { if (status > this.status) { this.status = status; @@ -71,7 +75,7 @@ class Check { * @param {string} message - string message to attach * @returns {number} */ - add(message = MESSAGES.UNKNOWN) { + add(message: string = MESSAGES.UNKNOWN): number { return this.messages.push(message); } } @@ -79,20 +83,19 @@ class Check { /** * health parent class */ -class Health { +abstract class Health { + + config: KafkaHealthConfig; + abstract client; + STATES = STATES; + MESSAGES = MESSAGES /** * creates a new instance - * @param {NConsumer|NProducer} client + * @param {config} config */ - constructor(client, config) { - this.client = client; - + constructor(config?: KafkaHealthConfig) { this.config = merge({}, defaultConfig, config); - - //make them accessable - this.STATES = STATES; - this.MESSAGES = MESSAGES; } /** @@ -100,7 +103,7 @@ class Health { * @param {number} status * @param {Array|string} message */ - createCheck(status, message) { + createCheck(status: number, message: string | string[]): Check { return new Check(status, message); } } @@ -109,14 +112,18 @@ class Health { * health check adapted for NConsumers * @extends Health */ -class ConsumerHealth extends Health { +export class ConsumerHealth extends Health { + + client: JSConsumer; /** * creates a new instance * @param {NConsumer} nconsumer + * @param {config} config optional */ - constructor(nconsumer, config) { - super(nconsumer, config); + constructor(nconsumer: JSConsumer, config?: KafkaHealthConfig) { + super(config); + this.client = nconsumer; } /** @@ -124,7 +131,7 @@ class ConsumerHealth extends Health { * @async * @returns {Promise.} */ - async check() { + async check(): Promise { /* ### preparation ### */ @@ -138,7 +145,7 @@ class ConsumerHealth extends Health { const analytics = this.client._analytics.getLastResult(); - if (!analytics) { + if (!analytics || Object.keys(analytics).length === 0) { return super.createCheck(STATES.NO_ANALYTICS, MESSAGES.NO_ANALYTICS); } @@ -182,14 +189,18 @@ class ConsumerHealth extends Health { * health check adapted for NProducers * @extends Health */ -class ProducerHealth extends Health { +export class ProducerHealth extends Health { + + client: JSProducer; /** * creates a new instance * @param {NProducer} nproducer + * @param {config} config */ - constructor(nproducer, config) { - super(nproducer, config); + constructor(nproducer: JSProducer, config?: KafkaHealthConfig) { + super(config); + this.client = nproducer; } /** @@ -197,7 +208,7 @@ class ProducerHealth extends Health { * @async * @returns {Promise.} */ - async check() { + async check(): Promise { /* ### preparation ### */ @@ -211,13 +222,13 @@ class ProducerHealth extends Health { const analytics = this.client._analytics.getLastResult(); - if (!analytics) { + if (!analytics || Object.keys(analytics).length === 0) { return super.createCheck(STATES.NO_ANALYTICS, MESSAGES.NO_ANALYTICS); } /* ### eof preparation ### */ - const check = new Check(STATES.HEALTHY, []); + const check = new Check(STATES.HEALTHY); if (analytics.errors !== null && analytics.errors >= this.config.thresholds.producer.errors) { check.changeStatus(STATES.CRITICAL); @@ -237,8 +248,3 @@ class ProducerHealth extends Health { return check; } } - -module.exports = { - ConsumerHealth, - ProducerHealth -}; diff --git a/lib/shared/Metadata.js b/src/lib/shared/Metadata.ts similarity index 58% rename from lib/shared/Metadata.js rename to src/lib/shared/Metadata.ts index cd86aaa..c68060a 100644 --- a/lib/shared/Metadata.js +++ b/src/lib/shared/Metadata.ts @@ -1,15 +1,34 @@ -"use strict"; +import { ITopicMetadata, PartitionMetadata } from "kafkajs"; + +// Defined from https://docs.confluent.io/current/kafka-rest/api.html#partitions +type PartitionRestFormat = { + partition: number; + leader: number; + replicas: PartitionReplicaRestFormat[] +} + +type PartitionReplicaRestFormat = { + broker: number; + leader: boolean; + in_sync: boolean; +} + +type TopicMetaData = { + topics: ITopicMetadata[] +} /** * wrapper arround node-librdkafka metadata object */ -class Metadata { +export class Metadata { + + raw: TopicMetaData = {topics: []}; /** * creates a new instance * @param {object} raw - metadata object response of node-librdkafka client */ - constructor(raw) { + constructor(raw: TopicMetaData) { this.raw = raw; } @@ -19,9 +38,9 @@ class Metadata { * @param {string} topicName - name of the kafka topic * @returns {number} */ - getPartitionCountOfTopic(topicName) { + getPartitionCountOfTopic(topicName: string): number { - const topic = this.raw.topics.filter(topic => topic.name === topicName)[0]; + const topic = this.raw.topics.filter(topic => topic.name === topicName).pop(); if (!topic) { throw new Error(topicName + " does not exist in fetched metadata."); @@ -36,25 +55,25 @@ class Metadata { * @param {string} topicName - name of the kafka topic * @returns {Array} */ - getPartitionsForTopic(topicName) { + getPartitionsForTopic(topicName: string): number[] { - const topic = this.raw.topics.filter(topic => topic.name === topicName)[0]; + const topic = this.raw.topics.filter((topic:ITopicMetadata) => topic.name === topicName).pop(); if (!topic) { throw new Error(topicName + " does not exist in fetched metadata."); } - return topic.partitions.map((partition) => partition.id); + return topic.partitions.map((partition) => partition.partitionId); } /** * @throws * returns a list of topic names */ - asTopicList() { - return this.raw.topics.map(topic => topic.name).filter(topic => { - return topic !== "__consumer_offsets"; - }); + asTopicList(): string[] { + return this.raw.topics + .filter((topic:ITopicMetadata) => topic.name !== "__consumer_offsets") + .map((topic: ITopicMetadata) => topic.name); } /** @@ -63,13 +82,13 @@ class Metadata { * @param {string} topicName - name of the kafka topic * @returns {object} */ - asTopicDescription(topicName) { + asTopicDescription(topicName: string): Record { if (!this.raw.topics || !this.raw.topics.length) { return {}; } - let topic = null; + let topic; for (let i = 0; i < this.raw.topics.length; i++) { if (this.raw.topics[i].name === topicName) { topic = this.raw.topics[i]; @@ -80,7 +99,7 @@ class Metadata { if (!topic) { return {}; } - + return { name: topic.name, configs: null, @@ -94,13 +113,13 @@ class Metadata { * @param {string} topicName - name of the kafka topic * @returns {Array} */ - asTopicPartitions(topicName) { + asTopicPartitions(topicName: string): PartitionRestFormat[] { if (!this.raw.topics || !this.raw.topics.length) { - return {}; + return []; } - let topic = null; + let topic: ITopicMetadata | null = null; for (let i = 0; i < this.raw.topics.length; i++) { if (this.raw.topics[i].name === topicName) { topic = this.raw.topics[i]; @@ -109,20 +128,21 @@ class Metadata { } if (!topic) { - return {}; + return []; } return Metadata.formatPartitions(topic.partitions); } /** + * @deprecated * @throws * gets a broker object (list of broker ids) * @returns {object} */ - asBrokers() { + asBrokers(): Record { return { - brokers: this.raw.brokers.map(broker => broker.id) + brokers: [] }; } @@ -132,15 +152,15 @@ class Metadata { * @param {Array} partitions - array of partitions * @returns {Array} */ - static formatPartitions(partitions) { - return partitions.map((p) => { - p.partition = p.id; - p.replicas = p.replicas.map((r) => ({ broker: r, in_sync: p.isrs.indexOf(r) !== -1, leader: r === p.leader })); - delete p.id; - delete p.isrs; - return p; - }); + static formatPartitions(partitions: PartitionMetadata[]): PartitionRestFormat[] { + return partitions.map((p) => ({ + partition: p.partitionId, + leader: p.leader, + replicas: p.replicas.map((r) => ({ + broker: r, + in_sync: p.isr.indexOf(r) !== -1, + leader: r === p.leader + })), + })); } } - -module.exports = Metadata; diff --git a/src/lib/shared/index.ts b/src/lib/shared/index.ts new file mode 100644 index 0000000..149d791 --- /dev/null +++ b/src/lib/shared/index.ts @@ -0,0 +1,6 @@ +import CompressionTypes from "./CompressionTypes"; + +export { CompressionTypes }; +export * from "./Health"; +export * from "./Analytics"; +export * from "./Metadata"; diff --git a/test/config.js b/test/config.js deleted file mode 100644 index b77dbb3..0000000 --- a/test/config.js +++ /dev/null @@ -1,88 +0,0 @@ -"use strict"; - -const config = { - /*logger: { - debug: console.log, - info: console.log, - warn: console.log, - error: console.log - }, */ - options: { - pollIntervalMs: 100, - consumeGraceMs: 22, - murmurHashVersion: "2", - }, -}; - -const producerConfig = Object.assign({}, config, { - noptions: { - "metadata.broker.list": "localhost:9092", - "client.id": "n-test-producer", - "compression.codec": "none", - "socket.keepalive.enable": true, - "queue.buffering.max.ms": 100, - "batch.num.messages": 5, - }, - tconf: { - "request.required.acks": 1, - }, -}); - -const consumerConfig = Object.assign({}, config, { - noptions: { - "metadata.broker.list": "localhost:9092", - "client.id": "n-test-consumer", - "group.id": "n-test-group", - "enable.auto.commit": false, - "socket.keepalive.enable": true, - "socket.blocking.max.ms": 5, - }, - tconf: { - "auto.offset.reset": "earliest", - }, -}); - -const jsProducerConfig = Object.assign({}, config, { - noptions: { - "metadata.broker.list": "localhost:9092", - "client.id": "n-test-produce-js", - "compression.codec": "none", - "socket.keepalive.enable": true, - "queue.buffering.max.ms": 100, - "batch.num.messages": 5, - }, - tconf: { - "request.required.acks": 1, - }, -}); - -const jsConsumerConfig = Object.assign({}, config, { - noptions: { - "metadata.broker.list": "localhost:9092", - "client.id": "n-test-consumer-js", - "group.id": "n-test-group-js", - "enable.auto.commit": false, - "socket.keepalive.enable": true, - "socket.blocking.max.ms": 5, - }, - tconf: { - "auto.offset.reset": "earliest", - }, -}); - -const topic = "n-test-topic"; - -const batchOptions = { - batchSize: 1000, - commitEveryNBatch: 1, - manualBatching: true, -}; - -module.exports = { - topic, - producerConfig, - consumerConfig, - batchOptions, - jsProducerConfig, - jsConsumerConfig, -}; diff --git a/test/config.ts b/test/config.ts new file mode 100644 index 0000000..ae9b6a9 --- /dev/null +++ b/test/config.ts @@ -0,0 +1,44 @@ +import { JSKafkaProducerConfig, JSKafkaConsumerConfig } from "../src/lib/interfaces"; +import { CompressionTypes } from "kafkajs"; + +export const jsProducerConfig: JSKafkaProducerConfig = { + noptions: { + "metadata.broker.list": "localhost:9092", + "client.id": "n-test-produce-js", + "compression.codec": CompressionTypes.None, + "socket.keepalive.enable": true, + "queue.buffering.max.ms": 100, + "batch.num.messages": 5, + "log_level": 0, + }, + options: { + murmurHashVersion: "2", + }, + tconf: { + "request.required.acks": 1, + "message.timeout.ms": 1000, + }, +}; + +export const jsConsumerConfig: JSKafkaConsumerConfig = { + noptions: { + "metadata.broker.list": "localhost:9092", + "client.id": "n-test-consumer-js", + "group.id": "n-test-group-js", + "enable.auto.commit": false, + "socket.keepalive.enable": true, + "socket.blocking.max.ms": 5, + }, + options: {}, + tconf: { + "auto.offset.reset": "earliest", + }, +}; + +export const topic = "n-test-topic"; + +export const batchOptions = { + batchSize: 1000, + commitEveryNBatch: 1, + manualBatching: true, +}; diff --git a/test/int/Health.test.js b/test/int/Health.test.ts similarity index 80% rename from test/int/Health.test.js rename to test/int/Health.test.ts index 94cd140..0057605 100644 --- a/test/int/Health.test.js +++ b/test/int/Health.test.ts @@ -1,14 +1,5 @@ -"use strict"; - -const assert = require("assert"); - -const { - Health -} = require("./../../index.js"); -const { - ConsumerHealth, - ProducerHealth -} = Health; +import assert from "assert"; +import { ConsumerHealth, ProducerHealth, STATES } from "../../src/index"; describe("Health UNIT", () => { @@ -101,9 +92,9 @@ describe("Health UNIT", () => { ph.check(), ch.check() ]).then(res => { - assert.equal(res[0].status, 3); - assert.equal(res[0].messages.length, 2); - assert.equal(res[1].status, 3); + assert.equal(res[0].status, STATES.CRITICAL); + assert.equal(res[0].messages.length, 3); + assert.equal(res[1].status, STATES.CRITICAL); assert.equal(res[1].messages.length, 2); }); }); @@ -117,8 +108,8 @@ describe("Health UNIT", () => { ph.check(), ch.check() ]).then(res => { - assert.equal(res[0].status, 1); - assert.equal(res[1].status, 1); + assert.equal(res[0].status, STATES.RISK); + assert.equal(res[1].status, STATES.RISK); }); }); @@ -129,22 +120,22 @@ describe("Health UNIT", () => { return Promise.all([ ch.check() ]).then(res => { - assert.equal(res[0].status, 2); + assert.equal(res[0].status, STATES.WARNING); assert.equal(res[0].messages.length, 1); }); }); it("should be no analytics", () => { - const ph = getPHI(getFakeProducer(null)); - const ch = getCHI(getFakeConsumer(null)); + const ph = getPHI(getFakeProducer()); + const ch = getCHI(getFakeConsumer()); return Promise.all([ ph.check(), ch.check() ]).then(res => { - assert.equal(res[0].status, -3); - assert.equal(res[1].status, -3); + assert.equal(res[0].status, STATES.NO_ANALYTICS); + assert.equal(res[1].status, STATES.NO_ANALYTICS); }); }); @@ -161,8 +152,8 @@ describe("Health UNIT", () => { ph.check(), ch.check() ]).then(res => { - assert.equal(res[0].status, -4); - assert.equal(res[1].status, -4); + assert.equal(res[0].status, STATES.DIS_ANALYTICS); + assert.equal(res[1].status, STATES.DIS_ANALYTICS); }); }); @@ -179,8 +170,8 @@ describe("Health UNIT", () => { ph.check(), ch.check() ]).then(res => { - assert.equal(res[0].status, -1); - assert.equal(res[1].status, -1); + assert.equal(res[0].status, STATES.UNCONNECTED); + assert.equal(res[1].status, STATES.UNCONNECTED); }); }); diff --git a/test/int/JSSinek.test.js b/test/int/JSSinek.test.js deleted file mode 100644 index f197aaa..0000000 --- a/test/int/JSSinek.test.js +++ /dev/null @@ -1,102 +0,0 @@ -"use strict"; - -const assert = require("assert"); -const { JSConsumer, JSProducer } = require("./../../index.js"); -const { jsProducerConfig, jsConsumerConfig, topic } = require("../config"); - -describe("Javascript Client INT", () => { - - let consumer = null; - let producer = null; - const consumedMessages = []; - let firstMessageReceived = false; - let messagesChecker; - - before(done => { - - producer = new JSProducer(jsProducerConfig); - consumer = new JSConsumer([topic], jsConsumerConfig); - - producer.on("error", error => console.error(error)); - consumer.on("error", error => console.error(error)); - - Promise.all([ - producer.connect(), - consumer.connect(false) - ]).then(() => { - consumer.consume(async (messages, callback) => { - messages.forEach((message) => { - if(!firstMessageReceived){ - firstMessageReceived = true; - } - consumedMessages.push(message); - }) - callback(); - }, true, false, { - batchSize: 1000, - commitEveryNBatch: 1, - manualBatching: true, - }); - // consumer.consume(); - // consumer.on("message", message => { - // consumedMessages.push(message); - // if(!firstMessageReceived){ - // firstMessageReceived = true; - // } - // }); - setTimeout(done, 1000); - }); - }); - - after(done => { - if (producer && consumer) { - - try { - producer.close(); - consumer.close(); //commit - } catch (error) { - console.error(error); - } - - setTimeout(done, 500); - } - }); - - it("should be able to produce messages", () => { - - const promises = [ - producer.send(topic, "a message"), - producer.bufferFormatPublish(topic, "1", { content: "a message 1" }, 1, null, 0), - producer.bufferFormatUpdate(topic, "2", { content: "a message 2" }, 1, null, 0), - producer.bufferFormatUnpublish(topic, "3", { content: "a message 3" }, 1, null, 0), - producer.send(topic, new Buffer("a message buffer")) - ]; - - return Promise.all(promises); - }); - - it("should be able to wait", done => { - messagesChecker = setInterval(() => { - if (consumedMessages.length >= 5) { - clearInterval(messagesChecker); - done(); - } - }, 500); - }); - - it("should have received first message", done => { - assert.ok(firstMessageReceived); - done(); - }); - - it("should be able to consume messages", done => { - assert.ok(consumedMessages.length); - assert.ok(!Buffer.isBuffer(consumedMessages[0].value)); - assert.equal(consumedMessages[0].value, "a message"); - assert.equal(JSON.parse(consumedMessages[1].value).payload.content, "a message 1"); - assert.equal(JSON.parse(consumedMessages[2].value).payload.content, "a message 2"); - assert.equal(JSON.parse(consumedMessages[3].value).payload.content, "a message 3"); - assert.equal(consumedMessages[4].value, "a message buffer"); - done(); - }); -}); diff --git a/test/int/JSSinek.test.ts b/test/int/JSSinek.test.ts new file mode 100644 index 0000000..93709e2 --- /dev/null +++ b/test/int/JSSinek.test.ts @@ -0,0 +1,104 @@ +import assert from "assert"; +import { JSConsumer, JSProducer } from '../../src' +import { jsProducerConfig, jsConsumerConfig, topic } from '../config'; + +describe("Javascript Client INT", () => { + + let consumer: JSConsumer; + let producer: JSProducer; + const consumedMessages: any[] = []; + let firstMessageReceived = false; + let messagesChecker; + + before(done => { + + try { + + producer = new JSProducer(jsProducerConfig); + consumer = new JSConsumer(topic, jsConsumerConfig); + + producer.on("error", error => console.error(error)); + consumer.on("error", error => console.error(error)); + + Promise.all([ + producer.connect(), + consumer.connect(false) + ]).then(() => { + consumer.consume(async (messages, callback) => { + messages.forEach((message) => { + if(!firstMessageReceived){ + firstMessageReceived = true; + } + consumedMessages.push(message); + }); + callback(); + }, true, false, { + batchSize: 1000, + commitEveryNBatch: 1, + manualBatching: true, + }); + setTimeout(done, 1000); + }); + } catch (e) { + console.log(e); + } + + }); + + after(done => { + if (producer && consumer) { + + try { + producer.close(); + consumer.close(); //commit + } catch (error) { + console.error(error); + } + + setTimeout(done, 500); + } + }); + + it("should be able to produce messages", async () => { + + // Change to await instead of promise.all in order to ensure the message order + // for tests further down. + try { + await producer.send(topic, "a message"); + await producer.bufferFormatPublish(topic, "1", { content: "a message 1" }, 1, null, null, 0); + await producer.bufferFormatUpdate(topic, "2", { content: "a message 2" }, 1, null, null, 0); + await producer.bufferFormatUnpublish(topic, "3", { content: "a message 3" }, 1, null, null, 0); + await producer.send(topic, new Buffer("a message buffer")); + + return true; + } catch(e) { + console.error(e) + return false; + } + }); + + it("should be able to wait", done => { + messagesChecker = setInterval(() => { + if (consumedMessages.length >= 5) { + clearInterval(messagesChecker); + done(); + } + }, 500); + }); + + it("should have received first message", done => { + assert.ok(firstMessageReceived); + done(); + }); + + it("should be able to consume messages", done => { + assert.ok(consumedMessages.length); + assert.ok(!Buffer.isBuffer(consumedMessages[0].value)); + assert.equal(consumedMessages[0].value, "a message"); + assert.equal(JSON.parse(consumedMessages[1].value).payload.content, "a message 1"); + assert.equal(JSON.parse(consumedMessages[2].value).payload.content, "a message 2"); + assert.equal(JSON.parse(consumedMessages[3].value).payload.content, "a message 3"); + assert.equal(consumedMessages[4].value, "a message buffer"); + done(); + }); +}); diff --git a/test/int/NSinek.test.js b/test/int/NSinek.test.js deleted file mode 100644 index 46ec95d..0000000 --- a/test/int/NSinek.test.js +++ /dev/null @@ -1,127 +0,0 @@ -"use strict"; - -const assert = require("assert"); -const { NConsumer, NProducer } = require("./../../index.js"); -const { producerConfig, consumerConfig, topic, batchOptions } = require("../config"); - -describe("Native Client INT", () => { - - let consumer = null; - let producer = null; - const consumedMessages = []; - let firstMessageReceived = false; - let messagesChecker; - let offsets = []; - - before(done => { - - producer = new NProducer(producerConfig, null, 1); - consumer = new NConsumer([topic], consumerConfig); - - producer.on("error", error => console.error(error)); - consumer.on("error", error => console.error(error)); - - Promise.all([ - producer.connect(), - consumer.connect(), - ]).then(() => { - - consumer.consume((messages, callback) => { - - messages.forEach((message) => { - consumedMessages.push(message); - if(!firstMessageReceived){ - firstMessageReceived = true; - } - }); - - callback(); - }, true, false, batchOptions); - - setTimeout(done, 100); - }); - }); - - after(done => { - if(producer && consumer){ - producer.close(); - consumer.close(true); //commit - setTimeout(done, 100); - } - }); - - it("should be able to produce messages", () => { - - const promises = [ - producer.send(topic, "a message"), - producer.bufferFormatPublish(topic, "1", {content: "a message 1"}, 1, null, 0), - producer.bufferFormatUpdate(topic, "2", {content: "a message 2"}, 1, null, 0), - producer.bufferFormatUnpublish(topic, "3", {content: "a message 3"}, 1, null, 0), - producer.send(topic, Buffer.from("a message buffer")), - producer.send(topic, "a message with headers", null, null, null, [{ myCustomKey: "myCustomValue" }]), - producer.bufferFormatPublish(topic, "1", {content: "a message with headers 1"}, 1, null, 0, null, [{ myCustomKey: "myCustomValue 1" }]), - producer.bufferFormatUpdate(topic, "2", {content: "a message with headers 2"}, 1, null, 0, null, [{ myCustomKey: "myCustomValue 2" }]), - producer.bufferFormatUnpublish(topic, "3", {content: "a message with headers 3"}, 1, null, 0, null, [{ myCustomKey: "myCustomValue 3" }]), - producer.send(topic, Buffer.from("a message buffer with headers"), null, null, null, [{ myCustomKey: "myCustomValue buffer" }]), - ]; - - return Promise.all(promises).then((produceResults) => { - produceResults.forEach((produceResult) => { - if(produceResult.offset !== null) { - offsets.push(produceResult.offset); - } - }); - assert.ok(offsets.length > 0); - }); - }); - - it("should be able to wait", done => { - messagesChecker = setInterval(()=>{ - if(consumedMessages.length >= 5){ - clearInterval(messagesChecker); - done(); - } - }, 100); - }); - - it("should have received first message", done => { - assert.ok(firstMessageReceived); - producer.getTopicList().then((topics) => { - assert.ok(topics.length); - done(); - }); - }); - - it("should be able to consume messages", done => { - // console.log(consumedMessages); - assert.ok(consumedMessages.length >= 10); - assert.ok(!Buffer.isBuffer(consumedMessages[0].value)); - assert.ok(consumedMessages[1].key, "1"); - assert.ok(consumedMessages[2].key, "2"); - assert.ok(consumedMessages[3].key, "3"); - assert.equal(consumedMessages[0].value, "a message"); - assert.equal(JSON.parse(consumedMessages[1].value).payload.content, "a message 1"); - assert.equal(JSON.parse(consumedMessages[2].value).payload.content, "a message 2"); - assert.equal(JSON.parse(consumedMessages[3].value).payload.content, "a message 3"); - assert.equal(consumedMessages[4].value, "a message buffer"); - assert.equal(consumedMessages[5].headers[0].myCustomKey, "myCustomValue"); - assert.equal(consumedMessages[6].headers[0].myCustomKey, "myCustomValue 1"); - assert.equal(consumedMessages[7].headers[0].myCustomKey, "myCustomValue 2"); - assert.equal(consumedMessages[8].headers[0].myCustomKey, "myCustomValue 3"); - assert.equal(consumedMessages[9].headers[0].myCustomKey, "myCustomValue buffer"); - - done(); - }); - - it("should see produced offset in consumed messages", done => { - const anOffset = offsets[0]; - let seen = false; - consumedMessages.forEach((message) => { - if(message.offset === anOffset){ - seen = true; - } - }); - assert.ok(seen); - done(); - }); -}); diff --git a/tsconfig.dist.json b/tsconfig.dist.json new file mode 100644 index 0000000..6289d4f --- /dev/null +++ b/tsconfig.dist.json @@ -0,0 +1,14 @@ +{ + "extends": "./tsconfig", + "compilerOptions": { + "rootDir": "src/", + "noEmit": false, + "outDir": "dist", + "declaration": true, + "declarationMap": true, + "sourceMap": true + }, + "include": [ + "src" + ] +} \ No newline at end of file diff --git a/tsconfig.json b/tsconfig.json new file mode 100644 index 0000000..6ca5bc1 --- /dev/null +++ b/tsconfig.json @@ -0,0 +1,21 @@ +{ + "compilerOptions": { + "module": "commonjs", + "esModuleInterop": true, + "target": "es2016", + "sourceMap": true, + "noEmit": true, + + "noImplicitReturns": true, + "noFallthroughCasesInSwitch": true, + "noUnusedParameters": true, + "noUnusedLocals": true, + + "strict": true, + + "noImplicitAny": false, + "strictBindCallApply": true, + "strictNullChecks": true + }, + "include": ["./src/**/*"] +} \ No newline at end of file diff --git a/yarn.lock b/yarn.lock index baebaad..af4cc28 100644 --- a/yarn.lock +++ b/yarn.lock @@ -64,6 +64,86 @@ resolved "https://registry.yarnpkg.com/@types/color-name/-/color-name-1.1.1.tgz#1c1261bbeaa10a8055bbc5d8ab84b7b2afc846a0" integrity sha512-rr+OQyAjxze7GgWrSaJwydHStIhHq2lvY3BOC2Mj7KnzI7XK0Uw1TOOdI9lDoajEbSWLiYgoo4f1R51erQfhPQ== +"@types/eslint-visitor-keys@^1.0.0": + version "1.0.0" + resolved "https://registry.yarnpkg.com/@types/eslint-visitor-keys/-/eslint-visitor-keys-1.0.0.tgz#1ee30d79544ca84d68d4b3cdb0af4f205663dd2d" + integrity sha512-OCutwjDZ4aFS6PB1UZ988C4YgwlBHJd6wCeQqaLdmadZ/7e+w79+hbMUFC1QXDNCmdyoRfAFdm0RypzwR+Qpag== + +"@types/json-schema@^7.0.3": + version "7.0.5" + resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.5.tgz#dcce4430e64b443ba8945f0290fb564ad5bac6dd" + integrity sha512-7+2BITlgjgDhH0vvwZU/HZJVyk+2XUlvxXe8dFMedNX/aMkaOq++rMAFXc0tM7ij15QaWlbdQASBR9dihi+bDQ== + +"@types/mocha@^8.0.1": + version "8.0.1" + resolved "https://registry.yarnpkg.com/@types/mocha/-/mocha-8.0.1.tgz#813b4ff8dd9920d652e1e6b451ff1e371a4561d2" + integrity sha512-TBZ6YdX7IZz4U9/mBoB8zCMRN1vXw8QdihRcZxD3I0Cv/r8XF8RggZ8WiXFws4aj5atzRR5hJrYer7g8nXwpnQ== + +"@types/node@^14.0.27": + version "14.0.27" + resolved "https://registry.yarnpkg.com/@types/node/-/node-14.0.27.tgz#a151873af5a5e851b51b3b065c9e63390a9e0eb1" + integrity sha512-kVrqXhbclHNHGu9ztnAwSncIgJv/FaxmzXJvGXNdcCpV1b8u1/Mi6z6m0vwy0LzKeXFTPLH0NzwmoJ3fNCIq0g== + +"@typescript-eslint/eslint-plugin@^3.9.0": + version "3.9.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/eslint-plugin/-/eslint-plugin-3.9.0.tgz#0fe529b33d63c9a94f7503ca2bb12c84b9477ff3" + integrity sha512-UD6b4p0/hSe1xdTvRCENSx7iQ+KR6ourlZFfYuPC7FlXEzdHuLPrEmuxZ23b2zW96KJX9Z3w05GE/wNOiEzrVg== + dependencies: + "@typescript-eslint/experimental-utils" "3.9.0" + debug "^4.1.1" + functional-red-black-tree "^1.0.1" + regexpp "^3.0.0" + semver "^7.3.2" + tsutils "^3.17.1" + +"@typescript-eslint/experimental-utils@3.9.0": + version "3.9.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/experimental-utils/-/experimental-utils-3.9.0.tgz#3171d8ddba0bf02a8c2034188593630914fcf5ee" + integrity sha512-/vSHUDYizSOhrOJdjYxPNGfb4a3ibO8zd4nUKo/QBFOmxosT3cVUV7KIg8Dwi6TXlr667G7YPqFK9+VSZOorNA== + dependencies: + "@types/json-schema" "^7.0.3" + "@typescript-eslint/types" "3.9.0" + "@typescript-eslint/typescript-estree" "3.9.0" + eslint-scope "^5.0.0" + eslint-utils "^2.0.0" + +"@typescript-eslint/parser@^3.9.0": + version "3.9.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/parser/-/parser-3.9.0.tgz#344978a265d9a5c7c8f13e62c78172a4374dabea" + integrity sha512-rDHOKb6uW2jZkHQniUQVZkixQrfsZGUCNWWbKWep4A5hGhN5dLHMUCNAWnC4tXRlHedXkTDptIpxs6e4Pz8UfA== + dependencies: + "@types/eslint-visitor-keys" "^1.0.0" + "@typescript-eslint/experimental-utils" "3.9.0" + "@typescript-eslint/types" "3.9.0" + "@typescript-eslint/typescript-estree" "3.9.0" + eslint-visitor-keys "^1.1.0" + +"@typescript-eslint/types@3.9.0": + version "3.9.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-3.9.0.tgz#be9d0aa451e1bf3ce99f2e6920659e5b2e6bfe18" + integrity sha512-rb6LDr+dk9RVVXO/NJE8dT1pGlso3voNdEIN8ugm4CWM5w5GimbThCMiMl4da1t5u3YwPWEwOnKAULCZgBtBHg== + +"@typescript-eslint/typescript-estree@3.9.0": + version "3.9.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-3.9.0.tgz#c6abbb50fa0d715cab46fef67ca6378bf2eaca13" + integrity sha512-N+158NKgN4rOmWVfvKOMoMFV5n8XxAliaKkArm/sOypzQ0bUL8MSnOEBW3VFIeffb/K5ce/cAV0yYhR7U4ALAA== + dependencies: + "@typescript-eslint/types" "3.9.0" + "@typescript-eslint/visitor-keys" "3.9.0" + debug "^4.1.1" + glob "^7.1.6" + is-glob "^4.0.1" + lodash "^4.17.15" + semver "^7.3.2" + tsutils "^3.17.1" + +"@typescript-eslint/visitor-keys@3.9.0": + version "3.9.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-3.9.0.tgz#44de8e1b1df67adaf3b94d6b60b80f8faebc8dd3" + integrity sha512-O1qeoGqDbu0EZUC/MZ6F1WHTIzcBVhGqDj3LhTnj65WUA548RXVxUHbYhAW9bZWfb2rnX9QsbbP5nmeJ5Z4+ng== + dependencies: + eslint-visitor-keys "^1.1.0" + abbrev@1: version "1.1.1" resolved "https://registry.yarnpkg.com/abbrev/-/abbrev-1.1.1.tgz#f8f2c887ad10bf67f634f005b6987fed3179aac8" @@ -120,11 +200,6 @@ ansi-escapes@^4.2.1: dependencies: type-fest "^0.11.0" -ansi-regex@^2.0.0: - version "2.1.1" - resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-2.1.1.tgz#c3b33ab5ee360d86e0e628f0468ae7ef27d654df" - integrity sha1-w7M6te42DYbg5ijwRorn7yfWVN8= - ansi-regex@^3.0.0: version "3.0.0" resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-3.0.0.tgz#ed0317c322064f79466c02966bddb605ab37d998" @@ -163,18 +238,10 @@ anymatch@~3.1.1: normalize-path "^3.0.0" picomatch "^2.0.4" -aproba@^1.0.3: - version "1.2.0" - resolved "https://registry.yarnpkg.com/aproba/-/aproba-1.2.0.tgz#6802e6264efd18c790a1b0d517f0f2627bf2c94a" - integrity sha512-Y9J6ZjXtoYh8RnXVCMOU/ttDmk1aBjunq9vO0ta5x85WDQiQfUF9sIPBITdbiiIVcBo03Hi3jMxigBtsddlXRw== - -are-we-there-yet@~1.1.2: - version "1.1.5" - resolved "https://registry.yarnpkg.com/are-we-there-yet/-/are-we-there-yet-1.1.5.tgz#4b35c2944f062a8bfcda66410760350fe9ddfc21" - integrity sha512-5hYdAkZlcG8tOLujVDTgCT+uPX0VnpAH28gWsLfzpXYm7wP6mp5Q/gYyR7YQ0cKVJcXJnl3j2kpBan13PtQf6w== - dependencies: - delegates "^1.0.0" - readable-stream "^2.0.6" +arg@^4.1.0: + version "4.1.3" + resolved "https://registry.yarnpkg.com/arg/-/arg-4.1.3.tgz#269fc7ad5b8e42cb63c896d5666017261c144089" + integrity sha512-58S9QDqG0Xx27YwPSt9fJxivjYl432YCwfDMfZ+71RAqUrZef7LrKQZ3LHLOwCS4FLNBplP533Zx895SeOCHvA== argparse@^1.0.7: version "1.0.10" @@ -198,13 +265,6 @@ async@1.x: resolved "https://registry.yarnpkg.com/async/-/async-1.5.2.tgz#ec6a61ae56480c0c3cb241c95618e20892f9672a" integrity sha1-7GphrlZIDAw8skHJVhjiCJL5Zyo= -async@^2.6.2: - version "2.6.3" - resolved "https://registry.yarnpkg.com/async/-/async-2.6.3.tgz#d72625e2344a3656e3a3ad4fa749fa83299d82ff" - integrity sha512-zflvls11DCy+dQWzTW2dzuilv8Z5X/pjfmZOWba6TNIVDm+2UDaJmXSOXlasHKfNBs8oo3M0aT50fDEWfKZjXg== - dependencies: - lodash "^4.17.14" - async@~3.2.0: version "3.2.0" resolved "https://registry.yarnpkg.com/async/-/async-3.2.0.tgz#b3a2685c5ebb641d3de02d161002c60fc9f85720" @@ -215,48 +275,11 @@ balanced-match@^1.0.0: resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.0.tgz#89b4d199ab2bee49de164ea02b89ce462d71b767" integrity sha1-ibTRmasr7kneFk6gK4nORi1xt2c= -base64-js@^1.0.2: - version "1.3.1" - resolved "https://registry.yarnpkg.com/base64-js/-/base64-js-1.3.1.tgz#58ece8cb75dd07e71ed08c736abc5fac4dbf8df1" - integrity sha512-mLQ4i2QO1ytvGWFWmcngKO//JXAQueZvwEKtjgQFM4jIK0kU+ytMfplL8j+n5mspOfjHwoAg+9yhb7BwAHm36g== - binary-extensions@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/binary-extensions/-/binary-extensions-2.0.0.tgz#23c0df14f6a88077f5f986c0d167ec03c3d5537c" integrity sha512-Phlt0plgpIIBOGTT/ehfFnbNlfsDEiqmzE2KRXoX1bLIlir4X/MR+zSyBEkL05ffWgnRSf/DXv+WrUAVr93/ow== -binary@~0.3.0: - version "0.3.0" - resolved "https://registry.yarnpkg.com/binary/-/binary-0.3.0.tgz#9f60553bc5ce8c3386f3b553cff47462adecaa79" - integrity sha1-n2BVO8XOjDOG87VTz/R0Yq3sqnk= - dependencies: - buffers "~0.1.1" - chainsaw "~0.1.0" - -bindings@^1.3.1: - version "1.5.0" - resolved "https://registry.yarnpkg.com/bindings/-/bindings-1.5.0.tgz#10353c9e945334bc0511a6d90b38fbc7c9c504df" - integrity sha512-p2q/t/mhvuOj/UeLlV6566GD/guowlr0hHxClI0W9m7MWYkL1F0hLo+0Aexs9HSPCtR1SXQ0TD3MMKrXZajbiQ== - dependencies: - file-uri-to-path "1.0.0" - -bl@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/bl/-/bl-2.2.0.tgz#e1a574cdf528e4053019bb800b041c0ac88da493" - integrity sha512-wbgvOpqopSr7uq6fJrLH8EsvYMJf9gzfo2jCsL2eTy75qXPukA4pCgHamOQkZtY5vmfVtjB+P3LNlMHW5CEZXA== - dependencies: - readable-stream "^2.3.5" - safe-buffer "^5.1.1" - -bl@^4.0.1: - version "4.0.2" - resolved "https://registry.yarnpkg.com/bl/-/bl-4.0.2.tgz#52b71e9088515d0606d9dd9cc7aa48dc1f98e73a" - integrity sha512-j4OH8f6Qg2bGuWfRiltT2HYGx0e1QcBTrK9KAHNMwMZdQnDZFk0ZSYIpADjYCB3U12nicC5tVJwSIhwOWjb4RQ== - dependencies: - buffer "^5.5.0" - inherits "^2.0.4" - readable-stream "^3.4.0" - bluebird@~3.7.2: version "3.7.2" resolved "https://registry.yarnpkg.com/bluebird/-/bluebird-3.7.2.tgz#9f229c15be272454ffa973ace0dbee79a1b0c36f" @@ -298,30 +321,10 @@ browser-stdout@1.3.1: resolved "https://registry.yarnpkg.com/browser-stdout/-/browser-stdout-1.3.1.tgz#baa559ee14ced73452229bad7326467c61fabd60" integrity sha512-qhAVI1+Av2X7qelOfAIYwXONood6XlZE/fXaBSmW/T5SzLAmCgzi+eiWE7fUvbHaeNBQH13UftjpXxsfLkMpgw== -buffer-crc32@~0.2.5: - version "0.2.13" - resolved "https://registry.yarnpkg.com/buffer-crc32/-/buffer-crc32-0.2.13.tgz#0d333e3f00eac50aa1454abd30ef8c2a5d9a7242" - integrity sha1-DTM+PwDqxQqhRUq9MO+MKl2ackI= - -buffer@^5.5.0: - version "5.6.0" - resolved "https://registry.yarnpkg.com/buffer/-/buffer-5.6.0.tgz#a31749dc7d81d84db08abf937b6b8c4033f62786" - integrity sha512-/gDYp/UtU0eA1ys8bOs9J6a+E/KWIY+DZ+Q2WESNUA0jFRsJOc0SNUO6xJ5SGA1xueg3NL65W6s+NY5l9cunuw== - dependencies: - base64-js "^1.0.2" - ieee754 "^1.1.4" - -buffermaker@~1.2.0: - version "1.2.1" - resolved "https://registry.yarnpkg.com/buffermaker/-/buffermaker-1.2.1.tgz#0631f92b891a84b750f1036491ac857c734429f4" - integrity sha512-IdnyU2jDHU65U63JuVQNTHiWjPRH0CS3aYd/WPaEwyX84rFdukhOduAVb1jwUScmb5X0JWPw8NZOrhoLMiyAHQ== - dependencies: - long "1.1.2" - -buffers@~0.1.1: - version "0.1.1" - resolved "https://registry.yarnpkg.com/buffers/-/buffers-0.1.1.tgz#b24579c3bed4d6d396aeee6d9a8ae7f5482ab7bb" - integrity sha1-skV5w77U1tOWru5tmorn9Ugqt7s= +buffer-from@^1.0.0: + version "1.1.1" + resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.1.tgz#32713bc028f75c02fdb710d7c7bcec1f2c6070ef" + integrity sha512-MQcXEUbCKtEo7bhqEs6560Hyd4XaovZlO/k9V3hjVUF/zwW7KBVdSK4gIt/bzwS9MbR5qob+F5jusZsb0YQK2A== bytes@3.1.0: version "3.1.0" @@ -338,13 +341,6 @@ camelcase@^5.0.0: resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320" integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg== -chainsaw@~0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/chainsaw/-/chainsaw-0.1.0.tgz#5eab50b28afe58074d0d58291388828b5e5fbc98" - integrity sha1-XqtQsor+WAdNDVgpE4iCi15fvJg= - dependencies: - traverse ">=0.3.0 <0.4" - chalk@^2.0.0, chalk@^2.1.0, chalk@^2.4.2: version "2.4.2" resolved "https://registry.yarnpkg.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424" @@ -382,11 +378,6 @@ chokidar@3.3.0: optionalDependencies: fsevents "~2.1.1" -chownr@^1.1.1: - version "1.1.4" - resolved "https://registry.yarnpkg.com/chownr/-/chownr-1.1.4.tgz#6fc9d7b42d32a583596337666e7d08084da2cc6b" - integrity sha512-jJ0bqzaylmJtVnNgzTeSOs8DPavpbYgEr/b0YL8/2GO3xJEhInFmhKMUnEJQjZumK7KXGFhUy89PrsJWlakBVg== - cli-cursor@^3.1.0: version "3.1.0" resolved "https://registry.yarnpkg.com/cli-cursor/-/cli-cursor-3.1.0.tgz#264305a7ae490d1d03bf0c9ba7c925d1753af307" @@ -408,11 +399,6 @@ cliui@^5.0.0: strip-ansi "^5.2.0" wrap-ansi "^5.1.0" -code-point-at@^1.0.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/code-point-at/-/code-point-at-1.1.0.tgz#0d070b4d043a5bea33a2f1a40e2edb3d9a4ccf77" - integrity sha1-DQcLTQQ6W+ozovGkDi7bPZpMz3c= - color-convert@^1.9.0: version "1.9.3" resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" @@ -447,11 +433,6 @@ concat-map@0.0.1: resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b" integrity sha1-2Klr13/Wjfd5OnMDajug1UBdR3s= -console-control-strings@^1.0.0, console-control-strings@~1.1.0: - version "1.1.0" - resolved "https://registry.yarnpkg.com/console-control-strings/-/console-control-strings-1.1.0.tgz#3d7cf4464db6446ea644bf4b39507f9851008e8e" - integrity sha1-PXz0Rk22RG6mRL9LOVB/mFEAjo4= - content-disposition@0.5.3: version "0.5.3" resolved "https://registry.yarnpkg.com/content-disposition/-/content-disposition-0.5.3.tgz#e130caf7e7279087c5616c2007d0485698984fbd" @@ -474,11 +455,6 @@ cookie@0.4.0: resolved "https://registry.yarnpkg.com/cookie/-/cookie-0.4.0.tgz#beb437e7022b3b6d49019d088665303ebe9c14ba" integrity sha512-+Hp8fLp57wnUSt0tY0tHEXh4voZRDnoIrZPqlo3DPiI4y9lwg/jqx+1Om94/W6ZaPDOUbnjOt/99w66zk+l1Xg== -core-util-is@~1.0.0: - version "1.0.2" - resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.2.tgz#b5fd54220aa2bc5ab57aab7140c940754503c1a7" - integrity sha1-tf1UIgqivFq1eqtxQMlAdUUDwac= - cross-spawn@^6.0.5: version "6.0.5" resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-6.0.5.tgz#4a5ec7c64dfae22c3a14124dbacdee846d80cbc4" @@ -490,7 +466,7 @@ cross-spawn@^6.0.5: shebang-command "^1.2.0" which "^1.2.9" -debug@2.6.9, debug@^2.1.3: +debug@2.6.9: version "2.6.9" resolved "https://registry.yarnpkg.com/debug/-/debug-2.6.9.tgz#5d128515df134ff327e90a4c93f4e077a536341f" integrity sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA== @@ -504,7 +480,7 @@ debug@3.2.6: dependencies: ms "^2.1.1" -debug@^4.0.1, debug@~4.1.1: +debug@^4.0.1, debug@^4.1.1, debug@~4.1.1: version "4.1.1" resolved "https://registry.yarnpkg.com/debug/-/debug-4.1.1.tgz#3b72260255109c6b589cee050f1d516139664791" integrity sha512-pYAIzeRo8J6KPEaJ0VWOh5Pzkbw/RetuzehGM7QRRX5he4fPHx2rdKMB256ehJCkX+XRQm16eZLqLNS8RSZXZw== @@ -516,18 +492,6 @@ decamelize@^1.2.0: resolved "https://registry.yarnpkg.com/decamelize/-/decamelize-1.2.0.tgz#f6534d15148269b20352e7bee26f501f9a191290" integrity sha1-9lNNFRSCabIDUue+4m9QH5oZEpA= -decompress-response@^4.2.0: - version "4.2.1" - resolved "https://registry.yarnpkg.com/decompress-response/-/decompress-response-4.2.1.tgz#414023cc7a302da25ce2ec82d0d5238ccafd8986" - integrity sha512-jOSne2qbyE+/r8G1VU+G/82LBs2Fs4LAsTiLSHOCOMZQl2OKZ6i8i4IyHemTe+/yIXOtTcRQMzPcgyhoFlqPkw== - dependencies: - mimic-response "^2.0.0" - -deep-extend@^0.6.0: - version "0.6.0" - resolved "https://registry.yarnpkg.com/deep-extend/-/deep-extend-0.6.0.tgz#c4fa7c95404a17a9c3e8ca7e1537312b736330ac" - integrity sha512-LOHxIOaPYdHlJRtCQfDIVZtfw/ufM8+rVj649RIHzcm/vGwQRXFt6OPqIFWsm2XEMrNIEtWR64sY1LEKD2vAOA== - deep-is@~0.1.3: version "0.1.3" resolved "https://registry.yarnpkg.com/deep-is/-/deep-is-0.1.3.tgz#b369d6fb5dbc13eecf524f91b070feedc357cf34" @@ -540,16 +504,6 @@ define-properties@^1.1.2, define-properties@^1.1.3: dependencies: object-keys "^1.0.12" -delegates@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/delegates/-/delegates-1.0.0.tgz#84c6e159b81904fdca59a0ef44cd870d31250f9a" - integrity sha1-hMbhWbgZBP3KWaDvRM2HDTElD5o= - -denque@^1.3.0: - version "1.4.1" - resolved "https://registry.yarnpkg.com/denque/-/denque-1.4.1.tgz#6744ff7641c148c3f8a69c307e51235c1f4a37cf" - integrity sha512-OfzPuSZKGcgr96rf1oODnfjqBFmr1DVoc/TrItj3Ohe0Ah1C5WX5Baquw/9U9KovnQ88EqmJbD66rKYUQYN1tQ== - depd@~1.1.2: version "1.1.2" resolved "https://registry.yarnpkg.com/depd/-/depd-1.1.2.tgz#9bcd52e14c097763e749b274c4346ed2e560b5a9" @@ -560,17 +514,12 @@ destroy@~1.0.4: resolved "https://registry.yarnpkg.com/destroy/-/destroy-1.0.4.tgz#978857442c44749e4206613e37946205826abd80" integrity sha1-l4hXRCxEdJ5CBmE+N5RiBYJqvYA= -detect-libc@^1.0.3: - version "1.0.3" - resolved "https://registry.yarnpkg.com/detect-libc/-/detect-libc-1.0.3.tgz#fa137c4bd698edf55cd5cd02ac559f91a4c4ba9b" - integrity sha1-+hN8S9aY7fVc1c0CrFWfkaTEups= - diff@3.5.0: version "3.5.0" resolved "https://registry.yarnpkg.com/diff/-/diff-3.5.0.tgz#800c0dd1e0a8bfbc95835c202ad220fe317e5a12" integrity sha512-A46qtFgd+g7pDZinpnwiRJtxbC1hpgf0uzP3iG89scHk0AUC7A1TGxf5OiiOUv/JMZR8GOt8hL900hV0bOy5xA== -diff@^4.0.2: +diff@^4.0.1, diff@^4.0.2: version "4.0.2" resolved "https://registry.yarnpkg.com/diff/-/diff-4.0.2.tgz#60f3aecb89d5fae520c11aa19efc2bb982aade7d" integrity sha512-58lmxKSA4BNyLz+HHMUzlOEpg09FV+ev6ZMe3vJihgdxzgcwZ8VoEEPmALCZG9LmqfVoNMMKpttIYTVG6uDY7A== @@ -602,13 +551,6 @@ encodeurl@~1.0.2: resolved "https://registry.yarnpkg.com/encodeurl/-/encodeurl-1.0.2.tgz#ad3ff4c86ec2d029322f5a02c3a9a606c95b3f59" integrity sha1-rT/0yG7C0CkyL1oCw6mmBslbP1k= -end-of-stream@^1.1.0, end-of-stream@^1.4.1: - version "1.4.4" - resolved "https://registry.yarnpkg.com/end-of-stream/-/end-of-stream-1.4.4.tgz#5ae64a5f45057baf3626ec14da0ca5e4b2431eb0" - integrity sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q== - dependencies: - once "^1.4.0" - es-abstract@^1.17.0-next.1, es-abstract@^1.17.5: version "1.17.5" resolved "https://registry.yarnpkg.com/es-abstract/-/es-abstract-1.17.5.tgz#d8c9d1d66c8981fb9200e2251d799eee92774ae9" @@ -672,6 +614,13 @@ eslint-utils@^1.4.3: dependencies: eslint-visitor-keys "^1.1.0" +eslint-utils@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/eslint-utils/-/eslint-utils-2.1.0.tgz#d2de5e03424e707dc10c74068ddedae708741b27" + integrity sha512-w94dQYoauyvlDc43XnGB8lU3Zt713vNChgt4EWwhXAP2XkBvndfxF0AgIqKOOasjPIPzj9JqgwkwbCYD0/V3Zg== + dependencies: + eslint-visitor-keys "^1.1.0" + eslint-visitor-keys@^1.1.0: version "1.1.0" resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-1.1.0.tgz#e2a82cea84ff246ad6fb57f9bde5b46621459ec2" @@ -778,11 +727,6 @@ etag@~1.8.1: resolved "https://registry.yarnpkg.com/etag/-/etag-1.8.1.tgz#41ae2eeb65efa62268aebfea83ac7d79299b0887" integrity sha1-Qa4u62XvpiJorr/qg6x9eSmbCIc= -expand-template@^2.0.3: - version "2.0.3" - resolved "https://registry.yarnpkg.com/expand-template/-/expand-template-2.0.3.tgz#6e14b3fcee0f3a6340ecb57d2e8918692052a47c" - integrity sha512-XYfuKMvj4O35f/pOXLObndIRvyQ+/+6AhODh+OKWj9S9498pHHn/IMszH+gt0fBCRWMNfk1ZSp5x3AifmnI2vg== - expect.js@~0.3.1: version "0.3.1" resolved "https://registry.yarnpkg.com/expect.js/-/expect.js-0.3.1.tgz#b0a59a0d2eff5437544ebf0ceaa6015841d09b5b" @@ -862,11 +806,6 @@ file-entry-cache@^5.0.1: dependencies: flat-cache "^2.0.1" -file-uri-to-path@1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/file-uri-to-path/-/file-uri-to-path-1.0.0.tgz#553a7b8446ff6f684359c445f1e37a05dacc33dd" - integrity sha512-0Zt+s3L7Vf1biwWZ29aARiVYLx7iMGnEUl9x33fbB/j3jR81u/O2LbqK+Bm1CDSNDKVtJ/YjwY7TUd5SkeLQLw== - fill-range@^7.0.1: version "7.0.1" resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-7.0.1.tgz#1919a6a7c75fe38b2c7c77e5198535da9acdda40" @@ -925,11 +864,6 @@ fresh@0.5.2: resolved "https://registry.yarnpkg.com/fresh/-/fresh-0.5.2.tgz#3d8cadd90d976569fa835ab1f8e4b23a105605a7" integrity sha1-PYyt2Q2XZWn6g1qx+OSyOhBWBac= -fs-constants@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/fs-constants/-/fs-constants-1.0.0.tgz#6be0de9be998ce16af8afc24497b9ee9b7ccd9ad" - integrity sha512-y6OAwoSIf7FyjMIv94u+b5rdheZEjzR63GTyZJm5qh4Bi+2YgwLCcI/fPFZkL5PSixOt6ZNKm+w+Hfp/Bciwow== - fs.realpath@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f" @@ -950,30 +884,11 @@ functional-red-black-tree@^1.0.1: resolved "https://registry.yarnpkg.com/functional-red-black-tree/-/functional-red-black-tree-1.0.1.tgz#1b0ab3bd553b2a0d6399d29c0e3ea0b252078327" integrity sha1-GwqzvVU7Kg1jmdKcDj6gslIHgyc= -gauge@~2.7.3: - version "2.7.4" - resolved "https://registry.yarnpkg.com/gauge/-/gauge-2.7.4.tgz#2c03405c7538c39d7eb37b317022e325fb018bf7" - integrity sha1-LANAXHU4w51+s3sxcCLjJfsBi/c= - dependencies: - aproba "^1.0.3" - console-control-strings "^1.0.0" - has-unicode "^2.0.0" - object-assign "^4.1.0" - signal-exit "^3.0.0" - string-width "^1.0.1" - strip-ansi "^3.0.1" - wide-align "^1.1.0" - get-caller-file@^2.0.1: version "2.0.5" resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e" integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg== -github-from-package@0.0.0: - version "0.0.0" - resolved "https://registry.yarnpkg.com/github-from-package/-/github-from-package-0.0.0.tgz#97fb5d96bfde8973313f20e8288ef9a167fa64ce" - integrity sha1-l/tdlr/eiXMxPyDoKI75oWf6ZM4= - glob-parent@^5.0.0, glob-parent@~5.1.0: version "5.1.1" resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-5.1.1.tgz#b6c1ef417c4e5663ea498f1c45afac6916bbc229" @@ -1004,7 +919,7 @@ glob@^5.0.15: once "^1.3.0" path-is-absolute "^1.0.0" -glob@^7.1.3: +glob@^7.1.3, glob@^7.1.6: version "7.1.6" resolved "https://registry.yarnpkg.com/glob/-/glob-7.1.6.tgz#141f33b81a7c2492e125594307480c46679278a6" integrity sha512-LwaxwyZ72Lk7vZINtNNrywX0ZuLyStrdDtabefZKAY5ZGJhVtgdznluResxNmPitE0SAO+O26sWTHeKSI2wMBA== @@ -1060,11 +975,6 @@ has-symbols@^1.0.0, has-symbols@^1.0.1: resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.1.tgz#9f5214758a44196c406d9bd76cebf81ec2dd31e8" integrity sha512-PLcsoqu++dmEIZB+6totNFKq/7Do+Z0u4oT0zKOJNl3lYK6vGwwu2hjHs+68OEZbTjiUE9bgOABXbP/GvrS0Kg== -has-unicode@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/has-unicode/-/has-unicode-2.0.1.tgz#e0e6fe6a28cf51138855e086d1691e771de2a8b9" - integrity sha1-4Ob+aijPUROIVeCG0Wkedx3iqLk= - has@^1.0.3: version "1.0.3" resolved "https://registry.yarnpkg.com/has/-/has-1.0.3.tgz#722d7cbfc1f6aa8241f16dd814e011e1f41e8796" @@ -1106,11 +1016,6 @@ iconv-lite@0.4.24, iconv-lite@^0.4.24: dependencies: safer-buffer ">= 2.1.2 < 3" -ieee754@^1.1.4: - version "1.1.13" - resolved "https://registry.yarnpkg.com/ieee754/-/ieee754-1.1.13.tgz#ec168558e95aa181fd87d37f55c32bbcb6708b84" - integrity sha512-4vf7I2LYV/HaWerSo3XmlMkp5eZ83i+/CDluXi/IGTs/O1sejBNhTtnxzmRZfvOUqj7lZjqHkeTvpgSFDlWZTg== - ignore@^4.0.6: version "4.0.6" resolved "https://registry.yarnpkg.com/ignore/-/ignore-4.0.6.tgz#750e3db5862087b4737ebac8207ffd1ef27b25fc" @@ -1137,7 +1042,7 @@ inflight@^1.0.4: once "^1.3.0" wrappy "1" -inherits@2, inherits@2.0.4, inherits@^2.0.3, inherits@^2.0.4, inherits@~2.0.3: +inherits@2, inherits@2.0.4: version "2.0.4" resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c" integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ== @@ -1147,11 +1052,6 @@ inherits@2.0.3: resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.3.tgz#633c2c83e3da42a502f52466022480f4208261de" integrity sha1-Yzwsg+PaQqUC9SRmAiSA9CCCYd4= -ini@~1.3.0: - version "1.3.5" - resolved "https://registry.yarnpkg.com/ini/-/ini-1.3.5.tgz#eee25f56db1c9ec6085e0c22778083f596abf927" - integrity sha512-RZY5huIKCMRWDUqZlEi72f/lmXKMvuszcMBduliQ3nnWbx9X/ZBQO7DijMEYS9EhHBb2qacRUMtC7svLwe0lcw== - inquirer@^7.0.0: version "7.1.0" resolved "https://registry.yarnpkg.com/inquirer/-/inquirer-7.1.0.tgz#1298a01859883e17c7264b82870ae1034f92dd29" @@ -1203,13 +1103,6 @@ is-extglob@^2.1.1: resolved "https://registry.yarnpkg.com/is-extglob/-/is-extglob-2.1.1.tgz#a88c02535791f02ed37c76a1b9ea9773c833f8c2" integrity sha1-qIwCU1eR8C7TfHahueqXc8gz+MI= -is-fullwidth-code-point@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-1.0.0.tgz#ef9e31386f031a7f0d643af82fde50c457ef00cb" - integrity sha1-754xOG8DGn8NZDr4L95QxFfvAMs= - dependencies: - number-is-nan "^1.0.0" - is-fullwidth-code-point@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-2.0.0.tgz#a3b30a5c4f199183167aaab93beefae3ddfb654f" @@ -1256,11 +1149,6 @@ isarray@0.0.1: resolved "https://registry.yarnpkg.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf" integrity sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8= -isarray@~1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" - integrity sha1-u5NdSFgsuhaMBoNJV6VKPgcSTxE= - isexe@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" @@ -1314,27 +1202,6 @@ just-extend@^4.0.2: resolved "https://registry.yarnpkg.com/just-extend/-/just-extend-4.1.0.tgz#7278a4027d889601640ee0ce0e5a00b992467da4" integrity sha512-ApcjaOdVTJ7y4r08xI5wIqpvwS48Q0PBG4DJROcEkH1f8MdAiNFyFxz3xoL0LWAVwjrwPYZdVHHxhRHcx/uGLA== -kafka-node@~5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/kafka-node/-/kafka-node-5.0.0.tgz#4b6f65cc1d77ebe565859dfb8f9575ed15d543c0" - integrity sha512-dD2ga5gLcQhsq1yNoQdy1MU4x4z7YnXM5bcG9SdQuiNr5KKuAmXixH1Mggwdah5o7EfholFbcNDPSVA6BIfaug== - dependencies: - async "^2.6.2" - binary "~0.3.0" - bl "^2.2.0" - buffer-crc32 "~0.2.5" - buffermaker "~1.2.0" - debug "^2.1.3" - denque "^1.3.0" - lodash "^4.17.4" - minimatch "^3.0.2" - nested-error-stacks "^2.0.0" - optional "^0.1.3" - retry "^0.10.1" - uuid "^3.0.0" - optionalDependencies: - snappy "^6.0.1" - kafkajs@1.12.0: version "1.12.0" resolved "https://registry.yarnpkg.com/kafkajs/-/kafkajs-1.12.0.tgz#50ad336baee95f3324af8ae8df6fadc96e07c613" @@ -1368,7 +1235,7 @@ lodash.merge@~4.6.2: resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" integrity sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ== -lodash@^4.17.14, lodash@^4.17.15, lodash@^4.17.4: +lodash@^4.17.14, lodash@^4.17.15: version "4.17.15" resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.15.tgz#b447f6670a0455bbfeedd11392eff330ea097548" integrity sha512-8xOcRHvCjnocdS5cpwXQXVzmmh5e5+saE2QGoeQmbKmRS6J3VQppPOIt0MnmE+4xlZoumy0GPG0D0MVIQbNA1A== @@ -1380,16 +1247,16 @@ log-symbols@3.0.0: dependencies: chalk "^2.4.2" -long@1.1.2: - version "1.1.2" - resolved "https://registry.yarnpkg.com/long/-/long-1.1.2.tgz#eaef5951ca7551d96926b82da242db9d6b28fb53" - integrity sha1-6u9ZUcp1UdlpJrgtokLbnWso+1M= - long@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/long/-/long-4.0.0.tgz#9a7b71cfb7d361a194ea555241c92f7468d5bf28" integrity sha512-XsP+KhQif4bjX1kbuSiySJFNAehNxgLb6hPRGJ9QsUr8ajHkuXGdrHmFUTUUXhDwVX2R5bY4JNZEwbUiMhV+MA== +make-error@^1.1.1: + version "1.3.6" + resolved "https://registry.yarnpkg.com/make-error/-/make-error-1.3.6.tgz#2eb2e37ea9b67c4891f684a1394799af484cf7a2" + integrity sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw== + media-typer@0.3.0: version "0.3.0" resolved "https://registry.yarnpkg.com/media-typer/-/media-typer-0.3.0.tgz#8710d7af0aa626f8fffa1ce00168545263255748" @@ -1427,28 +1294,18 @@ mimic-fn@^2.1.0: resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== -mimic-response@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/mimic-response/-/mimic-response-2.1.0.tgz#d13763d35f613d09ec37ebb30bac0469c0ee8f43" - integrity sha512-wXqjST+SLt7R009ySCglWBCFpjUygmCIfD790/kVbiGmUgfYGuB14PiTd5DwVxSV4NcYHjzMkoj5LjQZwTQLEA== - -"minimatch@2 || 3", minimatch@3.0.4, minimatch@^3.0.2, minimatch@^3.0.4: +"minimatch@2 || 3", minimatch@3.0.4, minimatch@^3.0.4: version "3.0.4" resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.0.4.tgz#5166e286457f03306064be5497e8dbb0c3d32083" integrity sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA== dependencies: brace-expansion "^1.1.7" -minimist@^1.2.0, minimist@^1.2.5: +minimist@^1.2.5: version "1.2.5" resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.5.tgz#67d66014b66a6a8aaa0c083c5fd58df4e4e97602" integrity sha512-FM9nNUYrRBAELZQT3xeZQ7fmMOBg6nWNmJKTcgsJeaLstP/UODVpGsr5OhXhhXg6f+qtJ8uiZ+PUxkDWcgIXLw== -mkdirp-classic@^0.5.2: - version "0.5.2" - resolved "https://registry.yarnpkg.com/mkdirp-classic/-/mkdirp-classic-0.5.2.tgz#54c441ce4c96cd7790e10b41a87aa51068ecab2b" - integrity sha512-ejdnDQcR75gwknmMw/tx02AuRs8jCtqFoFqDZMjiNxsu85sRIJVXDKHuLYvUUPRBUtV2FpSZa9bL1BUa3BdR2g== - mkdirp@0.5.3: version "0.5.3" resolved "https://registry.yarnpkg.com/mkdirp/-/mkdirp-0.5.3.tgz#5a514b7179259287952881e94410ec5465659f8c" @@ -1530,16 +1387,6 @@ mute-stream@0.0.8: resolved "https://registry.yarnpkg.com/mute-stream/-/mute-stream-0.0.8.tgz#1630c42b2251ff81e2a283de96a5497ea92e5e0d" integrity sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA== -nan@^2.14.0: - version "2.14.0" - resolved "https://registry.yarnpkg.com/nan/-/nan-2.14.0.tgz#7818f722027b2459a86f0295d434d1fc2336c52c" - integrity sha512-INOFj37C7k3AfaNTtX8RhsTw7qRy7eLET14cROi9+5HAVbbHuIWUHEauBv5qT4Av2tWasiTY1Jw6puUNqRJXQg== - -napi-build-utils@^1.0.1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/napi-build-utils/-/napi-build-utils-1.0.2.tgz#b1fddc0b2c46e380a0b7a76f984dd47c41a13806" - integrity sha512-ONmRUqK7zj7DWX0D9ADe03wbwOBZxNAfF20PlGfCWQcD3+/MakShIHrMqx9YwPTfxDdF1zLeL+RGZiR9kGMLdg== - natural-compare@^1.4.0: version "1.4.0" resolved "https://registry.yarnpkg.com/natural-compare/-/natural-compare-1.4.0.tgz#4abebfeed7541f2c27acfb29bdbbd15c8d5ba4f7" @@ -1555,11 +1402,6 @@ neo-async@^2.6.0: resolved "https://registry.yarnpkg.com/neo-async/-/neo-async-2.6.1.tgz#ac27ada66167fa8849a6addd837f6b189ad2081c" integrity sha512-iyam8fBuCUpWeKPGpaNMetEocMt364qkCsfL9JuhjXX6dRnguRVOfk2GZaDpPjcOKiiXCPINZC1GczQ7iTq3Zw== -nested-error-stacks@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/nested-error-stacks/-/nested-error-stacks-2.1.0.tgz#0fbdcf3e13fe4994781280524f8b96b0cdff9c61" - integrity sha512-AO81vsIO1k1sM4Zrd6Hu7regmJN1NSiAja10gc4bX3F0wd+9rQmcuHQaHVQCYIEC8iFXnE+mavh23GOt7wBgug== - nice-try@^1.0.4: version "1.0.5" resolved "https://registry.yarnpkg.com/nice-try/-/nice-try-1.0.5.tgz#a3378a7696ce7d223e88fc9b764bd7ef1089e366" @@ -1576,13 +1418,6 @@ nise@^4.0.1: just-extend "^4.0.2" path-to-regexp "^1.7.0" -node-abi@^2.7.0: - version "2.15.0" - resolved "https://registry.yarnpkg.com/node-abi/-/node-abi-2.15.0.tgz#51d55cc711bd9e4a24a572ace13b9231945ccb10" - integrity sha512-FeLpTS0F39U7hHZU1srAK4Vx+5AHNVOTP+hxBNQknR/54laTHSFIJkDWDqiquY1LeLUgTfPN7sLPhMubx0PLAg== - dependencies: - semver "^5.4.1" - node-environment-flags@1.0.6: version "1.0.6" resolved "https://registry.yarnpkg.com/node-environment-flags/-/node-environment-flags-1.0.6.tgz#a30ac13621f6f7d674260a54dede048c3982c088" @@ -1591,11 +1426,6 @@ node-environment-flags@1.0.6: object.getownpropertydescriptors "^2.0.3" semver "^5.7.0" -noop-logger@^0.1.1: - version "0.1.1" - resolved "https://registry.yarnpkg.com/noop-logger/-/noop-logger-0.1.1.tgz#94a2b1633c4f1317553007d8966fd0e841b6a4c2" - integrity sha1-lKKxYzxPExdVMAfYlm/Q6EG2pMI= - nopt@3.x: version "3.0.6" resolved "https://registry.yarnpkg.com/nopt/-/nopt-3.0.6.tgz#c6465dbf08abcd4db359317f79ac68a646b28ff9" @@ -1608,26 +1438,6 @@ normalize-path@^3.0.0, normalize-path@~3.0.0: resolved "https://registry.yarnpkg.com/normalize-path/-/normalize-path-3.0.0.tgz#0dcd69ff23a1c9b11fd0978316644a0388216a65" integrity sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA== -npmlog@^4.0.1: - version "4.1.2" - resolved "https://registry.yarnpkg.com/npmlog/-/npmlog-4.1.2.tgz#08a7f2a8bf734604779a9efa4ad5cc717abb954b" - integrity sha512-2uUqazuKlTaSI/dC8AzicUck7+IrEaOnN/e0jd3Xtt1KcGpwx30v50mL7oPyr/h9bL3E4aZccVwpwP+5W9Vjkg== - dependencies: - are-we-there-yet "~1.1.2" - console-control-strings "~1.1.0" - gauge "~2.7.3" - set-blocking "~2.0.0" - -number-is-nan@^1.0.0: - version "1.0.1" - resolved "https://registry.yarnpkg.com/number-is-nan/-/number-is-nan-1.0.1.tgz#097b602b53422a522c1afb8790318336941a011d" - integrity sha1-CXtgK1NCKlIsGvuHkDGDNpQaAR0= - -object-assign@^4.1.0: - version "4.1.1" - resolved "https://registry.yarnpkg.com/object-assign/-/object-assign-4.1.1.tgz#2109adc7965887cfc05cbbd442cac8bfbb360863" - integrity sha1-IQmtx5ZYh8/AXLvUQsrIv7s2CGM= - object-inspect@^1.7.0: version "1.7.0" resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.7.0.tgz#f4f6bd181ad77f006b5ece60bd0b6f398ff74a67" @@ -1663,7 +1473,7 @@ on-finished@~2.3.0: dependencies: ee-first "1.1.1" -once@1.x, once@^1.3.0, once@^1.3.1, once@^1.4.0: +once@1.x, once@^1.3.0: version "1.4.0" resolved "https://registry.yarnpkg.com/once/-/once-1.4.0.tgz#583b1aa775961d4b113ac17d9c50baef9dd76bd1" integrity sha1-WDsap3WWHUsROsF9nFC6753Xa9E= @@ -1682,11 +1492,6 @@ opencollective-postinstall@^2.0.2: resolved "https://registry.yarnpkg.com/opencollective-postinstall/-/opencollective-postinstall-2.0.2.tgz#5657f1bede69b6e33a45939b061eb53d3c6c3a89" integrity sha512-pVOEP16TrAO2/fjej1IdOyupJY8KDUM1CvsaScRbw6oddvpQoOfGk4ywha0HKKVAD6RkW4x6Q+tNBwhf3Bgpuw== -optional@^0.1.3: - version "0.1.4" - resolved "https://registry.yarnpkg.com/optional/-/optional-0.1.4.tgz#cdb1a9bedc737d2025f690ceeb50e049444fd5b3" - integrity sha512-gtvrrCfkE08wKcgXaVwQVgwEQ8vel2dc5DDBn9RLQZ3YtmtkBss6A2HY6BnJH4N/4Ku97Ri/SF8sNWE2225WJw== - optionator@^0.8.1, optionator@^0.8.3: version "0.8.3" resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.8.3.tgz#84fa1d036fe9d3c7e21d99884b601167ec8fb495" @@ -1767,37 +1572,11 @@ picomatch@^2.0.4: resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.2.2.tgz#21f333e9b6b8eaff02468f5146ea406d345f4dad" integrity sha512-q0M/9eZHzmr0AulXyPwNfZjtwZ/RBZlbN3K3CErVrk50T2ASYI7Bye0EvekFY3IP1Nt2DHu0re+V2ZHIpMkuWg== -prebuild-install@^5.2.2: - version "5.3.3" - resolved "https://registry.yarnpkg.com/prebuild-install/-/prebuild-install-5.3.3.tgz#ef4052baac60d465f5ba6bf003c9c1de79b9da8e" - integrity sha512-GV+nsUXuPW2p8Zy7SarF/2W/oiK8bFQgJcncoJ0d7kRpekEA0ftChjfEaF9/Y+QJEc/wFR7RAEa8lYByuUIe2g== - dependencies: - detect-libc "^1.0.3" - expand-template "^2.0.3" - github-from-package "0.0.0" - minimist "^1.2.0" - mkdirp "^0.5.1" - napi-build-utils "^1.0.1" - node-abi "^2.7.0" - noop-logger "^0.1.1" - npmlog "^4.0.1" - pump "^3.0.0" - rc "^1.2.7" - simple-get "^3.0.3" - tar-fs "^2.0.0" - tunnel-agent "^0.6.0" - which-pm-runs "^1.0.0" - prelude-ls@~1.1.2: version "1.1.2" resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.1.2.tgz#21932a549f5e52ffd9a827f570e04be62a97da54" integrity sha1-IZMqVJ9eUv/ZqCf1cOBL5iqX2lQ= -process-nextick-args@~2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/process-nextick-args/-/process-nextick-args-2.0.1.tgz#7820d9b16120cc55ca9ae7792680ae7dba6d7fe2" - integrity sha512-3ouUOpQhtgrbOa17J7+uxOTpITYWaGP7/AhoR3+A+/1e9skrzelGi/dXzEYyvbxubEF6Wn2ypscTKiKJFFn1ag== - progress@^2.0.0: version "2.0.3" resolved "https://registry.yarnpkg.com/progress/-/progress-2.0.3.tgz#7e8cf8d8f5b8f239c1bc68beb4eb78567d572ef8" @@ -1811,14 +1590,6 @@ proxy-addr@~2.0.5: forwarded "~0.1.2" ipaddr.js "1.9.1" -pump@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/pump/-/pump-3.0.0.tgz#b4a2116815bde2f4e1ea602354e8c75565107a64" - integrity sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww== - dependencies: - end-of-stream "^1.1.0" - once "^1.3.1" - punycode@^2.1.0: version "2.1.1" resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.1.1.tgz#b58b010ac40c22c5657616c8d2c2c02c7bf479ec" @@ -1844,38 +1615,6 @@ raw-body@2.4.0: iconv-lite "0.4.24" unpipe "1.0.0" -rc@^1.2.7: - version "1.2.8" - resolved "https://registry.yarnpkg.com/rc/-/rc-1.2.8.tgz#cd924bf5200a075b83c188cd6b9e211b7fc0d3ed" - integrity sha512-y3bGgqKj3QBdxLbLkomlohkvsA8gdAiUQlSBJnBhfn+BPxg4bc62d8TcBW15wavDfgexCgccckhcZvywyQYPOw== - dependencies: - deep-extend "^0.6.0" - ini "~1.3.0" - minimist "^1.2.0" - strip-json-comments "~2.0.1" - -readable-stream@^2.0.6, readable-stream@^2.3.5: - version "2.3.7" - resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-2.3.7.tgz#1eca1cf711aef814c04f62252a36a62f6cb23b57" - integrity sha512-Ebho8K4jIbHAxnuxi7o42OrZgF/ZTNcsZj6nRKyUmkhLFq8CHItp/fy6hQZuZmP/n3yZ9VBUbp4zz/mX8hmYPw== - dependencies: - core-util-is "~1.0.0" - inherits "~2.0.3" - isarray "~1.0.0" - process-nextick-args "~2.0.0" - safe-buffer "~5.1.1" - string_decoder "~1.1.1" - util-deprecate "~1.0.1" - -readable-stream@^3.1.1, readable-stream@^3.4.0: - version "3.6.0" - resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-3.6.0.tgz#337bbda3adc0706bd3e024426a286d4b4b2c9198" - integrity sha512-BViHy7LKeTz4oNnkcLJ+lVSL6vpiFeX6/d3oSH8zCW7UxP2onchk+vTGB143xuFjHS3deTgkKoXXymXqymiIdA== - dependencies: - inherits "^2.0.3" - string_decoder "^1.1.1" - util-deprecate "^1.0.1" - readdirp@~3.2.0: version "3.2.0" resolved "https://registry.yarnpkg.com/readdirp/-/readdirp-3.2.0.tgz#c30c33352b12c96dfb4b895421a49fd5a9593839" @@ -1888,6 +1627,11 @@ regexpp@^2.0.1: resolved "https://registry.yarnpkg.com/regexpp/-/regexpp-2.0.1.tgz#8d19d31cf632482b589049f8281f93dbcba4d07f" integrity sha512-lv0M6+TkDVniA3aD1Eg0DVpfU/booSu7Eev3TDO/mZKHBfVjgCGTV4t4buppESEYDtkArYFOxTJWv6S5C+iaNw== +regexpp@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/regexpp/-/regexpp-3.1.0.tgz#206d0ad0a5648cffbdb8ae46438f3dc51c9f78e2" + integrity sha512-ZOIzd8yVsQQA7j8GCSlPGXwg5PfmA1mrq0JP4nGhh54LaKN3xdai/vHUDu74pKwV8OxseMS65u2NImosQcSD0Q== + require-directory@^2.1.1: version "2.1.1" resolved "https://registry.yarnpkg.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" @@ -1916,11 +1660,6 @@ restore-cursor@^3.1.0: onetime "^5.1.0" signal-exit "^3.0.2" -retry@^0.10.1: - version "0.10.1" - resolved "https://registry.yarnpkg.com/retry/-/retry-0.10.1.tgz#e76388d217992c252750241d3d3956fed98d8ff4" - integrity sha1-52OI0heZLCUnUCQdPTlW/tmNj/Q= - rimraf@2.6.3: version "2.6.3" resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-2.6.3.tgz#b2d104fe0d8fb27cf9e0a1cda8262dd3833c6cab" @@ -1942,22 +1681,17 @@ rxjs@^6.5.3: dependencies: tslib "^1.9.0" -safe-buffer@5.1.2, safe-buffer@~5.1.0, safe-buffer@~5.1.1: +safe-buffer@5.1.2: version "5.1.2" resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.1.2.tgz#991ec69d296e0313747d59bdfd2b745c35f8828d" integrity sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g== -safe-buffer@^5.0.1, safe-buffer@^5.1.1, safe-buffer@~5.2.0: - version "5.2.0" - resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.2.0.tgz#b74daec49b1148f88c64b68d49b1e815c1f2f519" - integrity sha512-fZEwUGbVl7kouZs1jCdMLdt95hdIv0ZeHg6L7qPeciMZhZ+/gdesW4wgTARkrFWEpspjEATAzUGPG8N2jJiwbg== - "safer-buffer@>= 2.1.2 < 3": version "2.1.2" resolved "https://registry.yarnpkg.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a" integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg== -semver@^5.4.1, semver@^5.5.0, semver@^5.7.0: +semver@^5.5.0, semver@^5.7.0: version "5.7.1" resolved "https://registry.yarnpkg.com/semver/-/semver-5.7.1.tgz#a954f931aeba508d307bbf069eff0c01c96116f7" integrity sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ== @@ -1967,6 +1701,11 @@ semver@^6.1.2: resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.0.tgz#ee0a64c8af5e8ceea67687b133761e1becbd1d3d" integrity sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw== +semver@^7.3.2: + version "7.3.2" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.2.tgz#604962b052b81ed0786aae84389ffba70ffd3938" + integrity sha512-OrOb32TeeambH6UrhtShmF7CRDqhL6/5XpPNp2DuRH6+9QLw/orhp72j87v8Qa1ScDkvrrBNpZcDejAirJmfXQ== + send@0.17.1: version "0.17.1" resolved "https://registry.yarnpkg.com/send/-/send-0.17.1.tgz#c1d8b059f7900f7466dd4938bdc44e11ddb376c8" @@ -1996,7 +1735,7 @@ serve-static@1.14.1: parseurl "~1.3.3" send "0.17.1" -set-blocking@^2.0.0, set-blocking@~2.0.0: +set-blocking@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/set-blocking/-/set-blocking-2.0.0.tgz#045f9782d011ae9a6803ddd382b24392b3d890f7" integrity sha1-BF+XgtARrppoA93TgrJDkrPYkPc= @@ -2018,25 +1757,11 @@ shebang-regex@^1.0.0: resolved "https://registry.yarnpkg.com/shebang-regex/-/shebang-regex-1.0.0.tgz#da42f49740c0b42db2ca9728571cb190c98efea3" integrity sha1-2kL0l0DAtC2yypcoVxyxkMmO/qM= -signal-exit@^3.0.0, signal-exit@^3.0.2: +signal-exit@^3.0.2: version "3.0.3" resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.3.tgz#a1410c2edd8f077b08b4e253c8eacfcaf057461c" integrity sha512-VUJ49FC8U1OxwZLxIbTTrDvLnf/6TDgxZcK8wxR8zs13xpx7xbG60ndBlhNrFi2EMuFRoeDoJO7wthSLq42EjA== -simple-concat@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/simple-concat/-/simple-concat-1.0.0.tgz#7344cbb8b6e26fb27d66b2fc86f9f6d5997521c6" - integrity sha1-c0TLuLbib7J9ZrL8hvn21Zl1IcY= - -simple-get@^3.0.3: - version "3.1.0" - resolved "https://registry.yarnpkg.com/simple-get/-/simple-get-3.1.0.tgz#b45be062435e50d159540b576202ceec40b9c6b3" - integrity sha512-bCR6cP+aTdScaQCnQKbPKtJOKDp/hj9EDLJo3Nw4y1QksqaovlW/bnptB6/c1e+qmNIDHRK+oXFDdEqBT8WzUA== - dependencies: - decompress-response "^4.2.0" - once "^1.3.1" - simple-concat "^1.0.0" - sinon@~9.0.2: version "9.0.2" resolved "https://registry.yarnpkg.com/sinon/-/sinon-9.0.2.tgz#b9017e24633f4b1c98dfb6e784a5f0509f5fd85d" @@ -2059,16 +1784,15 @@ slice-ansi@^2.1.0: astral-regex "^1.0.0" is-fullwidth-code-point "^2.0.0" -snappy@^6.0.1: - version "6.2.3" - resolved "https://registry.yarnpkg.com/snappy/-/snappy-6.2.3.tgz#11690a8fcf0c71ff6b04e72ab32631804b2d265a" - integrity sha512-HZpVoIxMfQ4fL3iDuMdI1R5xycw1o9YDCAndTKZCY/EHRoKFvzwplttuBBVGeEg2fd1hYiwAXos/sM24W7N1LA== +source-map-support@^0.5.17: + version "0.5.19" + resolved "https://registry.yarnpkg.com/source-map-support/-/source-map-support-0.5.19.tgz#a98b62f86dcaf4f67399648c085291ab9e8fed61" + integrity sha512-Wonm7zOCIJzBGQdB+thsPar0kYuCIzYvxZwlBa87yi/Mdjv7Tip2cyVbLj5o0cFPN4EVkuTwb3GDDyUx2DGnGw== dependencies: - bindings "^1.3.1" - nan "^2.14.0" - prebuild-install "^5.2.2" + buffer-from "^1.0.0" + source-map "^0.6.0" -source-map@^0.6.1: +source-map@^0.6.0, source-map@^0.6.1: version "0.6.1" resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== @@ -2090,15 +1814,6 @@ sprintf-js@~1.0.2: resolved "https://registry.yarnpkg.com/statuses/-/statuses-1.5.0.tgz#161c7dac177659fd9811f43771fa99381478628c" integrity sha1-Fhx9rBd2Wf2YEfQ3cfqZOBR4Yow= -string-width@^1.0.1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/string-width/-/string-width-1.0.2.tgz#118bdf5b8cdc51a2a7e70d211e07e2b0b9b107d3" - integrity sha1-EYvfW4zcUaKn5w0hHgfisLmxB9M= - dependencies: - code-point-at "^1.0.0" - is-fullwidth-code-point "^1.0.0" - strip-ansi "^3.0.0" - "string-width@^1.0.2 || 2": version "2.1.1" resolved "https://registry.yarnpkg.com/string-width/-/string-width-2.1.1.tgz#ab93f27a8dc13d28cac815c462143a6d9012ae9e" @@ -2159,27 +1874,6 @@ string.prototype.trimstart@^1.0.0: define-properties "^1.1.3" es-abstract "^1.17.5" -string_decoder@^1.1.1: - version "1.3.0" - resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-1.3.0.tgz#42f114594a46cf1a8e30b0a84f56c78c3edac21e" - integrity sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA== - dependencies: - safe-buffer "~5.2.0" - -string_decoder@~1.1.1: - version "1.1.1" - resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-1.1.1.tgz#9cf1611ba62685d7030ae9e4ba34149c3af03fc8" - integrity sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg== - dependencies: - safe-buffer "~5.1.0" - -strip-ansi@^3.0.0, strip-ansi@^3.0.1: - version "3.0.1" - resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-3.0.1.tgz#6a385fb8853d952d5ff05d0e8aaf94278dc63dcf" - integrity sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8= - dependencies: - ansi-regex "^2.0.0" - strip-ansi@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-4.0.0.tgz#a8479022eb1ac368a871389b635262c505ee368f" @@ -2201,7 +1895,7 @@ strip-ansi@^6.0.0: dependencies: ansi-regex "^5.0.0" -strip-json-comments@2.0.1, strip-json-comments@~2.0.1: +strip-json-comments@2.0.1: version "2.0.1" resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-2.0.1.tgz#3c531942e908c2697c0ec344858c286c7ca0a60a" integrity sha1-PFMZQukIwml8DsNEhYwobHygpgo= @@ -2249,27 +1943,6 @@ table@^5.2.3: slice-ansi "^2.1.0" string-width "^3.0.0" -tar-fs@^2.0.0: - version "2.0.1" - resolved "https://registry.yarnpkg.com/tar-fs/-/tar-fs-2.0.1.tgz#e44086c1c60d31a4f0cf893b1c4e155dabfae9e2" - integrity sha512-6tzWDMeroL87uF/+lin46k+Q+46rAJ0SyPGz7OW7wTgblI273hsBqk2C1j0/xNadNLKDTUL9BukSjB7cwgmlPA== - dependencies: - chownr "^1.1.1" - mkdirp-classic "^0.5.2" - pump "^3.0.0" - tar-stream "^2.0.0" - -tar-stream@^2.0.0: - version "2.1.2" - resolved "https://registry.yarnpkg.com/tar-stream/-/tar-stream-2.1.2.tgz#6d5ef1a7e5783a95ff70b69b97455a5968dc1325" - integrity sha512-UaF6FoJ32WqALZGOIAApXx+OdxhekNMChu6axLJR85zMMjXKWFGjbIRe+J6P4UnRGg9rAwWvbTT0oI7hD/Un7Q== - dependencies: - bl "^4.0.1" - end-of-stream "^1.4.1" - fs-constants "^1.0.0" - inherits "^2.0.3" - readable-stream "^3.1.1" - text-table@^0.2.0: version "0.2.0" resolved "https://registry.yarnpkg.com/text-table/-/text-table-0.2.0.tgz#7f5ee823ae805207c00af2df4a84ec3fcfa570b4" @@ -2299,22 +1972,33 @@ toidentifier@1.0.0: resolved "https://registry.yarnpkg.com/toidentifier/-/toidentifier-1.0.0.tgz#7e1be3470f1e77948bc43d94a3c8f4d7752ba553" integrity sha512-yaOH/Pk/VEhBWWTlhI+qXxDFXlejDGcQipMlyxda9nthulaxLZUNcUqFxokp0vcYnvteJln5FNQDRrxj3YcbVw== -"traverse@>=0.3.0 <0.4": - version "0.3.9" - resolved "https://registry.yarnpkg.com/traverse/-/traverse-0.3.9.tgz#717b8f220cc0bb7b44e40514c22b2e8bbc70d8b9" - integrity sha1-cXuPIgzAu3tE5AUUwisui7xw2Lk= +ts-node@^8.10.2: + version "8.10.2" + resolved "https://registry.yarnpkg.com/ts-node/-/ts-node-8.10.2.tgz#eee03764633b1234ddd37f8db9ec10b75ec7fb8d" + integrity sha512-ISJJGgkIpDdBhWVu3jufsWpK3Rzo7bdiIXJjQc0ynKxVOVcg2oIrf2H2cejminGrptVc6q6/uynAHNCuWGbpVA== + dependencies: + arg "^4.1.0" + diff "^4.0.1" + make-error "^1.1.1" + source-map-support "^0.5.17" + yn "3.1.1" + +tslib@^1.8.1: + version "1.13.0" + resolved "https://registry.yarnpkg.com/tslib/-/tslib-1.13.0.tgz#c881e13cc7015894ed914862d276436fa9a47043" + integrity sha512-i/6DQjL8Xf3be4K/E6Wgpekn5Qasl1usyw++dAA35Ue5orEn65VIxOA+YvNNl9HV3qv70T7CNwjODHZrLwvd1Q== tslib@^1.9.0: version "1.11.1" resolved "https://registry.yarnpkg.com/tslib/-/tslib-1.11.1.tgz#eb15d128827fbee2841549e171f45ed338ac7e35" integrity sha512-aZW88SY8kQbU7gpV19lN24LtXh/yD4ZZg6qieAJDDg+YBsJcSmLGK9QpnUjAKVG/xefmvJGd1WUmfpT/g6AJGA== -tunnel-agent@^0.6.0: - version "0.6.0" - resolved "https://registry.yarnpkg.com/tunnel-agent/-/tunnel-agent-0.6.0.tgz#27a5dea06b36b04a0a9966774b290868f0fc40fd" - integrity sha1-J6XeoGs2sEoKmWZ3SykIaPD8QP0= +tsutils@^3.17.1: + version "3.17.1" + resolved "https://registry.yarnpkg.com/tsutils/-/tsutils-3.17.1.tgz#ed719917f11ca0dee586272b2ac49e015a2dd759" + integrity sha512-kzeQ5B8H3w60nFY2g8cJIuH7JDpsALXySGtwGJ0p2LSjLgay3NdIpqq5SoOBe46bKDW2iq25irHCr8wjomUS2g== dependencies: - safe-buffer "^5.0.1" + tslib "^1.8.1" type-check@~0.3.2: version "0.3.2" @@ -2346,6 +2030,11 @@ type-is@~1.6.17, type-is@~1.6.18: media-typer "0.3.0" mime-types "~2.1.24" +typescript@^3.9.7: + version "3.9.7" + resolved "https://registry.yarnpkg.com/typescript/-/typescript-3.9.7.tgz#98d600a5ebdc38f40cb277522f12dc800e9e25fa" + integrity sha512-BLbiRkiBzAwsjut4x/dsibSTB6yWpwT5qWmC2OfuCg3GgVQCSgMs4vEctYPhsaGtd0AeuuHMkjZ2h2WG8MSzRw== + uglify-js@^3.1.4: version "3.9.1" resolved "https://registry.yarnpkg.com/uglify-js/-/uglify-js-3.9.1.tgz#a56a71c8caa2d36b5556cc1fd57df01ae3491539" @@ -2365,21 +2054,11 @@ uri-js@^4.2.2: dependencies: punycode "^2.1.0" -util-deprecate@^1.0.1, util-deprecate@~1.0.1: - version "1.0.2" - resolved "https://registry.yarnpkg.com/util-deprecate/-/util-deprecate-1.0.2.tgz#450d4dc9fa70de732762fbd2d4a28981419a0ccf" - integrity sha1-RQ1Nyfpw3nMnYvvS1KKJgUGaDM8= - utils-merge@1.0.1: version "1.0.1" resolved "https://registry.yarnpkg.com/utils-merge/-/utils-merge-1.0.1.tgz#9f95710f50a267947b2ccc124741c1028427e713" integrity sha1-n5VxD1CiZ5R7LMwSR0HBAoQn5xM= -uuid@^3.0.0: - version "3.4.0" - resolved "https://registry.yarnpkg.com/uuid/-/uuid-3.4.0.tgz#b23e4358afa8a202fe7a100af1f5f883f02007ee" - integrity sha512-HjSDRw6gZE5JMggctHBcjVak08+KEVhSIiDzFnT9S9aegmp85S/bReBVTb4QTFaRNptJ9kuYaNhnbNEOkbKb/A== - uuid@~7.0.3: version "7.0.3" resolved "https://registry.yarnpkg.com/uuid/-/uuid-7.0.3.tgz#c5c9f2c8cf25dc0a372c4df1441c41f5bd0c680b" @@ -2400,11 +2079,6 @@ which-module@^2.0.0: resolved "https://registry.yarnpkg.com/which-module/-/which-module-2.0.0.tgz#d9ef07dce77b9902b8a3a8fa4b31c3e3f7e6e87a" integrity sha1-2e8H3Od7mQK4o6j6SzHD4/fm6Ho= -which-pm-runs@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/which-pm-runs/-/which-pm-runs-1.0.0.tgz#670b3afbc552e0b55df6b7780ca74615f23ad1cb" - integrity sha1-Zws6+8VS4LVd9rd4DKdGFfI60cs= - which@1.3.1, which@^1.1.1, which@^1.2.9: version "1.3.1" resolved "https://registry.yarnpkg.com/which/-/which-1.3.1.tgz#a45043d54f5805316da8d62f9f50918d3da70b0a" @@ -2412,7 +2086,7 @@ which@1.3.1, which@^1.1.1, which@^1.2.9: dependencies: isexe "^2.0.0" -wide-align@1.1.3, wide-align@^1.1.0: +wide-align@1.1.3: version "1.1.3" resolved "https://registry.yarnpkg.com/wide-align/-/wide-align-1.1.3.tgz#ae074e6bdc0c14a431e804e624549c633b000457" integrity sha512-QGkOQc8XL6Bt5PwnsExKBPuMKBxnGxWWW3fU55Xt4feHozMUhdUMaBCk290qpm/wG5u/RSKzwdAC4i51YigihA== @@ -2487,3 +2161,8 @@ yargs@13.3.2, yargs@^13.3.0: which-module "^2.0.0" y18n "^4.0.0" yargs-parser "^13.1.2" + +yn@3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/yn/-/yn-3.1.1.tgz#1e87401a09d767c1d5eab26a6e4c185182d2eb50" + integrity sha512-Ux4ygGWsu2c7isFWe8Yu1YluJmqVhxqK2cLXNQA5AcC3QfbGNpM7fu0Y8b/z16pXLnFxZYvWhd3fhBY9DLmC6Q==