Store offsets manually for each message (#6872)

(cherry picked from commit 6c32fc3fc1)
This commit is contained in:
Ivan 2019-09-09 19:59:51 +03:00 committed by Ivan Lezhankin
parent 49d683c34c
commit 9ed197c24f
2 changed files with 8 additions and 6 deletions

View File

@ -72,9 +72,7 @@ void ReadBufferFromKafkaConsumer::commit()
PrintOffsets("Polled offset", consumer->get_offsets_position(consumer->get_assignment()));
/// Since we can poll more messages than we already processed - commit only processed messages.
if (!messages.empty())
consumer->async_commit(*std::prev(current));
consumer->async_commit();
PrintOffsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment()));
@ -186,6 +184,9 @@ bool ReadBufferFromKafkaConsumer::nextImpl()
auto new_position = reinterpret_cast<char *>(const_cast<unsigned char *>(current->get_payload().get_data()));
BufferBase::set(new_position, current->get_payload().get_size(), 0);
/// Since we can poll more messages than we already processed - commit only processed messages.
consumer->store_offset(*current);
++current;
return true;

View File

@ -261,9 +261,10 @@ ConsumerBufferPtr StorageKafka::createReadBuffer()
conf.set("metadata.broker.list", brokers);
conf.set("group.id", group);
conf.set("client.id", VERSION_FULL);
conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start
conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished
conf.set("enable.partition.eof", "false"); // Ignore EOF messages
conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start
conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished
conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once.
conf.set("enable.partition.eof", "false"); // Ignore EOF messages
updateConfiguration(conf);
// Create a consumer and subscribe to topics