Skip to content

Commit

Permalink
Add tests for consumer.seek()
Browse files Browse the repository at this point in the history
  • Loading branch information
milindl committed Dec 11, 2023
1 parent d1c30f4 commit 85b1341
Show file tree
Hide file tree
Showing 4 changed files with 522 additions and 21 deletions.
3 changes: 3 additions & 0 deletions MIGRATION.md
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,9 @@
* `commitOffsets` does not (YET) support sending metadata for topic partitions being committed.
* `paused()` is not (YET) supported.
* Custom partition assignors are not supported.
* Changes to `seek`:
* The restriction to call seek only after `run` is removed.
* Rather than the `autoCommit` property of `run` deciding if the offset is committed, the librdkafka property `enable.auto.commit` of the consumer config is used.

### Admin Client

Expand Down
173 changes: 155 additions & 18 deletions lib/kafkajs/_consumer.js
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,19 @@ class Consumer {
*/
#state = ConsumerState.INIT;

/**
* Denotes if there are any new pending seeks we need to check.
* @type {boolean}
*/
#checkPendingSeeks = false;

/**
* Contains a mapping of topic+partition to an offset that the user wants to seek to.
* The keys are of the type "<topic>|<partition>".
* @type {Map<string, number>}
*/
#pendingSeeks = new Map();

/**
* @constructor
* @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig
Expand Down Expand Up @@ -99,13 +112,32 @@ class Consumer {
}

call
.finally(() => {
.finally(async () => {
// Emit the event
this.#internalClient.emit('rebalance', err, assignment);

try {
if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) {

if (this.#checkPendingSeeks)
assignment = this.#assignAsPerSeekedOffsets(assignment);

this.#internalClient.assign(assignment);

if (this.#checkPendingSeeks) {
const offsetsToCommit = assignment
.filter((topicPartition) => topicPartition.offset !== undefined)
.map((topicPartition) => ({
topic: topicPartition.topic,
partition: topicPartition.partition,
offset: String(topicPartition.offset),
}));

if (offsetsToCommit.length !== 0 && this.#rdKafkaConfig.globalConfig['enable.auto.commit']) {
await this.#commitOffsetsUntilNoStateErr(offsetsToCommit);
}
}

} else {
this.#internalClient.unassign();
}
Expand Down Expand Up @@ -191,9 +223,13 @@ class Consumer {
}

globalConfig['offset_commit_cb'] = true;
if (this.#kJSConfig.rebalanceListener) {
globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this);

if (!Object.hasOwn(this.#kJSConfig, 'rebalanceListener')) {
/* We might want to do certain things to maintain internal state in rebalance listener, so we need to set it to an empty object. */
this.#kJSConfig.rebalanceListener = {};
}
globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this);

return { globalConfig, topicConfig };
}

Expand Down Expand Up @@ -417,6 +453,14 @@ class Consumer {
* array/list until it can be processed, because librdkafka marks it as
* 'stored'... but anyway - we can implement something like this.
*/
if (this.#checkPendingSeeks) {
const invalidateMessage = await this.#seekInternal({ topic: m.topic, partition: m.partition });
if (invalidateMessage) {
/* Don't pass this message on to the user if this topic partition was seeked to. */
continue;
}
}

await config.eachMessage(
this.#createPayload(m)
)
Expand All @@ -429,6 +473,17 @@ class Consumer {
}
}

async #commitOffsetsUntilNoStateErr(offsetsToCommit) {
let err = { code: error.ErrorCodes.ERR_NO_ERROR };
do {
try {
await this.commitOffsets(offsetsToCommit);
} catch (e) {
err = e;
}
} while (err.code && err.code === error.ErrorCodes.ERR__STATE);
}

/**
* Commit offsets for the given topic partitions. If topic partitions are not specified, commits all offsets.
* @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]?} topicPartitions
Expand All @@ -443,9 +498,9 @@ class Consumer {
if (topicPartitions === null) {
this.#internalClient.commitSync();
} else {
const topicPartitions = topicPartitions.map(
const topicPartitionsRdKafka = topicPartitions.map(
topicPartitionOffsetToRdKafka);
this.#internalClient.commitSync(topicPartitions);
this.#internalClient.commitSync(topicPartitionsRdKafka);
}
} catch (e) {
if (!e.code || e.code !== error.ErrorCodes.ERR__NO_OFFSET) {
Expand All @@ -454,27 +509,109 @@ class Consumer {
}
}

/**
* Apply pending seeks to topic partitions we have just obtained as a result of a rebalance.
* @param {{topic: string, partition: number}[]} assignment The list of topic partitions to check for pending seeks.
* @returns {{topic: string, partition: number, offset: number}[]} the new assignment with the offsets seeked to, which can be passed to assign().
*/
#assignAsPerSeekedOffsets(assignment) {
const offsetsToCommit = [];

