Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CreateTopics: only suppress topic already exists errors #1136

Merged
merged 3 commits into from
May 26, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 4 additions & 12 deletions createtopics.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@ package kafka
import (
"bufio"
"context"
"errors"
"fmt"
"net"
"time"
Expand Down Expand Up @@ -65,7 +64,6 @@ func (c *Client) CreateTopics(ctx context.Context, req *CreateTopicsRequest) (*C
TimeoutMs: c.timeoutMs(ctx, defaultCreateTopicsTimeout),
ValidateOnly: req.ValidateOnly,
})

if err != nil {
return nil, fmt.Errorf("kafka.(*Client).CreateTopics: %w", err)
}
Expand Down Expand Up @@ -363,6 +361,9 @@ func (c *Conn) createTopics(request createTopicsRequestV0) (createTopicsResponse
return response, err
}
for _, tr := range response.TopicErrors {
if tr.ErrorCode == int16(TopicAlreadyExists) {
continue
}
if tr.ErrorCode != 0 {
return response, Error(tr.ErrorCode)
}
Expand All @@ -385,14 +386,5 @@ func (c *Conn) CreateTopics(topics ...TopicConfig) error {
_, err := c.createTopics(createTopicsRequestV0{
Topics: requestV0Topics,
})
if err != nil {
if errors.Is(err, TopicAlreadyExists) {
// ok
return nil
}

return err
}

return nil
return err
}
77 changes: 76 additions & 1 deletion createtopics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,10 +4,86 @@ import (
"bufio"
"bytes"
"context"
"errors"
"net"
"reflect"
"strconv"
"testing"
)

func TestConnCreateTopics(t *testing.T) {
topic1 := makeTopic()
topic2 := makeTopic()

conn, err := DialContext(context.Background(), "tcp", "localhost:9092")
if err != nil {
t.Fatal(err)
}

defer func() {
err := conn.Close()
if err != nil {
t.Fatalf("failed to close connection: %v", err)
}
}()

controller, _ := conn.Controller()

controllerConn, err := Dial("tcp", net.JoinHostPort(controller.Host, strconv.Itoa(controller.Port)))
if err != nil {
t.Fatal(err)
}
defer controllerConn.Close()

err = controllerConn.CreateTopics(TopicConfig{
Topic: topic1,
NumPartitions: 1,
ReplicationFactor: 1,
})
if err != nil {
t.Fatalf("unexpected error creating topic: %s", err.Error())
}

err = controllerConn.CreateTopics(TopicConfig{
Topic: topic1,
NumPartitions: 1,
ReplicationFactor: 1,
})

// Duplicate topic should not return an error
if err != nil {
t.Fatalf("unexpected error creating duplicate topic topic: %v", err)
}

err = controllerConn.CreateTopics(
TopicConfig{
Topic: topic1,
NumPartitions: 1,
ReplicationFactor: 1,
},
TopicConfig{
Topic: topic2,
NumPartitions: 1,
ReplicationFactor: 1,
},
TopicConfig{
Topic: topic2,
NumPartitions: 1,
ReplicationFactor: 1,
},
)

if err == nil {
t.Fatal("CreateTopics should have returned an error for invalid requests")
}

if !errors.Is(err, InvalidRequest) {
t.Fatalf("expected invalid request: %v", err)
}

deleteTopic(t, topic1)
}

func TestClientCreateTopics(t *testing.T) {
const (
topic1 = "client-topic-1"
Expand Down Expand Up @@ -59,7 +135,6 @@ func TestClientCreateTopics(t *testing.T) {
},
},
})

if err != nil {
t.Fatal(err)
}
Expand Down