Skip to content
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

[admin] KIP-430: Add authorized operations to describe API #964

Merged
merged 19 commits into from
Oct 19, 2023
Merged
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
7 changes: 7 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,13 @@

This is a feature release.

* Add support for AdminAPI `DescribeCluster()` and `DescribeTopics()`
(#964, @jainruchir).
* [KIP-430](https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses):
Return authorized operations in Describe Responses.
(#964, @jainruchir).
* Add `Rack` to the `Node` type, so AdminAPI calls can expose racks for brokers
(currently, all Describe Responses) (#964, @jainruchir).
* MockCluster can now be shutdown and started again to test broker
availability problems (#998, @kkoehler).
* Fixes a bug in the mock schema registry client where the wrong ID was being
Expand Down
2 changes: 2 additions & 0 deletions examples/.gitignore
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
admin_alter_consumer_group_offsets/admin_alter_consumer_group_offsets
admin_create_acls/admin_create_acls
admin_create_topic/admin_create_topic
admin_describe_topics/admin_describe_topics
admin_describe_cluster/admin_describe_cluster
admin_delete_acls/admin_delete_acls
admin_delete_consumer_groups/admin_delete_consumer_groups
admin_delete_topics/admin_delete_topics
Expand Down
6 changes: 5 additions & 1 deletion examples/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,15 @@ Examples
[admin_delete_topics](admin_delete_topics) - Delete topics

[admin_describe_acls](admin_describe_acls) - Find Access Control Lists using a filter


[admin_describe_cluster](admin_describe_cluster) - Describe cluster

[admin_describe_config](admin_describe_config) - Describe broker, topic or group configs

[admin_describe_consumer_groups](admin_describe_consumer_groups) - Describe one or more consumer groups

[admin_describe_topics](admin_describe_topics) - Describe topics

[admin_list_consumer_group_offsets](admin_list_consumer_group_offsets) - List consumer group offsets

[admin_list_consumer_groups](admin_list_consumer_groups) - List consumer groups
Expand Down
75 changes: 75 additions & 0 deletions examples/admin_describe_cluster/admin_describe_cluster.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/**
* Copyright 2023 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

// Describe Cluster
package main

import (
"context"
"fmt"
"os"
"strconv"
"time"

"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)

func main() {
if len(os.Args) < 3 {
fmt.Fprintf(
os.Stderr,
"Usage: %s <bootstrap-servers> <includeAuthorizedOperations>\n",
os.Args[0])
os.Exit(1)
}

bootstrapServers := os.Args[1]
includeAuthorizedOperations, errOperations := strconv.ParseBool(os.Args[2])
if errOperations != nil {
fmt.Printf(
"Failed to parse value of includeAuthorizedOperations %s: %s\n",
os.Args[2], errOperations)
os.Exit(1)
}

// Create a new AdminClient.
a, err := kafka.NewAdminClient(&kafka.ConfigMap{
"bootstrap.servers": bootstrapServers,
})
if err != nil {
fmt.Printf("Failed to create Admin client: %s\n", err)
os.Exit(1)
}
defer a.Close()

// Call DescribeCluster.
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
clusterDesc, err := a.DescribeCluster(
ctx, kafka.SetAdminOptionIncludeAuthorizedOperations(
includeAuthorizedOperations))
if err != nil {
fmt.Printf("Failed to describe cluster: %s\n", err)
os.Exit(1)
}

// Print results
fmt.Printf("ClusterId: %s\nController: %s\nNodes: %s\n",
clusterDesc.ClusterID, clusterDesc.Controller, clusterDesc.Nodes)
if includeAuthorizedOperations {
fmt.Printf("Allowed operations: %s\n", clusterDesc.AuthorizedOperations)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,25 +21,36 @@ import (
"context"
"fmt"
"os"
"strconv"
"time"

"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)

func main() {
if len(os.Args) < 3 {
if len(os.Args) < 4 {
fmt.Fprintf(
os.Stderr,
"Usage: %s <bootstrap-servers> <group1> [<group2> ...]\n",
"Usage: %s <bootstrap-servers> <includeAuthorizedOperations>"+
" <group1> [<group2> ...]\n",
os.Args[0])
os.Exit(1)
}

bootstrapServers := os.Args[1]
groups := os.Args[2:]
includeAuthorizedOperations, errOperations := strconv.ParseBool(os.Args[2])
if errOperations != nil {
fmt.Printf(
"Failed to parse value of includeAuthorizedOperations %s: %s\n", os.Args[2], errOperations)
os.Exit(1)
}

groups := os.Args[3:]

// Create a new AdminClient.
a, err := kafka.NewAdminClient(&kafka.ConfigMap{"bootstrap.servers": bootstrapServers})
a, err := kafka.NewAdminClient(&kafka.ConfigMap{
"bootstrap.servers": bootstrapServers,
})
if err != nil {
fmt.Printf("Failed to create Admin client: %s\n", err)
os.Exit(1)
Expand All @@ -49,7 +60,8 @@ func main() {
// Call DescribeConsumerGroups.
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
describeGroupsResult, err := a.DescribeConsumerGroups(ctx, groups)
describeGroupsResult, err := a.DescribeConsumerGroups(ctx, groups,
kafka.SetAdminOptionIncludeAuthorizedOperations(includeAuthorizedOperations))
if err != nil {
fmt.Printf("Failed to describe groups: %s\n", err)
os.Exit(1)
Expand All @@ -65,8 +77,12 @@ func main() {
"PartitionAssignor: %s\n"+
"State: %s\n"+
"Coordinator: %+v\n"+
"Members: %+v\n\n",
"Members: %+v\n",
g.GroupID, g.Error, g.IsSimpleConsumerGroup, g.PartitionAssignor,
g.State, g.Coordinator, g.Members)
if includeAuthorizedOperations {
fmt.Printf("Allowed operations: %s\n", g.AuthorizedOperations)
}
fmt.Printf("\n")
}
}
95 changes: 95 additions & 0 deletions examples/admin_describe_topics/admin_describe_topics.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/**
* Copyright 2023 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

// Describe topics
package main

import (
"context"
"fmt"
"os"
"strconv"
"time"

"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)

func main() {
if len(os.Args) < 4 {
fmt.Fprintf(
os.Stderr,
"Usage: %s <bootstrap-servers> <includeAuthorizedOperations>"+
" <topic1> [<topic2> ...]\n",
os.Args[0])
os.Exit(1)
}

bootstrapServers := os.Args[1]
includeAuthorizedOperations, errOperations := strconv.ParseBool(os.Args[2])
if errOperations != nil {
fmt.Printf(
"Failed to parse value of includeAuthorizedOperations %s: %s\n",
os.Args[2], errOperations)
os.Exit(1)
}
topics := os.Args[3:]

// Create a new AdminClient.
a, err := kafka.NewAdminClient(&kafka.ConfigMap{
"bootstrap.servers": bootstrapServers,
})
if err != nil {
fmt.Printf("Failed to create Admin client: %s\n", err)
os.Exit(1)
}
defer a.Close()

// Call DescribeTopics.
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
describeTopicsResult, err := a.DescribeTopics(
ctx, kafka.NewTopicCollectionOfTopicNames(topics),
kafka.SetAdminOptionIncludeAuthorizedOperations(
includeAuthorizedOperations))
if err != nil {
fmt.Printf("Failed to describe topics: %s\n", err)
os.Exit(1)
}

// Print results
fmt.Printf("A total of %d topic(s) described:\n\n",
len(describeTopicsResult.TopicDescriptions))
for _, t := range describeTopicsResult.TopicDescriptions {
if t.Error.Code() != 0 {
fmt.Printf("Topic: %s has error: %s\n",
t.Name, t.Error)
continue
}
fmt.Printf("Topic: %s has succeeded\n", t.Name)
if includeAuthorizedOperations {
fmt.Printf("Allowed operations: %s\n", t.AuthorizedOperations)
}
for i := 0; i < len(t.Partitions); i++ {
fmt.Printf("\tPartition id: %d with leader: %s\n",
t.Partitions[i].Partition, t.Partitions[i].Leader)
fmt.Printf("\t\tThe in-sync replica count is: %d, they are: \n\t\t%s\n",
len(t.Partitions[i].Isr), t.Partitions[i].Isr)
fmt.Printf("\t\tThe replica count is: %d, they are: \n\t\t%s\n",
len(t.Partitions[i].Replicas), t.Partitions[i].Replicas)
}
fmt.Printf("\n")
}
}
Loading