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

Added Consumer.OffsetsStore() #72

Merged
merged 4 commits into from
Aug 8, 2017
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
23 changes: 23 additions & 0 deletions kafka/consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,29 @@ func (c *Consumer) CommitOffsets(offsets []TopicPartition) ([]TopicPartition, er
return c.commit(offsets)
}

// OffsetsStore stores the provided list of offsets that will be committed
// to the offset store according to `auto.commit.interval.ms` or manual
// offset-less Commit().
//
// Returns the stored offsets on success. If at least one offset couldn't be stored,
// an error and a list of offsets is returned. Each offset can be checked for
// specific errors via its `.Error` member.
func (c *Consumer) OffsetsStore(offsets []TopicPartition) (storedOffsets []TopicPartition, err error) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Should probably be called StoreOffsets to be consistent with CommitOffsets (even though it makes it inconsistent with librdkafka, but that's a hidden gem).

coffsets := newCPartsFromTopicPartitions(offsets)
defer C.rd_kafka_topic_partition_list_destroy(coffsets)

cErr := C.rd_kafka_offsets_store(c.handle.rk, coffsets)

// coffsets might be annotated with an error
storedOffsets = newTopicPartitionsFromCparts(coffsets)

if cErr != C.RD_KAFKA_RESP_ERR_NO_ERROR {
return storedOffsets, newError(cErr)
}

return storedOffsets, nil
}

// Seek seeks the given topic partitions using the offset from the TopicPartition.
//
// If timeoutMs is not 0 the call will wait this long for the
Expand Down
15 changes: 14 additions & 1 deletion kafka/consumer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,6 @@ func TestConsumerAPIs(t *testing.T) {
if err != nil {
t.Errorf("SubscribeTopics failed: %s", err)
}

Copy link
Contributor

Choose a reason for hiding this comment

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

Remove whitespace diff

_, err = c.Commit()
if err != nil && err.(Error).Code() != ErrNoOffset {
t.Errorf("Commit() failed: %s", err)
Expand All @@ -65,6 +64,20 @@ func TestConsumerAPIs(t *testing.T) {
t.Errorf("Unsubscribe failed: %s", err)
}

topic := "gotest"
stored, err := c.OffsetsStore([]TopicPartition{{Topic: &topic, Partition: 0, Offset: 1}})
if err != nil && err.(Error).Code() != ErrUnknownPartition {
t.Errorf("OffsetsStore() failed: %s", err)
toppar := stored[0]
if toppar.Error.(Error).Code() == ErrUnknownPartition {
t.Errorf("OffsetsStore() TopicPartition error: %s", toppar.Error)
}
}
stored, err = c.OffsetsStore(nil)
Copy link
Contributor

Choose a reason for hiding this comment

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

Is there a use for accepting a nil partition set?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

nil is a valid value for a slice, in fact it is the default value. I wanted to make sure it is explicitly handled. In the fixup I changed that to an uninitialized (default) slice.

Copy link
Contributor

Choose a reason for hiding this comment

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

Right, the data type allows it, but is there a point for the StoreOffsets API to allow it? Passing a nil slice should be considered a usage error and not something we need to explicitly handle (I do agree that error handling is necessary, but parameter checking in APIs is not as fruitful as it seems IMHO unless there is a security concern to it).

Copy link
Contributor Author

@ctrochalakis ctrochalakis Aug 8, 2017

Choose a reason for hiding this comment

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

Let's call it an empty slice. I think that the API should allow passing an empty slice, whatever that means in Go. If someone chooses to pass an empty slice no error is returned. We don't explicitly handle it (as in consumer.go code) but we explicitly test that an empty []TopicPartition is properly treated.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

note that this part of the diff is now changed to:

        var empty []TopicPartition
        stored, err = c.StoreOffsets(empty)

which seems like a valid usage scenario.

if err != nil {
t.Errorf("OffsetsStore(nil) failed: %s", err)
}

topic1 := "gotest1"
topic2 := "gotest2"
err = c.Assign([]TopicPartition{{Topic: &topic1, Partition: 2},
Expand Down