for (let i = 0; i < assignment.length; i++) {
const topicPartition = assignment[i];
const key = `${topicPartition.topic}|${topicPartition.partition}`;
if (!this.#pendingSeeks.has(key))
continue;

const offset = this.#pendingSeeks.get(key);
this.#pendingSeeks.delete(key);

assignment[i].offset = offset;

offsetsToCommit.push({
topic: topicPartition.topic,
partition: topicPartition.partition,
offset: String(offset),
});
}
return assignment;
}

/**
* This method processes any pending seeks on partitions that are assigned to this consumer.
* @param {{topic: string, partition: number}} messageTopicPartition If this method was triggered by a message, pass the topic partition of the message, else it's optional.
* @returns whether the message that triggered this should be invalidated (if any).
*/
async #seekInternal(messageTopicPartition) {
this.#checkPendingSeeks = false;

const assignment = this.assignment();
const offsetsToCommit = [];
let invalidateMessage = false;

for (const topicPartition of assignment) {
const key = `${topicPartition.topic}|${topicPartition.partition}`;
if (!this.#pendingSeeks.has(key))
continue;

const offset = this.#pendingSeeks.get(key);
this.#pendingSeeks.delete(key);

const topicPartitionOffset = {
topic: topicPartition.topic,
partition: topicPartition.partition,
offset
};

/* It's assumed that topicPartition is already assigned, and thus can be seeked to and committed to.
* Errors are logged to detect bugs in the internal code. */
this.#internalClient.seek(topicPartitionOffset, 0, err => err ? console.error(err) : null);
offsetsToCommit.push({
topic: topicPartition.topic,
partition: topicPartition.partition,
offset: String(offset),
});

/* If we're seeking the same topic partition as in the message that triggers it, invalidate
* the message. */
if (messageTopicPartition && topicPartition.topic === messageTopicPartition.topic && topicPartition.partition === messageTopicPartition.partition) {
invalidateMessage = true;
}
}

if (offsetsToCommit.length !== 0 && this.#rdKafkaConfig.globalConfig['enable.auto.commit']) {
await this.#commitOffsetsUntilNoStateErr(offsetsToCommit);
}

return invalidateMessage;
}

/**
* Seek to the given offset for the topic partition.
* This method is completely asynchronous, and does not wait for the seek to complete.
* In case any partitions that are seeked to, are not a part of the current assignment, they are stored internally.
* If at any time, the consumer is assigned the partition, the seek will be performed.
* Depending on the value of the librdkafka property 'enable.auto.commit', the consumer will commit the offset seeked to.
* @param {import("../../types/kafkajs").TopicPartitionOffset} topicPartitionOffset
* @returns {Promise<void>} a promise that resolves when the consumer has seeked.
*/
seek(topicPartitionOffset) {
if (this.#state !== ConsumerState.CONNECTED) {
throw new error.KafkaJSError('Seek can only be called while connected.', { code: error.ErrorCodes.ERR__STATE });
}

return new Promise((resolve, reject) => {
const rdKafkaTopicPartitionOffset =
topicPartitionOffsetToRdKafka(topicPartitionOffset);
this.#internalClient.seek(rdKafkaTopicPartitionOffset, 0, (err) => {
if (err) {
reject(createKafkaJsErrorFromLibRdKafkaError(err));
} else {
resolve();
}
});
});
const rdKafkaTopicPartitionOffset =
topicPartitionOffsetToRdKafka(topicPartitionOffset);

if (typeof rdKafkaTopicPartitionOffset.topic !== 'string') {
throw new error.KafkaJSError('Topic must be a string.', { code: error.ErrorCodes.ERR__INVALID_ARG });
}

if (isNaN(rdKafkaTopicPartitionOffset.offset) || (rdKafkaTopicPartitionOffset.offset < 0 && rdKafkaTopicPartitionOffset.offset !== -2 && rdKafkaTopicPartitionOffset.offset !== -3)) {
throw new error.KafkaJSError('Offset must be >= 0, or a special value.', { code: error.ErrorCodes.ERR__INVALID_ARG });
}

this.#checkPendingSeeks = true;
this.#pendingSeeks.set(`${rdKafkaTopicPartitionOffset.topic}|${rdKafkaTopicPartitionOffset.partition}`, rdKafkaTopicPartitionOffset.offset);
}

async describeGroup() {
Expand All @@ -483,7 +620,7 @@ class Consumer {

/**
* Find the assigned topic partitions for the consumer.
* @returns {import("../../types").TopicPartition[]} the current assignment.
* @returns {import("../../types/kafkajs").TopicPartition[]} the current assignment.
*/
assignment() {
if (this.#state !== ConsumerState.CONNECTED) {
Expand Down
Loading

0 comments on commit 85b1341

Please sign in to comment.