From 7f21d715d7f18cda2c5b9ea3aa90fd56e0fdc07f Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 21 Mar 2023 14:43:40 +0530 Subject: [PATCH 01/12] desccribe topics/cluster with integration tests failing --- .../admin_describe_cluster.go | 86 + .../admin_describe_consumer_groups.go | 34 +- .../admin_describe_topics.go | 115 + kafka/adminapi.go | 313 ++ kafka/adminapi_test.go | 418 ++- kafka/adminoptions.go | 143 + kafka/integration_test.go | 3258 +++++++++-------- kafka/librdkafka_vendor/rdkafka.h | 478 ++- 8 files changed, 3143 insertions(+), 1702 deletions(-) create mode 100644 examples/admin_describe_cluster/admin_describe_cluster.go create mode 100644 examples/admin_describe_topics/admin_describe_topics.go diff --git a/examples/admin_describe_cluster/admin_describe_cluster.go b/examples/admin_describe_cluster/admin_describe_cluster.go new file mode 100644 index 000000000..6b6a66be7 --- /dev/null +++ b/examples/admin_describe_cluster/admin_describe_cluster.go @@ -0,0 +1,86 @@ +/** + * Copyright 2022 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) < 3 { + fmt.Fprintf( + os.Stderr, + "Usage: %s [ ...]\n", + "Usage: %s "+ + " [ ...]\n", os.Args[0]) os.Exit(1) } bootstrapServers := os.Args[1] - groups := os.Args[2:] + include_authorized_operations, err_operations := strconv.ParseBool(os.Args[2]) + if err_operations != nil { + fmt.Printf( + "Failed to parse value of include_authorized_operations %s: %s\n", os.Args[2], err_operations) + 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, + "sasl.mechanisms": "SCRAM-SHA-256", + "security.protocol": "SASL_PLAINTEXT", + "sasl.username": "broker", + "sasl.password": "broker", + }) if err != nil { fmt.Printf("Failed to create Admin client: %s\n", err) os.Exit(1) @@ -49,7 +64,7 @@ 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(include_authorized_operations)) if err != nil { fmt.Printf("Failed to describe groups: %s\n", err) os.Exit(1) @@ -65,8 +80,15 @@ 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 include_authorized_operations == true { + fmt.Printf("Allowed acl operations:\n") + for i := 0; i < len(g.AuthorizedOperations); i++ { + fmt.Printf("\t%s\n", g.AuthorizedOperations[i]) + } + } + fmt.Printf("\n") } } diff --git a/examples/admin_describe_topics/admin_describe_topics.go b/examples/admin_describe_topics/admin_describe_topics.go new file mode 100644 index 000000000..6c2274706 --- /dev/null +++ b/examples/admin_describe_topics/admin_describe_topics.go @@ -0,0 +1,115 @@ +/** + * Copyright 2022 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 [ ...]\n", + os.Args[0]) + os.Exit(1) + } + + bootstrapServers := os.Args[1] + include_topic_authorized_operations, err_operations := strconv.ParseBool(os.Args[2]) + if err_operations != nil { + fmt.Printf( + "Failed to parse value of include_topic_authorized_operations %s: %s\n", os.Args[2], err_operations) + os.Exit(1) + } + topics := os.Args[3:] + + // Create a new AdminClient. + a, err := kafka.NewAdminClient(&kafka.ConfigMap{ + "bootstrap.servers": bootstrapServers, + "sasl.mechanisms": "SCRAM-SHA-256", + "security.protocol": "SASL_PLAINTEXT", + "sasl.username": "broker", + "sasl.password": "broker", + }) + 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, topics, kafka.SetAdminOptionIncludeTopicAuthorizedOperations( + include_topic_authorized_operations)) + 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.Topic, t.Error) + continue + } + fmt.Printf("Topic: %s has succeeded\n", + t.Topic) + if include_topic_authorized_operations == true { + fmt.Printf("Allowed acl operations:\n") + for i := 0; i < len(t.TopicAuthorizedOperations); i++ { + fmt.Printf("\t%s\n", t.TopicAuthorizedOperations[i]) + } + } + for i := 0; i < len(t.Partitions); i++ { + if t.Partitions[i].Error.Code() != 0 { + fmt.Printf("Partition with id: %d"+ + "has error: %s\n\n", + t.Partitions[i].Id, t.Partitions[i].Error) + continue + } + fmt.Printf("Partition id: %d with leader: %d\n", + t.Partitions[i].Id, t.Partitions[i].Leader) + fmt.Printf("The in-sync replica count is: %d, they are: ", + len(t.Partitions[i].ISRs)) + for j := 0; j < len(t.Partitions[i].ISRs); j++ { + fmt.Printf("%d ", t.Partitions[i].ISRs[j]) + } + fmt.Printf("\n") + fmt.Printf("The replica count is: %d, they are: ", + len(t.Partitions[i].Replicas)) + for j := 0; j < len(t.Partitions[i].Replicas); j++ { + fmt.Printf("%d ", t.Partitions[i].Replicas[j]) + } + fmt.Printf("\n\n") + } + fmt.Printf("\n") + } +} diff --git a/kafka/adminapi.go b/kafka/adminapi.go index f7d7b8ac2..b3c6914d3 100644 --- a/kafka/adminapi.go +++ b/kafka/adminapi.go @@ -92,6 +92,13 @@ ConsumerGroupDescription_by_idx(const rd_kafka_ConsumerGroupDescription_t **resu return result_groups[idx]; } +static const rd_kafka_TopicDescription_t * +TopicDescription_by_idx(const rd_kafka_TopicDescription_t **result_topics, size_t cnt, size_t idx) { + if (idx >= cnt) + return NULL; + return result_topics[idx]; +} + static const rd_kafka_error_t * error_by_idx(const rd_kafka_error_t **errors, size_t cnt, size_t idx) { if (idx >= cnt) @@ -248,6 +255,8 @@ type ConsumerGroupDescription struct { Coordinator Node // Members list. Members []MemberDescription + // Acl operations allowed list + AuthorizedOperations []ACLOperation } // DescribeConsumerGroupsResult represents the result of a @@ -257,6 +266,53 @@ type DescribeConsumerGroupsResult struct { ConsumerGroupDescriptions []ConsumerGroupDescription } +// Partition information +type PartitionInfo struct { + // Partition Id. + Id int + // Error, if any, of partition. Check with `Error.Code() != ErrNoError. + Error Error + // Leader broker. + Leader int + // Replica broker. + Replicas []int + // In-Sync-Replica brokers. + ISRs []int +} + +// TopicDescription represents the result of DescribeTopics for +// a single topic. +type TopicDescription struct { + // Topic name. + Topic string + // Error, if any, of result. Check with `Error.Code() != ErrNoError`. + Error Error + // Partitions' information list. + Partitions []PartitionInfo + // Topic Acl operations allowed list + TopicAuthorizedOperations []ACLOperation +} + +// DescribeTopicsResult represents the result of a +// DescribeTopics call. +type DescribeTopicsResult struct { + // Slice of TopicDescription. + TopicDescriptions []TopicDescription +} + +// ClusterDescription represents the result of DescribeCluster for +// the cluster. +type ClusterDescription struct { + // Current cluster id in cluster. + ClusterID string + // Current controller id in cluster. + ControllerID int + // Slice of nodes in cluster. + Nodes []Node + // Cluster Acl operations allowed slice. + ClusterAuthorizedOperations []ACLOperation +} + // DeleteConsumerGroupsResult represents the result of a DeleteConsumerGroups // call. type DeleteConsumerGroupsResult struct { @@ -883,6 +939,15 @@ func (a *AdminClient) cToConsumerGroupDescriptions( } } + acl_operations_cnt := int( + C.rd_kafka_ConsumerGroupDescription_authorized_operations_count(cGroup)) + authorizedOperations := make([]ACLOperation, acl_operations_cnt) + for aclidx := 0; aclidx < acl_operations_cnt; aclidx++ { + aclop := int( + C.rd_kafka_ConsumerGroupDescription_authorized_operation(cGroup, C.size_t(aclidx))) + authorizedOperations[aclidx] = ACLOperation(aclop) + } + result[idx] = ConsumerGroupDescription{ GroupID: groupID, Error: err, @@ -891,8 +956,117 @@ func (a *AdminClient) cToConsumerGroupDescriptions( State: state, Coordinator: coordinator, Members: members, + AuthorizedOperations: authorizedOperations, + } + } + return result +} + +// cToTopicDescriptions converts a C rd_kafka_TopicDescription_t +// array to a Go TopicDescription slice. +func (a *AdminClient) cToTopicDescriptions( + cTopics **C.rd_kafka_TopicDescription_t, + cTopicCount C.size_t) (result []TopicDescription) { + result = make([]TopicDescription, cTopicCount) + for idx := 0; idx < int(cTopicCount); idx++ { + cTopic := C.TopicDescription_by_idx( + cTopics, cTopicCount, C.size_t(idx)) + + topic := C.GoString( + C.rd_kafka_TopicDescription_topic_name(cTopic)) + err := newErrorFromCError( + C.rd_kafka_TopicDescription_error(cTopic)) + partition_cnt := int( + C.rd_kafka_TopicDescription_topic_partition_cnt(cTopic)) + + partitions := make([]PartitionInfo, partition_cnt) + + for pidx := 0; pidx < partition_cnt; pidx++ { + id := int( + C.rd_kafka_TopicDescription_partiton_id(cTopic, C.int(pidx))) + leader := int( + C.rd_kafka_TopicDescription_partiton_leader(cTopic, C.int(pidx))) + partition_error := newErrorFromCError( + C.rd_kafka_TopicDescription_partition_error(cTopic, C.int(pidx))) + replicas_cnt := int( + C.rd_kafka_TopicDescription_partiton_replica_cnt(cTopic, C.int(pidx))) + isrs_cnt := int( + C.rd_kafka_TopicDescription_partiton_isr_cnt(cTopic, C.int(pidx))) + replicas := make([]int, replicas_cnt) + isrs := make([]int, isrs_cnt) + for ridx := 0; ridx < replicas_cnt; ridx++ { + replicas[ridx] = int( + C.rd_kafka_TopicDescription_partiton_replica_idx(cTopic, C.int(pidx), C.int(ridx))) + } + for isridx := 0; isridx < isrs_cnt; isridx++ { + isrs[isridx] = int( + C.rd_kafka_TopicDescription_partiton_isrs_idx(cTopic, C.int(pidx), C.int(isridx))) + } + partitions[pidx] = PartitionInfo{ + Id: id, + Error: partition_error, + Leader: leader, + Replicas: replicas, + ISRs: isrs, + } + } + + topic_authorized_operations_cnt := int( + C.rd_kafka_TopicDescription_topic_authorized_operations_cnt(cTopic)) + topicauthorizedOperations := make([]ACLOperation, topic_authorized_operations_cnt) + for aclidx := 0; aclidx < topic_authorized_operations_cnt; aclidx++ { + aclop := int( + C.rd_kafka_TopicDescription_authorized_operation_idx(cTopic, C.size_t(aclidx))) + topicauthorizedOperations[aclidx] = ACLOperation(aclop) + } + + result[idx] = TopicDescription{ + Topic: topic, + Error: err, + Partitions: partitions, + TopicAuthorizedOperations: topicauthorizedOperations, + } + } + return result +} + +// cToClusterDescriptions converts a C rd_kafka_TopicDescription_t +// to a Go ClusterDescription. +func (a *AdminClient) cToClusterDescription( + cClusterDesc *C.rd_kafka_ClusterDescription_t) (result ClusterDescription) { + clusterID := C.GoString( + C.rd_kafka_ClusterDescription_cluster_id(cClusterDesc)) + controllerID := int( + C.rd_kafka_ClusterDescription_controller_id(cClusterDesc)) + node_cnt := int( + C.rd_kafka_ClusterDescription_node_cnt(cClusterDesc)) + nodes := make([]Node, node_cnt) + for nidx := 0; nidx < node_cnt; nidx++ { + cNode := C.rd_kafka_ClusterDescription_node_idx(cClusterDesc, C.int(nidx)) + id := int(C.rd_kafka_Node_id(cNode)) + host := C.GoString(C.rd_kafka_Node_host(cNode)) + port := int(C.rd_kafka_Node_port(cNode)) + nodes[nidx] = Node{ + ID: id, + Host: host, + Port: port, } } + cluster_authorized_operations_cnt := int( + C.rd_kafka_ClusterDescription_cluster_acl_operations_cnt(cClusterDesc)) + clusterAuthorizedOperations := make([]ACLOperation, cluster_authorized_operations_cnt) + for aclidx := 0; aclidx < cluster_authorized_operations_cnt; aclidx++ { + aclop := int( + C.rd_kafka_ClusterDescription_authorized_operation_idx(cClusterDesc, C.size_t(aclidx))) + clusterAuthorizedOperations[aclidx] = ACLOperation(aclop) + } + + result = ClusterDescription{ + ClusterID: clusterID, + ControllerID: controllerID, + Nodes: nodes, + ClusterAuthorizedOperations: clusterAuthorizedOperations, + } return result } @@ -2080,6 +2254,145 @@ func (a *AdminClient) DescribeConsumerGroups( return describeResult, nil } +// DescribeTopics describes topics from cluster as specified by the +// topics list. +// +// Parameters: +// - `ctx` - context with the maximum amount of time to block, or nil for +// indefinite. +// - `topics` - Slice of groups to describe. This should not be nil/empty. +// - `options` - DescribeTopicsAdminOption options. +// +// Returns DescribeTopicsResult, which contains a slice of +// TopicDescriptions corresponding to the input topics, plus an error +// that is not `nil` for client level errors. Individual +// TopicDescriptions inside the slice should also be checked for +// errors. Individual TopicDescriptions also have a +// slice of allowed ACLOperations. +func (a *AdminClient) DescribeTopics( + ctx context.Context, topics []string, + options ...DescribeTopicsAdminOption) (result DescribeTopicsResult, err error) { + + describeResult := DescribeTopicsResult{} + err = a.verifyClient() + if err != nil { + return result, err + } + + // Convert topic names into char** required by the implementation. + cTopicNameList := make([]*C.char, len(topics)) + cTopicNameCount := C.size_t(len(topics)) + + for idx, topic := range topics { + cTopicNameList[idx] = C.CString(topic) + defer C.free(unsafe.Pointer(cTopicNameList[idx])) + } + + var cTopicNameListPtr **C.char + if cTopicNameCount > 0 { + cTopicNameListPtr = ((**C.char)(&cTopicNameList[0])) + } + + // Convert Go AdminOptions (if any) to C AdminOptions. + genericOptions := make([]AdminOption, len(options)) + for i := range options { + genericOptions[i] = options[i] + } + cOptions, err := adminOptionsSetup( + a.handle, C.RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, genericOptions) + if err != nil { + return describeResult, err + } + defer C.rd_kafka_AdminOptions_destroy(cOptions) + + // Create temporary queue for async operation. + cQueue := C.rd_kafka_queue_new(a.handle.rk) + defer C.rd_kafka_queue_destroy(cQueue) + + // Call rd_kafka_DescribeTopics (asynchronous). + C.rd_kafka_DescribeTopics( + a.handle.rk, + cTopicNameListPtr, + cTopicNameCount, + cOptions, + cQueue) + + // Wait for result, error or context timeout. + rkev, err := a.waitResult( + ctx, cQueue, C.RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT) + if err != nil { + return describeResult, err + } + defer C.rd_kafka_event_destroy(rkev) + + cRes := C.rd_kafka_event_DescribeTopics_result(rkev) + + // Convert result from C to Go. + var cTopicCount C.size_t + cTopics := C.rd_kafka_DescribeTopics_result_topics(cRes, &cTopicCount) + describeResult.TopicDescriptions = a.cToTopicDescriptions(cTopics, cTopicCount) + + return describeResult, nil +} + +// DescribeCluster describes the cluster +// +// Parameters: +// - `ctx` - context with the maximum amount of time to block, or nil for +// indefinite. +// - `options` - DescribeClusterAdminOption options. +// +// Returns ClusterDescription, which contains current cluster and controller +// id along with a slice of Nodes. It also has a slice of allowed ACLOperations. +func (a *AdminClient) DescribeCluster( + ctx context.Context, + options ...DescribeClusterAdminOption) (result ClusterDescription, err error) { + + clusterDesc := ClusterDescription{} + err = a.verifyClient() + if err != nil { + return result, err + } + + // Convert Go AdminOptions (if any) to C AdminOptions. + genericOptions := make([]AdminOption, len(options)) + for i := range options { + genericOptions[i] = options[i] + } + cOptions, err := adminOptionsSetup( + a.handle, C.RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, genericOptions) + if err != nil { + return clusterDesc, err + } + defer C.rd_kafka_AdminOptions_destroy(cOptions) + + // Create temporary queue for async operation. + cQueue := C.rd_kafka_queue_new(a.handle.rk) + defer C.rd_kafka_queue_destroy(cQueue) + + // Call rd_kafka_DescribeCluster (asynchronous). + C.rd_kafka_DescribeCluster( + a.handle.rk, + cOptions, + cQueue) + + // Wait for result, error or context timeout. + rkev, err := a.waitResult( + ctx, cQueue, C.RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT) + if err != nil { + return clusterDesc, err + } + defer C.rd_kafka_event_destroy(rkev) + + cRes := C.rd_kafka_event_DescribeCluster_result(rkev) + + // Convert result from C to Go. + cClusterDesc := C.rd_kafka_DescribeCluster_result_description(cRes) + clusterDesc = a.cToClusterDescription(cClusterDesc) + + return clusterDesc, nil +} + // DeleteConsumerGroups deletes a batch of consumer groups. // Parameters: // - `ctx` - context with the maximum amount of time to block, or nil for diff --git a/kafka/adminapi_test.go b/kafka/adminapi_test.go index 331872c64..6d1abf53e 100644 --- a/kafka/adminapi_test.go +++ b/kafka/adminapi_test.go @@ -485,6 +485,48 @@ func testAdminAPIsDescribeConsumerGroups( } } +func testAdminAPIsDescribeTopics( + what string, a *AdminClient, expDuration time.Duration, t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), expDuration) + defer cancel() + descres, err := a.DescribeTopics( + ctx, nil, SetAdminRequestTimeout(time.Second)) + if descres.TopicDescriptions != nil || err == nil { + t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", + descres, err) + } + if err.(Error).Code() != ErrInvalidArg { + t.Fatalf("Expected ErrInvalidArg with empty topics list, but got %s", err) + } + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + descres, err = a.DescribeTopics( + ctx, []string{"test"}, SetAdminRequestTimeout(time.Second)) + if descres.TopicDescriptions != nil || err == nil { + t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", + descres, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %s %v", err.(Error).Code(), ctx.Err()) + } +} + +func testAdminAPIsDescribeCluster( + what string, a *AdminClient, expDuration time.Duration, t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), expDuration) + defer cancel() + descres, err := a.DescribeCluster( + ctx, SetAdminRequestTimeout(time.Second)) + if descres.Nodes != nil || err == nil { + t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", + descres, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %s %v", err.(Error).Code(), ctx.Err()) + } +} + func testAdminAPIsDeleteConsumerGroups( what string, a *AdminClient, expDuration time.Duration, t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), expDuration) @@ -603,195 +645,197 @@ func testAdminAPIs(what string, a *AdminClient, t *testing.T) { t.Fatalf("Expected DeadlineExceeded, not %v, %v", ctx.Err(), err) } - // Incorrect input, fail with ErrInvalidArg - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - res, err = a.CreateTopics( - ctx, - []TopicSpecification{ - { - // Must not specify both ReplicationFactor and ReplicaAssignment - Topic: "mytopic", - NumPartitions: 2, - ReplicationFactor: 3, - ReplicaAssignment: [][]int32{ - []int32{1, 2, 3}, - []int32{3, 2, 1}, - }, - }, - }) - if res != nil || err == nil { - t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) - } - if ctx.Err() != nil { - t.Fatalf("Did not expect context to fail: %v", ctx.Err()) - } - if err.(Error).Code() != ErrInvalidArg { - t.Fatalf("Expected ErrInvalidArg, not %v", err) - } - - // Incorrect input, fail with ErrInvalidArg - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - res, err = a.CreateTopics( - ctx, - []TopicSpecification{ - { - // ReplicaAssignment must be same length as Numpartitions - Topic: "mytopic", - NumPartitions: 7, - ReplicaAssignment: [][]int32{ - []int32{1, 2, 3}, - []int32{3, 2, 1}, - }, - }, - }) - if res != nil || err == nil { - t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) - } - if ctx.Err() != nil { - t.Fatalf("Did not expect context to fail: %v", ctx.Err()) - } - if err.(Error).Code() != ErrInvalidArg { - t.Fatalf("Expected ErrInvalidArg, not %v", err) - } - - // Correct input, using options - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - res, err = a.CreateTopics( - ctx, - []TopicSpecification{ - { - Topic: "mytopic4", - NumPartitions: 9, - ReplicaAssignment: [][]int32{ - []int32{1}, - []int32{2}, - []int32{3}, - []int32{4}, - []int32{1}, - []int32{2}, - []int32{3}, - []int32{4}, - []int32{1}, - }, - Config: map[string]string{ - "some.topic.config": "unchecked", - "these.are.verified": "on the broker", - "and.this.is": "just", - "a": "unit test", - }, - }, - }, - SetAdminValidateOnly(false)) - if res != nil || err == nil { - t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) - } - if ctx.Err() != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - } - - // - // Remaining APIs - // Timeout code is identical for all APIs, no need to test - // them for each API. - // - - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - res, err = a.CreatePartitions( - ctx, - []PartitionsSpecification{ - { - Topic: "topic", - IncreaseTo: 19, - ReplicaAssignment: [][]int32{ - []int32{3234522}, - []int32{99999}, - }, - }, - { - Topic: "topic2", - IncreaseTo: 2, - ReplicaAssignment: [][]int32{ - []int32{99999}, - }, - }, - }) - if res != nil || err == nil { - t.Fatalf("Expected CreatePartitions to fail, but got result: %v, err: %v", res, err) - } - if ctx.Err() != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - } - - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - res, err = a.DeleteTopics( - ctx, - []string{"topic1", "topic2"}) - if res != nil || err == nil { - t.Fatalf("Expected DeleteTopics to fail, but got result: %v, err: %v", res, err) - } - if ctx.Err() != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v for error %v", ctx.Err(), err) - } - - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - cres, err := a.AlterConfigs( - ctx, - []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) - if cres != nil || err == nil { - t.Fatalf("Expected AlterConfigs to fail, but got result: %v, err: %v", cres, err) - } - if ctx.Err() != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - } - - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - cres, err = a.DescribeConfigs( - ctx, - []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) - if cres != nil || err == nil { - t.Fatalf("Expected DescribeConfigs to fail, but got result: %v, err: %v", cres, err) - } - if ctx.Err() != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - } - - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - clusterID, err := a.ClusterID(ctx) - if err == nil { - t.Fatalf("Expected ClusterID to fail, but got result: %v", clusterID) - } - if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - } - - ctx, cancel = context.WithTimeout(context.Background(), expDuration) - defer cancel() - controllerID, err := a.ControllerID(ctx) - if err == nil { - t.Fatalf("Expected ControllerID to fail, but got result: %v", controllerID) - } - if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { - t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - } - - testAdminAPIsCreateACLs(what, a, t) - testAdminAPIsDescribeACLs(what, a, t) - testAdminAPIsDeleteACLs(what, a, t) - - testAdminAPIsListConsumerGroups(what, a, expDuration, t) + // // Incorrect input, fail with ErrInvalidArg + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // res, err = a.CreateTopics( + // ctx, + // []TopicSpecification{ + // { + // // Must not specify both ReplicationFactor and ReplicaAssignment + // Topic: "mytopic", + // NumPartitions: 2, + // ReplicationFactor: 3, + // ReplicaAssignment: [][]int32{ + // []int32{1, 2, 3}, + // []int32{3, 2, 1}, + // }, + // }, + // }) + // if res != nil || err == nil { + // t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) + // } + // if ctx.Err() != nil { + // t.Fatalf("Did not expect context to fail: %v", ctx.Err()) + // } + // if err.(Error).Code() != ErrInvalidArg { + // t.Fatalf("Expected ErrInvalidArg, not %v", err) + // } + + // // Incorrect input, fail with ErrInvalidArg + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // res, err = a.CreateTopics( + // ctx, + // []TopicSpecification{ + // { + // // ReplicaAssignment must be same length as Numpartitions + // Topic: "mytopic", + // NumPartitions: 7, + // ReplicaAssignment: [][]int32{ + // []int32{1, 2, 3}, + // []int32{3, 2, 1}, + // }, + // }, + // }) + // if res != nil || err == nil { + // t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) + // } + // if ctx.Err() != nil { + // t.Fatalf("Did not expect context to fail: %v", ctx.Err()) + // } + // if err.(Error).Code() != ErrInvalidArg { + // t.Fatalf("Expected ErrInvalidArg, not %v", err) + // } + + // // Correct input, using options + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // res, err = a.CreateTopics( + // ctx, + // []TopicSpecification{ + // { + // Topic: "mytopic4", + // NumPartitions: 9, + // ReplicaAssignment: [][]int32{ + // []int32{1}, + // []int32{2}, + // []int32{3}, + // []int32{4}, + // []int32{1}, + // []int32{2}, + // []int32{3}, + // []int32{4}, + // []int32{1}, + // }, + // Config: map[string]string{ + // "some.topic.config": "unchecked", + // "these.are.verified": "on the broker", + // "and.this.is": "just", + // "a": "unit test", + // }, + // }, + // }, + // SetAdminValidateOnly(false)) + // if res != nil || err == nil { + // t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) + // } + // if ctx.Err() != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + // } + + // // + // // Remaining APIs + // // Timeout code is identical for all APIs, no need to test + // // them for each API. + // // + + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // res, err = a.CreatePartitions( + // ctx, + // []PartitionsSpecification{ + // { + // Topic: "topic", + // IncreaseTo: 19, + // ReplicaAssignment: [][]int32{ + // []int32{3234522}, + // []int32{99999}, + // }, + // }, + // { + // Topic: "topic2", + // IncreaseTo: 2, + // ReplicaAssignment: [][]int32{ + // []int32{99999}, + // }, + // }, + // }) + // if res != nil || err == nil { + // t.Fatalf("Expected CreatePartitions to fail, but got result: %v, err: %v", res, err) + // } + // if ctx.Err() != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + // } + + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // res, err = a.DeleteTopics( + // ctx, + // []string{"topic1", "topic2"}) + // if res != nil || err == nil { + // t.Fatalf("Expected DeleteTopics to fail, but got result: %v, err: %v", res, err) + // } + // if ctx.Err() != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v for error %v", ctx.Err(), err) + // } + + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // cres, err := a.AlterConfigs( + // ctx, + // []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) + // if cres != nil || err == nil { + // t.Fatalf("Expected AlterConfigs to fail, but got result: %v, err: %v", cres, err) + // } + // if ctx.Err() != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + // } + + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // cres, err = a.DescribeConfigs( + // ctx, + // []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) + // if cres != nil || err == nil { + // t.Fatalf("Expected DescribeConfigs to fail, but got result: %v, err: %v", cres, err) + // } + // if ctx.Err() != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + // } + + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // clusterID, err := a.ClusterID(ctx) + // if err == nil { + // t.Fatalf("Expected ClusterID to fail, but got result: %v", clusterID) + // } + // if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + // } + + // ctx, cancel = context.WithTimeout(context.Background(), expDuration) + // defer cancel() + // controllerID, err := a.ControllerID(ctx) + // if err == nil { + // t.Fatalf("Expected ControllerID to fail, but got result: %v", controllerID) + // } + // if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { + // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + // } + + // testAdminAPIsCreateACLs(what, a, t) + // testAdminAPIsDescribeACLs(what, a, t) + // testAdminAPIsDeleteACLs(what, a, t) + + // testAdminAPIsListConsumerGroups(what, a, expDuration, t) testAdminAPIsDescribeConsumerGroups(what, a, expDuration, t) - testAdminAPIsDeleteConsumerGroups(what, a, expDuration, t) + testAdminAPIsDescribeTopics(what, a, expDuration, t) + testAdminAPIsDescribeCluster(what, a, expDuration, t) + // testAdminAPIsDeleteConsumerGroups(what, a, expDuration, t) - testAdminAPIsListConsumerGroupOffsets(what, a, expDuration, t) - testAdminAPIsAlterConsumerGroupOffsets(what, a, expDuration, t) + // testAdminAPIsListConsumerGroupOffsets(what, a, expDuration, t) + // testAdminAPIsAlterConsumerGroupOffsets(what, a, expDuration, t) } // TestAdminAPIs dry-tests most Admin APIs, no broker is needed. diff --git a/kafka/adminoptions.go b/kafka/adminoptions.go index c0b9eb690..ffd063241 100644 --- a/kafka/adminoptions.go +++ b/kafka/adminoptions.go @@ -127,6 +127,10 @@ func (ao AdminOptionRequestTimeout) supportsListConsumerGroups() { } func (ao AdminOptionRequestTimeout) supportsDescribeConsumerGroups() { } +func (ao AdminOptionRequestTimeout) supportsDescribeTopics() { +} +func (ao AdminOptionRequestTimeout) supportsDescribeCluster() { +} func (ao AdminOptionRequestTimeout) supportsDeleteConsumerGroups() { } func (ao AdminOptionRequestTimeout) supportsListConsumerGroupOffsets() { @@ -299,6 +303,129 @@ func (ao AdminOptionMatchConsumerGroupStates) apply(cOptions *C.rd_kafka_AdminOp return nil } +// AdminOptionIncludeAuthorizedOperations decides if the broker should return include +// authorized operations. +// +// Default: false +// +// Valid for DescribeConsumerGroups. +type AdminOptionIncludeAuthorizedOperations struct { + isSet bool + val bool +} + +func (ao AdminOptionIncludeAuthorizedOperations) supportsDescribeConsumerGroups() { +} + +func (ao AdminOptionIncludeAuthorizedOperations) apply(cOptions *C.rd_kafka_AdminOptions_t) error { + if !ao.isSet { + return nil + } + + cError := C.rd_kafka_AdminOptions_set_include_authorized_operations( + cOptions, bool2cint(ao.val)) + if cError != nil { + C.rd_kafka_AdminOptions_destroy(cOptions) + return newErrorFromCErrorDestroy(cError) + } + + return nil +} + +// SetAdminOptionIncludeAuthorizedOperations decides if the broker should return include +// authorized operations. +// +// Default: false +// +// Valid for DescribeConsumerGroups. +func SetAdminOptionIncludeAuthorizedOperations(val bool) (ao AdminOptionIncludeAuthorizedOperations) { + ao.isSet = true + ao.val = val + return ao +} + +// AdminOptionIncludeTopicAuthorizedOperations decides if the broker should return include +// topic authorized operations. +// +// Default: false +// +// Valid for DescribeTopics. +type AdminOptionIncludeTopicAuthorizedOperations struct { + isSet bool + val bool +} + +func (ao AdminOptionIncludeTopicAuthorizedOperations) supportsDescribeTopics() { +} + +func (ao AdminOptionIncludeTopicAuthorizedOperations) apply(cOptions *C.rd_kafka_AdminOptions_t) error { + if !ao.isSet { + return nil + } + + cError := C.rd_kafka_AdminOptions_set_include_topic_authorized_operations( + cOptions, bool2cint(ao.val)) + if cError != nil { + C.rd_kafka_AdminOptions_destroy(cOptions) + return newErrorFromCErrorDestroy(cError) + } + + return nil +} + +// SetAdminOptionIncludeTopicAuthorizedOperations decides if the broker should return include +// topic authorized operations. +// +// Default: false +// +// Valid for DescribeTopics. +func SetAdminOptionIncludeTopicAuthorizedOperations(val bool) (ao AdminOptionIncludeTopicAuthorizedOperations) { + ao.isSet = true + ao.val = val + return ao +} + +// AdminOptionIncludeClusterAuthorizedOperations decides if the broker should return include +// cluster authorized operations. +// +// Default: false +// +// Valid for DescribeCluster. +type AdminOptionIncludeClusterAuthorizedOperations struct { + isSet bool + val bool +} + +func (ao AdminOptionIncludeClusterAuthorizedOperations) supportsDescribeCluster() { +} + +func (ao AdminOptionIncludeClusterAuthorizedOperations) apply(cOptions *C.rd_kafka_AdminOptions_t) error { + if !ao.isSet { + return nil + } + + cError := C.rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + cOptions, bool2cint(ao.val)) + if cError != nil { + C.rd_kafka_AdminOptions_destroy(cOptions) + return newErrorFromCErrorDestroy(cError) + } + + return nil +} + +// SetAdminOptionIncludeClusterAuthorizedOperations decides if the broker should return include +// cluster authorized operations. +// +// Default: false +// +// Valid for DescribeCluster. +func SetAdminOptionIncludeClusterAuthorizedOperations(val bool) (ao AdminOptionIncludeClusterAuthorizedOperations) { + ao.isSet = true + ao.val = val + return ao +} + // SetAdminMatchConsumerGroupStates decides groups in which state(s) should be // listed. // @@ -391,6 +518,22 @@ type DescribeConsumerGroupsAdminOption interface { apply(cOptions *C.rd_kafka_AdminOptions_t) error } +// DescribeTopicsAdminOption - see setter. +// +// See SetAdminRequestTimeout. +type DescribeTopicsAdminOption interface { + supportsDescribeTopics() + apply(cOptions *C.rd_kafka_AdminOptions_t) error +} + +// DescribeClusterAdminOption - see setter. +// +// See SetAdminRequestTimeout. +type DescribeClusterAdminOption interface { + supportsDescribeCluster() + apply(cOptions *C.rd_kafka_AdminOptions_t) error +} + // DeleteConsumerGroupsAdminOption - see setters. // // See SetAdminRequestTimeout. diff --git a/kafka/integration_test.go b/kafka/integration_test.go index 211713bfa..3442806d6 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -18,7 +18,6 @@ package kafka import ( "context" - "encoding/binary" "fmt" "math/rand" "path" @@ -603,243 +602,600 @@ func (its *IntegrationTestSuite) TearDownSuite() { } // TestConsumerSeekPartitions tests seeking of partitions using SeekPartitions(). -func (its *IntegrationTestSuite) TestConsumerSeekPartitions() { - t := its.T() - - numMessages := 10 // should be more than or equal to 2. - - // Produce `numMessages` messages to Topic. - conf := ConfigMap{"bootstrap.servers": testconf.Brokers} - conf.updateFromTestconf() - - producer, err := NewProducer(&conf) - if err != nil { - t.Fatalf("Failed to create producer: %s", err) - } - - for idx := 0; idx < numMessages; idx++ { - if err = producer.Produce(&Message{ - TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, - }, nil); err != nil { - t.Fatalf("Failed to produce message: %s", err) - } - } - - producer.Flush(10 * 1000) - producer.Close() - - // Assign partition, seek to `numMessages`/2, and check by reading the message. - conf = ConfigMap{ - "bootstrap.servers": testconf.Brokers, - "group.id": testconf.GroupID, - "auto.offset.reset": "end", - } - conf.updateFromTestconf() - - consumer, err := NewConsumer(&conf) - if err != nil { - t.Fatalf("Failed to create consumer: %s", err) - } - - tps := []TopicPartition{ - {Topic: &testconf.Topic, Partition: 0}, - } - err = consumer.Assign(tps) - if err != nil { - t.Fatalf("Failed to assign partition: %s", err) - } - - tps[0].Offset = Offset(numMessages / 2) - seekedPartitions, err := consumer.SeekPartitions(tps) - if err != nil { - t.Errorf("SeekPartitions failed: %s", err) - } - if len(seekedPartitions) != len(tps) { - t.Errorf( - "SeekPartitions should return result for %d partitions, %d returned", - len(tps), len(seekedPartitions)) - } - for _, seekedPartition := range seekedPartitions { - if seekedPartition.Error != nil { - t.Errorf("Seek error for partition %v", seekedPartition) - } - } - - msg, err := consumer.ReadMessage(10 * time.Second) - if err != nil { - t.Fatalf("ReadMessage failed: %s", err) - } - if msg.TopicPartition.Offset != Offset(numMessages/2) { - t.Errorf("Expected offset of read message is %d, got %d", - numMessages/2, msg.TopicPartition.Offset) - } -} - -// TestAdminClient_DeleteConsumerGroups verifies the working of the -// DeleteConsumerGroups API in the admin client. -// It does so by listing consumer groups before/after deletion. -func (its *IntegrationTestSuite) TestAdminClient_DeleteConsumerGroups() { +// func (its *IntegrationTestSuite) TestConsumerSeekPartitions() { +// t := its.T() + +// numMessages := 10 // should be more than or equal to 2. + +// // Produce `numMessages` messages to Topic. +// conf := ConfigMap{"bootstrap.servers": testconf.Brokers} +// conf.updateFromTestconf() + +// producer, err := NewProducer(&conf) +// if err != nil { +// t.Fatalf("Failed to create producer: %s", err) +// } + +// for idx := 0; idx < numMessages; idx++ { +// if err = producer.Produce(&Message{ +// TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, +// }, nil); err != nil { +// t.Fatalf("Failed to produce message: %s", err) +// } +// } + +// producer.Flush(10 * 1000) +// producer.Close() + +// // Assign partition, seek to `numMessages`/2, and check by reading the message. +// conf = ConfigMap{ +// "bootstrap.servers": testconf.Brokers, +// "group.id": testconf.GroupID, +// "auto.offset.reset": "end", +// } +// conf.updateFromTestconf() + +// consumer, err := NewConsumer(&conf) +// if err != nil { +// t.Fatalf("Failed to create consumer: %s", err) +// } + +// tps := []TopicPartition{ +// {Topic: &testconf.Topic, Partition: 0}, +// } +// err = consumer.Assign(tps) +// if err != nil { +// t.Fatalf("Failed to assign partition: %s", err) +// } + +// tps[0].Offset = Offset(numMessages / 2) +// seekedPartitions, err := consumer.SeekPartitions(tps) +// if err != nil { +// t.Errorf("SeekPartitions failed: %s", err) +// } +// if len(seekedPartitions) != len(tps) { +// t.Errorf( +// "SeekPartitions should return result for %d partitions, %d returned", +// len(tps), len(seekedPartitions)) +// } +// for _, seekedPartition := range seekedPartitions { +// if seekedPartition.Error != nil { +// t.Errorf("Seek error for partition %v", seekedPartition) +// } +// } + +// msg, err := consumer.ReadMessage(10 * time.Second) +// if err != nil { +// t.Fatalf("ReadMessage failed: %s", err) +// } +// if msg.TopicPartition.Offset != Offset(numMessages/2) { +// t.Errorf("Expected offset of read message is %d, got %d", +// numMessages/2, msg.TopicPartition.Offset) +// } +// } + +// // TestAdminClient_DeleteConsumerGroups verifies the working of the +// // DeleteConsumerGroups API in the admin client. +// // It does so by listing consumer groups before/after deletion. +// func (its *IntegrationTestSuite) TestAdminClient_DeleteConsumerGroups() { +// t := its.T() +// if testconf.Semaphore { +// t.Skipf("Skipping TestAdminClient_DeleteConsumerGroups since it is flaky[Does not run when tested with all the other integration tests]") +// return +// } +// rand.Seed(time.Now().Unix()) + +// // Generating new groupID to ensure a fresh group is created. +// groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) + +// ac := createAdminClient(t) +// defer ac.Close() + +// // Check that our group is not present initially. +// ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) +// if err != nil { +// t.Errorf("Error listing consumer groups %s\n", err) +// return +// } + +// if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { +// t.Errorf("Consumer group present before consumer created: %s\n", groupID) +// return +// } + +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() + +// // Create consumer +// config := &ConfigMap{ +// "bootstrap.servers": testconf.Brokers, +// "group.id": groupID, +// "auto.offset.reset": "earliest", +// "enable.auto.offset.store": false, +// } +// config.updateFromTestconf() +// consumer, err := NewConsumer(config) +// if err != nil { +// t.Errorf("Failed to create consumer: %s\n", err) +// return +// } +// consumerClosed := false +// defer func() { +// if !consumerClosed { +// consumer.Close() +// } +// }() + +// if err := consumer.Subscribe(testconf.Topic, nil); err != nil { +// t.Errorf("Failed to subscribe to %s: %s\n", testconf.Topic, err) +// return +// } + +// // This ReadMessage gives some time for the rebalance to happen. +// _, err = consumer.ReadMessage(5 * time.Second) +// if err != nil && err.(Error).Code() != ErrTimedOut { +// t.Errorf("Failed while reading message: %s\n", err) +// return +// } + +// // Check that the group exists. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) +// if err != nil { +// t.Errorf("Error listing consumer groups %s\n", err) +// return +// } + +// if findConsumerGroupListing(listGroupResult.Valid, groupID) == nil { +// t.Errorf("Consumer group %s should be present\n", groupID) +// return +// } + +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() + +// // Try deleting the group while consumer is active. It should fail. +// result, err := ac.DeleteConsumerGroups(ctx, []string{groupID}) +// if err != nil { +// t.Errorf("DeleteConsumerGroups() failed: %s", err) +// return +// } +// resultGroups := result.ConsumerGroupResults + +// if len(resultGroups) != 1 || resultGroups[0].Group != groupID { +// t.Errorf("Wrong group affected/no group affected") +// return +// } + +// if resultGroups[0].Error.code != ErrNonEmptyGroup { +// t.Errorf("Encountered the wrong error after calling DeleteConsumerGroups %s", resultGroups[0].Error) +// return +// } + +// // Close the consumer. +// if err = consumer.Close(); err != nil { +// t.Errorf("Could not close consumer %s", err) +// return +// } +// consumerClosed = true + +// // Delete the consumer group now. +// result, err = ac.DeleteConsumerGroups(ctx, []string{groupID}) +// if err != nil { +// t.Errorf("DeleteConsumerGroups() failed: %s", err) +// return +// } +// resultGroups = result.ConsumerGroupResults + +// if len(resultGroups) != 1 || resultGroups[0].Group != groupID { +// t.Errorf("Wrong group affected/no group affected") +// return +// } + +// if resultGroups[0].Error.code != ErrNoError { +// t.Errorf("Encountered an error after calling DeleteConsumerGroups %s", resultGroups[0].Error) +// return +// } + +// // Check for the absence of the consumer group after deletion. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) +// if err != nil { +// t.Errorf("Error listing consumer groups %s\n", err) +// return +// } + +// if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { +// t.Errorf("Consumer group %s should not be present\n", groupID) +// return +// } +// } + +// // TestAdminClient_ListAndDescribeConsumerGroups validates the working of the +// // list consumer groups and describe consumer group APIs of the admin client. +// // +// // We test the following situations: +// // +// // 1. One consumer group with one client. +// // 2. One consumer group with two clients. +// // 3. Empty consumer group. +// func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() { +// t := its.T() + +// // Generating a new topic/groupID to ensure a fresh group/topic is created. +// rand.Seed(time.Now().Unix()) +// groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) +// topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) +// nonExistentGroupID := fmt.Sprintf("%s-nonexistent-%d", testconf.GroupID, rand.Int()) + +// clientID1 := "test.client.1" +// clientID2 := "test.client.2" + +// ac := createAdminClient(t) +// defer ac.Close() + +// // Create a topic - we need to create here because we need 2 partitions. +// ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) +// defer cancel() +// _, err := ac.CreateTopics(ctx, []TopicSpecification{ +// { +// Topic: topic, +// NumPartitions: 2, +// }, +// }) +// if err != nil { +// t.Errorf("Topic creation failed with error %v", err) +// return +// } + +// // Delete the topic after the test is done. +// defer func() { +// ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) +// defer cancel() +// _, err = ac.DeleteTopics(ctx, []string{topic}) +// if err != nil { +// t.Errorf("Topic deletion failed with error %v", err) +// } +// }() + +// // Check the non-existence of consumer groups initially. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) +// if err != nil || len(listGroupResult.Errors) > 0 { +// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) +// return +// } + +// groups := listGroupResult.Valid +// if findConsumerGroupListing(groups, groupID) != nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { +// t.Errorf("Consumer groups %s and %s should not be present\n", groupID, nonExistentGroupID) +// return +// } + +// // 1. One consumer group with one client. +// // Create the first consumer. +// config := &ConfigMap{ +// "bootstrap.servers": testconf.Brokers, +// "group.id": groupID, +// "auto.offset.reset": "earliest", +// "enable.auto.offset.store": false, +// "client.id": clientID1, +// "partition.assignment.strategy": "range", +// } +// config.updateFromTestconf() +// consumer1, err := NewConsumer(config) +// if err != nil { +// t.Errorf("Failed to create consumer: %s\n", err) +// return +// } +// consumer1Closed := false +// defer func() { +// if !consumer1Closed { +// consumer1.Close() +// } +// }() +// consumer1.Subscribe(topic, nil) + +// // Call Poll to trigger a rebalance and give it enough time to finish. +// consumer1.Poll(10 * 1000) + +// // Check the existence of the group. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) +// if err != nil || len(listGroupResult.Errors) > 0 { +// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) +// return +// } +// groups = listGroupResult.Valid + +// if findConsumerGroupListing(groups, groupID) == nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { +// t.Errorf("Consumer groups %s should be present and %s should not be\n", groupID, nonExistentGroupID) +// return +// } + +// // Test the description of the consumer group. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// groupDescResult, err := ac.DescribeConsumerGroups( +// ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) +// if err != nil { +// t.Errorf("Error describing consumer groups %s\n", err) +// return +// } + +// groupDescs := groupDescResult.ConsumerGroupDescriptions +// if len(groupDescs) != 1 { +// t.Errorf("Describing one group should give exactly one result %s\n", err) +// return +// } + +// groupDesc := &groupDescs[0] + +// clientIDToPartitions := make(map[string][]TopicPartition) +// clientIDToPartitions[clientID1] = []TopicPartition{ +// {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, +// {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, +// } +// if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { +// t.Errorf("Expected description for consumer group %s is not same as actual: %v", groupID, groupDesc) +// return +// } + +// // 2. One consumer group with two clients. +// // Add another consumer to the same group. +// config = &ConfigMap{ +// "bootstrap.servers": testconf.Brokers, +// "group.id": groupID, +// "auto.offset.reset": "earliest", +// "enable.auto.offset.store": false, +// "client.id": clientID2, +// "partition.assignment.strategy": "range", +// } +// config.updateFromTestconf() +// consumer2, err := NewConsumer(config) +// if err != nil { +// t.Errorf("Failed to create consumer: %s\n", err) +// return +// } +// consumer2Closed := false +// defer func() { +// if !consumer2Closed { +// consumer2.Close() +// } +// }() +// consumer2.Subscribe(topic, nil) + +// // Call Poll to start triggering the rebalance. Give it enough time to run +// // that it becomes stable. +// // We need to make sure that the consumer group stabilizes since we will +// // check for the state later on. +// consumer2.Poll(5 * 1000) +// consumer1.Poll(5 * 1000) +// isGroupStable := false +// for !isGroupStable { +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) +// if err != nil { +// t.Errorf("Error describing consumer groups %s\n", err) +// return +// } +// groupDescs = groupDescResult.ConsumerGroupDescriptions +// groupDesc = findConsumerGroupDescription(groupDescs, groupID) +// if groupDesc == nil { +// t.Errorf("Consumer group %s should be present\n", groupID) +// return +// } +// isGroupStable = groupDesc.State == ConsumerGroupStateStable +// time.Sleep(time.Second) +// } + +// clientIDToPartitions[clientID1] = []TopicPartition{ +// {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, +// } +// clientIDToPartitions[clientID2] = []TopicPartition{ +// {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, +// } +// if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { +// t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) +// return +// } + +// // 3. Empty consumer group. +// // Close the existing consumers. +// if consumer1.Close() != nil { +// t.Errorf("Error closing the first consumer\n") +// return +// } +// consumer1Closed = true + +// if consumer2.Close() != nil { +// t.Errorf("Error closing the second consumer\n") +// return +// } +// consumer2Closed = true + +// // Try describing an empty group. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) +// groupDescs = groupDescResult.ConsumerGroupDescriptions + +// if err != nil { +// t.Errorf("Error describing consumer groups %s\n", err) +// return +// } + +// groupDesc = findConsumerGroupDescription(groupDescs, groupID) +// if groupDesc == nil { +// t.Errorf("Consumer group %s should be present\n", groupID) +// return +// } + +// clientIDToPartitions = make(map[string][]TopicPartition) +// if !checkGroupDesc(groupDesc, ConsumerGroupStateEmpty, groupID, "", clientIDToPartitions) { +// t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) +// } + +// // Try listing the Empty consumer group, and make sure that the States option +// // works while listing. +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err = ac.ListConsumerGroups( +// ctx, SetAdminRequestTimeout(30*time.Second), +// SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateEmpty})) +// if err != nil || len(listGroupResult.Errors) > 0 { +// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) +// return +// } +// groups = listGroupResult.Valid + +// groupInfo := findConsumerGroupListing(listGroupResult.Valid, groupID) +// if groupInfo == nil { +// t.Errorf("Consumer group %s should be present\n", groupID) +// return +// } + +// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) +// defer cancel() +// listGroupResult, err = ac.ListConsumerGroups( +// ctx, SetAdminRequestTimeout(30*time.Second), +// SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateStable})) +// if err != nil || len(listGroupResult.Errors) > 0 { +// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) +// return +// } +// groups = listGroupResult.Valid + +// groupInfo = findConsumerGroupListing(groups, groupID) +// if groupInfo != nil { +// t.Errorf("Consumer group %s should not be present\n", groupID) +// return +// } +// } + +// TestAdminClient_DescribeCluster validates the working of the +// describe cluster API of the admin client. +// +// We test the following situations: +// +// 1. DescribeCluster without createAcl. +// 2. DescribeCluster with Acl. +func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { t := its.T() - if testconf.Semaphore { - t.Skipf("Skipping TestAdminClient_DeleteConsumerGroups since it is flaky[Does not run when tested with all the other integration tests]") - return - } - rand.Seed(time.Now().Unix()) - - // Generating new groupID to ensure a fresh group is created. - groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) - ac := createAdminClient(t) defer ac.Close() + noError := NewError(ErrNoError, "", false) + checkExpectedResult := func(expected interface{}, result interface{}) { + if !reflect.DeepEqual(result, expected) { + t.Fatalf("Expected result to deep equal to %v, but found %v", expected, result) + } + } - // Check that our group is not present initially. + // Check the non-existence of consumer groups initially. ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() - listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) - if err != nil { - t.Errorf("Error listing consumer groups %s\n", err) - return + descres, err := ac.DescribeCluster( + ctx, SetAdminRequestTimeout(time.Second), SetAdminOptionIncludeClusterAuthorizedOperations(true)) + if descres.Nodes == nil || err != nil { + t.Fatalf("Expected DescribeTopics to pass, but not %s %v", + err.(Error).Code(), ctx.Err()) } - - if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { - t.Errorf("Consumer group present before consumer created: %s\n", groupID) - return + initialLen := len(descres.ClusterAuthorizedOperations) + if initialLen == 0 { + t.Fatalf("Expected cluster authorized operations>0" + + "as they are being requested") } + fmt.Printf("Initial length: %d\n", initialLen) - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - - // Create consumer - config := &ConfigMap{ - "bootstrap.servers": testconf.Brokers, - "group.id": groupID, - "auto.offset.reset": "earliest", - "enable.auto.offset.store": false, + newACLs := ACLBindings{ + { + Type: ResourceBroker, + Name: "kafka-cluster", + ResourcePatternType: ResourcePatternTypeLiteral, + Principal: "User:*", + Host: "*", + Operation: ACLOperationAlter, + PermissionType: ACLPermissionTypeAllow, + }, } - config.updateFromTestconf() - consumer, err := NewConsumer(config) + maxDuration, err := time.ParseDuration("30s") if err != nil { - t.Errorf("Failed to create consumer: %s\n", err) - return - } - consumerClosed := false - defer func() { - if !consumerClosed { - consumer.Close() - } - }() - - if err := consumer.Subscribe(testconf.Topic, nil); err != nil { - t.Errorf("Failed to subscribe to %s: %s\n", testconf.Topic, err) - return - } - - // This ReadMessage gives some time for the rebalance to happen. - _, err = consumer.ReadMessage(5 * time.Second) - if err != nil && err.(Error).Code() != ErrTimedOut { - t.Errorf("Failed while reading message: %s\n", err) - return + t.Fatalf("%s", err) } - - // Check that the group exists. - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + requestTimeout, err := time.ParseDuration("20s") if err != nil { - t.Errorf("Error listing consumer groups %s\n", err) - return - } - - if findConsumerGroupListing(listGroupResult.Valid, groupID) == nil { - t.Errorf("Consumer group %s should be present\n", groupID) - return + t.Fatalf("%s", err) } - - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) defer cancel() - // Try deleting the group while consumer is active. It should fail. - result, err := ac.DeleteConsumerGroups(ctx, []string{groupID}) + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) if err != nil { - t.Errorf("DeleteConsumerGroups() failed: %s", err) - return - } - resultGroups := result.ConsumerGroupResults - - if len(resultGroups) != 1 || resultGroups[0].Group != groupID { - t.Errorf("Wrong group affected/no group affected") - return - } - - if resultGroups[0].Error.code != ErrNonEmptyGroup { - t.Errorf("Encountered the wrong error after calling DeleteConsumerGroups %s", resultGroups[0].Error) - return - } - - // Close the consumer. - if err = consumer.Close(); err != nil { - t.Errorf("Could not close consumer %s", err) - return + t.Fatalf("CreateACLs() failed: %s", err) } - consumerClosed = true + expectedCreateACLs := []CreateACLResult{{Error: noError}} + checkExpectedResult(expectedCreateACLs, resultCreateACLs) - // Delete the consumer group now. - result, err = ac.DeleteConsumerGroups(ctx, []string{groupID}) - if err != nil { - t.Errorf("DeleteConsumerGroups() failed: %s", err) - return + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + descres, err = ac.DescribeCluster( + ctx, SetAdminRequestTimeout(time.Second), SetAdminOptionIncludeClusterAuthorizedOperations(true)) + if descres.Nodes == nil || err != nil { + t.Fatalf("Expected DescribeTopics to pass, but not %s %v", + err.(Error).Code(), ctx.Err()) } - resultGroups = result.ConsumerGroupResults - - if len(resultGroups) != 1 || resultGroups[0].Group != groupID { - t.Errorf("Wrong group affected/no group affected") - return + finalLen := len(descres.ClusterAuthorizedOperations) + fmt.Printf("Final length: %d\n", finalLen) + if initialLen <= finalLen { + t.Fatalf("Expected final acl count to have reduced after createAcl") } - if resultGroups[0].Error.code != ErrNoError { - t.Errorf("Encountered an error after calling DeleteConsumerGroups %s", resultGroups[0].Error) - return + aclBindingFilters := ACLBindingFilters{ + { + Type: ResourceBroker, + Name: "kafka-cluster", + ResourcePatternType: ResourcePatternTypeMatch, + Principal: "User:*", + Host: "*", + Operation: ACLOperationAlter, + PermissionType: ACLPermissionTypeAllow, + }, } - - // Check for the absence of the consumer group after deletion. - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) defer cancel() - listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + _, err = ac.DeleteACLs(ctx, aclBindingFilters, SetAdminRequestTimeout(requestTimeout)) if err != nil { - t.Errorf("Error listing consumer groups %s\n", err) - return - } - - if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { - t.Errorf("Consumer group %s should not be present\n", groupID) - return + t.Fatalf("%s", err) } } -// TestAdminClient_ListAndDescribeConsumerGroups validates the working of the -// list consumer groups and describe consumer group APIs of the admin client. +// TestAdminClient_DescribeTopics validates the working of the +// describe topics API of the admin client. // // We test the following situations: // -// 1. One consumer group with one client. -// 2. One consumer group with two clients. -// 3. Empty consumer group. -func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() { +// 1. DescribeTopics without createAcl. +// 2. DescribeTopics with Acl. +func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { t := its.T() - - // Generating a new topic/groupID to ensure a fresh group/topic is created. - rand.Seed(time.Now().Unix()) - groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) - topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) - nonExistentGroupID := fmt.Sprintf("%s-nonexistent-%d", testconf.GroupID, rand.Int()) - - clientID1 := "test.client.1" - clientID2 := "test.client.2" - ac := createAdminClient(t) defer ac.Close() + noError := NewError(ErrNoError, "", false) + checkExpectedResult := func(expected interface{}, result interface{}) { + if !reflect.DeepEqual(result, expected) { + t.Fatalf("Expected result to deep equal to %v, but found %v", expected, result) + } + } - // Create a topic - we need to create here because we need 2 partitions. + // Create a topic + topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() _, err := ac.CreateTopics(ctx, []TopicSpecification{ @@ -863,622 +1219,508 @@ func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() } }() - // Check the non-existence of consumer groups initially. - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) - if err != nil || len(listGroupResult.Errors) > 0 { - t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) - return - } - - groups := listGroupResult.Valid - if findConsumerGroupListing(groups, groupID) != nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { - t.Errorf("Consumer groups %s and %s should not be present\n", groupID, nonExistentGroupID) - return - } - - // 1. One consumer group with one client. - // Create the first consumer. - config := &ConfigMap{ - "bootstrap.servers": testconf.Brokers, - "group.id": groupID, - "auto.offset.reset": "earliest", - "enable.auto.offset.store": false, - "client.id": clientID1, - "partition.assignment.strategy": "range", - } - config.updateFromTestconf() - consumer1, err := NewConsumer(config) - if err != nil { - t.Errorf("Failed to create consumer: %s\n", err) - return - } - consumer1Closed := false - defer func() { - if !consumer1Closed { - consumer1.Close() - } - }() - consumer1.Subscribe(topic, nil) - - // Call Poll to trigger a rebalance and give it enough time to finish. - consumer1.Poll(10 * 1000) - - // Check the existence of the group. - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) - if err != nil || len(listGroupResult.Errors) > 0 { - t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) - return - } - groups = listGroupResult.Valid - - if findConsumerGroupListing(groups, groupID) == nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { - t.Errorf("Consumer groups %s should be present and %s should not be\n", groupID, nonExistentGroupID) - return - } - - // Test the description of the consumer group. + // Test the description of the topic. ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() - groupDescResult, err := ac.DescribeConsumerGroups( - ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) + topicDescResult, err := ac.DescribeTopics( + ctx, []string{topic, "failure"}, SetAdminRequestTimeout(30*time.Second), + SetAdminOptionIncludeTopicAuthorizedOperations(true)) if err != nil { - t.Errorf("Error describing consumer groups %s\n", err) + t.Errorf("Error describing topics %s\n", err) return } - groupDescs := groupDescResult.ConsumerGroupDescriptions - if len(groupDescs) != 1 { - t.Errorf("Describing one group should give exactly one result %s\n", err) + topicDescs := topicDescResult.TopicDescriptions + if len(topicDescs) != 2 { + t.Errorf("Describing two topics should give exactly two results %s\n", err) return } - - groupDesc := &groupDescs[0] - - clientIDToPartitions := make(map[string][]TopicPartition) - clientIDToPartitions[clientID1] = []TopicPartition{ - {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, - {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, + if topicDescs[1].Error.Code() != 3 { + t.Fatalf("Expected expected unknown Topic or partition, not %s\n", + topicDescs[1].Error) } - if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { - t.Errorf("Expected description for consumer group %s is not same as actual: %v", groupID, groupDesc) - return + initialLen := len(topicDescs[0].TopicAuthorizedOperations) + if initialLen == 0 { + t.Fatalf("Expected topic authorized operations>0" + + "as they are being requested") } + fmt.Printf("Initial length: %d\n", initialLen) - // 2. One consumer group with two clients. - // Add another consumer to the same group. - config = &ConfigMap{ - "bootstrap.servers": testconf.Brokers, - "group.id": groupID, - "auto.offset.reset": "earliest", - "enable.auto.offset.store": false, - "client.id": clientID2, - "partition.assignment.strategy": "range", + newACLs := ACLBindings{ + { + Type: ResourceTopic, + Name: topic, + ResourcePatternType: ResourcePatternTypeLiteral, + Principal: "User:*", + Host: "*", + Operation: ACLOperationRead, + PermissionType: ACLPermissionTypeAllow, + }, } - config.updateFromTestconf() - consumer2, err := NewConsumer(config) + maxDuration, err := time.ParseDuration("30s") if err != nil { - t.Errorf("Failed to create consumer: %s\n", err) - return - } - consumer2Closed := false - defer func() { - if !consumer2Closed { - consumer2.Close() - } - }() - consumer2.Subscribe(topic, nil) - - // Call Poll to start triggering the rebalance. Give it enough time to run - // that it becomes stable. - // We need to make sure that the consumer group stabilizes since we will - // check for the state later on. - consumer2.Poll(5 * 1000) - consumer1.Poll(5 * 1000) - isGroupStable := false - for !isGroupStable { - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) - if err != nil { - t.Errorf("Error describing consumer groups %s\n", err) - return - } - groupDescs = groupDescResult.ConsumerGroupDescriptions - groupDesc = findConsumerGroupDescription(groupDescs, groupID) - if groupDesc == nil { - t.Errorf("Consumer group %s should be present\n", groupID) - return - } - isGroupStable = groupDesc.State == ConsumerGroupStateStable - time.Sleep(time.Second) - } - - clientIDToPartitions[clientID1] = []TopicPartition{ - {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, - } - clientIDToPartitions[clientID2] = []TopicPartition{ - {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, - } - if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { - t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) - return + t.Fatalf("%s", err) } - - // 3. Empty consumer group. - // Close the existing consumers. - if consumer1.Close() != nil { - t.Errorf("Error closing the first consumer\n") - return + requestTimeout, err := time.ParseDuration("20s") + if err != nil { + t.Fatalf("%s", err) } - consumer1Closed = true + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + defer cancel() - if consumer2.Close() != nil { - t.Errorf("Error closing the second consumer\n") - return + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) + if err != nil { + t.Fatalf("CreateACLs() failed: %s", err) } - consumer2Closed = true + expectedCreateACLs := []CreateACLResult{{Error: noError}} + checkExpectedResult(expectedCreateACLs, resultCreateACLs) - // Try describing an empty group. ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() - groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) - groupDescs = groupDescResult.ConsumerGroupDescriptions - + topicDescResult, err = ac.DescribeTopics( + ctx, []string{topic}, SetAdminRequestTimeout(30*time.Second), + SetAdminOptionIncludeTopicAuthorizedOperations(true)) if err != nil { - t.Errorf("Error describing consumer groups %s\n", err) + t.Errorf("Error describing topics %s\n", err) return } - groupDesc = findConsumerGroupDescription(groupDescs, groupID) - if groupDesc == nil { - t.Errorf("Consumer group %s should be present\n", groupID) + topicDescs = topicDescResult.TopicDescriptions + if len(topicDescs) != 1 { + t.Errorf("Describing one topics should give exactly one result %s\n", err) return } - - clientIDToPartitions = make(map[string][]TopicPartition) - if !checkGroupDesc(groupDesc, ConsumerGroupStateEmpty, groupID, "", clientIDToPartitions) { - t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) + finalLen := len(topicDescs[0].TopicAuthorizedOperations) + if finalLen == 0 { + t.Fatalf("Expected topic authorized operations>0" + + "as they are being requested") } - - // Try listing the Empty consumer group, and make sure that the States option - // works while listing. - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - listGroupResult, err = ac.ListConsumerGroups( - ctx, SetAdminRequestTimeout(30*time.Second), - SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateEmpty})) - if err != nil || len(listGroupResult.Errors) > 0 { - t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) - return + fmt.Printf("Initial length: %d\n", initialLen) + fmt.Printf("Final length: %d\n", finalLen) + if initialLen <= finalLen { + t.Fatalf("Expected final acl count to have reduced after createAcl") } - groups = listGroupResult.Valid - groupInfo := findConsumerGroupListing(listGroupResult.Valid, groupID) - if groupInfo == nil { - t.Errorf("Consumer group %s should be present\n", groupID) - return + newACLs = ACLBindings{ + { + Type: ResourceTopic, + Name: topic, + ResourcePatternType: ResourcePatternTypeLiteral, + Principal: "User:*", + Host: "*", + Operation: ACLOperationDelete, + PermissionType: ACLPermissionTypeAllow, + }, } - - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) defer cancel() - listGroupResult, err = ac.ListConsumerGroups( - ctx, SetAdminRequestTimeout(30*time.Second), - SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateStable})) - if err != nil || len(listGroupResult.Errors) > 0 { - t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) - return - } - groups = listGroupResult.Valid - groupInfo = findConsumerGroupListing(groups, groupID) - if groupInfo != nil { - t.Errorf("Consumer group %s should not be present\n", groupID) - return + resultCreateACLs, err = ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) + if err != nil { + t.Fatalf("CreateACLs() failed: %s", err) } + expectedCreateACLs = []CreateACLResult{{Error: noError}} + checkExpectedResult(expectedCreateACLs, resultCreateACLs) } -func (its *IntegrationTestSuite) TestAdminTopics() { +// func (its *IntegrationTestSuite) TestAdminTopics() { +// t := its.T() +// rand.Seed(time.Now().Unix()) + +// a := createAdminClient(t) +// defer a.Close() + +// brokerList, err := getBrokerList(a) +// if err != nil { +// t.Fatalf("Failed to retrieve broker list: %v", err) +// } + +// // Few and Many replica sets use in these tests +// var fewReplicas []int32 +// if len(brokerList) < 2 { +// fewReplicas = brokerList +// } else { +// fewReplicas = brokerList[0:2] +// } + +// var manyReplicas []int32 +// if len(brokerList) < 5 { +// manyReplicas = brokerList +// } else { +// manyReplicas = brokerList[0:5] +// } + +// const topicCnt = 7 +// newTopics := make([]TopicSpecification, topicCnt) + +// expError := map[string]Error{} + +// for i := 0; i < topicCnt; i++ { +// topic := fmt.Sprintf("%s-create-%d-%d", testconf.Topic, i, rand.Intn(100000)) +// newTopics[i] = TopicSpecification{ +// Topic: topic, +// NumPartitions: 1 + i*2, +// } + +// if (i % 1) == 0 { +// newTopics[i].ReplicationFactor = len(fewReplicas) +// } else { +// newTopics[i].ReplicationFactor = len(manyReplicas) +// } + +// expError[newTopics[i].Topic] = Error{} // No error + +// var useReplicas []int32 +// if i == 2 { +// useReplicas = fewReplicas +// } else if i == 3 { +// useReplicas = manyReplicas +// } else if i == topicCnt-1 { +// newTopics[i].ReplicationFactor = len(brokerList) + 10 +// expError[newTopics[i].Topic] = Error{code: ErrInvalidReplicationFactor} +// } + +// if len(useReplicas) > 0 { +// newTopics[i].ReplicaAssignment = make([][]int32, newTopics[i].NumPartitions) +// newTopics[i].ReplicationFactor = 0 +// for p := 0; p < newTopics[i].NumPartitions; p++ { +// newTopics[i].ReplicaAssignment[p] = useReplicas +// } +// } +// } + +// maxDuration, err := time.ParseDuration("30s") +// if err != nil { +// t.Fatalf("%s", err) +// } + +// // First just validate the topics, don't create +// t.Logf("Validating topics before creation\n") +// ctx, cancel := context.WithTimeout(context.Background(), maxDuration) +// defer cancel() +// result, err := a.CreateTopics(ctx, newTopics, +// SetAdminValidateOnly(true)) +// if err != nil { +// t.Fatalf("CreateTopics(ValidateOnly) failed: %s", err) +// } + +// validateTopicResult(t, result, expError) + +// // Now create the topics +// t.Logf("Creating topics\n") +// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) +// defer cancel() +// result, err = a.CreateTopics(ctx, newTopics, SetAdminValidateOnly(false)) +// if err != nil { +// t.Fatalf("CreateTopics() failed: %s", err) +// } + +// validateTopicResult(t, result, expError) + +// // Attempt to create the topics again, should all fail. +// t.Logf("Attempt to re-create topics, should all fail\n") +// for k := range expError { +// if expError[k].code == ErrNoError { +// expError[k] = Error{code: ErrTopicAlreadyExists} +// } +// } +// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) +// defer cancel() +// result, err = a.CreateTopics(ctx, newTopics) +// if err != nil { +// t.Fatalf("CreateTopics#2() failed: %s", err) +// } + +// validateTopicResult(t, result, expError) + +// // Add partitions to some of the topics +// t.Logf("Create new partitions for a subset of topics\n") +// newParts := make([]PartitionsSpecification, topicCnt/2) +// expError = map[string]Error{} +// for i := 0; i < topicCnt/2; i++ { +// topic := newTopics[i].Topic +// newParts[i] = PartitionsSpecification{ +// Topic: topic, +// IncreaseTo: newTopics[i].NumPartitions + 3, +// } +// if i == 1 { +// // Invalid partition count (less than current) +// newParts[i].IncreaseTo = newTopics[i].NumPartitions - 1 +// expError[topic] = Error{code: ErrInvalidPartitions} +// } else { +// expError[topic] = Error{} +// } +// t.Logf("Creating new partitions for %s: %d -> %d: expecting %v\n", +// topic, newTopics[i].NumPartitions, newParts[i].IncreaseTo, expError[topic]) +// } + +// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) +// defer cancel() +// result, err = a.CreatePartitions(ctx, newParts) +// if err != nil { +// t.Fatalf("CreatePartitions() failed: %s", err) +// } + +// validateTopicResult(t, result, expError) + +// // FIXME: wait for topics to become available in metadata instead +// time.Sleep(5000 * time.Millisecond) + +// // Delete the topics +// deleteTopics := make([]string, topicCnt) +// for i := 0; i < topicCnt; i++ { +// deleteTopics[i] = newTopics[i].Topic +// if i == topicCnt-1 { +// expError[deleteTopics[i]] = Error{code: ErrUnknownTopicOrPart} +// } else { +// expError[deleteTopics[i]] = Error{} +// } +// } + +// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) +// defer cancel() +// result2, err := a.DeleteTopics(ctx, deleteTopics) +// if err != nil { +// t.Fatalf("DeleteTopics() failed: %s", err) +// } + +// validateTopicResult(t, result2, expError) +// } + +// func (its *IntegrationTestSuite) TestAdminConfig() { +// t := its.T() +// rand.Seed(time.Now().Unix()) + +// a := createAdminClient(t) +// defer a.Close() + +// // Steps: +// // 1) Create a topic, providing initial non-default configuration +// // 2) Read back config to verify +// // 3) Alter config +// // 4) Read back config to verify +// // 5) Delete the topic + +// topic := fmt.Sprintf("%s-config-%d", testconf.Topic, rand.Intn(100000)) + +// // Expected config +// expResources := []ConfigResourceResult{ +// { +// Type: ResourceTopic, +// Name: topic, +// Config: map[string]ConfigEntryResult{ +// "compression.type": ConfigEntryResult{ +// Name: "compression.type", +// Value: "snappy", +// }, +// }, +// }, +// } +// // Create topic +// newTopics := []TopicSpecification{{ +// Topic: topic, +// NumPartitions: 1, +// ReplicationFactor: 1, +// Config: map[string]string{"compression.type": "snappy"}, +// }} + +// ctx, cancel := context.WithCancel(context.Background()) +// defer cancel() +// topicResult, err := a.CreateTopics(ctx, newTopics) +// if err != nil { +// t.Fatalf("Create topic request failed: %v", err) +// } + +// if topicResult[0].Error.Code() != ErrNoError { +// t.Fatalf("Failed to create topic %s: %s", topic, topicResult[0].Error) +// } + +// // Wait for topic to show up in metadata before performing +// // subsequent operations on it, otherwise we risk DescribeConfigs() +// // to fail with UnknownTopic.. (this is really a broker issue). +// // Sometimes even the metadata is not enough, so we add an +// // arbitrary 10s sleep too. +// t.Logf("Waiting for new topic %s to show up in metadata and stabilize", topic) +// err = waitTopicInMetadata(a, topic, 10*1000) // 10s +// if err != nil { +// t.Fatalf("%v", err) +// } +// t.Logf("Topic %s now in metadata, waiting another 10s for stabilization", topic) +// time.Sleep(10 * time.Second) + +// // Read back config to validate +// configResources := []ConfigResource{{Type: ResourceTopic, Name: topic}} +// describeRes, err := a.DescribeConfigs(ctx, configResources) +// if err != nil { +// t.Fatalf("Describe configs request failed: %v", err) +// } + +// validateConfig(t, describeRes, expResources, true) + +// // Alter some configs. +// // Configuration alterations are currently atomic, all values +// // need to be passed, otherwise non-passed values will be reverted +// // to their default values. +// // Future versions will allow incremental updates: +// // https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API +// newConfig := make(map[string]string) +// for _, entry := range describeRes[0].Config { +// newConfig[entry.Name] = entry.Value +// } + +// // Change something +// newConfig["retention.ms"] = "86400000" +// newConfig["message.timestamp.type"] = "LogAppendTime" + +// for k, v := range newConfig { +// expResources[0].Config[k] = ConfigEntryResult{Name: k, Value: v} +// } + +// configResources = []ConfigResource{{Type: ResourceTopic, Name: topic, Config: StringMapToConfigEntries(newConfig, AlterOperationSet)}} +// alterRes, err := a.AlterConfigs(ctx, configResources) +// if err != nil { +// t.Fatalf("Alter configs request failed: %v", err) +// } + +// validateConfig(t, alterRes, expResources, false) + +// // Read back config to validate +// configResources = []ConfigResource{{Type: ResourceTopic, Name: topic}} +// describeRes, err = a.DescribeConfigs(ctx, configResources) +// if err != nil { +// t.Fatalf("Describe configs request failed: %v", err) +// } + +// validateConfig(t, describeRes, expResources, true) + +// // Delete the topic +// // FIXME: wait for topics to become available in metadata instead +// time.Sleep(5000 * time.Millisecond) + +// topicResult, err = a.DeleteTopics(ctx, []string{topic}) +// if err != nil { +// t.Fatalf("DeleteTopics() failed: %s", err) +// } + +// if topicResult[0].Error.Code() != ErrNoError { +// t.Fatalf("Failed to delete topic %s: %s", topic, topicResult[0].Error) +// } +// } + +// // Test AdminClient GetMetadata API +// func (its *IntegrationTestSuite) TestAdminGetMetadata() { +// t := its.T() + +// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} +// config.updateFromTestconf() + +// // Create Admin client +// a, err := NewAdminClient(config) +// if err != nil { +// t.Errorf("Failed to create Admin client: %s\n", err) +// return +// } +// defer a.Close() + +// metaData, err := a.GetMetadata(&testconf.Topic, false, 5*1000) +// if err != nil { +// t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) +// return +// } +// t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) + +// metaData, err = a.GetMetadata(nil, true, 5*1000) +// if err != nil { +// t.Errorf("Failed to get meta data, Error: %s\n", err) +// return +// } +// t.Logf("Meta data for admin client: %v\n", metaData) +// } + +// // Test AdminClient ClusterID. +// func (its *IntegrationTestSuite) TestAdminClient_ClusterID() { +// t := its.T() + +// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} +// if err := config.updateFromTestconf(); err != nil { +// t.Fatalf("Failed to update test configuration: %s\n", err) +// } + +// admin, err := NewAdminClient(config) +// if err != nil { +// t.Fatalf("Failed to create Admin client: %s\n", err) +// } +// defer admin.Close() + +// ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) +// defer cancel() +// clusterID, err := admin.ClusterID(ctx) +// if err != nil { +// t.Fatalf("Failed to get ClusterID: %s\n", err) +// } +// if clusterID == "" { +// t.Fatal("ClusterID is empty.") +// } + +// t.Logf("ClusterID: %s\n", clusterID) +// } + +// // Test AdminClient ControllerID. +// func (its *IntegrationTestSuite) TestAdminClient_ControllerID() { +// t := its.T() + +// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} +// if err := config.updateFromTestconf(); err != nil { +// t.Fatalf("Failed to update test configuration: %s\n", err) +// } + +// producer, err := NewProducer(config) +// if err != nil { +// t.Fatalf("Failed to create Producer client: %s\n", err) +// } +// admin, err := NewAdminClientFromProducer(producer) +// if err != nil { +// t.Fatalf("Failed to create Admin client: %s\n", err) +// } +// defer admin.Close() + +// ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) +// defer cancel() +// controllerID, err := admin.ControllerID(ctx) +// if err != nil { +// t.Fatalf("Failed to get ControllerID: %s\n", err) +// } +// if controllerID < 0 { +// t.Fatalf("ControllerID is negative: %d\n", controllerID) +// } + +// t.Logf("ControllerID: %d\n", controllerID) +// } + +func (its *IntegrationTestSuite) TestAdminACLs() { t := its.T() + rand.Seed(time.Now().Unix()) + topic := testconf.Topic + group := testconf.GroupID + noError := NewError(ErrNoError, "", false) + unknownError := NewError(ErrUnknown, "Unknown broker error", false) + var expectedCreateACLs []CreateACLResult + var expectedDescribeACLs DescribeACLsResult + var expectedDeleteACLs []DeleteACLsResult + var ctx context.Context + var cancel context.CancelFunc a := createAdminClient(t) defer a.Close() - brokerList, err := getBrokerList(a) - if err != nil { - t.Fatalf("Failed to retrieve broker list: %v", err) - } - - // Few and Many replica sets use in these tests - var fewReplicas []int32 - if len(brokerList) < 2 { - fewReplicas = brokerList - } else { - fewReplicas = brokerList[0:2] - } - - var manyReplicas []int32 - if len(brokerList) < 5 { - manyReplicas = brokerList - } else { - manyReplicas = brokerList[0:5] - } - - const topicCnt = 7 - newTopics := make([]TopicSpecification, topicCnt) - - expError := map[string]Error{} - - for i := 0; i < topicCnt; i++ { - topic := fmt.Sprintf("%s-create-%d-%d", testconf.Topic, i, rand.Intn(100000)) - newTopics[i] = TopicSpecification{ - Topic: topic, - NumPartitions: 1 + i*2, - } - - if (i % 1) == 0 { - newTopics[i].ReplicationFactor = len(fewReplicas) - } else { - newTopics[i].ReplicationFactor = len(manyReplicas) - } - - expError[newTopics[i].Topic] = Error{} // No error - - var useReplicas []int32 - if i == 2 { - useReplicas = fewReplicas - } else if i == 3 { - useReplicas = manyReplicas - } else if i == topicCnt-1 { - newTopics[i].ReplicationFactor = len(brokerList) + 10 - expError[newTopics[i].Topic] = Error{code: ErrInvalidReplicationFactor} - } - - if len(useReplicas) > 0 { - newTopics[i].ReplicaAssignment = make([][]int32, newTopics[i].NumPartitions) - newTopics[i].ReplicationFactor = 0 - for p := 0; p < newTopics[i].NumPartitions; p++ { - newTopics[i].ReplicaAssignment[p] = useReplicas - } - } - } - maxDuration, err := time.ParseDuration("30s") if err != nil { t.Fatalf("%s", err) } - - // First just validate the topics, don't create - t.Logf("Validating topics before creation\n") - ctx, cancel := context.WithTimeout(context.Background(), maxDuration) - defer cancel() - result, err := a.CreateTopics(ctx, newTopics, - SetAdminValidateOnly(true)) + requestTimeout, err := time.ParseDuration("20s") if err != nil { - t.Fatalf("CreateTopics(ValidateOnly) failed: %s", err) - } - - validateTopicResult(t, result, expError) - - // Now create the topics - t.Logf("Creating topics\n") - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) - defer cancel() - result, err = a.CreateTopics(ctx, newTopics, SetAdminValidateOnly(false)) - if err != nil { - t.Fatalf("CreateTopics() failed: %s", err) - } - - validateTopicResult(t, result, expError) - - // Attempt to create the topics again, should all fail. - t.Logf("Attempt to re-create topics, should all fail\n") - for k := range expError { - if expError[k].code == ErrNoError { - expError[k] = Error{code: ErrTopicAlreadyExists} - } - } - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) - defer cancel() - result, err = a.CreateTopics(ctx, newTopics) - if err != nil { - t.Fatalf("CreateTopics#2() failed: %s", err) - } - - validateTopicResult(t, result, expError) - - // Add partitions to some of the topics - t.Logf("Create new partitions for a subset of topics\n") - newParts := make([]PartitionsSpecification, topicCnt/2) - expError = map[string]Error{} - for i := 0; i < topicCnt/2; i++ { - topic := newTopics[i].Topic - newParts[i] = PartitionsSpecification{ - Topic: topic, - IncreaseTo: newTopics[i].NumPartitions + 3, - } - if i == 1 { - // Invalid partition count (less than current) - newParts[i].IncreaseTo = newTopics[i].NumPartitions - 1 - expError[topic] = Error{code: ErrInvalidPartitions} - } else { - expError[topic] = Error{} - } - t.Logf("Creating new partitions for %s: %d -> %d: expecting %v\n", - topic, newTopics[i].NumPartitions, newParts[i].IncreaseTo, expError[topic]) - } - - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) - defer cancel() - result, err = a.CreatePartitions(ctx, newParts) - if err != nil { - t.Fatalf("CreatePartitions() failed: %s", err) - } - - validateTopicResult(t, result, expError) - - // FIXME: wait for topics to become available in metadata instead - time.Sleep(5000 * time.Millisecond) - - // Delete the topics - deleteTopics := make([]string, topicCnt) - for i := 0; i < topicCnt; i++ { - deleteTopics[i] = newTopics[i].Topic - if i == topicCnt-1 { - expError[deleteTopics[i]] = Error{code: ErrUnknownTopicOrPart} - } else { - expError[deleteTopics[i]] = Error{} - } - } - - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) - defer cancel() - result2, err := a.DeleteTopics(ctx, deleteTopics) - if err != nil { - t.Fatalf("DeleteTopics() failed: %s", err) - } - - validateTopicResult(t, result2, expError) -} - -func (its *IntegrationTestSuite) TestAdminConfig() { - t := its.T() - rand.Seed(time.Now().Unix()) - - a := createAdminClient(t) - defer a.Close() - - // Steps: - // 1) Create a topic, providing initial non-default configuration - // 2) Read back config to verify - // 3) Alter config - // 4) Read back config to verify - // 5) Delete the topic - - topic := fmt.Sprintf("%s-config-%d", testconf.Topic, rand.Intn(100000)) - - // Expected config - expResources := []ConfigResourceResult{ - { - Type: ResourceTopic, - Name: topic, - Config: map[string]ConfigEntryResult{ - "compression.type": ConfigEntryResult{ - Name: "compression.type", - Value: "snappy", - }, - }, - }, - } - // Create topic - newTopics := []TopicSpecification{{ - Topic: topic, - NumPartitions: 1, - ReplicationFactor: 1, - Config: map[string]string{"compression.type": "snappy"}, - }} - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - topicResult, err := a.CreateTopics(ctx, newTopics) - if err != nil { - t.Fatalf("Create topic request failed: %v", err) - } - - if topicResult[0].Error.Code() != ErrNoError { - t.Fatalf("Failed to create topic %s: %s", topic, topicResult[0].Error) - } - - // Wait for topic to show up in metadata before performing - // subsequent operations on it, otherwise we risk DescribeConfigs() - // to fail with UnknownTopic.. (this is really a broker issue). - // Sometimes even the metadata is not enough, so we add an - // arbitrary 10s sleep too. - t.Logf("Waiting for new topic %s to show up in metadata and stabilize", topic) - err = waitTopicInMetadata(a, topic, 10*1000) // 10s - if err != nil { - t.Fatalf("%v", err) - } - t.Logf("Topic %s now in metadata, waiting another 10s for stabilization", topic) - time.Sleep(10 * time.Second) - - // Read back config to validate - configResources := []ConfigResource{{Type: ResourceTopic, Name: topic}} - describeRes, err := a.DescribeConfigs(ctx, configResources) - if err != nil { - t.Fatalf("Describe configs request failed: %v", err) - } - - validateConfig(t, describeRes, expResources, true) - - // Alter some configs. - // Configuration alterations are currently atomic, all values - // need to be passed, otherwise non-passed values will be reverted - // to their default values. - // Future versions will allow incremental updates: - // https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API - newConfig := make(map[string]string) - for _, entry := range describeRes[0].Config { - newConfig[entry.Name] = entry.Value - } - - // Change something - newConfig["retention.ms"] = "86400000" - newConfig["message.timestamp.type"] = "LogAppendTime" - - for k, v := range newConfig { - expResources[0].Config[k] = ConfigEntryResult{Name: k, Value: v} - } - - configResources = []ConfigResource{{Type: ResourceTopic, Name: topic, Config: StringMapToConfigEntries(newConfig, AlterOperationSet)}} - alterRes, err := a.AlterConfigs(ctx, configResources) - if err != nil { - t.Fatalf("Alter configs request failed: %v", err) - } - - validateConfig(t, alterRes, expResources, false) - - // Read back config to validate - configResources = []ConfigResource{{Type: ResourceTopic, Name: topic}} - describeRes, err = a.DescribeConfigs(ctx, configResources) - if err != nil { - t.Fatalf("Describe configs request failed: %v", err) - } - - validateConfig(t, describeRes, expResources, true) - - // Delete the topic - // FIXME: wait for topics to become available in metadata instead - time.Sleep(5000 * time.Millisecond) - - topicResult, err = a.DeleteTopics(ctx, []string{topic}) - if err != nil { - t.Fatalf("DeleteTopics() failed: %s", err) - } - - if topicResult[0].Error.Code() != ErrNoError { - t.Fatalf("Failed to delete topic %s: %s", topic, topicResult[0].Error) - } -} - -//Test AdminClient GetMetadata API -func (its *IntegrationTestSuite) TestAdminGetMetadata() { - t := its.T() - - config := &ConfigMap{"bootstrap.servers": testconf.Brokers} - config.updateFromTestconf() - - // Create Admin client - a, err := NewAdminClient(config) - if err != nil { - t.Errorf("Failed to create Admin client: %s\n", err) - return - } - defer a.Close() - - metaData, err := a.GetMetadata(&testconf.Topic, false, 5*1000) - if err != nil { - t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) - return - } - t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) - - metaData, err = a.GetMetadata(nil, true, 5*1000) - if err != nil { - t.Errorf("Failed to get meta data, Error: %s\n", err) - return - } - t.Logf("Meta data for admin client: %v\n", metaData) -} - -// Test AdminClient ClusterID. -func (its *IntegrationTestSuite) TestAdminClient_ClusterID() { - t := its.T() - - config := &ConfigMap{"bootstrap.servers": testconf.Brokers} - if err := config.updateFromTestconf(); err != nil { - t.Fatalf("Failed to update test configuration: %s\n", err) - } - - admin, err := NewAdminClient(config) - if err != nil { - t.Fatalf("Failed to create Admin client: %s\n", err) - } - defer admin.Close() - - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - clusterID, err := admin.ClusterID(ctx) - if err != nil { - t.Fatalf("Failed to get ClusterID: %s\n", err) - } - if clusterID == "" { - t.Fatal("ClusterID is empty.") - } - - t.Logf("ClusterID: %s\n", clusterID) -} - -// Test AdminClient ControllerID. -func (its *IntegrationTestSuite) TestAdminClient_ControllerID() { - t := its.T() - - config := &ConfigMap{"bootstrap.servers": testconf.Brokers} - if err := config.updateFromTestconf(); err != nil { - t.Fatalf("Failed to update test configuration: %s\n", err) - } - - producer, err := NewProducer(config) - if err != nil { - t.Fatalf("Failed to create Producer client: %s\n", err) - } - admin, err := NewAdminClientFromProducer(producer) - if err != nil { - t.Fatalf("Failed to create Admin client: %s\n", err) - } - defer admin.Close() - - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - controllerID, err := admin.ControllerID(ctx) - if err != nil { - t.Fatalf("Failed to get ControllerID: %s\n", err) - } - if controllerID < 0 { - t.Fatalf("ControllerID is negative: %d\n", controllerID) - } - - t.Logf("ControllerID: %d\n", controllerID) -} - -func (its *IntegrationTestSuite) TestAdminACLs() { - t := its.T() - - rand.Seed(time.Now().Unix()) - topic := testconf.Topic - group := testconf.GroupID - noError := NewError(ErrNoError, "", false) - unknownError := NewError(ErrUnknown, "Unknown broker error", false) - var expectedCreateACLs []CreateACLResult - var expectedDescribeACLs DescribeACLsResult - var expectedDeleteACLs []DeleteACLsResult - var ctx context.Context - var cancel context.CancelFunc - - a := createAdminClient(t) - defer a.Close() - - maxDuration, err := time.ParseDuration("30s") - if err != nil { - t.Fatalf("%s", err) - } - requestTimeout, err := time.ParseDuration("20s") - if err != nil { - t.Fatalf("%s", err) + t.Fatalf("%s", err) } checkExpectedResult := func(expected interface{}, result interface{}) { @@ -1647,741 +1889,741 @@ func (its *IntegrationTestSuite) TestAdminACLs() { checkExpectedResult(expectedDescribeACLs, *resultDescribeACLs) } -//Test consumer QueryWatermarkOffsets API -func (its *IntegrationTestSuite) TestConsumerQueryWatermarkOffsets() { - t := its.T() - - // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to - // get the number of messages in a topic - msgcnt, err := getMessageCountInTopic(testconf.Topic) - if err != nil { - t.Errorf("Cannot get message size. Error: %s\n", err) - } - - // Prime topic with test messages - createTestMessages() - producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) - - // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to - // get the number of messages in a topic - newmsgcnt, err := getMessageCountInTopic(testconf.Topic) - if err != nil { - t.Errorf("Cannot get message size. Error: %s\n", err) - } - - if newmsgcnt-msgcnt != len(p0TestMsgs) { - t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newmsgcnt-msgcnt) - } - -} - -//Test consumer GetWatermarkOffsets API -func (its *IntegrationTestSuite) TestConsumerGetWatermarkOffsets() { - t := its.T() - - // Create consumer - config := &ConfigMap{ - "go.events.channel.enable": true, - "bootstrap.servers": testconf.Brokers, - "group.id": testconf.GroupID, - "session.timeout.ms": 6000, - "enable.auto.commit": false, - "auto.offset.reset": "earliest", - } - _ = config.updateFromTestconf() - - c, err := NewConsumer(config) - if err != nil { - t.Fatalf("Unable to create consumer: %s", err) - } - defer func() { _ = c.Close() }() - - err = c.Subscribe(testconf.Topic, nil) - - // Prime topic with test messages - createTestMessages() - producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) - - // Wait for messages to be received so that we know the watermark offsets have been delivered - // with the fetch response - for ev := range c.Events() { - if _, ok := ev.(*Message); ok { - break - } - } - - _, queryHigh, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) - if err != nil { - t.Fatalf("Error querying watermark offsets: %s", err) - } - - // We are not currently testing the low watermark offset as it only gets set every 10s by the stits timer - _, getHigh, err := c.GetWatermarkOffsets(testconf.Topic, 0) - if err != nil { - t.Fatalf("Error getting watermark offsets: %s", err) - } - - if queryHigh != getHigh { - t.Errorf("QueryWatermarkOffsets high[%d] does not equal GetWatermarkOffsets high[%d]", queryHigh, getHigh) - } - -} - -//TestConsumerOffsetsForTimes -func (its *IntegrationTestSuite) TestConsumerOffsetsForTimes() { - t := its.T() - - conf := ConfigMap{"bootstrap.servers": testconf.Brokers, - "group.id": testconf.GroupID, - "api.version.request": true} - - conf.updateFromTestconf() - - c, err := NewConsumer(&conf) - - if err != nil { - panic(err) - } - defer c.Close() - - // Prime topic with test messages - createTestMessages() - producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) - - times := make([]TopicPartition, 1) - times[0] = TopicPartition{Topic: &testconf.Topic, Partition: 0, Offset: 12345} - offsets, err := c.OffsetsForTimes(times, 5000) - if err != nil { - t.Errorf("OffsetsForTimes() failed: %s\n", err) - return - } - - if len(offsets) != 1 { - t.Errorf("OffsetsForTimes() returned wrong length %d, expected 1\n", len(offsets)) - return - } - - if *offsets[0].Topic != testconf.Topic || offsets[0].Partition != 0 { - t.Errorf("OffsetsForTimes() returned wrong topic/partition\n") - return - } - - if offsets[0].Error != nil { - t.Errorf("OffsetsForTimes() returned error for partition 0: %s\n", err) - return - } - - low, _, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) - if err != nil { - t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) - return - } - - t.Logf("OffsetsForTimes() returned offset %d for timestamp %d\n", offsets[0].Offset, times[0].Offset) - - // Since we're using a phony low timestamp it is assumed that the returned - // offset will be oldest message. - if offsets[0].Offset != Offset(low) { - t.Errorf("OffsetsForTimes() returned invalid offset %d for timestamp %d, expected %d\n", offsets[0].Offset, times[0].Offset, low) - return - } - -} - -// test consumer GetMetadata API -func (its *IntegrationTestSuite) TestConsumerGetMetadata() { - t := its.T() - - config := &ConfigMap{"bootstrap.servers": testconf.Brokers, - "group.id": testconf.GroupID} - config.updateFromTestconf() - - // Create consumer - c, err := NewConsumer(config) - if err != nil { - t.Errorf("Failed to create consumer: %s\n", err) - return - } - defer c.Close() - - metaData, err := c.GetMetadata(&testconf.Topic, false, 5*1000) - if err != nil { - t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) - return - } - t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) - - metaData, err = c.GetMetadata(nil, true, 5*1000) - if err != nil { - t.Errorf("Failed to get meta data, Error: %s\n", err) - return - } - t.Logf("Meta data for consumer: %v\n", metaData) -} - -//Test producer QueryWatermarkOffsets API -func (its *IntegrationTestSuite) TestProducerQueryWatermarkOffsets() { - t := its.T() - - config := &ConfigMap{"bootstrap.servers": testconf.Brokers} - config.updateFromTestconf() - - // Create producer - p, err := NewProducer(config) - if err != nil { - t.Errorf("Failed to create producer: %s\n", err) - return - } - defer p.Close() - - low, high, err := p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) - if err != nil { - t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) - return - } - cnt := high - low - t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) - - createTestMessages() - producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) - - low, high, err = p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) - if err != nil { - t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) - return - } - t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) - newcnt := high - low - t.Logf("count = %d, New count = %d\n", cnt, newcnt) - if newcnt-cnt != int64(len(p0TestMsgs)) { - t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newcnt-cnt) - } -} - -//Test producer GetMetadata API -func (its *IntegrationTestSuite) TestProducerGetMetadata() { - t := its.T() - - config := &ConfigMap{"bootstrap.servers": testconf.Brokers} - config.updateFromTestconf() - - // Create producer - p, err := NewProducer(config) - if err != nil { - t.Errorf("Failed to create producer: %s\n", err) - return - } - defer p.Close() - - metaData, err := p.GetMetadata(&testconf.Topic, false, 5*1000) - if err != nil { - t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) - return - } - t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) - - metaData, err = p.GetMetadata(nil, true, 5*1000) - if err != nil { - t.Errorf("Failed to get meta data, Error: %s\n", err) - return - } - t.Logf("Meta data for producer: %v\n", metaData) - -} - -// test producer function-based API without delivery report -func (its *IntegrationTestSuite) TestProducerFunc() { - t := its.T() - producerTest(t, "Function producer (without DR)", - nil, producerCtrl{}, - func(p *Producer, m *Message, drChan chan Event) { - err := p.Produce(m, drChan) - if err != nil { - t.Errorf("Produce() failed: %v", err) - } - }) -} - -// test producer function-based API with delivery report -func (its *IntegrationTestSuite) TestProducerFuncDR() { - t := its.T() - producerTest(t, "Function producer (with DR)", - nil, producerCtrl{withDr: true}, - func(p *Producer, m *Message, drChan chan Event) { - err := p.Produce(m, drChan) - if err != nil { - t.Errorf("Produce() failed: %v", err) - } - }) -} - -// test producer with bad messages -func (its *IntegrationTestSuite) TestProducerWithBadMessages() { - t := its.T() - conf := ConfigMap{"bootstrap.servers": testconf.Brokers} - conf.updateFromTestconf() - - p, err := NewProducer(&conf) - if err != nil { - panic(err) - } - defer p.Close() - - // producing a nil message should return an error without crash - err = p.Produce(nil, p.Events()) - if err == nil { - t.Errorf("Producing a nil message should return error\n") - } else { - t.Logf("Producing a nil message returns expected error: %s\n", err) - } - - // producing a blank message (with nil Topic) should return an error without crash - err = p.Produce(&Message{}, p.Events()) - if err == nil { - t.Errorf("Producing a blank message should return error\n") - } else { - t.Logf("Producing a blank message returns expected error: %s\n", err) - } -} - -// test producer channel-based API without delivery report -func (its *IntegrationTestSuite) TestProducerChannel() { - t := its.T() - producerTest(t, "Channel producer (without DR)", - nil, producerCtrl{}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) -} - -// test producer channel-based API with delivery report -func (its *IntegrationTestSuite) TestProducerChannelDR() { - t := its.T() - producerTest(t, "Channel producer (with DR)", - nil, producerCtrl{withDr: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) - -} - -// test batch producer channel-based API without delivery report -func (its *IntegrationTestSuite) TestProducerBatchChannel() { - t := its.T() - producerTest(t, "Channel producer (without DR, batch channel)", - nil, producerCtrl{batchProducer: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) -} - -// test batch producer channel-based API with delivery report -func (its *IntegrationTestSuite) TestProducerBatchChannelDR() { - t := its.T() - producerTest(t, "Channel producer (DR, batch channel)", - nil, producerCtrl{withDr: true, batchProducer: true}, - func(p *Producer, m *Message, drChan chan Event) { - p.ProduceChannel() <- m - }) -} - -// test consumer channel-based API -func (its *IntegrationTestSuite) TestConsumerChannel() { - t := its.T() - consumerTestWithCommits(t, "Channel Consumer", - "", 0, true, eventTestChannelConsumer, nil) -} - -// test consumer channel-based API with incremental rebalancing -func (its *IntegrationTestSuite) TestConsumerChannelIncremental() { - t := its.T() - consumerTestWithCommits(t, "Channel Consumer Incremental", - "cooperative-sticky", 0, true, eventTestChannelConsumer, nil) -} - -// test consumer poll-based API -func (its *IntegrationTestSuite) TestConsumerPoll() { - t := its.T() - consumerTestWithCommits(t, "Poll Consumer", "", 0, false, eventTestPollConsumer, nil) -} - -// test consumer poll-based API with incremental rebalancing -func (its *IntegrationTestSuite) TestConsumerPollIncremental() { - t := its.T() - consumerTestWithCommits(t, "Poll Consumer ncremental", - "cooperative-sticky", 0, false, eventTestPollConsumer, nil) -} - -// test consumer poll-based API with rebalance callback -func (its *IntegrationTestSuite) TestConsumerPollRebalance() { - t := its.T() - consumerTestWithCommits(t, "Poll Consumer (rebalance callback)", - "", 0, false, eventTestPollConsumer, - func(c *Consumer, event Event) error { - t.Logf("Rebalanced: %s", event) - return nil - }) -} - -// test consumer poll-based API with incremental no-op rebalance callback -func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncrementalNoop() { - t := its.T() - consumerTestWithCommits(t, "Poll Consumer (incremental no-op rebalance callback)", - "cooperative-sticky", 0, false, eventTestPollConsumer, - func(c *Consumer, event Event) error { - t.Logf("Rebalanced: %s", event) - return nil - }) -} - -// test consumer poll-based API with incremental rebalance callback -func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncremental() { - t := its.T() - consumerTestWithCommits(t, "Poll Consumer (incremental rebalance callback)", - "cooperative-sticky", 0, false, eventTestPollConsumer, - func(c *Consumer, event Event) error { - t.Logf("Rebalanced: %s (RebalanceProtocol=%s, AssignmentLost=%v)", - event, c.GetRebalanceProtocol(), c.AssignmentLost()) - - switch e := event.(type) { - case AssignedPartitions: - err := c.IncrementalAssign(e.Partitions) - if err != nil { - t.Errorf("IncrementalAssign() failed: %s\n", err) - return err - } - case RevokedPartitions: - err := c.IncrementalUnassign(e.Partitions) - if err != nil { - t.Errorf("IncrementalUnassign() failed: %s\n", err) - return err - } - default: - t.Fatalf("Unexpected rebalance event: %v\n", e) - } - - return nil - }) -} - -// Test Committed() API -func (its *IntegrationTestSuite) TestConsumerCommitted() { - t := its.T() - if testconf.Semaphore { - t.Skipf("Skipping TestConsumerCommitted since it is flaky[Does not run when tested with all the other integration tests]") - return - } - - consumerTestWithCommits(t, "Poll Consumer (rebalance callback, verify Committed())", - "", 0, false, eventTestPollConsumer, - func(c *Consumer, event Event) error { - t.Logf("Rebalanced: %s", event) - rp, ok := event.(RevokedPartitions) - if ok { - offsets, err := c.Committed(rp.Partitions, 5000) - if err != nil { - t.Errorf("Failed to get committed offsets: %s\n", err) - return nil - } - - t.Logf("Retrieved Committed offsets: %s\n", offsets) - - if len(offsets) != len(rp.Partitions) || len(rp.Partitions) == 0 { - t.Errorf("Invalid number of partitions %d, should be %d (and >0)\n", len(offsets), len(rp.Partitions)) - } - - // Verify proper offsets: at least one partition needs - // to have a committed offset. - validCnt := 0 - for _, p := range offsets { - if p.Error != nil { - t.Errorf("Committed() partition error: %v: %v", p, p.Error) - } else if p.Offset >= 0 { - validCnt++ - } - } - - if validCnt == 0 { - t.Errorf("Committed(): no partitions with valid offsets: %v", offsets) - } - } - return nil - }) -} - -// TestProducerConsumerTimestamps produces messages with timestamps -// and verifies them on consumption. -// Requires librdkafka >=0.9.4 and Kafka >=0.10.0.0 -func (its *IntegrationTestSuite) TestProducerConsumerTimestamps() { - t := its.T() - numver, strver := LibraryVersion() - if numver < 0x00090400 { - t.Skipf("Requires librdkafka >=0.9.4 (currently on %s)", strver) - } - - consumerConf := ConfigMap{"bootstrap.servers": testconf.Brokers, - "go.events.channel.enable": true, - "group.id": testconf.Topic, - "enable.partition.eof": true, - } - - consumerConf.updateFromTestconf() - - /* Create consumer and find recognizable message, verify timestamp. - * The consumer is started before the producer to make sure - * the message isn't missed. */ - t.Logf("Creating consumer") - c, err := NewConsumer(&consumerConf) - if err != nil { - t.Fatalf("NewConsumer: %v", err) - } - - t.Logf("Assign %s [0]", testconf.Topic) - err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, - Offset: OffsetEnd}}) - if err != nil { - t.Fatalf("Assign: %v", err) - } - - /* Wait until EOF is reached so we dont miss the produced message */ - for ev := range c.Events() { - t.Logf("Awaiting initial EOF") - _, ok := ev.(PartitionEOF) - if ok { - break - } - } - - /* - * Create producer and produce one recognizable message with timestamp - */ - producerConf := ConfigMap{"bootstrap.servers": testconf.Brokers} - producerConf.updateFromTestconf() - - t.Logf("Creating producer") - p, err := NewProducer(&producerConf) - if err != nil { - t.Fatalf("NewProducer: %v", err) - } - - drChan := make(chan Event, 1) - - /* Offset the timestamp to avoid comparison with system clock */ - future, _ := time.ParseDuration("87658h") // 10y - timestamp := time.Now().Add(future) - key := fmt.Sprintf("TS: %v", timestamp) - t.Logf("Producing message with timestamp %v", timestamp) - err = p.Produce(&Message{ - TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, - Key: []byte(key), - Timestamp: timestamp}, - drChan) - - if err != nil { - t.Fatalf("Produce: %v", err) - } - - // Wait for delivery - t.Logf("Awaiting delivery report") - ev := <-drChan - m, ok := ev.(*Message) - if !ok { - t.Fatalf("drChan: Expected *Message, got %v", ev) - } - if m.TopicPartition.Error != nil { - t.Fatalf("Delivery failed: %v", m.TopicPartition) - } - t.Logf("Produced message to %v", m.TopicPartition) - producedOffset := m.TopicPartition.Offset - - p.Close() - - /* Now consume messages, waiting for that recognizable one. */ - t.Logf("Consuming messages") -outer: - for ev := range c.Events() { - switch m := ev.(type) { - case *Message: - if m.TopicPartition.Error != nil { - continue - } - if m.Key == nil || string(m.Key) != key { - continue - } - - t.Logf("Found message at %v with timestamp %s %s", - m.TopicPartition, - m.TimestampType, m.Timestamp) - - if m.TopicPartition.Offset != producedOffset { - t.Fatalf("Produced Offset %d does not match consumed offset %d", producedOffset, m.TopicPartition.Offset) - } - - if m.TimestampType != TimestampCreateTime { - t.Fatalf("Expected timestamp CreateTime, not %s", - m.TimestampType) - } - - /* Since Kafka timestamps are milliseconds we need to - * shave off some precision for the comparison */ - if m.Timestamp.UnixNano()/1000000 != - timestamp.UnixNano()/1000000 { - t.Fatalf("Expected timestamp %v (%d), not %v (%d)", - timestamp, timestamp.UnixNano(), - m.Timestamp, m.Timestamp.UnixNano()) - } - break outer - default: - } - } - - c.Close() -} - -// TestProducerConsumerHeaders produces messages with headers -// and verifies them on consumption. -// Requires librdkafka >=0.11.4 and Kafka >=0.11.0.0 -func (its *IntegrationTestSuite) TestProducerConsumerHeaders() { - t := its.T() - numver, strver := LibraryVersion() - if numver < 0x000b0400 { - t.Skipf("Requires librdkafka >=0.11.4 (currently on %s, 0x%x)", strver, numver) - } - - conf := ConfigMap{"bootstrap.servers": testconf.Brokers, - "api.version.request": true, - "enable.auto.commit": false, - "group.id": testconf.Topic, - } - - conf.updateFromTestconf() - - /* - * Create producer and produce a couple of messages with and without - * headers. - */ - t.Logf("Creating producer") - p, err := NewProducer(&conf) - if err != nil { - t.Fatalf("NewProducer: %v", err) - } - - drChan := make(chan Event, 1) - - // prepare some header values - bigBytes := make([]byte, 2500) - for i := 0; i < len(bigBytes); i++ { - bigBytes[i] = byte(i) - } - - myVarint := make([]byte, binary.MaxVarintLen64) - myVarintLen := binary.PutVarint(myVarint, 12345678901234) - - expMsgHeaders := [][]Header{ - { - {"msgid", []byte("1")}, - {"a key with SPACES ", bigBytes[:15]}, - {"BIGONE!", bigBytes}, - }, - { - {"msgid", []byte("2")}, - {"myVarint", myVarint[:myVarintLen]}, - {"empty", []byte("")}, - {"theNullIsNil", nil}, - }, - nil, // no headers - { - {"msgid", []byte("4")}, - {"order", []byte("1")}, - {"order", []byte("2")}, - {"order", nil}, - {"order", []byte("4")}, - }, - } - - t.Logf("Producing %d messages", len(expMsgHeaders)) - for _, hdrs := range expMsgHeaders { - err = p.Produce(&Message{ - TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, - Headers: hdrs}, - drChan) - } - - if err != nil { - t.Fatalf("Produce: %v", err) - } - - var firstOffset Offset = OffsetInvalid - for range expMsgHeaders { - ev := <-drChan - m, ok := ev.(*Message) - if !ok { - t.Fatalf("drChan: Expected *Message, got %v", ev) - } - if m.TopicPartition.Error != nil { - t.Fatalf("Delivery failed: %v", m.TopicPartition) - } - t.Logf("Produced message to %v", m.TopicPartition) - if firstOffset == OffsetInvalid { - firstOffset = m.TopicPartition.Offset - } - } - - p.Close() - - /* Now consume the produced messages and verify the headers */ - t.Logf("Creating consumer starting at offset %v", firstOffset) - c, err := NewConsumer(&conf) - if err != nil { - t.Fatalf("NewConsumer: %v", err) - } - - err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, - Offset: firstOffset}}) - if err != nil { - t.Fatalf("Assign: %v", err) - } - - for n, hdrs := range expMsgHeaders { - m, err := c.ReadMessage(-1) - if err != nil { - t.Fatalf("Expected message #%d, not error %v", n, err) - } - - if m.Headers == nil { - if hdrs == nil { - continue - } - t.Fatalf("Expected message #%d to have headers", n) - } - - if hdrs == nil { - t.Fatalf("Expected message #%d not to have headers, but found %v", n, m.Headers) - } - - // Compare headers - if !reflect.DeepEqual(hdrs, m.Headers) { - t.Fatalf("Expected message #%d headers to match %v, but found %v", n, hdrs, m.Headers) - } - - t.Logf("Message #%d headers matched: %v", n, m.Headers) - } - - c.Close() - -} +// // Test consumer QueryWatermarkOffsets API +// func (its *IntegrationTestSuite) TestConsumerQueryWatermarkOffsets() { +// t := its.T() + +// // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to +// // get the number of messages in a topic +// msgcnt, err := getMessageCountInTopic(testconf.Topic) +// if err != nil { +// t.Errorf("Cannot get message size. Error: %s\n", err) +// } + +// // Prime topic with test messages +// createTestMessages() +// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) + +// // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to +// // get the number of messages in a topic +// newmsgcnt, err := getMessageCountInTopic(testconf.Topic) +// if err != nil { +// t.Errorf("Cannot get message size. Error: %s\n", err) +// } + +// if newmsgcnt-msgcnt != len(p0TestMsgs) { +// t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newmsgcnt-msgcnt) +// } + +// } + +// // Test consumer GetWatermarkOffsets API +// func (its *IntegrationTestSuite) TestConsumerGetWatermarkOffsets() { +// t := its.T() + +// // Create consumer +// config := &ConfigMap{ +// "go.events.channel.enable": true, +// "bootstrap.servers": testconf.Brokers, +// "group.id": testconf.GroupID, +// "session.timeout.ms": 6000, +// "enable.auto.commit": false, +// "auto.offset.reset": "earliest", +// } +// _ = config.updateFromTestconf() + +// c, err := NewConsumer(config) +// if err != nil { +// t.Fatalf("Unable to create consumer: %s", err) +// } +// defer func() { _ = c.Close() }() + +// err = c.Subscribe(testconf.Topic, nil) + +// // Prime topic with test messages +// createTestMessages() +// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) + +// // Wait for messages to be received so that we know the watermark offsets have been delivered +// // with the fetch response +// for ev := range c.Events() { +// if _, ok := ev.(*Message); ok { +// break +// } +// } + +// _, queryHigh, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) +// if err != nil { +// t.Fatalf("Error querying watermark offsets: %s", err) +// } + +// // We are not currently testing the low watermark offset as it only gets set every 10s by the stits timer +// _, getHigh, err := c.GetWatermarkOffsets(testconf.Topic, 0) +// if err != nil { +// t.Fatalf("Error getting watermark offsets: %s", err) +// } + +// if queryHigh != getHigh { +// t.Errorf("QueryWatermarkOffsets high[%d] does not equal GetWatermarkOffsets high[%d]", queryHigh, getHigh) +// } + +// } + +// // TestConsumerOffsetsForTimes +// func (its *IntegrationTestSuite) TestConsumerOffsetsForTimes() { +// t := its.T() + +// conf := ConfigMap{"bootstrap.servers": testconf.Brokers, +// "group.id": testconf.GroupID, +// "api.version.request": true} + +// conf.updateFromTestconf() + +// c, err := NewConsumer(&conf) + +// if err != nil { +// panic(err) +// } +// defer c.Close() + +// // Prime topic with test messages +// createTestMessages() +// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) + +// times := make([]TopicPartition, 1) +// times[0] = TopicPartition{Topic: &testconf.Topic, Partition: 0, Offset: 12345} +// offsets, err := c.OffsetsForTimes(times, 5000) +// if err != nil { +// t.Errorf("OffsetsForTimes() failed: %s\n", err) +// return +// } + +// if len(offsets) != 1 { +// t.Errorf("OffsetsForTimes() returned wrong length %d, expected 1\n", len(offsets)) +// return +// } + +// if *offsets[0].Topic != testconf.Topic || offsets[0].Partition != 0 { +// t.Errorf("OffsetsForTimes() returned wrong topic/partition\n") +// return +// } + +// if offsets[0].Error != nil { +// t.Errorf("OffsetsForTimes() returned error for partition 0: %s\n", err) +// return +// } + +// low, _, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) +// if err != nil { +// t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) +// return +// } + +// t.Logf("OffsetsForTimes() returned offset %d for timestamp %d\n", offsets[0].Offset, times[0].Offset) + +// // Since we're using a phony low timestamp it is assumed that the returned +// // offset will be oldest message. +// if offsets[0].Offset != Offset(low) { +// t.Errorf("OffsetsForTimes() returned invalid offset %d for timestamp %d, expected %d\n", offsets[0].Offset, times[0].Offset, low) +// return +// } + +// } + +// // test consumer GetMetadata API +// func (its *IntegrationTestSuite) TestConsumerGetMetadata() { +// t := its.T() + +// config := &ConfigMap{"bootstrap.servers": testconf.Brokers, +// "group.id": testconf.GroupID} +// config.updateFromTestconf() + +// // Create consumer +// c, err := NewConsumer(config) +// if err != nil { +// t.Errorf("Failed to create consumer: %s\n", err) +// return +// } +// defer c.Close() + +// metaData, err := c.GetMetadata(&testconf.Topic, false, 5*1000) +// if err != nil { +// t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) +// return +// } +// t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) + +// metaData, err = c.GetMetadata(nil, true, 5*1000) +// if err != nil { +// t.Errorf("Failed to get meta data, Error: %s\n", err) +// return +// } +// t.Logf("Meta data for consumer: %v\n", metaData) +// } + +// // Test producer QueryWatermarkOffsets API +// func (its *IntegrationTestSuite) TestProducerQueryWatermarkOffsets() { +// t := its.T() + +// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} +// config.updateFromTestconf() + +// // Create producer +// p, err := NewProducer(config) +// if err != nil { +// t.Errorf("Failed to create producer: %s\n", err) +// return +// } +// defer p.Close() + +// low, high, err := p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) +// if err != nil { +// t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) +// return +// } +// cnt := high - low +// t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) + +// createTestMessages() +// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) + +// low, high, err = p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) +// if err != nil { +// t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) +// return +// } +// t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) +// newcnt := high - low +// t.Logf("count = %d, New count = %d\n", cnt, newcnt) +// if newcnt-cnt != int64(len(p0TestMsgs)) { +// t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newcnt-cnt) +// } +// } + +// // Test producer GetMetadata API +// func (its *IntegrationTestSuite) TestProducerGetMetadata() { +// t := its.T() + +// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} +// config.updateFromTestconf() + +// // Create producer +// p, err := NewProducer(config) +// if err != nil { +// t.Errorf("Failed to create producer: %s\n", err) +// return +// } +// defer p.Close() + +// metaData, err := p.GetMetadata(&testconf.Topic, false, 5*1000) +// if err != nil { +// t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) +// return +// } +// t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) + +// metaData, err = p.GetMetadata(nil, true, 5*1000) +// if err != nil { +// t.Errorf("Failed to get meta data, Error: %s\n", err) +// return +// } +// t.Logf("Meta data for producer: %v\n", metaData) + +// } + +// // test producer function-based API without delivery report +// func (its *IntegrationTestSuite) TestProducerFunc() { +// t := its.T() +// producerTest(t, "Function producer (without DR)", +// nil, producerCtrl{}, +// func(p *Producer, m *Message, drChan chan Event) { +// err := p.Produce(m, drChan) +// if err != nil { +// t.Errorf("Produce() failed: %v", err) +// } +// }) +// } + +// // test producer function-based API with delivery report +// func (its *IntegrationTestSuite) TestProducerFuncDR() { +// t := its.T() +// producerTest(t, "Function producer (with DR)", +// nil, producerCtrl{withDr: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// err := p.Produce(m, drChan) +// if err != nil { +// t.Errorf("Produce() failed: %v", err) +// } +// }) +// } + +// // test producer with bad messages +// func (its *IntegrationTestSuite) TestProducerWithBadMessages() { +// t := its.T() +// conf := ConfigMap{"bootstrap.servers": testconf.Brokers} +// conf.updateFromTestconf() + +// p, err := NewProducer(&conf) +// if err != nil { +// panic(err) +// } +// defer p.Close() + +// // producing a nil message should return an error without crash +// err = p.Produce(nil, p.Events()) +// if err == nil { +// t.Errorf("Producing a nil message should return error\n") +// } else { +// t.Logf("Producing a nil message returns expected error: %s\n", err) +// } + +// // producing a blank message (with nil Topic) should return an error without crash +// err = p.Produce(&Message{}, p.Events()) +// if err == nil { +// t.Errorf("Producing a blank message should return error\n") +// } else { +// t.Logf("Producing a blank message returns expected error: %s\n", err) +// } +// } + +// // test producer channel-based API without delivery report +// func (its *IntegrationTestSuite) TestProducerChannel() { +// t := its.T() +// producerTest(t, "Channel producer (without DR)", +// nil, producerCtrl{}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) +// } + +// // test producer channel-based API with delivery report +// func (its *IntegrationTestSuite) TestProducerChannelDR() { +// t := its.T() +// producerTest(t, "Channel producer (with DR)", +// nil, producerCtrl{withDr: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) + +// } + +// // test batch producer channel-based API without delivery report +// func (its *IntegrationTestSuite) TestProducerBatchChannel() { +// t := its.T() +// producerTest(t, "Channel producer (without DR, batch channel)", +// nil, producerCtrl{batchProducer: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) +// } + +// // test batch producer channel-based API with delivery report +// func (its *IntegrationTestSuite) TestProducerBatchChannelDR() { +// t := its.T() +// producerTest(t, "Channel producer (DR, batch channel)", +// nil, producerCtrl{withDr: true, batchProducer: true}, +// func(p *Producer, m *Message, drChan chan Event) { +// p.ProduceChannel() <- m +// }) +// } + +// // test consumer channel-based API +// func (its *IntegrationTestSuite) TestConsumerChannel() { +// t := its.T() +// consumerTestWithCommits(t, "Channel Consumer", +// "", 0, true, eventTestChannelConsumer, nil) +// } + +// // test consumer channel-based API with incremental rebalancing +// func (its *IntegrationTestSuite) TestConsumerChannelIncremental() { +// t := its.T() +// consumerTestWithCommits(t, "Channel Consumer Incremental", +// "cooperative-sticky", 0, true, eventTestChannelConsumer, nil) +// } + +// // test consumer poll-based API +// func (its *IntegrationTestSuite) TestConsumerPoll() { +// t := its.T() +// consumerTestWithCommits(t, "Poll Consumer", "", 0, false, eventTestPollConsumer, nil) +// } + +// // test consumer poll-based API with incremental rebalancing +// func (its *IntegrationTestSuite) TestConsumerPollIncremental() { +// t := its.T() +// consumerTestWithCommits(t, "Poll Consumer ncremental", +// "cooperative-sticky", 0, false, eventTestPollConsumer, nil) +// } + +// // test consumer poll-based API with rebalance callback +// func (its *IntegrationTestSuite) TestConsumerPollRebalance() { +// t := its.T() +// consumerTestWithCommits(t, "Poll Consumer (rebalance callback)", +// "", 0, false, eventTestPollConsumer, +// func(c *Consumer, event Event) error { +// t.Logf("Rebalanced: %s", event) +// return nil +// }) +// } + +// // test consumer poll-based API with incremental no-op rebalance callback +// func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncrementalNoop() { +// t := its.T() +// consumerTestWithCommits(t, "Poll Consumer (incremental no-op rebalance callback)", +// "cooperative-sticky", 0, false, eventTestPollConsumer, +// func(c *Consumer, event Event) error { +// t.Logf("Rebalanced: %s", event) +// return nil +// }) +// } + +// // test consumer poll-based API with incremental rebalance callback +// func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncremental() { +// t := its.T() +// consumerTestWithCommits(t, "Poll Consumer (incremental rebalance callback)", +// "cooperative-sticky", 0, false, eventTestPollConsumer, +// func(c *Consumer, event Event) error { +// t.Logf("Rebalanced: %s (RebalanceProtocol=%s, AssignmentLost=%v)", +// event, c.GetRebalanceProtocol(), c.AssignmentLost()) + +// switch e := event.(type) { +// case AssignedPartitions: +// err := c.IncrementalAssign(e.Partitions) +// if err != nil { +// t.Errorf("IncrementalAssign() failed: %s\n", err) +// return err +// } +// case RevokedPartitions: +// err := c.IncrementalUnassign(e.Partitions) +// if err != nil { +// t.Errorf("IncrementalUnassign() failed: %s\n", err) +// return err +// } +// default: +// t.Fatalf("Unexpected rebalance event: %v\n", e) +// } + +// return nil +// }) +// } + +// // Test Committed() API +// func (its *IntegrationTestSuite) TestConsumerCommitted() { +// t := its.T() +// if testconf.Semaphore { +// t.Skipf("Skipping TestConsumerCommitted since it is flaky[Does not run when tested with all the other integration tests]") +// return +// } + +// consumerTestWithCommits(t, "Poll Consumer (rebalance callback, verify Committed())", +// "", 0, false, eventTestPollConsumer, +// func(c *Consumer, event Event) error { +// t.Logf("Rebalanced: %s", event) +// rp, ok := event.(RevokedPartitions) +// if ok { +// offsets, err := c.Committed(rp.Partitions, 5000) +// if err != nil { +// t.Errorf("Failed to get committed offsets: %s\n", err) +// return nil +// } + +// t.Logf("Retrieved Committed offsets: %s\n", offsets) + +// if len(offsets) != len(rp.Partitions) || len(rp.Partitions) == 0 { +// t.Errorf("Invalid number of partitions %d, should be %d (and >0)\n", len(offsets), len(rp.Partitions)) +// } + +// // Verify proper offsets: at least one partition needs +// // to have a committed offset. +// validCnt := 0 +// for _, p := range offsets { +// if p.Error != nil { +// t.Errorf("Committed() partition error: %v: %v", p, p.Error) +// } else if p.Offset >= 0 { +// validCnt++ +// } +// } + +// if validCnt == 0 { +// t.Errorf("Committed(): no partitions with valid offsets: %v", offsets) +// } +// } +// return nil +// }) +// } + +// // TestProducerConsumerTimestamps produces messages with timestamps +// // and verifies them on consumption. +// // Requires librdkafka >=0.9.4 and Kafka >=0.10.0.0 +// func (its *IntegrationTestSuite) TestProducerConsumerTimestamps() { +// t := its.T() +// numver, strver := LibraryVersion() +// if numver < 0x00090400 { +// t.Skipf("Requires librdkafka >=0.9.4 (currently on %s)", strver) +// } + +// consumerConf := ConfigMap{"bootstrap.servers": testconf.Brokers, +// "go.events.channel.enable": true, +// "group.id": testconf.Topic, +// "enable.partition.eof": true, +// } + +// consumerConf.updateFromTestconf() + +// /* Create consumer and find recognizable message, verify timestamp. +// * The consumer is started before the producer to make sure +// * the message isn't missed. */ +// t.Logf("Creating consumer") +// c, err := NewConsumer(&consumerConf) +// if err != nil { +// t.Fatalf("NewConsumer: %v", err) +// } + +// t.Logf("Assign %s [0]", testconf.Topic) +// err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, +// Offset: OffsetEnd}}) +// if err != nil { +// t.Fatalf("Assign: %v", err) +// } + +// /* Wait until EOF is reached so we dont miss the produced message */ +// for ev := range c.Events() { +// t.Logf("Awaiting initial EOF") +// _, ok := ev.(PartitionEOF) +// if ok { +// break +// } +// } + +// /* +// * Create producer and produce one recognizable message with timestamp +// */ +// producerConf := ConfigMap{"bootstrap.servers": testconf.Brokers} +// producerConf.updateFromTestconf() + +// t.Logf("Creating producer") +// p, err := NewProducer(&producerConf) +// if err != nil { +// t.Fatalf("NewProducer: %v", err) +// } + +// drChan := make(chan Event, 1) + +// /* Offset the timestamp to avoid comparison with system clock */ +// future, _ := time.ParseDuration("87658h") // 10y +// timestamp := time.Now().Add(future) +// key := fmt.Sprintf("TS: %v", timestamp) +// t.Logf("Producing message with timestamp %v", timestamp) +// err = p.Produce(&Message{ +// TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, +// Key: []byte(key), +// Timestamp: timestamp}, +// drChan) + +// if err != nil { +// t.Fatalf("Produce: %v", err) +// } + +// // Wait for delivery +// t.Logf("Awaiting delivery report") +// ev := <-drChan +// m, ok := ev.(*Message) +// if !ok { +// t.Fatalf("drChan: Expected *Message, got %v", ev) +// } +// if m.TopicPartition.Error != nil { +// t.Fatalf("Delivery failed: %v", m.TopicPartition) +// } +// t.Logf("Produced message to %v", m.TopicPartition) +// producedOffset := m.TopicPartition.Offset + +// p.Close() + +// /* Now consume messages, waiting for that recognizable one. */ +// t.Logf("Consuming messages") +// outer: +// for ev := range c.Events() { +// switch m := ev.(type) { +// case *Message: +// if m.TopicPartition.Error != nil { +// continue +// } +// if m.Key == nil || string(m.Key) != key { +// continue +// } + +// t.Logf("Found message at %v with timestamp %s %s", +// m.TopicPartition, +// m.TimestampType, m.Timestamp) + +// if m.TopicPartition.Offset != producedOffset { +// t.Fatalf("Produced Offset %d does not match consumed offset %d", producedOffset, m.TopicPartition.Offset) +// } + +// if m.TimestampType != TimestampCreateTime { +// t.Fatalf("Expected timestamp CreateTime, not %s", +// m.TimestampType) +// } + +// /* Since Kafka timestamps are milliseconds we need to +// * shave off some precision for the comparison */ +// if m.Timestamp.UnixNano()/1000000 != +// timestamp.UnixNano()/1000000 { +// t.Fatalf("Expected timestamp %v (%d), not %v (%d)", +// timestamp, timestamp.UnixNano(), +// m.Timestamp, m.Timestamp.UnixNano()) +// } +// break outer +// default: +// } +// } + +// c.Close() +// } + +// // TestProducerConsumerHeaders produces messages with headers +// // and verifies them on consumption. +// // Requires librdkafka >=0.11.4 and Kafka >=0.11.0.0 +// func (its *IntegrationTestSuite) TestProducerConsumerHeaders() { +// t := its.T() +// numver, strver := LibraryVersion() +// if numver < 0x000b0400 { +// t.Skipf("Requires librdkafka >=0.11.4 (currently on %s, 0x%x)", strver, numver) +// } + +// conf := ConfigMap{"bootstrap.servers": testconf.Brokers, +// "api.version.request": true, +// "enable.auto.commit": false, +// "group.id": testconf.Topic, +// } + +// conf.updateFromTestconf() + +// /* +// * Create producer and produce a couple of messages with and without +// * headers. +// */ +// t.Logf("Creating producer") +// p, err := NewProducer(&conf) +// if err != nil { +// t.Fatalf("NewProducer: %v", err) +// } + +// drChan := make(chan Event, 1) + +// // prepare some header values +// bigBytes := make([]byte, 2500) +// for i := 0; i < len(bigBytes); i++ { +// bigBytes[i] = byte(i) +// } + +// myVarint := make([]byte, binary.MaxVarintLen64) +// myVarintLen := binary.PutVarint(myVarint, 12345678901234) + +// expMsgHeaders := [][]Header{ +// { +// {"msgid", []byte("1")}, +// {"a key with SPACES ", bigBytes[:15]}, +// {"BIGONE!", bigBytes}, +// }, +// { +// {"msgid", []byte("2")}, +// {"myVarint", myVarint[:myVarintLen]}, +// {"empty", []byte("")}, +// {"theNullIsNil", nil}, +// }, +// nil, // no headers +// { +// {"msgid", []byte("4")}, +// {"order", []byte("1")}, +// {"order", []byte("2")}, +// {"order", nil}, +// {"order", []byte("4")}, +// }, +// } + +// t.Logf("Producing %d messages", len(expMsgHeaders)) +// for _, hdrs := range expMsgHeaders { +// err = p.Produce(&Message{ +// TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, +// Headers: hdrs}, +// drChan) +// } + +// if err != nil { +// t.Fatalf("Produce: %v", err) +// } + +// var firstOffset Offset = OffsetInvalid +// for range expMsgHeaders { +// ev := <-drChan +// m, ok := ev.(*Message) +// if !ok { +// t.Fatalf("drChan: Expected *Message, got %v", ev) +// } +// if m.TopicPartition.Error != nil { +// t.Fatalf("Delivery failed: %v", m.TopicPartition) +// } +// t.Logf("Produced message to %v", m.TopicPartition) +// if firstOffset == OffsetInvalid { +// firstOffset = m.TopicPartition.Offset +// } +// } + +// p.Close() + +// /* Now consume the produced messages and verify the headers */ +// t.Logf("Creating consumer starting at offset %v", firstOffset) +// c, err := NewConsumer(&conf) +// if err != nil { +// t.Fatalf("NewConsumer: %v", err) +// } + +// err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, +// Offset: firstOffset}}) +// if err != nil { +// t.Fatalf("Assign: %v", err) +// } + +// for n, hdrs := range expMsgHeaders { +// m, err := c.ReadMessage(-1) +// if err != nil { +// t.Fatalf("Expected message #%d, not error %v", n, err) +// } + +// if m.Headers == nil { +// if hdrs == nil { +// continue +// } +// t.Fatalf("Expected message #%d to have headers", n) +// } + +// if hdrs == nil { +// t.Fatalf("Expected message #%d not to have headers, but found %v", n, m.Headers) +// } + +// // Compare headers +// if !reflect.DeepEqual(hdrs, m.Headers) { +// t.Fatalf("Expected message #%d headers to match %v, but found %v", n, hdrs, m.Headers) +// } + +// t.Logf("Message #%d headers matched: %v", n, m.Headers) +// } + +// c.Close() + +// } func TestIntegration(t *testing.T) { its := new(IntegrationTestSuite) diff --git a/kafka/librdkafka_vendor/rdkafka.h b/kafka/librdkafka_vendor/rdkafka.h index d77216f0e..bd22401b0 100644 --- a/kafka/librdkafka_vendor/rdkafka.h +++ b/kafka/librdkafka_vendor/rdkafka.h @@ -5282,6 +5282,10 @@ typedef int rd_kafka_event_type_t; #define RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT 0x8000 /** AlterConsumerGroupOffsets_result_t */ #define RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT 0x10000 +/** DescribeTopics_result_t */ +#define RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT 0x20000 +/** DescribeCluster_result_t */ +#define RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT 0x40000 /** @@ -5437,6 +5441,8 @@ int rd_kafka_event_error_is_fatal(rd_kafka_event_t *rkev); * - RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT * - RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT * - RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT + * - RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + * - RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT */ RD_EXPORT void *rd_kafka_event_opaque(rd_kafka_event_t *rkev); @@ -5548,7 +5554,10 @@ typedef rd_kafka_event_t rd_kafka_DeleteConsumerGroupOffsets_result_t; typedef rd_kafka_event_t rd_kafka_AlterConsumerGroupOffsets_result_t; /*! ListConsumerGroupOffsets result type */ typedef rd_kafka_event_t rd_kafka_ListConsumerGroupOffsets_result_t; - +/*! DescribeTopics result type */ +typedef rd_kafka_event_t rd_kafka_DescribeTopics_result_t; +/*! DescribeCluster result type */ +typedef rd_kafka_event_t rd_kafka_DescribeCluster_result_t; /** * @brief Get CreateTopics result. * @@ -5649,6 +5658,36 @@ rd_kafka_event_ListConsumerGroups_result(rd_kafka_event_t *rkev); RD_EXPORT const rd_kafka_DescribeConsumerGroups_result_t * rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_t *rkev); +/** + * @brief Get DescribeTopics result. + * + * @returns the result of a DescribeTopics request, or NULL if event is + * of different type. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p rkev object. + * + * Event types: + * RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ +RD_EXPORT const rd_kafka_DescribeTopics_result_t * +rd_kafka_event_DescribeTopics_result(rd_kafka_event_t *rkev); + +/** + * @brief Get DescribeCluster result. + * + * @returns the result of a DescribeCluster request, or NULL if event is + * of different type. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p rkev object. + * + * Event types: + * RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT + */ +RD_EXPORT const rd_kafka_DescribeCluster_result_t * +rd_kafka_event_DescribeCluster_result(rd_kafka_event_t *rkev); + /** * @brief Get DeleteGroups result. * @@ -6635,6 +6674,10 @@ typedef enum rd_kafka_admin_op_t { RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPOFFSETS, /** AlterConsumerGroupOffsets */ RD_KAFKA_ADMIN_OP_ALTERCONSUMERGROUPOFFSETS, + /**< DescribeTopics */ + RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, + /**< DescribeCluster */ + RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */ } rd_kafka_admin_op_t; @@ -6813,6 +6856,39 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_require_stable_offsets( rd_kafka_AdminOptions_t *options, int true_or_false); +/** + * @brief Whether broker should return authorized operations + * (DescribeConsumerGroups). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeConsumerGroups. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); + +/** + * @brief Whether broker should return topic authorized operations + * (DescribeTopic). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeTopic. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_topic_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); /** * @brief Set consumer groups states to query for. * @@ -7868,6 +7944,29 @@ RD_EXPORT const char *rd_kafka_ConsumerGroupDescription_partition_assignor( const rd_kafka_ConsumerGroupDescription_t *grpdesc); +/** + * @brief Gets count of authorized operations for the \p grpdesc group. + * + * @param grpdesc The group description. + * + * @return count of Authorized operations allowed, 0 if authorized operations list is NULL or empty. + */ +RD_EXPORT +size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( + const rd_kafka_ConsumerGroupDescription_t *grpdesc); + +/** + * @brief Gets operation at idx index of authorized operations for the \p grpdesc group. + * + * @param grpdesc The group description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +int rd_kafka_ConsumerGroupDescription_authorized_operation( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t idx); /** * @brief Gets state for the \p grpdesc group. @@ -8008,6 +8107,383 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( /**@}*/ +/**@}*/ + +/** + * @name Admin API - DescribeTopic + * @{ + */ +/** + * @brief Describe a topic and the authorized acl operations + * + * @param rk Client instance. + * @param topics Topic names + * @param topics_cnt Count of topics sent in topic names. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ + +/** + * @brief DescribeTopics result type. + * + */ +typedef struct rd_kafka_TopicDescription_s rd_kafka_TopicDescription_t; + +/** + * @brief Gets the partiton id for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton id. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_id( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton leader for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton leader. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_leader( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton in-sync replica count for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton replica count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_isr_cnt( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton replica count for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton replica count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_replica_cnt( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton in-sync replica at isr index + * for partition at partition index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param partition_idx Index for the partitions. + * @param isr_idx Index for the in-sync replica. + * + * @return The partiton in-sync replica. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_isrs_idx( + const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int isr_idx); + +/** + * @brief Gets the partiton replica at replica index + * for partition at partition index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param partition_idx Index for the partitions. + * @param replica_idx Index for the in-sync replica. + * + * @return The partiton replica. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_replica_idx( + const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int replica_idx); + +/** + * @brief Gets the topic partiton count for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic partiton count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_topic_partition_cnt( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the partiton error for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton error. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_error_t *rd_kafka_TopicDescription_partition_error( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets operation at idx index of topic authorized operations for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_authorized_operation_idx( + const rd_kafka_TopicDescription_t *topicdesc, + size_t idx); + +/** + * @brief Gets the topic authorized acl operations count for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic authorized operations count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_topic_authorized_operations_cnt( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the topic name for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic name. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const char *rd_kafka_TopicDescription_topic_name( + const rd_kafka_TopicDescription_t *topicdesc) ; + +/** + * @brief Gets the error for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic description error. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_error_t *rd_kafka_TopicDescription_error( + const rd_kafka_TopicDescription_t *topicdesc) ; +/** + * @brief Get an array of topic results from a DescribeTopics result. + * + * The returned topics life-time is the same as the \p result object. + * + * @param result Result to get topics results from. + * @param cntp is updated to the number of elements in the array. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_TopicDescription_t ** +rd_kafka_DescribeTopics_result_topics( + const rd_kafka_DescribeTopics_result_t *result, + size_t *cntp); +/** + * @brief Describe topics as specified by the \p topics + * array of size \p topics_cnt elements. + * + * @param rk Client instance. + * @param topics Array of topics to describe. + * @param topics_cnt Number of elements in \p topics array. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ +RD_EXPORT +void rd_kafka_DescribeTopics(rd_kafka_t *rk, + const char** topics, + size_t topics_cnt, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); +/**@}*/ + +/** + * @name Admin API - DescribeCluster + * @{ + */ + +typedef struct rd_kafka_ClusterDescription_s rd_kafka_ClusterDescription_t; + +/** + * @brief Gets the node for the \p clusterdesc cluster at idx position. + * + * @param clusterdesc The cluster description. + * + * @return The node at idx position. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const rd_kafka_Node_t *rd_kafka_ClusterDescription_node_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, int idx); + +/** + * @brief Gets operation at idx index of cluster authorized operations for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_authorized_operation_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + size_t idx); + +/** + * @brief Gets the cluster authorized acl operations for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster authorized operations. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the cluster current controller id for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster current controller id. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_controller_id( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the cluster current cluster id for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster current cluster id (char*). + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const char *rd_kafka_ClusterDescription_cluster_id( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the node count for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The node count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_node_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc) ; + +/** + * @brief Get the DescribeCluster result. + * + * The returned description life-time is the same as the \p result object. + * + * @param result Result to get group results from. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_ClusterDescription_t * +rd_kafka_DescribeCluster_result_description( + const rd_kafka_DescribeCluster_result_t *result); + +/** + * @brief Describes the cluster. + * + * @param rk Client instance. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT + */ +RD_EXPORT +void rd_kafka_DescribeCluster(rd_kafka_t *rk, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); + +/** + * @brief Whether broker should return cluster authorized operations + * (DescribeCluster). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeCluster. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); +/**@}*/ + /** * @name Admin API - DeleteGroups * @brief Delete groups from cluster From 199c599d19ecfe260b8ca9c9d53b5a2c63fcba71 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 11 Apr 2023 10:17:02 +0530 Subject: [PATCH 02/12] integration tests needs reviews --- kafka/integration_test.go | 3203 +++++++++++++++++++------------------ 1 file changed, 1602 insertions(+), 1601 deletions(-) diff --git a/kafka/integration_test.go b/kafka/integration_test.go index 3442806d6..5496977ef 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -18,6 +18,7 @@ package kafka import ( "context" + "encoding/binary" "fmt" "math/rand" "path" @@ -602,483 +603,483 @@ func (its *IntegrationTestSuite) TearDownSuite() { } // TestConsumerSeekPartitions tests seeking of partitions using SeekPartitions(). -// func (its *IntegrationTestSuite) TestConsumerSeekPartitions() { -// t := its.T() - -// numMessages := 10 // should be more than or equal to 2. - -// // Produce `numMessages` messages to Topic. -// conf := ConfigMap{"bootstrap.servers": testconf.Brokers} -// conf.updateFromTestconf() - -// producer, err := NewProducer(&conf) -// if err != nil { -// t.Fatalf("Failed to create producer: %s", err) -// } - -// for idx := 0; idx < numMessages; idx++ { -// if err = producer.Produce(&Message{ -// TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, -// }, nil); err != nil { -// t.Fatalf("Failed to produce message: %s", err) -// } -// } - -// producer.Flush(10 * 1000) -// producer.Close() - -// // Assign partition, seek to `numMessages`/2, and check by reading the message. -// conf = ConfigMap{ -// "bootstrap.servers": testconf.Brokers, -// "group.id": testconf.GroupID, -// "auto.offset.reset": "end", -// } -// conf.updateFromTestconf() - -// consumer, err := NewConsumer(&conf) -// if err != nil { -// t.Fatalf("Failed to create consumer: %s", err) -// } - -// tps := []TopicPartition{ -// {Topic: &testconf.Topic, Partition: 0}, -// } -// err = consumer.Assign(tps) -// if err != nil { -// t.Fatalf("Failed to assign partition: %s", err) -// } - -// tps[0].Offset = Offset(numMessages / 2) -// seekedPartitions, err := consumer.SeekPartitions(tps) -// if err != nil { -// t.Errorf("SeekPartitions failed: %s", err) -// } -// if len(seekedPartitions) != len(tps) { -// t.Errorf( -// "SeekPartitions should return result for %d partitions, %d returned", -// len(tps), len(seekedPartitions)) -// } -// for _, seekedPartition := range seekedPartitions { -// if seekedPartition.Error != nil { -// t.Errorf("Seek error for partition %v", seekedPartition) -// } -// } - -// msg, err := consumer.ReadMessage(10 * time.Second) -// if err != nil { -// t.Fatalf("ReadMessage failed: %s", err) -// } -// if msg.TopicPartition.Offset != Offset(numMessages/2) { -// t.Errorf("Expected offset of read message is %d, got %d", -// numMessages/2, msg.TopicPartition.Offset) -// } -// } - -// // TestAdminClient_DeleteConsumerGroups verifies the working of the -// // DeleteConsumerGroups API in the admin client. -// // It does so by listing consumer groups before/after deletion. -// func (its *IntegrationTestSuite) TestAdminClient_DeleteConsumerGroups() { -// t := its.T() -// if testconf.Semaphore { -// t.Skipf("Skipping TestAdminClient_DeleteConsumerGroups since it is flaky[Does not run when tested with all the other integration tests]") -// return -// } -// rand.Seed(time.Now().Unix()) - -// // Generating new groupID to ensure a fresh group is created. -// groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) - -// ac := createAdminClient(t) -// defer ac.Close() - -// // Check that our group is not present initially. -// ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) -// if err != nil { -// t.Errorf("Error listing consumer groups %s\n", err) -// return -// } - -// if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { -// t.Errorf("Consumer group present before consumer created: %s\n", groupID) -// return -// } - -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() - -// // Create consumer -// config := &ConfigMap{ -// "bootstrap.servers": testconf.Brokers, -// "group.id": groupID, -// "auto.offset.reset": "earliest", -// "enable.auto.offset.store": false, -// } -// config.updateFromTestconf() -// consumer, err := NewConsumer(config) -// if err != nil { -// t.Errorf("Failed to create consumer: %s\n", err) -// return -// } -// consumerClosed := false -// defer func() { -// if !consumerClosed { -// consumer.Close() -// } -// }() - -// if err := consumer.Subscribe(testconf.Topic, nil); err != nil { -// t.Errorf("Failed to subscribe to %s: %s\n", testconf.Topic, err) -// return -// } - -// // This ReadMessage gives some time for the rebalance to happen. -// _, err = consumer.ReadMessage(5 * time.Second) -// if err != nil && err.(Error).Code() != ErrTimedOut { -// t.Errorf("Failed while reading message: %s\n", err) -// return -// } - -// // Check that the group exists. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) -// if err != nil { -// t.Errorf("Error listing consumer groups %s\n", err) -// return -// } - -// if findConsumerGroupListing(listGroupResult.Valid, groupID) == nil { -// t.Errorf("Consumer group %s should be present\n", groupID) -// return -// } - -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() - -// // Try deleting the group while consumer is active. It should fail. -// result, err := ac.DeleteConsumerGroups(ctx, []string{groupID}) -// if err != nil { -// t.Errorf("DeleteConsumerGroups() failed: %s", err) -// return -// } -// resultGroups := result.ConsumerGroupResults - -// if len(resultGroups) != 1 || resultGroups[0].Group != groupID { -// t.Errorf("Wrong group affected/no group affected") -// return -// } - -// if resultGroups[0].Error.code != ErrNonEmptyGroup { -// t.Errorf("Encountered the wrong error after calling DeleteConsumerGroups %s", resultGroups[0].Error) -// return -// } - -// // Close the consumer. -// if err = consumer.Close(); err != nil { -// t.Errorf("Could not close consumer %s", err) -// return -// } -// consumerClosed = true - -// // Delete the consumer group now. -// result, err = ac.DeleteConsumerGroups(ctx, []string{groupID}) -// if err != nil { -// t.Errorf("DeleteConsumerGroups() failed: %s", err) -// return -// } -// resultGroups = result.ConsumerGroupResults - -// if len(resultGroups) != 1 || resultGroups[0].Group != groupID { -// t.Errorf("Wrong group affected/no group affected") -// return -// } - -// if resultGroups[0].Error.code != ErrNoError { -// t.Errorf("Encountered an error after calling DeleteConsumerGroups %s", resultGroups[0].Error) -// return -// } - -// // Check for the absence of the consumer group after deletion. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) -// if err != nil { -// t.Errorf("Error listing consumer groups %s\n", err) -// return -// } - -// if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { -// t.Errorf("Consumer group %s should not be present\n", groupID) -// return -// } -// } - -// // TestAdminClient_ListAndDescribeConsumerGroups validates the working of the -// // list consumer groups and describe consumer group APIs of the admin client. -// // -// // We test the following situations: -// // -// // 1. One consumer group with one client. -// // 2. One consumer group with two clients. -// // 3. Empty consumer group. -// func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() { -// t := its.T() - -// // Generating a new topic/groupID to ensure a fresh group/topic is created. -// rand.Seed(time.Now().Unix()) -// groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) -// topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) -// nonExistentGroupID := fmt.Sprintf("%s-nonexistent-%d", testconf.GroupID, rand.Int()) - -// clientID1 := "test.client.1" -// clientID2 := "test.client.2" - -// ac := createAdminClient(t) -// defer ac.Close() - -// // Create a topic - we need to create here because we need 2 partitions. -// ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) -// defer cancel() -// _, err := ac.CreateTopics(ctx, []TopicSpecification{ -// { -// Topic: topic, -// NumPartitions: 2, -// }, -// }) -// if err != nil { -// t.Errorf("Topic creation failed with error %v", err) -// return -// } - -// // Delete the topic after the test is done. -// defer func() { -// ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) -// defer cancel() -// _, err = ac.DeleteTopics(ctx, []string{topic}) -// if err != nil { -// t.Errorf("Topic deletion failed with error %v", err) -// } -// }() - -// // Check the non-existence of consumer groups initially. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) -// if err != nil || len(listGroupResult.Errors) > 0 { -// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) -// return -// } - -// groups := listGroupResult.Valid -// if findConsumerGroupListing(groups, groupID) != nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { -// t.Errorf("Consumer groups %s and %s should not be present\n", groupID, nonExistentGroupID) -// return -// } - -// // 1. One consumer group with one client. -// // Create the first consumer. -// config := &ConfigMap{ -// "bootstrap.servers": testconf.Brokers, -// "group.id": groupID, -// "auto.offset.reset": "earliest", -// "enable.auto.offset.store": false, -// "client.id": clientID1, -// "partition.assignment.strategy": "range", -// } -// config.updateFromTestconf() -// consumer1, err := NewConsumer(config) -// if err != nil { -// t.Errorf("Failed to create consumer: %s\n", err) -// return -// } -// consumer1Closed := false -// defer func() { -// if !consumer1Closed { -// consumer1.Close() -// } -// }() -// consumer1.Subscribe(topic, nil) - -// // Call Poll to trigger a rebalance and give it enough time to finish. -// consumer1.Poll(10 * 1000) - -// // Check the existence of the group. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) -// if err != nil || len(listGroupResult.Errors) > 0 { -// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) -// return -// } -// groups = listGroupResult.Valid - -// if findConsumerGroupListing(groups, groupID) == nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { -// t.Errorf("Consumer groups %s should be present and %s should not be\n", groupID, nonExistentGroupID) -// return -// } - -// // Test the description of the consumer group. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// groupDescResult, err := ac.DescribeConsumerGroups( -// ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) -// if err != nil { -// t.Errorf("Error describing consumer groups %s\n", err) -// return -// } - -// groupDescs := groupDescResult.ConsumerGroupDescriptions -// if len(groupDescs) != 1 { -// t.Errorf("Describing one group should give exactly one result %s\n", err) -// return -// } - -// groupDesc := &groupDescs[0] - -// clientIDToPartitions := make(map[string][]TopicPartition) -// clientIDToPartitions[clientID1] = []TopicPartition{ -// {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, -// {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, -// } -// if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { -// t.Errorf("Expected description for consumer group %s is not same as actual: %v", groupID, groupDesc) -// return -// } - -// // 2. One consumer group with two clients. -// // Add another consumer to the same group. -// config = &ConfigMap{ -// "bootstrap.servers": testconf.Brokers, -// "group.id": groupID, -// "auto.offset.reset": "earliest", -// "enable.auto.offset.store": false, -// "client.id": clientID2, -// "partition.assignment.strategy": "range", -// } -// config.updateFromTestconf() -// consumer2, err := NewConsumer(config) -// if err != nil { -// t.Errorf("Failed to create consumer: %s\n", err) -// return -// } -// consumer2Closed := false -// defer func() { -// if !consumer2Closed { -// consumer2.Close() -// } -// }() -// consumer2.Subscribe(topic, nil) - -// // Call Poll to start triggering the rebalance. Give it enough time to run -// // that it becomes stable. -// // We need to make sure that the consumer group stabilizes since we will -// // check for the state later on. -// consumer2.Poll(5 * 1000) -// consumer1.Poll(5 * 1000) -// isGroupStable := false -// for !isGroupStable { -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) -// if err != nil { -// t.Errorf("Error describing consumer groups %s\n", err) -// return -// } -// groupDescs = groupDescResult.ConsumerGroupDescriptions -// groupDesc = findConsumerGroupDescription(groupDescs, groupID) -// if groupDesc == nil { -// t.Errorf("Consumer group %s should be present\n", groupID) -// return -// } -// isGroupStable = groupDesc.State == ConsumerGroupStateStable -// time.Sleep(time.Second) -// } - -// clientIDToPartitions[clientID1] = []TopicPartition{ -// {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, -// } -// clientIDToPartitions[clientID2] = []TopicPartition{ -// {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, -// } -// if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { -// t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) -// return -// } - -// // 3. Empty consumer group. -// // Close the existing consumers. -// if consumer1.Close() != nil { -// t.Errorf("Error closing the first consumer\n") -// return -// } -// consumer1Closed = true - -// if consumer2.Close() != nil { -// t.Errorf("Error closing the second consumer\n") -// return -// } -// consumer2Closed = true - -// // Try describing an empty group. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) -// groupDescs = groupDescResult.ConsumerGroupDescriptions - -// if err != nil { -// t.Errorf("Error describing consumer groups %s\n", err) -// return -// } - -// groupDesc = findConsumerGroupDescription(groupDescs, groupID) -// if groupDesc == nil { -// t.Errorf("Consumer group %s should be present\n", groupID) -// return -// } - -// clientIDToPartitions = make(map[string][]TopicPartition) -// if !checkGroupDesc(groupDesc, ConsumerGroupStateEmpty, groupID, "", clientIDToPartitions) { -// t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) -// } - -// // Try listing the Empty consumer group, and make sure that the States option -// // works while listing. -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err = ac.ListConsumerGroups( -// ctx, SetAdminRequestTimeout(30*time.Second), -// SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateEmpty})) -// if err != nil || len(listGroupResult.Errors) > 0 { -// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) -// return -// } -// groups = listGroupResult.Valid - -// groupInfo := findConsumerGroupListing(listGroupResult.Valid, groupID) -// if groupInfo == nil { -// t.Errorf("Consumer group %s should be present\n", groupID) -// return -// } - -// ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) -// defer cancel() -// listGroupResult, err = ac.ListConsumerGroups( -// ctx, SetAdminRequestTimeout(30*time.Second), -// SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateStable})) -// if err != nil || len(listGroupResult.Errors) > 0 { -// t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) -// return -// } -// groups = listGroupResult.Valid - -// groupInfo = findConsumerGroupListing(groups, groupID) -// if groupInfo != nil { -// t.Errorf("Consumer group %s should not be present\n", groupID) -// return -// } -// } +func (its *IntegrationTestSuite) TestConsumerSeekPartitions() { + t := its.T() + + numMessages := 10 // should be more than or equal to 2. + + // Produce `numMessages` messages to Topic. + conf := ConfigMap{"bootstrap.servers": testconf.Brokers} + conf.updateFromTestconf() + + producer, err := NewProducer(&conf) + if err != nil { + t.Fatalf("Failed to create producer: %s", err) + } + + for idx := 0; idx < numMessages; idx++ { + if err = producer.Produce(&Message{ + TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, + }, nil); err != nil { + t.Fatalf("Failed to produce message: %s", err) + } + } + + producer.Flush(10 * 1000) + producer.Close() + + // Assign partition, seek to `numMessages`/2, and check by reading the message. + conf = ConfigMap{ + "bootstrap.servers": testconf.Brokers, + "group.id": testconf.GroupID, + "auto.offset.reset": "end", + } + conf.updateFromTestconf() + + consumer, err := NewConsumer(&conf) + if err != nil { + t.Fatalf("Failed to create consumer: %s", err) + } + + tps := []TopicPartition{ + {Topic: &testconf.Topic, Partition: 0}, + } + err = consumer.Assign(tps) + if err != nil { + t.Fatalf("Failed to assign partition: %s", err) + } + + tps[0].Offset = Offset(numMessages / 2) + seekedPartitions, err := consumer.SeekPartitions(tps) + if err != nil { + t.Errorf("SeekPartitions failed: %s", err) + } + if len(seekedPartitions) != len(tps) { + t.Errorf( + "SeekPartitions should return result for %d partitions, %d returned", + len(tps), len(seekedPartitions)) + } + for _, seekedPartition := range seekedPartitions { + if seekedPartition.Error != nil { + t.Errorf("Seek error for partition %v", seekedPartition) + } + } + + msg, err := consumer.ReadMessage(10 * time.Second) + if err != nil { + t.Fatalf("ReadMessage failed: %s", err) + } + if msg.TopicPartition.Offset != Offset(numMessages/2) { + t.Errorf("Expected offset of read message is %d, got %d", + numMessages/2, msg.TopicPartition.Offset) + } +} + +// TestAdminClient_DeleteConsumerGroups verifies the working of the +// DeleteConsumerGroups API in the admin client. +// It does so by listing consumer groups before/after deletion. +func (its *IntegrationTestSuite) TestAdminClient_DeleteConsumerGroups() { + t := its.T() + if testconf.Semaphore { + t.Skipf("Skipping TestAdminClient_DeleteConsumerGroups since it is flaky[Does not run when tested with all the other integration tests]") + return + } + rand.Seed(time.Now().Unix()) + + // Generating new groupID to ensure a fresh group is created. + groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) + + ac := createAdminClient(t) + defer ac.Close() + + // Check that our group is not present initially. + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + if err != nil { + t.Errorf("Error listing consumer groups %s\n", err) + return + } + + if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { + t.Errorf("Consumer group present before consumer created: %s\n", groupID) + return + } + + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + + // Create consumer + config := &ConfigMap{ + "bootstrap.servers": testconf.Brokers, + "group.id": groupID, + "auto.offset.reset": "earliest", + "enable.auto.offset.store": false, + } + config.updateFromTestconf() + consumer, err := NewConsumer(config) + if err != nil { + t.Errorf("Failed to create consumer: %s\n", err) + return + } + consumerClosed := false + defer func() { + if !consumerClosed { + consumer.Close() + } + }() + + if err := consumer.Subscribe(testconf.Topic, nil); err != nil { + t.Errorf("Failed to subscribe to %s: %s\n", testconf.Topic, err) + return + } + + // This ReadMessage gives some time for the rebalance to happen. + _, err = consumer.ReadMessage(5 * time.Second) + if err != nil && err.(Error).Code() != ErrTimedOut { + t.Errorf("Failed while reading message: %s\n", err) + return + } + + // Check that the group exists. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + if err != nil { + t.Errorf("Error listing consumer groups %s\n", err) + return + } + + if findConsumerGroupListing(listGroupResult.Valid, groupID) == nil { + t.Errorf("Consumer group %s should be present\n", groupID) + return + } + + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + + // Try deleting the group while consumer is active. It should fail. + result, err := ac.DeleteConsumerGroups(ctx, []string{groupID}) + if err != nil { + t.Errorf("DeleteConsumerGroups() failed: %s", err) + return + } + resultGroups := result.ConsumerGroupResults + + if len(resultGroups) != 1 || resultGroups[0].Group != groupID { + t.Errorf("Wrong group affected/no group affected") + return + } + + if resultGroups[0].Error.code != ErrNonEmptyGroup { + t.Errorf("Encountered the wrong error after calling DeleteConsumerGroups %s", resultGroups[0].Error) + return + } + + // Close the consumer. + if err = consumer.Close(); err != nil { + t.Errorf("Could not close consumer %s", err) + return + } + consumerClosed = true + + // Delete the consumer group now. + result, err = ac.DeleteConsumerGroups(ctx, []string{groupID}) + if err != nil { + t.Errorf("DeleteConsumerGroups() failed: %s", err) + return + } + resultGroups = result.ConsumerGroupResults + + if len(resultGroups) != 1 || resultGroups[0].Group != groupID { + t.Errorf("Wrong group affected/no group affected") + return + } + + if resultGroups[0].Error.code != ErrNoError { + t.Errorf("Encountered an error after calling DeleteConsumerGroups %s", resultGroups[0].Error) + return + } + + // Check for the absence of the consumer group after deletion. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + if err != nil { + t.Errorf("Error listing consumer groups %s\n", err) + return + } + + if findConsumerGroupListing(listGroupResult.Valid, groupID) != nil { + t.Errorf("Consumer group %s should not be present\n", groupID) + return + } +} + +// TestAdminClient_ListAndDescribeConsumerGroups validates the working of the +// list consumer groups and describe consumer group APIs of the admin client. +// +// We test the following situations: +// +// 1. One consumer group with one client. +// 2. One consumer group with two clients. +// 3. Empty consumer group. +func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() { + t := its.T() + + // Generating a new topic/groupID to ensure a fresh group/topic is created. + rand.Seed(time.Now().Unix()) + groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) + topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) + nonExistentGroupID := fmt.Sprintf("%s-nonexistent-%d", testconf.GroupID, rand.Int()) + + clientID1 := "test.client.1" + clientID2 := "test.client.2" + + ac := createAdminClient(t) + defer ac.Close() + + // Create a topic - we need to create here because we need 2 partitions. + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + _, err := ac.CreateTopics(ctx, []TopicSpecification{ + { + Topic: topic, + NumPartitions: 2, + }, + }) + if err != nil { + t.Errorf("Topic creation failed with error %v", err) + return + } + + // Delete the topic after the test is done. + defer func() { + ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + _, err = ac.DeleteTopics(ctx, []string{topic}) + if err != nil { + t.Errorf("Topic deletion failed with error %v", err) + } + }() + + // Check the non-existence of consumer groups initially. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err := ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + if err != nil || len(listGroupResult.Errors) > 0 { + t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) + return + } + + groups := listGroupResult.Valid + if findConsumerGroupListing(groups, groupID) != nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { + t.Errorf("Consumer groups %s and %s should not be present\n", groupID, nonExistentGroupID) + return + } + + // 1. One consumer group with one client. + // Create the first consumer. + config := &ConfigMap{ + "bootstrap.servers": testconf.Brokers, + "group.id": groupID, + "auto.offset.reset": "earliest", + "enable.auto.offset.store": false, + "client.id": clientID1, + "partition.assignment.strategy": "range", + } + config.updateFromTestconf() + consumer1, err := NewConsumer(config) + if err != nil { + t.Errorf("Failed to create consumer: %s\n", err) + return + } + consumer1Closed := false + defer func() { + if !consumer1Closed { + consumer1.Close() + } + }() + consumer1.Subscribe(topic, nil) + + // Call Poll to trigger a rebalance and give it enough time to finish. + consumer1.Poll(10 * 1000) + + // Check the existence of the group. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err = ac.ListConsumerGroups(ctx, SetAdminRequestTimeout(30*time.Second)) + if err != nil || len(listGroupResult.Errors) > 0 { + t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) + return + } + groups = listGroupResult.Valid + + if findConsumerGroupListing(groups, groupID) == nil || findConsumerGroupListing(groups, nonExistentGroupID) != nil { + t.Errorf("Consumer groups %s should be present and %s should not be\n", groupID, nonExistentGroupID) + return + } + + // Test the description of the consumer group. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + groupDescResult, err := ac.DescribeConsumerGroups( + ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) + if err != nil { + t.Errorf("Error describing consumer groups %s\n", err) + return + } + + groupDescs := groupDescResult.ConsumerGroupDescriptions + if len(groupDescs) != 1 { + t.Errorf("Describing one group should give exactly one result %s\n", err) + return + } + + groupDesc := &groupDescs[0] + + clientIDToPartitions := make(map[string][]TopicPartition) + clientIDToPartitions[clientID1] = []TopicPartition{ + {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, + {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, + } + if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { + t.Errorf("Expected description for consumer group %s is not same as actual: %v", groupID, groupDesc) + return + } + + // 2. One consumer group with two clients. + // Add another consumer to the same group. + config = &ConfigMap{ + "bootstrap.servers": testconf.Brokers, + "group.id": groupID, + "auto.offset.reset": "earliest", + "enable.auto.offset.store": false, + "client.id": clientID2, + "partition.assignment.strategy": "range", + } + config.updateFromTestconf() + consumer2, err := NewConsumer(config) + if err != nil { + t.Errorf("Failed to create consumer: %s\n", err) + return + } + consumer2Closed := false + defer func() { + if !consumer2Closed { + consumer2.Close() + } + }() + consumer2.Subscribe(topic, nil) + + // Call Poll to start triggering the rebalance. Give it enough time to run + // that it becomes stable. + // We need to make sure that the consumer group stabilizes since we will + // check for the state later on. + consumer2.Poll(5 * 1000) + consumer1.Poll(5 * 1000) + isGroupStable := false + for !isGroupStable { + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) + if err != nil { + t.Errorf("Error describing consumer groups %s\n", err) + return + } + groupDescs = groupDescResult.ConsumerGroupDescriptions + groupDesc = findConsumerGroupDescription(groupDescs, groupID) + if groupDesc == nil { + t.Errorf("Consumer group %s should be present\n", groupID) + return + } + isGroupStable = groupDesc.State == ConsumerGroupStateStable + time.Sleep(time.Second) + } + + clientIDToPartitions[clientID1] = []TopicPartition{ + {Topic: &topic, Partition: 0, Offset: OffsetInvalid}, + } + clientIDToPartitions[clientID2] = []TopicPartition{ + {Topic: &topic, Partition: 1, Offset: OffsetInvalid}, + } + if !checkGroupDesc(groupDesc, ConsumerGroupStateStable, groupID, "range", clientIDToPartitions) { + t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) + return + } + + // 3. Empty consumer group. + // Close the existing consumers. + if consumer1.Close() != nil { + t.Errorf("Error closing the first consumer\n") + return + } + consumer1Closed = true + + if consumer2.Close() != nil { + t.Errorf("Error closing the second consumer\n") + return + } + consumer2Closed = true + + // Try describing an empty group. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + groupDescResult, err = ac.DescribeConsumerGroups(ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second)) + groupDescs = groupDescResult.ConsumerGroupDescriptions + + if err != nil { + t.Errorf("Error describing consumer groups %s\n", err) + return + } + + groupDesc = findConsumerGroupDescription(groupDescs, groupID) + if groupDesc == nil { + t.Errorf("Consumer group %s should be present\n", groupID) + return + } + + clientIDToPartitions = make(map[string][]TopicPartition) + if !checkGroupDesc(groupDesc, ConsumerGroupStateEmpty, groupID, "", clientIDToPartitions) { + t.Errorf("Expected description for consumer group %s is not same as actual %v\n", groupID, groupDesc) + } + + // Try listing the Empty consumer group, and make sure that the States option + // works while listing. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err = ac.ListConsumerGroups( + ctx, SetAdminRequestTimeout(30*time.Second), + SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateEmpty})) + if err != nil || len(listGroupResult.Errors) > 0 { + t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) + return + } + groups = listGroupResult.Valid + + groupInfo := findConsumerGroupListing(listGroupResult.Valid, groupID) + if groupInfo == nil { + t.Errorf("Consumer group %s should be present\n", groupID) + return + } + + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + listGroupResult, err = ac.ListConsumerGroups( + ctx, SetAdminRequestTimeout(30*time.Second), + SetAdminMatchConsumerGroupStates([]ConsumerGroupState{ConsumerGroupStateStable})) + if err != nil || len(listGroupResult.Errors) > 0 { + t.Errorf("Error listing consumer groups %s %v\n", err, listGroupResult.Errors) + return + } + groups = listGroupResult.Valid + + groupInfo = findConsumerGroupListing(groups, groupID) + if groupInfo != nil { + t.Errorf("Consumer group %s should not be present\n", groupID) + return + } +} // TestAdminClient_DescribeCluster validates the working of the // describe cluster API of the admin client. @@ -1323,403 +1324,403 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { checkExpectedResult(expectedCreateACLs, resultCreateACLs) } -// func (its *IntegrationTestSuite) TestAdminTopics() { -// t := its.T() -// rand.Seed(time.Now().Unix()) - -// a := createAdminClient(t) -// defer a.Close() - -// brokerList, err := getBrokerList(a) -// if err != nil { -// t.Fatalf("Failed to retrieve broker list: %v", err) -// } - -// // Few and Many replica sets use in these tests -// var fewReplicas []int32 -// if len(brokerList) < 2 { -// fewReplicas = brokerList -// } else { -// fewReplicas = brokerList[0:2] -// } - -// var manyReplicas []int32 -// if len(brokerList) < 5 { -// manyReplicas = brokerList -// } else { -// manyReplicas = brokerList[0:5] -// } - -// const topicCnt = 7 -// newTopics := make([]TopicSpecification, topicCnt) - -// expError := map[string]Error{} - -// for i := 0; i < topicCnt; i++ { -// topic := fmt.Sprintf("%s-create-%d-%d", testconf.Topic, i, rand.Intn(100000)) -// newTopics[i] = TopicSpecification{ -// Topic: topic, -// NumPartitions: 1 + i*2, -// } - -// if (i % 1) == 0 { -// newTopics[i].ReplicationFactor = len(fewReplicas) -// } else { -// newTopics[i].ReplicationFactor = len(manyReplicas) -// } - -// expError[newTopics[i].Topic] = Error{} // No error - -// var useReplicas []int32 -// if i == 2 { -// useReplicas = fewReplicas -// } else if i == 3 { -// useReplicas = manyReplicas -// } else if i == topicCnt-1 { -// newTopics[i].ReplicationFactor = len(brokerList) + 10 -// expError[newTopics[i].Topic] = Error{code: ErrInvalidReplicationFactor} -// } - -// if len(useReplicas) > 0 { -// newTopics[i].ReplicaAssignment = make([][]int32, newTopics[i].NumPartitions) -// newTopics[i].ReplicationFactor = 0 -// for p := 0; p < newTopics[i].NumPartitions; p++ { -// newTopics[i].ReplicaAssignment[p] = useReplicas -// } -// } -// } - -// maxDuration, err := time.ParseDuration("30s") -// if err != nil { -// t.Fatalf("%s", err) -// } - -// // First just validate the topics, don't create -// t.Logf("Validating topics before creation\n") -// ctx, cancel := context.WithTimeout(context.Background(), maxDuration) -// defer cancel() -// result, err := a.CreateTopics(ctx, newTopics, -// SetAdminValidateOnly(true)) -// if err != nil { -// t.Fatalf("CreateTopics(ValidateOnly) failed: %s", err) -// } - -// validateTopicResult(t, result, expError) - -// // Now create the topics -// t.Logf("Creating topics\n") -// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) -// defer cancel() -// result, err = a.CreateTopics(ctx, newTopics, SetAdminValidateOnly(false)) -// if err != nil { -// t.Fatalf("CreateTopics() failed: %s", err) -// } - -// validateTopicResult(t, result, expError) - -// // Attempt to create the topics again, should all fail. -// t.Logf("Attempt to re-create topics, should all fail\n") -// for k := range expError { -// if expError[k].code == ErrNoError { -// expError[k] = Error{code: ErrTopicAlreadyExists} -// } -// } -// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) -// defer cancel() -// result, err = a.CreateTopics(ctx, newTopics) -// if err != nil { -// t.Fatalf("CreateTopics#2() failed: %s", err) -// } - -// validateTopicResult(t, result, expError) - -// // Add partitions to some of the topics -// t.Logf("Create new partitions for a subset of topics\n") -// newParts := make([]PartitionsSpecification, topicCnt/2) -// expError = map[string]Error{} -// for i := 0; i < topicCnt/2; i++ { -// topic := newTopics[i].Topic -// newParts[i] = PartitionsSpecification{ -// Topic: topic, -// IncreaseTo: newTopics[i].NumPartitions + 3, -// } -// if i == 1 { -// // Invalid partition count (less than current) -// newParts[i].IncreaseTo = newTopics[i].NumPartitions - 1 -// expError[topic] = Error{code: ErrInvalidPartitions} -// } else { -// expError[topic] = Error{} -// } -// t.Logf("Creating new partitions for %s: %d -> %d: expecting %v\n", -// topic, newTopics[i].NumPartitions, newParts[i].IncreaseTo, expError[topic]) -// } - -// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) -// defer cancel() -// result, err = a.CreatePartitions(ctx, newParts) -// if err != nil { -// t.Fatalf("CreatePartitions() failed: %s", err) -// } - -// validateTopicResult(t, result, expError) - -// // FIXME: wait for topics to become available in metadata instead -// time.Sleep(5000 * time.Millisecond) - -// // Delete the topics -// deleteTopics := make([]string, topicCnt) -// for i := 0; i < topicCnt; i++ { -// deleteTopics[i] = newTopics[i].Topic -// if i == topicCnt-1 { -// expError[deleteTopics[i]] = Error{code: ErrUnknownTopicOrPart} -// } else { -// expError[deleteTopics[i]] = Error{} -// } -// } - -// ctx, cancel = context.WithTimeout(context.Background(), maxDuration) -// defer cancel() -// result2, err := a.DeleteTopics(ctx, deleteTopics) -// if err != nil { -// t.Fatalf("DeleteTopics() failed: %s", err) -// } - -// validateTopicResult(t, result2, expError) -// } - -// func (its *IntegrationTestSuite) TestAdminConfig() { -// t := its.T() -// rand.Seed(time.Now().Unix()) - -// a := createAdminClient(t) -// defer a.Close() - -// // Steps: -// // 1) Create a topic, providing initial non-default configuration -// // 2) Read back config to verify -// // 3) Alter config -// // 4) Read back config to verify -// // 5) Delete the topic - -// topic := fmt.Sprintf("%s-config-%d", testconf.Topic, rand.Intn(100000)) - -// // Expected config -// expResources := []ConfigResourceResult{ -// { -// Type: ResourceTopic, -// Name: topic, -// Config: map[string]ConfigEntryResult{ -// "compression.type": ConfigEntryResult{ -// Name: "compression.type", -// Value: "snappy", -// }, -// }, -// }, -// } -// // Create topic -// newTopics := []TopicSpecification{{ -// Topic: topic, -// NumPartitions: 1, -// ReplicationFactor: 1, -// Config: map[string]string{"compression.type": "snappy"}, -// }} - -// ctx, cancel := context.WithCancel(context.Background()) -// defer cancel() -// topicResult, err := a.CreateTopics(ctx, newTopics) -// if err != nil { -// t.Fatalf("Create topic request failed: %v", err) -// } - -// if topicResult[0].Error.Code() != ErrNoError { -// t.Fatalf("Failed to create topic %s: %s", topic, topicResult[0].Error) -// } - -// // Wait for topic to show up in metadata before performing -// // subsequent operations on it, otherwise we risk DescribeConfigs() -// // to fail with UnknownTopic.. (this is really a broker issue). -// // Sometimes even the metadata is not enough, so we add an -// // arbitrary 10s sleep too. -// t.Logf("Waiting for new topic %s to show up in metadata and stabilize", topic) -// err = waitTopicInMetadata(a, topic, 10*1000) // 10s -// if err != nil { -// t.Fatalf("%v", err) -// } -// t.Logf("Topic %s now in metadata, waiting another 10s for stabilization", topic) -// time.Sleep(10 * time.Second) - -// // Read back config to validate -// configResources := []ConfigResource{{Type: ResourceTopic, Name: topic}} -// describeRes, err := a.DescribeConfigs(ctx, configResources) -// if err != nil { -// t.Fatalf("Describe configs request failed: %v", err) -// } - -// validateConfig(t, describeRes, expResources, true) - -// // Alter some configs. -// // Configuration alterations are currently atomic, all values -// // need to be passed, otherwise non-passed values will be reverted -// // to their default values. -// // Future versions will allow incremental updates: -// // https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API -// newConfig := make(map[string]string) -// for _, entry := range describeRes[0].Config { -// newConfig[entry.Name] = entry.Value -// } - -// // Change something -// newConfig["retention.ms"] = "86400000" -// newConfig["message.timestamp.type"] = "LogAppendTime" - -// for k, v := range newConfig { -// expResources[0].Config[k] = ConfigEntryResult{Name: k, Value: v} -// } - -// configResources = []ConfigResource{{Type: ResourceTopic, Name: topic, Config: StringMapToConfigEntries(newConfig, AlterOperationSet)}} -// alterRes, err := a.AlterConfigs(ctx, configResources) -// if err != nil { -// t.Fatalf("Alter configs request failed: %v", err) -// } - -// validateConfig(t, alterRes, expResources, false) - -// // Read back config to validate -// configResources = []ConfigResource{{Type: ResourceTopic, Name: topic}} -// describeRes, err = a.DescribeConfigs(ctx, configResources) -// if err != nil { -// t.Fatalf("Describe configs request failed: %v", err) -// } - -// validateConfig(t, describeRes, expResources, true) - -// // Delete the topic -// // FIXME: wait for topics to become available in metadata instead -// time.Sleep(5000 * time.Millisecond) - -// topicResult, err = a.DeleteTopics(ctx, []string{topic}) -// if err != nil { -// t.Fatalf("DeleteTopics() failed: %s", err) -// } - -// if topicResult[0].Error.Code() != ErrNoError { -// t.Fatalf("Failed to delete topic %s: %s", topic, topicResult[0].Error) -// } -// } - -// // Test AdminClient GetMetadata API -// func (its *IntegrationTestSuite) TestAdminGetMetadata() { -// t := its.T() - -// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} -// config.updateFromTestconf() - -// // Create Admin client -// a, err := NewAdminClient(config) -// if err != nil { -// t.Errorf("Failed to create Admin client: %s\n", err) -// return -// } -// defer a.Close() - -// metaData, err := a.GetMetadata(&testconf.Topic, false, 5*1000) -// if err != nil { -// t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) -// return -// } -// t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) - -// metaData, err = a.GetMetadata(nil, true, 5*1000) -// if err != nil { -// t.Errorf("Failed to get meta data, Error: %s\n", err) -// return -// } -// t.Logf("Meta data for admin client: %v\n", metaData) -// } - -// // Test AdminClient ClusterID. -// func (its *IntegrationTestSuite) TestAdminClient_ClusterID() { -// t := its.T() - -// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} -// if err := config.updateFromTestconf(); err != nil { -// t.Fatalf("Failed to update test configuration: %s\n", err) -// } - -// admin, err := NewAdminClient(config) -// if err != nil { -// t.Fatalf("Failed to create Admin client: %s\n", err) -// } -// defer admin.Close() - -// ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) -// defer cancel() -// clusterID, err := admin.ClusterID(ctx) -// if err != nil { -// t.Fatalf("Failed to get ClusterID: %s\n", err) -// } -// if clusterID == "" { -// t.Fatal("ClusterID is empty.") -// } - -// t.Logf("ClusterID: %s\n", clusterID) -// } - -// // Test AdminClient ControllerID. -// func (its *IntegrationTestSuite) TestAdminClient_ControllerID() { -// t := its.T() - -// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} -// if err := config.updateFromTestconf(); err != nil { -// t.Fatalf("Failed to update test configuration: %s\n", err) -// } - -// producer, err := NewProducer(config) -// if err != nil { -// t.Fatalf("Failed to create Producer client: %s\n", err) -// } -// admin, err := NewAdminClientFromProducer(producer) -// if err != nil { -// t.Fatalf("Failed to create Admin client: %s\n", err) -// } -// defer admin.Close() - -// ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) -// defer cancel() -// controllerID, err := admin.ControllerID(ctx) -// if err != nil { -// t.Fatalf("Failed to get ControllerID: %s\n", err) -// } -// if controllerID < 0 { -// t.Fatalf("ControllerID is negative: %d\n", controllerID) -// } - -// t.Logf("ControllerID: %d\n", controllerID) -// } - -func (its *IntegrationTestSuite) TestAdminACLs() { +func (its *IntegrationTestSuite) TestAdminTopics() { t := its.T() - rand.Seed(time.Now().Unix()) - topic := testconf.Topic - group := testconf.GroupID - noError := NewError(ErrNoError, "", false) - unknownError := NewError(ErrUnknown, "Unknown broker error", false) - var expectedCreateACLs []CreateACLResult - var expectedDescribeACLs DescribeACLsResult - var expectedDeleteACLs []DeleteACLsResult - var ctx context.Context - var cancel context.CancelFunc a := createAdminClient(t) defer a.Close() - maxDuration, err := time.ParseDuration("30s") + brokerList, err := getBrokerList(a) if err != nil { - t.Fatalf("%s", err) + t.Fatalf("Failed to retrieve broker list: %v", err) } - requestTimeout, err := time.ParseDuration("20s") - if err != nil { + + // Few and Many replica sets use in these tests + var fewReplicas []int32 + if len(brokerList) < 2 { + fewReplicas = brokerList + } else { + fewReplicas = brokerList[0:2] + } + + var manyReplicas []int32 + if len(brokerList) < 5 { + manyReplicas = brokerList + } else { + manyReplicas = brokerList[0:5] + } + + const topicCnt = 7 + newTopics := make([]TopicSpecification, topicCnt) + + expError := map[string]Error{} + + for i := 0; i < topicCnt; i++ { + topic := fmt.Sprintf("%s-create-%d-%d", testconf.Topic, i, rand.Intn(100000)) + newTopics[i] = TopicSpecification{ + Topic: topic, + NumPartitions: 1 + i*2, + } + + if (i % 1) == 0 { + newTopics[i].ReplicationFactor = len(fewReplicas) + } else { + newTopics[i].ReplicationFactor = len(manyReplicas) + } + + expError[newTopics[i].Topic] = Error{} // No error + + var useReplicas []int32 + if i == 2 { + useReplicas = fewReplicas + } else if i == 3 { + useReplicas = manyReplicas + } else if i == topicCnt-1 { + newTopics[i].ReplicationFactor = len(brokerList) + 10 + expError[newTopics[i].Topic] = Error{code: ErrInvalidReplicationFactor} + } + + if len(useReplicas) > 0 { + newTopics[i].ReplicaAssignment = make([][]int32, newTopics[i].NumPartitions) + newTopics[i].ReplicationFactor = 0 + for p := 0; p < newTopics[i].NumPartitions; p++ { + newTopics[i].ReplicaAssignment[p] = useReplicas + } + } + } + + maxDuration, err := time.ParseDuration("30s") + if err != nil { + t.Fatalf("%s", err) + } + + // First just validate the topics, don't create + t.Logf("Validating topics before creation\n") + ctx, cancel := context.WithTimeout(context.Background(), maxDuration) + defer cancel() + result, err := a.CreateTopics(ctx, newTopics, + SetAdminValidateOnly(true)) + if err != nil { + t.Fatalf("CreateTopics(ValidateOnly) failed: %s", err) + } + + validateTopicResult(t, result, expError) + + // Now create the topics + t.Logf("Creating topics\n") + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + defer cancel() + result, err = a.CreateTopics(ctx, newTopics, SetAdminValidateOnly(false)) + if err != nil { + t.Fatalf("CreateTopics() failed: %s", err) + } + + validateTopicResult(t, result, expError) + + // Attempt to create the topics again, should all fail. + t.Logf("Attempt to re-create topics, should all fail\n") + for k := range expError { + if expError[k].code == ErrNoError { + expError[k] = Error{code: ErrTopicAlreadyExists} + } + } + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + defer cancel() + result, err = a.CreateTopics(ctx, newTopics) + if err != nil { + t.Fatalf("CreateTopics#2() failed: %s", err) + } + + validateTopicResult(t, result, expError) + + // Add partitions to some of the topics + t.Logf("Create new partitions for a subset of topics\n") + newParts := make([]PartitionsSpecification, topicCnt/2) + expError = map[string]Error{} + for i := 0; i < topicCnt/2; i++ { + topic := newTopics[i].Topic + newParts[i] = PartitionsSpecification{ + Topic: topic, + IncreaseTo: newTopics[i].NumPartitions + 3, + } + if i == 1 { + // Invalid partition count (less than current) + newParts[i].IncreaseTo = newTopics[i].NumPartitions - 1 + expError[topic] = Error{code: ErrInvalidPartitions} + } else { + expError[topic] = Error{} + } + t.Logf("Creating new partitions for %s: %d -> %d: expecting %v\n", + topic, newTopics[i].NumPartitions, newParts[i].IncreaseTo, expError[topic]) + } + + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + defer cancel() + result, err = a.CreatePartitions(ctx, newParts) + if err != nil { + t.Fatalf("CreatePartitions() failed: %s", err) + } + + validateTopicResult(t, result, expError) + + // FIXME: wait for topics to become available in metadata instead + time.Sleep(5000 * time.Millisecond) + + // Delete the topics + deleteTopics := make([]string, topicCnt) + for i := 0; i < topicCnt; i++ { + deleteTopics[i] = newTopics[i].Topic + if i == topicCnt-1 { + expError[deleteTopics[i]] = Error{code: ErrUnknownTopicOrPart} + } else { + expError[deleteTopics[i]] = Error{} + } + } + + ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + defer cancel() + result2, err := a.DeleteTopics(ctx, deleteTopics) + if err != nil { + t.Fatalf("DeleteTopics() failed: %s", err) + } + + validateTopicResult(t, result2, expError) +} + +func (its *IntegrationTestSuite) TestAdminConfig() { + t := its.T() + rand.Seed(time.Now().Unix()) + + a := createAdminClient(t) + defer a.Close() + + // Steps: + // 1) Create a topic, providing initial non-default configuration + // 2) Read back config to verify + // 3) Alter config + // 4) Read back config to verify + // 5) Delete the topic + + topic := fmt.Sprintf("%s-config-%d", testconf.Topic, rand.Intn(100000)) + + // Expected config + expResources := []ConfigResourceResult{ + { + Type: ResourceTopic, + Name: topic, + Config: map[string]ConfigEntryResult{ + "compression.type": ConfigEntryResult{ + Name: "compression.type", + Value: "snappy", + }, + }, + }, + } + // Create topic + newTopics := []TopicSpecification{{ + Topic: topic, + NumPartitions: 1, + ReplicationFactor: 1, + Config: map[string]string{"compression.type": "snappy"}, + }} + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + topicResult, err := a.CreateTopics(ctx, newTopics) + if err != nil { + t.Fatalf("Create topic request failed: %v", err) + } + + if topicResult[0].Error.Code() != ErrNoError { + t.Fatalf("Failed to create topic %s: %s", topic, topicResult[0].Error) + } + + // Wait for topic to show up in metadata before performing + // subsequent operations on it, otherwise we risk DescribeConfigs() + // to fail with UnknownTopic.. (this is really a broker issue). + // Sometimes even the metadata is not enough, so we add an + // arbitrary 10s sleep too. + t.Logf("Waiting for new topic %s to show up in metadata and stabilize", topic) + err = waitTopicInMetadata(a, topic, 10*1000) // 10s + if err != nil { + t.Fatalf("%v", err) + } + t.Logf("Topic %s now in metadata, waiting another 10s for stabilization", topic) + time.Sleep(10 * time.Second) + + // Read back config to validate + configResources := []ConfigResource{{Type: ResourceTopic, Name: topic}} + describeRes, err := a.DescribeConfigs(ctx, configResources) + if err != nil { + t.Fatalf("Describe configs request failed: %v", err) + } + + validateConfig(t, describeRes, expResources, true) + + // Alter some configs. + // Configuration alterations are currently atomic, all values + // need to be passed, otherwise non-passed values will be reverted + // to their default values. + // Future versions will allow incremental updates: + // https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API + newConfig := make(map[string]string) + for _, entry := range describeRes[0].Config { + newConfig[entry.Name] = entry.Value + } + + // Change something + newConfig["retention.ms"] = "86400000" + newConfig["message.timestamp.type"] = "LogAppendTime" + + for k, v := range newConfig { + expResources[0].Config[k] = ConfigEntryResult{Name: k, Value: v} + } + + configResources = []ConfigResource{{Type: ResourceTopic, Name: topic, Config: StringMapToConfigEntries(newConfig, AlterOperationSet)}} + alterRes, err := a.AlterConfigs(ctx, configResources) + if err != nil { + t.Fatalf("Alter configs request failed: %v", err) + } + + validateConfig(t, alterRes, expResources, false) + + // Read back config to validate + configResources = []ConfigResource{{Type: ResourceTopic, Name: topic}} + describeRes, err = a.DescribeConfigs(ctx, configResources) + if err != nil { + t.Fatalf("Describe configs request failed: %v", err) + } + + validateConfig(t, describeRes, expResources, true) + + // Delete the topic + // FIXME: wait for topics to become available in metadata instead + time.Sleep(5000 * time.Millisecond) + + topicResult, err = a.DeleteTopics(ctx, []string{topic}) + if err != nil { + t.Fatalf("DeleteTopics() failed: %s", err) + } + + if topicResult[0].Error.Code() != ErrNoError { + t.Fatalf("Failed to delete topic %s: %s", topic, topicResult[0].Error) + } +} + +// Test AdminClient GetMetadata API +func (its *IntegrationTestSuite) TestAdminGetMetadata() { + t := its.T() + + config := &ConfigMap{"bootstrap.servers": testconf.Brokers} + config.updateFromTestconf() + + // Create Admin client + a, err := NewAdminClient(config) + if err != nil { + t.Errorf("Failed to create Admin client: %s\n", err) + return + } + defer a.Close() + + metaData, err := a.GetMetadata(&testconf.Topic, false, 5*1000) + if err != nil { + t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) + return + } + t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) + + metaData, err = a.GetMetadata(nil, true, 5*1000) + if err != nil { + t.Errorf("Failed to get meta data, Error: %s\n", err) + return + } + t.Logf("Meta data for admin client: %v\n", metaData) +} + +// Test AdminClient ClusterID. +func (its *IntegrationTestSuite) TestAdminClient_ClusterID() { + t := its.T() + + config := &ConfigMap{"bootstrap.servers": testconf.Brokers} + if err := config.updateFromTestconf(); err != nil { + t.Fatalf("Failed to update test configuration: %s\n", err) + } + + admin, err := NewAdminClient(config) + if err != nil { + t.Fatalf("Failed to create Admin client: %s\n", err) + } + defer admin.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + clusterID, err := admin.ClusterID(ctx) + if err != nil { + t.Fatalf("Failed to get ClusterID: %s\n", err) + } + if clusterID == "" { + t.Fatal("ClusterID is empty.") + } + + t.Logf("ClusterID: %s\n", clusterID) +} + +// Test AdminClient ControllerID. +func (its *IntegrationTestSuite) TestAdminClient_ControllerID() { + t := its.T() + + config := &ConfigMap{"bootstrap.servers": testconf.Brokers} + if err := config.updateFromTestconf(); err != nil { + t.Fatalf("Failed to update test configuration: %s\n", err) + } + + producer, err := NewProducer(config) + if err != nil { + t.Fatalf("Failed to create Producer client: %s\n", err) + } + admin, err := NewAdminClientFromProducer(producer) + if err != nil { + t.Fatalf("Failed to create Admin client: %s\n", err) + } + defer admin.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + controllerID, err := admin.ControllerID(ctx) + if err != nil { + t.Fatalf("Failed to get ControllerID: %s\n", err) + } + if controllerID < 0 { + t.Fatalf("ControllerID is negative: %d\n", controllerID) + } + + t.Logf("ControllerID: %d\n", controllerID) +} + +func (its *IntegrationTestSuite) TestAdminACLs() { + t := its.T() + + rand.Seed(time.Now().Unix()) + topic := testconf.Topic + group := testconf.GroupID + noError := NewError(ErrNoError, "", false) + unknownError := NewError(ErrUnknown, "Unknown broker error", false) + var expectedCreateACLs []CreateACLResult + var expectedDescribeACLs DescribeACLsResult + var expectedDeleteACLs []DeleteACLsResult + var ctx context.Context + var cancel context.CancelFunc + + a := createAdminClient(t) + defer a.Close() + + maxDuration, err := time.ParseDuration("30s") + if err != nil { + t.Fatalf("%s", err) + } + requestTimeout, err := time.ParseDuration("20s") + if err != nil { t.Fatalf("%s", err) } @@ -1889,741 +1890,741 @@ func (its *IntegrationTestSuite) TestAdminACLs() { checkExpectedResult(expectedDescribeACLs, *resultDescribeACLs) } -// // Test consumer QueryWatermarkOffsets API -// func (its *IntegrationTestSuite) TestConsumerQueryWatermarkOffsets() { -// t := its.T() - -// // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to -// // get the number of messages in a topic -// msgcnt, err := getMessageCountInTopic(testconf.Topic) -// if err != nil { -// t.Errorf("Cannot get message size. Error: %s\n", err) -// } - -// // Prime topic with test messages -// createTestMessages() -// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) - -// // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to -// // get the number of messages in a topic -// newmsgcnt, err := getMessageCountInTopic(testconf.Topic) -// if err != nil { -// t.Errorf("Cannot get message size. Error: %s\n", err) -// } - -// if newmsgcnt-msgcnt != len(p0TestMsgs) { -// t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newmsgcnt-msgcnt) -// } - -// } - -// // Test consumer GetWatermarkOffsets API -// func (its *IntegrationTestSuite) TestConsumerGetWatermarkOffsets() { -// t := its.T() - -// // Create consumer -// config := &ConfigMap{ -// "go.events.channel.enable": true, -// "bootstrap.servers": testconf.Brokers, -// "group.id": testconf.GroupID, -// "session.timeout.ms": 6000, -// "enable.auto.commit": false, -// "auto.offset.reset": "earliest", -// } -// _ = config.updateFromTestconf() - -// c, err := NewConsumer(config) -// if err != nil { -// t.Fatalf("Unable to create consumer: %s", err) -// } -// defer func() { _ = c.Close() }() - -// err = c.Subscribe(testconf.Topic, nil) - -// // Prime topic with test messages -// createTestMessages() -// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) - -// // Wait for messages to be received so that we know the watermark offsets have been delivered -// // with the fetch response -// for ev := range c.Events() { -// if _, ok := ev.(*Message); ok { -// break -// } -// } - -// _, queryHigh, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) -// if err != nil { -// t.Fatalf("Error querying watermark offsets: %s", err) -// } - -// // We are not currently testing the low watermark offset as it only gets set every 10s by the stits timer -// _, getHigh, err := c.GetWatermarkOffsets(testconf.Topic, 0) -// if err != nil { -// t.Fatalf("Error getting watermark offsets: %s", err) -// } - -// if queryHigh != getHigh { -// t.Errorf("QueryWatermarkOffsets high[%d] does not equal GetWatermarkOffsets high[%d]", queryHigh, getHigh) -// } - -// } - -// // TestConsumerOffsetsForTimes -// func (its *IntegrationTestSuite) TestConsumerOffsetsForTimes() { -// t := its.T() - -// conf := ConfigMap{"bootstrap.servers": testconf.Brokers, -// "group.id": testconf.GroupID, -// "api.version.request": true} - -// conf.updateFromTestconf() - -// c, err := NewConsumer(&conf) - -// if err != nil { -// panic(err) -// } -// defer c.Close() - -// // Prime topic with test messages -// createTestMessages() -// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) - -// times := make([]TopicPartition, 1) -// times[0] = TopicPartition{Topic: &testconf.Topic, Partition: 0, Offset: 12345} -// offsets, err := c.OffsetsForTimes(times, 5000) -// if err != nil { -// t.Errorf("OffsetsForTimes() failed: %s\n", err) -// return -// } - -// if len(offsets) != 1 { -// t.Errorf("OffsetsForTimes() returned wrong length %d, expected 1\n", len(offsets)) -// return -// } - -// if *offsets[0].Topic != testconf.Topic || offsets[0].Partition != 0 { -// t.Errorf("OffsetsForTimes() returned wrong topic/partition\n") -// return -// } - -// if offsets[0].Error != nil { -// t.Errorf("OffsetsForTimes() returned error for partition 0: %s\n", err) -// return -// } - -// low, _, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) -// if err != nil { -// t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) -// return -// } - -// t.Logf("OffsetsForTimes() returned offset %d for timestamp %d\n", offsets[0].Offset, times[0].Offset) - -// // Since we're using a phony low timestamp it is assumed that the returned -// // offset will be oldest message. -// if offsets[0].Offset != Offset(low) { -// t.Errorf("OffsetsForTimes() returned invalid offset %d for timestamp %d, expected %d\n", offsets[0].Offset, times[0].Offset, low) -// return -// } - -// } - -// // test consumer GetMetadata API -// func (its *IntegrationTestSuite) TestConsumerGetMetadata() { -// t := its.T() - -// config := &ConfigMap{"bootstrap.servers": testconf.Brokers, -// "group.id": testconf.GroupID} -// config.updateFromTestconf() - -// // Create consumer -// c, err := NewConsumer(config) -// if err != nil { -// t.Errorf("Failed to create consumer: %s\n", err) -// return -// } -// defer c.Close() - -// metaData, err := c.GetMetadata(&testconf.Topic, false, 5*1000) -// if err != nil { -// t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) -// return -// } -// t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) - -// metaData, err = c.GetMetadata(nil, true, 5*1000) -// if err != nil { -// t.Errorf("Failed to get meta data, Error: %s\n", err) -// return -// } -// t.Logf("Meta data for consumer: %v\n", metaData) -// } - -// // Test producer QueryWatermarkOffsets API -// func (its *IntegrationTestSuite) TestProducerQueryWatermarkOffsets() { -// t := its.T() - -// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} -// config.updateFromTestconf() - -// // Create producer -// p, err := NewProducer(config) -// if err != nil { -// t.Errorf("Failed to create producer: %s\n", err) -// return -// } -// defer p.Close() - -// low, high, err := p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) -// if err != nil { -// t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) -// return -// } -// cnt := high - low -// t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) - -// createTestMessages() -// producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) - -// low, high, err = p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) -// if err != nil { -// t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) -// return -// } -// t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) -// newcnt := high - low -// t.Logf("count = %d, New count = %d\n", cnt, newcnt) -// if newcnt-cnt != int64(len(p0TestMsgs)) { -// t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newcnt-cnt) -// } -// } - -// // Test producer GetMetadata API -// func (its *IntegrationTestSuite) TestProducerGetMetadata() { -// t := its.T() - -// config := &ConfigMap{"bootstrap.servers": testconf.Brokers} -// config.updateFromTestconf() - -// // Create producer -// p, err := NewProducer(config) -// if err != nil { -// t.Errorf("Failed to create producer: %s\n", err) -// return -// } -// defer p.Close() - -// metaData, err := p.GetMetadata(&testconf.Topic, false, 5*1000) -// if err != nil { -// t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) -// return -// } -// t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) - -// metaData, err = p.GetMetadata(nil, true, 5*1000) -// if err != nil { -// t.Errorf("Failed to get meta data, Error: %s\n", err) -// return -// } -// t.Logf("Meta data for producer: %v\n", metaData) - -// } - -// // test producer function-based API without delivery report -// func (its *IntegrationTestSuite) TestProducerFunc() { -// t := its.T() -// producerTest(t, "Function producer (without DR)", -// nil, producerCtrl{}, -// func(p *Producer, m *Message, drChan chan Event) { -// err := p.Produce(m, drChan) -// if err != nil { -// t.Errorf("Produce() failed: %v", err) -// } -// }) -// } - -// // test producer function-based API with delivery report -// func (its *IntegrationTestSuite) TestProducerFuncDR() { -// t := its.T() -// producerTest(t, "Function producer (with DR)", -// nil, producerCtrl{withDr: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// err := p.Produce(m, drChan) -// if err != nil { -// t.Errorf("Produce() failed: %v", err) -// } -// }) -// } - -// // test producer with bad messages -// func (its *IntegrationTestSuite) TestProducerWithBadMessages() { -// t := its.T() -// conf := ConfigMap{"bootstrap.servers": testconf.Brokers} -// conf.updateFromTestconf() - -// p, err := NewProducer(&conf) -// if err != nil { -// panic(err) -// } -// defer p.Close() - -// // producing a nil message should return an error without crash -// err = p.Produce(nil, p.Events()) -// if err == nil { -// t.Errorf("Producing a nil message should return error\n") -// } else { -// t.Logf("Producing a nil message returns expected error: %s\n", err) -// } - -// // producing a blank message (with nil Topic) should return an error without crash -// err = p.Produce(&Message{}, p.Events()) -// if err == nil { -// t.Errorf("Producing a blank message should return error\n") -// } else { -// t.Logf("Producing a blank message returns expected error: %s\n", err) -// } -// } - -// // test producer channel-based API without delivery report -// func (its *IntegrationTestSuite) TestProducerChannel() { -// t := its.T() -// producerTest(t, "Channel producer (without DR)", -// nil, producerCtrl{}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) -// } - -// // test producer channel-based API with delivery report -// func (its *IntegrationTestSuite) TestProducerChannelDR() { -// t := its.T() -// producerTest(t, "Channel producer (with DR)", -// nil, producerCtrl{withDr: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) - -// } - -// // test batch producer channel-based API without delivery report -// func (its *IntegrationTestSuite) TestProducerBatchChannel() { -// t := its.T() -// producerTest(t, "Channel producer (without DR, batch channel)", -// nil, producerCtrl{batchProducer: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) -// } - -// // test batch producer channel-based API with delivery report -// func (its *IntegrationTestSuite) TestProducerBatchChannelDR() { -// t := its.T() -// producerTest(t, "Channel producer (DR, batch channel)", -// nil, producerCtrl{withDr: true, batchProducer: true}, -// func(p *Producer, m *Message, drChan chan Event) { -// p.ProduceChannel() <- m -// }) -// } - -// // test consumer channel-based API -// func (its *IntegrationTestSuite) TestConsumerChannel() { -// t := its.T() -// consumerTestWithCommits(t, "Channel Consumer", -// "", 0, true, eventTestChannelConsumer, nil) -// } - -// // test consumer channel-based API with incremental rebalancing -// func (its *IntegrationTestSuite) TestConsumerChannelIncremental() { -// t := its.T() -// consumerTestWithCommits(t, "Channel Consumer Incremental", -// "cooperative-sticky", 0, true, eventTestChannelConsumer, nil) -// } - -// // test consumer poll-based API -// func (its *IntegrationTestSuite) TestConsumerPoll() { -// t := its.T() -// consumerTestWithCommits(t, "Poll Consumer", "", 0, false, eventTestPollConsumer, nil) -// } - -// // test consumer poll-based API with incremental rebalancing -// func (its *IntegrationTestSuite) TestConsumerPollIncremental() { -// t := its.T() -// consumerTestWithCommits(t, "Poll Consumer ncremental", -// "cooperative-sticky", 0, false, eventTestPollConsumer, nil) -// } - -// // test consumer poll-based API with rebalance callback -// func (its *IntegrationTestSuite) TestConsumerPollRebalance() { -// t := its.T() -// consumerTestWithCommits(t, "Poll Consumer (rebalance callback)", -// "", 0, false, eventTestPollConsumer, -// func(c *Consumer, event Event) error { -// t.Logf("Rebalanced: %s", event) -// return nil -// }) -// } - -// // test consumer poll-based API with incremental no-op rebalance callback -// func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncrementalNoop() { -// t := its.T() -// consumerTestWithCommits(t, "Poll Consumer (incremental no-op rebalance callback)", -// "cooperative-sticky", 0, false, eventTestPollConsumer, -// func(c *Consumer, event Event) error { -// t.Logf("Rebalanced: %s", event) -// return nil -// }) -// } - -// // test consumer poll-based API with incremental rebalance callback -// func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncremental() { -// t := its.T() -// consumerTestWithCommits(t, "Poll Consumer (incremental rebalance callback)", -// "cooperative-sticky", 0, false, eventTestPollConsumer, -// func(c *Consumer, event Event) error { -// t.Logf("Rebalanced: %s (RebalanceProtocol=%s, AssignmentLost=%v)", -// event, c.GetRebalanceProtocol(), c.AssignmentLost()) - -// switch e := event.(type) { -// case AssignedPartitions: -// err := c.IncrementalAssign(e.Partitions) -// if err != nil { -// t.Errorf("IncrementalAssign() failed: %s\n", err) -// return err -// } -// case RevokedPartitions: -// err := c.IncrementalUnassign(e.Partitions) -// if err != nil { -// t.Errorf("IncrementalUnassign() failed: %s\n", err) -// return err -// } -// default: -// t.Fatalf("Unexpected rebalance event: %v\n", e) -// } - -// return nil -// }) -// } - -// // Test Committed() API -// func (its *IntegrationTestSuite) TestConsumerCommitted() { -// t := its.T() -// if testconf.Semaphore { -// t.Skipf("Skipping TestConsumerCommitted since it is flaky[Does not run when tested with all the other integration tests]") -// return -// } - -// consumerTestWithCommits(t, "Poll Consumer (rebalance callback, verify Committed())", -// "", 0, false, eventTestPollConsumer, -// func(c *Consumer, event Event) error { -// t.Logf("Rebalanced: %s", event) -// rp, ok := event.(RevokedPartitions) -// if ok { -// offsets, err := c.Committed(rp.Partitions, 5000) -// if err != nil { -// t.Errorf("Failed to get committed offsets: %s\n", err) -// return nil -// } - -// t.Logf("Retrieved Committed offsets: %s\n", offsets) - -// if len(offsets) != len(rp.Partitions) || len(rp.Partitions) == 0 { -// t.Errorf("Invalid number of partitions %d, should be %d (and >0)\n", len(offsets), len(rp.Partitions)) -// } - -// // Verify proper offsets: at least one partition needs -// // to have a committed offset. -// validCnt := 0 -// for _, p := range offsets { -// if p.Error != nil { -// t.Errorf("Committed() partition error: %v: %v", p, p.Error) -// } else if p.Offset >= 0 { -// validCnt++ -// } -// } - -// if validCnt == 0 { -// t.Errorf("Committed(): no partitions with valid offsets: %v", offsets) -// } -// } -// return nil -// }) -// } - -// // TestProducerConsumerTimestamps produces messages with timestamps -// // and verifies them on consumption. -// // Requires librdkafka >=0.9.4 and Kafka >=0.10.0.0 -// func (its *IntegrationTestSuite) TestProducerConsumerTimestamps() { -// t := its.T() -// numver, strver := LibraryVersion() -// if numver < 0x00090400 { -// t.Skipf("Requires librdkafka >=0.9.4 (currently on %s)", strver) -// } - -// consumerConf := ConfigMap{"bootstrap.servers": testconf.Brokers, -// "go.events.channel.enable": true, -// "group.id": testconf.Topic, -// "enable.partition.eof": true, -// } - -// consumerConf.updateFromTestconf() - -// /* Create consumer and find recognizable message, verify timestamp. -// * The consumer is started before the producer to make sure -// * the message isn't missed. */ -// t.Logf("Creating consumer") -// c, err := NewConsumer(&consumerConf) -// if err != nil { -// t.Fatalf("NewConsumer: %v", err) -// } - -// t.Logf("Assign %s [0]", testconf.Topic) -// err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, -// Offset: OffsetEnd}}) -// if err != nil { -// t.Fatalf("Assign: %v", err) -// } - -// /* Wait until EOF is reached so we dont miss the produced message */ -// for ev := range c.Events() { -// t.Logf("Awaiting initial EOF") -// _, ok := ev.(PartitionEOF) -// if ok { -// break -// } -// } - -// /* -// * Create producer and produce one recognizable message with timestamp -// */ -// producerConf := ConfigMap{"bootstrap.servers": testconf.Brokers} -// producerConf.updateFromTestconf() - -// t.Logf("Creating producer") -// p, err := NewProducer(&producerConf) -// if err != nil { -// t.Fatalf("NewProducer: %v", err) -// } - -// drChan := make(chan Event, 1) - -// /* Offset the timestamp to avoid comparison with system clock */ -// future, _ := time.ParseDuration("87658h") // 10y -// timestamp := time.Now().Add(future) -// key := fmt.Sprintf("TS: %v", timestamp) -// t.Logf("Producing message with timestamp %v", timestamp) -// err = p.Produce(&Message{ -// TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, -// Key: []byte(key), -// Timestamp: timestamp}, -// drChan) - -// if err != nil { -// t.Fatalf("Produce: %v", err) -// } - -// // Wait for delivery -// t.Logf("Awaiting delivery report") -// ev := <-drChan -// m, ok := ev.(*Message) -// if !ok { -// t.Fatalf("drChan: Expected *Message, got %v", ev) -// } -// if m.TopicPartition.Error != nil { -// t.Fatalf("Delivery failed: %v", m.TopicPartition) -// } -// t.Logf("Produced message to %v", m.TopicPartition) -// producedOffset := m.TopicPartition.Offset - -// p.Close() - -// /* Now consume messages, waiting for that recognizable one. */ -// t.Logf("Consuming messages") -// outer: -// for ev := range c.Events() { -// switch m := ev.(type) { -// case *Message: -// if m.TopicPartition.Error != nil { -// continue -// } -// if m.Key == nil || string(m.Key) != key { -// continue -// } - -// t.Logf("Found message at %v with timestamp %s %s", -// m.TopicPartition, -// m.TimestampType, m.Timestamp) - -// if m.TopicPartition.Offset != producedOffset { -// t.Fatalf("Produced Offset %d does not match consumed offset %d", producedOffset, m.TopicPartition.Offset) -// } - -// if m.TimestampType != TimestampCreateTime { -// t.Fatalf("Expected timestamp CreateTime, not %s", -// m.TimestampType) -// } - -// /* Since Kafka timestamps are milliseconds we need to -// * shave off some precision for the comparison */ -// if m.Timestamp.UnixNano()/1000000 != -// timestamp.UnixNano()/1000000 { -// t.Fatalf("Expected timestamp %v (%d), not %v (%d)", -// timestamp, timestamp.UnixNano(), -// m.Timestamp, m.Timestamp.UnixNano()) -// } -// break outer -// default: -// } -// } - -// c.Close() -// } - -// // TestProducerConsumerHeaders produces messages with headers -// // and verifies them on consumption. -// // Requires librdkafka >=0.11.4 and Kafka >=0.11.0.0 -// func (its *IntegrationTestSuite) TestProducerConsumerHeaders() { -// t := its.T() -// numver, strver := LibraryVersion() -// if numver < 0x000b0400 { -// t.Skipf("Requires librdkafka >=0.11.4 (currently on %s, 0x%x)", strver, numver) -// } - -// conf := ConfigMap{"bootstrap.servers": testconf.Brokers, -// "api.version.request": true, -// "enable.auto.commit": false, -// "group.id": testconf.Topic, -// } - -// conf.updateFromTestconf() - -// /* -// * Create producer and produce a couple of messages with and without -// * headers. -// */ -// t.Logf("Creating producer") -// p, err := NewProducer(&conf) -// if err != nil { -// t.Fatalf("NewProducer: %v", err) -// } - -// drChan := make(chan Event, 1) - -// // prepare some header values -// bigBytes := make([]byte, 2500) -// for i := 0; i < len(bigBytes); i++ { -// bigBytes[i] = byte(i) -// } - -// myVarint := make([]byte, binary.MaxVarintLen64) -// myVarintLen := binary.PutVarint(myVarint, 12345678901234) - -// expMsgHeaders := [][]Header{ -// { -// {"msgid", []byte("1")}, -// {"a key with SPACES ", bigBytes[:15]}, -// {"BIGONE!", bigBytes}, -// }, -// { -// {"msgid", []byte("2")}, -// {"myVarint", myVarint[:myVarintLen]}, -// {"empty", []byte("")}, -// {"theNullIsNil", nil}, -// }, -// nil, // no headers -// { -// {"msgid", []byte("4")}, -// {"order", []byte("1")}, -// {"order", []byte("2")}, -// {"order", nil}, -// {"order", []byte("4")}, -// }, -// } - -// t.Logf("Producing %d messages", len(expMsgHeaders)) -// for _, hdrs := range expMsgHeaders { -// err = p.Produce(&Message{ -// TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, -// Headers: hdrs}, -// drChan) -// } - -// if err != nil { -// t.Fatalf("Produce: %v", err) -// } - -// var firstOffset Offset = OffsetInvalid -// for range expMsgHeaders { -// ev := <-drChan -// m, ok := ev.(*Message) -// if !ok { -// t.Fatalf("drChan: Expected *Message, got %v", ev) -// } -// if m.TopicPartition.Error != nil { -// t.Fatalf("Delivery failed: %v", m.TopicPartition) -// } -// t.Logf("Produced message to %v", m.TopicPartition) -// if firstOffset == OffsetInvalid { -// firstOffset = m.TopicPartition.Offset -// } -// } - -// p.Close() - -// /* Now consume the produced messages and verify the headers */ -// t.Logf("Creating consumer starting at offset %v", firstOffset) -// c, err := NewConsumer(&conf) -// if err != nil { -// t.Fatalf("NewConsumer: %v", err) -// } - -// err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, -// Offset: firstOffset}}) -// if err != nil { -// t.Fatalf("Assign: %v", err) -// } - -// for n, hdrs := range expMsgHeaders { -// m, err := c.ReadMessage(-1) -// if err != nil { -// t.Fatalf("Expected message #%d, not error %v", n, err) -// } - -// if m.Headers == nil { -// if hdrs == nil { -// continue -// } -// t.Fatalf("Expected message #%d to have headers", n) -// } - -// if hdrs == nil { -// t.Fatalf("Expected message #%d not to have headers, but found %v", n, m.Headers) -// } - -// // Compare headers -// if !reflect.DeepEqual(hdrs, m.Headers) { -// t.Fatalf("Expected message #%d headers to match %v, but found %v", n, hdrs, m.Headers) -// } - -// t.Logf("Message #%d headers matched: %v", n, m.Headers) -// } - -// c.Close() - -// } +// Test consumer QueryWatermarkOffsets API +func (its *IntegrationTestSuite) TestConsumerQueryWatermarkOffsets() { + t := its.T() + + // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to + // get the number of messages in a topic + msgcnt, err := getMessageCountInTopic(testconf.Topic) + if err != nil { + t.Errorf("Cannot get message size. Error: %s\n", err) + } + + // Prime topic with test messages + createTestMessages() + producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) + + // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to + // get the number of messages in a topic + newmsgcnt, err := getMessageCountInTopic(testconf.Topic) + if err != nil { + t.Errorf("Cannot get message size. Error: %s\n", err) + } + + if newmsgcnt-msgcnt != len(p0TestMsgs) { + t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newmsgcnt-msgcnt) + } + +} + +// Test consumer GetWatermarkOffsets API +func (its *IntegrationTestSuite) TestConsumerGetWatermarkOffsets() { + t := its.T() + + // Create consumer + config := &ConfigMap{ + "go.events.channel.enable": true, + "bootstrap.servers": testconf.Brokers, + "group.id": testconf.GroupID, + "session.timeout.ms": 6000, + "enable.auto.commit": false, + "auto.offset.reset": "earliest", + } + _ = config.updateFromTestconf() + + c, err := NewConsumer(config) + if err != nil { + t.Fatalf("Unable to create consumer: %s", err) + } + defer func() { _ = c.Close() }() + + err = c.Subscribe(testconf.Topic, nil) + + // Prime topic with test messages + createTestMessages() + producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) + + // Wait for messages to be received so that we know the watermark offsets have been delivered + // with the fetch response + for ev := range c.Events() { + if _, ok := ev.(*Message); ok { + break + } + } + + _, queryHigh, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) + if err != nil { + t.Fatalf("Error querying watermark offsets: %s", err) + } + + // We are not currently testing the low watermark offset as it only gets set every 10s by the stits timer + _, getHigh, err := c.GetWatermarkOffsets(testconf.Topic, 0) + if err != nil { + t.Fatalf("Error getting watermark offsets: %s", err) + } + + if queryHigh != getHigh { + t.Errorf("QueryWatermarkOffsets high[%d] does not equal GetWatermarkOffsets high[%d]", queryHigh, getHigh) + } + +} + +// TestConsumerOffsetsForTimes +func (its *IntegrationTestSuite) TestConsumerOffsetsForTimes() { + t := its.T() + + conf := ConfigMap{"bootstrap.servers": testconf.Brokers, + "group.id": testconf.GroupID, + "api.version.request": true} + + conf.updateFromTestconf() + + c, err := NewConsumer(&conf) + + if err != nil { + panic(err) + } + defer c.Close() + + // Prime topic with test messages + createTestMessages() + producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) + + times := make([]TopicPartition, 1) + times[0] = TopicPartition{Topic: &testconf.Topic, Partition: 0, Offset: 12345} + offsets, err := c.OffsetsForTimes(times, 5000) + if err != nil { + t.Errorf("OffsetsForTimes() failed: %s\n", err) + return + } + + if len(offsets) != 1 { + t.Errorf("OffsetsForTimes() returned wrong length %d, expected 1\n", len(offsets)) + return + } + + if *offsets[0].Topic != testconf.Topic || offsets[0].Partition != 0 { + t.Errorf("OffsetsForTimes() returned wrong topic/partition\n") + return + } + + if offsets[0].Error != nil { + t.Errorf("OffsetsForTimes() returned error for partition 0: %s\n", err) + return + } + + low, _, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) + if err != nil { + t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) + return + } + + t.Logf("OffsetsForTimes() returned offset %d for timestamp %d\n", offsets[0].Offset, times[0].Offset) + + // Since we're using a phony low timestamp it is assumed that the returned + // offset will be oldest message. + if offsets[0].Offset != Offset(low) { + t.Errorf("OffsetsForTimes() returned invalid offset %d for timestamp %d, expected %d\n", offsets[0].Offset, times[0].Offset, low) + return + } + +} + +// test consumer GetMetadata API +func (its *IntegrationTestSuite) TestConsumerGetMetadata() { + t := its.T() + + config := &ConfigMap{"bootstrap.servers": testconf.Brokers, + "group.id": testconf.GroupID} + config.updateFromTestconf() + + // Create consumer + c, err := NewConsumer(config) + if err != nil { + t.Errorf("Failed to create consumer: %s\n", err) + return + } + defer c.Close() + + metaData, err := c.GetMetadata(&testconf.Topic, false, 5*1000) + if err != nil { + t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) + return + } + t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) + + metaData, err = c.GetMetadata(nil, true, 5*1000) + if err != nil { + t.Errorf("Failed to get meta data, Error: %s\n", err) + return + } + t.Logf("Meta data for consumer: %v\n", metaData) +} + +// Test producer QueryWatermarkOffsets API +func (its *IntegrationTestSuite) TestProducerQueryWatermarkOffsets() { + t := its.T() + + config := &ConfigMap{"bootstrap.servers": testconf.Brokers} + config.updateFromTestconf() + + // Create producer + p, err := NewProducer(config) + if err != nil { + t.Errorf("Failed to create producer: %s\n", err) + return + } + defer p.Close() + + low, high, err := p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) + if err != nil { + t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) + return + } + cnt := high - low + t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) + + createTestMessages() + producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) + + low, high, err = p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) + if err != nil { + t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) + return + } + t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) + newcnt := high - low + t.Logf("count = %d, New count = %d\n", cnt, newcnt) + if newcnt-cnt != int64(len(p0TestMsgs)) { + t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newcnt-cnt) + } +} + +// Test producer GetMetadata API +func (its *IntegrationTestSuite) TestProducerGetMetadata() { + t := its.T() + + config := &ConfigMap{"bootstrap.servers": testconf.Brokers} + config.updateFromTestconf() + + // Create producer + p, err := NewProducer(config) + if err != nil { + t.Errorf("Failed to create producer: %s\n", err) + return + } + defer p.Close() + + metaData, err := p.GetMetadata(&testconf.Topic, false, 5*1000) + if err != nil { + t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) + return + } + t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) + + metaData, err = p.GetMetadata(nil, true, 5*1000) + if err != nil { + t.Errorf("Failed to get meta data, Error: %s\n", err) + return + } + t.Logf("Meta data for producer: %v\n", metaData) + +} + +// test producer function-based API without delivery report +func (its *IntegrationTestSuite) TestProducerFunc() { + t := its.T() + producerTest(t, "Function producer (without DR)", + nil, producerCtrl{}, + func(p *Producer, m *Message, drChan chan Event) { + err := p.Produce(m, drChan) + if err != nil { + t.Errorf("Produce() failed: %v", err) + } + }) +} + +// test producer function-based API with delivery report +func (its *IntegrationTestSuite) TestProducerFuncDR() { + t := its.T() + producerTest(t, "Function producer (with DR)", + nil, producerCtrl{withDr: true}, + func(p *Producer, m *Message, drChan chan Event) { + err := p.Produce(m, drChan) + if err != nil { + t.Errorf("Produce() failed: %v", err) + } + }) +} + +// test producer with bad messages +func (its *IntegrationTestSuite) TestProducerWithBadMessages() { + t := its.T() + conf := ConfigMap{"bootstrap.servers": testconf.Brokers} + conf.updateFromTestconf() + + p, err := NewProducer(&conf) + if err != nil { + panic(err) + } + defer p.Close() + + // producing a nil message should return an error without crash + err = p.Produce(nil, p.Events()) + if err == nil { + t.Errorf("Producing a nil message should return error\n") + } else { + t.Logf("Producing a nil message returns expected error: %s\n", err) + } + + // producing a blank message (with nil Topic) should return an error without crash + err = p.Produce(&Message{}, p.Events()) + if err == nil { + t.Errorf("Producing a blank message should return error\n") + } else { + t.Logf("Producing a blank message returns expected error: %s\n", err) + } +} + +// test producer channel-based API without delivery report +func (its *IntegrationTestSuite) TestProducerChannel() { + t := its.T() + producerTest(t, "Channel producer (without DR)", + nil, producerCtrl{}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) +} + +// test producer channel-based API with delivery report +func (its *IntegrationTestSuite) TestProducerChannelDR() { + t := its.T() + producerTest(t, "Channel producer (with DR)", + nil, producerCtrl{withDr: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) + +} + +// test batch producer channel-based API without delivery report +func (its *IntegrationTestSuite) TestProducerBatchChannel() { + t := its.T() + producerTest(t, "Channel producer (without DR, batch channel)", + nil, producerCtrl{batchProducer: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) +} + +// test batch producer channel-based API with delivery report +func (its *IntegrationTestSuite) TestProducerBatchChannelDR() { + t := its.T() + producerTest(t, "Channel producer (DR, batch channel)", + nil, producerCtrl{withDr: true, batchProducer: true}, + func(p *Producer, m *Message, drChan chan Event) { + p.ProduceChannel() <- m + }) +} + +// test consumer channel-based API +func (its *IntegrationTestSuite) TestConsumerChannel() { + t := its.T() + consumerTestWithCommits(t, "Channel Consumer", + "", 0, true, eventTestChannelConsumer, nil) +} + +// test consumer channel-based API with incremental rebalancing +func (its *IntegrationTestSuite) TestConsumerChannelIncremental() { + t := its.T() + consumerTestWithCommits(t, "Channel Consumer Incremental", + "cooperative-sticky", 0, true, eventTestChannelConsumer, nil) +} + +// test consumer poll-based API +func (its *IntegrationTestSuite) TestConsumerPoll() { + t := its.T() + consumerTestWithCommits(t, "Poll Consumer", "", 0, false, eventTestPollConsumer, nil) +} + +// test consumer poll-based API with incremental rebalancing +func (its *IntegrationTestSuite) TestConsumerPollIncremental() { + t := its.T() + consumerTestWithCommits(t, "Poll Consumer ncremental", + "cooperative-sticky", 0, false, eventTestPollConsumer, nil) +} + +// test consumer poll-based API with rebalance callback +func (its *IntegrationTestSuite) TestConsumerPollRebalance() { + t := its.T() + consumerTestWithCommits(t, "Poll Consumer (rebalance callback)", + "", 0, false, eventTestPollConsumer, + func(c *Consumer, event Event) error { + t.Logf("Rebalanced: %s", event) + return nil + }) +} + +// test consumer poll-based API with incremental no-op rebalance callback +func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncrementalNoop() { + t := its.T() + consumerTestWithCommits(t, "Poll Consumer (incremental no-op rebalance callback)", + "cooperative-sticky", 0, false, eventTestPollConsumer, + func(c *Consumer, event Event) error { + t.Logf("Rebalanced: %s", event) + return nil + }) +} + +// test consumer poll-based API with incremental rebalance callback +func (its *IntegrationTestSuite) TestConsumerPollRebalanceIncremental() { + t := its.T() + consumerTestWithCommits(t, "Poll Consumer (incremental rebalance callback)", + "cooperative-sticky", 0, false, eventTestPollConsumer, + func(c *Consumer, event Event) error { + t.Logf("Rebalanced: %s (RebalanceProtocol=%s, AssignmentLost=%v)", + event, c.GetRebalanceProtocol(), c.AssignmentLost()) + + switch e := event.(type) { + case AssignedPartitions: + err := c.IncrementalAssign(e.Partitions) + if err != nil { + t.Errorf("IncrementalAssign() failed: %s\n", err) + return err + } + case RevokedPartitions: + err := c.IncrementalUnassign(e.Partitions) + if err != nil { + t.Errorf("IncrementalUnassign() failed: %s\n", err) + return err + } + default: + t.Fatalf("Unexpected rebalance event: %v\n", e) + } + + return nil + }) +} + +// Test Committed() API +func (its *IntegrationTestSuite) TestConsumerCommitted() { + t := its.T() + if testconf.Semaphore { + t.Skipf("Skipping TestConsumerCommitted since it is flaky[Does not run when tested with all the other integration tests]") + return + } + + consumerTestWithCommits(t, "Poll Consumer (rebalance callback, verify Committed())", + "", 0, false, eventTestPollConsumer, + func(c *Consumer, event Event) error { + t.Logf("Rebalanced: %s", event) + rp, ok := event.(RevokedPartitions) + if ok { + offsets, err := c.Committed(rp.Partitions, 5000) + if err != nil { + t.Errorf("Failed to get committed offsets: %s\n", err) + return nil + } + + t.Logf("Retrieved Committed offsets: %s\n", offsets) + + if len(offsets) != len(rp.Partitions) || len(rp.Partitions) == 0 { + t.Errorf("Invalid number of partitions %d, should be %d (and >0)\n", len(offsets), len(rp.Partitions)) + } + + // Verify proper offsets: at least one partition needs + // to have a committed offset. + validCnt := 0 + for _, p := range offsets { + if p.Error != nil { + t.Errorf("Committed() partition error: %v: %v", p, p.Error) + } else if p.Offset >= 0 { + validCnt++ + } + } + + if validCnt == 0 { + t.Errorf("Committed(): no partitions with valid offsets: %v", offsets) + } + } + return nil + }) +} + +// TestProducerConsumerTimestamps produces messages with timestamps +// and verifies them on consumption. +// Requires librdkafka >=0.9.4 and Kafka >=0.10.0.0 +func (its *IntegrationTestSuite) TestProducerConsumerTimestamps() { + t := its.T() + numver, strver := LibraryVersion() + if numver < 0x00090400 { + t.Skipf("Requires librdkafka >=0.9.4 (currently on %s)", strver) + } + + consumerConf := ConfigMap{"bootstrap.servers": testconf.Brokers, + "go.events.channel.enable": true, + "group.id": testconf.Topic, + "enable.partition.eof": true, + } + + consumerConf.updateFromTestconf() + + /* Create consumer and find recognizable message, verify timestamp. + * The consumer is started before the producer to make sure + * the message isn't missed. */ + t.Logf("Creating consumer") + c, err := NewConsumer(&consumerConf) + if err != nil { + t.Fatalf("NewConsumer: %v", err) + } + + t.Logf("Assign %s [0]", testconf.Topic) + err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, + Offset: OffsetEnd}}) + if err != nil { + t.Fatalf("Assign: %v", err) + } + + /* Wait until EOF is reached so we dont miss the produced message */ + for ev := range c.Events() { + t.Logf("Awaiting initial EOF") + _, ok := ev.(PartitionEOF) + if ok { + break + } + } + + /* + * Create producer and produce one recognizable message with timestamp + */ + producerConf := ConfigMap{"bootstrap.servers": testconf.Brokers} + producerConf.updateFromTestconf() + + t.Logf("Creating producer") + p, err := NewProducer(&producerConf) + if err != nil { + t.Fatalf("NewProducer: %v", err) + } + + drChan := make(chan Event, 1) + + /* Offset the timestamp to avoid comparison with system clock */ + future, _ := time.ParseDuration("87658h") // 10y + timestamp := time.Now().Add(future) + key := fmt.Sprintf("TS: %v", timestamp) + t.Logf("Producing message with timestamp %v", timestamp) + err = p.Produce(&Message{ + TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, + Key: []byte(key), + Timestamp: timestamp}, + drChan) + + if err != nil { + t.Fatalf("Produce: %v", err) + } + + // Wait for delivery + t.Logf("Awaiting delivery report") + ev := <-drChan + m, ok := ev.(*Message) + if !ok { + t.Fatalf("drChan: Expected *Message, got %v", ev) + } + if m.TopicPartition.Error != nil { + t.Fatalf("Delivery failed: %v", m.TopicPartition) + } + t.Logf("Produced message to %v", m.TopicPartition) + producedOffset := m.TopicPartition.Offset + + p.Close() + + /* Now consume messages, waiting for that recognizable one. */ + t.Logf("Consuming messages") +outer: + for ev := range c.Events() { + switch m := ev.(type) { + case *Message: + if m.TopicPartition.Error != nil { + continue + } + if m.Key == nil || string(m.Key) != key { + continue + } + + t.Logf("Found message at %v with timestamp %s %s", + m.TopicPartition, + m.TimestampType, m.Timestamp) + + if m.TopicPartition.Offset != producedOffset { + t.Fatalf("Produced Offset %d does not match consumed offset %d", producedOffset, m.TopicPartition.Offset) + } + + if m.TimestampType != TimestampCreateTime { + t.Fatalf("Expected timestamp CreateTime, not %s", + m.TimestampType) + } + + /* Since Kafka timestamps are milliseconds we need to + * shave off some precision for the comparison */ + if m.Timestamp.UnixNano()/1000000 != + timestamp.UnixNano()/1000000 { + t.Fatalf("Expected timestamp %v (%d), not %v (%d)", + timestamp, timestamp.UnixNano(), + m.Timestamp, m.Timestamp.UnixNano()) + } + break outer + default: + } + } + + c.Close() +} + +// TestProducerConsumerHeaders produces messages with headers +// and verifies them on consumption. +// Requires librdkafka >=0.11.4 and Kafka >=0.11.0.0 +func (its *IntegrationTestSuite) TestProducerConsumerHeaders() { + t := its.T() + numver, strver := LibraryVersion() + if numver < 0x000b0400 { + t.Skipf("Requires librdkafka >=0.11.4 (currently on %s, 0x%x)", strver, numver) + } + + conf := ConfigMap{"bootstrap.servers": testconf.Brokers, + "api.version.request": true, + "enable.auto.commit": false, + "group.id": testconf.Topic, + } + + conf.updateFromTestconf() + + /* + * Create producer and produce a couple of messages with and without + * headers. + */ + t.Logf("Creating producer") + p, err := NewProducer(&conf) + if err != nil { + t.Fatalf("NewProducer: %v", err) + } + + drChan := make(chan Event, 1) + + // prepare some header values + bigBytes := make([]byte, 2500) + for i := 0; i < len(bigBytes); i++ { + bigBytes[i] = byte(i) + } + + myVarint := make([]byte, binary.MaxVarintLen64) + myVarintLen := binary.PutVarint(myVarint, 12345678901234) + + expMsgHeaders := [][]Header{ + { + {"msgid", []byte("1")}, + {"a key with SPACES ", bigBytes[:15]}, + {"BIGONE!", bigBytes}, + }, + { + {"msgid", []byte("2")}, + {"myVarint", myVarint[:myVarintLen]}, + {"empty", []byte("")}, + {"theNullIsNil", nil}, + }, + nil, // no headers + { + {"msgid", []byte("4")}, + {"order", []byte("1")}, + {"order", []byte("2")}, + {"order", nil}, + {"order", []byte("4")}, + }, + } + + t.Logf("Producing %d messages", len(expMsgHeaders)) + for _, hdrs := range expMsgHeaders { + err = p.Produce(&Message{ + TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, + Headers: hdrs}, + drChan) + } + + if err != nil { + t.Fatalf("Produce: %v", err) + } + + var firstOffset Offset = OffsetInvalid + for range expMsgHeaders { + ev := <-drChan + m, ok := ev.(*Message) + if !ok { + t.Fatalf("drChan: Expected *Message, got %v", ev) + } + if m.TopicPartition.Error != nil { + t.Fatalf("Delivery failed: %v", m.TopicPartition) + } + t.Logf("Produced message to %v", m.TopicPartition) + if firstOffset == OffsetInvalid { + firstOffset = m.TopicPartition.Offset + } + } + + p.Close() + + /* Now consume the produced messages and verify the headers */ + t.Logf("Creating consumer starting at offset %v", firstOffset) + c, err := NewConsumer(&conf) + if err != nil { + t.Fatalf("NewConsumer: %v", err) + } + + err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, + Offset: firstOffset}}) + if err != nil { + t.Fatalf("Assign: %v", err) + } + + for n, hdrs := range expMsgHeaders { + m, err := c.ReadMessage(-1) + if err != nil { + t.Fatalf("Expected message #%d, not error %v", n, err) + } + + if m.Headers == nil { + if hdrs == nil { + continue + } + t.Fatalf("Expected message #%d to have headers", n) + } + + if hdrs == nil { + t.Fatalf("Expected message #%d not to have headers, but found %v", n, m.Headers) + } + + // Compare headers + if !reflect.DeepEqual(hdrs, m.Headers) { + t.Fatalf("Expected message #%d headers to match %v, but found %v", n, hdrs, m.Headers) + } + + t.Logf("Message #%d headers matched: %v", n, m.Headers) + } + + c.Close() + +} func TestIntegration(t *testing.T) { its := new(IntegrationTestSuite) From 1a7b40cafae33f42224987960ecfc72ee76e81d1 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 11 Apr 2023 10:39:05 +0530 Subject: [PATCH 03/12] adminapi_test needs reviews --- .../admin_describe_cluster.go | 4 - .../admin_describe_consumer_groups.go | 4 - .../admin_describe_topics.go | 4 - kafka/adminapi_test.go | 374 +++++++++--------- 4 files changed, 187 insertions(+), 199 deletions(-) diff --git a/examples/admin_describe_cluster/admin_describe_cluster.go b/examples/admin_describe_cluster/admin_describe_cluster.go index 6b6a66be7..8cececa59 100644 --- a/examples/admin_describe_cluster/admin_describe_cluster.go +++ b/examples/admin_describe_cluster/admin_describe_cluster.go @@ -47,10 +47,6 @@ func main() { // Create a new AdminClient. a, err := kafka.NewAdminClient(&kafka.ConfigMap{ "bootstrap.servers": bootstrapServers, - "sasl.mechanisms": "SCRAM-SHA-256", - "security.protocol": "SASL_PLAINTEXT", - "sasl.username": "broker", - "sasl.password": "broker", }) if err != nil { fmt.Printf("Failed to create Admin client: %s\n", err) diff --git a/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go b/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go index ea09d9cdd..8aff82178 100644 --- a/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go +++ b/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go @@ -50,10 +50,6 @@ func main() { // Create a new AdminClient. a, err := kafka.NewAdminClient(&kafka.ConfigMap{ "bootstrap.servers": bootstrapServers, - "sasl.mechanisms": "SCRAM-SHA-256", - "security.protocol": "SASL_PLAINTEXT", - "sasl.username": "broker", - "sasl.password": "broker", }) if err != nil { fmt.Printf("Failed to create Admin client: %s\n", err) diff --git a/examples/admin_describe_topics/admin_describe_topics.go b/examples/admin_describe_topics/admin_describe_topics.go index 6c2274706..9241b22c4 100644 --- a/examples/admin_describe_topics/admin_describe_topics.go +++ b/examples/admin_describe_topics/admin_describe_topics.go @@ -49,10 +49,6 @@ func main() { // Create a new AdminClient. a, err := kafka.NewAdminClient(&kafka.ConfigMap{ "bootstrap.servers": bootstrapServers, - "sasl.mechanisms": "SCRAM-SHA-256", - "security.protocol": "SASL_PLAINTEXT", - "sasl.username": "broker", - "sasl.password": "broker", }) if err != nil { fmt.Printf("Failed to create Admin client: %s\n", err) diff --git a/kafka/adminapi_test.go b/kafka/adminapi_test.go index 6d1abf53e..919a7485b 100644 --- a/kafka/adminapi_test.go +++ b/kafka/adminapi_test.go @@ -645,197 +645,197 @@ func testAdminAPIs(what string, a *AdminClient, t *testing.T) { t.Fatalf("Expected DeadlineExceeded, not %v, %v", ctx.Err(), err) } - // // Incorrect input, fail with ErrInvalidArg - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // res, err = a.CreateTopics( - // ctx, - // []TopicSpecification{ - // { - // // Must not specify both ReplicationFactor and ReplicaAssignment - // Topic: "mytopic", - // NumPartitions: 2, - // ReplicationFactor: 3, - // ReplicaAssignment: [][]int32{ - // []int32{1, 2, 3}, - // []int32{3, 2, 1}, - // }, - // }, - // }) - // if res != nil || err == nil { - // t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) - // } - // if ctx.Err() != nil { - // t.Fatalf("Did not expect context to fail: %v", ctx.Err()) - // } - // if err.(Error).Code() != ErrInvalidArg { - // t.Fatalf("Expected ErrInvalidArg, not %v", err) - // } - - // // Incorrect input, fail with ErrInvalidArg - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // res, err = a.CreateTopics( - // ctx, - // []TopicSpecification{ - // { - // // ReplicaAssignment must be same length as Numpartitions - // Topic: "mytopic", - // NumPartitions: 7, - // ReplicaAssignment: [][]int32{ - // []int32{1, 2, 3}, - // []int32{3, 2, 1}, - // }, - // }, - // }) - // if res != nil || err == nil { - // t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) - // } - // if ctx.Err() != nil { - // t.Fatalf("Did not expect context to fail: %v", ctx.Err()) - // } - // if err.(Error).Code() != ErrInvalidArg { - // t.Fatalf("Expected ErrInvalidArg, not %v", err) - // } - - // // Correct input, using options - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // res, err = a.CreateTopics( - // ctx, - // []TopicSpecification{ - // { - // Topic: "mytopic4", - // NumPartitions: 9, - // ReplicaAssignment: [][]int32{ - // []int32{1}, - // []int32{2}, - // []int32{3}, - // []int32{4}, - // []int32{1}, - // []int32{2}, - // []int32{3}, - // []int32{4}, - // []int32{1}, - // }, - // Config: map[string]string{ - // "some.topic.config": "unchecked", - // "these.are.verified": "on the broker", - // "and.this.is": "just", - // "a": "unit test", - // }, - // }, - // }, - // SetAdminValidateOnly(false)) - // if res != nil || err == nil { - // t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) - // } - // if ctx.Err() != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - // } - - // // - // // Remaining APIs - // // Timeout code is identical for all APIs, no need to test - // // them for each API. - // // - - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // res, err = a.CreatePartitions( - // ctx, - // []PartitionsSpecification{ - // { - // Topic: "topic", - // IncreaseTo: 19, - // ReplicaAssignment: [][]int32{ - // []int32{3234522}, - // []int32{99999}, - // }, - // }, - // { - // Topic: "topic2", - // IncreaseTo: 2, - // ReplicaAssignment: [][]int32{ - // []int32{99999}, - // }, - // }, - // }) - // if res != nil || err == nil { - // t.Fatalf("Expected CreatePartitions to fail, but got result: %v, err: %v", res, err) - // } - // if ctx.Err() != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - // } - - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // res, err = a.DeleteTopics( - // ctx, - // []string{"topic1", "topic2"}) - // if res != nil || err == nil { - // t.Fatalf("Expected DeleteTopics to fail, but got result: %v, err: %v", res, err) - // } - // if ctx.Err() != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v for error %v", ctx.Err(), err) - // } - - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // cres, err := a.AlterConfigs( - // ctx, - // []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) - // if cres != nil || err == nil { - // t.Fatalf("Expected AlterConfigs to fail, but got result: %v, err: %v", cres, err) - // } - // if ctx.Err() != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - // } - - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // cres, err = a.DescribeConfigs( - // ctx, - // []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) - // if cres != nil || err == nil { - // t.Fatalf("Expected DescribeConfigs to fail, but got result: %v, err: %v", cres, err) - // } - // if ctx.Err() != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - // } - - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // clusterID, err := a.ClusterID(ctx) - // if err == nil { - // t.Fatalf("Expected ClusterID to fail, but got result: %v", clusterID) - // } - // if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - // } - - // ctx, cancel = context.WithTimeout(context.Background(), expDuration) - // defer cancel() - // controllerID, err := a.ControllerID(ctx) - // if err == nil { - // t.Fatalf("Expected ControllerID to fail, but got result: %v", controllerID) - // } - // if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { - // t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) - // } - - // testAdminAPIsCreateACLs(what, a, t) - // testAdminAPIsDescribeACLs(what, a, t) - // testAdminAPIsDeleteACLs(what, a, t) - - // testAdminAPIsListConsumerGroups(what, a, expDuration, t) + // Incorrect input, fail with ErrInvalidArg + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + res, err = a.CreateTopics( + ctx, + []TopicSpecification{ + { + // Must not specify both ReplicationFactor and ReplicaAssignment + Topic: "mytopic", + NumPartitions: 2, + ReplicationFactor: 3, + ReplicaAssignment: [][]int32{ + []int32{1, 2, 3}, + []int32{3, 2, 1}, + }, + }, + }) + if res != nil || err == nil { + t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) + } + if ctx.Err() != nil { + t.Fatalf("Did not expect context to fail: %v", ctx.Err()) + } + if err.(Error).Code() != ErrInvalidArg { + t.Fatalf("Expected ErrInvalidArg, not %v", err) + } + + // Incorrect input, fail with ErrInvalidArg + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + res, err = a.CreateTopics( + ctx, + []TopicSpecification{ + { + // ReplicaAssignment must be same length as Numpartitions + Topic: "mytopic", + NumPartitions: 7, + ReplicaAssignment: [][]int32{ + []int32{1, 2, 3}, + []int32{3, 2, 1}, + }, + }, + }) + if res != nil || err == nil { + t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) + } + if ctx.Err() != nil { + t.Fatalf("Did not expect context to fail: %v", ctx.Err()) + } + if err.(Error).Code() != ErrInvalidArg { + t.Fatalf("Expected ErrInvalidArg, not %v", err) + } + + // Correct input, using options + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + res, err = a.CreateTopics( + ctx, + []TopicSpecification{ + { + Topic: "mytopic4", + NumPartitions: 9, + ReplicaAssignment: [][]int32{ + []int32{1}, + []int32{2}, + []int32{3}, + []int32{4}, + []int32{1}, + []int32{2}, + []int32{3}, + []int32{4}, + []int32{1}, + }, + Config: map[string]string{ + "some.topic.config": "unchecked", + "these.are.verified": "on the broker", + "and.this.is": "just", + "a": "unit test", + }, + }, + }, + SetAdminValidateOnly(false)) + if res != nil || err == nil { + t.Fatalf("Expected CreateTopics to fail, but got result: %v, err: %v", res, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + } + + // + // Remaining APIs + // Timeout code is identical for all APIs, no need to test + // them for each API. + // + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + res, err = a.CreatePartitions( + ctx, + []PartitionsSpecification{ + { + Topic: "topic", + IncreaseTo: 19, + ReplicaAssignment: [][]int32{ + []int32{3234522}, + []int32{99999}, + }, + }, + { + Topic: "topic2", + IncreaseTo: 2, + ReplicaAssignment: [][]int32{ + []int32{99999}, + }, + }, + }) + if res != nil || err == nil { + t.Fatalf("Expected CreatePartitions to fail, but got result: %v, err: %v", res, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + } + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + res, err = a.DeleteTopics( + ctx, + []string{"topic1", "topic2"}) + if res != nil || err == nil { + t.Fatalf("Expected DeleteTopics to fail, but got result: %v, err: %v", res, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v for error %v", ctx.Err(), err) + } + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + cres, err := a.AlterConfigs( + ctx, + []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) + if cres != nil || err == nil { + t.Fatalf("Expected AlterConfigs to fail, but got result: %v, err: %v", cres, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + } + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + cres, err = a.DescribeConfigs( + ctx, + []ConfigResource{{Type: ResourceTopic, Name: "topic"}}) + if cres != nil || err == nil { + t.Fatalf("Expected DescribeConfigs to fail, but got result: %v, err: %v", cres, err) + } + if ctx.Err() != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + } + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + clusterID, err := a.ClusterID(ctx) + if err == nil { + t.Fatalf("Expected ClusterID to fail, but got result: %v", clusterID) + } + if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + } + + ctx, cancel = context.WithTimeout(context.Background(), expDuration) + defer cancel() + controllerID, err := a.ControllerID(ctx) + if err == nil { + t.Fatalf("Expected ControllerID to fail, but got result: %v", controllerID) + } + if ctx.Err() != context.DeadlineExceeded || err != context.DeadlineExceeded { + t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err()) + } + + testAdminAPIsCreateACLs(what, a, t) + testAdminAPIsDescribeACLs(what, a, t) + testAdminAPIsDeleteACLs(what, a, t) + + testAdminAPIsListConsumerGroups(what, a, expDuration, t) testAdminAPIsDescribeConsumerGroups(what, a, expDuration, t) testAdminAPIsDescribeTopics(what, a, expDuration, t) testAdminAPIsDescribeCluster(what, a, expDuration, t) - // testAdminAPIsDeleteConsumerGroups(what, a, expDuration, t) + testAdminAPIsDeleteConsumerGroups(what, a, expDuration, t) - // testAdminAPIsListConsumerGroupOffsets(what, a, expDuration, t) - // testAdminAPIsAlterConsumerGroupOffsets(what, a, expDuration, t) + testAdminAPIsListConsumerGroupOffsets(what, a, expDuration, t) + testAdminAPIsAlterConsumerGroupOffsets(what, a, expDuration, t) } // TestAdminAPIs dry-tests most Admin APIs, no broker is needed. From fd994b446e00faa85bfab70f4b27067ca03bd9f8 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 5 Sep 2023 11:43:39 +0530 Subject: [PATCH 04/12] Address public API changes in librdkafka PR --- .../admin_describe_cluster.go | 28 +- .../admin_describe_topics.go | 44 +-- kafka/adminapi.go | 287 ++++++++++-------- kafka/adminoptions.go | 90 +----- kafka/kafka.go | 2 + kafka/testresources/docker-compose.yaml | 3 +- 6 files changed, 195 insertions(+), 259 deletions(-) diff --git a/examples/admin_describe_cluster/admin_describe_cluster.go b/examples/admin_describe_cluster/admin_describe_cluster.go index 8cececa59..29219d21c 100644 --- a/examples/admin_describe_cluster/admin_describe_cluster.go +++ b/examples/admin_describe_cluster/admin_describe_cluster.go @@ -31,16 +31,16 @@ func main() { if len(os.Args) < 3 { fmt.Fprintf( os.Stderr, - "Usage: %s [ ...]\n", os.Args[0]) os.Exit(1) } bootstrapServers := os.Args[1] - include_topic_authorized_operations, err_operations := strconv.ParseBool(os.Args[2]) + include_authorized_operations, err_operations := strconv.ParseBool(os.Args[2]) if err_operations != nil { fmt.Printf( - "Failed to parse value of include_topic_authorized_operations %s: %s\n", os.Args[2], err_operations) + "Failed to parse value of include_authorized_operations %s: %s\n", os.Args[2], err_operations) os.Exit(1) } topics := os.Args[3:] @@ -60,8 +60,8 @@ func main() { ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() describeTopicsResult, err := a.DescribeTopics( - ctx, topics, kafka.SetAdminOptionIncludeTopicAuthorizedOperations( - include_topic_authorized_operations)) + ctx, topics, kafka.SetAdminOptionIncludeAuthorizedOperations( + include_authorized_operations)) if err != nil { fmt.Printf("Failed to describe topics: %s\n", err) os.Exit(1) @@ -78,33 +78,17 @@ func main() { } fmt.Printf("Topic: %s has succeeded\n", t.Topic) - if include_topic_authorized_operations == true { - fmt.Printf("Allowed acl operations:\n") - for i := 0; i < len(t.TopicAuthorizedOperations); i++ { - fmt.Printf("\t%s\n", t.TopicAuthorizedOperations[i]) - } + if include_authorized_operations == true { + fmt.Printf("Allowed operations: %s\n", t.AuthorizedOperations) } for i := 0; i < len(t.Partitions); i++ { - if t.Partitions[i].Error.Code() != 0 { - fmt.Printf("Partition with id: %d"+ - "has error: %s\n\n", - t.Partitions[i].Id, t.Partitions[i].Error) - continue - } - fmt.Printf("Partition id: %d with leader: %d\n", - t.Partitions[i].Id, t.Partitions[i].Leader) - fmt.Printf("The in-sync replica count is: %d, they are: ", - len(t.Partitions[i].ISRs)) - for j := 0; j < len(t.Partitions[i].ISRs); j++ { - fmt.Printf("%d ", t.Partitions[i].ISRs[j]) - } - fmt.Printf("\n") - fmt.Printf("The replica count is: %d, they are: ", - len(t.Partitions[i].Replicas)) - for j := 0; j < len(t.Partitions[i].Replicas); j++ { - fmt.Printf("%d ", t.Partitions[i].Replicas[j]) - } - fmt.Printf("\n\n") + 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") } diff --git a/kafka/adminapi.go b/kafka/adminapi.go index 6879f667a..f6b4e0a63 100644 --- a/kafka/adminapi.go +++ b/kafka/adminapi.go @@ -99,6 +99,25 @@ TopicDescription_by_idx(const rd_kafka_TopicDescription_t **result_topics, size_ return result_topics[idx]; } +static const rd_kafka_TopicPartitionInfo_t * +TopicPartitionInfo_by_idx(const rd_kafka_TopicPartitionInfo_t **partitions, size_t cnt, size_t idx) { + if (idx >= cnt) + return NULL; + return partitions[idx]; +} + +static const rd_kafka_AclOperation_t AclOperation_by_idx(const rd_kafka_AclOperation_t *acl_operations, size_t cnt, size_t idx) { + if (idx >= cnt) + return RD_KAFKA_ACL_OPERATION_UNKNOWN; + return acl_operations[idx]; +} + +static const rd_kafka_Node_t *Node_by_idx(const rd_kafka_Node_t **nodes, size_t cnt, size_t idx) { + if (idx >= cnt) + return NULL; + return nodes[idx]; +} + static const rd_kafka_UserScramCredentialsDescription_t * DescribeUserScramCredentials_result_description_by_idx(const rd_kafka_UserScramCredentialsDescription_t **descriptions, size_t cnt, size_t idx) { if (idx >= cnt) @@ -269,7 +288,7 @@ type ConsumerGroupDescription struct { Coordinator Node // Members list. Members []MemberDescription - // Acl operations allowed list + // Operations allowed for group AuthorizedOperations []ACLOperation } @@ -280,18 +299,16 @@ type DescribeConsumerGroupsResult struct { ConsumerGroupDescriptions []ConsumerGroupDescription } -// Partition information -type PartitionInfo struct { - // Partition Id. - Id int - // Error, if any, of partition. Check with `Error.Code() != ErrNoError. - Error Error +// Topic Partition information +type TopicPartitionInfo struct { + // Partition id. + Partition int // Leader broker. - Leader int - // Replica broker. - Replicas []int - // In-Sync-Replica brokers. - ISRs []int + Leader *Node + // Replicas of partition. + Replicas []Node + // In-Sync-Replicas of partition. + Isr []Node } // TopicDescription represents the result of DescribeTopics for @@ -301,10 +318,12 @@ type TopicDescription struct { Topic string // Error, if any, of result. Check with `Error.Code() != ErrNoError`. Error Error + // Is the topic is internal to Kafka? + IsInternal bool // Partitions' information list. - Partitions []PartitionInfo - // Topic Acl operations allowed list - TopicAuthorizedOperations []ACLOperation + Partitions []TopicPartitionInfo + // Operations allowed for topic. + AuthorizedOperations []ACLOperation } // DescribeTopicsResult represents the result of a @@ -314,17 +333,16 @@ type DescribeTopicsResult struct { TopicDescriptions []TopicDescription } -// ClusterDescription represents the result of DescribeCluster for -// the cluster. -type ClusterDescription struct { - // Current cluster id in cluster. - ClusterID string - // Current controller id in cluster. - ControllerID int - // Slice of nodes in cluster. +// DescribeClusterResult represents the result of DescribeCluster. +type DescribeClusterResult struct { + // Cluster id for cluster. + ClusterId string + // Current controller node for cluster. + Controller *Node + // List of nodes in cluster. Nodes []Node - // Cluster Acl operations allowed slice. - ClusterAuthorizedOperations []ACLOperation + // Operations allowed for cluster. + AuthorizedOperations []ACLOperation } // DeleteConsumerGroupsResult represents the result of a DeleteConsumerGroups @@ -1046,6 +1064,40 @@ func (a *AdminClient) cToTopicResults(cTopicRes **C.rd_kafka_topic_result_t, cCn return result, nil } +func (a *AdminClient) cToAuthorizedOperations( + cAuthorizedOperations *C.rd_kafka_AclOperation_t, cAuthorizedOperationCnt C.size_t) []ACLOperation { + if cAuthorizedOperations == nil { + return nil + } + + authorizedOperations := make([]ACLOperation, int(cAuthorizedOperationCnt)) + for i := 0; i < int(cAuthorizedOperationCnt); i++ { + cAuthorizedOperation := C.AclOperation_by_idx( + cAuthorizedOperations, cAuthorizedOperationCnt, C.size_t(i)) + authorizedOperations[i] = ACLOperation(cAuthorizedOperation) + } + + return authorizedOperations +} + +func (a *AdminClient) cToNode(cNode *C.rd_kafka_Node_t) Node { + node := Node{ID: -1} + if cNode == nil { + return node + } + + node.Host = C.GoString(C.rd_kafka_Node_host(cNode)) + node.Port = int(C.rd_kafka_Node_port(cNode)) + + cRack := C.rd_kafka_Node_rack_id(cNode) + if cRack != nil { + rackId := C.GoString(cRack) + node.RackId = &rackId + } + + return node +} + // cToConsumerGroupDescriptions converts a C rd_kafka_ConsumerGroupDescription_t // array to a Go ConsumerGroupDescription slice. func (a *AdminClient) cToConsumerGroupDescriptions( @@ -1068,11 +1120,7 @@ func (a *AdminClient) cToConsumerGroupDescriptions( C.rd_kafka_ConsumerGroupDescription_state(cGroup)) cNode := C.rd_kafka_ConsumerGroupDescription_coordinator(cGroup) - coordinator := Node{ - ID: int(C.rd_kafka_Node_id(cNode)), - Host: C.GoString(C.rd_kafka_Node_host(cNode)), - Port: int(C.rd_kafka_Node_port(cNode)), - } + coordinator := a.cToNode(cNode) membersCount := int( C.rd_kafka_ConsumerGroupDescription_member_count(cGroup)) @@ -1102,14 +1150,10 @@ func (a *AdminClient) cToConsumerGroupDescriptions( } } - acl_operations_cnt := int( - C.rd_kafka_ConsumerGroupDescription_authorized_operations_count(cGroup)) - authorizedOperations := make([]ACLOperation, acl_operations_cnt) - for aclidx := 0; aclidx < acl_operations_cnt; aclidx++ { - aclop := int( - C.rd_kafka_ConsumerGroupDescription_authorized_operation(cGroup, C.size_t(aclidx))) - authorizedOperations[aclidx] = ACLOperation(aclop) - } + cAuthorizedOperationsCnt := C.size_t(0) + cAuthorizedOperations := C.rd_kafka_ConsumerGroupDescription_authorized_operations( + cGroup, &cAuthorizedOperationsCnt) + authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, cAuthorizedOperationsCnt) result[idx] = ConsumerGroupDescription{ GroupID: groupID, @@ -1125,6 +1169,38 @@ func (a *AdminClient) cToConsumerGroupDescriptions( return result } +func (a *AdminClient) cToNodes(cNodes **C.rd_kafka_Node_t, cNodeCnt C.size_t) []Node { + nodes := make([]Node, int(cNodeCnt)) + for i := 0; i < int(cNodeCnt); i++ { + cNode := C.Node_by_idx(cNodes, cNodeCnt, C.size_t(i)) + nodes[i] = a.cToNode(cNode) + } + return nodes +} + +func (a *AdminClient) cToTopicPartitionInfo(partitionInfo *C.rd_kafka_TopicPartitionInfo_t) TopicPartitionInfo { + cPartitionId := C.rd_kafka_TopicPartitionInfo_partition(partitionInfo) + info := TopicPartitionInfo{ + Partition: int(cPartitionId), + } + + cLeader := C.rd_kafka_TopicPartitionInfo_leader(partitionInfo) + if cLeader != nil { + leader := a.cToNode(cLeader) + info.Leader = &leader + } + + cReplicaCnt := C.size_t(0) + cReplicas := C.rd_kafka_TopicPartitionInfo_replicas(partitionInfo, &cReplicaCnt) + info.Replicas = a.cToNodes(cReplicas, cReplicaCnt) + + cIsrCnt := C.size_t(0) + cIsr := C.rd_kafka_TopicPartitionInfo_isr(partitionInfo, &cIsrCnt) + info.Isr = a.cToNodes(cIsr, cIsrCnt) + + return info +} + // cToTopicDescriptions converts a C rd_kafka_TopicDescription_t // array to a Go TopicDescription slice. func (a *AdminClient) cToTopicDescriptions( @@ -1136,58 +1212,30 @@ func (a *AdminClient) cToTopicDescriptions( cTopics, cTopicCount, C.size_t(idx)) topic := C.GoString( - C.rd_kafka_TopicDescription_topic_name(cTopic)) + C.rd_kafka_TopicDescription_name(cTopic)) err := newErrorFromCError( C.rd_kafka_TopicDescription_error(cTopic)) - partition_cnt := int( - C.rd_kafka_TopicDescription_topic_partition_cnt(cTopic)) - - partitions := make([]PartitionInfo, partition_cnt) - - for pidx := 0; pidx < partition_cnt; pidx++ { - id := int( - C.rd_kafka_TopicDescription_partiton_id(cTopic, C.int(pidx))) - leader := int( - C.rd_kafka_TopicDescription_partiton_leader(cTopic, C.int(pidx))) - partition_error := newErrorFromCError( - C.rd_kafka_TopicDescription_partition_error(cTopic, C.int(pidx))) - replicas_cnt := int( - C.rd_kafka_TopicDescription_partiton_replica_cnt(cTopic, C.int(pidx))) - isrs_cnt := int( - C.rd_kafka_TopicDescription_partiton_isr_cnt(cTopic, C.int(pidx))) - replicas := make([]int, replicas_cnt) - isrs := make([]int, isrs_cnt) - for ridx := 0; ridx < replicas_cnt; ridx++ { - replicas[ridx] = int( - C.rd_kafka_TopicDescription_partiton_replica_idx(cTopic, C.int(pidx), C.int(ridx))) - } - for isridx := 0; isridx < isrs_cnt; isridx++ { - isrs[isridx] = int( - C.rd_kafka_TopicDescription_partiton_isrs_idx(cTopic, C.int(pidx), C.int(isridx))) - } - partitions[pidx] = PartitionInfo{ - Id: id, - Error: partition_error, - Leader: leader, - Replicas: replicas, - ISRs: isrs, - } - } - topic_authorized_operations_cnt := int( - C.rd_kafka_TopicDescription_topic_authorized_operations_cnt(cTopic)) - topicauthorizedOperations := make([]ACLOperation, topic_authorized_operations_cnt) - for aclidx := 0; aclidx < topic_authorized_operations_cnt; aclidx++ { - aclop := int( - C.rd_kafka_TopicDescription_authorized_operation_idx(cTopic, C.size_t(aclidx))) - topicauthorizedOperations[aclidx] = ACLOperation(aclop) + cPartitionInfoCnt := C.size_t(0) + cPartitionInfos := C.rd_kafka_TopicDescription_partitions(cTopic, &cPartitionInfoCnt) + + partitions := make([]TopicPartitionInfo, int(cPartitionInfoCnt)) + + for pidx := 0; pidx < int(cPartitionInfoCnt); pidx++ { + cPartitionInfo := C.TopicPartitionInfo_by_idx(cPartitionInfos, cPartitionInfoCnt, C.size_t(pidx)) + partitions[pidx] = a.cToTopicPartitionInfo(cPartitionInfo) } + cAuthorizedOperationsCnt := C.size_t(0) + cAuthorizedOperations := C.rd_kafka_TopicDescription_authorized_operations( + cTopic, &cAuthorizedOperationsCnt) + authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, cAuthorizedOperationsCnt) + result[idx] = TopicDescription{ - Topic: topic, - Error: err, - Partitions: partitions, - TopicAuthorizedOperations: topicauthorizedOperations, + Topic: topic, + Error: err, + Partitions: partitions, + AuthorizedOperations: authorizedOperations, } } return result @@ -1196,42 +1244,31 @@ func (a *AdminClient) cToTopicDescriptions( // cToClusterDescriptions converts a C rd_kafka_TopicDescription_t // to a Go ClusterDescription. func (a *AdminClient) cToClusterDescription( - cClusterDesc *C.rd_kafka_ClusterDescription_t) (result ClusterDescription) { - clusterID := C.GoString( - C.rd_kafka_ClusterDescription_cluster_id(cClusterDesc)) - controllerID := int( - C.rd_kafka_ClusterDescription_controller_id(cClusterDesc)) - node_cnt := int( - C.rd_kafka_ClusterDescription_node_cnt(cClusterDesc)) - nodes := make([]Node, node_cnt) - for nidx := 0; nidx < node_cnt; nidx++ { - cNode := C.rd_kafka_ClusterDescription_node_idx(cClusterDesc, C.int(nidx)) - id := int(C.rd_kafka_Node_id(cNode)) - host := C.GoString(C.rd_kafka_Node_host(cNode)) - port := int(C.rd_kafka_Node_port(cNode)) - nodes[nidx] = Node{ - ID: id, - Host: host, - Port: port, - } - } - cluster_authorized_operations_cnt := int( - C.rd_kafka_ClusterDescription_cluster_acl_operations_cnt(cClusterDesc)) - clusterAuthorizedOperations := make([]ACLOperation, cluster_authorized_operations_cnt) - for aclidx := 0; aclidx < cluster_authorized_operations_cnt; aclidx++ { - aclop := int( - C.rd_kafka_ClusterDescription_authorized_operation_idx(cClusterDesc, C.size_t(aclidx))) - clusterAuthorizedOperations[aclidx] = ACLOperation(aclop) - } + cResult *C.rd_kafka_DescribeTopics_result_t) (result DescribeClusterResult) { + clusterID := C.GoString(C.rd_kafka_DescribeCluster_result_cluster_id(cResult)) - result = ClusterDescription{ - ClusterID: clusterID, - ControllerID: controllerID, - Nodes: nodes, - ClusterAuthorizedOperations: clusterAuthorizedOperations, + var controller *Node = nil + cController := C.rd_kafka_DescribeCluster_result_controller(cResult) + if cController != nil { + controllerValue := a.cToNode(cController) + controller = &controllerValue } - return result + cNodeCnt := C.size_t(0) + cNodes := C.rd_kafka_DescribeCluster_result_nodes(cResult, &cNodeCnt) + nodes := a.cToNodes(cNodes, cNodeCnt) + + cAuthorizedOperationsCnt := C.size_t(0) + cAuthorizedOperations := C.rd_kafka_DescribeCluster_result_authorized_operations( + cResult, &cAuthorizedOperationsCnt) + authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, cAuthorizedOperationsCnt) + + return DescribeClusterResult{ + ClusterId: clusterID, + Controller: controller, + Nodes: nodes, + AuthorizedOperations: authorizedOperations, + } } // cToDescribeUserScramCredentialsResult converts a C @@ -2585,7 +2622,7 @@ func (a *AdminClient) DescribeTopics( return result, err } - // Convert topic names into char** required by the implementation. + // Convert topic names into char**. cTopicNameList := make([]*C.char, len(topics)) cTopicNameCount := C.size_t(len(topics)) @@ -2599,6 +2636,9 @@ func (a *AdminClient) DescribeTopics( cTopicNameListPtr = ((**C.char)(&cTopicNameList[0])) } + // Convert char** of topic names into rd_kafka_TopicCollection_t* + cTopicCollection := C.rd_kafka_TopicCollection_new_from_names(cTopicNameListPtr, cTopicNameCount) + // Convert Go AdminOptions (if any) to C AdminOptions. genericOptions := make([]AdminOption, len(options)) for i := range options { @@ -2618,8 +2658,7 @@ func (a *AdminClient) DescribeTopics( // Call rd_kafka_DescribeTopics (asynchronous). C.rd_kafka_DescribeTopics( a.handle.rk, - cTopicNameListPtr, - cTopicNameCount, + cTopicCollection, cOptions, cQueue) @@ -2652,13 +2691,12 @@ func (a *AdminClient) DescribeTopics( // id along with a slice of Nodes. It also has a slice of allowed ACLOperations. func (a *AdminClient) DescribeCluster( ctx context.Context, - options ...DescribeClusterAdminOption) (result ClusterDescription, err error) { - - clusterDesc := ClusterDescription{} + options ...DescribeClusterAdminOption) (result DescribeClusterResult, err error) { err = a.verifyClient() if err != nil { return result, err } + clusterDesc := DescribeClusterResult{} // Convert Go AdminOptions (if any) to C AdminOptions. genericOptions := make([]AdminOption, len(options)) @@ -2693,8 +2731,7 @@ func (a *AdminClient) DescribeCluster( cRes := C.rd_kafka_event_DescribeCluster_result(rkev) // Convert result from C to Go. - cClusterDesc := C.rd_kafka_DescribeCluster_result_description(cRes) - clusterDesc = a.cToClusterDescription(cClusterDesc) + clusterDesc = a.cToClusterDescription(cRes) return clusterDesc, nil } diff --git a/kafka/adminoptions.go b/kafka/adminoptions.go index 70029b608..c1d2966cb 100644 --- a/kafka/adminoptions.go +++ b/kafka/adminoptions.go @@ -311,7 +311,7 @@ func (ao AdminOptionMatchConsumerGroupStates) apply(cOptions *C.rd_kafka_AdminOp // // Default: false // -// Valid for DescribeConsumerGroups. +// Valid for DescribeConsumerGroups, DescribeTopics, DescribeCluster. type AdminOptionIncludeAuthorizedOperations struct { isSet bool val bool @@ -319,6 +319,10 @@ type AdminOptionIncludeAuthorizedOperations struct { func (ao AdminOptionIncludeAuthorizedOperations) supportsDescribeConsumerGroups() { } +func (ao AdminOptionIncludeAuthorizedOperations) supportsDescribeTopics() { +} +func (ao AdminOptionIncludeAuthorizedOperations) supportsDescribeCluster() { +} func (ao AdminOptionIncludeAuthorizedOperations) apply(cOptions *C.rd_kafka_AdminOptions_t) error { if !ao.isSet { @@ -340,95 +344,13 @@ func (ao AdminOptionIncludeAuthorizedOperations) apply(cOptions *C.rd_kafka_Admi // // Default: false // -// Valid for DescribeConsumerGroups. +// Valid for DescribeConsumerGroups, DescribeTopics, DescribeCluster. func SetAdminOptionIncludeAuthorizedOperations(val bool) (ao AdminOptionIncludeAuthorizedOperations) { ao.isSet = true ao.val = val return ao } -// AdminOptionIncludeTopicAuthorizedOperations decides if the broker should return include -// topic authorized operations. -// -// Default: false -// -// Valid for DescribeTopics. -type AdminOptionIncludeTopicAuthorizedOperations struct { - isSet bool - val bool -} - -func (ao AdminOptionIncludeTopicAuthorizedOperations) supportsDescribeTopics() { -} - -func (ao AdminOptionIncludeTopicAuthorizedOperations) apply(cOptions *C.rd_kafka_AdminOptions_t) error { - if !ao.isSet { - return nil - } - - cError := C.rd_kafka_AdminOptions_set_include_topic_authorized_operations( - cOptions, bool2cint(ao.val)) - if cError != nil { - C.rd_kafka_AdminOptions_destroy(cOptions) - return newErrorFromCErrorDestroy(cError) - } - - return nil -} - -// SetAdminOptionIncludeTopicAuthorizedOperations decides if the broker should return include -// topic authorized operations. -// -// Default: false -// -// Valid for DescribeTopics. -func SetAdminOptionIncludeTopicAuthorizedOperations(val bool) (ao AdminOptionIncludeTopicAuthorizedOperations) { - ao.isSet = true - ao.val = val - return ao -} - -// AdminOptionIncludeClusterAuthorizedOperations decides if the broker should return include -// cluster authorized operations. -// -// Default: false -// -// Valid for DescribeCluster. -type AdminOptionIncludeClusterAuthorizedOperations struct { - isSet bool - val bool -} - -func (ao AdminOptionIncludeClusterAuthorizedOperations) supportsDescribeCluster() { -} - -func (ao AdminOptionIncludeClusterAuthorizedOperations) apply(cOptions *C.rd_kafka_AdminOptions_t) error { - if !ao.isSet { - return nil - } - - cError := C.rd_kafka_AdminOptions_set_include_cluster_authorized_operations( - cOptions, bool2cint(ao.val)) - if cError != nil { - C.rd_kafka_AdminOptions_destroy(cOptions) - return newErrorFromCErrorDestroy(cError) - } - - return nil -} - -// SetAdminOptionIncludeClusterAuthorizedOperations decides if the broker should return include -// cluster authorized operations. -// -// Default: false -// -// Valid for DescribeCluster. -func SetAdminOptionIncludeClusterAuthorizedOperations(val bool) (ao AdminOptionIncludeClusterAuthorizedOperations) { - ao.isSet = true - ao.val = val - return ao -} - // SetAdminMatchConsumerGroupStates decides groups in which state(s) should be // listed. // diff --git a/kafka/kafka.go b/kafka/kafka.go index 9bcc9e7a7..1dae2a32f 100644 --- a/kafka/kafka.go +++ b/kafka/kafka.go @@ -326,6 +326,8 @@ type Node struct { Host string // Node port. Port int + // Node rack (may be nil) + RackId *string } func (n Node) String() string { diff --git a/kafka/testresources/docker-compose.yaml b/kafka/testresources/docker-compose.yaml index 51d3bc2aa..f2d3968d1 100644 --- a/kafka/testresources/docker-compose.yaml +++ b/kafka/testresources/docker-compose.yaml @@ -21,5 +21,4 @@ services: KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 KAFKA_AUTHORIZER_CLASS_NAME: kafka.security.authorizer.AclAuthorizer - KAFKA_SUPER_USERS: "User:ANONYMOUS" - \ No newline at end of file + KAFKA_SUPER_USERS: "User:ANONYMOUS" \ No newline at end of file From e941f64fe563250a85be5f67989dc2d9fbcef302 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 6 Sep 2023 11:23:56 +0530 Subject: [PATCH 05/12] Add SASL authentication enabled broker to go integration tests --- kafka/testresources/docker-compose.yaml | 40 +++++++++++++++++++++++-- kafka/testresources/kafka_jaas.conf | 14 +++++++++ kafka/testresources/zookeeper_jaas.conf | 5 ++++ 3 files changed, 56 insertions(+), 3 deletions(-) create mode 100644 kafka/testresources/kafka_jaas.conf create mode 100644 kafka/testresources/zookeeper_jaas.conf diff --git a/kafka/testresources/docker-compose.yaml b/kafka/testresources/docker-compose.yaml index f2d3968d1..b8f5577be 100644 --- a/kafka/testresources/docker-compose.yaml +++ b/kafka/testresources/docker-compose.yaml @@ -4,16 +4,25 @@ services: image: confluentinc/cp-zookeeper environment: ZOOKEEPER_CLIENT_PORT: 2181 + + zookeeper_auth: + image: confluentinc/cp-zookeeper + environment: + ZOOKEEPER_CLIENT_PORT: 2182 + KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/zookeeper/zookeeper_jaas.conf" + volumes: + - ./zookeeper_jaas.conf:/etc/zookeeper/zookeeper_jaas.conf + kafka: image: confluentinc/cp-kafka depends_on: - zookeeper ports: - - 9092:9092 + - 9092:9092 environment: KAFKA_BROKER_ID: 0 KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:29092, PLAINTEXT_HOST://localhost:9092 + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:19092, PLAINTEXT_HOST://localhost:9092 KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 18000 @@ -21,4 +30,29 @@ services: KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 KAFKA_AUTHORIZER_CLASS_NAME: kafka.security.authorizer.AclAuthorizer - KAFKA_SUPER_USERS: "User:ANONYMOUS" \ No newline at end of file + KAFKA_SUPER_USERS: "User:ANONYMOUS" + + kafka_auth: + image: confluentinc/cp-kafka:7.5.0 + depends_on: + - zookeeper_auth + ports: + - 9093:9093 + environment: + KAFKA_BROKER_ID: 0 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: SASL_PLAINTEXT:SASL_PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: SASL_PLAINTEXT://localhost:9093 + KAFKA_ZOOKEEPER_CONNECT: zookeeper_auth:2182 + KAFKA_SASL_ENABLED_MECHANISMS: PLAIN + KAFKA_INTER_BROKER_LISTENER_NAME: SASL_PLAINTEXT + KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: PLAIN + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: 18000 + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 + KAFKA_AUTHORIZER_CLASS_NAME: kafka.security.authorizer.AclAuthorizer + KAFKA_SUPER_USERS: "User:admin" + KAFKA_ALLOW_EVERYONE_IF_NO_ACL_FOUND: true + KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/kafka_jaas.conf" + volumes: + - ./kafka_jaas.conf:/etc/kafka/kafka_jaas.conf \ No newline at end of file diff --git a/kafka/testresources/kafka_jaas.conf b/kafka/testresources/kafka_jaas.conf new file mode 100644 index 000000000..b86fa85ea --- /dev/null +++ b/kafka/testresources/kafka_jaas.conf @@ -0,0 +1,14 @@ +KafkaServer { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin" + user_admin="admin" + user_testuser="testpass"; +}; + +Client { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin"; +}; + diff --git a/kafka/testresources/zookeeper_jaas.conf b/kafka/testresources/zookeeper_jaas.conf new file mode 100644 index 000000000..404823cb6 --- /dev/null +++ b/kafka/testresources/zookeeper_jaas.conf @@ -0,0 +1,5 @@ +Server { + org.apache.zookeeper.server.auth.DigestLoginModule required + user_admin="admin"; +}; + From 9b7a2387ac7de235cbe437edce189b8e4fbaf94e Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 6 Sep 2023 11:25:24 +0530 Subject: [PATCH 06/12] Fix integration tests --- CHANGELOG.md | 8 +- examples/.gitignore | 2 + kafka/integration_test.go | 463 ++++++++++++++++++++++++++------------ kafka/testhelpers.go | 82 +++++-- 4 files changed, 392 insertions(+), 163 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7fef94a09..95fb281cc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,7 +2,13 @@ # v2.3.0 -This is a maintenance release. +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). ## Fixes diff --git a/examples/.gitignore b/examples/.gitignore index 5d2756d79..23181bcd0 100644 --- a/examples/.gitignore +++ b/examples/.gitignore @@ -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 diff --git a/kafka/integration_test.go b/kafka/integration_test.go index 0371834d3..d9cca8597 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -110,7 +110,9 @@ func checkGroupDesc( groupDesc.Coordinator.Host == "" || // We will run all our tests on non-simple consumer groups only. groupDesc.IsSimpleConsumerGroup || - len(groupDesc.Members) != len(clientIDToPartitions) { + len(groupDesc.Members) != len(clientIDToPartitions) || + // We don't set IncludeAuthorizedOperations while using helper. + len(groupDesc.AuthorizedOperations) > 0 { return false } @@ -1093,40 +1095,187 @@ func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() } } +// TestAdminClient_DescribeConsumerGroupsAuthorizedOperations validates the +// working of the DescribeConsumerGroups API of the admin client for fetching +// authorized operations (KIP-430). +// +// We test the following situations: +// +// 1. Default ACLs on group. +// 2. Modified ACLs on group. +func (its *IntegrationTestSuite) TestAdminClient_DescribeConsumerGroupsAuthorizedOperations() { + t := its.T() + assert := its.Assert() + + // Generating a new topic/groupID to ensure a fresh group/topic is created. + rand.Seed(time.Now().Unix()) + groupID := fmt.Sprintf("%s-%d", testconf.GroupID, rand.Int()) + topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) + + clientID := "test.client.1" + + ac := createAdminClientWithSasl(t) + defer ac.Close() + + // Create a topic - we need to create here because we need 2 partitions. + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + _, err := ac.CreateTopics(ctx, []TopicSpecification{ + { + Topic: topic, + NumPartitions: 2, + }, + }) + assert.Nil(err, "CreateTopics should succeed") + + // Delete the topic after the test is done. + defer func() { + ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + _, err = ac.DeleteTopics(ctx, []string{topic}) + assert.Nil(err, "DeleteTopics should succeed") + }() + + // Create a consumer so that a consumer group might be created + config := &ConfigMap{ + "bootstrap.servers": testconf.Brokers, + "group.id": groupID, + "client.id": clientID, + } + config.updateFromTestconf() + consumer, err := NewConsumer(config) + assert.Nil(err, "NewConsumer should succeed") + + // Close the consumer after the test is done + defer consumer.Close() + + consumer.Subscribe(topic, nil) + + // Call Poll to trigger a rebalance and give it enough time to finish. + consumer.Poll(10 * 1000) + + // 1. Default ACLs on group. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + groupDescResult, err := ac.DescribeConsumerGroups( + ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second), + SetAdminOptionIncludeAuthorizedOperations(true)) + assert.Nil(err, "DescribeConsumerGroups should succeed") + + groupDescs := groupDescResult.ConsumerGroupDescriptions + assert.Len(groupDescs, 1, "Describing one group should give exactly one result") + + groupDesc := &groupDescs[0] + assert.NotEmpty(groupDesc.AuthorizedOperations, + "Authorized operations should not be empty") + assert.ElementsMatch(groupDesc.AuthorizedOperations, + []ACLOperation{ + ACLOperationRead, + ACLOperationDelete, + ACLOperationDescribe}) + + // Change the ACLs on the group + newACLs := ACLBindings{ + { + Type: ResourceGroup, + Name: groupID, + ResourcePatternType: ResourcePatternTypeLiteral, + Principal: "User:*", + Host: "*", + Operation: ACLOperationRead, + PermissionType: ACLPermissionTypeAllow, + }, + } + + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, + SetAdminRequestTimeout(time.Second)) + assert.Nil(err, "CreateACLs should not throw an error") + assert.Len(resultCreateACLs, 1, + "CreateACLs result should contain on result") + assert.Equal( + resultCreateACLs[0].Error.Code(), ErrNoError, + "CreateACLs result should not have an error") + + // 2. Modified ACLs on group. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + groupDescResult, err = ac.DescribeConsumerGroups( + ctx, []string{groupID}, SetAdminRequestTimeout(30*time.Second), + SetAdminOptionIncludeAuthorizedOperations(true)) + assert.Nil(err, "DescribeConsumerGroups should succeed") + + groupDescs = groupDescResult.ConsumerGroupDescriptions + assert.Len(groupDescs, 1, + "Describing one group should give exactly one result") + + groupDesc = &groupDescs[0] + assert.NotEmpty(groupDesc.AuthorizedOperations, + "Authorized operations should not be empty") + // Read permissions implicitly allows Describe. + assert.ElementsMatch(groupDesc.AuthorizedOperations, + []ACLOperation{ACLOperationRead, ACLOperationDescribe}) +} + // TestAdminClient_DescribeCluster validates the working of the -// describe cluster API of the admin client. +// DescribeCluster API of the admin client. // // We test the following situations: // -// 1. DescribeCluster without createAcl. -// 2. DescribeCluster with Acl. +// 1. DescribeCluster without ACLs. +// 2. DescribeCluster with default ACLs. +// 3. DescribeCluster with modified ACLs. func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { t := its.T() + assert := its.Assert() ac := createAdminClient(t) defer ac.Close() - noError := NewError(ErrNoError, "", false) - checkExpectedResult := func(expected interface{}, result interface{}) { - if !reflect.DeepEqual(result, expected) { - t.Fatalf("Expected result to deep equal to %v, but found %v", expected, result) - } - } - // Check the non-existence of consumer groups initially. + // 1. DescribeCluster without ACLs. ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() descres, err := ac.DescribeCluster( - ctx, SetAdminRequestTimeout(time.Second), SetAdminOptionIncludeClusterAuthorizedOperations(true)) - if descres.Nodes == nil || err != nil { - t.Fatalf("Expected DescribeTopics to pass, but not %s %v", - err.(Error).Code(), ctx.Err()) - } - initialLen := len(descres.ClusterAuthorizedOperations) - if initialLen == 0 { - t.Fatalf("Expected cluster authorized operations>0" + - "as they are being requested") - } - fmt.Printf("Initial length: %d\n", initialLen) + ctx, SetAdminRequestTimeout(time.Second*30)) + + // There are fields which cannot be checked, like controller, or the value + // of the cluster ID. We try checking for the existence in cases we can. + assert.Nil(err, "DescribeCluster should not throw an error") + assert.NotEmpty(descres.Nodes, "Cluster nodes should not be empty") + assert.NotEmpty(descres.ClusterId, "Cluster id should be set") + assert.NotEmpty(descres.Nodes[0].Host, + "First node's host should be non-empty") + assert.Empty(descres.AuthorizedOperations, + "Authorized operations should be empty, not requested") + + // Tests for Authorized Operations need a broker with SASL authentication. + // This may be a different broker than the usual broker, so we create a + // new AdminClient. + ac = createAdminClientWithSasl(t) + defer ac.Close() + // 2. DescribeCluster with default ACLs. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + descres, err = ac.DescribeCluster( + ctx, SetAdminRequestTimeout(time.Second*30), + SetAdminOptionIncludeAuthorizedOperations(true)) + + assert.Nil(err, "DescribeCluster should not throw an error") + assert.NotEmpty(descres.Nodes, "Cluster nodes should not be empty") + assert.NotEmpty(descres.ClusterId, "Cluster id should be set") + assert.NotEmpty(descres.Nodes[0].Host, + "First node's host should be non-empty") + assert.NotEmpty(descres.AuthorizedOperations, + "Authorized operations should not be empty") + assert.ElementsMatch(descres.AuthorizedOperations, + []ACLOperation{ + ACLOperationCreate, ACLOperationAlter, ACLOperationDescribe, + ACLOperationClusterAction, ACLOperationDescribeConfigs, + ACLOperationAlterConfigs, ACLOperationIdempotentWrite}) + + // Create some ACL bindings on the cluster. newACLs := ACLBindings{ { Type: ResourceBroker, @@ -1138,38 +1287,38 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { PermissionType: ACLPermissionTypeAllow, }, } - maxDuration, err := time.ParseDuration("30s") - if err != nil { - t.Fatalf("%s", err) - } - requestTimeout, err := time.ParseDuration("20s") - if err != nil { - t.Fatalf("%s", err) - } - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() - resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) - if err != nil { - t.Fatalf("CreateACLs() failed: %s", err) - } - expectedCreateACLs := []CreateACLResult{{Error: noError}} - checkExpectedResult(expectedCreateACLs, resultCreateACLs) + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, + SetAdminRequestTimeout(time.Second*30)) + assert.Nil(err, "CreateACLs should not throw an error") + assert.Len(resultCreateACLs, 1, + "CreateACLs result should contain on result") + assert.Equal( + resultCreateACLs[0].Error.Code(), ErrNoError, + "CreateACLs result should not have an error") + // 3. DescribeCluster with modified ACLs. ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() descres, err = ac.DescribeCluster( - ctx, SetAdminRequestTimeout(time.Second), SetAdminOptionIncludeClusterAuthorizedOperations(true)) - if descres.Nodes == nil || err != nil { - t.Fatalf("Expected DescribeTopics to pass, but not %s %v", - err.(Error).Code(), ctx.Err()) - } - finalLen := len(descres.ClusterAuthorizedOperations) - fmt.Printf("Final length: %d\n", finalLen) - if initialLen <= finalLen { - t.Fatalf("Expected final acl count to have reduced after createAcl") - } - + ctx, SetAdminRequestTimeout(time.Second*30), + SetAdminOptionIncludeAuthorizedOperations(true)) + + assert.Nil(err, "DescribeCluster should not throw an error") + assert.NotEmpty(descres.Nodes, "Cluster nodes should not be empty") + assert.NotEmpty(descres.ClusterId, "Cluster id should be set") + assert.NotEmpty(descres.Nodes[0].Host, + "First node's host should be non-empty") + assert.NotEmpty(descres.AuthorizedOperations, + "Authorized operations should not be empty") + // Alter permissions implicitly allow Describe. + assert.ElementsMatch(descres.AuthorizedOperations, + []ACLOperation{ACLOperationDescribe, ACLOperationAlter}) + + // Clean up cluster ACLs for subsequent tests. aclBindingFilters := ACLBindingFilters{ { Type: ResourceBroker, @@ -1181,31 +1330,28 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { PermissionType: ACLPermissionTypeAllow, }, } - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() - _, err = ac.DeleteACLs(ctx, aclBindingFilters, SetAdminRequestTimeout(requestTimeout)) - if err != nil { - t.Fatalf("%s", err) - } + _, err = ac.DeleteACLs(ctx, aclBindingFilters, + SetAdminRequestTimeout(time.Second*30)) + assert.Nil(err, "DeleteACLs should not throw an error") } // TestAdminClient_DescribeTopics validates the working of the -// describe topics API of the admin client. +// DescribeTopics API of the admin client. // // We test the following situations: // -// 1. DescribeTopics without createAcl. -// 2. DescribeTopics with Acl. +// 1. DescribeTopics without ACLs. +// 2. DescribeTopics with default ACLs. +// 3. DescribeTopics with modified ACLs. func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { t := its.T() + assert := its.Assert() + rand.Seed(time.Now().Unix()) + ac := createAdminClient(t) defer ac.Close() - noError := NewError(ErrNoError, "", false) - checkExpectedResult := func(expected interface{}, result interface{}) { - if !reflect.DeepEqual(result, expected) { - t.Fatalf("Expected result to deep equal to %v, but found %v", expected, result) - } - } // Create a topic topic := fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) @@ -1217,123 +1363,146 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { NumPartitions: 2, }, }) - if err != nil { - t.Errorf("Topic creation failed with error %v", err) - return - } + assert.Nil(err, "CreateTopics should not fail") // Delete the topic after the test is done. - defer func() { + defer func(ac *AdminClient) { ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) defer cancel() _, err = ac.DeleteTopics(ctx, []string{topic}) - if err != nil { - t.Errorf("Topic deletion failed with error %v", err) - } - }() + assert.Nil(err, "DeleteTopics should not fail") + }(ac) - // Test the description of the topic. + // 1. DescribeTopics without ACLs. ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err := ac.DescribeTopics( - ctx, []string{topic, "failure"}, SetAdminRequestTimeout(30*time.Second), - SetAdminOptionIncludeTopicAuthorizedOperations(true)) - if err != nil { - t.Errorf("Error describing topics %s\n", err) - return - } + ctx, []string{topic, "nonexistent"}, + SetAdminRequestTimeout(30*time.Second)) + assert.Nil(err, "DescribeTopics should not fail") topicDescs := topicDescResult.TopicDescriptions - if len(topicDescs) != 2 { - t.Errorf("Describing two topics should give exactly two results %s\n", err) - return - } - if topicDescs[1].Error.Code() != 3 { - t.Fatalf("Expected expected unknown Topic or partition, not %s\n", - topicDescs[1].Error) - } - initialLen := len(topicDescs[0].TopicAuthorizedOperations) - if initialLen == 0 { - t.Fatalf("Expected topic authorized operations>0" + - "as they are being requested") - } - fmt.Printf("Initial length: %d\n", initialLen) + assert.Len(topicDescs, 2, + "Describing two topics should give exactly two results") + assert.Equal(topicDescs[0].Topic, topic, + "First result topic should match request topic") + assert.Equal(topicDescs[1].Topic, "nonexistent", + "Second result topic should match request topic") + assert.Equal(topicDescs[1].Error.Code(), ErrUnknownTopicOrPart, + "Expected correct error for nonexistent topic") + + topicDesc := topicDescs[0] + assert.Equal(topicDesc.Error.Code(), ErrNoError, + "Topic description should not have an error") + assert.False(topicDesc.IsInternal, "Topic should not be internal") + assert.Empty(topicDesc.AuthorizedOperations, + "Topic should not have authorized operations") + + assert.Len(topicDesc.Partitions, 2, "Topic should have two partitions") + assert.GreaterOrEqual(len(topicDesc.Partitions[0].Replicas), 1, + "At least one replica should exist for partition") + + // Tests for Authorized Operations need a broker with SASL authentication. + // This may be a different broker than the usual broker, so we create a + // new AdminClient. + ac = createAdminClientWithSasl(t) + defer ac.Close() - newACLs := ACLBindings{ + // Create a topic - the broker may be different for SASL, so we need to + // ensure that a topic is created. + topic = fmt.Sprintf("%s-%d", testconf.Topic, rand.Int()) + ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + _, err = ac.CreateTopics(ctx, []TopicSpecification{ { - Type: ResourceTopic, - Name: topic, - ResourcePatternType: ResourcePatternTypeLiteral, - Principal: "User:*", - Host: "*", - Operation: ACLOperationRead, - PermissionType: ACLPermissionTypeAllow, + Topic: topic, + NumPartitions: 2, }, - } - maxDuration, err := time.ParseDuration("30s") - if err != nil { - t.Fatalf("%s", err) - } - requestTimeout, err := time.ParseDuration("20s") - if err != nil { - t.Fatalf("%s", err) - } - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) - defer cancel() + }) + assert.Nil(err, "CreateTopics should not fail") - resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) - if err != nil { - t.Fatalf("CreateACLs() failed: %s", err) - } - expectedCreateACLs := []CreateACLResult{{Error: noError}} - checkExpectedResult(expectedCreateACLs, resultCreateACLs) + // Delete the second topic after the test is done. + defer func(ac *AdminClient) { + ctx, cancel = context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + _, err = ac.DeleteTopics(ctx, []string{topic}) + assert.Nil(err, "DeleteTopics should not fail") + }(ac) + // 2. DescribeTopics with default ACLs. ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err = ac.DescribeTopics( - ctx, []string{topic}, SetAdminRequestTimeout(30*time.Second), - SetAdminOptionIncludeTopicAuthorizedOperations(true)) - if err != nil { - t.Errorf("Error describing topics %s\n", err) - return - } + ctx, []string{topic}, + SetAdminRequestTimeout(30*time.Second), + SetAdminOptionIncludeAuthorizedOperations(true)) - topicDescs = topicDescResult.TopicDescriptions - if len(topicDescs) != 1 { - t.Errorf("Describing one topics should give exactly one result %s\n", err) - return - } - finalLen := len(topicDescs[0].TopicAuthorizedOperations) - if finalLen == 0 { - t.Fatalf("Expected topic authorized operations>0" + - "as they are being requested") - } - fmt.Printf("Initial length: %d\n", initialLen) - fmt.Printf("Final length: %d\n", finalLen) - if initialLen <= finalLen { - t.Fatalf("Expected final acl count to have reduced after createAcl") - } + assert.Nil(err, "DescribeTopics should not fail") - newACLs = ACLBindings{ + topicDescs = topicDescResult.TopicDescriptions + assert.Len(topicDescs, 1, + "Describing one topic should give exactly one result") + assert.Equal(topicDescs[0].Topic, topic, + "First result topic should match request topic") + + topicDesc = topicDescs[0] + assert.Equal(topicDesc.Error.Code(), ErrNoError, + "Topic description should not have an error") + assert.NotEmpty(topicDesc.AuthorizedOperations, + "Topic should have authorized operations") + assert.ElementsMatch(topicDesc.AuthorizedOperations, []ACLOperation{ + ACLOperationRead, ACLOperationWrite, ACLOperationCreate, + ACLOperationDelete, ACLOperationAlter, ACLOperationDescribe, + ACLOperationDescribeConfigs, ACLOperationAlterConfigs}) + + // Create some ACL bindings on the topic. + newACLs := ACLBindings{ { Type: ResourceTopic, Name: topic, ResourcePatternType: ResourcePatternTypeLiteral, Principal: "User:*", Host: "*", - Operation: ACLOperationDelete, + Operation: ACLOperationRead, PermissionType: ACLPermissionTypeAllow, }, } - ctx, cancel = context.WithTimeout(context.Background(), maxDuration) + + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() - resultCreateACLs, err = ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) - if err != nil { - t.Fatalf("CreateACLs() failed: %s", err) - } - expectedCreateACLs = []CreateACLResult{{Error: noError}} - checkExpectedResult(expectedCreateACLs, resultCreateACLs) + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, + SetAdminRequestTimeout(time.Second)) + assert.Nil(err, "CreateACLs should not throw an error") + assert.Len(resultCreateACLs, 1, + "CreateACLs result should contain on result") + assert.Equal( + resultCreateACLs[0].Error.Code(), ErrNoError, + "CreateACLs result should not have an error") + + // 3. DescribeTopics with modified ACLs. + ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + topicDescResult, err = ac.DescribeTopics( + ctx, []string{topic}, SetAdminRequestTimeout(time.Second), + SetAdminOptionIncludeAuthorizedOperations(true)) + + assert.Nil(err, "DescribeTopics should not fail") + + topicDescs = topicDescResult.TopicDescriptions + assert.Len(topicDescs, 1, + "Describing one topic should give exactly one result") + assert.Equal(topicDescs[0].Topic, topic, + "First result topic should match request topic") + + topicDesc = topicDescs[0] + assert.Equal(topicDesc.Error.Code(), ErrNoError, + "Topic description should not have an error") + assert.NotEmpty(topicDesc.AuthorizedOperations, + "Topic should have authorized operations") + // Read permissions implicitly allows Describe. + assert.ElementsMatch(topicDesc.AuthorizedOperations, + []ACLOperation{ACLOperationRead, ACLOperationDescribe}) } func (its *IntegrationTestSuite) TestAdminTopics() { diff --git a/kafka/testhelpers.go b/kafka/testhelpers.go index 899ed33f6..42e0a774c 100644 --- a/kafka/testhelpers.go +++ b/kafka/testhelpers.go @@ -19,6 +19,7 @@ package kafka import ( "context" "encoding/json" + "errors" "flag" "fmt" "math/rand" @@ -33,15 +34,19 @@ import ( import "C" var testconf struct { - Docker bool - Semaphore bool - Brokers string - Topic string - GroupID string - PerfMsgCount int - PerfMsgSize int - Config []string - conf ConfigMap + Docker bool + Semaphore bool + Brokers string + BrokersSasl string + SaslUsername string + SaslPassword string + SaslMechanism string + Topic string + GroupID string + PerfMsgCount int + PerfMsgSize int + Config []string + conf ConfigMap } const defaulttestconfTopic = "test" @@ -52,6 +57,10 @@ const defaulttestconfPerfMsgSize = 100 var defaulttestconfConfig = [1]string{"api.version.request=true"} const defaulttestconfBrokers = "localhost:9092" +const defaulttestconfBrokersSasl = "localhost:9093" +const defaultSaslUsername = "testuser" +const defaultSaslPassword = "testpass" +const defaultSaslMechanism = "PLAIN" // flag for semaphore job var semaphoreJob = flag.Bool("clients.semaphore", false, "Tells if the job is running on Semaphore") @@ -83,9 +92,14 @@ func testconfRead() bool { testconf.GroupID = defaulttestconfGroupID testconf.Topic = defaulttestconfTopic testconf.Brokers = "" + testconf.BrokersSasl = "" if testconf.Docker || testconf.Semaphore { testconf.Brokers = defaulttestconfBrokers + testconf.BrokersSasl = defaulttestconfBrokersSasl + testconf.SaslUsername = defaultSaslUsername + testconf.SaslPassword = defaultSaslPassword + testconf.SaslMechanism = defaultSaslMechanism return true } @@ -111,6 +125,10 @@ func testconfRead() bool { testconf.Brokers = os.Getenv(testconf.Brokers[1:]) } + if len(testconf.BrokersSasl) > 0 && testconf.BrokersSasl[0] == '$' { + testconf.BrokersSasl = os.Getenv(testconf.BrokersSasl[1:]) + } + return true } @@ -132,6 +150,29 @@ func (cm *ConfigMap) updateFromTestconf() error { } +func (cm *ConfigMap) updateToSaslAuthentication() error { + if testconf.BrokersSasl == "" { + return errors.New("BrokersSasl must be set in test config") + } + if len(testconf.SaslMechanism) == 0 { + return errors.New("SaslMechanism must be set in test config") + } + if len(testconf.SaslPassword) == 0 { + return errors.New("SaslPassword must be set in test config") + } + if len(testconf.SaslUsername) == 0 { + return errors.New("SaslUsername must be set in test config") + } + + cm.SetKey("bootstrap.servers", testconf.BrokersSasl) + cm.SetKey("sasl.username", testconf.SaslUsername) + cm.SetKey("sasl.password", testconf.SaslPassword) + cm.SetKey("sasl.mechanisms", testconf.SaslMechanism) + cm.SetKey("security.protocol", "SASL_PLAINTEXT") + + return nil +} + // Return the number of messages available in all partitions of a topic. // WARNING: This uses watermark offsets so it will be incorrect for compacted topics. func getMessageCountInTopic(topic string) (int, error) { @@ -219,7 +260,7 @@ func waitTopicInMetadata(H Handle, topic string, timeoutMs int) error { } -func createAdminClient(t *testing.T) (a *AdminClient) { +func createAdminClientImpl(t *testing.T, withSasl bool) (a *AdminClient) { numver, strver := LibraryVersion() if numver < 0x000b0500 { t.Skipf("Requires librdkafka >=0.11.5 (currently on %s, 0x%x)", strver, numver) @@ -229,13 +270,16 @@ func createAdminClient(t *testing.T) (a *AdminClient) { t.Skipf("Missing testconf.json") } - conf := ConfigMap{"bootstrap.servers": testconf.Brokers} + brokers := testconf.Brokers + conf := ConfigMap{"bootstrap.servers": brokers} conf.updateFromTestconf() + if withSasl { + if err := conf.updateToSaslAuthentication(); err != nil { + t.Skipf("Test requires SASL Authentication, but failed to set it up: %s", err) + return + } + } - /* - * Create producer and produce a couple of messages with and without - * headers. - */ a, err := NewAdminClient(&conf) if err != nil { t.Fatalf("NewAdminClient: %v", err) @@ -244,6 +288,14 @@ func createAdminClient(t *testing.T) (a *AdminClient) { return a } +func createAdminClient(t *testing.T) (a *AdminClient) { + return createAdminClientImpl(t, false) +} + +func createAdminClientWithSasl(t *testing.T) (a *AdminClient) { + return createAdminClientImpl(t, true) +} + func createTestTopic(t *testing.T, suffix string, numPartitions int, replicationFactor int) string { rand.Seed(time.Now().Unix()) From 6a754e424cd7a21d0c886b5bb2cb08ca72664a89 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 6 Sep 2023 12:20:04 +0530 Subject: [PATCH 07/12] Change minor details and public API for DescribeTopics --- examples/README.md | 6 +- .../admin_describe_cluster.go | 9 +- .../admin_describe_consumer_groups.go | 10 +-- .../admin_describe_topics.go | 14 +-- kafka/adminapi.go | 86 ++++++++++++------- kafka/adminapi_test.go | 5 +- kafka/adminoptions.go | 10 +-- kafka/integration_test.go | 17 ++-- kafka/testhelpers.go | 5 ++ kafka/testresources/docker-compose.yaml | 4 +- kafka/testresources/kafka_jaas.conf | 1 - kafka/testresources/zookeeper_jaas.conf | 1 - 12 files changed, 98 insertions(+), 70 deletions(-) diff --git a/examples/README.md b/examples/README.md index ee737dae5..e383a9416 100644 --- a/examples/README.md +++ b/examples/README.md @@ -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 diff --git a/examples/admin_describe_cluster/admin_describe_cluster.go b/examples/admin_describe_cluster/admin_describe_cluster.go index 29219d21c..18eee9319 100644 --- a/examples/admin_describe_cluster/admin_describe_cluster.go +++ b/examples/admin_describe_cluster/admin_describe_cluster.go @@ -1,5 +1,5 @@ /** - * Copyright 2022 Confluent Inc. + * 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. @@ -14,7 +14,7 @@ * limitations under the License. */ -// Describe topics +// Describe Cluster package main import ( @@ -40,7 +40,8 @@ func main() { include_authorized_operations, err_operations := strconv.ParseBool(os.Args[2]) if err_operations != nil { fmt.Printf( - "Failed to parse value of include_authorized_operations %s: %s\n", os.Args[2], err_operations) + "Failed to parse value of include_authorized_operations %s: %s\n", + os.Args[2], err_operations) os.Exit(1) } @@ -68,7 +69,7 @@ func main() { // Print results fmt.Printf("ClusterId: %s\nController: %s\nNodes: %s\n", clusterDesc.ClusterId, clusterDesc.Controller, clusterDesc.Nodes) - if include_authorized_operations == true { + if include_authorized_operations { fmt.Printf("Allowed operations: %s\n", clusterDesc.AuthorizedOperations) } } diff --git a/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go b/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go index 8aff82178..18adb66ce 100644 --- a/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go +++ b/examples/admin_describe_consumer_groups/admin_describe_consumer_groups.go @@ -60,7 +60,8 @@ func main() { // Call DescribeConsumerGroups. ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() - describeGroupsResult, err := a.DescribeConsumerGroups(ctx, groups, kafka.SetAdminOptionIncludeAuthorizedOperations(include_authorized_operations)) + describeGroupsResult, err := a.DescribeConsumerGroups(ctx, groups, + kafka.SetAdminOptionIncludeAuthorizedOperations(include_authorized_operations)) if err != nil { fmt.Printf("Failed to describe groups: %s\n", err) os.Exit(1) @@ -79,11 +80,8 @@ func main() { "Members: %+v\n", g.GroupID, g.Error, g.IsSimpleConsumerGroup, g.PartitionAssignor, g.State, g.Coordinator, g.Members) - if include_authorized_operations == true { - fmt.Printf("Allowed acl operations:\n") - for i := 0; i < len(g.AuthorizedOperations); i++ { - fmt.Printf("\t%s\n", g.AuthorizedOperations[i]) - } + if include_authorized_operations { + fmt.Printf("Allowed operations: %s\n", g.AuthorizedOperations) } fmt.Printf("\n") } diff --git a/examples/admin_describe_topics/admin_describe_topics.go b/examples/admin_describe_topics/admin_describe_topics.go index 35f14bb8b..7aed6acc2 100644 --- a/examples/admin_describe_topics/admin_describe_topics.go +++ b/examples/admin_describe_topics/admin_describe_topics.go @@ -1,5 +1,5 @@ /** - * Copyright 2022 Confluent Inc. + * 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. @@ -41,7 +41,8 @@ func main() { include_authorized_operations, err_operations := strconv.ParseBool(os.Args[2]) if err_operations != nil { fmt.Printf( - "Failed to parse value of include_authorized_operations %s: %s\n", os.Args[2], err_operations) + "Failed to parse value of include_authorized_operations %s: %s\n", + os.Args[2], err_operations) os.Exit(1) } topics := os.Args[3:] @@ -60,7 +61,8 @@ func main() { ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() describeTopicsResult, err := a.DescribeTopics( - ctx, topics, kafka.SetAdminOptionIncludeAuthorizedOperations( + ctx, kafka.TopicCollection{Names: topics}, + kafka.SetAdminOptionIncludeAuthorizedOperations( include_authorized_operations)) if err != nil { fmt.Printf("Failed to describe topics: %s\n", err) @@ -76,9 +78,8 @@ func main() { t.Topic, t.Error) continue } - fmt.Printf("Topic: %s has succeeded\n", - t.Topic) - if include_authorized_operations == true { + fmt.Printf("Topic: %s has succeeded\n", t.Topic) + if include_authorized_operations { fmt.Printf("Allowed operations: %s\n", t.AuthorizedOperations) } for i := 0; i < len(t.Partitions); i++ { @@ -86,7 +87,6 @@ func main() { 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) } diff --git a/kafka/adminapi.go b/kafka/adminapi.go index f6b4e0a63..6f9665648 100644 --- a/kafka/adminapi.go +++ b/kafka/adminapi.go @@ -299,6 +299,12 @@ type DescribeConsumerGroupsResult struct { ConsumerGroupDescriptions []ConsumerGroupDescription } +// TopicCollection represents a collection of topics. +type TopicCollection struct { + // Slice of topic names. + Names []string +} + // Topic Partition information type TopicPartitionInfo struct { // Partition id. @@ -1064,8 +1070,11 @@ func (a *AdminClient) cToTopicResults(cTopicRes **C.rd_kafka_topic_result_t, cCn return result, nil } +// cToAuthorizedOperations converts a C AclOperation_t array to a Go +// ACLOperation list. func (a *AdminClient) cToAuthorizedOperations( - cAuthorizedOperations *C.rd_kafka_AclOperation_t, cAuthorizedOperationCnt C.size_t) []ACLOperation { + cAuthorizedOperations *C.rd_kafka_AclOperation_t, + cAuthorizedOperationCnt C.size_t) []ACLOperation { if cAuthorizedOperations == nil { return nil } @@ -1080,6 +1089,7 @@ func (a *AdminClient) cToAuthorizedOperations( return authorizedOperations } +// cToNode converts a C Node_t* to a Go Node. func (a *AdminClient) cToNode(cNode *C.rd_kafka_Node_t) Node { node := Node{ID: -1} if cNode == nil { @@ -1098,6 +1108,17 @@ func (a *AdminClient) cToNode(cNode *C.rd_kafka_Node_t) Node { return node } +// cToNode converts a C Node_t array to a Go Node list. +func (a *AdminClient) cToNodes( + cNodes **C.rd_kafka_Node_t, cNodeCnt C.size_t) []Node { + nodes := make([]Node, int(cNodeCnt)) + for i := 0; i < int(cNodeCnt); i++ { + cNode := C.Node_by_idx(cNodes, cNodeCnt, C.size_t(i)) + nodes[i] = a.cToNode(cNode) + } + return nodes +} + // cToConsumerGroupDescriptions converts a C rd_kafka_ConsumerGroupDescription_t // array to a Go ConsumerGroupDescription slice. func (a *AdminClient) cToConsumerGroupDescriptions( @@ -1153,7 +1174,8 @@ func (a *AdminClient) cToConsumerGroupDescriptions( cAuthorizedOperationsCnt := C.size_t(0) cAuthorizedOperations := C.rd_kafka_ConsumerGroupDescription_authorized_operations( cGroup, &cAuthorizedOperationsCnt) - authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, cAuthorizedOperationsCnt) + authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, + cAuthorizedOperationsCnt) result[idx] = ConsumerGroupDescription{ GroupID: groupID, @@ -1169,16 +1191,10 @@ func (a *AdminClient) cToConsumerGroupDescriptions( return result } -func (a *AdminClient) cToNodes(cNodes **C.rd_kafka_Node_t, cNodeCnt C.size_t) []Node { - nodes := make([]Node, int(cNodeCnt)) - for i := 0; i < int(cNodeCnt); i++ { - cNode := C.Node_by_idx(cNodes, cNodeCnt, C.size_t(i)) - nodes[i] = a.cToNode(cNode) - } - return nodes -} - -func (a *AdminClient) cToTopicPartitionInfo(partitionInfo *C.rd_kafka_TopicPartitionInfo_t) TopicPartitionInfo { +// cToTopicPartitionInfo converts a C TopicPartitionInfo_t into a Go +// TopicPartitionInfo. +func (a *AdminClient) cToTopicPartitionInfo( + partitionInfo *C.rd_kafka_TopicPartitionInfo_t) TopicPartitionInfo { cPartitionId := C.rd_kafka_TopicPartitionInfo_partition(partitionInfo) info := TopicPartitionInfo{ Partition: int(cPartitionId), @@ -1191,7 +1207,8 @@ func (a *AdminClient) cToTopicPartitionInfo(partitionInfo *C.rd_kafka_TopicParti } cReplicaCnt := C.size_t(0) - cReplicas := C.rd_kafka_TopicPartitionInfo_replicas(partitionInfo, &cReplicaCnt) + cReplicas := C.rd_kafka_TopicPartitionInfo_replicas( + partitionInfo, &cReplicaCnt) info.Replicas = a.cToNodes(cReplicas, cReplicaCnt) cIsrCnt := C.size_t(0) @@ -1201,8 +1218,8 @@ func (a *AdminClient) cToTopicPartitionInfo(partitionInfo *C.rd_kafka_TopicParti return info } -// cToTopicDescriptions converts a C rd_kafka_TopicDescription_t -// array to a Go TopicDescription slice. +// cToTopicDescriptions converts a C TopicDescription_t +// array to a Go TopicDescription list. func (a *AdminClient) cToTopicDescriptions( cTopics **C.rd_kafka_TopicDescription_t, cTopicCount C.size_t) (result []TopicDescription) { @@ -1241,9 +1258,9 @@ func (a *AdminClient) cToTopicDescriptions( return result } -// cToClusterDescriptions converts a C rd_kafka_TopicDescription_t -// to a Go ClusterDescription. -func (a *AdminClient) cToClusterDescription( +// cToDescribeClusterResult converts a C DescribeTopics_result_t to a Go +// DescribeClusterResult. +func (a *AdminClient) cToDescribeClusterResult( cResult *C.rd_kafka_DescribeTopics_result_t) (result DescribeClusterResult) { clusterID := C.GoString(C.rd_kafka_DescribeCluster_result_cluster_id(cResult)) @@ -1259,9 +1276,11 @@ func (a *AdminClient) cToClusterDescription( nodes := a.cToNodes(cNodes, cNodeCnt) cAuthorizedOperationsCnt := C.size_t(0) - cAuthorizedOperations := C.rd_kafka_DescribeCluster_result_authorized_operations( - cResult, &cAuthorizedOperationsCnt) - authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, cAuthorizedOperationsCnt) + cAuthorizedOperations := + C.rd_kafka_DescribeCluster_result_authorized_operations( + cResult, &cAuthorizedOperationsCnt) + authorizedOperations := a.cToAuthorizedOperations( + cAuthorizedOperations, cAuthorizedOperationsCnt) return DescribeClusterResult{ ClusterId: clusterID, @@ -2603,7 +2622,7 @@ func (a *AdminClient) DescribeConsumerGroups( // Parameters: // - `ctx` - context with the maximum amount of time to block, or nil for // indefinite. -// - `topics` - Slice of groups to describe. This should not be nil/empty. +// - `topics` - Collection of topics to describe. This should not be nil/empty. // - `options` - DescribeTopicsAdminOption options. // // Returns DescribeTopicsResult, which contains a slice of @@ -2613,7 +2632,7 @@ func (a *AdminClient) DescribeConsumerGroups( // errors. Individual TopicDescriptions also have a // slice of allowed ACLOperations. func (a *AdminClient) DescribeTopics( - ctx context.Context, topics []string, + ctx context.Context, topics TopicCollection, options ...DescribeTopicsAdminOption) (result DescribeTopicsResult, err error) { describeResult := DescribeTopicsResult{} @@ -2623,10 +2642,10 @@ func (a *AdminClient) DescribeTopics( } // Convert topic names into char**. - cTopicNameList := make([]*C.char, len(topics)) - cTopicNameCount := C.size_t(len(topics)) + cTopicNameList := make([]*C.char, len(topics.Names)) + cTopicNameCount := C.size_t(len(topics.Names)) - for idx, topic := range topics { + for idx, topic := range topics.Names { cTopicNameList[idx] = C.CString(topic) defer C.free(unsafe.Pointer(cTopicNameList[idx])) } @@ -2637,7 +2656,9 @@ func (a *AdminClient) DescribeTopics( } // Convert char** of topic names into rd_kafka_TopicCollection_t* - cTopicCollection := C.rd_kafka_TopicCollection_new_from_names(cTopicNameListPtr, cTopicNameCount) + cTopicCollection := C.rd_kafka_TopicCollection_new_from_names( + cTopicNameListPtr, cTopicNameCount) + defer C.rd_kafka_TopicCollection_destroy(cTopicCollection) // Convert Go AdminOptions (if any) to C AdminOptions. genericOptions := make([]AdminOption, len(options)) @@ -2687,8 +2708,8 @@ func (a *AdminClient) DescribeTopics( // indefinite. // - `options` - DescribeClusterAdminOption options. // -// Returns ClusterDescription, which contains current cluster and controller -// id along with a slice of Nodes. It also has a slice of allowed ACLOperations. +// Returns ClusterDescription, which contains current cluster ID and controller +// along with a slice of Nodes. It also has a slice of allowed ACLOperations. func (a *AdminClient) DescribeCluster( ctx context.Context, options ...DescribeClusterAdminOption) (result DescribeClusterResult, err error) { @@ -2731,7 +2752,7 @@ func (a *AdminClient) DescribeCluster( cRes := C.rd_kafka_event_DescribeCluster_result(rkev) // Convert result from C to Go. - clusterDesc = a.cToClusterDescription(cRes) + clusterDesc = a.cToDescribeClusterResult(cRes) return clusterDesc, nil } @@ -2744,9 +2765,8 @@ func (a *AdminClient) DescribeCluster( // - `options` - DeleteConsumerGroupsAdminOption options. // // Returns a DeleteConsumerGroupsResult containing a slice of ConsumerGroupResult, with -// -// group-level errors, (if any) contained inside; and an error that is not nil -// for client level errors. +// group-level errors, (if any) contained inside; and an error that is not nil +// for client level errors. func (a *AdminClient) DeleteConsumerGroups( ctx context.Context, groups []string, options ...DeleteConsumerGroupsAdminOption) (result DeleteConsumerGroupsResult, err error) { diff --git a/kafka/adminapi_test.go b/kafka/adminapi_test.go index f2b6ea483..cdf9ae8bc 100644 --- a/kafka/adminapi_test.go +++ b/kafka/adminapi_test.go @@ -490,7 +490,7 @@ func testAdminAPIsDescribeTopics( ctx, cancel := context.WithTimeout(context.Background(), expDuration) defer cancel() descres, err := a.DescribeTopics( - ctx, nil, SetAdminRequestTimeout(time.Second)) + ctx, TopicCollection{}, SetAdminRequestTimeout(time.Second)) if descres.TopicDescriptions != nil || err == nil { t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", descres, err) @@ -502,7 +502,8 @@ func testAdminAPIsDescribeTopics( ctx, cancel = context.WithTimeout(context.Background(), expDuration) defer cancel() descres, err = a.DescribeTopics( - ctx, []string{"test"}, SetAdminRequestTimeout(time.Second)) + ctx, TopicCollection{Names: []string{"test"}}, + SetAdminRequestTimeout(time.Second)) if descres.TopicDescriptions != nil || err == nil { t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", descres, err) diff --git a/kafka/adminoptions.go b/kafka/adminoptions.go index c1d2966cb..3d4c88342 100644 --- a/kafka/adminoptions.go +++ b/kafka/adminoptions.go @@ -306,7 +306,7 @@ func (ao AdminOptionMatchConsumerGroupStates) apply(cOptions *C.rd_kafka_AdminOp return nil } -// AdminOptionIncludeAuthorizedOperations decides if the broker should return include +// AdminOptionIncludeAuthorizedOperations decides if the broker should return // authorized operations. // // Default: false @@ -339,7 +339,7 @@ func (ao AdminOptionIncludeAuthorizedOperations) apply(cOptions *C.rd_kafka_Admi return nil } -// SetAdminOptionIncludeAuthorizedOperations decides if the broker should return include +// SetAdminOptionIncludeAuthorizedOperations decides if the broker should return // authorized operations. // // Default: false @@ -445,7 +445,7 @@ type DescribeConsumerGroupsAdminOption interface { // DescribeTopicsAdminOption - see setter. // -// See SetAdminRequestTimeout. +// See SetAdminRequestTimeout, SetAdminOptionIncludeAuthorizedOperations. type DescribeTopicsAdminOption interface { supportsDescribeTopics() apply(cOptions *C.rd_kafka_AdminOptions_t) error @@ -453,7 +453,7 @@ type DescribeTopicsAdminOption interface { // DescribeClusterAdminOption - see setter. // -// See SetAdminRequestTimeout. +// See SetAdminRequestTimeout, SetAdminOptionIncludeAuthorizedOperations. type DescribeClusterAdminOption interface { supportsDescribeCluster() apply(cOptions *C.rd_kafka_AdminOptions_t) error @@ -461,7 +461,7 @@ type DescribeClusterAdminOption interface { // DeleteConsumerGroupsAdminOption - see setters. // -// See SetAdminRequestTimeout. +// See SetAdminRequestTimeout, SetAdminOptionIncludeAuthorizedOperations. type DeleteConsumerGroupsAdminOption interface { supportsDeleteConsumerGroups() apply(cOptions *C.rd_kafka_AdminOptions_t) error diff --git a/kafka/integration_test.go b/kafka/integration_test.go index d9cca8597..d48da7317 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -833,7 +833,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DeleteConsumerGroups() { // TestAdminClient_ListAndDescribeConsumerGroups validates the working of the // list consumer groups and describe consumer group APIs of the admin client. // -// We test the following situations: +// We test the following situations: // // 1. One consumer group with one client. // 2. One consumer group with two clients. @@ -1099,7 +1099,7 @@ func (its *IntegrationTestSuite) TestAdminClient_ListAndDescribeConsumerGroups() // working of the DescribeConsumerGroups API of the admin client for fetching // authorized operations (KIP-430). // -// We test the following situations: +// We test the following situations: // // 1. Default ACLs on group. // 2. Modified ACLs on group. @@ -1117,7 +1117,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeConsumerGroupsAuthorize ac := createAdminClientWithSasl(t) defer ac.Close() - // Create a topic - we need to create here because we need 2 partitions. + // Create a topic. ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() _, err := ac.CreateTopics(ctx, []TopicSpecification{ @@ -1222,7 +1222,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeConsumerGroupsAuthorize // TestAdminClient_DescribeCluster validates the working of the // DescribeCluster API of the admin client. // -// We test the following situations: +// We test the following situations: // // 1. DescribeCluster without ACLs. // 2. DescribeCluster with default ACLs. @@ -1340,7 +1340,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { // TestAdminClient_DescribeTopics validates the working of the // DescribeTopics API of the admin client. // -// We test the following situations: +// We test the following situations: // // 1. DescribeTopics without ACLs. // 2. DescribeTopics with default ACLs. @@ -1377,7 +1377,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err := ac.DescribeTopics( - ctx, []string{topic, "nonexistent"}, + ctx, TopicCollection{Names: []string{topic, "nonexistent"}}, SetAdminRequestTimeout(30*time.Second)) assert.Nil(err, "DescribeTopics should not fail") @@ -1433,7 +1433,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err = ac.DescribeTopics( - ctx, []string{topic}, + ctx, TopicCollection{Names: []string{topic}}, SetAdminRequestTimeout(30*time.Second), SetAdminOptionIncludeAuthorizedOperations(true)) @@ -1484,7 +1484,8 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err = ac.DescribeTopics( - ctx, []string{topic}, SetAdminRequestTimeout(time.Second), + ctx, TopicCollection{Names: []string{topic}}, + SetAdminRequestTimeout(time.Second*30), SetAdminOptionIncludeAuthorizedOperations(true)) assert.Nil(err, "DescribeTopics should not fail") diff --git a/kafka/testhelpers.go b/kafka/testhelpers.go index 42e0a774c..f9d0d928f 100644 --- a/kafka/testhelpers.go +++ b/kafka/testhelpers.go @@ -150,6 +150,8 @@ func (cm *ConfigMap) updateFromTestconf() error { } +// updateToSaslAuthentication updates an existing ConfigMap with SASL related +// settings. Returns error in case a broker with SASL is not configured. func (cm *ConfigMap) updateToSaslAuthentication() error { if testconf.BrokersSasl == "" { return errors.New("BrokersSasl must be set in test config") @@ -260,6 +262,9 @@ func waitTopicInMetadata(H Handle, topic string, timeoutMs int) error { } +// createAdminClientImpl is the implementation for createAdminClient and +// createAdminClientWithSasl. It creates a new admin client, or skips the test +// in case it can't be created. func createAdminClientImpl(t *testing.T, withSasl bool) (a *AdminClient) { numver, strver := LibraryVersion() if numver < 0x000b0500 { diff --git a/kafka/testresources/docker-compose.yaml b/kafka/testresources/docker-compose.yaml index b8f5577be..cc170e562 100644 --- a/kafka/testresources/docker-compose.yaml +++ b/kafka/testresources/docker-compose.yaml @@ -33,7 +33,7 @@ services: KAFKA_SUPER_USERS: "User:ANONYMOUS" kafka_auth: - image: confluentinc/cp-kafka:7.5.0 + image: confluentinc/cp-kafka depends_on: - zookeeper_auth ports: @@ -55,4 +55,4 @@ services: KAFKA_ALLOW_EVERYONE_IF_NO_ACL_FOUND: true KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/kafka_jaas.conf" volumes: - - ./kafka_jaas.conf:/etc/kafka/kafka_jaas.conf \ No newline at end of file + - ./kafka_jaas.conf:/etc/kafka/kafka_jaas.conf diff --git a/kafka/testresources/kafka_jaas.conf b/kafka/testresources/kafka_jaas.conf index b86fa85ea..b241ce229 100644 --- a/kafka/testresources/kafka_jaas.conf +++ b/kafka/testresources/kafka_jaas.conf @@ -11,4 +11,3 @@ Client { username="admin" password="admin"; }; - diff --git a/kafka/testresources/zookeeper_jaas.conf b/kafka/testresources/zookeeper_jaas.conf index 404823cb6..29bf4faa8 100644 --- a/kafka/testresources/zookeeper_jaas.conf +++ b/kafka/testresources/zookeeper_jaas.conf @@ -2,4 +2,3 @@ Server { org.apache.zookeeper.server.auth.DigestLoginModule required user_admin="admin"; }; - From f49edf8024e91f8c2c6f4677ca8c5be51b92a86e Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 22 Sep 2023 09:55:09 +0530 Subject: [PATCH 08/12] Address review comments --- .../admin_describe_cluster.go | 2 +- .../admin_describe_topics.go | 8 +- kafka/adminapi.go | 85 +++++++++------ kafka/adminapi_test.go | 4 +- kafka/integration_test.go | 100 +++++++++++++----- kafka/kafka.go | 2 +- kafka/testhelpers.go | 3 +- 7 files changed, 136 insertions(+), 68 deletions(-) diff --git a/examples/admin_describe_cluster/admin_describe_cluster.go b/examples/admin_describe_cluster/admin_describe_cluster.go index 18eee9319..79bcf78c6 100644 --- a/examples/admin_describe_cluster/admin_describe_cluster.go +++ b/examples/admin_describe_cluster/admin_describe_cluster.go @@ -31,7 +31,7 @@ func main() { if len(os.Args) < 3 { fmt.Fprintf( os.Stderr, - "Usage: %s \n", os.Args[0]) os.Exit(1) } diff --git a/examples/admin_describe_topics/admin_describe_topics.go b/examples/admin_describe_topics/admin_describe_topics.go index 7aed6acc2..b1e2aef08 100644 --- a/examples/admin_describe_topics/admin_describe_topics.go +++ b/examples/admin_describe_topics/admin_describe_topics.go @@ -31,7 +31,7 @@ func main() { if len(os.Args) < 4 { fmt.Fprintf( os.Stderr, - "Usage: %s "+ " [ ...]\n", os.Args[0]) os.Exit(1) @@ -61,7 +61,7 @@ func main() { ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() describeTopicsResult, err := a.DescribeTopics( - ctx, kafka.TopicCollection{Names: topics}, + ctx, kafka.NewTopicCollectionOfTopicNames(topics), kafka.SetAdminOptionIncludeAuthorizedOperations( include_authorized_operations)) if err != nil { @@ -75,10 +75,10 @@ func main() { for _, t := range describeTopicsResult.TopicDescriptions { if t.Error.Code() != 0 { fmt.Printf("Topic: %s has error: %s\n", - t.Topic, t.Error) + t.Name, t.Error) continue } - fmt.Printf("Topic: %s has succeeded\n", t.Topic) + fmt.Printf("Topic: %s has succeeded\n", t.Name) if include_authorized_operations { fmt.Printf("Allowed operations: %s\n", t.AuthorizedOperations) } diff --git a/kafka/adminapi.go b/kafka/adminapi.go index 6f9665648..a6a34bd07 100644 --- a/kafka/adminapi.go +++ b/kafka/adminapi.go @@ -302,7 +302,15 @@ type DescribeConsumerGroupsResult struct { // TopicCollection represents a collection of topics. type TopicCollection struct { // Slice of topic names. - Names []string + topicNames []string +} + +// NewTopicCollectionOfTopicNames creates a new TopicCollection based on a list +// of topic names. +func NewTopicCollectionOfTopicNames(names []string) TopicCollection { + return TopicCollection{ + topicNames: names, + } } // Topic Partition information @@ -321,7 +329,7 @@ type TopicPartitionInfo struct { // a single topic. type TopicDescription struct { // Topic name. - Topic string + Name string // Error, if any, of result. Check with `Error.Code() != ErrNoError`. Error Error // Is the topic is internal to Kafka? @@ -1090,24 +1098,33 @@ func (a *AdminClient) cToAuthorizedOperations( } // cToNode converts a C Node_t* to a Go Node. +// cNode must not be nil. func (a *AdminClient) cToNode(cNode *C.rd_kafka_Node_t) Node { - node := Node{ID: -1} - if cNode == nil { - return node + node := Node{ + ID: int(C.rd_kafka_Node_id(cNode)), + Host: C.GoString(C.rd_kafka_Node_host(cNode)), + Port: int(C.rd_kafka_Node_port(cNode)), } - node.Host = C.GoString(C.rd_kafka_Node_host(cNode)) - node.Port = int(C.rd_kafka_Node_port(cNode)) - - cRack := C.rd_kafka_Node_rack_id(cNode) + cRack := C.rd_kafka_Node_rack(cNode) if cRack != nil { rackId := C.GoString(cRack) - node.RackId = &rackId + node.Rack = &rackId } return node } +// cToNodePtr converts a C Node_t* to a Go *Node. +func (a *AdminClient) cToNodePtr(cNode *C.rd_kafka_Node_t) *Node { + if cNode == nil { + return nil + } + + node := a.cToNode(cNode) + return &node +} + // cToNode converts a C Node_t array to a Go Node list. func (a *AdminClient) cToNodes( cNodes **C.rd_kafka_Node_t, cNodeCnt C.size_t) []Node { @@ -1201,10 +1218,7 @@ func (a *AdminClient) cToTopicPartitionInfo( } cLeader := C.rd_kafka_TopicPartitionInfo_leader(partitionInfo) - if cLeader != nil { - leader := a.cToNode(cLeader) - info.Leader = &leader - } + info.Leader = a.cToNodePtr(cLeader) cReplicaCnt := C.size_t(0) cReplicas := C.rd_kafka_TopicPartitionInfo_replicas( @@ -1221,18 +1235,26 @@ func (a *AdminClient) cToTopicPartitionInfo( // cToTopicDescriptions converts a C TopicDescription_t // array to a Go TopicDescription list. func (a *AdminClient) cToTopicDescriptions( - cTopics **C.rd_kafka_TopicDescription_t, - cTopicCount C.size_t) (result []TopicDescription) { - result = make([]TopicDescription, cTopicCount) - for idx := 0; idx < int(cTopicCount); idx++ { + cTopicDescriptions **C.rd_kafka_TopicDescription_t, + cTopicDescriptionCount C.size_t) (result []TopicDescription) { + result = make([]TopicDescription, cTopicDescriptionCount) + for idx := 0; idx < int(cTopicDescriptionCount); idx++ { cTopic := C.TopicDescription_by_idx( - cTopics, cTopicCount, C.size_t(idx)) + cTopicDescriptions, cTopicDescriptionCount, C.size_t(idx)) - topic := C.GoString( + topicName := C.GoString( C.rd_kafka_TopicDescription_name(cTopic)) err := newErrorFromCError( C.rd_kafka_TopicDescription_error(cTopic)) + if err.Code() != ErrNoError { + result[idx] = TopicDescription{ + Name: topicName, + Error: err, + } + continue + } + cPartitionInfoCnt := C.size_t(0) cPartitionInfos := C.rd_kafka_TopicDescription_partitions(cTopic, &cPartitionInfoCnt) @@ -1249,7 +1271,7 @@ func (a *AdminClient) cToTopicDescriptions( authorizedOperations := a.cToAuthorizedOperations(cAuthorizedOperations, cAuthorizedOperationsCnt) result[idx] = TopicDescription{ - Topic: topic, + Name: topicName, Error: err, Partitions: partitions, AuthorizedOperations: authorizedOperations, @@ -1266,10 +1288,7 @@ func (a *AdminClient) cToDescribeClusterResult( var controller *Node = nil cController := C.rd_kafka_DescribeCluster_result_controller(cResult) - if cController != nil { - controllerValue := a.cToNode(cController) - controller = &controllerValue - } + controller = a.cToNodePtr(cController) cNodeCnt := C.size_t(0) cNodes := C.rd_kafka_DescribeCluster_result_nodes(cResult, &cNodeCnt) @@ -2642,10 +2661,10 @@ func (a *AdminClient) DescribeTopics( } // Convert topic names into char**. - cTopicNameList := make([]*C.char, len(topics.Names)) - cTopicNameCount := C.size_t(len(topics.Names)) + cTopicNameList := make([]*C.char, len(topics.topicNames)) + cTopicNameCount := C.size_t(len(topics.topicNames)) - for idx, topic := range topics.Names { + for idx, topic := range topics.topicNames { cTopicNameList[idx] = C.CString(topic) defer C.free(unsafe.Pointer(cTopicNameList[idx])) } @@ -2656,7 +2675,7 @@ func (a *AdminClient) DescribeTopics( } // Convert char** of topic names into rd_kafka_TopicCollection_t* - cTopicCollection := C.rd_kafka_TopicCollection_new_from_names( + cTopicCollection := C.rd_kafka_TopicCollection_of_topic_names( cTopicNameListPtr, cTopicNameCount) defer C.rd_kafka_TopicCollection_destroy(cTopicCollection) @@ -2694,9 +2713,11 @@ func (a *AdminClient) DescribeTopics( cRes := C.rd_kafka_event_DescribeTopics_result(rkev) // Convert result from C to Go. - var cTopicCount C.size_t - cTopics := C.rd_kafka_DescribeTopics_result_topics(cRes, &cTopicCount) - describeResult.TopicDescriptions = a.cToTopicDescriptions(cTopics, cTopicCount) + var cTopicDescriptionCount C.size_t + cTopicDescriptions := + C.rd_kafka_DescribeTopics_result_topics(cRes, &cTopicDescriptionCount) + describeResult.TopicDescriptions = + a.cToTopicDescriptions(cTopicDescriptions, cTopicDescriptionCount) return describeResult, nil } diff --git a/kafka/adminapi_test.go b/kafka/adminapi_test.go index cdf9ae8bc..abbc47bfd 100644 --- a/kafka/adminapi_test.go +++ b/kafka/adminapi_test.go @@ -490,7 +490,7 @@ func testAdminAPIsDescribeTopics( ctx, cancel := context.WithTimeout(context.Background(), expDuration) defer cancel() descres, err := a.DescribeTopics( - ctx, TopicCollection{}, SetAdminRequestTimeout(time.Second)) + ctx, NewTopicCollectionOfTopicNames(nil), SetAdminRequestTimeout(time.Second)) if descres.TopicDescriptions != nil || err == nil { t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", descres, err) @@ -502,7 +502,7 @@ func testAdminAPIsDescribeTopics( ctx, cancel = context.WithTimeout(context.Background(), expDuration) defer cancel() descres, err = a.DescribeTopics( - ctx, TopicCollection{Names: []string{"test"}}, + ctx, NewTopicCollectionOfTopicNames([]string{"test"}), SetAdminRequestTimeout(time.Second)) if descres.TopicDescriptions != nil || err == nil { t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v", diff --git a/kafka/integration_test.go b/kafka/integration_test.go index d48da7317..f1681894a 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -1138,9 +1138,13 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeConsumerGroupsAuthorize // Create a consumer so that a consumer group might be created config := &ConfigMap{ - "bootstrap.servers": testconf.Brokers, + "bootstrap.servers": testconf.BrokersSasl, "group.id": groupID, "client.id": clientID, + "sasl.username": testconf.SaslUsername, + "sasl.password": testconf.SaslPassword, + "sasl.mechanism": testconf.SaslMechanism, + "security.protocol": "SASL_PLAINTEXT", } config.updateFromTestconf() consumer, err := NewConsumer(config) @@ -1166,6 +1170,8 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeConsumerGroupsAuthorize assert.Len(groupDescs, 1, "Describing one group should give exactly one result") groupDesc := &groupDescs[0] + assert.Equal(groupDesc.Error.Code(), ErrNoError, + "Group description should succeed") assert.NotEmpty(groupDesc.AuthorizedOperations, "Authorized operations should not be empty") assert.ElementsMatch(groupDesc.AuthorizedOperations, @@ -1190,6 +1196,26 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeConsumerGroupsAuthorize ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() + // Delete group ACLs to keep the test cluster clean. + defer func() { + aclBindingFilters := ACLBindingFilters{ + { + Type: ResourceGroup, + Name: groupID, + ResourcePatternType: ResourcePatternTypeLiteral, + Principal: "User:*", + Host: "*", + Operation: ACLOperationRead, + PermissionType: ACLPermissionTypeAllow, + }, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + _, err = ac.DeleteACLs(ctx, aclBindingFilters, + SetAdminRequestTimeout(time.Second*30)) + assert.Nil(err, "DeleteACLs should not throw an error") + }() + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(time.Second)) assert.Nil(err, "CreateACLs should not throw an error") @@ -1300,6 +1326,26 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { resultCreateACLs[0].Error.Code(), ErrNoError, "CreateACLs result should not have an error") + // Clean up cluster ACLs for subsequent tests. + defer func() { + aclBindingFilters := ACLBindingFilters{ + { + Type: ResourceBroker, + Name: "kafka-cluster", + ResourcePatternType: ResourcePatternTypeMatch, + Principal: "User:*", + Host: "*", + Operation: ACLOperationAlter, + PermissionType: ACLPermissionTypeAllow, + }, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + _, err = ac.DeleteACLs(ctx, aclBindingFilters, + SetAdminRequestTimeout(time.Second*30)) + assert.Nil(err, "DeleteACLs should not throw an error") + }() + // 3. DescribeCluster with modified ACLs. ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() @@ -1317,24 +1363,6 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeCluster() { // Alter permissions implicitly allow Describe. assert.ElementsMatch(descres.AuthorizedOperations, []ACLOperation{ACLOperationDescribe, ACLOperationAlter}) - - // Clean up cluster ACLs for subsequent tests. - aclBindingFilters := ACLBindingFilters{ - { - Type: ResourceBroker, - Name: "kafka-cluster", - ResourcePatternType: ResourcePatternTypeMatch, - Principal: "User:*", - Host: "*", - Operation: ACLOperationAlter, - PermissionType: ACLPermissionTypeAllow, - }, - } - ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - _, err = ac.DeleteACLs(ctx, aclBindingFilters, - SetAdminRequestTimeout(time.Second*30)) - assert.Nil(err, "DeleteACLs should not throw an error") } // TestAdminClient_DescribeTopics validates the working of the @@ -1377,16 +1405,16 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err := ac.DescribeTopics( - ctx, TopicCollection{Names: []string{topic, "nonexistent"}}, + ctx, NewTopicCollectionOfTopicNames([]string{topic, "nonexistent"}), SetAdminRequestTimeout(30*time.Second)) assert.Nil(err, "DescribeTopics should not fail") topicDescs := topicDescResult.TopicDescriptions assert.Len(topicDescs, 2, "Describing two topics should give exactly two results") - assert.Equal(topicDescs[0].Topic, topic, + assert.Equal(topicDescs[0].Name, topic, "First result topic should match request topic") - assert.Equal(topicDescs[1].Topic, "nonexistent", + assert.Equal(topicDescs[1].Name, "nonexistent", "Second result topic should match request topic") assert.Equal(topicDescs[1].Error.Code(), ErrUnknownTopicOrPart, "Expected correct error for nonexistent topic") @@ -1433,7 +1461,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err = ac.DescribeTopics( - ctx, TopicCollection{Names: []string{topic}}, + ctx, NewTopicCollectionOfTopicNames([]string{topic}), SetAdminRequestTimeout(30*time.Second), SetAdminOptionIncludeAuthorizedOperations(true)) @@ -1442,7 +1470,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { topicDescs = topicDescResult.TopicDescriptions assert.Len(topicDescs, 1, "Describing one topic should give exactly one result") - assert.Equal(topicDescs[0].Topic, topic, + assert.Equal(topicDescs[0].Name, topic, "First result topic should match request topic") topicDesc = topicDescs[0] @@ -1471,6 +1499,26 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() + // Delete topic ACLs to keep the test cluster clean. + defer func() { + aclBindingFilters := ACLBindingFilters{ + { + Type: ResourceTopic, + Name: topic, + ResourcePatternType: ResourcePatternTypeLiteral, + Principal: "User:*", + Host: "*", + Operation: ACLOperationRead, + PermissionType: ACLPermissionTypeAllow, + }, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + _, err = ac.DeleteACLs(ctx, aclBindingFilters, + SetAdminRequestTimeout(time.Second*30)) + assert.Nil(err, "DeleteACLs should not throw an error") + }() + resultCreateACLs, err := ac.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(time.Second)) assert.Nil(err, "CreateACLs should not throw an error") @@ -1484,7 +1532,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) defer cancel() topicDescResult, err = ac.DescribeTopics( - ctx, TopicCollection{Names: []string{topic}}, + ctx, NewTopicCollectionOfTopicNames([]string{topic}), SetAdminRequestTimeout(time.Second*30), SetAdminOptionIncludeAuthorizedOperations(true)) @@ -1493,7 +1541,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { topicDescs = topicDescResult.TopicDescriptions assert.Len(topicDescs, 1, "Describing one topic should give exactly one result") - assert.Equal(topicDescs[0].Topic, topic, + assert.Equal(topicDescs[0].Name, topic, "First result topic should match request topic") topicDesc = topicDescs[0] diff --git a/kafka/kafka.go b/kafka/kafka.go index 1dae2a32f..69550b1ec 100644 --- a/kafka/kafka.go +++ b/kafka/kafka.go @@ -327,7 +327,7 @@ type Node struct { // Node port. Port int // Node rack (may be nil) - RackId *string + Rack *string } func (n Node) String() string { diff --git a/kafka/testhelpers.go b/kafka/testhelpers.go index f9d0d928f..dcffe576e 100644 --- a/kafka/testhelpers.go +++ b/kafka/testhelpers.go @@ -275,8 +275,7 @@ func createAdminClientImpl(t *testing.T, withSasl bool) (a *AdminClient) { t.Skipf("Missing testconf.json") } - brokers := testconf.Brokers - conf := ConfigMap{"bootstrap.servers": brokers} + conf := ConfigMap{"bootstrap.servers": testconf.Brokers} conf.updateFromTestconf() if withSasl { if err := conf.updateToSaslAuthentication(); err != nil { From d7dc5af12c987bb9342bd486516675f0243fe589 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Fri, 13 Oct 2023 00:30:01 +0530 Subject: [PATCH 09/12] Added topic id to describe topics response --- .../admin_describe_topics.go | 1 + kafka/adminapi.go | 14 ++++++++++++ kafka/integration_test.go | 1 + kafka/kafka.go | 22 +++++++++++++++++++ 4 files changed, 38 insertions(+) diff --git a/examples/admin_describe_topics/admin_describe_topics.go b/examples/admin_describe_topics/admin_describe_topics.go index b1e2aef08..3a505c76e 100644 --- a/examples/admin_describe_topics/admin_describe_topics.go +++ b/examples/admin_describe_topics/admin_describe_topics.go @@ -79,6 +79,7 @@ func main() { continue } fmt.Printf("Topic: %s has succeeded\n", t.Name) + fmt.Printf("Topic Id: %s\n", t.TopicId) if include_authorized_operations { fmt.Printf("Allowed operations: %s\n", t.AuthorizedOperations) } diff --git a/kafka/adminapi.go b/kafka/adminapi.go index a6a34bd07..a84166b88 100644 --- a/kafka/adminapi.go +++ b/kafka/adminapi.go @@ -330,6 +330,8 @@ type TopicPartitionInfo struct { type TopicDescription struct { // Topic name. Name string + // Topic Id + TopicId Uuid // Error, if any, of result. Check with `Error.Code() != ErrNoError`. Error Error // Is the topic is internal to Kafka? @@ -1097,6 +1099,16 @@ func (a *AdminClient) cToAuthorizedOperations( return authorizedOperations } +// cToUuid converts a C rd_kafka_Uuid_t to a Go Uuid. +func (a *AdminClient) cToUuid(cUuid *C.rd_kafka_Uuid_t) Uuid { + uuid := Uuid{ + mostSignificantBits: int64(C.rd_kafka_Uuid_most_significant_bits(cUuid)), + leastSignificantBits: int64(C.rd_kafka_Uuid_least_significant_bits(cUuid)), + base64str: C.GoString(C.rd_kafka_Uuid_base64str(cUuid)), + } + return uuid +} + // cToNode converts a C Node_t* to a Go Node. // cNode must not be nil. func (a *AdminClient) cToNode(cNode *C.rd_kafka_Node_t) Node { @@ -1244,6 +1256,7 @@ func (a *AdminClient) cToTopicDescriptions( topicName := C.GoString( C.rd_kafka_TopicDescription_name(cTopic)) + topicId := a.cToUuid(C.rd_kafka_TopicDescription_topic_id(cTopic)) err := newErrorFromCError( C.rd_kafka_TopicDescription_error(cTopic)) @@ -1272,6 +1285,7 @@ func (a *AdminClient) cToTopicDescriptions( result[idx] = TopicDescription{ Name: topicName, + TopicId: topicId, Error: err, Partitions: partitions, AuthorizedOperations: authorizedOperations, diff --git a/kafka/integration_test.go b/kafka/integration_test.go index f1681894a..e82461d1a 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -1420,6 +1420,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { "Expected correct error for nonexistent topic") topicDesc := topicDescs[0] + assert.NotNil(topicDesc.TopicId) assert.Equal(topicDesc.Error.Code(), ErrNoError, "Topic description should not have an error") assert.False(topicDesc.IsInternal, "Topic should not be internal") diff --git a/kafka/kafka.go b/kafka/kafka.go index 69550b1ec..92cb0a125 100644 --- a/kafka/kafka.go +++ b/kafka/kafka.go @@ -334,6 +334,28 @@ func (n Node) String() string { return fmt.Sprintf("[%s:%d]/%d", n.Host, n.Port, n.ID) } +// Uuid. +type Uuid struct { + // Most Significant Bits. + mostSignificantBits int64 + // Least Significant Bits. + leastSignificantBits int64 + // Base64 representation + base64str string +} + +func (uuid Uuid) String() string { + return uuid.base64str +} + +func (uuid Uuid) GetMostSignificantBits() int64 { + return uuid.mostSignificantBits +} + +func (uuid Uuid) GetLeastSignificantBits() int64 { + return uuid.leastSignificantBits +} + // ConsumerGroupTopicPartitions represents a consumer group's TopicPartitions. type ConsumerGroupTopicPartitions struct { // Group name From a0f0db72b7c53373db67f8aa7bd7b28c10eec3c4 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Fri, 20 Oct 2023 18:31:38 +0530 Subject: [PATCH 10/12] Fixed lint issue --- .../admin_describe_topics.go | 2 +- kafka/adminapi.go | 18 +++++++++--------- kafka/integration_test.go | 2 +- kafka/kafka.go | 13 ++++++++----- 4 files changed, 19 insertions(+), 16 deletions(-) diff --git a/examples/admin_describe_topics/admin_describe_topics.go b/examples/admin_describe_topics/admin_describe_topics.go index 0a04f5f91..42ad101aa 100644 --- a/examples/admin_describe_topics/admin_describe_topics.go +++ b/examples/admin_describe_topics/admin_describe_topics.go @@ -79,7 +79,7 @@ func main() { continue } fmt.Printf("Topic: %s has succeeded\n", t.Name) - fmt.Printf("Topic Id: %s\n", t.TopicId) + fmt.Printf("Topic Id: %s\n", t.TopicID) if includeAuthorizedOperations { fmt.Printf("Allowed operations: %s\n", t.AuthorizedOperations) } diff --git a/kafka/adminapi.go b/kafka/adminapi.go index bcd34ffb5..970ad0f59 100644 --- a/kafka/adminapi.go +++ b/kafka/adminapi.go @@ -332,7 +332,7 @@ type TopicDescription struct { // Topic name. Name string // Topic Id - TopicId Uuid + TopicID UUID // Error, if any, of the result. Check with `Error.Code() != ErrNoError`. Error Error // Is the topic internal to Kafka? @@ -1100,12 +1100,12 @@ func (a *AdminClient) cToAuthorizedOperations( return authorizedOperations } -// cToUuid converts a C rd_kafka_Uuid_t to a Go Uuid. -func (a *AdminClient) cToUuid(cUuid *C.rd_kafka_Uuid_t) Uuid { - uuid := Uuid{ - mostSignificantBits: int64(C.rd_kafka_Uuid_most_significant_bits(cUuid)), - leastSignificantBits: int64(C.rd_kafka_Uuid_least_significant_bits(cUuid)), - base64str: C.GoString(C.rd_kafka_Uuid_base64str(cUuid)), +// cToUUID converts a C rd_kafka_Uuid_t to a Go UUID. +func (a *AdminClient) cToUUID(cUUID *C.rd_kafka_Uuid_t) UUID { + uuid := UUID{ + mostSignificantBits: int64(C.rd_kafka_Uuid_most_significant_bits(cUUID)), + leastSignificantBits: int64(C.rd_kafka_Uuid_least_significant_bits(cUUID)), + base64str: C.GoString(C.rd_kafka_Uuid_base64str(cUUID)), } return uuid } @@ -1257,7 +1257,7 @@ func (a *AdminClient) cToTopicDescriptions( topicName := C.GoString( C.rd_kafka_TopicDescription_name(cTopic)) - topicId := a.cToUuid(C.rd_kafka_TopicDescription_topic_id(cTopic)) + TopicID := a.cToUUID(C.rd_kafka_TopicDescription_topic_id(cTopic)) err := newErrorFromCError( C.rd_kafka_TopicDescription_error(cTopic)) @@ -1286,7 +1286,7 @@ func (a *AdminClient) cToTopicDescriptions( result[idx] = TopicDescription{ Name: topicName, - TopicId: topicId, + TopicID: TopicID, Error: err, Partitions: partitions, AuthorizedOperations: authorizedOperations, diff --git a/kafka/integration_test.go b/kafka/integration_test.go index c921d80c4..24ba97f36 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -1420,7 +1420,7 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { "Expected correct error for nonexistent topic") topicDesc := topicDescs[0] - assert.NotNil(topicDesc.TopicId) + assert.NotNil(topicDesc.TopicID) assert.Equal(topicDesc.Error.Code(), ErrNoError, "Topic description should not have an error") assert.False(topicDesc.IsInternal, "Topic should not be internal") diff --git a/kafka/kafka.go b/kafka/kafka.go index 92cb0a125..cf53c00a5 100644 --- a/kafka/kafka.go +++ b/kafka/kafka.go @@ -334,8 +334,8 @@ func (n Node) String() string { return fmt.Sprintf("[%s:%d]/%d", n.Host, n.Port, n.ID) } -// Uuid. -type Uuid struct { +// Kafka UUID representation +type UUID struct { // Most Significant Bits. mostSignificantBits int64 // Least Significant Bits. @@ -344,15 +344,18 @@ type Uuid struct { base64str string } -func (uuid Uuid) String() string { +// Base64 string representation of the UUID +func (uuid UUID) String() string { return uuid.base64str } -func (uuid Uuid) GetMostSignificantBits() int64 { +// Returns Most Significant 64 bits of the 128 bits UUID +func (uuid UUID) GetMostSignificantBits() int64 { return uuid.mostSignificantBits } -func (uuid Uuid) GetLeastSignificantBits() int64 { +// Returns Least Significant 64 bits of the 128 bits UUID +func (uuid UUID) GetLeastSignificantBits() int64 { return uuid.leastSignificantBits } From e1116a78fb823cf11ae1439381a1395924929095 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Fri, 20 Oct 2023 18:45:09 +0530 Subject: [PATCH 11/12] Fixed more lint issues --- kafka/kafka.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/kafka.go b/kafka/kafka.go index cf53c00a5..d349f0e86 100644 --- a/kafka/kafka.go +++ b/kafka/kafka.go @@ -334,7 +334,7 @@ func (n Node) String() string { return fmt.Sprintf("[%s:%d]/%d", n.Host, n.Port, n.ID) } -// Kafka UUID representation +// UUID Kafka UUID representation type UUID struct { // Most Significant Bits. mostSignificantBits int64 @@ -349,12 +349,12 @@ func (uuid UUID) String() string { return uuid.base64str } -// Returns Most Significant 64 bits of the 128 bits UUID +// GetMostSignificantBits returns Most Significant 64 bits of the 128 bits UUID func (uuid UUID) GetMostSignificantBits() int64 { return uuid.mostSignificantBits } -// Returns Least Significant 64 bits of the 128 bits UUID +// GetLeastSignificantBits returns Least Significant 64 bits of the 128 bits UUID func (uuid UUID) GetLeastSignificantBits() int64 { return uuid.leastSignificantBits } From 1d0eb62dc9ba50a5fae31c81d5c490a66f47ee0f Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Sat, 21 Oct 2023 19:44:04 +0530 Subject: [PATCH 12/12] PR comment --- kafka/integration_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/integration_test.go b/kafka/integration_test.go index 24ba97f36..ed57d20ee 100644 --- a/kafka/integration_test.go +++ b/kafka/integration_test.go @@ -1420,7 +1420,9 @@ func (its *IntegrationTestSuite) TestAdminClient_DescribeTopics() { "Expected correct error for nonexistent topic") topicDesc := topicDescs[0] - assert.NotNil(topicDesc.TopicID) + assert.NotZero(topicDesc.TopicID.GetLeastSignificantBits()) + assert.NotZero(topicDesc.TopicID.GetMostSignificantBits()) + assert.NotEmpty(topicDesc.TopicID.String()) assert.Equal(topicDesc.Error.Code(), ErrNoError, "Topic description should not have an error") assert.False(topicDesc.IsInternal, "Topic should not be internal")