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

Refactor the producer, part 2 #550

Merged
merged 1 commit into from
Oct 15, 2015
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
146 changes: 78 additions & 68 deletions async_producer.go
Original file line number Diff line number Diff line change
Expand Up @@ -508,13 +508,14 @@ func (pp *partitionProducer) updateLeader() error {
// one per broker, constructs both an aggregator and a flusher
func (p *asyncProducer) newBrokerProducer(broker *Broker) chan<- *ProducerMessage {
input := make(chan *ProducerMessage)
bridge := make(chan []*ProducerMessage)
bridge := make(chan *produceSet)

a := &aggregator{
parent: p,
broker: broker,
input: input,
output: bridge,
buffer: newProduceSet(p),
}
go withRecover(a.run)

Expand All @@ -535,15 +536,14 @@ type aggregator struct {
parent *asyncProducer
broker *Broker
input <-chan *ProducerMessage
output chan<- []*ProducerMessage
output chan<- *produceSet

buffer []*ProducerMessage
bufferBytes int
timer <-chan time.Time
buffer *produceSet
timer <-chan time.Time
}

func (a *aggregator) run() {
var output chan<- []*ProducerMessage
var output chan<- *produceSet

for {
select {
Expand All @@ -552,17 +552,19 @@ func (a *aggregator) run() {
goto shutdown
}

if a.wouldOverflow(msg) {
if a.buffer.wouldOverflow(msg) {
Logger.Printf("producer/aggregator/%d maximum request accumulated, forcing blocking flush\n", a.broker.ID())
a.output <- a.buffer
a.reset()
output = nil
}

a.buffer = append(a.buffer, msg)
a.bufferBytes += msg.byteSize()
if err := a.buffer.add(msg); err != nil {
a.parent.returnError(msg, err)
continue
}

if a.readyToFlush(msg) {
if a.buffer.readyToFlush(msg) {
output = a.output
} else if a.parent.conf.Producer.Flush.Frequency > 0 && a.timer == nil {
a.timer = time.After(a.parent.conf.Producer.Flush.Frequency)
Expand All @@ -576,58 +578,22 @@ func (a *aggregator) run() {
}

shutdown:
if len(a.buffer) > 0 {
if !a.buffer.empty() {
a.output <- a.buffer
}
close(a.output)
}

func (a *aggregator) wouldOverflow(msg *ProducerMessage) bool {
switch {
// Would we overflow our maximum possible size-on-the-wire? 10KiB is arbitrary overhead for safety.
case a.bufferBytes+msg.byteSize() >= int(MaxRequestSize-(10*1024)):
return true
// Would we overflow the size-limit of a compressed message-batch?
case a.parent.conf.Producer.Compression != CompressionNone && a.bufferBytes+msg.byteSize() >= a.parent.conf.Producer.MaxMessageBytes:
return true
// Would we overflow simply in number of messages?
case a.parent.conf.Producer.Flush.MaxMessages > 0 && len(a.buffer) >= a.parent.conf.Producer.Flush.MaxMessages:
return true
default:
return false
}
}

func (a *aggregator) readyToFlush(msg *ProducerMessage) bool {
switch {
// If all three config values are 0, we always flush as-fast-as-possible
case a.parent.conf.Producer.Flush.Frequency == 0 && a.parent.conf.Producer.Flush.Bytes == 0 && a.parent.conf.Producer.Flush.Messages == 0:
return true
// If the messages is a chaser we must flush to maintain the state-machine
case msg.flags&chaser == chaser:
return true
// If we've passed the message trigger-point
case a.parent.conf.Producer.Flush.Messages > 0 && len(a.buffer) >= a.parent.conf.Producer.Flush.Messages:
return true
// If we've passed the byte trigger-point
case a.parent.conf.Producer.Flush.Bytes > 0 && a.bufferBytes >= a.parent.conf.Producer.Flush.Bytes:
return true
default:
return false
}
}

func (a *aggregator) reset() {
a.timer = nil
a.buffer = nil
a.bufferBytes = 0
a.buffer = newProduceSet(a.parent)
}

// takes a batch at a time from the aggregator and sends to the broker
type flusher struct {
parent *asyncProducer
broker *Broker
input <-chan []*ProducerMessage
input <-chan *produceSet

currentRetries map[string]map[int32]error
}
Expand All @@ -639,11 +605,13 @@ func (f *flusher) run() {

for batch := range f.input {
if closing != nil {
f.parent.retryMessages(batch, closing)
batch.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) {
f.parent.retryMessages(msgs, closing)
})
continue
}

set := f.groupAndFilter(batch)
set := f.filter(batch)
if set.empty() {
continue
}
Expand Down Expand Up @@ -683,30 +651,32 @@ func (f *flusher) run() {
Logger.Printf("producer/flusher/%d shut down\n", f.broker.ID())
}

func (f *flusher) groupAndFilter(batch []*ProducerMessage) *produceSet {
func (f *flusher) filter(batch *produceSet) *produceSet {
set := newProduceSet(f.parent)

for _, msg := range batch {
batch.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) {
for _, msg := range msgs {

if f.currentRetries[msg.Topic] != nil && f.currentRetries[msg.Topic][msg.Partition] != nil {
// we're currently retrying this partition so we need to filter out this message
f.parent.retryMessages([]*ProducerMessage{msg}, f.currentRetries[msg.Topic][msg.Partition])
if f.currentRetries[msg.Topic] != nil && f.currentRetries[msg.Topic][msg.Partition] != nil {
// we're currently retrying this partition so we need to filter out this message
f.parent.retryMessages([]*ProducerMessage{msg}, f.currentRetries[msg.Topic][msg.Partition])

if msg.flags&chaser == chaser {
// ...but now we can start processing future messages again
Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
f.broker.ID(), msg.Topic, msg.Partition)
delete(f.currentRetries[msg.Topic], msg.Partition)
}
if msg.flags&chaser == chaser {
// ...but now we can start processing future messages again
Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
f.broker.ID(), msg.Topic, msg.Partition)
delete(f.currentRetries[msg.Topic], msg.Partition)
}

continue
}
continue
}

if err := set.add(msg); err != nil {
f.parent.returnError(msg, err)
continue
if err := set.add(msg); err != nil {
f.parent.returnError(msg, err)
continue
}
}
}
})

return set
}
Expand Down Expand Up @@ -874,6 +844,46 @@ func (ps *produceSet) eachPartition(cb func(topic string, partition int32, msgs
}
}

func (ps *produceSet) wouldOverflow(msg *ProducerMessage) bool {
switch {
// Would we overflow our maximum possible size-on-the-wire? 10KiB is arbitrary overhead for safety.
case ps.bufferBytes+msg.byteSize() >= int(MaxRequestSize-(10*1024)):
return true
// Would we overflow the size-limit of a compressed message-batch for this partition?
case ps.parent.conf.Producer.Compression != CompressionNone &&
ps.msgs[msg.Topic] != nil && ps.msgs[msg.Topic][msg.Partition] != nil &&
ps.msgs[msg.Topic][msg.Partition].bufferBytes+msg.byteSize() >= ps.parent.conf.Producer.MaxMessageBytes:
return true
// Would we overflow simply in number of messages?
case ps.parent.conf.Producer.Flush.MaxMessages > 0 && ps.bufferCount >= ps.parent.conf.Producer.Flush.MaxMessages:
return true
default:
return false
}
}

func (ps *produceSet) readyToFlush(msg *ProducerMessage) bool {
switch {
// If we don't have any messages, nothing else matters
case ps.empty():
return false
// If all three config values are 0, we always flush as-fast-as-possible
case ps.parent.conf.Producer.Flush.Frequency == 0 && ps.parent.conf.Producer.Flush.Bytes == 0 && ps.parent.conf.Producer.Flush.Messages == 0:
return true
// If the messages is ps chaser we must flush to maintain the state-machine
case msg.flags&chaser == chaser:
return true
// If we've passed the message trigger-point
case ps.parent.conf.Producer.Flush.Messages > 0 && ps.bufferCount >= ps.parent.conf.Producer.Flush.Messages:
return true
// If we've passed the byte trigger-point
case ps.parent.conf.Producer.Flush.Bytes > 0 && ps.bufferBytes >= ps.parent.conf.Producer.Flush.Bytes:
return true
default:
return false
}
}

func (ps *produceSet) empty() bool {
return ps.bufferCount == 0
}
Expand Down
4 changes: 2 additions & 2 deletions async_producer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,7 +320,7 @@ func TestAsyncProducerEncoderFailures(t *testing.T) {
leader.Returns(prodSuccess)

config := NewConfig()
config.Producer.Flush.Messages = 3
config.Producer.Flush.Messages = 1
Copy link
Contributor

Choose a reason for hiding this comment

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

This was needed because the 2 failed to encode messages are not counted towards the batch anymore I assume?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

bingo

config.Producer.Return.Successes = true
config.Producer.Partitioner = NewManualPartitioner
producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
Expand All @@ -330,8 +330,8 @@ func TestAsyncProducerEncoderFailures(t *testing.T) {

for flush := 0; flush < 3; flush++ {
producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(true), Value: flakyEncoder(false)}
producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(true), Value: flakyEncoder(true)}
producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(false), Value: flakyEncoder(true)}
producer.Input() <- &ProducerMessage{Topic: "my_topic", Key: flakyEncoder(true), Value: flakyEncoder(true)}
expectResults(t, producer, 1, 2)
}

Expand Down