Skip to content

Add highwatermark offset to eachBatch callback #317

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,3 +1,12 @@
# confluent-kafka-javascript v1.4.0

v1.4.0 is a feature release. It is supported for all usage.

## Enhancements

1. Adds support for `highWatermark`, `offsetLag()`, and `offsetLagLow()` in `eachBatch` callback (#317).


# confluent-kafka-javascript v1.3.0

v1.3.0 is a feature release. It is supported for all usage.
Expand Down
25 changes: 22 additions & 3 deletions lib/kafkajs/_consumer.js
Original file line number Diff line number Diff line change
Expand Up @@ -863,6 +863,25 @@ class Consumer {
#createBatchPayload(messages) {
const topic = messages[0].topic;
const partition = messages[0].partition;
let watermarkOffsets = {};
let highWatermark = '-1001';
let offsetLag_ = -1;
let offsetLagLow_ = -1;

try {
watermarkOffsets = this.#internalClient.getWatermarkOffsets(topic, partition);
} catch (e) {
/* Only warn. The batch as a whole remains valid but for the fact that the highwatermark won't be there. */
this.#logger.warn(`Could not get watermark offsets for batch: ${e}`, this.#createConsumerBindingMessageMetadata());
}

if (Number.isInteger(watermarkOffsets.highOffset)) {
highWatermark = watermarkOffsets.highOffset.toString();
/* While calculating lag, we subtract 1 from the high offset
* for compatibility reasons with KafkaJS's API */
offsetLag_ = (watermarkOffsets.highOffset - 1) - messages[messages.length - 1].offset;
Copy link
Preview

Copilot AI May 16, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Calling getWatermarkOffsets on every batch may incur extra broker requests; consider caching per-partition watermarks or fetching them less frequently.

Copilot uses AI. Check for mistakes.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For reference to other reviewers: getWatermarkOffsets is locally cached and does not have a discernible impact on performance.

offsetLagLow_ = (watermarkOffsets.highOffset - 1) - messages[0].offset;
}

const messagesConverted = [];
for (let i = 0; i < messages.length; i++) {
Expand Down Expand Up @@ -892,13 +911,13 @@ class Consumer {
const batch = {
topic,
partition,
highWatermark: '-1001', /* We don't fetch it yet. We can call committed() to fetch it but that might incur network calls. */
highWatermark,
messages: messagesConverted,
isEmpty: () => false,
firstOffset: () => (messagesConverted[0].offset).toString(),
lastOffset: () => (messagesConverted[messagesConverted.length - 1].offset).toString(),
offsetLag: () => notImplemented(),
offsetLagLow: () => notImplemented(),
offsetLag: () => offsetLag_.toString(),
offsetLagLow: () => offsetLagLow_.toString(),
};

const returnPayload = {
Expand Down
62 changes: 62 additions & 0 deletions test/promisified/consumer/consumeMessages.spec.js
Original file line number Diff line number Diff line change
Expand Up @@ -858,4 +858,66 @@ describe.each(cases)('Consumer - partitionsConsumedConcurrently = %s -', (partit
await waitFor(() => (messagesConsumed === messages.length || batchesCountExceeds1), () => { }, 100);
expect(batchesCountExceeds1).toBe(false);
});

it('shows the correct high watermark and lag for partition', async () => {
await producer.connect();

let messages0 = Array(10).fill().map(() => {
const value = secureRandom();
return { value: `value-${value}`, partition: 0 };
});
let partition0ProducedMessages = messages0.length;

const messages1 = Array(5).fill().map(() => {
const value = secureRandom();
return { value: `value-${value}`, partition: 1 };
});

const messages2 = Array(2).fill().map(() => {
const value = secureRandom();
return { value: `value-${value}`, partition: 2 };
});

for (const messages of [messages0, messages1, messages2]) {
await producer.send({
topic: topicName,
messages: messages,
});
}

await consumer.connect();
await consumer.subscribe({ topic: topicName });

let messagesConsumed = 0;
consumer.run({
partitionsConsumedConcurrently,
eachBatch: async ({ batch }) => {
if (batch.partition === 0) {
expect(batch.highWatermark).toEqual(String(partition0ProducedMessages));
} else if (batch.partition === 1) {
expect(batch.highWatermark).toEqual(String(messages1.length));
} else if (batch.partition === 2) {
expect(batch.highWatermark).toEqual(String(messages2.length));
}
expect(batch.offsetLag()).toEqual(String(+batch.highWatermark - 1 - +batch.lastOffset()));
expect(batch.offsetLagLow()).toEqual(String(+batch.highWatermark - 1 - +batch.firstOffset()));
messagesConsumed += batch.messages.length;
}
});
await waitFor(() => (messagesConsumed === (partition0ProducedMessages + messages1.length + messages2.length)), () => { }, 100);

/* Add some more messages to partition 0 to make sure high watermark is updated. */
messages0 = Array(15).fill().map(() => {
const value = secureRandom();
return { value: `value-${value}`, partition: 0 };
});
partition0ProducedMessages += messages0.length;
await producer.send({
topic: topicName,
messages: messages0,
});

await waitFor(() => (messagesConsumed === (partition0ProducedMessages + messages1.length + messages2.length)), () => { }, 100);
});

});
2 changes: 2 additions & 0 deletions types/kafkajs.d.ts
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,8 @@ export type Batch = {
isEmpty(): boolean
firstOffset(): string | null
lastOffset(): string
offsetLag(): string
offsetLagLow(): string
}

export type KafkaMessage = MessageSetEntry | RecordBatchEntry
Expand Down