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

Add the field to avoid creating missing topic. #873

Merged
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 5 additions & 2 deletions writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,9 @@ type Writer struct {
// If nil, DefaultTransport is used.
Transport RoundTripper

// AllowAutoTopicCreation notifies writer to create topic is missing.
Copy link
Contributor

Choose a reason for hiding this comment

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

While it's not a breaking change per-say since the code will continue to compile, the new behavior we introduce is the inverse of what the program used to do.

Do you think we could add a bit more documentation to reflect that fact?

Copy link
Contributor Author

@kislerdm kislerdm Mar 27, 2022

Choose a reason for hiding this comment

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

@achille-roussel hey! indeed, you are right wrt docu update. I update the test case as well.

AllowAutoTopicCreation bool
Copy link

Choose a reason for hiding this comment

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

Any reason to not introduce this in the writer config too?
#1278

If I am using WriterConfig to create the writer, is it safe to just set this config after calling NewWriter?

Copy link

Choose a reason for hiding this comment

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

@erikdw @achille-roussel Here's a PR in case 1278 is a desirable feature request: #1279 if


// Manages the current set of partition-topic writers.
group sync.WaitGroup
mutex sync.Mutex
Expand Down Expand Up @@ -733,7 +736,7 @@ func (w *Writer) partitions(ctx context.Context, topic string) (int, error) {
// caching recent results (the kafka.Transport types does).
r, err := client.transport().RoundTrip(ctx, client.Addr, &metadataAPI.Request{
TopicNames: []string{topic},
AllowAutoTopicCreation: true,
AllowAutoTopicCreation: w.AllowAutoTopicCreation,
})
if err != nil {
return 0, err
Expand Down Expand Up @@ -941,7 +944,7 @@ func newBatchQueue(initialSize int) batchQueue {
bq := batchQueue{
queue: make([]*writeBatch, 0, initialSize),
mutex: &sync.Mutex{},
cond: &sync.Cond{},
cond: &sync.Cond{},
}

bq.cond.L = bq.mutex
Expand Down
28 changes: 28 additions & 0 deletions writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,10 @@ func TestWriter(t *testing.T) {
scenario: "writing a message to a non-existant topic creates the topic",
function: testWriterAutoCreateTopic,
},
{
scenario: "terminates on an attempts to write a message to a non-existant topic",
kislerdm marked this conversation as resolved.
Show resolved Hide resolved
function: testWriterTerminateMissingTopic,
},
}

for _, test := range tests {
Expand Down Expand Up @@ -737,6 +741,30 @@ func testWriterAutoCreateTopic(t *testing.T) {
}
}

func testWriterTerminateMissingTopic(t *testing.T) {
topic := makeTopic()

transport := &Transport{}
defer transport.CloseIdleConnections()

writer := &Writer{
Addr: TCP("localhost:9092"),
Topic: topic,
Balancer: &RoundRobin{},
RequiredAcks: RequireNone,
AllowAutoTopicCreation: false,
Transport: transport,
}
defer writer.Close()

msg := Message{Value: []byte("FooBar")}

if err := writer.WriteMessages(context.Background(), msg); err == nil {
t.Fatal("Kafka error [3] UNKNOWN_TOPIC_OR_PARTITION is expected")
return
}
}

type staticBalancer struct {
partition int
}
Expand Down