Skip to content

List consumer group offsets #49

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

Merged
merged 118 commits into from
Nov 4, 2024
Merged

List consumer group offsets #49

merged 118 commits into from
Nov 4, 2024

Conversation

PratRanj07
Copy link
Contributor

Created ListConsumerGroupOffsets API with the name of fetchOffsets and wrote examples and test cases for it.

@PratRanj07 PratRanj07 requested a review from a team as a code owner May 21, 2024 10:01
@PratRanj07 PratRanj07 requested a review from milindl May 21, 2024 10:01
Copy link

cla-assistant bot commented May 21, 2024

CLA assistant check
All committers have signed the CLA.

Copy link
Contributor

@milindl milindl left a comment

Choose a reason for hiding this comment

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

Good work @PratRanj07 to implement this.

I have done a first round of review.
Things still pending to be reviewed by me:

  1. example added
  2. run leak checks with valgrind

and of course any changes you make due to the comments.

@@ -366,6 +366,38 @@ class Admin {
});
});
}

/**
* Fetch Offsets
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Just 1 space before Fetch Offsets


/**
* Fetch Offsets
* @param {string} options.groupId - The group id.
Copy link
Contributor

Choose a reason for hiding this comment

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

documentation:

Suggested change
* @param {string} options.groupId - The group id.
* @param {string} options.groupId - The group ID to fetch offsets for.

if (this.#state !== AdminState.CONNECTED) {
throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE });
}
//console.log(options);
Copy link
Contributor

Choose a reason for hiding this comment

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

remove

}
//console.log(options);
if(options.resolveOffsets){
throw new Error("resolveOffsets is not yet implemented.");
Copy link
Contributor

Choose a reason for hiding this comment

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

Throw the correct error type. Check the file to see how it's done for waitForLeaders.

Change the check within the if block also. Currently the check won't work if someone has set resolveOffsets as false. We want to indicate non-support of the key entirely.

* @param {boolean} options.resolveOffsets - not yet implemented
* @param {number?} options.timeout - The request timeout in milliseconds.
* May be unset (default: 5000)
* @param {boolean?} options.requireStableOffsets
Copy link
Contributor

Choose a reason for hiding this comment

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

documentation: Requires documentation and default value

src/common.h Outdated
@@ -38,6 +38,8 @@ template<> std::vector<std::string> GetParameter<std::vector<std::string> >(
template<> v8::Local<v8::Array> GetParameter<v8::Local<v8::Array> >(
v8::Local<v8::Object>, std::string, v8::Local<v8::Array>);
// template int GetParameter<int>(v8::Local<v8::Object, std::string, int);
rd_kafka_topic_partition_list_t *v8ArrayToTopicPartitionList(
Copy link
Contributor

Choose a reason for hiding this comment

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

This should probably be within namespace TopicPartition. With a different name. Maybe FromGroupedTopicPartitionV8Array. Not really sure about the name for now.

).rejects.toHaveProperty("code", ErrorCodes.ERR__TIMED_OUT);
});

test("should return correct offset after consuming messages", async () => {
Copy link
Contributor

Choose a reason for hiding this comment

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

  • add a test with topics unset or null too
  • add a test with multiple topics each with more than 1 partition so the result format can be verified

@@ -0,0 +1,129 @@
jest.setTimeout(30000);
Copy link
Contributor

Choose a reason for hiding this comment

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

rename this file to be same as the function name. Let's call it 'list consumer groups' only when we actually need to call into librdkafka. Otherwise it gets confusing to mix the names.

@@ -259,6 +259,10 @@ export interface PartitionOffset {
offset: string
}

export type FetchOffsetsPartition = PartitionOffset & { metadata: string | null }

export type TopicInput = string | { topic: string; partitions: number[] }
Copy link
Contributor

Choose a reason for hiding this comment

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

To be consistent with how we have done things before, define these types in rdkafka.d.ts, and import them here. Re-export from this file. (See the top of this file for examples, one such type is GroupOverview)

@@ -11,7 +11,7 @@ import {

export * from './config';
export * from './errors';
import { Kafka } from './kafkajs';
import { FetchOffsetsPartition, Kafka, TopicInput } from './kafkajs';
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove this line entirely after following the comment from kafkajs.d.ts (Kafka was unused anyway)

@PratRanj07 PratRanj07 requested a review from milindl June 3, 2024 13:43
@PratRanj07 PratRanj07 requested a review from a team as a code owner June 4, 2024 06:36
@EricMCornelius
Copy link

Any ETA on this? You guys are promoting this client as a KafkaJS replacement while lacking some pretty basic functionality on the admin side here...

@EricMCornelius
Copy link

https://github.com/confluentinc/confluent-kafka-javascript/blob/master/MIGRATION.md#admin-client

At the very least, maybe keep major omissions up to date in any migration guides?

@@ -438,6 +442,12 @@ export interface IAdminClient {
deleteGroups(groupIds: string[],
options?: { timeout?: number },
cb?: (err: LibrdKafkaError, result: DeleteGroupsResult[]) => any): void;
fetchOffsets(options: { groupId: string,
Copy link
Contributor

Choose a reason for hiding this comment

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

for the node-rdkafka API, it's a new method, so we should have the same signature and name as listConsumerGroupOffsets, allowing multiple groups. In future, we'll deprecate fetchOffsets and have a new method named listConsumerGroupOffsets in KafkaJS API too.

Copy link
Contributor

@milindl milindl left a comment

Choose a reason for hiding this comment

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

Thanks for the changes!

Please also add a CHANGELOG.md entry, and merge the master branch to it. Once you do that we will change the base branch of this PR to master.

lib/admin.js Outdated
* @param {number?} options.timeout - The request timeout in milliseconds.
* May be unset (default: 5000)
* @param {boolean?} options.requireStableOffsets - Whether broker should return stable offsets
* (transaction-committed). (default: false)
Copy link
Contributor

Choose a reason for hiding this comment

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

indentation for the * is off

lib/admin.js Outdated
options.requireStableOffsets = false;
}

let originalTopics = null;
Copy link
Contributor

Choose a reason for hiding this comment

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

About the comment:
I see that you have added the comment which explains what you are doing, but a better comment should explain why you are doing it rather than the 'what' (which a developer can figure out through reading the code also). However, the 'why' is something that the other developer can't figure out on their own.

So for instance, you should say that, 'If the input is a list of topic string, the user expects us to fetch offsets for all all partitions of all the input topics. In librdkafka, we can only fetch offsets by topic partitions, or else, we can fetch all of them. This, we must fetch offsets for all topic partitions (by settings topics to null) and filter by the topic strings later.'

^ something like this. You can add this to your comment to annotate both the 'what' and the 'why'.

* @param {number?} options.timeout - The request timeout in milliseconds.
* May be unset (default: 5000)
* @param {boolean?} options.requireStableOffsets - Whether broker should return stable offsets
* (transaction-committed). (default: false)
Copy link
Contributor

Choose a reason for hiding this comment

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

again the indentation is off for this *

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: "(transaction-committed)" should be aligned to "Whether"

lib/admin.js Outdated
* Fetch Offsets
*
* @param {string} options.groupId - The group ID to fetch offsets for.
* @param {import('../types/rdkafka').TopicInput} options.topics - The topics to fetch offsets for.
Copy link
Contributor

Choose a reason for hiding this comment

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

Use instead

import("../../types/kafkajs").TopicInput

* Fetch Offsets
*
* @param {string} options.groupId - The group ID to fetch offsets for.
* @param {import('../../types/rdkafka').TopicInput} options.topics - The topics to fetch offsets for.
Copy link
Contributor

Choose a reason for hiding this comment

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

Use instead

import("../../types/kafkajs").TopicInput

src/admin.cc Outdated
const rd_kafka_group_result_t **results =
rd_kafka_ListConsumerGroupOffsets_result_groups(result, &result_cnt);

// Change the type of the 'error' pointer to 'const rd_kafka_error_t *'
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's move this processing outside this function and let the caller process this. This sort of works for now, but when/if we add the capacity for multiple groups, it won't.

Let's handle only top level errors here, and move this group-specific errors out.

Copy link
Contributor

Choose a reason for hiding this comment

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

So with the change that Emanuele suggested, you will need to change the FromFetchOffsetsResult function as well, since the return type signature will be different for listConsumerGroupOffsets.

However, the kafkajs-compatible signature remains the same, so you'll need to convert it in fetchOffsets in _admin.js

offset: (parseInt(message.offset, 10) + 1).toString(),
},
]);
await consumer.stop();
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove as this has no effect, here and elsewhere

await consumer.stop();
}
} catch (error) {
if (error.message.includes("Offset out of range")) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure when you'll get this error, because it's handled automatically. Maybe remove the try-catch block entirely?

Here, and elsewhere.

src/common.h Outdated

v8::Local<v8::Array> FromFetchOffsetsResult(
const rd_kafka_ListConsumerGroupOffsets_result_t *result);
}// namespace Admin
Copy link
Contributor

Choose a reason for hiding this comment

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

nit

Suggested change
}// namespace Admin
} // namespace Admin

src/workers.cc Outdated
* @brief Fetch Offsets in an asynchronous worker
*
* This callback will list all the consumer group offsets for the specified
* topic offsets.
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
* topic offsets.
* groups' topic partitions.

@confluent-cla-assistant
Copy link

🎉 All Contributor License Agreements have been signed. Ready to merge.
Please push an empty commit if you would like to re-run the checks to verify CLA status for all contributors.

@PratRanj07 PratRanj07 changed the base branch from dev_early_access_development_branch to master October 25, 2024 08:46
Copy link
Contributor

@milindl milindl left a comment

Choose a reason for hiding this comment

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

Pretty good work to address the review comments. I added a final set of comments, please take a look

@@ -86,6 +86,10 @@ export interface TopicPartitionOffsetAndMetadata extends TopicPartitionOffset {

export type TopicPartitionTime = TopicPartitionOffset;

export type FetchOffsetsPartition = PartitionOffset & { metadata: string | null, leaderEpoch: number | null, error?: LibrdKafkaError };
Copy link
Contributor

Choose a reason for hiding this comment

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

This isn't used in this file. It's just exported. Define this instead in kafkajs.d.ts.

@@ -11,7 +11,7 @@ import {

export * from './config';
export * from './errors';
import { Kafka } from './kafkajs';
import { PartitionOffset } from './kafkajs';
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove this line entirely after following the comment related to FetchOffsetsPartition.

@@ -86,6 +86,10 @@ export interface TopicPartitionOffsetAndMetadata extends TopicPartitionOffset {

export type TopicPartitionTime = TopicPartitionOffset;

export type FetchOffsetsPartition = PartitionOffset & { metadata: string | null, leaderEpoch: number | null, error?: LibrdKafkaError };

export type TopicInput = string[] | { topic: string; partitions: number[] }[]
Copy link
Contributor

Choose a reason for hiding this comment

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

Same as FetchOffsetsPartition, it is not used in this file so define in the other file only.

*/
const topicPartitionMap = new Map();

