From f4b68b136a47c5e9e17149d944756bd2a6f40e4b Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 15:21:35 +0100 Subject: [PATCH 01/50] Configurable batch size --- CHANGELOG.md | 9 ++++++ MIGRATION.md | 6 ++-- lib/kafkajs/_consumer.js | 69 +++++++++++----------------------------- types/kafkajs.d.ts | 12 ++++++- 4 files changed, 42 insertions(+), 54 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b5823789..60b57794 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +# confluent-kafka-javascript 1.6.1 + +v1.6.1 is a maintenance release. It is supported for all usage. + +### Enhancements + +1. Configurable batch size through the `js.max.batch.size` property (#389). + + # confluent-kafka-javascript 1.6.0 v1.6.0 is a feature release. It is supported for all usage. diff --git a/MIGRATION.md b/MIGRATION.md index 27ae6438..13cd5621 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -303,9 +303,9 @@ producerRun().then(consumerRun).catch(console.error); - The `heartbeat()` no longer needs to be called by the user in the `eachMessage/eachBatch` callback. Heartbeats are automatically managed by librdkafka. - The `partitionsConsumedConcurrently` is supported by both `eachMessage` and `eachBatch`. - - An API compatible version of `eachBatch` is available, but the batch size calculation is not - as per configured parameters, rather, a constant maximum size is configured internally. This is subject - to change. + - An API compatible version of `eachBatch` is available, maximum batch size + can be configured through the `js.max.batch.size` configuration property + and defaults to 32. The property `eachBatchAutoResolve` is supported. Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, and within the returned batch, `offsetLag` and `offsetLagLow` are supported. diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index ada184f8..20d093fb 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -155,17 +155,6 @@ class Consumer { */ #messageCache = null; - /** - * The maximum size of the message cache. - * Will be adjusted dynamically. - */ - #messageCacheMaxSize = 1; - - /** - * Number of times we tried to increase the cache. - */ - #increaseCount = 0; - /** * Whether the user has enabled manual offset management (commits). */ @@ -182,6 +171,11 @@ class Consumer { */ #partitionCount = 0; + /** + * Maximum batch size passed in eachBatch calls. + */ + #maxBatchSize = 32; + /** * Whether worker termination has been scheduled. */ @@ -311,8 +305,6 @@ class Consumer { * consumed messages upto N from the internalClient, but the user has stale'd the cache * after consuming just k (< N) messages. We seek back to last consumed offset + 1. */ this.#messageCache.clear(); - this.#messageCacheMaxSize = 1; - this.#increaseCount = 0; const clearPartitions = this.assignment(); const seeks = []; for (const topicPartition of clearPartitions) { @@ -691,6 +683,17 @@ class Consumer { this.#cacheExpirationTimeoutMs = this.#maxPollIntervalMs; rdKafkaConfig['max.poll.interval.ms'] = this.#maxPollIntervalMs * 2; + if (rdKafkaConfig['js.max.batch.size'] !== undefined) { + const maxBatchSize = +rdKafkaConfig['js.max.batch.size']; + if (!Number.isInteger(maxBatchSize) || (maxBatchSize <= 0 && maxBatchSize !== -1)) { + throw new error.KafkaJSError( + "'js.max.batch.size' must be a positive integer or -1 for unlimited batch size.", + { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + this.#maxBatchSize = maxBatchSize; + delete rdKafkaConfig['js.max.batch.size']; + } + return rdKafkaConfig; } @@ -844,33 +847,6 @@ class Consumer { await this.commitOffsets(); } - /** - * Request a size increase. - * It increases the size by 2x, but only if the size is less than 1024, - * only if the size has been requested to be increased twice in a row. - * @private - */ - #increaseMaxSize() { - if (this.#messageCacheMaxSize === 1024) - return; - this.#increaseCount++; - if (this.#increaseCount <= 1) - return; - this.#messageCacheMaxSize = Math.min(this.#messageCacheMaxSize << 1, 1024); - this.#increaseCount = 0; - } - - /** - * Request a size decrease. - * It decreases the size to 80% of the last received size, with a minimum of 1. - * @param {number} recvdSize - the number of messages received in the last poll. - * @private - */ - #decreaseMaxSize(recvdSize) { - this.#messageCacheMaxSize = Math.max(Math.floor((recvdSize * 8) / 10), 1); - this.#increaseCount = 0; - } - /** * Converts a list of messages returned by node-rdkafka into a message that can be used by the eachBatch callback. * @param {import("../..").Message[]} messages - must not be empty. Must contain messages from the same topic and partition. @@ -1001,11 +977,6 @@ class Consumer { const res = takeFromCache(); this.#lastFetchClockNs = hrtime.bigint(); this.#maxPollIntervalRestart.resolve(); - if (messages.length === this.#messageCacheMaxSize) { - this.#increaseMaxSize(); - } else { - this.#decreaseMaxSize(messages.length); - } return res; } finally { this.#fetchInProgress.resolve(); @@ -1040,7 +1011,7 @@ class Consumer { } return this.#fetchAndResolveWith(() => this.#messageCache.next(), - this.#messageCacheMaxSize); + Number.MAX_SAFE_INTEGER); } /** @@ -1071,7 +1042,7 @@ class Consumer { return this.#fetchAndResolveWith(() => this.#messageCache.nextN(null, size), - this.#messageCacheMaxSize); + Number.MAX_SAFE_INTEGER); } /** @@ -1559,11 +1530,9 @@ class Consumer { async #runInternal(config) { this.#concurrency = config.partitionsConsumedConcurrently; const perMessageProcessor = config.eachMessage ? this.#messageProcessor : this.#batchProcessor; - /* TODO: make this dynamic, based on max batch size / size of last message seen. */ - const maxBatchSize = 32; const fetcher = config.eachMessage ? (savedIdx) => this.#consumeSingleCached(savedIdx) - : (savedIdx) => this.#consumeCachedN(savedIdx, maxBatchSize); + : (savedIdx) => this.#consumeCachedN(savedIdx, this.#maxBatchSize); this.#workers = []; await this.#lock.write(async () => { diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 6045a77e..9048fb4e 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -244,7 +244,17 @@ export interface ConsumerConfig { partitionAssignors?: PartitionAssignors[], } -export type ConsumerGlobalAndTopicConfig = ConsumerGlobalConfig & ConsumerTopicConfig; +export interface JSConsumerConfig { + /** + * Maximum batch size passed in eachBatch calls. + * A value of -1 means no limit. + * + * @default 32 + */ + 'js.max.batch.size'?: string | number +} + +export type ConsumerGlobalAndTopicConfig = ConsumerGlobalConfig & ConsumerTopicConfig & JSConsumerConfig; export interface ConsumerConstructorConfig extends ConsumerGlobalAndTopicConfig { kafkaJS?: ConsumerConfig; From 09c6197cd573525f272c6f54b73588900cf61edf Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 19 Oct 2025 11:20:27 +0200 Subject: [PATCH 02/50] Keep 1.5s of data in cache --- lib/kafkajs/_consumer.js | 73 ++++++++++++++++++- .../consumer/consumeMessages.spec.js | 24 ++---- 2 files changed, 74 insertions(+), 23 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 20d093fb..c7ce6289 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -155,6 +155,12 @@ class Consumer { */ #messageCache = null; + /** + * The maximum size of the message cache. + * Will be adjusted dynamically. + */ + #messageCacheMaxSize = 1; + /** * Whether the user has enabled manual offset management (commits). */ @@ -175,6 +181,7 @@ class Consumer { * Maximum batch size passed in eachBatch calls. */ #maxBatchSize = 32; + #maxBatchesSize = 32; /** * Whether worker termination has been scheduled. @@ -191,6 +198,9 @@ class Consumer { */ #concurrency = 1; + + #runConfig = null; + /** * Promise that resolves together with last in progress fetch. * It's set to null when no fetch is in progress. @@ -228,7 +238,15 @@ class Consumer { /** * Last fetch real time clock in nanoseconds. */ - #lastFetchClockNs = 0; + #lastFetchClockNs = 0n; + /** + * Last number of messages fetched. + */ + #lastFetchedMessageCnt = 0n; + /** + * Last fetch concurrency used. + */ + #lastFetchedConcurrency = 0n; /** * List of pending operations to be executed after @@ -691,6 +709,10 @@ class Consumer { { code: error.ErrorCodes.ERR__INVALID_ARG }); } this.#maxBatchSize = maxBatchSize; + this.#maxBatchesSize = maxBatchSize; + if (maxBatchSize === -1) { + this.#messageCacheMaxSize = Number.MAX_SAFE_INTEGER; + } delete rdKafkaConfig['js.max.batch.size']; } @@ -933,6 +955,43 @@ class Consumer { return returnPayload; } + #updateMaxMessageCacheSize() { + if (this.#maxBatchSize === -1) { + return; + } + + const nowNs = hrtime.bigint(); + if (this.#lastFetchedMessageCnt > 0 && this.#lastFetchClockNs > 0n && + nowNs > this.#lastFetchClockNs) { + const consumptionDurationSeconds = Number(nowNs - this.#lastFetchClockNs) / 1e9; + const messagesPerSecondSingleWorker = this.#lastFetchedMessageCnt / this.#lastFetchedConcurrency / consumptionDurationSeconds; + // Keep enough messages in the cache for 1.5 seconds of consumption. + this.#messageCacheMaxSize = Math.round(1.5 * messagesPerSecondSingleWorker) * this.#concurrency; + const minCacheSize = this.#runConfig.eachBatch ? this.#maxBatchesSize : this.#concurrency; + if (this.#messageCacheMaxSize < minCacheSize) + this.#messageCacheMaxSize = minCacheSize; + else if (this.#messageCacheMaxSize > minCacheSize * 10) + this.#messageCacheMaxSize = minCacheSize * 10; + if (this.#messageCacheMaxSize < 1024) + this.#messageCacheMaxSize = 1024; + } + } + + #saveFetchStats(messages) { + this.#lastFetchClockNs = hrtime.bigint(); + const partitionsNum = new Map(); + for (const msg of messages) { + const key = partitionKey(msg); + partitionsNum.set(key, 1); + if (partitionsNum.size >= this.#concurrency) { + break; + } + } + this.#lastFetchedConcurrency = partitionsNum.size; + this.#lastFetchedMessageCnt = messages.length; + } + + async #fetchAndResolveWith(takeFromCache, size) { if (this.#fetchInProgress) { await this.#fetchInProgress; @@ -959,6 +1018,8 @@ class Consumer { const fetchResult = new DeferredPromise(); this.#logger.debug(`Attempting to fetch ${size} messages to the message cache`, this.#createConsumerBindingMessageMetadata()); + + this.#updateMaxMessageCacheSize(); this.#internalClient.consume(size, (err, messages) => fetchResult.resolve([err, messages])); @@ -975,7 +1036,7 @@ class Consumer { this.#messageCache.addMessages(messages); const res = takeFromCache(); - this.#lastFetchClockNs = hrtime.bigint(); + this.#saveFetchStats(messages); this.#maxPollIntervalRestart.resolve(); return res; } finally { @@ -1011,7 +1072,7 @@ class Consumer { } return this.#fetchAndResolveWith(() => this.#messageCache.next(), - Number.MAX_SAFE_INTEGER); + this.#messageCacheMaxSize); } /** @@ -1042,7 +1103,7 @@ class Consumer { return this.#fetchAndResolveWith(() => this.#messageCache.nextN(null, size), - Number.MAX_SAFE_INTEGER); + this.#messageCacheMaxSize); } /** @@ -1528,7 +1589,9 @@ class Consumer { * @private */ async #runInternal(config) { + this.#runConfig = config; this.#concurrency = config.partitionsConsumedConcurrently; + this.#maxBatchesSize = this.#maxBatchSize * this.#concurrency; const perMessageProcessor = config.eachMessage ? this.#messageProcessor : this.#batchProcessor; const fetcher = config.eachMessage ? (savedIdx) => this.#consumeSingleCached(savedIdx) @@ -1543,6 +1606,8 @@ class Consumer { this.#workerTerminationScheduled = new DeferredPromise(); this.#lastFetchClockNs = hrtime.bigint(); + this.#lastFetchedMessageCnt = 0; + this.#lastFetchedConcurrency = 0; if (this.#pendingOperations.length === 0) { const workersToSpawn = Math.max(1, Math.min(this.#concurrency, this.#partitionCount)); const cacheExpirationLoop = this.#cacheExpirationLoop(); diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index ca9204c6..2fc0ccd3 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -642,9 +642,6 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit let errors = false; let receivedMessages = 0; - const batchLengths = [1, 1, 2, - /* cache reset */ - 1, 1]; consumer.run({ partitionsConsumedConcurrently, eachBatchAutoResolve: true, @@ -652,10 +649,7 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit receivedMessages++; try { - expect(event.batch.messages.length) - .toEqual(batchLengths[receivedMessages - 1]); - - if (receivedMessages === 3) { + if (event.batch.messages.length >= 32) { expect(event.isStale()).toEqual(false); await sleep(7500); /* 7.5s 'processing' @@ -732,13 +726,7 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit let errors = false; let receivedMessages = 0; - const batchLengths = [/* first we reach batches of 32 message and fetches of 64 - * max poll interval exceeded happens on second - * 32 messages batch of the 64 msg fetch. */ - 1, 1, 2, 2, 4, 4, 8, 8, 16, 16, 32, 32, 32, 32, - /* max poll interval exceeded, 32 reprocessed + - * 1 new message. */ - 1, 1, 2, 2, 4, 4, 8, 8, 3]; + let firstLongBatchProcessing; consumer.run({ partitionsConsumedConcurrently, eachBatchAutoResolve: true, @@ -746,17 +734,15 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit receivedMessages++; try { - expect(event.batch.messages.length) - .toEqual(batchLengths[receivedMessages - 1]); - - if (receivedMessages === 13) { + if (!firstLongBatchProcessing && event.batch.messages.length >= 32) { expect(event.isStale()).toEqual(false); await sleep(6000); /* 6s 'processing' * cache clearance starts at 7000 */ expect(event.isStale()).toEqual(false); + firstLongBatchProcessing = receivedMessages; } - if ( receivedMessages === 14) { + if (firstLongBatchProcessing && receivedMessages === firstLongBatchProcessing + 1) { expect(event.isStale()).toEqual(false); await sleep(10000); /* 10s 'processing' From df6c83388f6569afb0571f110ff8c9974412fa77 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 15:14:56 +0100 Subject: [PATCH 03/50] Remove minimum 1024 cache size for faster rebalances in case of long processing time --- lib/kafkajs/_consumer.js | 12 +++++--- .../consumer/consumeMessages.spec.js | 28 +++++++++++++------ 2 files changed, 28 insertions(+), 12 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index c7ce6289..77fa73a8 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -957,6 +957,9 @@ class Consumer { #updateMaxMessageCacheSize() { if (this.#maxBatchSize === -1) { + // In case of unbounded max batch size it returns all available messages + // for a partition in each batch. Cache is unbounded as well as + // it takes only one call to process each partition. return; } @@ -965,15 +968,16 @@ class Consumer { nowNs > this.#lastFetchClockNs) { const consumptionDurationSeconds = Number(nowNs - this.#lastFetchClockNs) / 1e9; const messagesPerSecondSingleWorker = this.#lastFetchedMessageCnt / this.#lastFetchedConcurrency / consumptionDurationSeconds; - // Keep enough messages in the cache for 1.5 seconds of consumption. + // Keep enough messages in the cache for 1.5 seconds of concurrent consumption. this.#messageCacheMaxSize = Math.round(1.5 * messagesPerSecondSingleWorker) * this.#concurrency; const minCacheSize = this.#runConfig.eachBatch ? this.#maxBatchesSize : this.#concurrency; if (this.#messageCacheMaxSize < minCacheSize) + // Keep at least one batch or one message per worker. + // It's possible less workers than requested were active in previous run. this.#messageCacheMaxSize = minCacheSize; else if (this.#messageCacheMaxSize > minCacheSize * 10) + // Keep at most 10 messages or batches per requested worker. this.#messageCacheMaxSize = minCacheSize * 10; - if (this.#messageCacheMaxSize < 1024) - this.#messageCacheMaxSize = 1024; } } @@ -1018,7 +1022,7 @@ class Consumer { const fetchResult = new DeferredPromise(); this.#logger.debug(`Attempting to fetch ${size} messages to the message cache`, this.#createConsumerBindingMessageMetadata()); - + this.#updateMaxMessageCacheSize(); this.#internalClient.consume(size, (err, messages) => fetchResult.resolve([err, messages])); diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 2fc0ccd3..5c0df12d 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -412,6 +412,11 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit partitions: partitions, }); + // If you have a large consume time and consuming one message at a time, + // you need to have very small batch sizes to keep the concurrency up. + // It's to avoid having a too large cache and postponing the next fetch + // and so the rebalance too much. + const producer = createProducer({}, {'batch.num.messages': '1'}); await producer.connect(); await consumer.connect(); await consumer.subscribe({ topic: topicName }); @@ -448,6 +453,7 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit await producer.send({ topic: topicName, messages }); await maxConcurrentWorkersReached; expect(inProgressMaxValue).toBe(expectedMaxConcurrentWorkers); + await producer.disconnect(); }); it('consume GZIP messages', async () => { @@ -612,6 +618,7 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit let assigns = 0; let revokes = 0; let lost = 0; + let firstBatchProcessing; consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, @@ -649,14 +656,14 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit receivedMessages++; try { - if (event.batch.messages.length >= 32) { - expect(event.isStale()).toEqual(false); - await sleep(7500); - /* 7.5s 'processing' - * doesn't exceed max poll interval. - * Cache reset is transparent */ - expect(event.isStale()).toEqual(false); - } + expect(event.isStale()).toEqual(false); + await sleep(7500); + /* 7.5s 'processing' + * doesn't exceed max poll interval. + * Cache reset is transparent */ + expect(event.isStale()).toEqual(false); + if (firstBatchProcessing === undefined) + firstBatchProcessing = receivedMessages; } catch (e) { console.error(e); errors = true; @@ -680,6 +687,8 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit /* Triggers revocation */ await consumer.disconnect(); + expect(firstBatchProcessing).toBeDefined(); + expect(receivedMessages).toBeGreaterThan(firstBatchProcessing); /* First assignment */ expect(assigns).toEqual(1); /* Revocation on disconnect */ @@ -777,6 +786,9 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit /* Triggers revocation */ await consumer.disconnect(); + expect(firstLongBatchProcessing).toBeDefined(); + expect(receivedMessages).toBeGreaterThan(firstLongBatchProcessing); + /* First assignment + assignment after partitions lost */ expect(assigns).toEqual(2); /* Partitions lost + revocation on disconnect */ From cf45a3c523aadf15ec1705d73176b6e6cf456ef2 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 20 Oct 2025 13:50:32 +0200 Subject: [PATCH 04/50] Changed property name to `js.consumer.max.batch.size` --- CHANGELOG.md | 2 +- MIGRATION.md | 2 +- lib/kafkajs/_consumer.js | 8 ++++---- types/kafkajs.d.ts | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 60b57794..3e031d8e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,7 +4,7 @@ v1.6.1 is a maintenance release. It is supported for all usage. ### Enhancements -1. Configurable batch size through the `js.max.batch.size` property (#389). +1. Configurable batch size through the `js.consumer.max.batch.size` property (#389). # confluent-kafka-javascript 1.6.0 diff --git a/MIGRATION.md b/MIGRATION.md index 13cd5621..26ad9efd 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -304,7 +304,7 @@ producerRun().then(consumerRun).catch(console.error); Heartbeats are automatically managed by librdkafka. - The `partitionsConsumedConcurrently` is supported by both `eachMessage` and `eachBatch`. - An API compatible version of `eachBatch` is available, maximum batch size - can be configured through the `js.max.batch.size` configuration property + can be configured through the `js.consumer.max.batch.size` configuration property and defaults to 32. The property `eachBatchAutoResolve` is supported. Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 77fa73a8..6a1d8987 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -701,11 +701,11 @@ class Consumer { this.#cacheExpirationTimeoutMs = this.#maxPollIntervalMs; rdKafkaConfig['max.poll.interval.ms'] = this.#maxPollIntervalMs * 2; - if (rdKafkaConfig['js.max.batch.size'] !== undefined) { - const maxBatchSize = +rdKafkaConfig['js.max.batch.size']; + if (rdKafkaConfig['js.consumer.max.batch.size'] !== undefined) { + const maxBatchSize = +rdKafkaConfig['js.consumer.max.batch.size']; if (!Number.isInteger(maxBatchSize) || (maxBatchSize <= 0 && maxBatchSize !== -1)) { throw new error.KafkaJSError( - "'js.max.batch.size' must be a positive integer or -1 for unlimited batch size.", + "'js.consumer.max.batch.size' must be a positive integer or -1 for unlimited batch size.", { code: error.ErrorCodes.ERR__INVALID_ARG }); } this.#maxBatchSize = maxBatchSize; @@ -713,7 +713,7 @@ class Consumer { if (maxBatchSize === -1) { this.#messageCacheMaxSize = Number.MAX_SAFE_INTEGER; } - delete rdKafkaConfig['js.max.batch.size']; + delete rdKafkaConfig['js.consumer.max.batch.size']; } return rdKafkaConfig; diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 9048fb4e..a9bb92ac 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -251,7 +251,7 @@ export interface JSConsumerConfig { * * @default 32 */ - 'js.max.batch.size'?: string | number + 'js.consumer.max.batch.size'?: string | number } export type ConsumerGlobalAndTopicConfig = ConsumerGlobalConfig & ConsumerTopicConfig & JSConsumerConfig; From 28edb19ecbb758087e07fcc5623cb3cd6543d99c Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 22 Oct 2025 17:21:08 +0200 Subject: [PATCH 05/50] Use only 1.5 seconds cache size estimation aligned at batch size * concurrency --- lib/kafkajs/_consumer.js | 80 +++---------------- .../consumer/consumeMessages.spec.js | 3 +- .../consumer/consumerCacheTests.spec.js | 26 +++--- 3 files changed, 26 insertions(+), 83 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 6a1d8987..1d067b09 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -197,10 +197,6 @@ class Consumer { * The number of partitions to consume concurrently as set by the user, or 1. */ #concurrency = 1; - - - #runConfig = null; - /** * Promise that resolves together with last in progress fetch. * It's set to null when no fetch is in progress. @@ -958,7 +954,7 @@ class Consumer { #updateMaxMessageCacheSize() { if (this.#maxBatchSize === -1) { // In case of unbounded max batch size it returns all available messages - // for a partition in each batch. Cache is unbounded as well as + // for a partition in each batch. Cache is unbounded given that // it takes only one call to process each partition. return; } @@ -969,15 +965,11 @@ class Consumer { const consumptionDurationSeconds = Number(nowNs - this.#lastFetchClockNs) / 1e9; const messagesPerSecondSingleWorker = this.#lastFetchedMessageCnt / this.#lastFetchedConcurrency / consumptionDurationSeconds; // Keep enough messages in the cache for 1.5 seconds of concurrent consumption. - this.#messageCacheMaxSize = Math.round(1.5 * messagesPerSecondSingleWorker) * this.#concurrency; - const minCacheSize = this.#runConfig.eachBatch ? this.#maxBatchesSize : this.#concurrency; - if (this.#messageCacheMaxSize < minCacheSize) - // Keep at least one batch or one message per worker. - // It's possible less workers than requested were active in previous run. - this.#messageCacheMaxSize = minCacheSize; - else if (this.#messageCacheMaxSize > minCacheSize * 10) - // Keep at most 10 messages or batches per requested worker. - this.#messageCacheMaxSize = minCacheSize * 10; + // Round up to the nearest multiple of `#maxBatchesSize`. + this.#messageCacheMaxSize = Math.ceil( + Math.round(1.5 * messagesPerSecondSingleWorker) * this.#concurrency + / this.#maxBatchesSize + ) * this.#maxBatchesSize; } } @@ -1110,24 +1102,6 @@ class Consumer { this.#messageCacheMaxSize); } - /** - * Consumes n messages from the internal consumer. - * @returns {Promise} A promise that resolves to a list of messages. The size of this list is guaranteed to be less than or equal to n. - * @note this method cannot be used in conjunction with #consumeSingleCached. - * @private - */ - async #consumeN(n) { - return new Promise((resolve, reject) => { - this.#internalClient.consume(n, (err, messages) => { - if (err) { - reject(createKafkaJsErrorFromLibRdKafkaError(err)); - return; - } - resolve(messages); - }); - }); - } - /** * Flattens a list of topics with partitions into a list of topic, partition. * @param {Array<({topic: string, partitions: Array}|{topic: string, partition: number})>} topics @@ -1593,16 +1567,18 @@ class Consumer { * @private */ async #runInternal(config) { - this.#runConfig = config; - this.#concurrency = config.partitionsConsumedConcurrently; - this.#maxBatchesSize = this.#maxBatchSize * this.#concurrency; const perMessageProcessor = config.eachMessage ? this.#messageProcessor : this.#batchProcessor; const fetcher = config.eachMessage ? (savedIdx) => this.#consumeSingleCached(savedIdx) : (savedIdx) => this.#consumeCachedN(savedIdx, this.#maxBatchSize); - this.#workers = []; await this.#lock.write(async () => { + this.#workers = []; + this.#concurrency = config.partitionsConsumedConcurrently; + this.#maxBatchesSize = ( + config.eachBatch && this.#maxBatchSize > 0 ? + this.#maxBatchSize : + 1) * this.#concurrency; while (!this.#disconnectStarted) { if (this.#maxPollIntervalRestart.resolved) @@ -1639,38 +1615,6 @@ class Consumer { this.#maxPollIntervalRestart.resolve(); } - /** - * Consumes a single message from the consumer within the given timeout. - * THIS METHOD IS NOT IMPLEMENTED. - * @note This method cannot be used with run(). Either that, or this must be used. - * - * @param {any} args - * @param {number} args.timeout - the timeout in milliseconds, defaults to 1000. - * @returns {import("../..").Message|null} a message, or null if the timeout was reached. - * @private - */ - async consume({ timeout } = { timeout: 1000 }) { - if (this.#state !== ConsumerState.CONNECTED) { - throw new error.KafkaJSError('consume can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); - } - - if (this.#running) { - throw new error.KafkaJSError('consume() and run() cannot be used together.', { code: error.ErrorCodes.ERR__CONFLICT }); - } - - this.#internalClient.setDefaultConsumeTimeout(timeout); - let m = null; - - try { - const ms = await this.#consumeN(1); - m = ms[0]; - } finally { - this.#internalClient.setDefaultConsumeTimeout(undefined); - } - - throw new error.KafkaJSError('consume() is not implemented.' + m, { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); - } - async #commitOffsetsUntilNoStateErr(offsetsToCommit) { let err = { code: error.ErrorCodes.ERR_NO_ERROR }; do { diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js index 5c0df12d..8ef27e04 100644 --- a/test/promisified/consumer/consumeMessages.spec.js +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -434,9 +434,8 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit inProgressMaxValue = Math.max(inProgress, inProgressMaxValue); if (inProgressMaxValue >= expectedMaxConcurrentWorkers) { maxConcurrentWorkersReached.resolve(); - } else if (messagesConsumed.length > 2048) { - await sleep(1000); } + await sleep(100); inProgress--; }, }); diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index f923c65f..5ba94d15 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -90,6 +90,7 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon }); it('is cleared on seek', async () => { + const producer = createProducer({}, {'batch.num.messages': '1'}); await consumer.connect(); await producer.connect(); await consumer.subscribe({ topic: topicName }); @@ -134,6 +135,8 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); // partition 2 expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); + + await producer.disconnect(); }); it('is cleared before rebalance', async () => { @@ -142,6 +145,8 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon * the consumers are created with the same groupId, we create them here. * TODO: verify correctness of theory. It's conjecture... which solves flakiness. */ let groupId = `consumer-group-id-${secureRandom()}`; + const multiplier = 9; + const numMessages = 16 * multiplier; consumer = createConsumer({ groupId, maxWaitTimeInMs: 100, @@ -156,6 +161,7 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon autoCommit: isAutoCommit, clientId: "consumer2", }); + const producer = createProducer({}, {'batch.num.messages': '1'}); await consumer.connect(); await producer.connect(); @@ -164,7 +170,6 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon const messagesConsumed = []; const messagesConsumedConsumer1 = []; const messagesConsumedConsumer2 = []; - let consumer2ConsumeRunning = false; consumer.run({ partitionsConsumedConcurrently, @@ -176,18 +181,13 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon { topic: event.topic, partition: event.partition, offset: Number(event.message.offset) + 1 }, ]); - /* Until the second consumer joins, consume messages slowly so as to not consume them all - * before the rebalance triggers. */ - if (messagesConsumed.length > 1024 && !consumer2ConsumeRunning) { - await sleep(10); - } + await sleep(100); } }); /* Evenly distribute 1024*9 messages across 3 partitions */ let i = 0; - const multiplier = 9; - const messages = Array(1024 * multiplier) + const messages = Array(numMessages) .fill() .map(() => { const value = secureRandom(); @@ -198,7 +198,7 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon // Wait for the messages - some of them, before starting the // second consumer. - await waitForMessages(messagesConsumed, { number: 1024 }); + await waitForMessages(messagesConsumed, { number: 16 }); await consumer2.connect(); await consumer2.subscribe({ topic: topicName }); @@ -210,23 +210,23 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon }); await waitFor(() => consumer2.assignment().length > 0, () => null); - consumer2ConsumeRunning = true; /* Now that both consumers have joined, wait for all msgs to be consumed */ - await waitForMessages(messagesConsumed, { number: 1024 * multiplier }); + await waitForMessages(messagesConsumed, { number: numMessages }); /* No extra messages should be consumed. */ await sleep(1000); - expect(messagesConsumed.length).toEqual(1024 * multiplier); + expect(messagesConsumed.length).toEqual(numMessages); /* Check if all messages were consumed. */ expect(messagesConsumed.map(event => (+event.message.offset)).sort((a, b) => a - b)) - .toEqual(Array(1024 * multiplier).fill().map((_, i) => Math.floor(i / 3))); + .toEqual(Array(numMessages).fill().map((_, i) => Math.floor(i / 3))); /* Consumer2 should have consumed at least one message. */ expect(messagesConsumedConsumer2.length).toBeGreaterThan(0); await consumer2.disconnect(); + await producer.disconnect(); }, 60000); it('does not hold up polling for non-message events', async () => { From e34123200b51ea11d47760e2b0eb68be33b83487 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 23 Oct 2025 11:06:23 +0200 Subject: [PATCH 06/50] Fix for at-least-once guarantee not ensured in case a seek happens on one partition and there are messages being fetched for other partitions --- lib/kafkajs/_consumer.js | 40 +++++++++++++++++++--------------- lib/kafkajs/_consumer_cache.js | 35 ++++++++++++++++++++++++----- 2 files changed, 53 insertions(+), 22 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 1d067b09..43d5d13a 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1402,21 +1402,21 @@ class Consumer { return ppc; } - #discardMessages(ms, ppc) { - if (ms) { - let m = ms[0]; - if (m.constructor === Array) { - m = m[0]; - } - ppc = ms[1]; - if (m && !this.#lastConsumedOffsets.has(ppc.key)) { + #returnMessages(ms) { + let m = ms[0]; + // ppc could have been change we must return it as well. + let ppc = ms[1]; + const messagesToReturn = m.constructor === Array ? m : [m]; + const firstMessage = messagesToReturn[0]; + if (firstMessage && !this.#lastConsumedOffsets.has(ppc.key)) { this.#lastConsumedOffsets.set(ppc.key, { - topic: m.topic, - partition: m.partition, - offset: m.offset - 1, + topic: firstMessage.topic, + partition: firstMessage.partition, + offset: firstMessage.offset - 1, }); - } } + + this.#messageCache.returnMessages(messagesToReturn); return ppc; } @@ -1463,11 +1463,15 @@ class Consumer { continue; if (this.#pendingOperations.length) { - ppc = this.#discardMessages(ms, ppc); - break; + /* + * Don't process messages anymore, execute the operations first. + * Return the messages to the cache that will be cleared if needed. + * `ppc` could have been changed, we must return it as well. + */ + ppc = this.#returnMessages(ms); + } else { + ppc = await perMessageProcessor(ms, config); } - - ppc = await perMessageProcessor(ms, config); } catch (e) { /* Since this error cannot be exposed to the user in the current situation, just log and retry. * This is due to restartOnFailure being set to always true. */ @@ -1548,7 +1552,9 @@ class Consumer { * @private */ async #executePendingOperations() { - for (const op of this.#pendingOperations) { + // Execute all pending operations, they could add more operations. + while (this.#pendingOperations.length > 0) { + const op = this.#pendingOperations.shift(); await op(); } this.#pendingOperations = []; diff --git a/lib/kafkajs/_consumer_cache.js b/lib/kafkajs/_consumer_cache.js index 9047688e..33a2e81c 100644 --- a/lib/kafkajs/_consumer_cache.js +++ b/lib/kafkajs/_consumer_cache.js @@ -28,12 +28,19 @@ class PerPartitionMessageCache { } /** - * Adds a message to the cache. + * Adds a message to the cache as last one. */ - _add(message) { + _addLast(message) { this.#cache.addLast(message); } + /** + * Adds a message to the cache as first one. + */ + _addFirst(message) { + this.#cache.addFirst(message); + } + get key() { return this.#key; } @@ -126,7 +133,7 @@ class MessageCache { * * @param {Object} message - the message to add to the cache. */ - #add(message) { + #add(message, head = false) { const key = partitionKey(message); let cache = this.#tpToPpc.get(key); if (!cache) { @@ -135,7 +142,11 @@ class MessageCache { cache._node = this.#availablePartitions.addLast(cache); this.notifyAvailablePartitions(); } - cache._add(message); + if (head) { + cache._addFirst(message); + } else { + cache._addLast(message); + } } get availableSize() { @@ -183,7 +194,21 @@ class MessageCache { */ addMessages(messages) { for (const message of messages) - this.#add(message); + this.#add(message, false); + this.#size += messages.length; + } + + /** + * Return messages to the cache, to be read again. + * + * @param {Array} messages - the messages to return to the cache. + */ + returnMessages(messages) { + let i = messages.length - 1; + while (i >= 0) { + this.#add(messages[i], true); + i--; + } this.#size += messages.length; } From 82a4867418dbc71b7eb777e8b8f6630c9ca26ab2 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 10:28:43 +0100 Subject: [PATCH 07/50] Configurable cache size in milliseconds --- CHANGELOG.md | 4 +++- MIGRATION.md | 4 +++- lib/kafkajs/_consumer.js | 26 ++++++++++++++++++++++---- types/kafkajs.d.ts | 9 ++++++++- 4 files changed, 36 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3e031d8e..19b33882 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,7 +4,9 @@ v1.6.1 is a maintenance release. It is supported for all usage. ### Enhancements -1. Configurable batch size through the `js.consumer.max.batch.size` property (#389). +1. Configurable batch size through the `js.consumer.max.batch.size` property + and cache size through the `js.consumer.max.cache.size.per.worker.ms` + property (#389). # confluent-kafka-javascript 1.6.0 diff --git a/MIGRATION.md b/MIGRATION.md index 26ad9efd..1945ae5d 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -305,7 +305,9 @@ producerRun().then(consumerRun).catch(console.error); - The `partitionsConsumedConcurrently` is supported by both `eachMessage` and `eachBatch`. - An API compatible version of `eachBatch` is available, maximum batch size can be configured through the `js.consumer.max.batch.size` configuration property - and defaults to 32. + and defaults to 32. `js.consumer.max.cache.size.per.worker.ms` allows to + configure the cache size estimated based on consumption rate and defaults + to 1.5 seconds. The property `eachBatchAutoResolve` is supported. Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, and within the returned batch, `offsetLag` and `offsetLagLow` are supported. diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 43d5d13a..1f316847 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -183,6 +183,14 @@ class Consumer { #maxBatchSize = 32; #maxBatchesSize = 32; + /** + * Maximum cache size in milliseconds per worker. + * Based on the consumer rate estimated through the eachMessage/eachBatch calls. + * + * @default 1500 + */ + #maxCacheSizePerWorkerMs = 1500; + /** * Whether worker termination has been scheduled. */ @@ -711,6 +719,16 @@ class Consumer { } delete rdKafkaConfig['js.consumer.max.batch.size']; } + if (rdKafkaConfig['js.consumer.max.cache.size.per.worker.ms'] !== undefined) { + const maxCacheSizePerWorkerMs = +rdKafkaConfig['js.consumer.max.cache.size.per.worker.ms']; + if (!Number.isInteger(maxCacheSizePerWorkerMs) || (maxCacheSizePerWorkerMs <= 0)) { + throw new error.KafkaJSError( + "'js.consumer.max.cache.size.per.worker.ms' must be a positive integer.", + { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + this.#maxCacheSizePerWorkerMs = maxCacheSizePerWorkerMs; + delete rdKafkaConfig['js.consumer.max.cache.size.per.worker.ms']; + } return rdKafkaConfig; } @@ -962,12 +980,12 @@ class Consumer { const nowNs = hrtime.bigint(); if (this.#lastFetchedMessageCnt > 0 && this.#lastFetchClockNs > 0n && nowNs > this.#lastFetchClockNs) { - const consumptionDurationSeconds = Number(nowNs - this.#lastFetchClockNs) / 1e9; - const messagesPerSecondSingleWorker = this.#lastFetchedMessageCnt / this.#lastFetchedConcurrency / consumptionDurationSeconds; - // Keep enough messages in the cache for 1.5 seconds of concurrent consumption. + const consumptionDurationMilliseconds = Number(nowNs - this.#lastFetchClockNs) / 1e6; + const messagesPerMillisecondSingleWorker = this.#lastFetchedMessageCnt / this.#lastFetchedConcurrency / consumptionDurationMilliseconds; + // Keep enough messages in the cache for this.#maxCacheSizePerWorkerMs of concurrent consumption by all workers. // Round up to the nearest multiple of `#maxBatchesSize`. this.#messageCacheMaxSize = Math.ceil( - Math.round(1.5 * messagesPerSecondSingleWorker) * this.#concurrency + Math.round(this.#maxCacheSizePerWorkerMs * messagesPerMillisecondSingleWorker) * this.#concurrency / this.#maxBatchesSize ) * this.#maxBatchesSize; } diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index a9bb92ac..c18bbc84 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -251,7 +251,14 @@ export interface JSConsumerConfig { * * @default 32 */ - 'js.consumer.max.batch.size'?: string | number + 'js.consumer.max.batch.size'?: string | number, + /** + * Maximum cache size per worker in milliseconds based on the + * consume rate estimated through the eachMessage/eachBatch calls. + * + * @default 1500 + */ + 'js.consumer.max.cache.size.per.worker.ms'?: string | number } export type ConsumerGlobalAndTopicConfig = ConsumerGlobalConfig & ConsumerTopicConfig & JSConsumerConfig; From 03369f426e7ede47f5b76be6e679f0c9b72aae4a Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 13:52:18 +0100 Subject: [PATCH 08/50] Add worker identifier to the payload for better debugging --- lib/kafkajs/_consumer.js | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 1f316847..6dad154f 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -800,7 +800,7 @@ class Consumer { * @returns {import("../../types/kafkajs").EachMessagePayload} * @private */ - #createPayload(message) { + #createPayload(message, worker) { let key = message.key; if (typeof key === 'string') { key = Buffer.from(key); @@ -824,6 +824,7 @@ class Consumer { }, heartbeat: async () => { /* no op */ }, pause: this.pause.bind(this, [{ topic: message.topic, partitions: [message.partition] }]), + _worker: worker, }; } @@ -889,7 +890,7 @@ class Consumer { * @returns {import("../../types/kafkajs").EachBatchPayload} * @private */ - #createBatchPayload(messages) { + #createBatchPayload(messages, worker) { const topic = messages[0].topic; const partition = messages[0].partition; let watermarkOffsets = {}; @@ -954,6 +955,7 @@ class Consumer { _stale: false, _seeked: false, _lastResolvedOffset: { offset: -1, leaderEpoch: -1 }, + _worker: worker, heartbeat: async () => { /* no op */ }, pause: this.pause.bind(this, [{ topic, partitions: [partition] }]), commitOffsetsIfNecessary: this.#eachBatchPayload_commitOffsetsIfNecessary.bind(this), @@ -1290,12 +1292,12 @@ class Consumer { * @returns {Promise} The cache index of the message that was processed. * @private */ - async #messageProcessor(m, config) { + async #messageProcessor(m, config, worker) { let ppc; [m, ppc] = m; let key = partitionKey(m); let eachMessageProcessed = false; - const payload = this.#createPayload(m); + const payload = this.#createPayload(m, worker); try { this.#lastConsumedOffsets.set(key, m); @@ -1351,11 +1353,11 @@ class Consumer { * the passed batch. * @private */ - async #batchProcessor(ms, config) { + async #batchProcessor(ms, config, worker) { let ppc; [ms, ppc] = ms; const key = partitionKey(ms[0]); - const payload = this.#createBatchPayload(ms); + const payload = this.#createBatchPayload(ms, worker); this.#topicPartitionToBatchPayload.set(key, payload); @@ -1474,6 +1476,7 @@ class Consumer { */ async #worker(config, perMessageProcessor, fetcher) { let ppc = null; + let workerId = Math.random().toString().slice(2); while (!this.#workerTerminationScheduled.resolved) { try { const ms = await fetcher(ppc); @@ -1488,7 +1491,7 @@ class Consumer { */ ppc = this.#returnMessages(ms); } else { - ppc = await perMessageProcessor(ms, config); + ppc = await perMessageProcessor(ms, config, workerId); } } catch (e) { /* Since this error cannot be exposed to the user in the current situation, just log and retry. From 940bc20b9864c2225fca16615fcc1bece6a901eb Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 20 Oct 2025 12:26:40 +0200 Subject: [PATCH 09/50] Fix for test flakyness --- test/promisified/admin/fetch_offsets.spec.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/promisified/admin/fetch_offsets.spec.js b/test/promisified/admin/fetch_offsets.spec.js index 08e5a81a..c6412530 100644 --- a/test/promisified/admin/fetch_offsets.spec.js +++ b/test/promisified/admin/fetch_offsets.spec.js @@ -132,8 +132,7 @@ describe("fetchOffset function", () => { await consumer.run({ eachMessage: async ({ topic, partition, message }) => { - messagesConsumed.push(message); // Populate messagesConsumed - if (messagesConsumed.length === 5) { + if (messagesConsumed.length === 4) { await consumer.commitOffsets([ { topic, @@ -142,6 +141,7 @@ describe("fetchOffset function", () => { }, ]); } + messagesConsumed.push(message); // Populate messagesConsumed }, }); From bd0537f6f89680801ca1d926607d907f63f3addd Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 23 Oct 2025 11:07:57 +0200 Subject: [PATCH 10/50] Make `is cleared before rebalance` less flaky in case of increased time before first assignment --- lib/kafkajs/_consumer.js | 4 ++-- .../consumer/consumerCacheTests.spec.js | 14 ++++++-------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 6dad154f..c81d494b 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1435,7 +1435,7 @@ class Consumer { offset: firstMessage.offset - 1, }); } - + this.#messageCache.returnMessages(messagesToReturn); return ppc; } @@ -1484,7 +1484,7 @@ class Consumer { continue; if (this.#pendingOperations.length) { - /* + /* * Don't process messages anymore, execute the operations first. * Return the messages to the cache that will be cleared if needed. * `ppc` could have been changed, we must return it as well. diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index 5ba94d15..f62bdb4e 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -90,7 +90,6 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon }); it('is cleared on seek', async () => { - const producer = createProducer({}, {'batch.num.messages': '1'}); await consumer.connect(); await producer.connect(); await consumer.subscribe({ topic: topicName }); @@ -135,8 +134,6 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon expect(messagesConsumed.filter(m => m.partition === 1).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); // partition 2 expect(messagesConsumed.filter(m => m.partition === 2).map(m => m.message.offset)).toEqual(Array(1024 * 3).fill().map((_, i) => `${i}`)); - - await producer.disconnect(); }); it('is cleared before rebalance', async () => { @@ -145,7 +142,7 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon * the consumers are created with the same groupId, we create them here. * TODO: verify correctness of theory. It's conjecture... which solves flakiness. */ let groupId = `consumer-group-id-${secureRandom()}`; - const multiplier = 9; + const multiplier = 18; const numMessages = 16 * multiplier; consumer = createConsumer({ groupId, @@ -161,7 +158,6 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon autoCommit: isAutoCommit, clientId: "consumer2", }); - const producer = createProducer({}, {'batch.num.messages': '1'}); await consumer.connect(); await producer.connect(); @@ -181,11 +177,14 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon { topic: event.topic, partition: event.partition, offset: Number(event.message.offset) + 1 }, ]); - await sleep(100); + // Simulate some processing time so we don't poll all messages + // and put them in the cache before consumer2 joins. + if (messagesConsumedConsumer2.length === 0) + await sleep(100); } }); - /* Evenly distribute 1024*9 messages across 3 partitions */ + /* Evenly distribute numMessages messages across 3 partitions */ let i = 0; const messages = Array(numMessages) .fill() @@ -226,7 +225,6 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon expect(messagesConsumedConsumer2.length).toBeGreaterThan(0); await consumer2.disconnect(); - await producer.disconnect(); }, 60000); it('does not hold up polling for non-message events', async () => { From dc21a5490f25b38d861d136eabaf470ec4b85e6e Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 09:33:21 +0100 Subject: [PATCH 11/50] Reduce flakyness of 'times out if messages are pending' --- test/promisified/producer/flush.spec.js | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/test/promisified/producer/flush.spec.js b/test/promisified/producer/flush.spec.js index 64a15b9f..e8dec677 100644 --- a/test/promisified/producer/flush.spec.js +++ b/test/promisified/producer/flush.spec.js @@ -71,6 +71,10 @@ describe('Producer > Flush', () => { it('times out if messages are pending', async () => { + const producer = createProducer({ + }, { + 'batch.num.messages': 1, + }); await producer.connect(); let messageSent = false; @@ -82,6 +86,7 @@ describe('Producer > Flush', () => { /* Small timeout */ await expect(producer.flush({ timeout: 1 })).rejects.toThrow(Kafka.KafkaJSTimeout); expect(messageSent).toBe(false); + await producer.disconnect(); } ); From 9ae3f5934a7b300b9f04197e4d36a335777c43ad Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 20 Oct 2025 12:43:58 +0200 Subject: [PATCH 12/50] 1.6.1-alpha.1 --- ci/update-version.js | 2 +- lib/util.js | 2 +- package-lock.json | 6 +++--- package.json | 2 +- schemaregistry/package.json | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/ci/update-version.js b/ci/update-version.js index 531ed2ef..796261f9 100644 --- a/ci/update-version.js +++ b/ci/update-version.js @@ -89,7 +89,7 @@ function getPackageVersion(tag, branch) { // publish with a -devel suffix for EA and RC releases. if (tag.prerelease.length > 0) { - baseVersion += '-' + tag.prerelease.join('-'); + baseVersion += '-' + tag.prerelease.join('.'); } console.log(`Package version is "${baseVersion}"`); diff --git a/lib/util.js b/lib/util.js index ae539b46..389c3411 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = '1.6.0'; +util.bindingVersion = '1.6.1-alpha.1'; diff --git a/package-lock.json b/package-lock.json index 45f6f0a3..e930a811 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "1.6.0", + "version": "1.6.1-alpha.1", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "1.6.0", + "version": "1.6.1-alpha.1", "hasInstallScript": true, "license": "MIT", "workspaces": [ @@ -10077,7 +10077,7 @@ }, "schemaregistry": { "name": "@confluentinc/schemaregistry", - "version": "1.6.0", + "version": "1.6.1-alpha.1", "license": "MIT", "dependencies": { "@aws-sdk/client-kms": "^3.637.0", diff --git a/package.json b/package.json index 4d36d2da..0e0e27c3 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "1.6.0", + "version": "1.6.1-alpha.1", "description": "Node.js bindings for librdkafka", "librdkafka": "2.12.0", "librdkafka_win": "2.12.0", diff --git a/schemaregistry/package.json b/schemaregistry/package.json index fe058a56..d6218c6d 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "1.6.0", + "version": "1.6.1-alpha.1", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts", From d30ea6d64193a3dbe0c7c79828fa73522142d136 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:40:29 +0100 Subject: [PATCH 13/50] fixup CHANGELOG and spaces --- CHANGELOG.md | 2 ++ lib/kafkajs/_consumer.js | 1 + 2 files changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 19b33882..a6b7bb9f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,8 @@ v1.6.1 is a maintenance release. It is supported for all usage. 1. Configurable batch size through the `js.consumer.max.batch.size` property and cache size through the `js.consumer.max.cache.size.per.worker.ms` property (#389). +2. Fix for at-least-once guarantee not ensured in case a seek happens on one +partition and there are messages being fetched about other partitions (#389). # confluent-kafka-javascript 1.6.0 diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index c81d494b..d9661ffb 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -205,6 +205,7 @@ class Consumer { * The number of partitions to consume concurrently as set by the user, or 1. */ #concurrency = 1; + /** * Promise that resolves together with last in progress fetch. * It's set to null when no fetch is in progress. From 5a7b93dde696ec9fa26625a0e94df1779f6fa4e2 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:42:16 +0100 Subject: [PATCH 14/50] 1.6.1 --- lib/util.js | 2 +- package-lock.json | 6 +++--- package.json | 2 +- schemaregistry/package.json | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/lib/util.js b/lib/util.js index 389c3411..d2dcc9e5 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = '1.6.1-alpha.1'; +util.bindingVersion = '1.6.1'; diff --git a/package-lock.json b/package-lock.json index e930a811..fe9eb280 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "1.6.1-alpha.1", + "version": "1.6.1", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "1.6.1-alpha.1", + "version": "1.6.1", "hasInstallScript": true, "license": "MIT", "workspaces": [ @@ -10077,7 +10077,7 @@ }, "schemaregistry": { "name": "@confluentinc/schemaregistry", - "version": "1.6.1-alpha.1", + "version": "1.6.1", "license": "MIT", "dependencies": { "@aws-sdk/client-kms": "^3.637.0", diff --git a/package.json b/package.json index 0e0e27c3..bebafa9c 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "1.6.1-alpha.1", + "version": "1.6.1", "description": "Node.js bindings for librdkafka", "librdkafka": "2.12.0", "librdkafka_win": "2.12.0", diff --git a/schemaregistry/package.json b/schemaregistry/package.json index d6218c6d..b15a5c8f 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "1.6.1-alpha.1", + "version": "1.6.1", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts", From 06e6bcb6a942f3b3a2293c54b7ea27ea7e5392d9 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:48:51 +0100 Subject: [PATCH 15/50] FIXME about KIP-848 autocommit issue --- test/promisified/consumer/consumerCacheTests.spec.js | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/test/promisified/consumer/consumerCacheTests.spec.js b/test/promisified/consumer/consumerCacheTests.spec.js index f62bdb4e..e5bf1950 100644 --- a/test/promisified/consumer/consumerCacheTests.spec.js +++ b/test/promisified/consumer/consumerCacheTests.spec.js @@ -232,6 +232,10 @@ describe.each(cases)('Consumer message cache - isAutoCommit = %s - partitionsCon * non-message events like rebalances, etc. Internally, this is to make sure that * we call poll() at least once within max.poll.interval.ms even if the cache is * still full. This depends on us expiring the cache on time. */ + + /* FIXME: this test can be flaky when using KIP-848 protocol and + * auto-commit. To check if there's something to fix about that case. + */ const impatientConsumer = createConsumer({ groupId, maxWaitTimeInMs: 100, From 697a98f4b6cea206fafec8697f070fdc593865ae Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:58:50 +0100 Subject: [PATCH 16/50] Updated PR in changelog entry --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a6b7bb9f..f6e5830a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,9 +6,9 @@ v1.6.1 is a maintenance release. It is supported for all usage. 1. Configurable batch size through the `js.consumer.max.batch.size` property and cache size through the `js.consumer.max.cache.size.per.worker.ms` - property (#389). + property (#393). 2. Fix for at-least-once guarantee not ensured in case a seek happens on one -partition and there are messages being fetched about other partitions (#389). +partition and there are messages being fetched about other partitions (#393). # confluent-kafka-javascript 1.6.0 From 3634aeff7460f1fd8788a3ba10395348ad1d326e Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 3 Nov 2025 11:03:20 +0100 Subject: [PATCH 17/50] Address comments --- CHANGELOG.md | 4 ++-- MIGRATION.md | 7 +++++-- lib/kafkajs/_consumer.js | 4 ++-- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f6e5830a..7c05b809 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ -# confluent-kafka-javascript 1.6.1 +# confluent-kafka-javascript 1.7.0 -v1.6.1 is a maintenance release. It is supported for all usage. +v1.7.0 is a feature release. It is supported for all usage. ### Enhancements diff --git a/MIGRATION.md b/MIGRATION.md index 1945ae5d..40f96f20 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -305,9 +305,12 @@ producerRun().then(consumerRun).catch(console.error); - The `partitionsConsumedConcurrently` is supported by both `eachMessage` and `eachBatch`. - An API compatible version of `eachBatch` is available, maximum batch size can be configured through the `js.consumer.max.batch.size` configuration property - and defaults to 32. `js.consumer.max.cache.size.per.worker.ms` allows to + and defaults to 32. It is not dependent on the size of the produced batches + present on the broker, as these are constructed client-side. Similar to + the Java client configuration `max.poll.records`. + `js.consumer.max.cache.size.per.worker.ms` allows to configure the cache size estimated based on consumption rate and defaults - to 1.5 seconds. + to the cache being sized to 1.5s worth of messages. The property `eachBatchAutoResolve` is supported. Within the `eachBatch` callback, use of `uncommittedOffsets` is unsupported, and within the returned batch, `offsetLag` and `offsetLagLow` are supported. diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index d9661ffb..92efb5c3 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -706,7 +706,7 @@ class Consumer { this.#cacheExpirationTimeoutMs = this.#maxPollIntervalMs; rdKafkaConfig['max.poll.interval.ms'] = this.#maxPollIntervalMs * 2; - if (rdKafkaConfig['js.consumer.max.batch.size'] !== undefined) { + if (Object.hasOwn(rdKafkaConfig, 'js.consumer.max.batch.size')) { const maxBatchSize = +rdKafkaConfig['js.consumer.max.batch.size']; if (!Number.isInteger(maxBatchSize) || (maxBatchSize <= 0 && maxBatchSize !== -1)) { throw new error.KafkaJSError( @@ -720,7 +720,7 @@ class Consumer { } delete rdKafkaConfig['js.consumer.max.batch.size']; } - if (rdKafkaConfig['js.consumer.max.cache.size.per.worker.ms'] !== undefined) { + if (Object.hasOwn(rdKafkaConfig, 'js.consumer.max.cache.size.per.worker.ms')) { const maxCacheSizePerWorkerMs = +rdKafkaConfig['js.consumer.max.cache.size.per.worker.ms']; if (!Number.isInteger(maxCacheSizePerWorkerMs) || (maxCacheSizePerWorkerMs <= 0)) { throw new error.KafkaJSError( From 1924126e4350c35f8306a2a2456bf157346f3002 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 3 Nov 2025 12:13:43 +0100 Subject: [PATCH 18/50] v1.7.0 --- lib/util.js | 2 +- package-lock.json | 6 +++--- package.json | 2 +- schemaregistry/package.json | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/lib/util.js b/lib/util.js index d2dcc9e5..a8a9322f 100644 --- a/lib/util.js +++ b/lib/util.js @@ -52,4 +52,4 @@ util.dictToStringList = function (mapOrObject) { return list; }; -util.bindingVersion = '1.6.1'; +util.bindingVersion = '1.7.0'; diff --git a/package-lock.json b/package-lock.json index fe9eb280..e9a84eb5 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "@confluentinc/kafka-javascript", - "version": "1.6.1", + "version": "1.7.0", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "@confluentinc/kafka-javascript", - "version": "1.6.1", + "version": "1.7.0", "hasInstallScript": true, "license": "MIT", "workspaces": [ @@ -10077,7 +10077,7 @@ }, "schemaregistry": { "name": "@confluentinc/schemaregistry", - "version": "1.6.1", + "version": "1.7.0", "license": "MIT", "dependencies": { "@aws-sdk/client-kms": "^3.637.0", diff --git a/package.json b/package.json index bebafa9c..1852a308 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/kafka-javascript", - "version": "1.6.1", + "version": "1.7.0", "description": "Node.js bindings for librdkafka", "librdkafka": "2.12.0", "librdkafka_win": "2.12.0", diff --git a/schemaregistry/package.json b/schemaregistry/package.json index b15a5c8f..b00c58f9 100644 --- a/schemaregistry/package.json +++ b/schemaregistry/package.json @@ -1,6 +1,6 @@ { "name": "@confluentinc/schemaregistry", - "version": "1.6.1", + "version": "1.7.0", "description": "Node.js client for Confluent Schema Registry", "main": "dist/index.js", "types": "dist/index.d.ts", From 6a68b4927f377b0a82b616b7b134075d8a3c482c Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 8 Oct 2025 14:41:51 +0200 Subject: [PATCH 19/50] Performance test improvements, measure eachBatch rate, time and lag, avg and max memory usage --- .gitignore | 2 + .semaphore/semaphore.yml | 2 +- ci/tests/run_perf_test.js | 147 ++++++++++++++++++ ci/tests/run_perf_test.sh | 64 -------- .../performance/performance-consolidated.js | 74 ++++++++- .../performance-primitives-common.js | 99 ++++++++++++ .../performance-primitives-kafkajs.js | 86 +++++----- .../performance/performance-primitives.js | 79 ++++------ 8 files changed, 393 insertions(+), 160 deletions(-) create mode 100644 ci/tests/run_perf_test.js delete mode 100755 ci/tests/run_perf_test.sh create mode 100644 examples/performance/performance-primitives-common.js diff --git a/.gitignore b/.gitignore index 07f5f6b2..d49064a2 100644 --- a/.gitignore +++ b/.gitignore @@ -21,3 +21,5 @@ coverage .nyc_output/ *lcov.info **/lcov-report +examples/performance/*.json +*.log diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 1457b69d..3eece201 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -191,7 +191,7 @@ blocks: - export NODE_OPTIONS='--max-old-space-size=1536' - cd examples/performance - npm install - - bash -c '../../ci/tests/run_perf_test.sh' + - node '../../ci/tests/run_perf_test.js' - rm -rf ./node_modules - name: "Linux amd64: Release" diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js new file mode 100644 index 00000000..889b58d7 --- /dev/null +++ b/ci/tests/run_perf_test.js @@ -0,0 +1,147 @@ +#!/usr/bin/env node + +const { execSync } = require('child_process'); + +function runCommand(command) { + try { + const output = execSync(command, { encoding: 'utf8', stdio: 'pipe' }); + console.log(output); + return output; + } catch (error) { + const errorOutput = error.stdout || error.stderr || error.message; + console.log(errorOutput); + return errorOutput; + } +} + +function extractValue(content, pattern) { + try { + const lines = content.split('\n'); + const matchingLine = lines.find(line => line.includes(pattern)); + if (matchingLine) { + const value = matchingLine.split(':')[1]?.trim(); + return Number(value || ''); + } + return NaN; + } catch (error) { + return NaN; + } +} + +function belowThreshold(value, target, threshold = 0.7) { + if (isNaN(value) || isNaN(target)) + throw new Error(`Invalid number comparison: value=${value}, target=${target}`); + return value < (target * threshold); +} + +function belowTarget(value, target) { + return belowThreshold(value, target, 1); +} + +// Run performance tests and store outputs in memory +console.log('Running Confluent Producer/Consumer test...'); +const outputConfluentProducerConsumer = runCommand('MODE=confluent MESSAGE_COUNT=50000 node performance-consolidated.js --create-topics --consumer --producer'); + +console.log('Running KafkaJS Producer/Consumer test...'); +const outputKjsProducerConsumer = runCommand('MODE=kafkajs MESSAGE_COUNT=50000 node performance-consolidated.js --create-topics --consumer --producer'); + +console.log('Running Confluent CTP test...'); +const outputConfluentCtp = runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); + +console.log('Running KafkaJS CTP test...'); +const outputKjsCtp = runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); + +// Extract Confluent results +const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); +const consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate (eachMessage):'); +const consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); +const consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate (eachBatch):'); +const consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); +const consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); +const consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); +const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); +const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); +const ctpConfluent = extractValue(outputConfluentCtp, '=== Consume-Transform-Produce Rate:'); + +// Extract KafkaJS results +const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); +const consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate (eachMessage):'); +const consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); +const consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate (eachBatch):'); +const consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); +const consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); +const consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); +const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); +const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); +const ctpKjs = extractValue(outputKjsCtp, '=== Consume-Transform-Produce Rate:'); + +// Print results +console.log(`Producer rates: confluent ${producerConfluent}, kafkajs ${producerKjs}`); +console.log(`Consumer rates (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); +console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); +console.log(`Consumer rates (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); +console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); +console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); +console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); +console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); +console.log(`Max RSS: confluent ${consumerConfluentMaxRSS}, kafkajs ${consumerKjsMaxRSS}`); +console.log(`CTP rates: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); + +let errcode = 0; +const maxPerformanceDifference = 0.7; + +// Compare against KJS (30% threshold) +if (belowThreshold(producerConfluent, producerKjs, maxPerformanceDifference)) { + console.log(`Producer rates differ by more than 30%: confluent ${producerConfluent}, kafkajs ${producerKjs}`); + errcode = 1; +} + +if (belowThreshold(consumerConfluentMessage, consumerKjsMessage, maxPerformanceDifference)) { + console.log(`Consumer rates (eachMessage) differ by more than 30%: confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); + // FIXME: improve consumer performance at least to KafkaJS level + errcode = 0; +} + +// Lower is better for time +if (belowThreshold(consumerKjsTime, consumerConfluentTime, maxPerformanceDifference)) { + console.log(`Consumption time (eachMessage) differ by more than 30%: confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); + errcode = 0; +} + +if (belowThreshold(consumerConfluentBatch, consumerKjsBatch, maxPerformanceDifference)) { + console.log(`Consumer rates (eachBatch) differ by more than 30%: confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); + errcode = 0; +} + +// Lower is better for time +if (belowThreshold(consumerKjsBatchTime, consumerConfluentBatchTime, maxPerformanceDifference)) { + console.log(`Consumption time (eachBatch) differ by more than 30%: confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); + errcode = 0; +} + +if (belowThreshold(ctpConfluent, ctpKjs, maxPerformanceDifference)) { + console.log(`CTP rates differ by more than 30%: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); + errcode = 1; +} + +// Compare against target numbers +const TARGET_PRODUCE = process.env.TARGET_PRODUCE_PERFORMANCE || '35'; +const TARGET_CONSUME = process.env.TARGET_CONSUME_PERFORMANCE || '18'; +const TARGET_CTP = process.env.TARGET_CTP_PERFORMANCE || '0.02'; + +if (belowTarget(producerConfluent, TARGET_PRODUCE)) { + console.log(`Confluent producer rate is below target: ${producerConfluent}`); + errcode = 1; +} + +if (belowTarget(consumerConfluentMessage, TARGET_CONSUME)) { + console.log(`Confluent consumer rate is below target: ${consumerConfluentMessage}`); + errcode = 1; +} + +if (belowTarget(ctpConfluent, TARGET_CTP)) { + console.log(`Confluent CTP rate is below target: ${ctpConfluent}`); + errcode = 1; +} + +process.exit(errcode); \ No newline at end of file diff --git a/ci/tests/run_perf_test.sh b/ci/tests/run_perf_test.sh deleted file mode 100755 index c55a67ab..00000000 --- a/ci/tests/run_perf_test.sh +++ /dev/null @@ -1,64 +0,0 @@ -#!/bin/bash - -testresultConfluentProducerConsumer=$(mktemp) -testresultConfluentCtp=$(mktemp) -testresultKjsProducerConsumer=$(mktemp) -testresultKjsCtp=$(mktemp) - -MODE=confluent MESSAGE_COUNT=500000 node performance-consolidated.js --create-topics --consumer --producer 2>&1 | tee "$testresultConfluentProducerConsumer" -MODE=kafkajs MESSAGE_COUNT=500000 node performance-consolidated.js --create-topics --consumer --producer 2>&1 | tee "$testresultKjsProducerConsumer" -MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp 2>&1 | tee "$testresultConfluentCtp" -MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp 2>&1 | tee "$testresultKjsCtp" - -producerConfluent=$(grep "=== Producer Rate:" "$testresultConfluentProducerConsumer" | cut -d':' -f2 | tr -d ' ') -consumerConfluent=$(grep "=== Consumer Rate:" "$testresultConfluentProducerConsumer" | cut -d':' -f2 | tr -d ' ') -ctpConfluent=$(grep "=== Consume-Transform-Produce Rate:" "$testresultConfluentCtp" | cut -d':' -f2 | tr -d ' ') -producerKjs=$(grep "=== Producer Rate:" "$testresultKjsProducerConsumer" | cut -d':' -f2 | tr -d ' ') -consumerKjs=$(grep "=== Consumer Rate:" "$testresultKjsProducerConsumer" | cut -d':' -f2 | tr -d ' ') -ctpKjs=$(grep "=== Consume-Transform-Produce Rate:" "$testresultKjsCtp" | cut -d':' -f2 | tr -d ' ') - -echo "Producer rates: confluent $producerConfluent, kafkajs $producerKjs" -echo "Consumer rates: confluent $consumerConfluent, kafkajs $consumerKjs" -echo "CTP rates: confluent $ctpConfluent, kafkajs $ctpKjs" - -errcode=0 - -# Compare against KJS -if [[ $(echo "$producerConfluent < $producerKjs * 70 / 100" | bc -l) -eq 1 ]]; then - echo "Producer rates differ by more than 30%: confluent $producerConfluent, kafkajs $producerKjs" - errcode=1 -fi - -if [[ $(echo "$consumerConfluent < $consumerKjs * 70 / 100" | bc -l) -eq 1 ]]; then - echo "Consumer rates differ by more than 30%: confluent $consumerConfluent, kafkajs $consumerKjs" - # FIXME: improve consumer performance at least to KafkaJS level - errcode=0 -fi - -if [[ $(echo "$ctpConfluent < $ctpKjs * 70 / 100" | bc -l) -eq 1 ]]; then - echo "CTP rates differ by more than 30%: confluent $ctpConfluent, kafkajs $ctpKjs" - errcode=1 -fi - -# Compare against numbers set within semaphore config -TARGET_PRODUCE="${TARGET_PRODUCE_PERFORMANCE:-35}" -TARGET_CONSUME="${TARGET_CONSUME_PERFORMANCE:-18}" -TARGET_CTP="${TARGET_CTP_PERFORMANCE:-0.02}" - -if [[ $(echo "$producerConfluent < $TARGET_PRODUCE" | bc -l) -eq 1 ]]; then - echo "Confluent producer rate is below target: $producerConfluent" - errcode=1 -fi - -if [[ $(echo "$consumerConfluent < $TARGET_CONSUME" | bc -l) -eq 1 ]]; then - echo "Confluent consumer rate is below target: $consumerConfluent" - errcode=1 -fi - -if [[ $(echo "$ctpConfluent < $TARGET_CTP" | bc -l) -eq 1 ]]; then - echo "Confluent CTP rate is below target: $ctpConfluent" - errcode=1 -fi - -exit $errcode - diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 7ef9bce6..b56f71b5 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -1,3 +1,4 @@ +const fs = require('fs'); const mode = process.env.MODE ? process.env.MODE : 'confluent'; let runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether; @@ -27,6 +28,44 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e const produceConsumeLatency = process.argv.includes('--latency'); const all = process.argv.includes('--all'); const createTopics = process.argv.includes('--create-topics'); + let maxAverageRSSKB, maxMaxRSSKB; + const stats = {}; + + let measures = []; + let interval; + const startTrackingMemory = () => { + interval = setInterval(() => { + measures.push({ rss: process.memoryUsage().rss, + timestamp: Date.now() }); + }, 100); + }; + + const datapointToJSON = (m) => + ({ rss: m.rss.toString(), timestamp: m.timestamp.toString() }); + + const endTrackingMemory = (fileName) => { + clearInterval(interval); + interval = null; + const averageRSS = measures.reduce((sum, m) => sum + m.rss, 0) / measures.length; + const averageRSSKB = averageRSS / 1024; + maxAverageRSSKB = !maxAverageRSSKB ? averageRSSKB : Math.max(averageRSSKB, maxAverageRSSKB); + console.log(`=== Average RSS: ${averageRSSKB} KB`); + const max = measures.reduce((prev, current) => (prev.rss > current.rss) ? prev : current); + const maxRSSKB = max.rss / 1024; + maxMaxRSSKB = !maxMaxRSSKB ? maxRSSKB : Math.max(maxRSSKB, maxMaxRSSKB); + console.log(`=== Max RSS: ${maxRSSKB} KB at ${new Date(max.timestamp).toISOString()}`); + if (fileName) { + const measuresJSON = JSON.stringify({ + measures: measures.map(datapointToJSON), + averageRSS: averageRSS.toString(), + maxRSS: datapointToJSON(max) + }, null, 2); + fs.writeFileSync(fileName, measuresJSON); + } + measures = []; + } + + console.log(`=== Starting Performance Tests - Mode ${mode} ===`); if (createTopics || all) { console.log("=== Creating Topics (deleting if they exist already):"); @@ -45,18 +84,38 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e console.log(` Batch Size: ${batchSize}`); console.log(` Compression: ${compression}`); console.log(` Warmup Messages: ${warmupMessages}`); + startTrackingMemory(); const producerRate = await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + endTrackingMemory(`producer-memory-${mode}.json`); console.log("=== Producer Rate: ", producerRate); } if (consumer || all) { // If user runs this without --producer then they are responsible for seeding the topic. - console.log("=== Running Basic Consumer Performance Test:") + console.log("=== Running Basic Consumer Performance Test (eachMessage):") console.log(` Brokers: ${brokers}`); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); - const consumerRate = await runConsumer(brokers, topic, messageCount); - console.log("=== Consumer Rate: ", consumerRate); + startTrackingMemory(); + const consumerRate = await runConsumer(brokers, topic, warmupMessages, messageCount, false, stats); + endTrackingMemory(`consumer-memory-message-${mode}.json`); + console.log("=== Consumer Rate (eachMessage): ", consumerRate); + console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); + } + + if (consumer || all) { + // If user runs this without --producer then they are responsible for seeding the topic. + console.log("=== Running Basic Consumer Performance Test (eachBatch):") + console.log(` Brokers: ${brokers}`); + console.log(` Topic: ${topic}`); + console.log(` Message Count: ${messageCount}`); + startTrackingMemory(); + const consumerRate = await runConsumer(brokers, topic, warmupMessages, messageCount, true, stats); + endTrackingMemory(`consumer-memory-batch-${mode}.json`); + console.log("=== Consumer Rate (eachBatch): ", consumerRate); + console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); + console.log("=== Max eachBatch lag: ", stats.maxOffsetLag); + console.log("=== Consumption time (eachBatch): ", stats.durationSeconds); } if (ctp || all) { @@ -67,7 +126,9 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e console.log(` Message Count: ${messageCount}`); // Seed the topic with messages await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + startTrackingMemory(); const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, warmupMessages, messageCount, messageProcessTimeMs, ctpConcurrency); + endTrackingMemory(`consume-transform-produce-${mode}.json`); console.log("=== Consume-Transform-Produce Rate: ", ctpRate); } @@ -78,7 +139,9 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e console.log(` Message Count: ${messageCount}`); console.log(` Consumer Processing Time: ${consumerProcessingTime}`); console.log(` Producer Processing Time: ${producerProcessingTime}`); + startTrackingMemory(); const { mean, p50, p90, p95, outliers } = await runProducerConsumerTogether(brokers, topic, messageCount, messageSize, producerProcessingTime, consumerProcessingTime); + endTrackingMemory(`producer-consumer-together-${mode}.json`); console.log(`=== Produce-To-Consume Latency (ms): Mean: ${mean}, P50: ${p50}, P90: ${p90}, P95: ${p95}`); // The presence of outliers invalidates the mean measurement, and rasies concerns as to why there are any. @@ -87,4 +150,9 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e console.log("=== Outliers (ms): ", outliers); } } + + if (maxAverageRSSKB !== undefined && maxMaxRSSKB !== undefined) { + console.log(`=== Max Average RSS across tests: `, maxAverageRSSKB); + console.log(`=== Max RSS across tests: `, maxMaxRSSKB); + } })(); \ No newline at end of file diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js new file mode 100644 index 00000000..e2f9f8dc --- /dev/null +++ b/examples/performance/performance-primitives-common.js @@ -0,0 +1,99 @@ +const { hrtime } = require('process'); + +async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { + await consumer.connect(); + await consumer.subscribe({ topic }); + + let messagesReceived = 0; + let messagesMeasured = 0; + let totalMessageSize = 0; + let totalBatches = 0; + let totalOffsetLag = 0; + let maxOffsetLag = 0; + let durationSeconds = 0; + let startTime; + let rate; + const skippedMessages = warmupMessages; + + console.log("Starting consumer."); + let consumeMethod = { + eachMessage: async ({ topic, partition, message }) => { + messagesReceived++; + + if (messagesReceived >= skippedMessages) { + messagesMeasured++; + totalMessageSize += message.value.length; + + if (messagesReceived === skippedMessages) { + startTime = hrtime.bigint(); + } else if (messagesMeasured === totalMessageCnt) { + let durationNanos = Number(hrtime.bigint() - startTime); + durationSeconds = durationNanos / 1e9; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } + } + } + } + if (eachBatch) { + consumeMethod = { + eachBatch: async ({ batch }) => { + const messagesBeforeBatch = messagesReceived; + const topic = batch.topic; + messagesReceived += batch.messages.length; + if (messagesReceived >= skippedMessages) { + const offsetLag = batch.offsetLag(); + totalOffsetLag += Number(offsetLag); + maxOffsetLag = Math.max(offsetLag, maxOffsetLag); + totalBatches++; + messagesMeasured = messagesReceived - skippedMessages; + let messages = batch.messages; + if (messagesBeforeBatch < skippedMessages) { + messages = messages.slice(messages.length - messagesMeasured); + } + for (const message of messages) + totalMessageSize += message.value.length; + + if (!startTime) { + startTime = hrtime.bigint(); + } else if (messagesMeasured === totalMessageCnt) { + let durationNanos = Number(hrtime.bigint() - startTime); + durationSeconds = durationNanos / 1e9; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } + } + } + }; + } + + consumer.run({ + ...consumeMethod + }); + + await new Promise((resolve) => { + let interval = setInterval(() => { + if (messagesMeasured >= totalMessageCnt) { + clearInterval(interval); + resolve(); + } + }, 1000); + }); + + await consumer.disconnect(); + + if (stats) { + if (eachBatch) { + stats.averageOffsetLag = totalBatches > 0 ? (totalOffsetLag / totalBatches) : 0; + stats.maxOffsetLag = maxOffsetLag; + } + stats.durationSeconds = durationSeconds; + } + return rate; +} + +module.exports = { + runConsumer +}; \ No newline at end of file diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 1dfc530e..3eac7236 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -1,6 +1,7 @@ const { Kafka, CompressionTypes } = require('kafkajs'); const { randomBytes } = require('crypto'); const { hrtime } = require('process'); +const { runConsumer: runConsumerCommon } = require('./performance-primitives-common'); module.exports = { runProducer, @@ -106,62 +107,52 @@ async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessa return rate; } -async function runConsumer(brokers, topic, totalMessageCnt) { - const kafka = new Kafka({ - clientId: 'kafka-test-performance', - brokers: brokers.split(','), - }); +class CompatibleConsumer { + constructor(consumer) { + this.consumer = consumer; + } - const consumer = kafka.consumer({ - groupId: 'test-group' + Math.random(), - }); - await consumer.connect(); - await consumer.subscribe({ topic, fromBeginning: true }); + async connect() { + return this.consumer.connect(); + } - let messagesReceived = 0; - let messagesMeasured = 0; - let totalMessageSize = 0; - let startTime; - let rate; - const skippedMessages = 100; + async disconnect() { + return this.consumer.disconnect(); + } - console.log("Starting consumer."); + async subscribe(opts) { + return this.consumer.subscribe({ + ...opts, + fromBeginning: true + }); + } - consumer.run({ - autoCommit: false, - eachMessage: async ({ topic, partition, message }) => { - messagesReceived++; + pause(topics) { + return this.consumer.pause(topics); + } - if (messagesReceived >= skippedMessages) { - messagesMeasured++; - totalMessageSize += message.value.length; + run(opts) { + return this.consumer.run({ + ...opts, + autoCommit: false, + }); + } +} - if (messagesReceived === skippedMessages) { - startTime = hrtime(); - } else if (messagesMeasured === totalMessageCnt) { - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); - } - } - } +function newCompatibleConsumer(brokers) { + const kafka = new Kafka({ + clientId: 'kafka-test-performance', + brokers: brokers.split(','), }); - totalMessageSize = 0; - - await new Promise((resolve) => { - let interval = setInterval(() => { - if (messagesReceived >= totalMessageCnt) { - clearInterval(interval); - resolve(); - } - }, 1000); + const consumer = kafka.consumer({ + groupId: 'test-group' + Math.random(), }); + return new CompatibleConsumer(consumer); +} - await consumer.disconnect(); - return rate; +async function runConsumer(brokers, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { + return runConsumerCommon(newCompatibleConsumer(brokers), topic, warmupMessages, totalMessageCnt, eachBatch, stats); } async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { @@ -213,8 +204,9 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, w if (messagesMeasured === totalMessageCnt) { let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + let durationSeconds = durationNanos / 1e9; rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd, transformed and sent ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + console.log(`Recvd, transformed and sent ${messagesMeasured} messages in ${durationSeconds}, ${totalMessageSize} bytes; rate is ${rate} MB/s`); consumer.pause([{ topic }]); } } else { diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 8dce6b54..af8b4519 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -1,6 +1,7 @@ const { Kafka, ErrorCodes, CompressionTypes } = require('../../').KafkaJS; const { randomBytes } = require('crypto'); const { hrtime } = require('process'); +const { runConsumer: runConsumerCommon } = require('./performance-primitives-common'); module.exports = { runProducer, @@ -111,7 +112,35 @@ async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessa return rate; } -async function runConsumer(brokers, topic, totalMessageCnt) { +class CompatibleConsumer { + constructor(consumer) { + this.consumer = consumer; + } + + async connect() { + return this.consumer.connect(); + } + + async disconnect() { + return this.consumer.disconnect(); + } + + async subscribe(opts) { + return this.consumer.subscribe(opts); + } + + pause(topics) { + return this.consumer.pause(topics); + } + + run(opts) { + return this.consumer.run({ + ...opts + }); + } +} + +function newCompatibleConsumer(brokers) { const kafka = new Kafka({ 'client.id': 'kafka-test-performance', 'metadata.broker.list': brokers, @@ -123,51 +152,11 @@ async function runConsumer(brokers, topic, totalMessageCnt) { 'auto.offset.reset': 'earliest', 'fetch.queue.backoff.ms': '100', }); - await consumer.connect(); - await consumer.subscribe({ topic }); - - let messagesReceived = 0; - let messagesMeasured = 0; - let totalMessageSize = 0; - let startTime; - let rate; - const skippedMessages = 100; - - console.log("Starting consumer."); - - consumer.run({ - eachMessage: async ({ topic, partition, message }) => { - messagesReceived++; - - if (messagesReceived >= skippedMessages) { - messagesMeasured++; - totalMessageSize += message.value.length; - - if (messagesReceived === skippedMessages) { - startTime = hrtime(); - } else if (messagesMeasured === totalMessageCnt) { - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd ${messagesMeasured} messages, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); - } - } - } - }); - - totalMessageSize = 0; - await new Promise((resolve) => { - let interval = setInterval(() => { - if (messagesMeasured >= totalMessageCnt) { - clearInterval(interval); - resolve(); - } - }, 1000); - }); + return new CompatibleConsumer(consumer); +} - await consumer.disconnect(); - return rate; +async function runConsumer(brokers, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { + return runConsumerCommon(newCompatibleConsumer(brokers), topic, warmupMessages, totalMessageCnt, eachBatch, stats); } async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { From d7d5b0ff346c13a3545f92b77cfe1d804158290b Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 9 Oct 2025 15:33:32 +0200 Subject: [PATCH 20/50] Run performance tests with SASL_PLAINTEXT or SASL_SSL and plain authentication --- .../performance/performance-consolidated.js | 46 ++++++++++----- .../performance-primitives-kafkajs.js | 58 ++++++++++--------- .../performance/performance-primitives.js | 54 +++++++++-------- 3 files changed, 95 insertions(+), 63 deletions(-) diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index b56f71b5..3993fad4 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -9,6 +9,9 @@ if (mode === 'confluent') { } const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; +const securityProtocol = process.env.SECURITY_PROTOCOL; +const saslUsername = process.env.SASL_USERNAME; +const saslPassword = process.env.SASL_PASSWORD; const topic = process.env.KAFKA_TOPIC || 'test-topic'; const topic2 = process.env.KAFKA_TOPIC2 || 'test-topic2'; const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; @@ -20,6 +23,23 @@ const messageProcessTimeMs = process.env.MESSAGE_PROCESS_TIME_MS ? +process.env. const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY : 1; const consumerProcessingTime = process.env.CONSUMER_PROCESSING_TIME ? +process.env.CONSUMER_PROCESSING_TIME : 100; const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.env.PRODUCER_PROCESSING_TIME : 100; +const parameters = { + brokers, + securityProtocol, + saslUsername, + saslPassword, +} + +function logParameters(parameters) { + console.log(` Brokers: ${parameters.brokers}`); + if (parameters.securityProtocol && parameters.saslUsername && parameters.saslPassword) { + console.log(` Security Protocol: ${parameters.securityProtocol}`); + console.log(` SASL Username: ${parameters.saslUsername ? parameters.saslUsername : 'not set'}`); + console.log(` SASL Password: ${parameters.saslPassword ? '******' : 'not set'}`); + } else { + console.log(" No security protocol configured"); + } +} (async function () { const producer = process.argv.includes('--producer'); @@ -69,15 +89,15 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e if (createTopics || all) { console.log("=== Creating Topics (deleting if they exist already):"); - console.log(` Brokers: ${brokers}`); + logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Topic2: ${topic2}`); - await runCreateTopics(brokers, topic, topic2); + await runCreateTopics(parameters, topic, topic2); } if (producer || all) { console.log("=== Running Basic Producer Performance Test:") - console.log(` Brokers: ${brokers}`); + logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); console.log(` Message Size: ${messageSize}`); @@ -85,7 +105,7 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e console.log(` Compression: ${compression}`); console.log(` Warmup Messages: ${warmupMessages}`); startTrackingMemory(); - const producerRate = await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + const producerRate = await runProducer(parameters, topic, batchSize, warmupMessages, messageCount, messageSize, compression); endTrackingMemory(`producer-memory-${mode}.json`); console.log("=== Producer Rate: ", producerRate); } @@ -93,11 +113,11 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e if (consumer || all) { // If user runs this without --producer then they are responsible for seeding the topic. console.log("=== Running Basic Consumer Performance Test (eachMessage):") - console.log(` Brokers: ${brokers}`); + logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); startTrackingMemory(); - const consumerRate = await runConsumer(brokers, topic, warmupMessages, messageCount, false, stats); + const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, false, stats); endTrackingMemory(`consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate (eachMessage): ", consumerRate); console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); @@ -106,11 +126,11 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e if (consumer || all) { // If user runs this without --producer then they are responsible for seeding the topic. console.log("=== Running Basic Consumer Performance Test (eachBatch):") - console.log(` Brokers: ${brokers}`); + logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); startTrackingMemory(); - const consumerRate = await runConsumer(brokers, topic, warmupMessages, messageCount, true, stats); + const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, true, stats); endTrackingMemory(`consumer-memory-batch-${mode}.json`); console.log("=== Consumer Rate (eachBatch): ", consumerRate); console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); @@ -120,27 +140,27 @@ const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.e if (ctp || all) { console.log("=== Running Consume-Transform-Produce Performance Test:") - console.log(` Brokers: ${brokers}`); + logParameters(parameters); console.log(` ConsumeTopic: ${topic}`); console.log(` ProduceTopic: ${topic2}`); console.log(` Message Count: ${messageCount}`); // Seed the topic with messages - await runProducer(brokers, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + await runProducer(parameters, topic, batchSize, warmupMessages, messageCount, messageSize, compression); startTrackingMemory(); - const ctpRate = await runConsumeTransformProduce(brokers, topic, topic2, warmupMessages, messageCount, messageProcessTimeMs, ctpConcurrency); + const ctpRate = await runConsumeTransformProduce(parameters, topic, topic2, warmupMessages, messageCount, messageProcessTimeMs, ctpConcurrency); endTrackingMemory(`consume-transform-produce-${mode}.json`); console.log("=== Consume-Transform-Produce Rate: ", ctpRate); } if (produceConsumeLatency || all) { console.log("=== Running Produce-To-Consume Latency Performance Test:") - console.log(` Brokers: ${brokers}`); + logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); console.log(` Consumer Processing Time: ${consumerProcessingTime}`); console.log(` Producer Processing Time: ${producerProcessingTime}`); startTrackingMemory(); - const { mean, p50, p90, p95, outliers } = await runProducerConsumerTogether(brokers, topic, messageCount, messageSize, producerProcessingTime, consumerProcessingTime); + const { mean, p50, p90, p95, outliers } = await runProducerConsumerTogether(parameters, topic, messageCount, messageSize, producerProcessingTime, consumerProcessingTime); endTrackingMemory(`producer-consumer-together-${mode}.json`); console.log(`=== Produce-To-Consume Latency (ms): Mean: ${mean}, P50: ${p50}, P90: ${p90}, P95: ${p95}`); diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 3eac7236..14bbe483 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -11,11 +11,29 @@ module.exports = { runProducerConsumerTogether, }; -async function runCreateTopics(brokers, topic, topic2) { - const kafka = new Kafka({ +function baseConfiguration(parameters) { + let ret = { clientId: 'kafka-test-performance', - brokers: brokers.split(','), - }); + brokers: parameters.brokers.split(','), + }; + if (parameters.securityProtocol && + parameters.saslUsername && + parameters.saslPassword) { + ret = { + ...ret, + ssl: parameters.securityProtocol.toLowerCase().includes('ssl'), + sasl: { + mechanism: 'plain', + username: parameters.saslUsername, + password: parameters.saslPassword + } + }; + } + return ret; +} + +async function runCreateTopics(parameters, topic, topic2) { + const kafka = new Kafka(baseConfiguration(parameters)); const admin = kafka.admin(); await admin.connect(); @@ -43,7 +61,7 @@ async function runCreateTopics(brokers, topic, topic2) { await admin.disconnect(); } -async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { +async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { let totalMessagesSent = 0; let totalBytesSent = 0; @@ -53,10 +71,7 @@ async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessa const messages = Array(batchSize).fill(message); - const kafka = new Kafka({ - clientId: 'kafka-test-performance', - brokers: brokers.split(','), - }); + const kafka = new Kafka(baseConfiguration(parameters)); const producer = kafka.producer(); await producer.connect(); @@ -139,11 +154,8 @@ class CompatibleConsumer { } } -function newCompatibleConsumer(brokers) { - const kafka = new Kafka({ - clientId: 'kafka-test-performance', - brokers: brokers.split(','), - }); +function newCompatibleConsumer(parameters) { + const kafka = new Kafka(baseConfiguration(parameters)); const consumer = kafka.consumer({ groupId: 'test-group' + Math.random(), @@ -151,15 +163,12 @@ function newCompatibleConsumer(brokers) { return new CompatibleConsumer(consumer); } -async function runConsumer(brokers, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { - return runConsumerCommon(newCompatibleConsumer(brokers), topic, warmupMessages, totalMessageCnt, eachBatch, stats); +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { + return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, stats); } -async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { - const kafka = new Kafka({ - clientId: 'kafka-test-performance', - brokers: brokers.split(','), - }); +async function runConsumeTransformProduce(parameters, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { + const kafka = new Kafka(baseConfiguration(parameters)); const producer = kafka.producer({}); await producer.connect(); @@ -234,11 +243,8 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, w } -async function runProducerConsumerTogether(brokers, topic, totalMessageCnt, msgSize, produceMessageProcessTimeMs, consumeMessageProcessTimeMs) { - const kafka = new Kafka({ - clientId: 'kafka-test-performance', - brokers: brokers.split(','), - }); +async function runProducerConsumerTogether(parameters, topic, totalMessageCnt, msgSize, produceMessageProcessTimeMs, consumeMessageProcessTimeMs) { + const kafka = new Kafka(baseConfiguration(parameters)); const producer = kafka.producer({}); await producer.connect(); diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index af8b4519..3c147b74 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -11,11 +11,27 @@ module.exports = { runProducerConsumerTogether, }; -async function runCreateTopics(brokers, topic, topic2) { - const kafka = new Kafka({ +function baseConfiguration(parameters) { + let ret = { 'client.id': 'kafka-test-performance', - "metadata.broker.list": brokers, - }); + 'metadata.broker.list': parameters.brokers, + }; + if (parameters.securityProtocol && + parameters.saslUsername && + parameters.saslPassword) { + ret = { + ...ret, + 'security.protocol': parameters.securityProtocol, + 'sasl.mechanism': 'plain', + 'sasl.username': parameters.saslUsername, + 'sasl.password': parameters.saslPassword, + }; + } + return ret; +} + +async function runCreateTopics(parameters, topic, topic2) { + const kafka = new Kafka(baseConfiguration(parameters)); const admin = kafka.admin(); await admin.connect(); @@ -44,7 +60,7 @@ async function runCreateTopics(brokers, topic, topic2) { await admin.disconnect(); } -async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { +async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { let totalMessagesSent = 0; let totalBytesSent = 0; @@ -55,8 +71,7 @@ async function runProducer(brokers, topic, batchSize, warmupMessages, totalMessa const messages = Array(batchSize).fill(message); const kafka = new Kafka({ - 'client.id': 'kafka-test-performance', - 'metadata.broker.list': brokers, + ...baseConfiguration(parameters), 'compression.codec': CompressionTypes[compression], }); @@ -140,11 +155,8 @@ class CompatibleConsumer { } } -function newCompatibleConsumer(brokers) { - const kafka = new Kafka({ - 'client.id': 'kafka-test-performance', - 'metadata.broker.list': brokers, - }); +function newCompatibleConsumer(parameters) { + const kafka = new Kafka(baseConfiguration(parameters)); const consumer = kafka.consumer({ 'group.id': 'test-group' + Math.random(), @@ -155,15 +167,12 @@ function newCompatibleConsumer(brokers) { return new CompatibleConsumer(consumer); } -async function runConsumer(brokers, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { - return runConsumerCommon(newCompatibleConsumer(brokers), topic, warmupMessages, totalMessageCnt, eachBatch, stats); +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { + return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, stats); } -async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { - const kafka = new Kafka({ - 'client.id': 'kafka-test-performance', - 'metadata.broker.list': brokers, - }); +async function runConsumeTransformProduce(parameters, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { + const kafka = new Kafka(baseConfiguration(parameters)); const producer = kafka.producer({ /* We want things to be flushed immediately as we'll be awaiting this. */ @@ -249,11 +258,8 @@ async function runConsumeTransformProduce(brokers, consumeTopic, produceTopic, w return rate; } -async function runProducerConsumerTogether(brokers, topic, totalMessageCnt, msgSize, produceMessageProcessTimeMs, consumeMessageProcessTimeMs) { - const kafka = new Kafka({ - 'client.id': 'kafka-test-performance', - 'metadata.broker.list': brokers, - }); +async function runProducerConsumerTogether(parameters, topic, totalMessageCnt, msgSize, produceMessageProcessTimeMs, consumeMessageProcessTimeMs) { + const kafka = new Kafka(baseConfiguration(parameters)); const producer = kafka.producer({ /* We want things to be flushed immediately as we'll be awaiting this. */ From b62ebc28a68ca0958e6fc39ec806d6f0d9f25357 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 9 Oct 2025 17:45:19 +0200 Subject: [PATCH 21/50] Fixes --- ci/tests/run_perf_test.js | 6 ++++-- examples/performance/performance-primitives-common.js | 5 +++-- examples/performance/performance-primitives.js | 2 +- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 889b58d7..1bc5ccde 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -9,7 +9,9 @@ function runCommand(command) { return output; } catch (error) { const errorOutput = error.stdout || error.stderr || error.message; - console.log(errorOutput); + console.log(error.stdout); + console.log(error.stderr); + console.log(error.message); return errorOutput; } } @@ -30,7 +32,7 @@ function extractValue(content, pattern) { function belowThreshold(value, target, threshold = 0.7) { if (isNaN(value) || isNaN(target)) - throw new Error(`Invalid number comparison: value=${value}, target=${target}`); + return false; return value < (target * threshold); } diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index e2f9f8dc..90b850c1 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -57,10 +57,11 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (!startTime) { startTime = hrtime.bigint(); - } else if (messagesMeasured === totalMessageCnt) { + } else if (messagesMeasured >= totalMessageCnt) { let durationNanos = Number(hrtime.bigint() - startTime); durationSeconds = durationNanos / 1e9; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + rate = durationNanos === 0 ? Infinity : + (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); consumer.pause([{ topic }]); } diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 3c147b74..33b24b19 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -22,7 +22,7 @@ function baseConfiguration(parameters) { ret = { ...ret, 'security.protocol': parameters.securityProtocol, - 'sasl.mechanism': 'plain', + 'sasl.mechanism': 'PLAIN', 'sasl.username': parameters.saslUsername, 'sasl.password': parameters.saslPassword, }; From 03f3458e07dd79b1996b63d855aa119e0bf73bb8 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 10:35:05 +0200 Subject: [PATCH 22/50] Improve KafkaJS producing, more reliable createTopics, configurable message count and partition number --- ci/tests/run_perf_test.js | 26 +++++++++++++------ .../performance/performance-consolidated.js | 7 +++-- .../performance-primitives-kafkajs.js | 11 +++++--- .../performance/performance-primitives.js | 11 +++++--- 4 files changed, 39 insertions(+), 16 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 1bc5ccde..22f2b42e 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -42,18 +42,22 @@ function belowTarget(value, target) { // Run performance tests and store outputs in memory console.log('Running Confluent Producer/Consumer test...'); -const outputConfluentProducerConsumer = runCommand('MODE=confluent MESSAGE_COUNT=50000 node performance-consolidated.js --create-topics --consumer --producer'); +const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 50000; +const skipCtpTest = process.env.SKIP_CTP_TEST ? process.env.SKIP_CTP_TEST === 'true' : false; +const outputConfluentProducerConsumer = runCommand(`MODE=confluent MESSAGE_COUNT=${messageCount} node performance-consolidated.js --create-topics --consumer --producer`); console.log('Running KafkaJS Producer/Consumer test...'); -const outputKjsProducerConsumer = runCommand('MODE=kafkajs MESSAGE_COUNT=50000 node performance-consolidated.js --create-topics --consumer --producer'); +const outputKjsProducerConsumer = runCommand(`MODE=kafkajs MESSAGE_COUNT=${messageCount} node performance-consolidated.js --create-topics --consumer --producer`); console.log('Running Confluent CTP test...'); const outputConfluentCtp = runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); console.log('Running KafkaJS CTP test...'); -const outputKjsCtp = runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); +const outputKjsCtp = skipCtpTest ? '' : + runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); // Extract Confluent results +let ctpConfluent, ctpKjs; const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); const consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate (eachMessage):'); const consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); @@ -63,7 +67,9 @@ const consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerCon const consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); -const ctpConfluent = extractValue(outputConfluentCtp, '=== Consume-Transform-Produce Rate:'); +if (!skipCtpTest) { + ctpConfluent = extractValue(outputConfluentCtp, '=== Consume-Transform-Produce Rate:'); +} // Extract KafkaJS results const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); @@ -75,7 +81,9 @@ const consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== const consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); -const ctpKjs = extractValue(outputKjsCtp, '=== Consume-Transform-Produce Rate:'); +if (!skipCtpTest) { + ctpKjs = extractValue(outputKjsCtp, '=== Consume-Transform-Produce Rate:'); +} // Print results console.log(`Producer rates: confluent ${producerConfluent}, kafkajs ${producerKjs}`); @@ -87,7 +95,9 @@ console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); console.log(`Max RSS: confluent ${consumerConfluentMaxRSS}, kafkajs ${consumerKjsMaxRSS}`); -console.log(`CTP rates: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); +if (!skipCtpTest) { + console.log(`CTP rates: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); +} let errcode = 0; const maxPerformanceDifference = 0.7; @@ -121,7 +131,7 @@ if (belowThreshold(consumerKjsBatchTime, consumerConfluentBatchTime, maxPerforma errcode = 0; } -if (belowThreshold(ctpConfluent, ctpKjs, maxPerformanceDifference)) { +if (!skipCtpTest && belowThreshold(ctpConfluent, ctpKjs, maxPerformanceDifference)) { console.log(`CTP rates differ by more than 30%: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); errcode = 1; } @@ -141,7 +151,7 @@ if (belowTarget(consumerConfluentMessage, TARGET_CONSUME)) { errcode = 1; } -if (belowTarget(ctpConfluent, TARGET_CTP)) { +if (!skipCtpTest && belowTarget(ctpConfluent, TARGET_CTP)) { console.log(`Confluent CTP rate is below target: ${ctpConfluent}`); errcode = 1; } diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 3993fad4..a5655893 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -5,7 +5,9 @@ let runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runPr if (mode === 'confluent') { ({ runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether } = require('./performance-primitives')); } else { - ({ runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether } = require('./performance-primitives-kafkajs')); + ({ runProducer, runConsumer, runConsumeTransformProduce, runProducerConsumerTogether } = require('./performance-primitives-kafkajs')); + /* createTopics is more reliable in CKJS */ + ({ runCreateTopics } = require('./performance-primitives')); } const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; @@ -18,6 +20,7 @@ const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 10 const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; const compression = process.env.COMPRESSION || 'None'; +const numPartitions = process.env.PARTITIONS ? +process.env.PARTITIONS : 3; const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); const messageProcessTimeMs = process.env.MESSAGE_PROCESS_TIME_MS ? +process.env.MESSAGE_PROCESS_TIME_MS : 5; const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY : 1; @@ -92,7 +95,7 @@ function logParameters(parameters) { logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Topic2: ${topic2}`); - await runCreateTopics(parameters, topic, topic2); + await runCreateTopics(parameters, topic, topic2, numPartitions); } if (producer || all) { diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 14bbe483..8e154841 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -32,7 +32,7 @@ function baseConfiguration(parameters) { return ret; } -async function runCreateTopics(parameters, topic, topic2) { +async function runCreateTopics(parameters, topic, topic2, numPartitions) { const kafka = new Kafka(baseConfiguration(parameters)); const admin = kafka.admin(); @@ -40,7 +40,7 @@ async function runCreateTopics(parameters, topic, topic2) { for (let t of [topic, topic2]) { let topicCreated = await admin.createTopics({ - topics: [{ topic: t, numPartitions: 3 }], + topics: [{ topic: t, numPartitions }], }).catch(console.error); if (topicCreated) { console.log(`Created topic ${t}`); @@ -52,7 +52,7 @@ async function runCreateTopics(parameters, topic, topic2) { await new Promise(resolve => setTimeout(resolve, 1000)); /* Propagate. */ await admin.createTopics({ topics: [ - { topic: t, numPartitions: 3 }, + { topic: t, numPartitions }, ], }).catch(console.error); console.log(`Created topic ${t}`); @@ -97,6 +97,7 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe // await them all at once. We need the second while loop to keep sending // in case of queue full errors, which surface only on awaiting. while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + let messagesNotAwaited = 0; while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { promises.push(producer.send({ topic, @@ -110,8 +111,12 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe throw err; })); messagesDispatched += batchSize; + messagesNotAwaited += batchSize; + if (messagesNotAwaited >= 10000) + break; } await Promise.all(promises); + promises = []; } let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 33b24b19..d79081e9 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -30,7 +30,7 @@ function baseConfiguration(parameters) { return ret; } -async function runCreateTopics(parameters, topic, topic2) { +async function runCreateTopics(parameters, topic, topic2, numPartitions) { const kafka = new Kafka(baseConfiguration(parameters)); const admin = kafka.admin(); @@ -38,7 +38,7 @@ async function runCreateTopics(parameters, topic, topic2) { for (let t of [topic, topic2]) { let topicCreated = await admin.createTopics({ - topics: [{ topic: t, numPartitions: 3 }], + topics: [{ topic: t, numPartitions }], }).catch(console.error); if (topicCreated) { console.log(`Created topic ${t}`); @@ -50,7 +50,7 @@ async function runCreateTopics(parameters, topic, topic2) { await new Promise(resolve => setTimeout(resolve, 1000)); /* Propagate. */ await admin.createTopics({ topics: [ - { topic: t, numPartitions: 3 }, + { topic: t, numPartitions }, ], }).catch(console.error); console.log(`Created topic ${t}`); @@ -98,6 +98,7 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe // await them all at once. We need the second while loop to keep sending // in case of queue full errors, which surface only on awaiting. while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + let messagesNotAwaited = 0; while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { promises.push(producer.send({ topic, @@ -115,8 +116,12 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe } })); messagesDispatched += batchSize; + messagesNotAwaited += batchSize; + if (messagesNotAwaited >= 10000) + break; } await Promise.all(promises); + promises = []; } let elapsed = hrtime(startTime); let durationNanos = elapsed[0] * 1e9 + elapsed[1]; From a9dd2c2743803c1fe0ca44a37420314abd2735c2 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 10:47:44 +0200 Subject: [PATCH 23/50] Log number of partitions --- examples/performance/performance-consolidated.js | 1 + examples/performance/performance-primitives-kafkajs.js | 4 ++-- examples/performance/performance-primitives.js | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index a5655893..ee551671 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -95,6 +95,7 @@ function logParameters(parameters) { logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Topic2: ${topic2}`); + console.log(` Partitions: ${numPartitions}`); await runCreateTopics(parameters, topic, topic2, numPartitions); } diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 8e154841..e89286dd 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -43,7 +43,7 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { topics: [{ topic: t, numPartitions }], }).catch(console.error); if (topicCreated) { - console.log(`Created topic ${t}`); + console.log(`Created topic ${t} with ${numPartitions} partitions`); continue; } @@ -55,7 +55,7 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { { topic: t, numPartitions }, ], }).catch(console.error); - console.log(`Created topic ${t}`); + console.log(`Created topic ${t} with ${numPartitions} partitions`); } await admin.disconnect(); diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index d79081e9..b17f0d55 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -41,7 +41,7 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { topics: [{ topic: t, numPartitions }], }).catch(console.error); if (topicCreated) { - console.log(`Created topic ${t}`); + console.log(`Created topic ${t} with ${numPartitions} partitions`); continue; } @@ -53,7 +53,7 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { { topic: t, numPartitions }, ], }).catch(console.error); - console.log(`Created topic ${t}`); + console.log(`Created topic ${t} with ${numPartitions} partitions`); await new Promise(resolve => setTimeout(resolve, 1000)); /* Propagate. */ } From b15bc1d23bf4e3237581d9505be44ef06d4319af Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 10:59:13 +0200 Subject: [PATCH 24/50] Fix skipCtpTest --- ci/tests/run_perf_test.js | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 22f2b42e..2c3817c6 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -50,7 +50,8 @@ console.log('Running KafkaJS Producer/Consumer test...'); const outputKjsProducerConsumer = runCommand(`MODE=kafkajs MESSAGE_COUNT=${messageCount} node performance-consolidated.js --create-topics --consumer --producer`); console.log('Running Confluent CTP test...'); -const outputConfluentCtp = runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); +const outputConfluentCtp = skipCtpTest ? '' : + runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); console.log('Running KafkaJS CTP test...'); const outputKjsCtp = skipCtpTest ? '' : From 90966c24dc4feb7508e2b390ba070d0878ea18ce Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 12:59:30 +0200 Subject: [PATCH 25/50] Configurable partitions consumed concurrently --- examples/performance/performance-consolidated.js | 5 +++-- examples/performance/performance-primitives-common.js | 4 +++- examples/performance/performance-primitives-kafkajs.js | 4 ++-- examples/performance/performance-primitives.js | 4 ++-- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index ee551671..7fe31310 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -21,6 +21,7 @@ const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; const compression = process.env.COMPRESSION || 'None'; const numPartitions = process.env.PARTITIONS ? +process.env.PARTITIONS : 3; +const partitionsConsumedConcurrently = process.env.PARTITIONS_CONSUMED_CONCURRENTLY ? +process.env.PARTITIONS_CONSUMED_CONCURRENTLY : 1; const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); const messageProcessTimeMs = process.env.MESSAGE_PROCESS_TIME_MS ? +process.env.MESSAGE_PROCESS_TIME_MS : 5; const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY : 1; @@ -121,7 +122,7 @@ function logParameters(parameters) { console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); startTrackingMemory(); - const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, false, stats); + const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, false, partitionsConsumedConcurrently, stats); endTrackingMemory(`consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate (eachMessage): ", consumerRate); console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); @@ -134,7 +135,7 @@ function logParameters(parameters) { console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); startTrackingMemory(); - const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, true, stats); + const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, true, partitionsConsumedConcurrently, stats); endTrackingMemory(`consumer-memory-batch-${mode}.json`); console.log("=== Consumer Rate (eachBatch): ", consumerRate); console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 90b850c1..96c2eed8 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -1,6 +1,6 @@ const { hrtime } = require('process'); -async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { +async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats) { await consumer.connect(); await consumer.subscribe({ topic }); @@ -17,6 +17,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac console.log("Starting consumer."); let consumeMethod = { + partitionsConsumedConcurrently, eachMessage: async ({ topic, partition, message }) => { messagesReceived++; @@ -38,6 +39,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac } if (eachBatch) { consumeMethod = { + partitionsConsumedConcurrently, eachBatch: async ({ batch }) => { const messagesBeforeBatch = messagesReceived; const topic = batch.topic; diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index e89286dd..2b823d45 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -168,8 +168,8 @@ function newCompatibleConsumer(parameters) { return new CompatibleConsumer(consumer); } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { - return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, stats); +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats) { + return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats); } async function runConsumeTransformProduce(parameters, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index b17f0d55..5a6a0783 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -172,8 +172,8 @@ function newCompatibleConsumer(parameters) { return new CompatibleConsumer(consumer); } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, stats) { - return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, stats); +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats) { + return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats); } async function runConsumeTransformProduce(parameters, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { From a8bb201393277388723b272601ac1427b21970b5 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 15:02:41 +0200 Subject: [PATCH 26/50] Consume rate in msg/s, average batch size, different messages depending on randomness --- .../performance/performance-consolidated.js | 22 ++++++++++++++----- .../performance-primitives-common.js | 3 +++ .../performance-primitives-kafkajs.js | 13 ++++++----- .../performance/performance-primitives.js | 13 ++++++----- 4 files changed, 36 insertions(+), 15 deletions(-) diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 7fe31310..b3b581e5 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -20,6 +20,8 @@ const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 10 const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; const compression = process.env.COMPRESSION || 'None'; +// Between 0 and 1, percentage of random bytes in each message +const randomness = process.env.RANDOMNESS ? +process.env.RANDOMNESS : 0.5; const numPartitions = process.env.PARTITIONS ? +process.env.PARTITIONS : 3; const partitionsConsumedConcurrently = process.env.PARTITIONS_CONSUMED_CONCURRENTLY ? +process.env.PARTITIONS_CONSUMED_CONCURRENTLY : 1; const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); @@ -110,7 +112,8 @@ function logParameters(parameters) { console.log(` Compression: ${compression}`); console.log(` Warmup Messages: ${warmupMessages}`); startTrackingMemory(); - const producerRate = await runProducer(parameters, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + const producerRate = await runProducer(parameters, topic, batchSize, + warmupMessages, messageCount, messageSize, compression, randomness); endTrackingMemory(`producer-memory-${mode}.json`); console.log("=== Producer Rate: ", producerRate); } @@ -121,10 +124,14 @@ function logParameters(parameters) { logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); + console.log(` Partitions consumed concurrently: ${partitionsConsumedConcurrently}`); startTrackingMemory(); - const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, false, partitionsConsumedConcurrently, stats); + const consumerRate = await runConsumer(parameters, topic, + warmupMessages, messageCount, + false, partitionsConsumedConcurrently, stats); endTrackingMemory(`consumer-memory-message-${mode}.json`); - console.log("=== Consumer Rate (eachMessage): ", consumerRate); + console.log("=== Consumer Rate MB/s (eachMessage): ", consumerRate); + console.log("=== Consumer Rate msg/s (eachMessage): ", stats.messageRate); console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); } @@ -134,12 +141,17 @@ function logParameters(parameters) { logParameters(parameters); console.log(` Topic: ${topic}`); console.log(` Message Count: ${messageCount}`); + console.log(` Partitions consumed concurrently: ${partitionsConsumedConcurrently}`); startTrackingMemory(); - const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, true, partitionsConsumedConcurrently, stats); + const consumerRate = await runConsumer(parameters, topic, + warmupMessages, messageCount, + true, partitionsConsumedConcurrently, stats); endTrackingMemory(`consumer-memory-batch-${mode}.json`); - console.log("=== Consumer Rate (eachBatch): ", consumerRate); + console.log("=== Consumer Rate MB/s (eachBatch): ", consumerRate); + console.log("=== Consumer Rate msg/s (eachBatch): ", stats.messageRate); console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); console.log("=== Max eachBatch lag: ", stats.maxOffsetLag); + console.log("=== Average eachBatch size: ", stats.averageBatchSize); console.log("=== Consumption time (eachBatch): ", stats.durationSeconds); } diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 96c2eed8..e2f44f46 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -91,7 +91,10 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (eachBatch) { stats.averageOffsetLag = totalBatches > 0 ? (totalOffsetLag / totalBatches) : 0; stats.maxOffsetLag = maxOffsetLag; + stats.averageBatchSize = totalBatches > 0 ? (messagesMeasured / totalBatches) : 0; } + stats.messageRate = durationSeconds > 0 ? + (messagesMeasured / durationSeconds) : Infinity; stats.durationSeconds = durationSeconds; } return rate; diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 2b823d45..d742a19c 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -61,16 +61,19 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { await admin.disconnect(); } -async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { +async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness) { let totalMessagesSent = 0; let totalBytesSent = 0; - const message = { - value: randomBytes(msgSize), + const messages = Array(batchSize); + let staticValue = randomBytes(Math.floor(msgSize * (1 - randomness))); + for (let i = 0; i < batchSize; i++) { + /* Generate a different random value for each message */ + messages[i] = { + value: Buffer.concat([staticValue, randomBytes(msgSize - staticValue.length)]), + }; } - const messages = Array(batchSize).fill(message); - const kafka = new Kafka(baseConfiguration(parameters)); const producer = kafka.producer(); diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 5a6a0783..052ab78c 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -60,16 +60,19 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { await admin.disconnect(); } -async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression) { +async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness) { let totalMessagesSent = 0; let totalBytesSent = 0; - const message = { - value: randomBytes(msgSize), + const messages = Array(batchSize); + let staticValue = randomBytes(Math.floor(msgSize * (1 - randomness))); + for (let i = 0; i < batchSize; i++) { + /* Generate a different random value for each message */ + messages[i] = { + value: Buffer.concat([staticValue, randomBytes(msgSize - staticValue.length)]), + }; } - const messages = Array(batchSize).fill(message); - const kafka = new Kafka({ ...baseConfiguration(parameters), 'compression.codec': CompressionTypes[compression], From 363d38a5f090b5df2cad5e094cbf2639bfaa9ee9 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 15:20:06 +0200 Subject: [PATCH 27/50] fixup --- ci/tests/run_perf_test.js | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 2c3817c6..307895c9 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -60,9 +60,11 @@ const outputKjsCtp = skipCtpTest ? '' : // Extract Confluent results let ctpConfluent, ctpKjs; const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); -const consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate (eachMessage):'); +const consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); +const consumerConfluentMessageRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); const consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); -const consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate (eachBatch):'); +const consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); +const consumerConfluentBatchRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); const consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); const consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); const consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); @@ -74,9 +76,11 @@ if (!skipCtpTest) { // Extract KafkaJS results const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); -const consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate (eachMessage):'); +const consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); +const consumerKjsMessageRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); const consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); -const consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate (eachBatch):'); +const consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); +const consumerKjsBatchRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); const consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); const consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); const consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); @@ -88,9 +92,11 @@ if (!skipCtpTest) { // Print results console.log(`Producer rates: confluent ${producerConfluent}, kafkajs ${producerKjs}`); -console.log(`Consumer rates (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); +console.log(`Consumer rates MB/s (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); +console.log(`Consumer rates msg/s (eachMessage): confluent ${consumerConfluentMessageRate}, kafkajs ${consumerKjsMessageRate}`); console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); -console.log(`Consumer rates (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); +console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); +console.log(`Consumer rates msg/s (eachBatch): confluent ${consumerConfluentBatchRate}, kafkajs ${consumerKjsBatchRate}`); console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); @@ -110,7 +116,7 @@ if (belowThreshold(producerConfluent, producerKjs, maxPerformanceDifference)) { } if (belowThreshold(consumerConfluentMessage, consumerKjsMessage, maxPerformanceDifference)) { - console.log(`Consumer rates (eachMessage) differ by more than 30%: confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); + console.log(`Consumer rates MB/s (eachMessage) differ by more than 30%: confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); // FIXME: improve consumer performance at least to KafkaJS level errcode = 0; } From 2cb7acde11c695b0c5ffe67277acf88b12e07c67 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 16:22:54 +0200 Subject: [PATCH 28/50] Average and max E2E latency --- ci/tests/run_perf_test.js | 12 ++++++++ .../performance/performance-consolidated.js | 4 +++ .../performance-primitives-common.js | 26 ++++++++++++++++- .../performance-primitives-kafkajs.js | 28 +++++++++++++++---- .../performance/performance-primitives.js | 28 +++++++++++++++---- 5 files changed, 85 insertions(+), 13 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 307895c9..6bea459f 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -62,9 +62,13 @@ let ctpConfluent, ctpKjs; const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); const consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); const consumerConfluentMessageRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); +const consumerConfluentMessageAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); +const consumerConfluentMessageMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); const consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); const consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); const consumerConfluentBatchRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); +const consumerConfluentBatchAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); +const consumerConfluentBatchMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); const consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); const consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); const consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); @@ -78,9 +82,13 @@ if (!skipCtpTest) { const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); const consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); const consumerKjsMessageRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); +const consumerKjsMessageAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); +const consumerKjsMessageMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); const consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); const consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); const consumerKjsBatchRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); +const consumerKjsBatchAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); +const consumerKjsBatchMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); const consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); const consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); const consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); @@ -94,9 +102,13 @@ if (!skipCtpTest) { console.log(`Producer rates: confluent ${producerConfluent}, kafkajs ${producerKjs}`); console.log(`Consumer rates MB/s (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); console.log(`Consumer rates msg/s (eachMessage): confluent ${consumerConfluentMessageRate}, kafkajs ${consumerKjsMessageRate}`); +console.log(`Consumer average E2E latency (eachMessage): confluent ${consumerConfluentMessageAvgLatency}, kafkajs ${consumerKjsMessageAvgLatency}`); +console.log(`Consumer max E2E latency (eachMessage): confluent ${consumerConfluentMessageMaxLatency}, kafkajs ${consumerKjsMessageMaxLatency}`); console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); console.log(`Consumer rates msg/s (eachBatch): confluent ${consumerConfluentBatchRate}, kafkajs ${consumerKjsBatchRate}`); +console.log(`Consumer average E2E latency (eachBatch): confluent ${consumerConfluentBatchAvgLatency}, kafkajs ${consumerKjsBatchAvgLatency}`); +console.log(`Consumer max E2E latency (eachBatch): confluent ${consumerConfluentBatchMaxLatency}, kafkajs ${consumerKjsBatchMaxLatency}`); console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index b3b581e5..83a8fe0b 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -132,6 +132,8 @@ function logParameters(parameters) { endTrackingMemory(`consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate MB/s (eachMessage): ", consumerRate); console.log("=== Consumer Rate msg/s (eachMessage): ", stats.messageRate); + console.log("=== Consumer average E2E latency (eachMessage): ", stats.avgLatency); + console.log("=== Consumer max E2E latency (eachMessage): ", stats.maxLatency); console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); } @@ -152,6 +154,8 @@ function logParameters(parameters) { console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); console.log("=== Max eachBatch lag: ", stats.maxOffsetLag); console.log("=== Average eachBatch size: ", stats.averageBatchSize); + console.log("=== Consumer average E2E latency (eachBatch): ", stats.avgLatency); + console.log("=== Consumer max E2E latency (eachBatch): ", stats.maxLatency); console.log("=== Consumption time (eachBatch): ", stats.durationSeconds); } diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index e2f44f46..e47587ad 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -14,6 +14,23 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac let startTime; let rate; const skippedMessages = warmupMessages; + const decoder = new TextDecoder('utf-8'); + + const updateLatency = (receivedAt, numMessages, message) => { + if (!stats) + return; + + const sentAt = Number(decoder.decode(message.value.slice(0, 13))); + const latency = receivedAt - sentAt; + + if (stats.maxLatency === undefined) { + stats.maxLatency = latency; + stats.avgLatency = latency; + } else { + stats.maxLatency = Math.max(stats.maxLatency, latency); + stats.avgLatency = ((stats.avgLatency * (numMessages - 1)) + latency) / numMessages; + } + }; console.log("Starting consumer."); let consumeMethod = { @@ -24,6 +41,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (messagesReceived >= skippedMessages) { messagesMeasured++; totalMessageSize += message.value.length; + updateLatency(Date.now(), messagesMeasured, message); if (messagesReceived === skippedMessages) { startTime = hrtime.bigint(); @@ -54,8 +72,14 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (messagesBeforeBatch < skippedMessages) { messages = messages.slice(messages.length - messagesMeasured); } - for (const message of messages) + const now = Date.now(); + const messagesBase = messagesMeasured - messages.length; + let i = 1; + for (const message of messages) { totalMessageSize += message.value.length; + updateLatency(now, messagesBase + i, message); + i++; + } if (!startTime) { startTime = hrtime.bigint(); diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index d742a19c..022e54cd 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -65,12 +65,22 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe let totalMessagesSent = 0; let totalBytesSent = 0; - const messages = Array(batchSize); - let staticValue = randomBytes(Math.floor(msgSize * (1 - randomness))); - for (let i = 0; i < batchSize; i++) { + const messages = Array(totalMessageCnt); + const encoder = new TextEncoder(); + let staticValueLength = Math.floor(msgSize * (1 - randomness)); + if (staticValueLength < 13) + staticValueLength = 13; + const staticValueRemainder = staticValueLength - 13; + if (staticValueRemainder > 0) { + staticValueRemainder = randomBytes(staticValueRemainder); + } else { + staticValueRemainder = Buffer.alloc(0); + } + + for (let i = 0; i < totalMessageCnt; i++) { /* Generate a different random value for each message */ messages[i] = { - value: Buffer.concat([staticValue, randomBytes(msgSize - staticValue.length)]), + value: Buffer.concat([staticValueRemainder, randomBytes(msgSize - staticValueLength)]), }; } @@ -83,7 +93,7 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe while (warmupMessages > 0) { await producer.send({ topic, - messages, + messages: messages.slice(0, batchSize), compression: CompressionTypes[compression], }); warmupMessages -= batchSize; @@ -102,9 +112,15 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { let messagesNotAwaited = 0; while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + const modifiedMessages = []; + for (const msg of messages.slice(messagesDispatched, messagesDispatched + batchSize)) { + modifiedMessages.push({ + value: Buffer.concat([encoder.encode(Date.now().toString()), msg.value]) + }); + } promises.push(producer.send({ topic, - messages, + messages: modifiedMessages, compression: CompressionTypes[compression], }).then(() => { totalMessagesSent += batchSize; diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 052ab78c..fd7a9670 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -64,12 +64,22 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe let totalMessagesSent = 0; let totalBytesSent = 0; - const messages = Array(batchSize); - let staticValue = randomBytes(Math.floor(msgSize * (1 - randomness))); - for (let i = 0; i < batchSize; i++) { + const messages = Array(totalMessageCnt); + const encoder = new TextEncoder(); + let staticValueLength = Math.floor(msgSize * (1 - randomness)); + if (staticValueLength < 13) + staticValueLength = 13; + const staticValueRemainder = staticValueLength - 13; + if (staticValueRemainder > 0) { + staticValueRemainder = randomBytes(staticValueRemainder); + } else { + staticValueRemainder = Buffer.alloc(0); + } + + for (let i = 0; i < totalMessageCnt; i++) { /* Generate a different random value for each message */ messages[i] = { - value: Buffer.concat([staticValue, randomBytes(msgSize - staticValue.length)]), + value: Buffer.concat([staticValueRemainder, randomBytes(msgSize - staticValueLength)]), }; } @@ -85,7 +95,7 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe while (warmupMessages > 0) { await producer.send({ topic, - messages, + messages: messages.slice(0, batchSize), }); warmupMessages -= batchSize; } @@ -103,9 +113,15 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { let messagesNotAwaited = 0; while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + const modifiedMessages = []; + for (const msg of messages.slice(messagesDispatched, messagesDispatched + batchSize)) { + modifiedMessages.push({ + value: Buffer.concat([encoder.encode(Date.now().toString()), msg.value]) + }); + } promises.push(producer.send({ topic, - messages, + messages: modifiedMessages, }).then(() => { totalMessagesSent += batchSize; totalBytesSent += batchSize * msgSize; From 5a66af1e3f20a7f8d938eabe4dfe4154acedb9c4 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 14 Oct 2025 17:59:45 +0200 Subject: [PATCH 29/50] Log average and max size --- ci/tests/run_perf_test.js | 3 +++ examples/performance/performance-primitives-common.js | 2 +- examples/performance/performance-primitives-kafkajs.js | 2 +- examples/performance/performance-primitives.js | 2 +- 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 6bea459f..34536da6 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -72,6 +72,7 @@ const consumerConfluentBatchMaxLatency = extractValue(outputConfluentProducerCon const consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); const consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); const consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); +const consumerConfluentBatchAverageSize = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch size:'); const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); if (!skipCtpTest) { @@ -92,6 +93,7 @@ const consumerKjsBatchMaxLatency = extractValue(outputKjsProducerConsumer, '=== const consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); const consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); const consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); +const consumerKjsBatchAverageSize = extractValue(outputKjsProducerConsumer, '=== Average eachBatch size:'); const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); if (!skipCtpTest) { @@ -112,6 +114,7 @@ console.log(`Consumer max E2E latency (eachBatch): confluent ${consumerConfluent console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); +console.log(`Average eachBatch size: confluent ${consumerConfluentBatchAverageSize}, kafkajs ${consumerKjsBatchAverageSize}`); console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); console.log(`Max RSS: confluent ${consumerConfluentMaxRSS}, kafkajs ${consumerKjsMaxRSS}`); if (!skipCtpTest) { diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index e47587ad..a4baa352 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -23,7 +23,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac const sentAt = Number(decoder.decode(message.value.slice(0, 13))); const latency = receivedAt - sentAt; - if (stats.maxLatency === undefined) { + if (!stats.maxLatency) { stats.maxLatency = latency; stats.avgLatency = latency; } else { diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 022e54cd..321028ab 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -70,7 +70,7 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe let staticValueLength = Math.floor(msgSize * (1 - randomness)); if (staticValueLength < 13) staticValueLength = 13; - const staticValueRemainder = staticValueLength - 13; + let staticValueRemainder = staticValueLength - 13; if (staticValueRemainder > 0) { staticValueRemainder = randomBytes(staticValueRemainder); } else { diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index fd7a9670..cc243d12 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -69,7 +69,7 @@ async function runProducer(parameters, topic, batchSize, warmupMessages, totalMe let staticValueLength = Math.floor(msgSize * (1 - randomness)); if (staticValueLength < 13) staticValueLength = 13; - const staticValueRemainder = staticValueLength - 13; + let staticValueRemainder = staticValueLength - 13; if (staticValueRemainder > 0) { staticValueRemainder = randomBytes(staticValueRemainder); } else { From 3243a755d6bb192c995e4c3482fd2aedbbdbc3a1 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 15 Oct 2025 11:51:37 +0200 Subject: [PATCH 30/50] Changes to run the test for a longer time --- ci/tests/run_perf_test.js | 402 ++++++++++++------ .../performance/performance-consolidated.js | 34 +- .../performance-primitives-common.js | 205 ++++++++- .../performance-primitives-kafkajs.js | 135 +++--- .../performance/performance-primitives.js | 143 +++---- 5 files changed, 598 insertions(+), 321 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 34536da6..578de361 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -1,17 +1,14 @@ #!/usr/bin/env node -const { execSync } = require('child_process'); +const util = require('util'); +const exec = util.promisify(require('child_process').exec); -function runCommand(command) { +async function runCommand(command) { try { - const output = execSync(command, { encoding: 'utf8', stdio: 'pipe' }); - console.log(output); - return output; + const output = await exec(command, { encoding: 'utf8', stdio: 'pipe' }); + return output.stdout; } catch (error) { const errorOutput = error.stdout || error.stderr || error.message; - console.log(error.stdout); - console.log(error.stderr); - console.log(error.message); return errorOutput; } } @@ -40,142 +37,283 @@ function belowTarget(value, target) { return belowThreshold(value, target, 1); } -// Run performance tests and store outputs in memory -console.log('Running Confluent Producer/Consumer test...'); -const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 50000; -const skipCtpTest = process.env.SKIP_CTP_TEST ? process.env.SKIP_CTP_TEST === 'true' : false; -const outputConfluentProducerConsumer = runCommand(`MODE=confluent MESSAGE_COUNT=${messageCount} node performance-consolidated.js --create-topics --consumer --producer`); - -console.log('Running KafkaJS Producer/Consumer test...'); -const outputKjsProducerConsumer = runCommand(`MODE=kafkajs MESSAGE_COUNT=${messageCount} node performance-consolidated.js --create-topics --consumer --producer`); - -console.log('Running Confluent CTP test...'); -const outputConfluentCtp = skipCtpTest ? '' : - runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); - -console.log('Running KafkaJS CTP test...'); -const outputKjsCtp = skipCtpTest ? '' : - runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); - -// Extract Confluent results -let ctpConfluent, ctpKjs; -const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); -const consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); -const consumerConfluentMessageRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); -const consumerConfluentMessageAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); -const consumerConfluentMessageMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); -const consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); -const consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); -const consumerConfluentBatchRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); -const consumerConfluentBatchAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); -const consumerConfluentBatchMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); -const consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); -const consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); -const consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); -const consumerConfluentBatchAverageSize = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch size:'); -const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); -const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); -if (!skipCtpTest) { - ctpConfluent = extractValue(outputConfluentCtp, '=== Consume-Transform-Produce Rate:'); -} -// Extract KafkaJS results -const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); -const consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); -const consumerKjsMessageRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); -const consumerKjsMessageAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); -const consumerKjsMessageMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); -const consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); -const consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); -const consumerKjsBatchRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); -const consumerKjsBatchAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); -const consumerKjsBatchMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); -const consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); -const consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); -const consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); -const consumerKjsBatchAverageSize = extractValue(outputKjsProducerConsumer, '=== Average eachBatch size:'); -const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); -const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); -if (!skipCtpTest) { - ctpKjs = extractValue(outputKjsCtp, '=== Consume-Transform-Produce Rate:'); -} -// Print results -console.log(`Producer rates: confluent ${producerConfluent}, kafkajs ${producerKjs}`); -console.log(`Consumer rates MB/s (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); -console.log(`Consumer rates msg/s (eachMessage): confluent ${consumerConfluentMessageRate}, kafkajs ${consumerKjsMessageRate}`); -console.log(`Consumer average E2E latency (eachMessage): confluent ${consumerConfluentMessageAvgLatency}, kafkajs ${consumerKjsMessageAvgLatency}`); -console.log(`Consumer max E2E latency (eachMessage): confluent ${consumerConfluentMessageMaxLatency}, kafkajs ${consumerKjsMessageMaxLatency}`); -console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); -console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); -console.log(`Consumer rates msg/s (eachBatch): confluent ${consumerConfluentBatchRate}, kafkajs ${consumerKjsBatchRate}`); -console.log(`Consumer average E2E latency (eachBatch): confluent ${consumerConfluentBatchAvgLatency}, kafkajs ${consumerKjsBatchAvgLatency}`); -console.log(`Consumer max E2E latency (eachBatch): confluent ${consumerConfluentBatchMaxLatency}, kafkajs ${consumerKjsBatchMaxLatency}`); -console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); -console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); -console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); -console.log(`Average eachBatch size: confluent ${consumerConfluentBatchAverageSize}, kafkajs ${consumerKjsBatchAverageSize}`); -console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); -console.log(`Max RSS: confluent ${consumerConfluentMaxRSS}, kafkajs ${consumerKjsMaxRSS}`); -if (!skipCtpTest) { - console.log(`CTP rates: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); -} -let errcode = 0; -const maxPerformanceDifference = 0.7; +async function main() { + // Run performance tests and store outputs in memory + const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 50000; + const skipCtpTest = process.env.SKIP_CTP_TEST ? process.env.SKIP_CTP_TEST === 'true' : false; + const concurrentRun = process.env.CONCURRENT_RUN ? process.env.CONCURRENT_RUN === 'true' : false; + const consumerMode = process.env.CONSUMER_MODE || 'all'; + const produceToSecondTopic = process.env.PRODUCE_TO_SECOND_TOPIC ? process.env.PRODUCE_TO_SECOND_TOPIC === 'true' : false; + const produceToSecondTopicParam = produceToSecondTopic ? '--produce-to-second-topic' : ''; + const consumerModeAll = consumerMode === 'all'; + const consumerModeEachMessage = consumerMode === 'eachMessage'; + const consumerModeEachBatch = consumerMode === 'eachBatch'; + let consumerParam = '--consumer'; + if (consumerModeEachMessage) { + consumerParam = '--consumer-each-message'; + } else if (consumerModeEachBatch) { + consumerParam = '--consumer-each-batch'; + } + let outputConfluentProducerConsumer; + let outputKjsProducerConsumer; + const groupIdEachMessageConfluent = `test-group-confluent-message-` + Math.random(); + const groupIdEachBatchConfluent = `test-group-confluent-batch-` + Math.random(); + const groupIdEachMessageKafkaJS = `test-group-kafkajs-message-` + Math.random(); + const groupIdEachBatchKafkaJS = `test-group-kafkajs-batch-` + Math.random(); + if (consumerModeAll || consumerModeEachMessage) { + console.log(`Confluent eachMessage group id: ${groupIdEachMessageConfluent}`); + console.log(`KafkaJS eachMessage group id: ${groupIdEachMessageKafkaJS}`); + } + if (consumerModeAll || consumerModeEachBatch) { + console.log(`Confluent eachBatch group id: ${groupIdEachBatchConfluent}`); + console.log(`KafkaJS eachBatch group id: ${groupIdEachBatchKafkaJS}`); + } -// Compare against KJS (30% threshold) -if (belowThreshold(producerConfluent, producerKjs, maxPerformanceDifference)) { - console.log(`Producer rates differ by more than 30%: confluent ${producerConfluent}, kafkajs ${producerKjs}`); - errcode = 1; -} + const runProducerConsumerMode = async (mode) => { + const modeLabel = mode === 'confluent' ? 'Confluent' : 'KafkaJS'; + const groupIdEachMessage = mode === 'confluent' ? groupIdEachMessageConfluent : groupIdEachMessageKafkaJS; + const groupIdEachBatch = mode === 'confluent' ? groupIdEachBatchConfluent : groupIdEachBatchKafkaJS; -if (belowThreshold(consumerConfluentMessage, consumerKjsMessage, maxPerformanceDifference)) { - console.log(`Consumer rates MB/s (eachMessage) differ by more than 30%: confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); - // FIXME: improve consumer performance at least to KafkaJS level - errcode = 0; -} + if (concurrentRun) { + console.log(`Running ${modeLabel} Producer/Consumer test (concurrently)...`); -// Lower is better for time -if (belowThreshold(consumerKjsTime, consumerConfluentTime, maxPerformanceDifference)) { - console.log(`Consumption time (eachMessage) differ by more than 30%: confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); - errcode = 0; -} + await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); + const allPromises = []; + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} node performance-consolidated.js --producer`)); + if (consumerModeAll || consumerModeEachMessage) { + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} GROUPID_MESSAGE=${groupIdEachMessage} node performance-consolidated.js --consumer-each-message ${produceToSecondTopicParam}`)); + } + if (consumerModeAll || consumerModeEachBatch) { + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --consumer-each-batch ${produceToSecondTopicParam}`)); + } + const results = await Promise.allSettled(allPromises); + return results.map(r => r.status === 'fulfilled' ? r.value : '').join('\n'); + } else { + console.log(`Running ${modeLabel} Producer/Consumer test...`); + return runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} GROUPID_MESSAGE=${groupIdEachMessage} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --create-topics ${consumerParam} ${produceToSecondTopicParam} --producer`); + } + } -if (belowThreshold(consumerConfluentBatch, consumerKjsBatch, maxPerformanceDifference)) { - console.log(`Consumer rates (eachBatch) differ by more than 30%: confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); - errcode = 0; -} + const runProducerConsumer = async () => { + outputConfluentProducerConsumer = await runProducerConsumerMode('confluent'); + outputKjsProducerConsumer = await runProducerConsumerMode('kafkajs'); + } -// Lower is better for time -if (belowThreshold(consumerKjsBatchTime, consumerConfluentBatchTime, maxPerformanceDifference)) { - console.log(`Consumption time (eachBatch) differ by more than 30%: confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); - errcode = 0; -} + await runProducerConsumer(); + console.log(outputConfluentProducerConsumer); + console.log(outputKjsProducerConsumer); -if (!skipCtpTest && belowThreshold(ctpConfluent, ctpKjs, maxPerformanceDifference)) { - console.log(`CTP rates differ by more than 30%: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); - errcode = 1; -} + console.log('Running Confluent CTP test...'); + const outputConfluentCtp = skipCtpTest ? '' : + runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); -// Compare against target numbers -const TARGET_PRODUCE = process.env.TARGET_PRODUCE_PERFORMANCE || '35'; -const TARGET_CONSUME = process.env.TARGET_CONSUME_PERFORMANCE || '18'; -const TARGET_CTP = process.env.TARGET_CTP_PERFORMANCE || '0.02'; + console.log('Running KafkaJS CTP test...'); + const outputKjsCtp = skipCtpTest ? '' : + runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); -if (belowTarget(producerConfluent, TARGET_PRODUCE)) { - console.log(`Confluent producer rate is below target: ${producerConfluent}`); - errcode = 1; -} + // Extract Confluent results + let ctpConfluent, ctpKjs; + let consumerConfluentMessage; + let consumerConfluentMessageRate; + let consumerConfluentMessageAvgLatency; + let consumerConfluentMessageMaxLatency; + let consumerConfluentTime; + let consumerConfluentMessageAverageRSS; + let consumerConfluentMessageMaxRSS; + let consumerConfluentBatch; + let consumerConfluentBatchRate; + let consumerConfluentBatchAvgLatency; + let consumerConfluentBatchMaxLatency; + let consumerConfluentBatchTime; + let consumerConfluentBatchAverageLag; + let consumerConfluentBatchMaxLag; + let consumerConfluentBatchAverageSize; + let consumerConfluentBatchAverageRSS; + let consumerConfluentBatchMaxRSS; -if (belowTarget(consumerConfluentMessage, TARGET_CONSUME)) { - console.log(`Confluent consumer rate is below target: ${consumerConfluentMessage}`); - errcode = 1; -} + const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); + const producerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average producer RSS KB:'); + const producerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max producer RSS KB:'); + if (consumerModeAll || consumerModeEachMessage) { + consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); + consumerConfluentMessageRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); + consumerConfluentMessageAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); + consumerConfluentMessageMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); + consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); + consumerConfluentMessageAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average consumer-each-message RSS KB:'); + consumerConfluentMessageMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-message RSS KB:'); + } + if (consumerModeAll || consumerModeEachBatch) { + consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); + consumerConfluentBatchRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); + consumerConfluentBatchAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); + consumerConfluentBatchMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); + consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); + consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); + consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); + consumerConfluentBatchAverageSize = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch size:'); + consumerConfluentBatchAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average consumer-each-batch RSS KB:'); + consumerConfluentBatchMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-batch RSS KB:'); + } + const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); + const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); + if (!skipCtpTest) { + ctpConfluent = extractValue(outputConfluentCtp, '=== Consume-Transform-Produce Rate:'); + } + + // Extract KafkaJS results + let consumerKjsMessage; + let consumerKjsMessageRate; + let consumerKjsMessageAvgLatency; + let consumerKjsMessageMaxLatency; + let consumerKjsTime; + let consumerKjsMessageAverageRSS; + let consumerKjsMessageMaxRSS; + let consumerKjsBatch; + let consumerKjsBatchRate; + let consumerKjsBatchAvgLatency; + let consumerKjsBatchMaxLatency; + let consumerKjsBatchTime; + let consumerKjsBatchAverageLag; + let consumerKjsBatchMaxLag; + let consumerKjsBatchAverageSize; + let consumerKjsBatchAverageRSS; + let consumerKjsBatchMaxRSS; + + const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); + const producerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average producer RSS KB:'); + const producerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max producer RSS KB:'); + if (consumerModeAll || consumerModeEachMessage) { + consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); + consumerKjsMessageRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); + consumerKjsMessageAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); + consumerKjsMessageMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); + consumerKjsMessageAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average consumer-each-message RSS KB:'); + consumerKjsMessageMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-message RSS KB:'); + } + if (consumerModeAll || consumerModeEachBatch) { + consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); + consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); + consumerKjsBatchRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); + consumerKjsBatchAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); + consumerKjsBatchMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); + consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); + consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); + consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); + consumerKjsBatchAverageSize = extractValue(outputKjsProducerConsumer, '=== Average eachBatch size:'); + consumerKjsBatchAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average consumer-each-batch RSS KB:'); + consumerKjsBatchMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-batch RSS KB:'); + } + const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); + const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); + if (!skipCtpTest) { + ctpKjs = extractValue(outputKjsCtp, '=== Consume-Transform-Produce Rate:'); + } + + // Print results + console.log(`Producer rates: confluent ${producerConfluent}, kafkajs ${producerKjs}`); + console.log(`Average RSS (produce): confluent ${producerConfluentAverageRSS}, kafkajs ${producerKjsAverageRSS}`); + console.log(`Max RSS (produce): confluent ${producerConfluentMaxRSS}, kafkajs ${producerKjsMaxRSS}`); + if (consumerModeAll || consumerModeEachMessage) { + console.log(`Consumer rates MB/s (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); + console.log(`Consumer rates msg/s (eachMessage): confluent ${consumerConfluentMessageRate}, kafkajs ${consumerKjsMessageRate}`); + console.log(`Consumer average E2E latency (eachMessage): confluent ${consumerConfluentMessageAvgLatency}, kafkajs ${consumerKjsMessageAvgLatency}`); + console.log(`Consumer max E2E latency (eachMessage): confluent ${consumerConfluentMessageMaxLatency}, kafkajs ${consumerKjsMessageMaxLatency}`); + console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); + console.log(`Average RSS (eachMessage): confluent ${consumerConfluentMessageAverageRSS}, kafkajs ${consumerKjsMessageAverageRSS}`); + console.log(`Max RSS (eachMessage): confluent ${consumerConfluentMessageMaxRSS}, kafkajs ${consumerKjsMessageMaxRSS}`); + } + if (consumerModeAll || consumerModeEachBatch) { + console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); + console.log(`Consumer rates msg/s (eachBatch): confluent ${consumerConfluentBatchRate}, kafkajs ${consumerKjsBatchRate}`); + console.log(`Consumer average E2E latency (eachBatch): confluent ${consumerConfluentBatchAvgLatency}, kafkajs ${consumerKjsBatchAvgLatency}`); + console.log(`Consumer max E2E latency (eachBatch): confluent ${consumerConfluentBatchMaxLatency}, kafkajs ${consumerKjsBatchMaxLatency}`); + console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); + console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); + console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); + console.log(`Average eachBatch size: confluent ${consumerConfluentBatchAverageSize}, kafkajs ${consumerKjsBatchAverageSize}`); + console.log(`Average RSS (eachBatch): confluent ${consumerConfluentBatchAverageRSS}, kafkajs ${consumerKjsBatchAverageRSS}`); + console.log(`Max RSS (eachBatch): confluent ${consumerConfluentBatchMaxRSS}, kafkajs ${consumerKjsBatchMaxRSS}`); + } + if (!concurrentRun) { + console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); + console.log(`Max RSS: confluent ${consumerConfluentMaxRSS}, kafkajs ${consumerKjsMaxRSS}`); + } + if (!skipCtpTest) { + console.log(`CTP rates: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); + } + + let errcode = 0; + const maxPerformanceDifference = 0.7; + + // Compare against KJS (30% threshold) + if (belowThreshold(producerConfluent, producerKjs, maxPerformanceDifference)) { + console.log(`Producer rates differ by more than 30%: confluent ${producerConfluent}, kafkajs ${producerKjs}`); + errcode = 1; + } + + if (consumerModeAll || consumerModeEachMessage) { + if (belowThreshold(consumerConfluentMessage, consumerKjsMessage, maxPerformanceDifference)) { + console.log(`Consumer rates MB/s (eachMessage) differ by more than 30%: confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); + // FIXME: improve consumer performance at least to KafkaJS level + errcode = 0; + } + + // Lower is better for time + if (belowThreshold(consumerKjsTime, consumerConfluentTime, maxPerformanceDifference)) { + console.log(`Consumption time (eachMessage) differ by more than 30%: confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); + errcode = 0; + } + } + + if (consumerModeAll || consumerModeEachBatch) { + if (belowThreshold(consumerConfluentBatch, consumerKjsBatch, maxPerformanceDifference)) { + console.log(`Consumer rates (eachBatch) differ by more than 30%: confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); + errcode = 0; + } + + // Lower is better for time + if (belowThreshold(consumerKjsBatchTime, consumerConfluentBatchTime, maxPerformanceDifference)) { + console.log(`Consumption time (eachBatch) differ by more than 30%: confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); + errcode = 0; + } + } + + if (!skipCtpTest && belowThreshold(ctpConfluent, ctpKjs, maxPerformanceDifference)) { + console.log(`CTP rates differ by more than 30%: confluent ${ctpConfluent}, kafkajs ${ctpKjs}`); + errcode = 1; + } + + // Compare against target numbers + const TARGET_PRODUCE = process.env.TARGET_PRODUCE_PERFORMANCE || '35'; + const TARGET_CONSUME = process.env.TARGET_CONSUME_PERFORMANCE || '18'; + const TARGET_CTP = process.env.TARGET_CTP_PERFORMANCE || '0.02'; + + if (belowTarget(producerConfluent, TARGET_PRODUCE)) { + console.log(`Confluent producer rate is below target: ${producerConfluent}`); + errcode = 1; + } + + if ((consumerModeAll || consumerModeEachMessage) && belowTarget(consumerConfluentMessage, TARGET_CONSUME)) { + console.log(`Confluent consumer rate (eachMessage) is below target: ${consumerConfluentMessage}`); + errcode = 1; + } + + if ((consumerModeAll || consumerModeEachBatch) && belowTarget(consumerConfluentBatch, TARGET_CONSUME)) { + console.log(`Confluent consumer rate (eachBatch) is below target: ${consumerConfluentBatch}`); + errcode = 1; + } + + if (!skipCtpTest && belowTarget(ctpConfluent, TARGET_CTP)) { + console.log(`Confluent CTP rate is below target: ${ctpConfluent}`); + errcode = 1; + } -if (!skipCtpTest && belowTarget(ctpConfluent, TARGET_CTP)) { - console.log(`Confluent CTP rate is below target: ${ctpConfluent}`); - errcode = 1; + process.exit(errcode); } -process.exit(errcode); \ No newline at end of file +if (require.main === module) + main(); \ No newline at end of file diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 83a8fe0b..f3217c06 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -29,6 +29,7 @@ const messageProcessTimeMs = process.env.MESSAGE_PROCESS_TIME_MS ? +process.env. const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY : 1; const consumerProcessingTime = process.env.CONSUMER_PROCESSING_TIME ? +process.env.CONSUMER_PROCESSING_TIME : 100; const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.env.PRODUCER_PROCESSING_TIME : 100; +const limitRPS = process.env.LIMIT_RPS ? +process.env.LIMIT_RPS : null; const parameters = { brokers, securityProtocol, @@ -50,6 +51,9 @@ function logParameters(parameters) { (async function () { const producer = process.argv.includes('--producer'); const consumer = process.argv.includes('--consumer'); + const consumerEachMessage = process.argv.includes('--consumer-each-message'); + const consumerEachBatch = process.argv.includes('--consumer-each-batch'); + const produceToSecondTopic = process.argv.includes('--produce-to-second-topic'); const ctp = process.argv.includes('--ctp'); const produceConsumeLatency = process.argv.includes('--latency'); const all = process.argv.includes('--all'); @@ -69,17 +73,17 @@ function logParameters(parameters) { const datapointToJSON = (m) => ({ rss: m.rss.toString(), timestamp: m.timestamp.toString() }); - const endTrackingMemory = (fileName) => { + const endTrackingMemory = (name, fileName) => { clearInterval(interval); interval = null; const averageRSS = measures.reduce((sum, m) => sum + m.rss, 0) / measures.length; const averageRSSKB = averageRSS / 1024; maxAverageRSSKB = !maxAverageRSSKB ? averageRSSKB : Math.max(averageRSSKB, maxAverageRSSKB); - console.log(`=== Average RSS: ${averageRSSKB} KB`); + console.log(`=== Average ${name} RSS KB: ${averageRSSKB}`); const max = measures.reduce((prev, current) => (prev.rss > current.rss) ? prev : current); const maxRSSKB = max.rss / 1024; maxMaxRSSKB = !maxMaxRSSKB ? maxRSSKB : Math.max(maxRSSKB, maxMaxRSSKB); - console.log(`=== Max RSS: ${maxRSSKB} KB at ${new Date(max.timestamp).toISOString()}`); + console.log(`=== Max ${name} RSS KB: ${maxRSSKB}`); if (fileName) { const measuresJSON = JSON.stringify({ measures: measures.map(datapointToJSON), @@ -110,15 +114,17 @@ function logParameters(parameters) { console.log(` Message Size: ${messageSize}`); console.log(` Batch Size: ${batchSize}`); console.log(` Compression: ${compression}`); + console.log(` Limit RPS: ${limitRPS}`); console.log(` Warmup Messages: ${warmupMessages}`); startTrackingMemory(); const producerRate = await runProducer(parameters, topic, batchSize, - warmupMessages, messageCount, messageSize, compression, randomness); - endTrackingMemory(`producer-memory-${mode}.json`); + warmupMessages, messageCount, messageSize, compression, + randomness, limitRPS); + endTrackingMemory('producer', `producer-memory-${mode}.json`); console.log("=== Producer Rate: ", producerRate); } - if (consumer || all) { + if (consumer || consumerEachMessage || all) { // If user runs this without --producer then they are responsible for seeding the topic. console.log("=== Running Basic Consumer Performance Test (eachMessage):") logParameters(parameters); @@ -128,8 +134,9 @@ function logParameters(parameters) { startTrackingMemory(); const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, - false, partitionsConsumedConcurrently, stats); - endTrackingMemory(`consumer-memory-message-${mode}.json`); + false, partitionsConsumedConcurrently, stats, + produceToSecondTopic ? topic2 : null, compression); + endTrackingMemory('consumer-each-message', `consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate MB/s (eachMessage): ", consumerRate); console.log("=== Consumer Rate msg/s (eachMessage): ", stats.messageRate); console.log("=== Consumer average E2E latency (eachMessage): ", stats.avgLatency); @@ -137,7 +144,7 @@ function logParameters(parameters) { console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); } - if (consumer || all) { + if (consumer || consumerEachBatch || all) { // If user runs this without --producer then they are responsible for seeding the topic. console.log("=== Running Basic Consumer Performance Test (eachBatch):") logParameters(parameters); @@ -147,8 +154,9 @@ function logParameters(parameters) { startTrackingMemory(); const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, - true, partitionsConsumedConcurrently, stats); - endTrackingMemory(`consumer-memory-batch-${mode}.json`); + true, partitionsConsumedConcurrently, stats, + produceToSecondTopic ? topic2 : null, compression); + endTrackingMemory('consumer-each-batch', `consumer-memory-batch-${mode}.json`); console.log("=== Consumer Rate MB/s (eachBatch): ", consumerRate); console.log("=== Consumer Rate msg/s (eachBatch): ", stats.messageRate); console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); @@ -169,7 +177,7 @@ function logParameters(parameters) { await runProducer(parameters, topic, batchSize, warmupMessages, messageCount, messageSize, compression); startTrackingMemory(); const ctpRate = await runConsumeTransformProduce(parameters, topic, topic2, warmupMessages, messageCount, messageProcessTimeMs, ctpConcurrency); - endTrackingMemory(`consume-transform-produce-${mode}.json`); + endTrackingMemory('consume-transform-produce', `consume-transform-produce-${mode}.json`); console.log("=== Consume-Transform-Produce Rate: ", ctpRate); } @@ -182,7 +190,7 @@ function logParameters(parameters) { console.log(` Producer Processing Time: ${producerProcessingTime}`); startTrackingMemory(); const { mean, p50, p90, p95, outliers } = await runProducerConsumerTogether(parameters, topic, messageCount, messageSize, producerProcessingTime, consumerProcessingTime); - endTrackingMemory(`producer-consumer-together-${mode}.json`); + endTrackingMemory('producer-consumer-together', `producer-consumer-together-${mode}.json`); console.log(`=== Produce-To-Consume Latency (ms): Mean: ${mean}, P50: ${p50}, P90: ${p90}, P95: ${p95}`); // The presence of outliers invalidates the mean measurement, and rasies concerns as to why there are any. diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index a4baa352..e48da4e0 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -1,6 +1,62 @@ const { hrtime } = require('process'); +const { randomBytes } = require('crypto'); -async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats) { +const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; +const AUTO_COMMIT = process.env.AUTO_COMMIT || 'false'; +let autoCommit; +if (AUTO_COMMIT && AUTO_COMMIT === 'false') + autoCommit = null; +else { + autoCommit = Number(AUTO_COMMIT); + if (isNaN(autoCommit)) { + autoCommit = null; + } +} + +function installHandlers() { + const handlers = { + terminationRequested: false, + terminateTimeout: null, + }; + const terminationRequestedCallback = () => { + console.log('Termination requested, waiting for current operation to finish...'); + handlers.terminationRequested = true; + } + process.on('SIGINT', terminationRequestedCallback); + process.on('SIGTERM', terminationRequestedCallback); + handlers.terminationRequestedCallback = terminationRequestedCallback; + handlers.terminateTimeout = setTimeout(terminationRequestedCallback, + TERMINATE_TIMEOUT_MS); + return handlers; +} + +function removeHandlers(handlers) { + process.off('SIGINT', handlers.terminationRequestedCallback); + process.off('SIGTERM', handlers.terminationRequestedCallback); + clearTimeout(handlers.terminateTimeout); +} + +function getAutoCommit() { + return autoCommit; +} + +function genericProduceToTopic(producer, topic, messages) { + return producer.send({ + topic, + messages, + }).catch((err) => { + if (producer.isQueueFullError(err)) { + /* do just send them again */ + return genericProduceToTopic(producer, topic, messages); + } else { + console.error(err); + throw err; + } + }); +} + +async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages) { + const handlers = installHandlers(); await consumer.connect(); await consumer.subscribe({ topic }); @@ -13,6 +69,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac let durationSeconds = 0; let startTime; let rate; + let consumptionStopped = false; const skippedMessages = warmupMessages; const decoder = new TextDecoder('utf-8'); @@ -32,11 +89,30 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac } }; + const shouldTerminate = () => { + return handlers.terminationRequested || + (totalMessageCnt > 0 && messagesMeasured >= totalMessageCnt); + }; + + const stopConsuming = () => { + if (consumptionStopped) + return; + consumptionStopped = true; + let durationNanos = Number(hrtime.bigint() - startTime); + durationSeconds = durationNanos / 1e9; + rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); + consumer.pause([{ topic }]); + } + console.log("Starting consumer."); let consumeMethod = { partitionsConsumedConcurrently, eachMessage: async ({ topic, partition, message }) => { messagesReceived++; + if (actionOnMessages) { + await actionOnMessages([message]); + } if (messagesReceived >= skippedMessages) { messagesMeasured++; @@ -45,12 +121,8 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (messagesReceived === skippedMessages) { startTime = hrtime.bigint(); - } else if (messagesMeasured === totalMessageCnt) { - let durationNanos = Number(hrtime.bigint() - startTime); - durationSeconds = durationNanos / 1e9; - rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); + } else if (totalMessageCnt > 0 && messagesMeasured === totalMessageCnt) { + stopConsuming(); } } } @@ -61,6 +133,10 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac eachBatch: async ({ batch }) => { const messagesBeforeBatch = messagesReceived; const topic = batch.topic; + if (actionOnMessages) { + await actionOnMessages(batch.messages); + } + messagesReceived += batch.messages.length; if (messagesReceived >= skippedMessages) { const offsetLag = batch.offsetLag(); @@ -83,7 +159,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (!startTime) { startTime = hrtime.bigint(); - } else if (messagesMeasured >= totalMessageCnt) { + } else if (totalMessageCnt > 0 && messagesMeasured >= totalMessageCnt) { let durationNanos = Number(hrtime.bigint() - startTime); durationSeconds = durationNanos / 1e9; rate = durationNanos === 0 ? Infinity : @@ -102,7 +178,8 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac await new Promise((resolve) => { let interval = setInterval(() => { - if (messagesMeasured >= totalMessageCnt) { + if (shouldTerminate()) { + stopConsuming(); clearInterval(interval); resolve(); } @@ -121,9 +198,117 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac (messagesMeasured / durationSeconds) : Infinity; stats.durationSeconds = durationSeconds; } + removeHandlers(handlers); + return rate; +} + +async function runProducer(producer, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS) { + const handlers = installHandlers(); + let totalMessagesSent = 0; + let totalBytesSent = 0; + + const encoder = new TextEncoder(); + let staticValueLength = Math.floor(msgSize * (1 - randomness)); + if (staticValueLength < 13) + staticValueLength = 13; + let staticValueRemainder = staticValueLength - 13; + if (staticValueRemainder > 0) { + staticValueRemainder = randomBytes(staticValueRemainder); + } else { + staticValueRemainder = Buffer.alloc(0); + } + + let messageCnt = totalMessageCnt; + if (messageCnt === -1) { + messageCnt = batchSize * 10000; + } + const messages = Array(messageCnt); + for (let i = 0; i < messageCnt; i++) { + /* Generate a different random value for each message */ + messages[i] = { + value: Buffer.concat([staticValueRemainder, randomBytes(msgSize - staticValueLength)]), + }; + } + + await producer.connect(); + + console.log('Sending ' + warmupMessages + ' warmup messages.'); + while (warmupMessages > 0) { + await producer.send({ + topic, + messages: messages.slice(0, batchSize) + }, compression); + warmupMessages -= batchSize; + } + console.log('Sent warmup messages'); + + // Now that warmup is done, start measuring... + let startTime; + let promises = []; + startTime = hrtime(); + let messagesDispatched = 0; + + // The double while-loop allows us to send a bunch of messages and then + // await them all at once. We need the second while loop to keep sending + // in case of queue full errors, which surface only on awaiting. + while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + const startTimeBatch = hrtime.bigint(); + const maxToAwait = limitRPS ? limitRPS : 10000; + let messagesNotAwaited = 0; + while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { + const modifiedMessages = []; + const batchStart = messagesDispatched % messageCnt; + for (const msg of messages.slice(batchStart, batchStart + batchSize)) { + modifiedMessages.push({ + value: Buffer.concat([encoder.encode(Date.now().toString()), msg.value]) + }); + } + promises.push(producer.send({ + topic, + messages: modifiedMessages, + }, compression).then(() => { + totalMessagesSent += batchSize; + totalBytesSent += batchSize * msgSize; + }).catch((err) => { + if (producer.isQueueFullError(err)) { + /* do nothing, just send them again */ + messagesDispatched -= batchSize; + totalMessagesSent -= batchSize; + totalBytesSent -= batchSize * msgSize; + } else { + console.error(err); + throw err; + } + })); + messagesDispatched += batchSize; + messagesNotAwaited += batchSize; + if (handlers.terminationRequested || messagesNotAwaited >= maxToAwait) + break; + } + await Promise.all(promises); + promises = []; + if (handlers.terminationRequested) + break; + + const now = hrtime.bigint(); + const elapsedBatch = now - startTimeBatch; + if (limitRPS && elapsedBatch < 1000000000n) { + await new Promise(resolve => setTimeout(resolve, Number(1000000000n - elapsedBatch) / 1e6)); + } + } + let elapsed = hrtime(startTime); + let durationNanos = elapsed[0] * 1e9 + elapsed[1]; + let rate = (totalBytesSent / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ + console.log(`Sent ${totalMessagesSent} messages, ${totalBytesSent} bytes; rate is ${rate} MB/s`); + + await producer.disconnect(); + removeHandlers(handlers); return rate; } module.exports = { - runConsumer + runConsumer, + runProducer, + genericProduceToTopic, + getAutoCommit, }; \ No newline at end of file diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 321028ab..240e931c 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -1,7 +1,12 @@ const { Kafka, CompressionTypes } = require('kafkajs'); const { randomBytes } = require('crypto'); const { hrtime } = require('process'); -const { runConsumer: runConsumerCommon } = require('./performance-primitives-common'); +const { + runConsumer: runConsumerCommon, + runProducer: runProducerCommon, + genericProduceToTopic, + getAutoCommit, + } = require('./performance-primitives-common'); module.exports = { runProducer, @@ -61,89 +66,38 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { await admin.disconnect(); } -async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness) { - let totalMessagesSent = 0; - let totalBytesSent = 0; - - const messages = Array(totalMessageCnt); - const encoder = new TextEncoder(); - let staticValueLength = Math.floor(msgSize * (1 - randomness)); - if (staticValueLength < 13) - staticValueLength = 13; - let staticValueRemainder = staticValueLength - 13; - if (staticValueRemainder > 0) { - staticValueRemainder = randomBytes(staticValueRemainder); - } else { - staticValueRemainder = Buffer.alloc(0); +class CompatibleProducer { + constructor(producer) { + this.producer = producer; } - for (let i = 0; i < totalMessageCnt; i++) { - /* Generate a different random value for each message */ - messages[i] = { - value: Buffer.concat([staticValueRemainder, randomBytes(msgSize - staticValueLength)]), - }; + async connect() { + return this.producer.connect(); } - const kafka = new Kafka(baseConfiguration(parameters)); + async disconnect() { + return this.producer.disconnect(); + } - const producer = kafka.producer(); - await producer.connect(); + isQueueFullError(err) { + return false; + } - console.log('Sending ' + warmupMessages + ' warmup messages.'); - while (warmupMessages > 0) { - await producer.send({ - topic, - messages: messages.slice(0, batchSize), + send(opts, compression) { + return this.producer.send({ + ...opts, compression: CompressionTypes[compression], }); - warmupMessages -= batchSize; } - console.log('Sent warmup messages'); - - // Now that warmup is done, start measuring... - let startTime; - let promises = []; - startTime = hrtime(); - let messagesDispatched = 0; - - // The double while-loop allows us to send a bunch of messages and then - // await them all at once. We need the second while loop to keep sending - // in case of queue full errors, which surface only on awaiting. - while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { - let messagesNotAwaited = 0; - while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { - const modifiedMessages = []; - for (const msg of messages.slice(messagesDispatched, messagesDispatched + batchSize)) { - modifiedMessages.push({ - value: Buffer.concat([encoder.encode(Date.now().toString()), msg.value]) - }); - } - promises.push(producer.send({ - topic, - messages: modifiedMessages, - compression: CompressionTypes[compression], - }).then(() => { - totalMessagesSent += batchSize; - totalBytesSent += batchSize * msgSize; - }).catch((err) => { - console.error(err); - throw err; - })); - messagesDispatched += batchSize; - messagesNotAwaited += batchSize; - if (messagesNotAwaited >= 10000) - break; - } - await Promise.all(promises); - promises = []; - } - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - let rate = (totalBytesSent / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Sent ${totalMessagesSent} messages, ${totalBytesSent} bytes; rate is ${rate} MB/s`); +} +function newCompatibleProducer(parameters, compression) { + return new CompatibleProducer( + new Kafka(baseConfiguration(parameters) + ).producer()); +} - await producer.disconnect(); - return rate; +async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS) { + return runProducerCommon(newCompatibleProducer(parameters, compression), topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS); } class CompatibleConsumer { @@ -171,24 +125,47 @@ class CompatibleConsumer { } run(opts) { + const autoCommit = getAutoCommit(); + const autoCommitOpts = autoCommit > 0 ? + { autoCommit: true, autoCommitInterval: autoCommit } : + { autoCommit: false }; return this.consumer.run({ ...opts, - autoCommit: false, + ...autoCommitOpts, }); } } -function newCompatibleConsumer(parameters) { + +function newCompatibleConsumer(parameters, eachBatch) { const kafka = new Kafka(baseConfiguration(parameters)); + let groupId = eachBatch ? process.env.GROUPID_BATCH : process.env.GROUPID_MESSAGE; + if (!groupId) { + groupId = 'test-group' + Math.random(); + } + console.log(`New KafkaJS group id: ${groupId}`); const consumer = kafka.consumer({ - groupId: 'test-group' + Math.random(), + groupId, }); return new CompatibleConsumer(consumer); } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats) { - return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats); + +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression) { + let actionOnMessages = null; + let producer; + if (produceToTopic) { + producer = newCompatibleProducer(parameters, produceCompression); + await producer.connect(); + actionOnMessages = (messages) => + genericProduceToTopic(producer, produceToTopic, messages); + } + const ret = await runConsumerCommon(newCompatibleConsumer(parameters, eachBatch), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages); + if (producer) { + await producer.disconnect(); + } + return ret; } async function runConsumeTransformProduce(parameters, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index cc243d12..a9d06d39 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -1,7 +1,12 @@ const { Kafka, ErrorCodes, CompressionTypes } = require('../../').KafkaJS; const { randomBytes } = require('crypto'); const { hrtime } = require('process'); -const { runConsumer: runConsumerCommon } = require('./performance-primitives-common'); +const { + runConsumer: runConsumerCommon, + runProducer: runProducerCommon, + genericProduceToTopic, + getAutoCommit, + } = require('./performance-primitives-common'); module.exports = { runProducer, @@ -60,95 +65,37 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { await admin.disconnect(); } -async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness) { - let totalMessagesSent = 0; - let totalBytesSent = 0; - - const messages = Array(totalMessageCnt); - const encoder = new TextEncoder(); - let staticValueLength = Math.floor(msgSize * (1 - randomness)); - if (staticValueLength < 13) - staticValueLength = 13; - let staticValueRemainder = staticValueLength - 13; - if (staticValueRemainder > 0) { - staticValueRemainder = randomBytes(staticValueRemainder); - } else { - staticValueRemainder = Buffer.alloc(0); +class CompatibleProducer { + constructor(producer) { + this.producer = producer; } - for (let i = 0; i < totalMessageCnt; i++) { - /* Generate a different random value for each message */ - messages[i] = { - value: Buffer.concat([staticValueRemainder, randomBytes(msgSize - staticValueLength)]), - }; + async connect() { + return this.producer.connect(); } - const kafka = new Kafka({ - ...baseConfiguration(parameters), - 'compression.codec': CompressionTypes[compression], - }); - - const producer = kafka.producer(); - await producer.connect(); + async disconnect() { + return this.producer.disconnect(); + } - console.log('Sending ' + warmupMessages + ' warmup messages.'); - while (warmupMessages > 0) { - await producer.send({ - topic, - messages: messages.slice(0, batchSize), - }); - warmupMessages -= batchSize; + isQueueFullError(err) { + return err.code === ErrorCodes.ERR__QUEUE_FULL; } - console.log('Sent warmup messages'); - // Now that warmup is done, start measuring... - let startTime; - let promises = []; - startTime = hrtime(); - let messagesDispatched = 0; - - // The double while-loop allows us to send a bunch of messages and then - // await them all at once. We need the second while loop to keep sending - // in case of queue full errors, which surface only on awaiting. - while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { - let messagesNotAwaited = 0; - while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { - const modifiedMessages = []; - for (const msg of messages.slice(messagesDispatched, messagesDispatched + batchSize)) { - modifiedMessages.push({ - value: Buffer.concat([encoder.encode(Date.now().toString()), msg.value]) - }); - } - promises.push(producer.send({ - topic, - messages: modifiedMessages, - }).then(() => { - totalMessagesSent += batchSize; - totalBytesSent += batchSize * msgSize; - }).catch((err) => { - if (err.code === ErrorCodes.ERR__QUEUE_FULL) { - /* do nothing, just send them again */ - messagesDispatched -= batchSize; - } else { - console.error(err); - throw err; - } - })); - messagesDispatched += batchSize; - messagesNotAwaited += batchSize; - if (messagesNotAwaited >= 10000) - break; - } - await Promise.all(promises); - promises = []; + send(opts) { + return this.producer.send(opts); } - let elapsed = hrtime(startTime); - let durationNanos = elapsed[0] * 1e9 + elapsed[1]; - let rate = (totalBytesSent / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Sent ${totalMessagesSent} messages, ${totalBytesSent} bytes; rate is ${rate} MB/s`); +} +function newCompatibleProducer(parameters, compression) { + return new CompatibleProducer( + new Kafka({ + ...baseConfiguration(parameters), + 'compression.codec': CompressionTypes[compression], + }).producer()); +} - await producer.disconnect(); - return rate; +async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS) { + return runProducerCommon(newCompatibleProducer(parameters, compression), topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS); } class CompatibleConsumer { @@ -179,20 +126,42 @@ class CompatibleConsumer { } } -function newCompatibleConsumer(parameters) { +function newCompatibleConsumer(parameters, eachBatch) { const kafka = new Kafka(baseConfiguration(parameters)); - + const autoCommit = getAutoCommit(); + const autoCommitOpts = autoCommit > 0 ? + { 'enable.auto.commit': true, 'auto.commit.interval.ms': autoCommit } : + { 'enable.auto.commit': false }; + + let groupId = eachBatch ? process.env.GROUPID_BATCH : process.env.GROUPID_MESSAGE; + if (!groupId) { + groupId = 'test-group' + Math.random(); + } + console.log(`New Confluent group id: ${groupId}`); const consumer = kafka.consumer({ - 'group.id': 'test-group' + Math.random(), - 'enable.auto.commit': false, + 'group.id': groupId, 'auto.offset.reset': 'earliest', 'fetch.queue.backoff.ms': '100', + ...autoCommitOpts, }); return new CompatibleConsumer(consumer); } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats) { - return runConsumerCommon(newCompatibleConsumer(parameters), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats); + +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression) { + let actionOnMessages = null; + let producer; + if (produceToTopic) { + producer = newCompatibleProducer(parameters, produceCompression); + await producer.connect(); + actionOnMessages = (messages) => + genericProduceToTopic(producer, produceToTopic, messages); + } + const ret = await runConsumerCommon(newCompatibleConsumer(parameters, eachBatch), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages); + if (producer) { + await producer.disconnect(); + } + return ret; } async function runConsumeTransformProduce(parameters, consumeTopic, produceTopic, warmupMessages, totalMessageCnt, messageProcessTimeMs, ctpConcurrency) { From 417c571532e0368e782250d9eac3d45f4d2c70fe Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 15 Oct 2025 18:44:25 +0200 Subject: [PATCH 31/50] Add broker lag monitoring --- ci/tests/run_perf_test.js | 45 ++++++++++- .../performance/performance-consolidated.js | 22 +++++- .../performance-primitives-common.js | 77 +++++++++++++++++++ .../performance/performance-primitives.js | 9 +++ 4 files changed, 149 insertions(+), 4 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 578de361..40f7aa7e 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -79,15 +79,24 @@ async function main() { if (concurrentRun) { console.log(`Running ${modeLabel} Producer/Consumer test (concurrently)...`); + const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; + // Wait 2s more to see if all lag is caught up + const TERMINATE_TIMEOUT_MS_CONSUMERS = TERMINATE_TIMEOUT_MS + 2000; await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); const allPromises = []; allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} node performance-consolidated.js --producer`)); if (consumerModeAll || consumerModeEachMessage) { - allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} GROUPID_MESSAGE=${groupIdEachMessage} node performance-consolidated.js --consumer-each-message ${produceToSecondTopicParam}`)); + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MESSAGE=${groupIdEachMessage} node performance-consolidated.js --consumer-each-message ${produceToSecondTopicParam}`)); } if (consumerModeAll || consumerModeEachBatch) { - allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --consumer-each-batch ${produceToSecondTopicParam}`)); + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --consumer-each-batch ${produceToSecondTopicParam}`)); + } + if (consumerModeAll || consumerModeEachMessage) { + allPromises.push(runCommand(`MODE=${mode} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachMessage} node performance-consolidated.js --monitor-lag`)); + } + if (consumerModeAll || consumerModeEachBatch) { + allPromises.push(runCommand(`MODE=${mode} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachBatch} node performance-consolidated.js --monitor-lag`)); } const results = await Promise.allSettled(allPromises); return results.map(r => r.status === 'fulfilled' ? r.value : '').join('\n'); @@ -123,6 +132,10 @@ async function main() { let consumerConfluentTime; let consumerConfluentMessageAverageRSS; let consumerConfluentMessageMaxRSS; + let consumerConfluentMessageAverageBrokerLag; + let consumerConfluentMessageMaxBrokerLag; + let consumerConfluentMessageTotalLagMeasurements; + let consumerConfluentBatch; let consumerConfluentBatchRate; let consumerConfluentBatchAvgLatency; @@ -133,6 +146,9 @@ async function main() { let consumerConfluentBatchAverageSize; let consumerConfluentBatchAverageRSS; let consumerConfluentBatchMaxRSS; + let consumerConfluentBatchAverageBrokerLag; + let consumerConfluentBatchMaxBrokerLag; + let consumerConfluentBatchTotalLagMeasurements; const producerConfluent = extractValue(outputConfluentProducerConsumer, '=== Producer Rate:'); const producerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average producer RSS KB:'); @@ -145,6 +161,9 @@ async function main() { consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); consumerConfluentMessageAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average consumer-each-message RSS KB:'); consumerConfluentMessageMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-message RSS KB:'); + consumerConfluentMessageAverageBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Average broker lag (${groupIdEachMessageConfluent}):`); + consumerConfluentMessageMaxBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Max broker lag (${groupIdEachMessageConfluent}):`); + consumerConfluentMessageTotalLagMeasurements = extractValue(outputConfluentProducerConsumer, `=== Total broker lag measurements (${groupIdEachMessageConfluent}):`); } if (consumerModeAll || consumerModeEachBatch) { consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); @@ -157,6 +176,9 @@ async function main() { consumerConfluentBatchAverageSize = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch size:'); consumerConfluentBatchAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average consumer-each-batch RSS KB:'); consumerConfluentBatchMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-batch RSS KB:'); + consumerConfluentBatchAverageBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Average broker lag (${groupIdEachBatchConfluent}):`); + consumerConfluentBatchMaxBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Max broker lag (${groupIdEachBatchConfluent}):`); + consumerConfluentBatchTotalLagMeasurements = extractValue(outputConfluentProducerConsumer, `=== Total broker lag measurements (${groupIdEachBatchConfluent}):`); } const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); @@ -172,6 +194,10 @@ async function main() { let consumerKjsTime; let consumerKjsMessageAverageRSS; let consumerKjsMessageMaxRSS; + let consumerKjsMessageAverageBrokerLag; + let consumerKjsMessageMaxBrokerLag; + let consumerKjsMessageTotalLagMeasurements; + let consumerKjsBatch; let consumerKjsBatchRate; let consumerKjsBatchAvgLatency; @@ -182,6 +208,9 @@ async function main() { let consumerKjsBatchAverageSize; let consumerKjsBatchAverageRSS; let consumerKjsBatchMaxRSS; + let consumerKjsBatchAverageBrokerLag; + let consumerKjsBatchMaxBrokerLag; + let consumerKjsBatchTotalLagMeasurements; const producerKjs = extractValue(outputKjsProducerConsumer, '=== Producer Rate:'); const producerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average producer RSS KB:'); @@ -193,6 +222,9 @@ async function main() { consumerKjsMessageMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); consumerKjsMessageAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average consumer-each-message RSS KB:'); consumerKjsMessageMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-message RSS KB:'); + consumerKjsMessageAverageBrokerLag = extractValue(outputKjsProducerConsumer, `=== Average broker lag (${groupIdEachMessageKafkaJS}):`); + consumerKjsMessageMaxBrokerLag = extractValue(outputKjsProducerConsumer, `=== Max broker lag (${groupIdEachMessageKafkaJS}):`); + consumerKjsMessageTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Total broker lag measurements (${groupIdEachMessageKafkaJS}):`); } if (consumerModeAll || consumerModeEachBatch) { consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); @@ -206,6 +238,9 @@ async function main() { consumerKjsBatchAverageSize = extractValue(outputKjsProducerConsumer, '=== Average eachBatch size:'); consumerKjsBatchAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average consumer-each-batch RSS KB:'); consumerKjsBatchMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-batch RSS KB:'); + consumerKjsBatchAverageBrokerLag = extractValue(outputKjsProducerConsumer, `=== Average broker lag (${groupIdEachBatchKafkaJS}):`); + consumerKjsBatchMaxBrokerLag = extractValue(outputKjsProducerConsumer, `=== Max broker lag (${groupIdEachBatchKafkaJS}):`); + consumerKjsBatchTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Total broker lag measurements (${groupIdEachBatchKafkaJS}):`); } const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); @@ -225,6 +260,9 @@ async function main() { console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); console.log(`Average RSS (eachMessage): confluent ${consumerConfluentMessageAverageRSS}, kafkajs ${consumerKjsMessageAverageRSS}`); console.log(`Max RSS (eachMessage): confluent ${consumerConfluentMessageMaxRSS}, kafkajs ${consumerKjsMessageMaxRSS}`); + console.log(`Average broker lag (eachMessage): confluent ${consumerConfluentMessageAverageBrokerLag}, kafkajs ${consumerKjsMessageAverageBrokerLag}`); + console.log(`Max broker lag (eachMessage): confluent ${consumerConfluentMessageMaxBrokerLag}, kafkajs ${consumerKjsMessageMaxBrokerLag}`); + console.log(`Total broker lag measurements (eachMessage): confluent ${consumerConfluentMessageTotalLagMeasurements}, kafkajs ${consumerKjsMessageTotalLagMeasurements}`); } if (consumerModeAll || consumerModeEachBatch) { console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); @@ -237,6 +275,9 @@ async function main() { console.log(`Average eachBatch size: confluent ${consumerConfluentBatchAverageSize}, kafkajs ${consumerKjsBatchAverageSize}`); console.log(`Average RSS (eachBatch): confluent ${consumerConfluentBatchAverageRSS}, kafkajs ${consumerKjsBatchAverageRSS}`); console.log(`Max RSS (eachBatch): confluent ${consumerConfluentBatchMaxRSS}, kafkajs ${consumerKjsBatchMaxRSS}`); + console.log(`Average broker lag (eachBatch): confluent ${consumerConfluentBatchAverageBrokerLag}, kafkajs ${consumerKjsBatchAverageBrokerLag}`); + console.log(`Max broker lag (eachBatch): confluent ${consumerConfluentBatchMaxBrokerLag}, kafkajs ${consumerKjsBatchMaxBrokerLag}`); + console.log(`Total broker lag measurements (eachBatch): confluent ${consumerConfluentBatchTotalLagMeasurements}, kafkajs ${consumerKjsBatchTotalLagMeasurements}`); } if (!concurrentRun) { console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index f3217c06..c30abc22 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -3,11 +3,13 @@ const mode = process.env.MODE ? process.env.MODE : 'confluent'; let runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether; if (mode === 'confluent') { - ({ runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether } = require('./performance-primitives')); + ({ runProducer, runConsumer, runConsumeTransformProduce, + runCreateTopics, runLagMonitoring, + runProducerConsumerTogether } = require('./performance-primitives')); } else { ({ runProducer, runConsumer, runConsumeTransformProduce, runProducerConsumerTogether } = require('./performance-primitives-kafkajs')); /* createTopics is more reliable in CKJS */ - ({ runCreateTopics } = require('./performance-primitives')); + ({ runCreateTopics, runLagMonitoring } = require('./performance-primitives')); } const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; @@ -58,6 +60,7 @@ function logParameters(parameters) { const produceConsumeLatency = process.argv.includes('--latency'); const all = process.argv.includes('--all'); const createTopics = process.argv.includes('--create-topics'); + const monitorLag = process.argv.includes('--monitor-lag'); let maxAverageRSSKB, maxMaxRSSKB; const stats = {}; @@ -106,6 +109,21 @@ function logParameters(parameters) { await runCreateTopics(parameters, topic, topic2, numPartitions); } + if (monitorLag) { + console.log("=== Starting Lag Monitoring:"); + logParameters(parameters); + console.log(` Topic: ${topic}`); + const { + averageLag, + maxLag, + totalMeasurements + } = await runLagMonitoring(parameters, topic); + const monitoredGroupId = process.env.GROUPID_MONITOR; + console.log(`=== Average broker lag (${monitoredGroupId}): `, averageLag); + console.log(`=== Max broker lag (${monitoredGroupId}): `, maxLag); + console.log(`=== Total broker lag measurements (${monitoredGroupId}): `, totalMeasurements); + } + if (producer || all) { console.log("=== Running Basic Producer Performance Test:") logParameters(parameters); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index e48da4e0..fab95aec 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -306,9 +306,86 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess return rate; } +async function runLagMonitoring(admin, topic) { + const handlers = installHandlers(); + let groupId = process.env.GROUPID_MONITOR; + if (!groupId) { + throw new Error("GROUPID_MONITOR environment variable not set"); + } + + await admin.connect(); + + const fetchTotalLag = async () => { + const partitionCompleteLag = {}; + const partitionHWM = {}; + const partitionLag = {}; + let totalLag = 0n; + const operations = [ + admin.fetchTopicOffsets(topic, { timeout: 30000 }), + admin.fetchOffsets({ groupId, topics: [topic], timeout: 30000 }), + ] + let [topicOffsets, groupOffsets] = await Promise.all(operations); + groupOffsets = groupOffsets[0]; + + for (const partitionOffset of topicOffsets) { + partitionHWM[partitionOffset.partition] = BigInt(partitionOffset.high); + partitionCompleteLag[partitionOffset.partition] = BigInt(partitionOffset.high) - BigInt(partitionOffset.low); + } + + if (groupOffsets && groupOffsets.partitions) { + for (const partitionOffset of groupOffsets.partitions) { + const partition = partitionOffset.partition; + const hwm = partitionHWM[partition]; + if (hwm && partitionOffset.offset && hwm >= BigInt(partitionOffset.offset)) { + const currentLag = hwm - BigInt(partitionOffset.offset); + partitionLag[partition] = currentLag; + totalLag += currentLag; + } + } + } else { + throw new Error(`No offsets found for group ${groupId} on topic ${topic}`); + } + for (const partition of Object.keys(partitionHWM)) { + if (partitionLag[partition] === undefined) { + const currentLag = partitionCompleteLag[partition]; + partitionLag[partition] = currentLag; + totalLag += currentLag; + } + } + return totalLag; + } + + let totalAverageLag = 0n; + let maxLag = 0n; + let totalMeasurements = 0; + + while (!handlers.terminationRequested) { + try { + const lag = await fetchTotalLag(); + totalAverageLag += lag; + maxLag = lag > maxLag ? lag : maxLag; + totalMeasurements++; + } catch (e) { + console.error(`Error fetching lag: ${e}`); + } + await new Promise(resolve => setTimeout(resolve, 100)); + } + const averageLag = totalMeasurements > 0 ? (Number(totalAverageLag) / totalMeasurements) : NaN; + maxLag = Number(maxLag); + + await admin.disconnect(); + removeHandlers(handlers); + return { + averageLag, + maxLag, + totalMeasurements + }; +} + module.exports = { runConsumer, runProducer, + runLagMonitoring, genericProduceToTopic, getAutoCommit, }; \ No newline at end of file diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index a9d06d39..9ec5bebb 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -4,6 +4,7 @@ const { hrtime } = require('process'); const { runConsumer: runConsumerCommon, runProducer: runProducerCommon, + runLagMonitoring: runLagMonitoringCommon, genericProduceToTopic, getAutoCommit, } = require('./performance-primitives-common'); @@ -14,6 +15,7 @@ module.exports = { runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether, + runLagMonitoring, }; function baseConfiguration(parameters) { @@ -65,6 +67,13 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { await admin.disconnect(); } +function runLagMonitoring(parameters, topic) { + const kafka = new Kafka(baseConfiguration(parameters)); + const admin = kafka.admin(); + + return runLagMonitoringCommon(admin, topic); +} + class CompatibleProducer { constructor(producer) { this.producer = producer; From fbff472ea036103da78ed4bd1346a2226a4a53ea Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 16 Oct 2025 14:03:41 +0200 Subject: [PATCH 32/50] fixup --- ci/tests/run_perf_test.js | 15 +++++++-------- examples/performance/performance-consolidated.js | 2 +- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 40f7aa7e..d5735e1e 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -8,8 +8,7 @@ async function runCommand(command) { const output = await exec(command, { encoding: 'utf8', stdio: 'pipe' }); return output.stdout; } catch (error) { - const errorOutput = error.stdout || error.stderr || error.message; - return errorOutput; + return (error.stdout || '') + (error.stderr || '') + (error.message || ''); } } @@ -163,7 +162,7 @@ async function main() { consumerConfluentMessageMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-message RSS KB:'); consumerConfluentMessageAverageBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Average broker lag (${groupIdEachMessageConfluent}):`); consumerConfluentMessageMaxBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Max broker lag (${groupIdEachMessageConfluent}):`); - consumerConfluentMessageTotalLagMeasurements = extractValue(outputConfluentProducerConsumer, `=== Total broker lag measurements (${groupIdEachMessageConfluent}):`); + consumerConfluentMessageTotalLagMeasurements = extractValue(outputConfluentProducerConsumer, `=== Sample size for broker lag measurement (${groupIdEachMessageConfluent}):`); } if (consumerModeAll || consumerModeEachBatch) { consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); @@ -178,7 +177,7 @@ async function main() { consumerConfluentBatchMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-batch RSS KB:'); consumerConfluentBatchAverageBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Average broker lag (${groupIdEachBatchConfluent}):`); consumerConfluentBatchMaxBrokerLag = extractValue(outputConfluentProducerConsumer, `=== Max broker lag (${groupIdEachBatchConfluent}):`); - consumerConfluentBatchTotalLagMeasurements = extractValue(outputConfluentProducerConsumer, `=== Total broker lag measurements (${groupIdEachBatchConfluent}):`); + consumerConfluentBatchTotalLagMeasurements = extractValue(outputConfluentProducerConsumer, `=== Sample size for broker lag measurement (${groupIdEachBatchConfluent}):`); } const consumerConfluentAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Max Average RSS across tests:'); const consumerConfluentMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max RSS across tests:'); @@ -224,7 +223,7 @@ async function main() { consumerKjsMessageMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-message RSS KB:'); consumerKjsMessageAverageBrokerLag = extractValue(outputKjsProducerConsumer, `=== Average broker lag (${groupIdEachMessageKafkaJS}):`); consumerKjsMessageMaxBrokerLag = extractValue(outputKjsProducerConsumer, `=== Max broker lag (${groupIdEachMessageKafkaJS}):`); - consumerKjsMessageTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Total broker lag measurements (${groupIdEachMessageKafkaJS}):`); + consumerKjsMessageTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Sample size for broker lag measurement (${groupIdEachMessageKafkaJS}):`); } if (consumerModeAll || consumerModeEachBatch) { consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); @@ -240,7 +239,7 @@ async function main() { consumerKjsBatchMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-batch RSS KB:'); consumerKjsBatchAverageBrokerLag = extractValue(outputKjsProducerConsumer, `=== Average broker lag (${groupIdEachBatchKafkaJS}):`); consumerKjsBatchMaxBrokerLag = extractValue(outputKjsProducerConsumer, `=== Max broker lag (${groupIdEachBatchKafkaJS}):`); - consumerKjsBatchTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Total broker lag measurements (${groupIdEachBatchKafkaJS}):`); + consumerKjsBatchTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Sample size for broker lag measurement (${groupIdEachBatchKafkaJS}):`); } const consumerKjsAverageRSS = extractValue(outputKjsProducerConsumer, '=== Max Average RSS across tests:'); const consumerKjsMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max RSS across tests:'); @@ -262,7 +261,7 @@ async function main() { console.log(`Max RSS (eachMessage): confluent ${consumerConfluentMessageMaxRSS}, kafkajs ${consumerKjsMessageMaxRSS}`); console.log(`Average broker lag (eachMessage): confluent ${consumerConfluentMessageAverageBrokerLag}, kafkajs ${consumerKjsMessageAverageBrokerLag}`); console.log(`Max broker lag (eachMessage): confluent ${consumerConfluentMessageMaxBrokerLag}, kafkajs ${consumerKjsMessageMaxBrokerLag}`); - console.log(`Total broker lag measurements (eachMessage): confluent ${consumerConfluentMessageTotalLagMeasurements}, kafkajs ${consumerKjsMessageTotalLagMeasurements}`); + console.log(`Sample size for broker lag measurement (eachMessage): confluent ${consumerConfluentMessageTotalLagMeasurements}, kafkajs ${consumerKjsMessageTotalLagMeasurements}`); } if (consumerModeAll || consumerModeEachBatch) { console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); @@ -277,7 +276,7 @@ async function main() { console.log(`Max RSS (eachBatch): confluent ${consumerConfluentBatchMaxRSS}, kafkajs ${consumerKjsBatchMaxRSS}`); console.log(`Average broker lag (eachBatch): confluent ${consumerConfluentBatchAverageBrokerLag}, kafkajs ${consumerKjsBatchAverageBrokerLag}`); console.log(`Max broker lag (eachBatch): confluent ${consumerConfluentBatchMaxBrokerLag}, kafkajs ${consumerKjsBatchMaxBrokerLag}`); - console.log(`Total broker lag measurements (eachBatch): confluent ${consumerConfluentBatchTotalLagMeasurements}, kafkajs ${consumerKjsBatchTotalLagMeasurements}`); + console.log(`Sample size for broker lag measurement (eachBatch): confluent ${consumerConfluentBatchTotalLagMeasurements}, kafkajs ${consumerKjsBatchTotalLagMeasurements}`); } if (!concurrentRun) { console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index c30abc22..b39d28b8 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -121,7 +121,7 @@ function logParameters(parameters) { const monitoredGroupId = process.env.GROUPID_MONITOR; console.log(`=== Average broker lag (${monitoredGroupId}): `, averageLag); console.log(`=== Max broker lag (${monitoredGroupId}): `, maxLag); - console.log(`=== Total broker lag measurements (${monitoredGroupId}): `, totalMeasurements); + console.log(`=== Sample size for broker lag measurement (${monitoredGroupId}): `, totalMeasurements); } if (producer || all) { From d1a3e8c510e20e0aec6965e7a684cb4398d77741 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 17 Oct 2025 11:08:04 +0200 Subject: [PATCH 33/50] Use same producer and different E2E latencies --- ci/tests/run_perf_test.js | 63 +++++++++++++------ .../performance/performance-consolidated.js | 33 +++++++--- .../performance-primitives-common.js | 58 +++++++++++------ .../performance-primitives-kafkajs.js | 10 ++- .../performance/performance-primitives.js | 3 +- 5 files changed, 117 insertions(+), 50 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index d5735e1e..b8e9e808 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -106,8 +106,11 @@ async function main() { } const runProducerConsumer = async () => { + console.log(`Running Producer/Consumer tests with Confluent Kafka JS at ${new Date().toISOString()}`); outputConfluentProducerConsumer = await runProducerConsumerMode('confluent'); + console.log(`Running Producer/Consumer tests with KafkaJS at ${new Date().toISOString()}`); outputKjsProducerConsumer = await runProducerConsumerMode('kafkajs'); + console.log(`Producer/Consumer tests completed at ${new Date().toISOString()}`); } await runProducerConsumer(); @@ -126,8 +129,10 @@ async function main() { let ctpConfluent, ctpKjs; let consumerConfluentMessage; let consumerConfluentMessageRate; - let consumerConfluentMessageAvgLatency; - let consumerConfluentMessageMaxLatency; + let consumerConfluentMessageAvgLatencyT0T1; + let consumerConfluentMessageMaxLatencyT0T1; + let consumerConfluentMessageAvgLatencyT0T2; + let consumerConfluentMessageMaxLatencyT0T2; let consumerConfluentTime; let consumerConfluentMessageAverageRSS; let consumerConfluentMessageMaxRSS; @@ -137,8 +142,10 @@ async function main() { let consumerConfluentBatch; let consumerConfluentBatchRate; - let consumerConfluentBatchAvgLatency; - let consumerConfluentBatchMaxLatency; + let consumerConfluentBatchAvgLatencyT0T1; + let consumerConfluentBatchMaxLatencyT0T1; + let consumerConfluentBatchAvgLatencyT0T2; + let consumerConfluentBatchMaxLatencyT0T2; let consumerConfluentBatchTime; let consumerConfluentBatchAverageLag; let consumerConfluentBatchMaxLag; @@ -155,8 +162,10 @@ async function main() { if (consumerModeAll || consumerModeEachMessage) { consumerConfluentMessage = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); consumerConfluentMessageRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); - consumerConfluentMessageAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); - consumerConfluentMessageMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); + consumerConfluentMessageAvgLatencyT0T1 = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency T0-T1 (eachMessage):'); + consumerConfluentMessageMaxLatencyT0T1 = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency T0-T1 (eachMessage):'); + consumerConfluentMessageAvgLatencyT0T2 = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency T0-T2 (eachMessage):'); + consumerConfluentMessageMaxLatencyT0T2 = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency T0-T2 (eachMessage):'); consumerConfluentTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachMessage):'); consumerConfluentMessageAverageRSS = extractValue(outputConfluentProducerConsumer, '=== Average consumer-each-message RSS KB:'); consumerConfluentMessageMaxRSS = extractValue(outputConfluentProducerConsumer, '=== Max consumer-each-message RSS KB:'); @@ -167,8 +176,10 @@ async function main() { if (consumerModeAll || consumerModeEachBatch) { consumerConfluentBatch = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); consumerConfluentBatchRate = extractValue(outputConfluentProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); - consumerConfluentBatchAvgLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); - consumerConfluentBatchMaxLatency = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); + consumerConfluentBatchAvgLatencyT0T1 = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency T0-T1 (eachBatch):'); + consumerConfluentBatchMaxLatencyT0T1 = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency T0-T1 (eachBatch):'); + consumerConfluentBatchAvgLatencyT0T2 = extractValue(outputConfluentProducerConsumer, '=== Consumer average E2E latency T0-T2 (eachBatch):'); + consumerConfluentBatchMaxLatencyT0T2 = extractValue(outputConfluentProducerConsumer, '=== Consumer max E2E latency T0-T2 (eachBatch):'); consumerConfluentBatchTime = extractValue(outputConfluentProducerConsumer, '=== Consumption time (eachBatch):'); consumerConfluentBatchAverageLag = extractValue(outputConfluentProducerConsumer, '=== Average eachBatch lag:'); consumerConfluentBatchMaxLag = extractValue(outputConfluentProducerConsumer, '=== Max eachBatch lag:'); @@ -188,8 +199,8 @@ async function main() { // Extract KafkaJS results let consumerKjsMessage; let consumerKjsMessageRate; - let consumerKjsMessageAvgLatency; - let consumerKjsMessageMaxLatency; + let consumerKjsMessageAvgLatencyT0T1; + let consumerKjsMessageMaxLatencyT0T1; let consumerKjsTime; let consumerKjsMessageAverageRSS; let consumerKjsMessageMaxRSS; @@ -199,8 +210,8 @@ async function main() { let consumerKjsBatch; let consumerKjsBatchRate; - let consumerKjsBatchAvgLatency; - let consumerKjsBatchMaxLatency; + let consumerKjsBatchAvgLatencyT0T1; + let consumerKjsBatchMaxLatencyT0T1; let consumerKjsBatchTime; let consumerKjsBatchAverageLag; let consumerKjsBatchMaxLag; @@ -217,8 +228,10 @@ async function main() { if (consumerModeAll || consumerModeEachMessage) { consumerKjsMessage = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachMessage):'); consumerKjsMessageRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachMessage):'); - consumerKjsMessageAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachMessage):'); - consumerKjsMessageMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachMessage):'); + consumerKjsMessageAvgLatencyT0T1 = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency T0-T1 (eachMessage):'); + consumerKjsMessageMaxLatencyT0T1 = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency T0-T1 (eachMessage):'); + consumerKjsMessageAvgLatencyT0T2 = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency T0-T2 (eachMessage):'); + consumerKjsMessageMaxLatencyT0T2 = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency T0-T2 (eachMessage):'); consumerKjsMessageAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average consumer-each-message RSS KB:'); consumerKjsMessageMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-message RSS KB:'); consumerKjsMessageAverageBrokerLag = extractValue(outputKjsProducerConsumer, `=== Average broker lag (${groupIdEachMessageKafkaJS}):`); @@ -229,8 +242,10 @@ async function main() { consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); consumerKjsBatchRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); - consumerKjsBatchAvgLatency = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency (eachBatch):'); - consumerKjsBatchMaxLatency = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency (eachBatch):'); + consumerKjsBatchAvgLatencyT0T1 = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency T0-T1 (eachBatch):'); + consumerKjsBatchMaxLatencyT0T1 = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency T0-T1 (eachBatch):'); + consumerKjsBatchAvgLatencyT0T2 = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency T0-T2 (eachBatch):'); + consumerKjsBatchMaxLatencyT0T2 = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency T0-T2 (eachBatch):'); consumerKjsBatchTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachBatch):'); consumerKjsBatchAverageLag = extractValue(outputKjsProducerConsumer, '=== Average eachBatch lag:'); consumerKjsBatchMaxLag = extractValue(outputKjsProducerConsumer, '=== Max eachBatch lag:'); @@ -254,8 +269,12 @@ async function main() { if (consumerModeAll || consumerModeEachMessage) { console.log(`Consumer rates MB/s (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); console.log(`Consumer rates msg/s (eachMessage): confluent ${consumerConfluentMessageRate}, kafkajs ${consumerKjsMessageRate}`); - console.log(`Consumer average E2E latency (eachMessage): confluent ${consumerConfluentMessageAvgLatency}, kafkajs ${consumerKjsMessageAvgLatency}`); - console.log(`Consumer max E2E latency (eachMessage): confluent ${consumerConfluentMessageMaxLatency}, kafkajs ${consumerKjsMessageMaxLatency}`); + console.log(`Consumer average E2E latency T0-T1 (eachMessage): confluent ${consumerConfluentMessageAvgLatencyT0T1}, kafkajs ${consumerKjsMessageAvgLatencyT0T1}`); + console.log(`Consumer max E2E latency T0-T1 (eachMessage): confluent ${consumerConfluentMessageMaxLatencyT0T1}, kafkajs ${consumerKjsMessageMaxLatencyT0T1}`); + if (produceToSecondTopic) { + console.log(`Consumer average E2E latency T0-T2 (eachMessage): confluent ${consumerConfluentMessageAvgLatencyT0T2}, kafkajs ${consumerKjsMessageAvgLatencyT0T2}`); + console.log(`Consumer max E2E latency T0-T2 (eachMessage): confluent ${consumerConfluentMessageMaxLatencyT0T2}, kafkajs ${consumerKjsMessageMaxLatencyT0T2}`); + } console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); console.log(`Average RSS (eachMessage): confluent ${consumerConfluentMessageAverageRSS}, kafkajs ${consumerKjsMessageAverageRSS}`); console.log(`Max RSS (eachMessage): confluent ${consumerConfluentMessageMaxRSS}, kafkajs ${consumerKjsMessageMaxRSS}`); @@ -266,8 +285,12 @@ async function main() { if (consumerModeAll || consumerModeEachBatch) { console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); console.log(`Consumer rates msg/s (eachBatch): confluent ${consumerConfluentBatchRate}, kafkajs ${consumerKjsBatchRate}`); - console.log(`Consumer average E2E latency (eachBatch): confluent ${consumerConfluentBatchAvgLatency}, kafkajs ${consumerKjsBatchAvgLatency}`); - console.log(`Consumer max E2E latency (eachBatch): confluent ${consumerConfluentBatchMaxLatency}, kafkajs ${consumerKjsBatchMaxLatency}`); + console.log(`Consumer average E2E latency T0-T1 (eachBatch): confluent ${consumerConfluentBatchAvgLatencyT0T1}, kafkajs ${consumerKjsBatchAvgLatencyT0T1}`); + console.log(`Consumer max E2E latency T0-T1 (eachBatch): confluent ${consumerConfluentBatchMaxLatencyT0T1}, kafkajs ${consumerKjsBatchMaxLatencyT0T1}`); + if (produceToSecondTopic) { + console.log(`Consumer average E2E latency T0-T2 (eachBatch): confluent ${consumerConfluentBatchAvgLatencyT0T2}, kafkajs ${consumerKjsBatchAvgLatencyT0T2}`); + console.log(`Consumer max E2E latency T0-T2 (eachBatch): confluent ${consumerConfluentBatchMaxLatencyT0T2}, kafkajs ${consumerKjsBatchMaxLatencyT0T2}`); + } console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index b39d28b8..4684a709 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -1,15 +1,16 @@ const fs = require('fs'); const mode = process.env.MODE ? process.env.MODE : 'confluent'; -let runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether; +let runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runProducerConsumerTogether, runProducerCKJS; if (mode === 'confluent') { ({ runProducer, runConsumer, runConsumeTransformProduce, runCreateTopics, runLagMonitoring, runProducerConsumerTogether } = require('./performance-primitives')); + runProducerCKJS = runProducer; } else { ({ runProducer, runConsumer, runConsumeTransformProduce, runProducerConsumerTogether } = require('./performance-primitives-kafkajs')); /* createTopics is more reliable in CKJS */ - ({ runCreateTopics, runLagMonitoring } = require('./performance-primitives')); + ({ runCreateTopics, runLagMonitoring, runProducer: runProducerCKJS } = require('./performance-primitives')); } const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; @@ -32,6 +33,7 @@ const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +proc const consumerProcessingTime = process.env.CONSUMER_PROCESSING_TIME ? +process.env.CONSUMER_PROCESSING_TIME : 100; const producerProcessingTime = process.env.PRODUCER_PROCESSING_TIME ? +process.env.PRODUCER_PROCESSING_TIME : 100; const limitRPS = process.env.LIMIT_RPS ? +process.env.LIMIT_RPS : null; +const useCKJSProducerEverywhere = process.env.USE_CKJS_PRODUCER_EVERYWHERE === 'true'; const parameters = { brokers, securityProtocol, @@ -134,8 +136,13 @@ function logParameters(parameters) { console.log(` Compression: ${compression}`); console.log(` Limit RPS: ${limitRPS}`); console.log(` Warmup Messages: ${warmupMessages}`); + console.log(` Use CKJS Producer Everywhere: ${useCKJSProducerEverywhere}`); startTrackingMemory(); - const producerRate = await runProducer(parameters, topic, batchSize, + let runProducerFunction = runProducer; + if (useCKJSProducerEverywhere) { + runProducerFunction = runProducerCKJS; + } + const producerRate = await runProducerFunction(parameters, topic, batchSize, warmupMessages, messageCount, messageSize, compression, randomness, limitRPS); endTrackingMemory('producer', `producer-memory-${mode}.json`); @@ -153,12 +160,16 @@ function logParameters(parameters) { const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, false, partitionsConsumedConcurrently, stats, - produceToSecondTopic ? topic2 : null, compression); + produceToSecondTopic ? topic2 : null, compression, useCKJSProducerEverywhere); endTrackingMemory('consumer-each-message', `consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate MB/s (eachMessage): ", consumerRate); console.log("=== Consumer Rate msg/s (eachMessage): ", stats.messageRate); - console.log("=== Consumer average E2E latency (eachMessage): ", stats.avgLatency); - console.log("=== Consumer max E2E latency (eachMessage): ", stats.maxLatency); + console.log("=== Consumer average E2E latency T0-T1 (eachMessage): ", stats.avgLatencyT0T1); + console.log("=== Consumer max E2E latency T0-T1 (eachMessage): ", stats.maxLatencyT0T1); + if (produceToSecondTopic) { + console.log("=== Consumer average E2E latency T0-T2 (eachMessage): ", stats.avgLatencyT0T2); + console.log("=== Consumer max E2E latency T0-T2 (eachMessage): ", stats.maxLatencyT0T2); + } console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); } @@ -173,15 +184,19 @@ function logParameters(parameters) { const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, true, partitionsConsumedConcurrently, stats, - produceToSecondTopic ? topic2 : null, compression); + produceToSecondTopic ? topic2 : null, compression, useCKJSProducerEverywhere); endTrackingMemory('consumer-each-batch', `consumer-memory-batch-${mode}.json`); console.log("=== Consumer Rate MB/s (eachBatch): ", consumerRate); console.log("=== Consumer Rate msg/s (eachBatch): ", stats.messageRate); console.log("=== Average eachBatch lag: ", stats.averageOffsetLag); console.log("=== Max eachBatch lag: ", stats.maxOffsetLag); console.log("=== Average eachBatch size: ", stats.averageBatchSize); - console.log("=== Consumer average E2E latency (eachBatch): ", stats.avgLatency); - console.log("=== Consumer max E2E latency (eachBatch): ", stats.maxLatency); + console.log("=== Consumer average E2E latency T0-T1 (eachBatch): ", stats.avgLatencyT0T1); + console.log("=== Consumer max E2E latency T0-T1 (eachBatch): ", stats.maxLatencyT0T1); + if (produceToSecondTopic) { + console.log("=== Consumer average E2E latency T0-T2 (eachBatch): ", stats.avgLatencyT0T2); + console.log("=== Consumer max E2E latency T0-T2 (eachBatch): ", stats.maxLatencyT0T2); + } console.log("=== Consumption time (eachBatch): ", stats.durationSeconds); } diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index fab95aec..d229091a 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -73,19 +73,29 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac const skippedMessages = warmupMessages; const decoder = new TextDecoder('utf-8'); - const updateLatency = (receivedAt, numMessages, message) => { + const updateLatency = (receivedAt, numMessages, message, isT0T2) => { if (!stats) return; const sentAt = Number(decoder.decode(message.value.slice(0, 13))); const latency = receivedAt - sentAt; - if (!stats.maxLatency) { - stats.maxLatency = latency; - stats.avgLatency = latency; + if (!isT0T2) { + if (!stats.maxLatencyT0T1) { + stats.maxLatencyT0T1 = latency; + stats.avgLatencyT0T1 = latency; + } else { + stats.maxLatencyT0T1 = Math.max(stats.maxLatencyT0T1, latency); + stats.avgLatencyT0T1 = ((stats.avgLatencyT0T1 * (numMessages - 1)) + latency) / numMessages; + } } else { - stats.maxLatency = Math.max(stats.maxLatency, latency); - stats.avgLatency = ((stats.avgLatency * (numMessages - 1)) + latency) / numMessages; + if (!stats.maxLatencyT0T2) { + stats.maxLatencyT0T2 = latency; + stats.avgLatencyT0T2 = latency; + } else { + stats.maxLatencyT0T2 = Math.max(stats.maxLatencyT0T2, latency); + stats.avgLatencyT0T2 = ((stats.avgLatencyT0T2 * (numMessages - 1)) + latency) / numMessages; + } } }; @@ -110,14 +120,10 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac partitionsConsumedConcurrently, eachMessage: async ({ topic, partition, message }) => { messagesReceived++; - if (actionOnMessages) { - await actionOnMessages([message]); - } - if (messagesReceived >= skippedMessages) { messagesMeasured++; totalMessageSize += message.value.length; - updateLatency(Date.now(), messagesMeasured, message); + updateLatency(Date.now(), messagesMeasured, message, false); if (messagesReceived === skippedMessages) { startTime = hrtime.bigint(); @@ -125,6 +131,11 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac stopConsuming(); } } + + if (actionOnMessages) { + await actionOnMessages([message]); + updateLatency(Date.now(), messagesMeasured, message, true); + } } } if (eachBatch) { @@ -133,10 +144,8 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac eachBatch: async ({ batch }) => { const messagesBeforeBatch = messagesReceived; const topic = batch.topic; - if (actionOnMessages) { - await actionOnMessages(batch.messages); - } - + let messagesBase; + let messages; messagesReceived += batch.messages.length; if (messagesReceived >= skippedMessages) { const offsetLag = batch.offsetLag(); @@ -144,16 +153,16 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac maxOffsetLag = Math.max(offsetLag, maxOffsetLag); totalBatches++; messagesMeasured = messagesReceived - skippedMessages; - let messages = batch.messages; + messages = batch.messages; if (messagesBeforeBatch < skippedMessages) { messages = messages.slice(messages.length - messagesMeasured); } const now = Date.now(); - const messagesBase = messagesMeasured - messages.length; + messagesBase = messagesMeasured - messages.length; let i = 1; for (const message of messages) { totalMessageSize += message.value.length; - updateLatency(now, messagesBase + i, message); + updateLatency(now, messagesBase + i, message, false); i++; } @@ -168,6 +177,19 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac consumer.pause([{ topic }]); } } + + if (actionOnMessages) { + await actionOnMessages(batch.messages); + if (messagesMeasured > 0) { + let i = 1; + const now = Date.now(); + for (const message of messages) { + totalMessageSize += message.value.length; + updateLatency(now, messagesBase + i, message, true); + i++; + } + } + } } }; } diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 240e931c..e8917ca7 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -8,6 +8,10 @@ const { getAutoCommit, } = require('./performance-primitives-common'); +const { + newCompatibleProducer: newCompatibleProducerCKJS, +} = require('./performance-primitives'); + module.exports = { runProducer, runConsumer, @@ -152,11 +156,13 @@ function newCompatibleConsumer(parameters, eachBatch) { } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression) { +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression, useCKJSProducerEverywhere) { let actionOnMessages = null; let producer; if (produceToTopic) { - producer = newCompatibleProducer(parameters, produceCompression); + const newCompatibleProducerFunction = useCKJSProducerEverywhere ? + newCompatibleProducerCKJS : newCompatibleProducer; + producer = newCompatibleProducerFunction(parameters, produceCompression); await producer.connect(); actionOnMessages = (messages) => genericProduceToTopic(producer, produceToTopic, messages); diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 9ec5bebb..d702432f 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -16,6 +16,7 @@ module.exports = { runCreateTopics, runProducerConsumerTogether, runLagMonitoring, + newCompatibleProducer, }; function baseConfiguration(parameters) { @@ -157,7 +158,7 @@ function newCompatibleConsumer(parameters, eachBatch) { } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression) { +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression, useCKJSProducerEverywhere) { let actionOnMessages = null; let producer; if (produceToTopic) { From 833580838dbf32d816bd95e220fda9285977a298 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 17 Oct 2025 14:24:59 +0200 Subject: [PATCH 34/50] Auto commit on batch end --- .../performance-primitives-common.js | 18 ++++++++++++++++++ .../performance-primitives-kafkajs.js | 4 ++++ examples/performance/performance-primitives.js | 4 ++++ 3 files changed, 26 insertions(+) diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index d229091a..1103a1fa 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -3,6 +3,7 @@ const { randomBytes } = require('crypto'); const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; const AUTO_COMMIT = process.env.AUTO_COMMIT || 'false'; +const AUTO_COMMIT_ON_BATCH_END = process.env.AUTO_COMMIT_ON_BATCH_END === 'true'; let autoCommit; if (AUTO_COMMIT && AUTO_COMMIT === 'false') autoCommit = null; @@ -136,6 +137,14 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac await actionOnMessages([message]); updateLatency(Date.now(), messagesMeasured, message, true); } + + if (autoCommit !== null && AUTO_COMMIT_ON_BATCH_END) { + await consumer.commitOffsetsOnBatchEnd([{ + topic, + partition, + offset: (Number(message.offset) + 1).toString(), + }]); + } } } if (eachBatch) { @@ -144,6 +153,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac eachBatch: async ({ batch }) => { const messagesBeforeBatch = messagesReceived; const topic = batch.topic; + const partition = batch.partition; let messagesBase; let messages; messagesReceived += batch.messages.length; @@ -190,6 +200,14 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac } } } + + if (autoCommit !== null && AUTO_COMMIT_ON_BATCH_END) { + await consumer.commitOffsetsOnBatchEnd([{ + topic, + partition, + offset: (Number(batch.lastOffset()) + 1).toString(), + }]); + } } }; } diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index e8917ca7..1b302297 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -128,6 +128,10 @@ class CompatibleConsumer { return this.consumer.pause(topics); } + commitOffsetsOnBatchEnd(offsets) { + // do nothing, done by KafkaJS after each* if autoCommit is enabled + } + run(opts) { const autoCommit = getAutoCommit(); const autoCommitOpts = autoCommit > 0 ? diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index d702432f..84d56383 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -129,6 +129,10 @@ class CompatibleConsumer { return this.consumer.pause(topics); } + commitOffsetsOnBatchEnd(offsets) { + return this.consumer.commitOffsets(offsets); + } + run(opts) { return this.consumer.run({ ...opts From 07bdc1168db1ba684ae08dac07b3ccadf8e48d19 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 15:15:31 +0100 Subject: [PATCH 35/50] fixup --- ci/tests/run_perf_test.js | 2 +- examples/performance/performance-primitives-common.js | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index b8e9e808..f588aa10 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -232,6 +232,7 @@ async function main() { consumerKjsMessageMaxLatencyT0T1 = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency T0-T1 (eachMessage):'); consumerKjsMessageAvgLatencyT0T2 = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency T0-T2 (eachMessage):'); consumerKjsMessageMaxLatencyT0T2 = extractValue(outputKjsProducerConsumer, '=== Consumer max E2E latency T0-T2 (eachMessage):'); + consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); consumerKjsMessageAverageRSS = extractValue(outputKjsProducerConsumer, '=== Average consumer-each-message RSS KB:'); consumerKjsMessageMaxRSS = extractValue(outputKjsProducerConsumer, '=== Max consumer-each-message RSS KB:'); consumerKjsMessageAverageBrokerLag = extractValue(outputKjsProducerConsumer, `=== Average broker lag (${groupIdEachMessageKafkaJS}):`); @@ -239,7 +240,6 @@ async function main() { consumerKjsMessageTotalLagMeasurements = extractValue(outputKjsProducerConsumer, `=== Sample size for broker lag measurement (${groupIdEachMessageKafkaJS}):`); } if (consumerModeAll || consumerModeEachBatch) { - consumerKjsTime = extractValue(outputKjsProducerConsumer, '=== Consumption time (eachMessage):'); consumerKjsBatch = extractValue(outputKjsProducerConsumer, '=== Consumer Rate MB/s (eachBatch):'); consumerKjsBatchRate = extractValue(outputKjsProducerConsumer, '=== Consumer Rate msg/s (eachBatch):'); consumerKjsBatchAvgLatencyT0T1 = extractValue(outputKjsProducerConsumer, '=== Consumer average E2E latency T0-T1 (eachBatch):'); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 1103a1fa..bb18c71e 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -190,7 +190,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (actionOnMessages) { await actionOnMessages(batch.messages); - if (messagesMeasured > 0) { + if (messagesMeasured > 0 && messages.length > 0) { let i = 1; const now = Date.now(); for (const message of messages) { From ec6f16cb6d84eb1ac9be6ef9a4c49bd98302106e Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 19 Oct 2025 18:16:55 +0200 Subject: [PATCH 36/50] Fix CI performance run --- ci/tests/run_perf_test.js | 25 ++++++++++++------- .../performance/performance-consolidated.js | 4 ++- .../performance-primitives-common.js | 21 +++++++--------- 3 files changed, 28 insertions(+), 22 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index f588aa10..a88870b0 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -42,8 +42,11 @@ function belowTarget(value, target) { async function main() { // Run performance tests and store outputs in memory const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 50000; - const skipCtpTest = process.env.SKIP_CTP_TEST ? process.env.SKIP_CTP_TEST === 'true' : false; + let skipCtpTest = process.env.SKIP_CTP_TEST ? process.env.SKIP_CTP_TEST === 'true' : false; const concurrentRun = process.env.CONCURRENT_RUN ? process.env.CONCURRENT_RUN === 'true' : false; + if (concurrentRun) { + skipCtpTest = true; + } const consumerMode = process.env.CONSUMER_MODE || 'all'; const produceToSecondTopic = process.env.PRODUCE_TO_SECOND_TOPIC ? process.env.PRODUCE_TO_SECOND_TOPIC === 'true' : false; const produceToSecondTopicParam = produceToSecondTopic ? '--produce-to-second-topic' : ''; @@ -119,11 +122,11 @@ async function main() { console.log('Running Confluent CTP test...'); const outputConfluentCtp = skipCtpTest ? '' : - runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); + (await runCommand('MODE=confluent MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp')); console.log('Running KafkaJS CTP test...'); const outputKjsCtp = skipCtpTest ? '' : - runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp'); + (await runCommand('MODE=kafkajs MESSAGE_COUNT=5000 node performance-consolidated.js --create-topics --ctp')); // Extract Confluent results let ctpConfluent, ctpKjs; @@ -278,9 +281,11 @@ async function main() { console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); console.log(`Average RSS (eachMessage): confluent ${consumerConfluentMessageAverageRSS}, kafkajs ${consumerKjsMessageAverageRSS}`); console.log(`Max RSS (eachMessage): confluent ${consumerConfluentMessageMaxRSS}, kafkajs ${consumerKjsMessageMaxRSS}`); - console.log(`Average broker lag (eachMessage): confluent ${consumerConfluentMessageAverageBrokerLag}, kafkajs ${consumerKjsMessageAverageBrokerLag}`); - console.log(`Max broker lag (eachMessage): confluent ${consumerConfluentMessageMaxBrokerLag}, kafkajs ${consumerKjsMessageMaxBrokerLag}`); - console.log(`Sample size for broker lag measurement (eachMessage): confluent ${consumerConfluentMessageTotalLagMeasurements}, kafkajs ${consumerKjsMessageTotalLagMeasurements}`); + if (concurrentRun) { + console.log(`Average broker lag (eachMessage): confluent ${consumerConfluentMessageAverageBrokerLag}, kafkajs ${consumerKjsMessageAverageBrokerLag}`); + console.log(`Max broker lag (eachMessage): confluent ${consumerConfluentMessageMaxBrokerLag}, kafkajs ${consumerKjsMessageMaxBrokerLag}`); + console.log(`Sample size for broker lag measurement (eachMessage): confluent ${consumerConfluentMessageTotalLagMeasurements}, kafkajs ${consumerKjsMessageTotalLagMeasurements}`); + } } if (consumerModeAll || consumerModeEachBatch) { console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); @@ -297,9 +302,11 @@ async function main() { console.log(`Average eachBatch size: confluent ${consumerConfluentBatchAverageSize}, kafkajs ${consumerKjsBatchAverageSize}`); console.log(`Average RSS (eachBatch): confluent ${consumerConfluentBatchAverageRSS}, kafkajs ${consumerKjsBatchAverageRSS}`); console.log(`Max RSS (eachBatch): confluent ${consumerConfluentBatchMaxRSS}, kafkajs ${consumerKjsBatchMaxRSS}`); - console.log(`Average broker lag (eachBatch): confluent ${consumerConfluentBatchAverageBrokerLag}, kafkajs ${consumerKjsBatchAverageBrokerLag}`); - console.log(`Max broker lag (eachBatch): confluent ${consumerConfluentBatchMaxBrokerLag}, kafkajs ${consumerKjsBatchMaxBrokerLag}`); - console.log(`Sample size for broker lag measurement (eachBatch): confluent ${consumerConfluentBatchTotalLagMeasurements}, kafkajs ${consumerKjsBatchTotalLagMeasurements}`); + if (concurrentRun) { + console.log(`Average broker lag (eachBatch): confluent ${consumerConfluentBatchAverageBrokerLag}, kafkajs ${consumerKjsBatchAverageBrokerLag}`); + console.log(`Max broker lag (eachBatch): confluent ${consumerConfluentBatchMaxBrokerLag}, kafkajs ${consumerKjsBatchMaxBrokerLag}`); + console.log(`Sample size for broker lag measurement (eachBatch): confluent ${consumerConfluentBatchTotalLagMeasurements}, kafkajs ${consumerKjsBatchTotalLagMeasurements}`); + } } if (!concurrentRun) { console.log(`Average RSS: confluent ${consumerConfluentAverageRSS}, kafkajs ${consumerKjsAverageRSS}`); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 4684a709..7c573980 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -207,7 +207,9 @@ function logParameters(parameters) { console.log(` ProduceTopic: ${topic2}`); console.log(` Message Count: ${messageCount}`); // Seed the topic with messages - await runProducer(parameters, topic, batchSize, warmupMessages, messageCount, messageSize, compression); + await runProducerCKJS(parameters, topic, batchSize, + warmupMessages, messageCount, messageSize, compression, + randomness, limitRPS); startTrackingMemory(); const ctpRate = await runConsumeTransformProduce(parameters, topic, topic2, warmupMessages, messageCount, messageProcessTimeMs, ctpConcurrency); endTrackingMemory('consume-transform-produce', `consume-transform-produce-${mode}.json`); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index bb18c71e..ff1f0027 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -14,7 +14,7 @@ else { } } -function installHandlers() { +function installHandlers(useTerminateTimeout) { const handlers = { terminationRequested: false, terminateTimeout: null, @@ -26,8 +26,10 @@ function installHandlers() { process.on('SIGINT', terminationRequestedCallback); process.on('SIGTERM', terminationRequestedCallback); handlers.terminationRequestedCallback = terminationRequestedCallback; - handlers.terminateTimeout = setTimeout(terminationRequestedCallback, - TERMINATE_TIMEOUT_MS); + if (useTerminateTimeout) { + handlers.terminateTimeout = setTimeout(terminationRequestedCallback, + TERMINATE_TIMEOUT_MS); + } return handlers; } @@ -57,7 +59,7 @@ function genericProduceToTopic(producer, topic, messages) { } async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages) { - const handlers = installHandlers(); + const handlers = installHandlers(totalMessageCnt === -1); await consumer.connect(); await consumer.subscribe({ topic }); @@ -179,12 +181,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (!startTime) { startTime = hrtime.bigint(); } else if (totalMessageCnt > 0 && messagesMeasured >= totalMessageCnt) { - let durationNanos = Number(hrtime.bigint() - startTime); - durationSeconds = durationNanos / 1e9; - rate = durationNanos === 0 ? Infinity : - (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ - console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); + stopConsuming(); } } @@ -243,7 +240,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac } async function runProducer(producer, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS) { - const handlers = installHandlers(); + const handlers = installHandlers(totalMessageCnt === -1); let totalMessagesSent = 0; let totalBytesSent = 0; @@ -347,7 +344,7 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess } async function runLagMonitoring(admin, topic) { - const handlers = installHandlers(); + const handlers = installHandlers(true); let groupId = process.env.GROUPID_MONITOR; if (!groupId) { throw new Error("GROUPID_MONITOR environment variable not set"); From 0725638544a88605f8e0c7b707e4f4ec9b596a02 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 15:50:17 +0100 Subject: [PATCH 37/50] Two partitions consumed concurrently --- ci/tests/run_perf_test.js | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index a88870b0..7f2ef4ed 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -47,6 +47,12 @@ async function main() { if (concurrentRun) { skipCtpTest = true; } + if (!process.env.PARTITIONS_CONSUMED_CONCURRENTLY) { + process.env.PARTITIONS_CONSUMED_CONCURRENTLY = '2'; + } + if (!process.env.COMPRESSION) { + process.env.COMPRESSION = 'GZIP'; + } const consumerMode = process.env.CONSUMER_MODE || 'all'; const produceToSecondTopic = process.env.PRODUCE_TO_SECOND_TOPIC ? process.env.PRODUCE_TO_SECOND_TOPIC === 'true' : false; const produceToSecondTopicParam = produceToSecondTopic ? '--produce-to-second-topic' : ''; From a230484a0a395177a898db2a5a2753f4c47ff5be Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 19 Oct 2025 21:05:20 +0200 Subject: [PATCH 38/50] Remove E2E latency metrics for serial run --- ci/tests/run_perf_test.js | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 7f2ef4ed..f1774adb 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -278,16 +278,16 @@ async function main() { if (consumerModeAll || consumerModeEachMessage) { console.log(`Consumer rates MB/s (eachMessage): confluent ${consumerConfluentMessage}, kafkajs ${consumerKjsMessage}`); console.log(`Consumer rates msg/s (eachMessage): confluent ${consumerConfluentMessageRate}, kafkajs ${consumerKjsMessageRate}`); - console.log(`Consumer average E2E latency T0-T1 (eachMessage): confluent ${consumerConfluentMessageAvgLatencyT0T1}, kafkajs ${consumerKjsMessageAvgLatencyT0T1}`); - console.log(`Consumer max E2E latency T0-T1 (eachMessage): confluent ${consumerConfluentMessageMaxLatencyT0T1}, kafkajs ${consumerKjsMessageMaxLatencyT0T1}`); - if (produceToSecondTopic) { - console.log(`Consumer average E2E latency T0-T2 (eachMessage): confluent ${consumerConfluentMessageAvgLatencyT0T2}, kafkajs ${consumerKjsMessageAvgLatencyT0T2}`); - console.log(`Consumer max E2E latency T0-T2 (eachMessage): confluent ${consumerConfluentMessageMaxLatencyT0T2}, kafkajs ${consumerKjsMessageMaxLatencyT0T2}`); - } console.log(`Consumption time (eachMessage): confluent ${consumerConfluentTime}, kafkajs ${consumerKjsTime}`); console.log(`Average RSS (eachMessage): confluent ${consumerConfluentMessageAverageRSS}, kafkajs ${consumerKjsMessageAverageRSS}`); console.log(`Max RSS (eachMessage): confluent ${consumerConfluentMessageMaxRSS}, kafkajs ${consumerKjsMessageMaxRSS}`); if (concurrentRun) { + console.log(`Consumer average E2E latency T0-T1 (eachMessage): confluent ${consumerConfluentMessageAvgLatencyT0T1}, kafkajs ${consumerKjsMessageAvgLatencyT0T1}`); + console.log(`Consumer max E2E latency T0-T1 (eachMessage): confluent ${consumerConfluentMessageMaxLatencyT0T1}, kafkajs ${consumerKjsMessageMaxLatencyT0T1}`); + if (produceToSecondTopic) { + console.log(`Consumer average E2E latency T0-T2 (eachMessage): confluent ${consumerConfluentMessageAvgLatencyT0T2}, kafkajs ${consumerKjsMessageAvgLatencyT0T2}`); + console.log(`Consumer max E2E latency T0-T2 (eachMessage): confluent ${consumerConfluentMessageMaxLatencyT0T2}, kafkajs ${consumerKjsMessageMaxLatencyT0T2}`); + } console.log(`Average broker lag (eachMessage): confluent ${consumerConfluentMessageAverageBrokerLag}, kafkajs ${consumerKjsMessageAverageBrokerLag}`); console.log(`Max broker lag (eachMessage): confluent ${consumerConfluentMessageMaxBrokerLag}, kafkajs ${consumerKjsMessageMaxBrokerLag}`); console.log(`Sample size for broker lag measurement (eachMessage): confluent ${consumerConfluentMessageTotalLagMeasurements}, kafkajs ${consumerKjsMessageTotalLagMeasurements}`); @@ -296,19 +296,19 @@ async function main() { if (consumerModeAll || consumerModeEachBatch) { console.log(`Consumer rates MB/s (eachBatch): confluent ${consumerConfluentBatch}, kafkajs ${consumerKjsBatch}`); console.log(`Consumer rates msg/s (eachBatch): confluent ${consumerConfluentBatchRate}, kafkajs ${consumerKjsBatchRate}`); - console.log(`Consumer average E2E latency T0-T1 (eachBatch): confluent ${consumerConfluentBatchAvgLatencyT0T1}, kafkajs ${consumerKjsBatchAvgLatencyT0T1}`); - console.log(`Consumer max E2E latency T0-T1 (eachBatch): confluent ${consumerConfluentBatchMaxLatencyT0T1}, kafkajs ${consumerKjsBatchMaxLatencyT0T1}`); - if (produceToSecondTopic) { - console.log(`Consumer average E2E latency T0-T2 (eachBatch): confluent ${consumerConfluentBatchAvgLatencyT0T2}, kafkajs ${consumerKjsBatchAvgLatencyT0T2}`); - console.log(`Consumer max E2E latency T0-T2 (eachBatch): confluent ${consumerConfluentBatchMaxLatencyT0T2}, kafkajs ${consumerKjsBatchMaxLatencyT0T2}`); - } console.log(`Consumption time (eachBatch): confluent ${consumerConfluentBatchTime}, kafkajs ${consumerKjsBatchTime}`); - console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); - console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); console.log(`Average eachBatch size: confluent ${consumerConfluentBatchAverageSize}, kafkajs ${consumerKjsBatchAverageSize}`); console.log(`Average RSS (eachBatch): confluent ${consumerConfluentBatchAverageRSS}, kafkajs ${consumerKjsBatchAverageRSS}`); console.log(`Max RSS (eachBatch): confluent ${consumerConfluentBatchMaxRSS}, kafkajs ${consumerKjsBatchMaxRSS}`); if (concurrentRun) { + console.log(`Consumer average E2E latency T0-T1 (eachBatch): confluent ${consumerConfluentBatchAvgLatencyT0T1}, kafkajs ${consumerKjsBatchAvgLatencyT0T1}`); + console.log(`Consumer max E2E latency T0-T1 (eachBatch): confluent ${consumerConfluentBatchMaxLatencyT0T1}, kafkajs ${consumerKjsBatchMaxLatencyT0T1}`); + if (produceToSecondTopic) { + console.log(`Consumer average E2E latency T0-T2 (eachBatch): confluent ${consumerConfluentBatchAvgLatencyT0T2}, kafkajs ${consumerKjsBatchAvgLatencyT0T2}`); + console.log(`Consumer max E2E latency T0-T2 (eachBatch): confluent ${consumerConfluentBatchMaxLatencyT0T2}, kafkajs ${consumerKjsBatchMaxLatencyT0T2}`); + } + console.log(`Average eachBatch lag: confluent ${consumerConfluentBatchAverageLag}, kafkajs ${consumerKjsBatchAverageLag}`); + console.log(`Max eachBatch lag: confluent ${consumerConfluentBatchMaxLag}, kafkajs ${consumerKjsBatchMaxLag}`); console.log(`Average broker lag (eachBatch): confluent ${consumerConfluentBatchAverageBrokerLag}, kafkajs ${consumerKjsBatchAverageBrokerLag}`); console.log(`Max broker lag (eachBatch): confluent ${consumerConfluentBatchMaxBrokerLag}, kafkajs ${consumerKjsBatchMaxBrokerLag}`); console.log(`Sample size for broker lag measurement (eachBatch): confluent ${consumerConfluentBatchTotalLagMeasurements}, kafkajs ${consumerKjsBatchTotalLagMeasurements}`); From f4905b05ca6c44192f1b61439356f4b13f50ee42 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 24 Oct 2025 00:05:29 +0200 Subject: [PATCH 39/50] More accurate latency and throughput measurement --- ci/tests/run_perf_test.js | 16 +++++---- .../performance/performance-consolidated.js | 5 +++ .../performance-primitives-common.js | 33 ++++++++++++++++--- 3 files changed, 42 insertions(+), 12 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index f1774adb..197b5992 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -87,24 +87,26 @@ async function main() { if (concurrentRun) { console.log(`Running ${modeLabel} Producer/Consumer test (concurrently)...`); + const INITIAL_DELAY_MS = 2000; const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; - // Wait 2s more to see if all lag is caught up - const TERMINATE_TIMEOUT_MS_CONSUMERS = TERMINATE_TIMEOUT_MS + 2000; + // Wait INITIAL_DELAY_MS more to see if all lag is caught up, start earlier than the producer to check + // E2E latencies more accurately. + const TERMINATE_TIMEOUT_MS_CONSUMERS = TERMINATE_TIMEOUT_MS + INITIAL_DELAY_MS * 2; await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); const allPromises = []; - allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} node performance-consolidated.js --producer`)); + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} INITIAL_DELAY_MS=${INITIAL_DELAY_MS} node performance-consolidated.js --producer`)); if (consumerModeAll || consumerModeEachMessage) { - allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MESSAGE=${groupIdEachMessage} node performance-consolidated.js --consumer-each-message ${produceToSecondTopicParam}`)); + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MESSAGE=${groupIdEachMessage} node performance-consolidated.js --consumer-each-message ${produceToSecondTopicParam}`)); } if (consumerModeAll || consumerModeEachBatch) { - allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --consumer-each-batch ${produceToSecondTopicParam}`)); + allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --consumer-each-batch ${produceToSecondTopicParam}`)); } if (consumerModeAll || consumerModeEachMessage) { - allPromises.push(runCommand(`MODE=${mode} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachMessage} node performance-consolidated.js --monitor-lag`)); + allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachMessage} node performance-consolidated.js --monitor-lag`)); } if (consumerModeAll || consumerModeEachBatch) { - allPromises.push(runCommand(`MODE=${mode} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachBatch} node performance-consolidated.js --monitor-lag`)); + allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachBatch} node performance-consolidated.js --monitor-lag`)); } const results = await Promise.allSettled(allPromises); return results.map(r => r.status === 'fulfilled' ? r.value : '').join('\n'); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 7c573980..24ef4cb7 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -25,6 +25,7 @@ const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; const compression = process.env.COMPRESSION || 'None'; // Between 0 and 1, percentage of random bytes in each message const randomness = process.env.RANDOMNESS ? +process.env.RANDOMNESS : 0.5; +const initialDelayMs = process.env.INITIAL_DELAY_MS ? +process.env.INITIAL_DELAY_MS : 0; const numPartitions = process.env.PARTITIONS ? +process.env.PARTITIONS : 3; const partitionsConsumedConcurrently = process.env.PARTITIONS_CONSUMED_CONCURRENTLY ? +process.env.PARTITIONS_CONSUMED_CONCURRENTLY : 1; const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); @@ -101,6 +102,10 @@ function logParameters(parameters) { } console.log(`=== Starting Performance Tests - Mode ${mode} ===`); + if (initialDelayMs > 0) { + console.log(`=== Initial delay of ${initialDelayMs}ms before starting tests ===`); + await new Promise(resolve => setTimeout(resolve, initialDelayMs)); + } if (createTopics || all) { console.log("=== Creating Topics (deleting if they exist already):"); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index ff1f0027..33fcbbb6 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -60,8 +60,22 @@ function genericProduceToTopic(producer, topic, messages) { async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages) { const handlers = installHandlers(totalMessageCnt === -1); - await consumer.connect(); - await consumer.subscribe({ topic }); + while (true) { + try { + await consumer.connect(); + break; + } catch (e) { + console.error(`Error connecting consumer: ${e}`); + } + } + while (true) { + try { + await consumer.subscribe({ topic }); + break; + } catch (e) { + console.error(`Error subscribing consumer: ${e}`); + } + } let messagesReceived = 0; let messagesMeasured = 0; @@ -73,6 +87,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac let startTime; let rate; let consumptionStopped = false; + let lastMessageReceivedAt; const skippedMessages = warmupMessages; const decoder = new TextDecoder('utf-8'); @@ -111,11 +126,16 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (consumptionStopped) return; consumptionStopped = true; - let durationNanos = Number(hrtime.bigint() - startTime); + const now = lastMessageReceivedAt || hrtime.bigint(); + let durationNanos = Number(now - startTime); durationSeconds = durationNanos / 1e9; rate = (totalMessageSize / durationNanos) * 1e9 / (1024 * 1024); /* MB/s */ console.log(`Recvd ${messagesMeasured} messages in ${durationSeconds} seconds, ${totalMessageSize} bytes; rate is ${rate} MB/s`); - consumer.pause([{ topic }]); + try { + consumer.pause([{ topic }]); + } catch (e) { + console.error(`Error pausing consumer: ${e}`); + } } console.log("Starting consumer."); @@ -153,6 +173,8 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac consumeMethod = { partitionsConsumedConcurrently, eachBatch: async ({ batch }) => { + if (!batch.messages) + return; const messagesBeforeBatch = messagesReceived; const topic = batch.topic; const partition = batch.partition; @@ -170,6 +192,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac messages = messages.slice(messages.length - messagesMeasured); } const now = Date.now(); + lastMessageReceivedAt = hrtime.bigint(); messagesBase = messagesMeasured - messages.length; let i = 1; for (const message of messages) { @@ -187,7 +210,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac if (actionOnMessages) { await actionOnMessages(batch.messages); - if (messagesMeasured > 0 && messages.length > 0) { + if (messagesMeasured > 0 && messages && messages.length > 0) { let i = 1; const now = Date.now(); for (const message of messages) { From 89f4ea413003e76bc86c7f804d6cbbcb00a700a6 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 24 Oct 2025 17:55:53 +0200 Subject: [PATCH 40/50] Add latency percentiles calculation --- ci/tests/run_perf_test.js | 14 ++- .../performance/performance-consolidated.js | 16 ++- .../performance-primitives-common.js | 105 +++++++++++++++++- .../performance-primitives-kafkajs.js | 6 + .../performance/performance-primitives.js | 14 +++ 5 files changed, 147 insertions(+), 8 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 197b5992..3a4228e9 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -87,13 +87,19 @@ async function main() { if (concurrentRun) { console.log(`Running ${modeLabel} Producer/Consumer test (concurrently)...`); - const INITIAL_DELAY_MS = 2000; + const INITIAL_DELAY_MS = 10000; const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; // Wait INITIAL_DELAY_MS more to see if all lag is caught up, start earlier than the producer to check // E2E latencies more accurately. - const TERMINATE_TIMEOUT_MS_CONSUMERS = TERMINATE_TIMEOUT_MS + INITIAL_DELAY_MS * 2; + const TERMINATE_TIMEOUT_MS_CONSUMERS = TERMINATE_TIMEOUT_MS + INITIAL_DELAY_MS + 2000; + const TERMINATE_TIMEOUT_MS_LAG_MONITORING = TERMINATE_TIMEOUT_MS + 1000; await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); + + console.log(`Waiting 10s ms after topic creation before starting producer and consumers...`); + await new Promise(resolve => setTimeout(resolve, 10000)); + + console.log(`Starting producer and consumers...`); const allPromises = []; allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} INITIAL_DELAY_MS=${INITIAL_DELAY_MS} node performance-consolidated.js --producer`)); if (consumerModeAll || consumerModeEachMessage) { @@ -103,10 +109,10 @@ async function main() { allPromises.push(runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --consumer-each-batch ${produceToSecondTopicParam}`)); } if (consumerModeAll || consumerModeEachMessage) { - allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachMessage} node performance-consolidated.js --monitor-lag`)); + allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=${INITIAL_DELAY_MS} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_LAG_MONITORING} GROUPID_MONITOR=${groupIdEachMessage} node performance-consolidated.js --monitor-lag`)); } if (consumerModeAll || consumerModeEachBatch) { - allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=0 TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_CONSUMERS} GROUPID_MONITOR=${groupIdEachBatch} node performance-consolidated.js --monitor-lag`)); + allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=${INITIAL_DELAY_MS} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_LAG_MONITORING} GROUPID_MONITOR=${groupIdEachBatch} node performance-consolidated.js --monitor-lag`)); } const results = await Promise.allSettled(allPromises); return results.map(r => r.status === 'fulfilled' ? r.value : '').join('\n'); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 24ef4cb7..bfa165aa 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -17,8 +17,8 @@ const brokers = process.env.KAFKA_BROKERS || 'localhost:9092'; const securityProtocol = process.env.SECURITY_PROTOCOL; const saslUsername = process.env.SASL_USERNAME; const saslPassword = process.env.SASL_PASSWORD; -const topic = process.env.KAFKA_TOPIC || 'test-topic'; -const topic2 = process.env.KAFKA_TOPIC2 || 'test-topic2'; +const topic = process.env.KAFKA_TOPIC || `test-topic-${mode}`; +const topic2 = process.env.KAFKA_TOPIC2 || `test-topic2-${mode}`; const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; @@ -53,6 +53,13 @@ function logParameters(parameters) { } } +function printPercentiles(percentiles, type) { + for (const { percentile, value, count, total } of percentiles) { + const percentileStr = `P${percentile}`.padStart(6, ' '); + console.log(`=== Consumer ${percentileStr} E2E latency ${type}: ${value.toFixed(2)} ms (${count}/${total})`); + } +} + (async function () { const producer = process.argv.includes('--producer'); const consumer = process.argv.includes('--consumer'); @@ -169,10 +176,11 @@ function logParameters(parameters) { endTrackingMemory('consumer-each-message', `consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate MB/s (eachMessage): ", consumerRate); console.log("=== Consumer Rate msg/s (eachMessage): ", stats.messageRate); - console.log("=== Consumer average E2E latency T0-T1 (eachMessage): ", stats.avgLatencyT0T1); + printPercentiles(stats.percentilesTOT1, 'T0-T1 (eachMessage)'); console.log("=== Consumer max E2E latency T0-T1 (eachMessage): ", stats.maxLatencyT0T1); if (produceToSecondTopic) { console.log("=== Consumer average E2E latency T0-T2 (eachMessage): ", stats.avgLatencyT0T2); + printPercentiles(stats.percentilesTOT2, 'T0-T2 (eachMessage)'); console.log("=== Consumer max E2E latency T0-T2 (eachMessage): ", stats.maxLatencyT0T2); } console.log("=== Consumption time (eachMessage): ", stats.durationSeconds); @@ -197,9 +205,11 @@ function logParameters(parameters) { console.log("=== Max eachBatch lag: ", stats.maxOffsetLag); console.log("=== Average eachBatch size: ", stats.averageBatchSize); console.log("=== Consumer average E2E latency T0-T1 (eachBatch): ", stats.avgLatencyT0T1); + printPercentiles(stats.percentilesTOT1, 'T0-T1 (eachBatch)'); console.log("=== Consumer max E2E latency T0-T1 (eachBatch): ", stats.maxLatencyT0T1); if (produceToSecondTopic) { console.log("=== Consumer average E2E latency T0-T2 (eachBatch): ", stats.avgLatencyT0T2); + printPercentiles(stats.percentilesTOT2, 'T0-T2 (eachBatch)'); console.log("=== Consumer max E2E latency T0-T2 (eachBatch): ", stats.maxLatencyT0T2); } console.log("=== Consumption time (eachBatch): ", stats.durationSeconds); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 33fcbbb6..f3c36e24 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -1,5 +1,6 @@ const { hrtime } = require('process'); const { randomBytes } = require('crypto'); +const PERCENTILES = [50, 75, 90, 95, 99, 99.9, 99.99, 100]; const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; const AUTO_COMMIT = process.env.AUTO_COMMIT || 'false'; @@ -58,8 +59,86 @@ function genericProduceToTopic(producer, topic, messages) { }); } + +// We use a simple count-sketch for latency percentiles to avoid storing all latencies in memory. +// because we're also measuring the memory usage of the consumer as part of the performance tests. +class LatencyCountSketch { + #numBuckets; + #minValue; + #maxValue; + #buckets; + #counts; + #changeBaseLogarithm; + #totalCount = 0; + #base; + + constructor({ + error = 0.01, // 1% error + minValue = 0.01, // min 10μs latency + maxValue = 60000, // max 60s latency + }) { + // Each bucket represents [x, x * (1 + error)) + this.#base = 1 + error; + // Change base from natural log to log base this.#base + this.#changeBaseLogarithm = Math.log(this.#base); + this.#numBuckets = Math.ceil(Math.log(maxValue / minValue) / Math.log(this.#base)); + this.#maxValue = maxValue; + + this.#buckets = new Array(this.#numBuckets + 2).fill(0); + this.#buckets[this.#numBuckets + 1] = Number.POSITIVE_INFINITY; + this.#buckets[this.#numBuckets] = this.#maxValue; + this.#buckets[0] = 0; + let i = this.#numBuckets - 1; + let currentValue = maxValue; + while (i >= 1) { + let nextMinimum = currentValue / this.#base; + this.#buckets[i] = nextMinimum; + currentValue = nextMinimum; + i--; + } + this.#minValue = this.#buckets[1]; + this.#counts = new Array(this.#numBuckets + 2).fill(0); + } + + add(latency) { + let idx = 0; + if (latency > 0) + idx = Math.ceil(Math.log(latency / this.#minValue) / this.#changeBaseLogarithm); + idx = (idx < 0) ? 0 : + (idx > this.#buckets.length - 2) ? (this.#buckets.length - 2) : + idx; + + this.#counts[idx]++; + this.#totalCount++; + } + + percentiles(percentilesArray) { + const percentileCounts = percentilesArray.map(p => Math.ceil(this.#totalCount * p / 100)); + const percentileResults = new Array(percentilesArray.length); + var totalCountSoFar = 0; + let j = 0; + let sum = 0; + for (let i = 0; i < this.#counts.length; i++) { + sum += this.#counts[i]; + } + for (let i = 0; i < percentileCounts.length; i++) { + while ((totalCountSoFar < percentileCounts[i]) && (j < this.#counts.length - 1)) { + totalCountSoFar += this.#counts[j]; + j++; + } + const bucketIndex = (j < this.#counts.length - 1) ? j : this.#counts.length - 2; + percentileResults[i] = [this.#buckets[bucketIndex], totalCountSoFar, this.#totalCount]; + } + return percentileResults; + } +} + async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages) { const handlers = installHandlers(totalMessageCnt === -1); + if (stats) { + stats.percentilesTOT1 = new LatencyCountSketch({}); + stats.percentilesTOT2 = new LatencyCountSketch({}); + } while (true) { try { await consumer.connect(); @@ -96,7 +175,17 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac return; const sentAt = Number(decoder.decode(message.value.slice(0, 13))); - const latency = receivedAt - sentAt; + let latency = receivedAt - sentAt; + + if (isNaN(latency)) { + console.log(`WARN: NaN latency received message timestamp: ${message.value.slice(0, 13)}`); + return; + } else if (latency < 0) { + console.log(`WARN: negative latency ${latency} sentAt ${sentAt} receivedAt ${receivedAt}`); + latency = 0; + } else if (latency > 60000) { + console.log(`WARN: received large latency ${latency} sentAt ${sentAt} receivedAt ${receivedAt}`); + } if (!isT0T2) { if (!stats.maxLatencyT0T1) { @@ -106,6 +195,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac stats.maxLatencyT0T1 = Math.max(stats.maxLatencyT0T1, latency); stats.avgLatencyT0T1 = ((stats.avgLatencyT0T1 * (numMessages - 1)) + latency) / numMessages; } + stats.percentilesTOT1.add(latency); } else { if (!stats.maxLatencyT0T2) { stats.maxLatencyT0T2 = latency; @@ -114,6 +204,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac stats.maxLatencyT0T2 = Math.max(stats.maxLatencyT0T2, latency); stats.avgLatencyT0T2 = ((stats.avgLatencyT0T2 * (numMessages - 1)) + latency) / numMessages; } + stats.percentilesTOT2.add(latency); } }; @@ -257,6 +348,18 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac stats.messageRate = durationSeconds > 0 ? (messagesMeasured / durationSeconds) : Infinity; stats.durationSeconds = durationSeconds; + stats.percentilesTOT1 = stats.percentilesTOT1.percentiles(PERCENTILES).map((value, index) => ({ + percentile: PERCENTILES[index], + value: value[0], + count: value[1], + total: value[2], + })); + stats.percentilesTOT2 = stats.percentilesTOT2.percentiles(PERCENTILES).map((value, index) => ({ + percentile: PERCENTILES[index], + value: value[0], + count: value[1], + total: value[2], + })); } removeHandlers(handlers); return rate; diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 1b302297..da240e3f 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -20,6 +20,8 @@ module.exports = { runProducerConsumerTogether, }; +const IS_HIGHER_LATENCY_CLUSTER = process.env.IS_HIGHER_LATENCY_CLUSTER === 'true'; + function baseConfiguration(parameters) { let ret = { clientId: 'kafka-test-performance', @@ -147,6 +149,9 @@ class CompatibleConsumer { function newCompatibleConsumer(parameters, eachBatch) { const kafka = new Kafka(baseConfiguration(parameters)); + const higherLatencyClusterOpts = IS_HIGHER_LATENCY_CLUSTER ? { + maxBytesPerPartition: 8388608 + } : {}; let groupId = eachBatch ? process.env.GROUPID_BATCH : process.env.GROUPID_MESSAGE; if (!groupId) { @@ -154,6 +159,7 @@ function newCompatibleConsumer(parameters, eachBatch) { } console.log(`New KafkaJS group id: ${groupId}`); const consumer = kafka.consumer({ + ...higherLatencyClusterOpts, groupId, }); return new CompatibleConsumer(consumer); diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index 84d56383..e9b28b6d 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -19,6 +19,9 @@ module.exports = { newCompatibleProducer, }; + +const IS_HIGHER_LATENCY_CLUSTER = process.env.IS_HIGHER_LATENCY_CLUSTER === 'true'; + function baseConfiguration(parameters) { let ret = { 'client.id': 'kafka-test-performance', @@ -97,9 +100,16 @@ class CompatibleProducer { } } function newCompatibleProducer(parameters, compression) { + const higherLatencyClusterOpts = IS_HIGHER_LATENCY_CLUSTER ? { + 'linger.ms': '200', + 'sticky.partitioning.linger.ms': '200', + 'message.max.bytes': '2148352', + 'batch.size': '2097152', + } : {}; return new CompatibleProducer( new Kafka({ ...baseConfiguration(parameters), + ...higherLatencyClusterOpts, 'compression.codec': CompressionTypes[compression], }).producer()); } @@ -146,6 +156,9 @@ function newCompatibleConsumer(parameters, eachBatch) { const autoCommitOpts = autoCommit > 0 ? { 'enable.auto.commit': true, 'auto.commit.interval.ms': autoCommit } : { 'enable.auto.commit': false }; + const higherLatencyClusterOpts = IS_HIGHER_LATENCY_CLUSTER ? { + 'max.partition.fetch.bytes': '8388608' + } : {}; let groupId = eachBatch ? process.env.GROUPID_BATCH : process.env.GROUPID_MESSAGE; if (!groupId) { @@ -157,6 +170,7 @@ function newCompatibleConsumer(parameters, eachBatch) { 'auto.offset.reset': 'earliest', 'fetch.queue.backoff.ms': '100', ...autoCommitOpts, + ...higherLatencyClusterOpts, }); return new CompatibleConsumer(consumer); } From 195cf73d267a25425fb9855cb33504d9e6f21c81 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:21:39 +0100 Subject: [PATCH 41/50] USE_KEYS and timestamp in headers --- .../performance-primitives-common.js | 32 ++++++++++++------- .../performance-primitives-kafkajs.js | 3 +- 2 files changed, 22 insertions(+), 13 deletions(-) diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index f3c36e24..bde4d900 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -5,6 +5,8 @@ const PERCENTILES = [50, 75, 90, 95, 99, 99.9, 99.99, 100]; const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; const AUTO_COMMIT = process.env.AUTO_COMMIT || 'false'; const AUTO_COMMIT_ON_BATCH_END = process.env.AUTO_COMMIT_ON_BATCH_END === 'true'; +const USE_KEYS = process.env.USE_KEYS === 'true'; + let autoCommit; if (AUTO_COMMIT && AUTO_COMMIT === 'false') autoCommit = null; @@ -168,13 +170,16 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac let consumptionStopped = false; let lastMessageReceivedAt; const skippedMessages = warmupMessages; - const decoder = new TextDecoder('utf-8'); const updateLatency = (receivedAt, numMessages, message, isT0T2) => { if (!stats) return; - const sentAt = Number(decoder.decode(message.value.slice(0, 13))); + if (!message.headers || !message.headers['timestamp']) { + console.log('WARN: message without timestamp header received, cannot measure latency'); + return; + } + const sentAt = Number(message.headers['timestamp']); let latency = receivedAt - sentAt; if (isNaN(latency)) { @@ -370,15 +375,11 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess let totalMessagesSent = 0; let totalBytesSent = 0; - const encoder = new TextEncoder(); let staticValueLength = Math.floor(msgSize * (1 - randomness)); - if (staticValueLength < 13) - staticValueLength = 13; - let staticValueRemainder = staticValueLength - 13; - if (staticValueRemainder > 0) { - staticValueRemainder = randomBytes(staticValueRemainder); + if (staticValueLength > 0) { + staticValueBytes = randomBytes(staticValueLength); } else { - staticValueRemainder = Buffer.alloc(0); + staticValueBytes = Buffer.alloc(0); } let messageCnt = totalMessageCnt; @@ -389,8 +390,11 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess for (let i = 0; i < messageCnt; i++) { /* Generate a different random value for each message */ messages[i] = { - value: Buffer.concat([staticValueRemainder, randomBytes(msgSize - staticValueLength)]), + value: Buffer.concat([staticValueBytes, randomBytes(msgSize - staticValueLength)]), }; + if (USE_KEYS) { + messages[i].key = randomBytes(8); + } } await producer.connect(); @@ -422,8 +426,12 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess const modifiedMessages = []; const batchStart = messagesDispatched % messageCnt; for (const msg of messages.slice(batchStart, batchStart + batchSize)) { - modifiedMessages.push({ - value: Buffer.concat([encoder.encode(Date.now().toString()), msg.value]) + modifiedMessages.push({ + key: msg.key, + value: msg.value, + headers: { + 'timestamp': Date.now().toString(), + } }); } promises.push(producer.send({ diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index da240e3f..951e5f6d 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -1,4 +1,4 @@ -const { Kafka, CompressionTypes } = require('kafkajs'); +const { Kafka, CompressionTypes, logLevel } = require('kafkajs'); const { randomBytes } = require('crypto'); const { hrtime } = require('process'); const { @@ -26,6 +26,7 @@ function baseConfiguration(parameters) { let ret = { clientId: 'kafka-test-performance', brokers: parameters.brokers.split(','), + logLevel: logLevel.WARN, }; if (parameters.securityProtocol && parameters.saslUsername && From f0fc2987b7c46ce8cd08967a094f09a8b88f7fba Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Oct 2025 12:37:54 +0100 Subject: [PATCH 42/50] Don't modify passed messages in sendOptions --- ci/tests/run_perf_test.js | 2 +- lib/kafkajs/_producer.js | 25 +++++++++++++++---------- 2 files changed, 16 insertions(+), 11 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 3a4228e9..aece4d3f 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -96,7 +96,7 @@ async function main() { await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); - console.log(`Waiting 10s ms after topic creation before starting producer and consumers...`); + console.log(`Waiting 10s after topic creation before starting producer and consumers...`); await new Promise(resolve => setTimeout(resolve, 10000)); console.log(`Starting producer and consumers...`); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index bf1e058d..d48c6551 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -622,27 +622,32 @@ class Producer { const msgPromises = []; for (let i = 0; i < sendOptions.messages.length; i++) { const msg = sendOptions.messages[i]; - - if (!Object.hasOwn(msg, "partition") || msg.partition === null) { - msg.partition = -1; + let key = msg.key; + let value = msg.value; + let partition = msg.partition; + let headers = msg.headers; + let timestamp = msg.timestamp; + + if (partition === undefined || msg.partition === null) { + partition = -1; } - if (typeof msg.value === 'string') { - msg.value = Buffer.from(msg.value); + if (typeof value === 'string') { + value = Buffer.from(value); } - if (Object.hasOwn(msg, "timestamp") && msg.timestamp) { - msg.timestamp = Number(msg.timestamp); + if (timestamp) { + timestamp = Number(timestamp); } else { - msg.timestamp = 0; + timestamp = 0; } - msg.headers = convertToRdKafkaHeaders(msg.headers); + headers = convertToRdKafkaHeaders(headers); msgPromises.push(new Promise((resolve, reject) => { const opaque = { resolve, reject }; try { - this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp, opaque, msg.headers); + this.#internalClient.produce(sendOptions.topic, partition, value, key, timestamp, opaque, headers); } catch (err) { reject(err); } From 560d492423256048fa89e9d8f36750153389ad92 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:34:39 +0100 Subject: [PATCH 43/50] CONSUMER_MAX_BATCH_SIZE in performance test --- ci/tests/run_perf_test.js | 3 +++ examples/performance/performance-consolidated.js | 4 ++-- examples/performance/performance-primitives.js | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index aece4d3f..86b78696 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -47,6 +47,9 @@ async function main() { if (concurrentRun) { skipCtpTest = true; } + if (!process.env.CONSUMER_MAX_BATCH_SIZE) { + process.env.CONSUMER_MAX_BATCH_SIZE = '-1'; + } if (!process.env.PARTITIONS_CONSUMED_CONCURRENTLY) { process.env.PARTITIONS_CONSUMED_CONCURRENTLY = '2'; } diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index bfa165aa..53fec870 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -20,8 +20,8 @@ const saslPassword = process.env.SASL_PASSWORD; const topic = process.env.KAFKA_TOPIC || `test-topic-${mode}`; const topic2 = process.env.KAFKA_TOPIC2 || `test-topic2-${mode}`; const messageCount = process.env.MESSAGE_COUNT ? +process.env.MESSAGE_COUNT : 1000000; -const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 256; -const batchSize = process.env.BATCH_SIZE ? +process.env.BATCH_SIZE : 100; +const messageSize = process.env.MESSAGE_SIZE ? +process.env.MESSAGE_SIZE : 4096; +const batchSize = process.env.PRODUCER_BATCH_SIZE ? +process.env.PRODUCER_BATCH_SIZE : 100; const compression = process.env.COMPRESSION || 'None'; // Between 0 and 1, percentage of random bytes in each message const randomness = process.env.RANDOMNESS ? +process.env.RANDOMNESS : 0.5; diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index e9b28b6d..a7349c6a 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -20,6 +20,7 @@ module.exports = { }; +const CONSUMER_MAX_BATCH_SIZE = process.env.CONSUMER_MAX_BATCH_SIZE ? +process.env.CONSUMER_MAX_BATCH_SIZE : null; const IS_HIGHER_LATENCY_CLUSTER = process.env.IS_HIGHER_LATENCY_CLUSTER === 'true'; function baseConfiguration(parameters) { @@ -156,6 +157,9 @@ function newCompatibleConsumer(parameters, eachBatch) { const autoCommitOpts = autoCommit > 0 ? { 'enable.auto.commit': true, 'auto.commit.interval.ms': autoCommit } : { 'enable.auto.commit': false }; + const jsOpts = (eachBatch && CONSUMER_MAX_BATCH_SIZE !== null) ? { + 'js.consumer.max.batch.size': CONSUMER_MAX_BATCH_SIZE + } : {}; const higherLatencyClusterOpts = IS_HIGHER_LATENCY_CLUSTER ? { 'max.partition.fetch.bytes': '8388608' } : {}; @@ -170,6 +174,7 @@ function newCompatibleConsumer(parameters, eachBatch) { 'auto.offset.reset': 'earliest', 'fetch.queue.backoff.ms': '100', ...autoCommitOpts, + ...jsOpts, ...higherLatencyClusterOpts, }); return new CompatibleConsumer(consumer); From 1f3db883f9068fae85463649ccef2345c436c9dc Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 16:54:01 +0100 Subject: [PATCH 44/50] Changelog entry for `producer.send` fix --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7c05b809..39c7cf2c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ v1.7.0 is a feature release. It is supported for all usage. property (#393). 2. Fix for at-least-once guarantee not ensured in case a seek happens on one partition and there are messages being fetched about other partitions (#393). +3. Messages passed to the producer `send` method are kept constant (#394). # confluent-kafka-javascript 1.6.0 From 58d611e23c9cee2c8d6b22eba5c13fe45f45bcfd Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 29 Oct 2025 17:58:42 +0100 Subject: [PATCH 45/50] Fix to avoid counting the message size two times --- examples/performance/performance-primitives-common.js | 1 - 1 file changed, 1 deletion(-) diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index bde4d900..50a05f96 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -310,7 +310,6 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac let i = 1; const now = Date.now(); for (const message of messages) { - totalMessageSize += message.value.length; updateLatency(now, messagesBase + i, message, true); i++; } From bd146693006c408813d1562b71d16977a5ba54f1 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 4 Nov 2025 12:06:10 +0100 Subject: [PATCH 46/50] Fix in cases for large batch sizes --- examples/performance/performance-primitives-common.js | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 50a05f96..7321522a 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -383,7 +383,9 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess let messageCnt = totalMessageCnt; if (messageCnt === -1) { - messageCnt = batchSize * 10000; + if (batchSize > 20000) + batchSize = 20000; + messageCnt = Math.ceil(20000 / batchSize) * batchSize; } const messages = Array(messageCnt); for (let i = 0; i < messageCnt; i++) { From 00187f4518e562899e6257695f98e9c1ad04e566 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 4 Nov 2025 15:41:33 +0100 Subject: [PATCH 47/50] Fix use warmup messages with headers in case messages from other partitions are consumed first --- ci/tests/run_perf_test.js | 5 +++-- examples/performance/performance-consolidated.js | 2 +- .../performance/performance-primitives-common.js | 13 ++++++++++--- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/ci/tests/run_perf_test.js b/ci/tests/run_perf_test.js index 86b78696..93e534a7 100644 --- a/ci/tests/run_perf_test.js +++ b/ci/tests/run_perf_test.js @@ -97,7 +97,8 @@ async function main() { const TERMINATE_TIMEOUT_MS_CONSUMERS = TERMINATE_TIMEOUT_MS + INITIAL_DELAY_MS + 2000; const TERMINATE_TIMEOUT_MS_LAG_MONITORING = TERMINATE_TIMEOUT_MS + 1000; - await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); + const createTopicResult = await runCommand(`MODE=${mode} node performance-consolidated.js --create-topics`); + let ret = [createTopicResult]; console.log(`Waiting 10s after topic creation before starting producer and consumers...`); await new Promise(resolve => setTimeout(resolve, 10000)); @@ -118,7 +119,7 @@ async function main() { allPromises.push(runCommand(`MODE=${mode} INITIAL_DELAY_MS=${INITIAL_DELAY_MS} TERMINATE_TIMEOUT_MS=${TERMINATE_TIMEOUT_MS_LAG_MONITORING} GROUPID_MONITOR=${groupIdEachBatch} node performance-consolidated.js --monitor-lag`)); } const results = await Promise.allSettled(allPromises); - return results.map(r => r.status === 'fulfilled' ? r.value : '').join('\n'); + return ret.concat(results.map(r => r.status === 'fulfilled' ? r.value : '')).join('\n'); } else { console.log(`Running ${modeLabel} Producer/Consumer test...`); return runCommand(`MODE=${mode} MESSAGE_COUNT=${messageCount} GROUPID_MESSAGE=${groupIdEachMessage} GROUPID_BATCH=${groupIdEachBatch} node performance-consolidated.js --create-topics ${consumerParam} ${produceToSecondTopicParam} --producer`); diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 53fec870..907b0a06 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -28,7 +28,7 @@ const randomness = process.env.RANDOMNESS ? +process.env.RANDOMNESS : 0.5; const initialDelayMs = process.env.INITIAL_DELAY_MS ? +process.env.INITIAL_DELAY_MS : 0; const numPartitions = process.env.PARTITIONS ? +process.env.PARTITIONS : 3; const partitionsConsumedConcurrently = process.env.PARTITIONS_CONSUMED_CONCURRENTLY ? +process.env.PARTITIONS_CONSUMED_CONCURRENTLY : 1; -const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : (batchSize * 10); +const warmupMessages = process.env.WARMUP_MESSAGES ? +process.env.WARMUP_MESSAGES : ((batchSize * 10 > 1024) ? 1024 : (batchSize * 10)); const messageProcessTimeMs = process.env.MESSAGE_PROCESS_TIME_MS ? +process.env.MESSAGE_PROCESS_TIME_MS : 5; const ctpConcurrency = process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY ? +process.env.CONSUME_TRANSFORM_PRODUCE_CONCURRENCY : 1; const consumerProcessingTime = process.env.CONSUMER_PROCESSING_TIME ? +process.env.CONSUMER_PROCESSING_TIME : 100; diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 7321522a..b6dc176c 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -172,7 +172,7 @@ async function runConsumer(consumer, topic, warmupMessages, totalMessageCnt, eac const skippedMessages = warmupMessages; const updateLatency = (receivedAt, numMessages, message, isT0T2) => { - if (!stats) + if (!stats || numMessages < 1) return; if (!message.headers || !message.headers['timestamp']) { @@ -402,11 +402,18 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess console.log('Sending ' + warmupMessages + ' warmup messages.'); while (warmupMessages > 0) { + const toProduce = Math.min(batchSize, warmupMessages); await producer.send({ topic, - messages: messages.slice(0, batchSize) + messages: messages.slice(0, toProduce).map((msg) => ({ + key: msg.key, + value: msg.value, + headers: { + 'timestamp': Date.now().toString(), + } + })) }, compression); - warmupMessages -= batchSize; + warmupMessages -= toProduce; } console.log('Sent warmup messages'); From c8d5c77641234c9ada562f80cb1a7e60ed905ec3 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 4 Nov 2025 21:39:42 +0100 Subject: [PATCH 48/50] Stats callback --- lib/kafkajs/_admin.js | 15 +++++++++++++++ lib/kafkajs/_consumer.js | 16 ++++++++++++++++ lib/kafkajs/_producer.js | 15 +++++++++++++++ 3 files changed, 46 insertions(+) diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index ed713546..a4ce1bac 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -174,6 +174,12 @@ class Admin { /* Delete properties which are already processed, or cannot be passed to node-rdkafka */ delete rdKafkaConfig.kafkaJS; + if (Object.hasOwn(rdKafkaConfig, 'stats_cb')) { + if (typeof rdKafkaConfig['stats_cb'] === 'function') + this.#statsCb = rdKafkaConfig['stats_cb']; + delete rdKafkaConfig['stats_cb']; + } + return rdKafkaConfig; } @@ -203,6 +209,12 @@ class Admin { } } + /** + * Callback for the event.stats event, if defined. + * @private + */ + #statsCb = null; + /** * Set up the client and connect to the bootstrap brokers. * @returns {Promise} Resolves when connection is complete, rejects on error. @@ -225,6 +237,9 @@ class Admin { 'ready': this.#readyCb.bind(this), 'event.log': (msg) => loggerTrampoline(msg, this.#logger), }); + if (this.#statsCb) { + this.#internalClient.on('event.stats', this.#statsCb.bind(this)); + } } else { const underlyingClient = this.#existingClient._getInternalClient(); if (!underlyingClient) { diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 92efb5c3..37b045de 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -731,6 +731,12 @@ class Consumer { delete rdKafkaConfig['js.consumer.max.cache.size.per.worker.ms']; } + if (Object.hasOwn(rdKafkaConfig, 'stats_cb')) { + if (typeof rdKafkaConfig['stats_cb'] === 'function') + this.#statsCb = rdKafkaConfig['stats_cb']; + delete rdKafkaConfig['stats_cb']; + } + return rdKafkaConfig; } @@ -770,6 +776,13 @@ class Consumer { } } + + /** + * Callback for the event.stats event, if defined. + * @private + */ + #statsCb = null; + /** * Converts headers returned by node-rdkafka into a format that can be used by the eachMessage/eachBatch callback. * @param {import("../..").MessageHeader[] | undefined} messageHeaders @@ -1167,6 +1180,9 @@ class Consumer { this.#internalClient.on('error', this.#errorCb.bind(this)); this.#internalClient.on('event.error', this.#errorCb.bind(this)); this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); + if (this.#statsCb) { + this.#internalClient.on('event.stats', this.#statsCb.bind(this)); + } return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index d48c6551..d3130d1f 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -269,6 +269,12 @@ class Producer { /* TODO: Add a warning if dr_cb is set? Or else, create a trampoline for it. */ rdKafkaConfig.dr_cb = true; + if (Object.hasOwn(rdKafkaConfig, 'stats_cb')) { + if (typeof rdKafkaConfig['stats_cb'] === 'function') + this.#statsCb = rdKafkaConfig['stats_cb']; + delete rdKafkaConfig['stats_cb']; + } + return rdKafkaConfig; } @@ -373,6 +379,12 @@ class Producer { this.#logger.error(err, this.#createProducerBindingMessageMetadata()); } + /** + * Callback for the event.stats event, if defined. + * @private + */ + #statsCb = null; + /** * Set up the client and connect to the bootstrap brokers. * @@ -394,6 +406,9 @@ class Producer { this.#internalClient.on('event.error', this.#errorCb.bind(this)); this.#internalClient.on('error', this.#errorCb.bind(this)); this.#internalClient.on('event.log', (msg) => loggerTrampoline(msg, this.#logger)); + if (this.#statsCb) { + this.#internalClient.on('event.stats', this.#statsCb.bind(this)); + } return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; From 5eaa4e2fc32bb63a44b27f1e3f854f98a5ed2f44 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 5 Nov 2025 11:05:50 +0100 Subject: [PATCH 49/50] fixup performance example --- .../performance-primitives-common.js | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index b6dc176c..687ab77e 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -433,7 +433,8 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess while (totalMessageCnt == -1 || messagesDispatched < totalMessageCnt) { const modifiedMessages = []; const batchStart = messagesDispatched % messageCnt; - for (const msg of messages.slice(batchStart, batchStart + batchSize)) { + for (let i = 0; i < batchSize; i++) { + const msg = messages[(batchStart + i) % messages.length]; modifiedMessages.push({ key: msg.key, value: msg.value, @@ -441,26 +442,27 @@ async function runProducer(producer, topic, batchSize, warmupMessages, totalMess 'timestamp': Date.now().toString(), } }); + if (messagesNotAwaited + i + 1 >= maxToAwait) + break; } + const toProduce = modifiedMessages.length; promises.push(producer.send({ topic, messages: modifiedMessages, }, compression).then(() => { - totalMessagesSent += batchSize; - totalBytesSent += batchSize * msgSize; + totalMessagesSent += toProduce; + totalBytesSent += toProduce * msgSize; }).catch((err) => { if (producer.isQueueFullError(err)) { /* do nothing, just send them again */ - messagesDispatched -= batchSize; - totalMessagesSent -= batchSize; - totalBytesSent -= batchSize * msgSize; + messagesDispatched -= toProduce; } else { console.error(err); throw err; } })); - messagesDispatched += batchSize; - messagesNotAwaited += batchSize; + messagesDispatched += toProduce; + messagesNotAwaited += toProduce; if (handlers.terminationRequested || messagesNotAwaited >= maxToAwait) break; } From d78d6e6fb5481cb0834f951b674075b57e8f03cc Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 5 Nov 2025 12:00:49 +0100 Subject: [PATCH 50/50] Logger and statistics callback --- .../performance/performance-consolidated.js | 6 +- .../performance-primitives-common.js | 38 +++++++++++++ .../performance-primitives-kafkajs.js | 33 +++++++++-- .../performance/performance-primitives.js | 57 +++++++++++++++++-- 4 files changed, 122 insertions(+), 12 deletions(-) diff --git a/examples/performance/performance-consolidated.js b/examples/performance/performance-consolidated.js index 907b0a06..c2191aa6 100644 --- a/examples/performance/performance-consolidated.js +++ b/examples/performance/performance-consolidated.js @@ -172,7 +172,8 @@ function printPercentiles(percentiles, type) { const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, false, partitionsConsumedConcurrently, stats, - produceToSecondTopic ? topic2 : null, compression, useCKJSProducerEverywhere); + produceToSecondTopic ? topic2 : null, compression, useCKJSProducerEverywhere, + messageSize, limitRPS); endTrackingMemory('consumer-each-message', `consumer-memory-message-${mode}.json`); console.log("=== Consumer Rate MB/s (eachMessage): ", consumerRate); console.log("=== Consumer Rate msg/s (eachMessage): ", stats.messageRate); @@ -197,7 +198,8 @@ function printPercentiles(percentiles, type) { const consumerRate = await runConsumer(parameters, topic, warmupMessages, messageCount, true, partitionsConsumedConcurrently, stats, - produceToSecondTopic ? topic2 : null, compression, useCKJSProducerEverywhere); + produceToSecondTopic ? topic2 : null, compression, useCKJSProducerEverywhere, + messageSize, limitRPS); endTrackingMemory('consumer-each-batch', `consumer-memory-batch-${mode}.json`); console.log("=== Consumer Rate MB/s (eachBatch): ", consumerRate); console.log("=== Consumer Rate msg/s (eachBatch): ", stats.messageRate); diff --git a/examples/performance/performance-primitives-common.js b/examples/performance/performance-primitives-common.js index 687ab77e..cf156aff 100644 --- a/examples/performance/performance-primitives-common.js +++ b/examples/performance/performance-primitives-common.js @@ -1,5 +1,7 @@ const { hrtime } = require('process'); const { randomBytes } = require('crypto'); +const fs = require('fs'); +const { logLevel } = require('../../').KafkaJS; const PERCENTILES = [50, 75, 90, 95, 99, 99.9, 99.99, 100]; const TERMINATE_TIMEOUT_MS = process.env.TERMINATE_TIMEOUT_MS ? +process.env.TERMINATE_TIMEOUT_MS : 600000; @@ -17,6 +19,41 @@ else { } } +class PerformanceLogger { + constructor(fileName) { + this.logLevel = logLevel.INFO; + this.writeStream = fs.createWriteStream(fileName); + } + + setLogLevel(logLevel) { + this.logLevel = logLevel; + } + + info(message, extra) { + if (this.logLevel >= logLevel.INFO) + this.writeStream.write(`INFO: ${message} ${JSON.stringify(extra)}\n`); + } + + error(message, extra) { + if (this.logLevel >= logLevel.ERROR) + this.writeStream.write(`ERROR: ${message} ${JSON.stringify(extra)}\n`); + } + + warn(message, extra) { + if (this.logLevel >= logLevel.WARN) + this.writeStream.write(`WARN: ${message} ${JSON.stringify(extra)}\n`); + } + + debug(message, extra) { + if (this.logLevel >= logLevel.DEBUG) + this.writeStream.write(`DEBUG: ${message} ${JSON.stringify(extra)}\n`); + } + + namespace() { + return this; + } +} + function installHandlers(useTerminateTimeout) { const handlers = { terminationRequested: false, @@ -569,4 +606,5 @@ module.exports = { runLagMonitoring, genericProduceToTopic, getAutoCommit, + PerformanceLogger, }; \ No newline at end of file diff --git a/examples/performance/performance-primitives-kafkajs.js b/examples/performance/performance-primitives-kafkajs.js index 951e5f6d..b777870d 100644 --- a/examples/performance/performance-primitives-kafkajs.js +++ b/examples/performance/performance-primitives-kafkajs.js @@ -6,6 +6,7 @@ const { runProducer: runProducerCommon, genericProduceToTopic, getAutoCommit, + PerformanceLogger, } = require('./performance-primitives-common'); const { @@ -21,6 +22,8 @@ module.exports = { }; const IS_HIGHER_LATENCY_CLUSTER = process.env.IS_HIGHER_LATENCY_CLUSTER === 'true'; +const DEBUG = process.env.DEBUG; +const ENABLE_LOGGING = DEBUG !== undefined; function baseConfiguration(parameters) { let ret = { @@ -41,11 +44,18 @@ function baseConfiguration(parameters) { } }; } + if (parameters.logToFile) { + ret.logger = new PerformanceLogger(parameters.logToFile); + } return ret; } async function runCreateTopics(parameters, topic, topic2, numPartitions) { - const kafka = new Kafka(baseConfiguration(parameters)); + const adminParameters = { + ...parameters, + disableLogging: true, + }; + const kafka = new Kafka(baseConfiguration(adminParameters)); const admin = kafka.admin(); await admin.connect(); @@ -104,6 +114,9 @@ function newCompatibleProducer(parameters, compression) { } async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS) { + if (ENABLE_LOGGING && !parameters.logToFile) { + parameters.logToFile = './kafkajs-producer.log'; + } return runProducerCommon(newCompatibleProducer(parameters, compression), topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS); } @@ -167,18 +180,30 @@ function newCompatibleConsumer(parameters, eachBatch) { } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression, useCKJSProducerEverywhere) { +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression, useCKJSProducerEverywhere, + messageSize, limitRPS) { let actionOnMessages = null; let producer; if (produceToTopic) { const newCompatibleProducerFunction = useCKJSProducerEverywhere ? newCompatibleProducerCKJS : newCompatibleProducer; - producer = newCompatibleProducerFunction(parameters, produceCompression); + const producerParameters = { + ...parameters, + }; + if (ENABLE_LOGGING) + producerParameters.logToFile = './kafkajs-consumer-producer.log'; + producer = newCompatibleProducerFunction(producerParameters, produceCompression); await producer.connect(); actionOnMessages = (messages) => genericProduceToTopic(producer, produceToTopic, messages); } - const ret = await runConsumerCommon(newCompatibleConsumer(parameters, eachBatch), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages); + const consumerParameters = { + ...parameters, + }; + if (ENABLE_LOGGING) + consumerParameters.logToFile = eachBatch ? './kafkajs-consumer-batch.log' : + './kafkajs-consumer-message.log'; + const ret = await runConsumerCommon(newCompatibleConsumer(consumerParameters, eachBatch), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages); if (producer) { await producer.disconnect(); } diff --git a/examples/performance/performance-primitives.js b/examples/performance/performance-primitives.js index a7349c6a..e1d66dd5 100644 --- a/examples/performance/performance-primitives.js +++ b/examples/performance/performance-primitives.js @@ -7,6 +7,7 @@ const { runLagMonitoring: runLagMonitoringCommon, genericProduceToTopic, getAutoCommit, + PerformanceLogger, } = require('./performance-primitives-common'); module.exports = { @@ -22,6 +23,9 @@ module.exports = { const CONSUMER_MAX_BATCH_SIZE = process.env.CONSUMER_MAX_BATCH_SIZE ? +process.env.CONSUMER_MAX_BATCH_SIZE : null; const IS_HIGHER_LATENCY_CLUSTER = process.env.IS_HIGHER_LATENCY_CLUSTER === 'true'; +const DEBUG = process.env.DEBUG; +const STATISTICS_INTERVAL_MS = process.env.STATISTICS_INTERVAL_MS ? +process.env.STATISTICS_INTERVAL_MS : null; +const ENABLE_LOGGING = DEBUG !== undefined || STATISTICS_INTERVAL_MS !== null; function baseConfiguration(parameters) { let ret = { @@ -39,11 +43,29 @@ function baseConfiguration(parameters) { 'sasl.password': parameters.saslPassword, }; } + if (DEBUG && !parameters.disableLogging) { + ret['debug'] = DEBUG; + } + if (parameters.logToFile) { + ret.kafkaJS = { + 'logger': new PerformanceLogger(parameters.logToFile), + }; + } + if (STATISTICS_INTERVAL_MS !== null) { + ret['statistics.interval.ms'] = STATISTICS_INTERVAL_MS; + ret['stats_cb'] = function (event) { + this.logger().info(event.message); + }; + } return ret; } async function runCreateTopics(parameters, topic, topic2, numPartitions) { - const kafka = new Kafka(baseConfiguration(parameters)); + const adminParameters = { + ...parameters, + disableLogging: true, + }; + const kafka = new Kafka(baseConfiguration(adminParameters)); const admin = kafka.admin(); await admin.connect(); @@ -73,7 +95,11 @@ async function runCreateTopics(parameters, topic, topic2, numPartitions) { } function runLagMonitoring(parameters, topic) { - const kafka = new Kafka(baseConfiguration(parameters)); + const monitoringParameters = { + ...parameters, + disableLogging: true, + }; + const kafka = new Kafka(baseConfiguration(monitoringParameters)); const admin = kafka.admin(); return runLagMonitoringCommon(admin, topic); @@ -116,6 +142,9 @@ function newCompatibleProducer(parameters, compression) { } async function runProducer(parameters, topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS) { + if (ENABLE_LOGGING && !parameters.logToFile) { + parameters.logToFile = './confluent-producer.log'; + } return runProducerCommon(newCompatibleProducer(parameters, compression), topic, batchSize, warmupMessages, totalMessageCnt, msgSize, compression, randomness, limitRPS); } @@ -151,7 +180,8 @@ class CompatibleConsumer { } } -function newCompatibleConsumer(parameters, eachBatch) { +function newCompatibleConsumer(parameters, eachBatch, messageSize, limitRPS) { + const minFetchBytes = messageSize * limitRPS; const kafka = new Kafka(baseConfiguration(parameters)); const autoCommit = getAutoCommit(); const autoCommitOpts = autoCommit > 0 ? @@ -173,6 +203,7 @@ function newCompatibleConsumer(parameters, eachBatch) { 'group.id': groupId, 'auto.offset.reset': 'earliest', 'fetch.queue.backoff.ms': '100', + 'fetch.min.bytes': minFetchBytes.toString(), ...autoCommitOpts, ...jsOpts, ...higherLatencyClusterOpts, @@ -181,16 +212,30 @@ function newCompatibleConsumer(parameters, eachBatch) { } -async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression, useCKJSProducerEverywhere) { +async function runConsumer(parameters, topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, produceToTopic, produceCompression, useCKJSProducerEverywhere, + messageSize, limitRPS) { let actionOnMessages = null; let producer; if (produceToTopic) { - producer = newCompatibleProducer(parameters, produceCompression); + const producerParameters = { + ...parameters, + }; + if (ENABLE_LOGGING) + producerParameters.logToFile = './confluent-consumer-producer.log'; + producer = newCompatibleProducer(producerParameters, produceCompression); await producer.connect(); actionOnMessages = (messages) => genericProduceToTopic(producer, produceToTopic, messages); } - const ret = await runConsumerCommon(newCompatibleConsumer(parameters, eachBatch), topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages); + const consumerParameters = { + ...parameters, + }; + if (ENABLE_LOGGING) + consumerParameters.logToFile = eachBatch ? './confluent-consumer-batch.log' : + './confluent-consumer-message.log'; + const ret = await runConsumerCommon( + newCompatibleConsumer(consumerParameters, eachBatch, messageSize, limitRPS), + topic, warmupMessages, totalMessageCnt, eachBatch, partitionsConsumedConcurrently, stats, actionOnMessages); if (producer) { await producer.disconnect(); }