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

Support request pipelining in AsyncProducer #2094

Merged
merged 6 commits into from
Jan 18, 2022
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
70 changes: 22 additions & 48 deletions async_producer.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,20 +670,12 @@ func (pp *partitionProducer) updateLeader() error {
})
}

type pendingResponse struct {
set *produceSet
version int16
promise *responsePromise
response *ProduceResponse
}

// one per broker; also constructs an associated flusher
func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {
var (
input = make(chan *ProducerMessage)
bridge = make(chan *produceSet)
responses = make(chan *brokerProducerResponse)
pendings = make(chan *pendingResponse, p.conf.Net.MaxOpenRequests-1)
)

bp := &brokerProducer{
Expand All @@ -700,54 +692,36 @@ func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {

// minimal bridge to make the network response `select`able
go withRecover(func() {
defer close(pendings)
for set := range bridge {
set := set
var err error
var response *ProduceResponse
var promise *responsePromise

request := set.buildRequest()
if request.RequiredAcks != NoResponse {
response = new(ProduceResponse)
}

responseHeaderVersion := int16(-1)
if response != nil {
responseHeaderVersion = response.headerVersion()
}

promise, err = broker.send(request, response != nil, responseHeaderVersion)

// return quickly if failed or ackMode: NoResponse
if err != nil || promise == nil {
responses <- &brokerProducerResponse{
set: set,
err: err,
res: response,
// Capture the current set to forward in the callback
sendResponse := func(set *produceSet) ProduceCallback {
return func(response *ProduceResponse, err error) {
responses <- &brokerProducerResponse{
set: set,
err: err,
res: response,
}
}
continue
}
pending := &pendingResponse{set: set, version: request.version(), response: response, promise: promise}
pendings <- pending
}
})
}(set)

go withRecover(func() {
defer close(responses)
for pending := range pendings {
var err error
select {
case buf := <-pending.promise.packets:
err = versionedDecode(buf, pending.response, pending.version)
case err = <-pending.promise.errors:
// Use AsyncProduce vs Produce to not block waiting for the response
// so that we can pipeline multiple produce requests and achieve higher throughput, see:
// https://kafka.apache.org/protocol#protocol_network
err := broker.AsyncProduce(request, sendResponse)
if err != nil {
// Request failed to be sent
sendResponse(nil, err)
continue
}
responses <- &brokerProducerResponse{
set: pending.set,
err: err,
res: pending.response,
// Callback is not called when using NoResponse
if p.conf.Producer.RequiredAcks == NoResponse {
// Provide the expected nil response
sendResponse(nil, nil)
}
}
close(responses)
})

if p.conf.Producer.Retry.Max <= 0 {
Expand Down
141 changes: 114 additions & 27 deletions broker.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ type Broker struct {
connErr error
lock sync.Mutex
opened int32
responses chan responsePromise
responses chan *responsePromise
done chan bool

registeredMetrics []string
Expand Down Expand Up @@ -121,10 +121,25 @@ type responsePromise struct {
requestTime time.Time
correlationID int32
headerVersion int16
handler func([]byte, error)
packets chan []byte
errors chan error
}

func (p *responsePromise) handle(packets []byte, err error) {
// Use callback when provided
if p.handler != nil {
p.handler(packets, err)
return
}
// Otherwise fallback to using channels
if err != nil {
p.errors <- err
return
}
p.packets <- packets
}

// NewBroker creates and returns a Broker targeting the given host:port address.
// This does not attempt to actually connect, you have to call Open() for that.
func NewBroker(addr string) *Broker {
Expand Down Expand Up @@ -219,7 +234,7 @@ func (b *Broker) Open(conf *Config) error {
}

b.done = make(chan bool)
b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1)
b.responses = make(chan *responsePromise, b.conf.Net.MaxOpenRequests-1)

if b.id >= 0 {
DebugLogger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id)
Expand Down Expand Up @@ -342,7 +357,55 @@ func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, e
return response, nil
}

// Produce returns a produce response or error
// ProduceCallback function is called once the produce response has been parsed
// or could not be read.
type ProduceCallback func(*ProduceResponse, error)

// AsyncProduce sends a produce request and eventually call the provided callback
// with a produce response or an error.
//
// Waiting for the response is generally not blocking on the contrary to using Produce.
// If the maximum number of in flight request configured is reached then
// the request will be blocked till a previous response is received.
//
// When configured with RequiredAcks == NoResponse, the callback will not be invoked.
// If an error is returned because the request could not be sent then the callback
// will not be invoked either.
func (b *Broker) AsyncProduce(request *ProduceRequest, cb ProduceCallback) error {
needAcks := request.RequiredAcks != NoResponse
// Use a nil promise when no acks is required
var promise *responsePromise

if needAcks {
// Create ProduceResponse early to provide the header version
res := new(ProduceResponse)
promise = &responsePromise{
headerVersion: res.headerVersion(),
// Packets will be converted to a ProduceResponse in the responseReceiver goroutine
handler: func(packets []byte, err error) {
if err != nil {
// Failed request
cb(nil, err)
return
}

if err := versionedDecode(packets, res, request.version()); err != nil {
// Malformed response
cb(nil, err)
return
}

// Wellformed response
b.updateThrottleMetric(res.ThrottleTime)
cb(res, nil)
},
}
}

return b.sendWithPromise(request, promise)
}

//Produce returns a produce response or error
func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
var (
response *ProduceResponse
Expand All @@ -354,15 +417,7 @@ func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
} else {
response = new(ProduceResponse)
err = b.sendAndReceive(request, response)
if response.ThrottleTime != time.Duration(0) {
DebugLogger.Printf(
"producer/broker/%d ProduceResponse throttled %v\n",
b.ID(), response.ThrottleTime)
if b.brokerThrottleTime != nil {
throttleTimeInMs := int64(response.ThrottleTime / time.Millisecond)
b.brokerThrottleTime.Update(throttleTimeInMs)
}
}
b.updateThrottleMetric(response.ThrottleTime)
}

if err != nil {
Expand Down Expand Up @@ -807,24 +862,43 @@ func (b *Broker) write(buf []byte) (n int, err error) {
}

func (b *Broker) send(rb protocolBody, promiseResponse bool, responseHeaderVersion int16) (*responsePromise, error) {
var promise *responsePromise
if promiseResponse {
// Packets or error will be sent to the following channels
// once the response is received
promise = &responsePromise{
headerVersion: responseHeaderVersion,
packets: make(chan []byte),
errors: make(chan error),
}
}

if err := b.sendWithPromise(rb, promise); err != nil {
return nil, err
}

return promise, nil
}

func (b *Broker) sendWithPromise(rb protocolBody, promise *responsePromise) error {
b.lock.Lock()
defer b.lock.Unlock()

if b.conn == nil {
if b.connErr != nil {
return nil, b.connErr
return b.connErr
}
return nil, ErrNotConnected
return ErrNotConnected
}

if !b.conf.Version.IsAtLeast(rb.requiredVersion()) {
return nil, ErrUnsupportedVersion
return ErrUnsupportedVersion
}

req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
buf, err := encode(req, b.conf.MetricRegistry)
if err != nil {
return nil, err
return err
}

requestTime := time.Now()
Expand All @@ -834,20 +908,21 @@ func (b *Broker) send(rb protocolBody, promiseResponse bool, responseHeaderVersi
b.updateOutgoingCommunicationMetrics(bytes)
if err != nil {
b.addRequestInFlightMetrics(-1)
return nil, err
return err
}
b.correlationID++

if !promiseResponse {
if promise == nil {
// Record request latency without the response
b.updateRequestLatencyAndInFlightMetrics(time.Since(requestTime))
return nil, nil
return nil
}

promise := responsePromise{requestTime, req.correlationID, responseHeaderVersion, make(chan []byte), make(chan error)}
promise.requestTime = requestTime
promise.correlationID = req.correlationID
b.responses <- promise

return &promise, nil
return nil
}

func (b *Broker) sendAndReceive(req protocolBody, res protocolBody) error {
Expand Down Expand Up @@ -942,7 +1017,7 @@ func (b *Broker) responseReceiver() {
// This was previously incremented in send() and
// we are not calling updateIncomingCommunicationMetrics()
b.addRequestInFlightMetrics(-1)
response.errors <- dead
response.handle(nil, dead)
continue
}

Expand All @@ -954,7 +1029,7 @@ func (b *Broker) responseReceiver() {
if err != nil {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
dead = err
response.errors <- err
response.handle(nil, err)
continue
}

Expand All @@ -963,15 +1038,15 @@ func (b *Broker) responseReceiver() {
if err != nil {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
dead = err
response.errors <- err
response.handle(nil, err)
continue
}
if decodedHeader.correlationID != response.correlationID {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
// TODO if decoded ID < cur ID, discard until we catch up
// TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response
dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)}
response.errors <- dead
response.handle(nil, dead)
continue
}

Expand All @@ -980,11 +1055,11 @@ func (b *Broker) responseReceiver() {
b.updateIncomingCommunicationMetrics(bytesReadHeader+bytesReadBody, requestLatency)
if err != nil {
dead = err
response.errors <- err
response.handle(nil, err)
continue
}

response.packets <- buf
response.handle(buf, nil)
}
close(b.done)
}
Expand Down Expand Up @@ -1545,6 +1620,18 @@ func (b *Broker) updateOutgoingCommunicationMetrics(bytes int) {
}
}

func (b *Broker) updateThrottleMetric(throttleTime time.Duration) {
if throttleTime != time.Duration(0) {
DebugLogger.Printf(
"producer/broker/%d ProduceResponse throttled %v\n",
b.ID(), throttleTime)
if b.brokerThrottleTime != nil {
throttleTimeInMs := int64(throttleTime / time.Millisecond)
b.brokerThrottleTime.Update(throttleTimeInMs)
}
}
}

func (b *Broker) registerMetrics() {
b.brokerIncomingByteRate = b.registerMeter("incoming-byte-rate")
b.brokerRequestRate = b.registerMeter("request-rate")
Expand Down
Loading