offsets.forEach(groupResult => {
Copy link
Contributor

Choose a reason for hiding this comment

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

You should make sure here that groupResult.length is exactly 1 and just use that element. The forEach is redundant

}));

if (originalTopics !== null) {
convertedOffsets = convertedOffsets.filter(convertedOffsets => originalTopics.includes(convertedOffsets.topic));
Copy link
Contributor

Choose a reason for hiding this comment

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

nit

Suggested change
convertedOffsets = convertedOffsets.filter(convertedOffsets => originalTopics.includes(convertedOffsets.topic));
convertedOffsets = convertedOffsets.filter(convertedOffset => originalTopics.includes(convertedOffset.topic));

@@ -0,0 +1,68 @@
// require('kafkajs') is replaced with require('@confluentinc/kafka-javascript').KafkaJS.
Copy link
Contributor

Choose a reason for hiding this comment

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

This file still needs to be moved to examples/kafkajs/admin/fetch-offsets.js, and use parseArgs appropriately for argument parsing. You can follow this file for an idea of how to do it: examples/kafkajs/admin/describe-groups.js

Copy link
Contributor

Choose a reason for hiding this comment

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

Use arguments for bootstrap server, requireStableOffsets and timeout. Use positionals for the group name and the topic/partitions as you are currently doing, it will be something like

  const args = parseArgs({
    allowPositionals: true,
    options: {
      'bootstrap-servers': {
        type: 'string',
        short: 'b',
        default: 'localhost:9092',
      },
      'timeout': {
        type: 'string',
        short: 'm',
        default: 5000,
      },
...
    },
  });

and then args.values will contain option values, and args.positionals will contain the rest of the arguments which you can parse with your function that you've already written.

src/admin.cc Outdated
Comment on lines 1096 to 1097
v8::Local<v8::Object> listGroupOffsetObj =
listGroupOffsets->Get(context, i).ToLocalChecked().As<v8::Object>();
Copy link
Contributor

Choose a reason for hiding this comment

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

Here and at other places, use Nan::Get instead. You don't need to make the context either. Handle the error case properly too, in case the user has not provided a proper object.

    v8::Local<v8::Value> listGroupOffsetValue = Nan::Get(listGroupOffsets, i).ToLocalChecked();
    if (!listGroupOffsetValue->IsObject()) {
      return Nan::ThrowError("Each entry must be an object");
    }
    v8::Local<v8::Object> listGroupOffsetObj = listGroupOffsetValue.As<v8::Object>();

src/common.cc Outdated
v8::Isolate* isolate = v8::Isolate::GetCurrent();
v8::Local<v8::Context> context = isolate->GetCurrentContext();

v8::Local<v8::Value> topicVal =
Copy link
Contributor

Choose a reason for hiding this comment

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

Just use GetParameter<std::string> to get the string of the topic name directly, no need for isolate/context. The "item" is an object and thus can be used with GetParameter<std::string>.

Same goes for partitionVal and offset also.

return Nan::ThrowError("'listGroupOffsets' cannot be empty");
}

