Skip to content

Commit cd781ba

Browse files
emasabmilindl
andauthored
Fix calls with empty lists (#1084)
* Empty topic collection and increased test coverage * Empty partition map returns empty ResultInfos Supports request timeout ResultsInfos -> ResultInfos Increased test coverage * Update documentation --------- Co-authored-by: Milind L <[email protected]> Co-authored-by: Milind L <[email protected]>
1 parent 1c0a3cd commit cd781ba

File tree

5 files changed

+219
-35
lines changed

5 files changed

+219
-35
lines changed

examples/admin_list_offsets/admin_list_offsets.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -106,7 +106,7 @@ func main() {
106106
}
107107
// map[TopicPartition]ListOffsetsResultInfo
108108
// Print results
109-
for tp, info := range results.ResultsInfos {
109+
for tp, info := range results.ResultInfos {
110110
fmt.Printf("Topic: %s Partition: %d\n", *tp.Topic, tp.Partition)
111111
if info.Error.Code() != kafka.ErrNoError {
112112
fmt.Printf(" ErrorCode: %d ErrorMessage: %s\n\n", info.Error.Code(), info.Error.String())

kafka/adminapi.go

Lines changed: 17 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1022,7 +1022,7 @@ type ListOffsetsResultInfo struct {
10221022

10231023
// ListOffsetsResult holds the map of TopicPartition to ListOffsetsResultInfo for a request.
10241024
type ListOffsetsResult struct {
1025-
ResultsInfos map[TopicPartition]ListOffsetsResultInfo
1025+
ResultInfos map[TopicPartition]ListOffsetsResultInfo
10261026
}
10271027

10281028
// waitResult waits for a result event on cQueue or the ctx to be cancelled, whichever happens
@@ -1140,9 +1140,9 @@ func (a *AdminClient) cToAuthorizedOperations(
11401140
// cToUUID converts a C rd_kafka_Uuid_t to a Go UUID.
11411141
func (a *AdminClient) cToUUID(cUUID *C.rd_kafka_Uuid_t) UUID {
11421142
uuid := UUID{
1143-
mostSignificantBits: int64(C.rd_kafka_Uuid_most_significant_bits(cUUID)),
1143+
mostSignificantBits: int64(C.rd_kafka_Uuid_most_significant_bits(cUUID)),
11441144
leastSignificantBits: int64(C.rd_kafka_Uuid_least_significant_bits(cUUID)),
1145-
base64str: C.GoString(C.rd_kafka_Uuid_base64str(cUUID)),
1145+
base64str: C.GoString(C.rd_kafka_Uuid_base64str(cUUID)),
11461146
}
11471147
return uuid
11481148
}
@@ -1420,7 +1420,7 @@ func cToDescribeUserScramCredentialsResult(
14201420
// cToListOffsetsResult converts a C
14211421
// rd_kafka_ListOffsets_result_t to a Go ListOffsetsResult
14221422
func cToListOffsetsResult(cRes *C.rd_kafka_ListOffsets_result_t) (result ListOffsetsResult) {
1423-
result = ListOffsetsResult{ResultsInfos: make(map[TopicPartition]ListOffsetsResultInfo)}
1423+
result = ListOffsetsResult{ResultInfos: make(map[TopicPartition]ListOffsetsResultInfo)}
14241424
var cPartitionCount C.size_t
14251425
cResultInfos := C.rd_kafka_ListOffsets_result_infos(cRes, &cPartitionCount)
14261426
for itr := 0; itr < int(cPartitionCount); itr++ {
@@ -1436,7 +1436,7 @@ func cToListOffsetsResult(cRes *C.rd_kafka_ListOffsets_result_t) (result ListOff
14361436
resultInfo.LeaderEpoch = &cLeaderEpoch
14371437
}
14381438
resultInfo.Error = newError(cPartition.err)
1439-
result.ResultsInfos[Partition] = resultInfo
1439+
result.ResultInfos[Partition] = resultInfo
14401440
}
14411441
return result
14421442
}
@@ -2728,7 +2728,8 @@ func (a *AdminClient) DescribeConsumerGroups(
27282728
// Parameters:
27292729
// - `ctx` - context with the maximum amount of time to block, or nil for
27302730
// indefinite.
2731-
// - `topics` - Collection of topics to describe. This should not be nil/empty.
2731+
// - `topics` - Collection of topics to describe. This should not have nil
2732+
// topic names.
27322733
// - `options` - DescribeTopicsAdminOption options.
27332734
//
27342735
// Returns DescribeTopicsResult, which contains a slice of
@@ -2751,6 +2752,11 @@ func (a *AdminClient) DescribeTopics(
27512752
cTopicNameList := make([]*C.char, len(topics.topicNames))
27522753
cTopicNameCount := C.size_t(len(topics.topicNames))
27532754

2755+
if topics.topicNames == nil {
2756+
return describeResult, newErrorFromString(ErrInvalidArg,
2757+
"TopicCollection of topic names cannot be nil")
2758+
}
2759+
27542760
for idx, topic := range topics.topicNames {
27552761
cTopicNameList[idx] = C.CString(topic)
27562762
defer C.free(unsafe.Pointer(cTopicNameList[idx]))
@@ -3216,18 +3222,20 @@ func (a *AdminClient) DescribeUserScramCredentials(
32163222
// specified TopicPartiton based on an OffsetSpec.
32173223
//
32183224
// Parameters:
3225+
//
32193226
// - `ctx` - context with the maximum amount of time to block, or nil for
32203227
// indefinite.
3221-
// - `topicPartitionOffsets` - a map from TopicPartition to OffsetSpec, it holds either the OffsetSpec enum value or timestamp.
3228+
// - `topicPartitionOffsets` - a map from TopicPartition to OffsetSpec, it
3229+
// holds either the OffsetSpec enum value or timestamp. Must not be nil.
32223230
// - `options` - ListOffsetsAdminOption options.
32233231
//
32243232
// Returns a ListOffsetsResult.
32253233
// Each TopicPartition's ListOffset can have an individual error.
32263234
func (a *AdminClient) ListOffsets(
32273235
ctx context.Context, topicPartitionOffsets map[TopicPartition]OffsetSpec,
32283236
options ...ListOffsetsAdminOption) (result ListOffsetsResult, err error) {
3229-
if len(topicPartitionOffsets) < 1 || topicPartitionOffsets == nil {
3230-
return result, newErrorFromString(ErrInvalidArg, "expected topicPartitionOffsets of size greater or equal 1.")
3237+
if topicPartitionOffsets == nil {
3238+
return result, newErrorFromString(ErrInvalidArg, "expected topicPartitionOffsets parameter.")
32313239
}
32323240

32333241
topicPartitions := C.rd_kafka_topic_partition_list_new(C.int(len(topicPartitionOffsets)))

kafka/adminapi_test.go

Lines changed: 196 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -487,29 +487,73 @@ func testAdminAPIsDescribeConsumerGroups(
487487

488488
func testAdminAPIsDescribeTopics(
489489
what string, a *AdminClient, expDuration time.Duration, t *testing.T) {
490-
ctx, cancel := context.WithTimeout(context.Background(), expDuration)
491-
defer cancel()
492-
descres, err := a.DescribeTopics(
493-
ctx, NewTopicCollectionOfTopicNames(nil), SetAdminRequestTimeout(time.Second))
494-
if descres.TopicDescriptions != nil || err == nil {
495-
t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v",
496-
descres, err)
497-
}
498-
if err.(Error).Code() != ErrInvalidArg {
499-
t.Fatalf("Expected ErrInvalidArg with empty topics list, but got %s", err)
500-
}
490+
requestTimeout := SetAdminRequestTimeout(time.Second)
491+
for _, options := range [][]DescribeTopicsAdminOption{
492+
{},
493+
{requestTimeout},
494+
{requestTimeout, SetAdminOptionIncludeAuthorizedOperations(true)},
495+
{SetAdminOptionIncludeAuthorizedOperations(false)},
496+
} {
497+
498+
// nil slice gives error
499+
ctx, cancel := context.WithTimeout(context.Background(), expDuration)
500+
defer cancel()
501+
descres, err := a.DescribeTopics(
502+
ctx, NewTopicCollectionOfTopicNames(nil), options...)
503+
if descres.TopicDescriptions != nil || err == nil {
504+
t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v",
505+
descres, err)
506+
}
507+
if err.(Error).Code() != ErrInvalidArg {
508+
t.Fatalf("Expected ErrInvalidArg with nil slice, but got %s", err)
509+
}
501510

502-
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
503-
defer cancel()
504-
descres, err = a.DescribeTopics(
505-
ctx, NewTopicCollectionOfTopicNames([]string{"test"}),
506-
SetAdminRequestTimeout(time.Second))
507-
if descres.TopicDescriptions != nil || err == nil {
508-
t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v",
509-
descres, err)
510-
}
511-
if ctx.Err() != context.DeadlineExceeded {
512-
t.Fatalf("Expected DeadlineExceeded, not %s %v", err.(Error).Code(), ctx.Err())
511+
// Empty slice returns empty TopicDescription slice
512+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
513+
defer cancel()
514+
descres, err = a.DescribeTopics(
515+
ctx, NewTopicCollectionOfTopicNames([]string{}), options...)
516+
if descres.TopicDescriptions == nil || err != nil {
517+
t.Fatalf("Expected DescribeTopics to succeed, but got result: %v, err: %v",
518+
descres, err)
519+
}
520+
if len(descres.TopicDescriptions) > 0 {
521+
t.Fatalf("Expected an empty TopicDescription slice, but got %d elements",
522+
len(descres.TopicDescriptions))
523+
}
524+
525+
// Empty topic names
526+
for _, topicCollection := range []TopicCollection{
527+
NewTopicCollectionOfTopicNames([]string{""}),
528+
NewTopicCollectionOfTopicNames([]string{"correct", ""}),
529+
} {
530+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
531+
defer cancel()
532+
descres, err = a.DescribeTopics(
533+
ctx, topicCollection,
534+
options...)
535+
if descres.TopicDescriptions != nil || err == nil {
536+
t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v",
537+
descres, err)
538+
}
539+
if err.(Error).Code() != ErrInvalidArg {
540+
t.Fatalf("Expected ErrInvalidArg, not %d %v", err.(Error).Code(), err.Error())
541+
}
542+
}
543+
544+
// Normal call
545+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
546+
defer cancel()
547+
descres, err = a.DescribeTopics(
548+
ctx, NewTopicCollectionOfTopicNames([]string{"test"}),
549+
options...)
550+
if descres.TopicDescriptions != nil || err == nil {
551+
t.Fatalf("Expected DescribeTopics to fail, but got result: %v, err: %v",
552+
descres, err)
553+
}
554+
if ctx.Err() != context.DeadlineExceeded {
555+
t.Fatalf("Expected DeadlineExceeded, not %s %v", err.(Error).Code(), ctx.Err())
556+
}
513557
}
514558
}
515559

@@ -638,6 +682,135 @@ func testAdminAPIsAlterConsumerGroupOffsets(
638682
t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err())
639683
}
640684
}
685+
686+
func testAdminAPIsListOffsets(
687+
what string, a *AdminClient, expDuration time.Duration, t *testing.T) {
688+
topic := "test"
689+
invalidTopic := ""
690+
requestTimeout := SetAdminRequestTimeout(time.Second)
691+
692+
// Invalid option value
693+
ctx, cancel := context.WithTimeout(context.Background(), expDuration)
694+
defer cancel()
695+
result, err := a.ListOffsets(
696+
ctx,
697+
map[TopicPartition]OffsetSpec{}, SetAdminRequestTimeout(-1))
698+
if result.ResultInfos != nil || err == nil {
699+
t.Fatalf("Expected ListOffsets to fail, but got result: %v, err: %v",
700+
result, err)
701+
}
702+
if err.(Error).Code() != ErrInvalidArg {
703+
t.Fatalf("Expected ErrInvalidArg, not %v", err)
704+
}
705+
706+
for _, options := range [][]ListOffsetsAdminOption{
707+
{},
708+
{requestTimeout},
709+
{requestTimeout, SetAdminIsolationLevel(IsolationLevelReadUncommitted)},
710+
{SetAdminIsolationLevel(IsolationLevelReadCommitted)},
711+
} {
712+
// nil argument should fail, not being treated as empty
713+
ctx, cancel := context.WithTimeout(context.Background(), expDuration)
714+
defer cancel()
715+
result, err := a.ListOffsets(
716+
ctx,
717+
nil, options...)
718+
if result.ResultInfos != nil || err == nil {
719+
t.Fatalf("Expected ListOffsets to fail, but got result: %v, err: %v",
720+
result, err)
721+
}
722+
if err.(Error).Code() != ErrInvalidArg {
723+
t.Fatalf("Expected ErrInvalidArg, not %v", err)
724+
}
725+
726+
// Empty map returns empty ResultInfos
727+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
728+
defer cancel()
729+
result, err = a.ListOffsets(
730+
ctx,
731+
map[TopicPartition]OffsetSpec{}, options...)
732+
if result.ResultInfos == nil || err != nil {
733+
t.Fatalf("Expected ListOffsets to succeed, but got result: %v, err: %v",
734+
result, err)
735+
}
736+
if len(result.ResultInfos) > 0 {
737+
t.Fatalf("Expected empty ResultInfos, not %v", result.ResultInfos)
738+
}
739+
740+
// Invalid TopicPartition
741+
for _, topicPartitionOffsets := range []map[TopicPartition]OffsetSpec{
742+
{{Topic: &invalidTopic, Partition: 0}: EarliestOffsetSpec},
743+
{{Topic: &topic, Partition: -1}: EarliestOffsetSpec},
744+
} {
745+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
746+
defer cancel()
747+
result, err = a.ListOffsets(
748+
ctx,
749+
topicPartitionOffsets, options...)
750+
if result.ResultInfos != nil || err == nil {
751+
t.Fatalf("Expected ListOffsets to fail, but got result: %v, err: %v",
752+
result, err)
753+
}
754+
if err.(Error).Code() != ErrInvalidArg {
755+
t.Fatalf("Expected ErrInvalidArg, not %v", err)
756+
}
757+
}
758+
759+
// Same partition with different offsets
760+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
761+
defer cancel()
762+
topicPartitionOffsets := map[TopicPartition]OffsetSpec{
763+
{Topic: &topic, Partition: 0, Offset: 10}: EarliestOffsetSpec,
764+
{Topic: &topic, Partition: 0, Offset: 20}: LatestOffsetSpec,
765+
}
766+
result, err = a.ListOffsets(
767+
ctx,
768+
topicPartitionOffsets, options...)
769+
if result.ResultInfos != nil || err == nil {
770+
t.Fatalf("Expected ListOffsets to fail, but got result: %v, err: %v",
771+
result, err)
772+
}
773+
if err.(Error).Code() != ErrInvalidArg {
774+
t.Fatalf("Expected ErrInvalidArg, not %v", err)
775+
}
776+
777+
// Two different partitions
778+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
779+
defer cancel()
780+
topicPartitionOffsets = map[TopicPartition]OffsetSpec{
781+
{Topic: &topic, Partition: 0, Offset: 10}: EarliestOffsetSpec,
782+
{Topic: &topic, Partition: 1, Offset: 20}: EarliestOffsetSpec,
783+
}
784+
result, err = a.ListOffsets(
785+
ctx,
786+
topicPartitionOffsets, options...)
787+
if result.ResultInfos != nil || err == nil {
788+
t.Fatalf("Expected ListOffsets to fail, but got result: %v, err: %v",
789+
result, err)
790+
}
791+
if ctx.Err() != context.DeadlineExceeded {
792+
t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err())
793+
}
794+
795+
// Single partition
796+
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
797+
defer cancel()
798+
topicPartitionOffsets = map[TopicPartition]OffsetSpec{
799+
{Topic: &topic, Partition: 0}: EarliestOffsetSpec,
800+
}
801+
result, err = a.ListOffsets(
802+
ctx,
803+
topicPartitionOffsets, options...)
804+
if result.ResultInfos != nil || err == nil {
805+
t.Fatalf("Expected ListOffsets to fail, but got result: %v, err: %v",
806+
result, err)
807+
}
808+
if ctx.Err() != context.DeadlineExceeded {
809+
t.Fatalf("Expected DeadlineExceeded, not %v", ctx.Err())
810+
}
811+
}
812+
}
813+
641814
func testAdminAPIsUserScramCredentials(what string, a *AdminClient, expDuration time.Duration, t *testing.T) {
642815
var users []string
643816

@@ -958,6 +1131,7 @@ func testAdminAPIs(what string, a *AdminClient, t *testing.T) {
9581131

9591132
testAdminAPIsListConsumerGroupOffsets(what, a, expDuration, t)
9601133
testAdminAPIsAlterConsumerGroupOffsets(what, a, expDuration, t)
1134+
testAdminAPIsListOffsets(what, a, expDuration, t)
9611135

9621136
testAdminAPIsUserScramCredentials(what, a, expDuration, t)
9631137
}

kafka/adminoptions.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -137,6 +137,8 @@ func (ao AdminOptionRequestTimeout) supportsListConsumerGroupOffsets() {
137137
}
138138
func (ao AdminOptionRequestTimeout) supportsAlterConsumerGroupOffsets() {
139139
}
140+
func (ao AdminOptionRequestTimeout) supportsListOffsets() {
141+
}
140142
func (ao AdminOptionRequestTimeout) supportsDescribeUserScramCredentials() {
141143
}
142144
func (ao AdminOptionRequestTimeout) supportsAlterUserScramCredentials() {

kafka/integration_test.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3206,7 +3206,7 @@ func (its *IntegrationTestSuite) TestAdminClient_ListOffsets() {
32063206
results, err = a.ListOffsets(ctx, topicPartitionOffsets, SetAdminIsolationLevel(IsolationLevelReadCommitted))
32073207
assert.Nil(err, "ListOffsets should not fail.")
32083208

3209-
for _, info := range results.ResultsInfos {
3209+
for _, info := range results.ResultInfos {
32103210
assert.Equal(info.Error.Code(), ErrNoError, "Error code should be ErrNoError.")
32113211
assert.Equal(info.Offset, int64(0), "Offset should be ErrNoError.")
32123212
}
@@ -3215,7 +3215,7 @@ func (its *IntegrationTestSuite) TestAdminClient_ListOffsets() {
32153215
results, err = a.ListOffsets(ctx, topicPartitionOffsets, SetAdminIsolationLevel(IsolationLevelReadCommitted))
32163216
assert.Nil(err, "ListOffsets should not fail.")
32173217

3218-
for _, info := range results.ResultsInfos {
3218+
for _, info := range results.ResultInfos {
32193219
assert.Equal(info.Error.Code(), ErrNoError, "Error code should be ErrNoError.")
32203220
assert.Equal(info.Offset, int64(3), "Offset should be 3.")
32213221
}
@@ -3224,7 +3224,7 @@ func (its *IntegrationTestSuite) TestAdminClient_ListOffsets() {
32243224
results, err = a.ListOffsets(ctx, topicPartitionOffsets, SetAdminIsolationLevel(IsolationLevelReadCommitted))
32253225
assert.Nil(err, "ListOffsets should not fail.")
32263226

3227-
for _, info := range results.ResultsInfos {
3227+
for _, info := range results.ResultInfos {
32283228
assert.Equal(info.Error.Code(), ErrNoError, "Error code should be ErrNoError.")
32293229
assert.Equal(info.Offset, int64(1), "Offset should be 1.")
32303230
}

0 commit comments

Comments
 (0)