rd_kafka_ListConsumerGroupOffsets_t **requests =
Copy link
Contributor

Choose a reason for hiding this comment

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

Add a comment above this telling the reader that the ownership for this is taken by Workers::AdminClientListConsumerGroupOffsets and freeing it is also handled by that class.

src/common.cc Outdated

// Set partition-level error (if any)
if (partition->err != RD_KAFKA_RESP_ERR_NO_ERROR) {
RdKafka::ErrorCode code = static_cast<RdKafka::ErrorCode>(partition->err);
Copy link
Contributor

Choose a reason for hiding this comment

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

Add // NOLINT or split into two lines.

Copy link
Contributor

@milindl milindl left a comment

Choose a reason for hiding this comment

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

Great work. I'm approving the PR. There are some minor changes suggested. You can make them and merge this in.

CHANGELOG.md Outdated
## Features

1. Add support for an Admin API to list a consumer group's offsets (#49).

# confluent-kafka-javascript v0.3.1
Copy link
Contributor

Choose a reason for hiding this comment

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

0.3.1 isn't released yet so remove this section and add "1. Fixes an issue where headers were not passed correctly to the eachBatch callback (#130)." within this list only. Call it "Enhancements" instead of features.

fetchOffsets(options: {
groupId: string,
topics?: TopicInput,
resolveOffsets?: boolean,
Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry I missed this earlier, remove resolveOffsets from the types. For users of typescript this makes their life better as they get a compile-time error instead of a run time error.

Copy link
Contributor

Choose a reason for hiding this comment

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

nit:
rename this file to fetch-offsets.js

@PratRanj07 PratRanj07 merged commit e1bd8e8 into master Nov 4, 2024
2 checks passed
@PratRanj07 PratRanj07 deleted the listConsumerGroupOffsets branch November 4, 2024 06:52
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy