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

Fix deadlock when closing Broker in brokerProducer #2133

Merged
merged 5 commits into from
Feb 13, 2022

Conversation

slaunay
Copy link
Contributor

@slaunay slaunay commented Feb 8, 2022

Bugfix

A regression was introduced in #2094 (Sarama 1.31.0) and can result in a deadlock inside the AsyncProducer (and indirectly the SyncProducer) when handling Produce response error (typically network errors, see #2129).
The root cause is an AsyncProduce callback used to pipeline Produce requests must not close the Broker directly.
Indeed, closing the Broker blocks till the responseReceiver goroutine is done yet the responseReceiver goroutine might be blocked invoking a callback.
In this specific case, we are not directly closing the Broker in the callback but from:

  • the brokerProducer run goroutine when calling handleError after receiving an error producing
  • while the brokerProducer bridge goroutine can be blocked if trying to send another response to the run goroutine
  • and the broker responseReceiver goroutine is blocked invoking the callback defined in the bridge goroutine

I believe this happens when more than one Produce request is in flight to a given broker and we are getting a missing or malformed response which is why it was not detected by the current test suite.
We have been using Broker.AsyncProduce and callbacks without issues in production ourselves but this is because we do not rely on the AsyncProducer and also made sure not to close the Broker in a blocking path.

A simple fix would be to close the broker asynchronously when we receive a response error:

--- a/async_producer.go
+++ b/async_producer.go
@@ -1028,7 +1028,7 @@ func (bp *brokerProducer) handleError(sent *produceSet, err error) {
        default:
                Logger.Printf("producer/broker/%d state change to [closing] because %s\n", bp.broker.ID(), err)
                bp.parent.abandonBrokerConnection(bp.broker)
-               _ = bp.broker.Close()
+               safeAsyncClose(bp.broker)
                bp.closing = err
                sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
                        bp.parent.retryMessages(pSet.msgs, err)

But that approach might leak such a goroutine once the brokerProducer is abandonned.
So the proposed fix consists of:

  • adding a new goroutine to the brokerProducer to receive a request to close the Broker
  • signal such goroutine once when we want to close the Broker (e.g. network error)
  • wait for it to be done (whether a closing request was sent or not) when we shutdown the brokerProducer

Changes

  • add unit test to reproduce the deadlock by simulating a network error
  • document possible deadlock when closing the Broker from an AsyncProduce callback when handling a response error
  • add closeBroker goroutine and channel to asynchronously close a Broker and only once
  • reuse the stopchan channel to signal that the closeBroker goroutine is done
  • update TestBrokerProducerShutdown to check goroutine leak by closing the input vs the stopchan channel

Testing done

I added a new unit test to check that the proposed fix works and possibly detect a similar regression in the future.
Here is the run using 1.31.1 (result in a deadlock):

=== RUN   TestAsyncProducerMultipleRetriesWithConcurrentRequests
[sarama] 2022/02/08 11:21:38 *** mockbroker/1 listening on 127.0.0.1:61958
[sarama] 2022/02/08 11:21:38 *** mockbroker/2 listening on 127.0.0.1:61959
[sarama] 2022/02/08 11:21:38 Initializing new client
[sarama] 2022/02/08 11:21:38 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:21:38 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:21:38 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:21:38 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:21:38 client/metadata fetching metadata for all topics from broker 127.0.0.1:61958
[sarama] 2022/02/08 11:21:38 *** mockbroker/1/0: connection opened
[sarama] 2022/02/08 11:21:38 Connected to broker at 127.0.0.1:61958 (unregistered)
[sarama] 2022/02/08 11:21:38 *** mockbroker/1/0: replied to *sarama.MetadataRequest with *sarama.MetadataResponse
-> (*sarama.MetadataRequest){Version:(int16)0 Topics:([]string)<nil> AllowAutoTopicCreation:(bool)false}
-> (*sarama.MetadataResponse){Version:(int16)0 ThrottleTimeMs:(int32)0 Brokers:([]*sarama.Broker)[<max>] ClusterID:(*string)<nil> ControllerID:(int32)0 Topics:([]*sarama.TopicMetadata)[<max>]}
[sarama] 2022/02/08 11:21:38 client/brokers registered new broker #2 at 127.0.0.1:61959
[sarama] 2022/02/08 11:21:38 Successfully initialized new client
[sarama] 2022/02/08 11:21:38 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:21:38 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:21:38 producer/broker/2 starting up
[sarama] 2022/02/08 11:21:38 producer/broker/2 state change to [open] on my_topic/0
[sarama] 2022/02/08 11:21:38 *** mockbroker/2/0: connection opened
[sarama] 2022/02/08 11:21:38 Connected to broker at 127.0.0.1:61959 (registered as #2)
[sarama] 2022/02/08 11:21:38 producer/broker/2 state change to [closing] because read tcp 127.0.0.1:61961->127.0.0.1:61959: i/o timeout
[sarama] 2022/02/08 11:21:38 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/08 11:21:38 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
coverage: 11.9% of statements
panic: test timed out after 5s

goroutine 40 [running]:
testing.(*M).startAlarm.func1()
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1788 +0xbb
created by time.goFunc
        /usr/local/Cellar/go/1.17.5/libexec/src/time/sleep.go:180 +0x4a

goroutine 1 [chan receive]:
testing.(*T).Run(0xc0001fe1a0, {0x1713d1f, 0x36}, 0x171f9c0)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1307 +0x752
testing.runTests.func1(0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1598 +0x9a
testing.tRunner(0xc0001fe1a0, 0xc000187b88)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1259 +0x230
testing.runTests(0xc000200000, {0x1ac0820, 0x178, 0x178}, {0xc000187bf0, 0x203000, 0x1ac4c60})
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1596 +0x7cb
testing.(*M).Run(0xc000200000)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1504 +0x9d2
main.main()
        _testmain.go:1115 +0x1e5

goroutine 19 [select]:
github.com/Shopify/sarama.expectResults(0xc0001fe340, {0x17c7658, 0xc0002be0e0}, 0x0, 0xa)
        /Users/slaunay/work/open-source/sarama/async_producer_test.go:43 +0x179
github.com/Shopify/sarama.TestAsyncProducerMultipleRetriesWithConcurrentRequests(0xc0001fe340)
        /Users/slaunay/work/open-source/sarama/async_producer_test.go:688 +0x465
testing.tRunner(0xc0001fe340, 0x171f9c0)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1259 +0x230
created by testing.(*T).Run
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1306 +0x727

goroutine 35 [select]:
github.com/Shopify/sarama.(*client).backgroundMetadataUpdater(0xc0000e63f0)
        /Users/slaunay/work/open-source/sarama/client.go:824 +0x2dd
github.com/Shopify/sarama.withRecover(0xc000298080)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.NewClient
        /Users/slaunay/work/open-source/sarama/client.go:183 +0x7d4

goroutine 36 [chan receive]:
github.com/Shopify/sarama.(*asyncProducer).dispatcher(0xc0002be0e0)
        /Users/slaunay/work/open-source/sarama/async_producer.go:331 +0x105
github.com/Shopify/sarama.withRecover(0xc0002980b0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.newAsyncProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:167 +0x513

goroutine 21 [IO wait]:
internal/poll.runtime_pollWait(0xa380740, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc000200098, 0xc00007e101, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0xbd
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Accept(0xc000200080)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:402 +0x425
net.(*netFD).accept(0xc000200080)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_unix.go:173 +0x4a
net.(*TCPListener).accept(0xc0000ba750)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock_posix.go:140 +0x45
net.(*TCPListener).Accept(0xc0000ba750)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock.go:262 +0x68
github.com/Shopify/sarama.(*MockBroker).serverLoop(0xc000200100)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:174 +0x1fb
created by github.com/Shopify/sarama.NewMockBrokerListener
        /Users/slaunay/work/open-source/sarama/mockbroker.go:419 +0x705

goroutine 24 [IO wait]:
internal/poll.runtime_pollWait(0xa380658, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc000200198, 0xc000292001, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0xbd
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Accept(0xc000200180)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:402 +0x425
net.(*netFD).accept(0xc000200180)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_unix.go:173 +0x4a
net.(*TCPListener).accept(0xc0000ba780)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock_posix.go:140 +0x45
net.(*TCPListener).Accept(0xc0000ba780)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock.go:262 +0x68
github.com/Shopify/sarama.(*MockBroker).serverLoop(0xc000200200)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:174 +0x1fb
created by github.com/Shopify/sarama.NewMockBrokerListener
        /Users/slaunay/work/open-source/sarama/mockbroker.go:419 +0x705

goroutine 28 [chan receive]:
github.com/Shopify/sarama.(*partitionProducer).dispatch(0xc0000a4ba0)
        /Users/slaunay/work/open-source/sarama/async_producer.go:546 +0x498
github.com/Shopify/sarama.withRecover(0xc000232630)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.(*asyncProducer).newPartitionProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:513 +0x3c5

goroutine 50 [chan receive]:
github.com/Shopify/sarama.(*MockBroker).serverLoop.func1()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:165 +0x5c
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:164 +0x138

goroutine 51 [chan receive]:
github.com/Shopify/sarama.(*MockBroker).serverLoop.func1()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:165 +0x5c
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:164 +0x138

goroutine 26 [chan receive]:
github.com/rcrowley/go-metrics.(*meterArbiter).tick(0x1ac49a0)
        /Users/slaunay/go/pkg/mod/github.com/rcrowley/[email protected]/meter.go:239 +0x59
created by github.com/rcrowley/go-metrics.NewMeter
        /Users/slaunay/go/pkg/mod/github.com/rcrowley/[email protected]/meter.go:46 +0x17f

goroutine 6 [IO wait]:
internal/poll.runtime_pollWait(0xa380488, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc00007e118, 0xc0002942dc, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0xbd
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Read(0xc00007e100, {0xc0002942dc, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:167 +0x419
net.(*netFD).Read(0xc00007e100, {0xc0002942dc, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_posix.go:56 +0x51
net.(*conn).Read(0xc000010018, {0xc0002942dc, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/net.go:183 +0xb1
io.ReadAtLeast({0x17bc580, 0xc000010018}, {0xc0002942dc, 0x4, 0x4}, 0x4)
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:328 +0xde
io.ReadFull({0x17bc580, 0xc000010018}, {0xc0002942dc, 0x0, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:347 +0x58
github.com/Shopify/sarama.(*MockBroker).readToBytes(0xc000200170, {0x17bc580, 0xc000010018})
        /Users/slaunay/work/open-source/sarama/mockbroker.go:193 +0x8f
github.com/Shopify/sarama.(*MockBroker).handleRequests(0xc000200100, {0xa3250d8, 0xc000010018}, 0x0, 0xc000122004)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:243 +0x4c9
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:176 +0x1d3

goroutine 7 [select]:
github.com/Shopify/sarama.(*MockBroker).handleRequests.func2()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:233 +0xda
created by github.com/Shopify/sarama.(*MockBroker).handleRequests
        /Users/slaunay/work/open-source/sarama/mockbroker.go:232 +0x41e

goroutine 27 [chan receive]:
github.com/Shopify/sarama.(*Broker).responseReceiver(0xc0000fd500)
        /Users/slaunay/work/open-source/sarama/broker.go:1033 +0xa5
github.com/Shopify/sarama.withRecover(0xc0002325f0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.(*Broker).Open.func1
        /Users/slaunay/work/open-source/sarama/broker.go:244 +0x110b

goroutine 37 [chan receive]:
github.com/Shopify/sarama.(*asyncProducer).retryHandler(0xc0002be0e0)
        /Users/slaunay/work/open-source/sarama/async_producer.go:1052 +0xbb
github.com/Shopify/sarama.withRecover(0xc0002980d0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.newAsyncProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:168 +0x5d9

goroutine 38 [chan receive]:
github.com/Shopify/sarama.(*topicProducer).dispatch(0xc00028c280)
        /Users/slaunay/work/open-source/sarama/async_producer.go:413 +0x9c
github.com/Shopify/sarama.withRecover(0xc0002980f0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.(*asyncProducer).newTopicProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:408 +0x391

# Trying to close the Broker but blocked by responseReceiver (see below)
goroutine 53 [chan receive]:
github.com/Shopify/sarama.(*Broker).Close(0xc0002a0380)
        /Users/slaunay/work/open-source/sarama/broker.go:287 +0x152
github.com/Shopify/sarama.(*brokerProducer).handleError(0xc000130000, 0x10ad42c, {0x17bc540, 0xc00029a1e0})
        /Users/slaunay/work/open-source/sarama/async_producer.go:1031 +0x2ba
github.com/Shopify/sarama.(*brokerProducer).handleResponse(0xc000130000, 0xc0002a21e0)
        /Users/slaunay/work/open-source/sarama/async_producer.go:898 +0xaf
github.com/Shopify/sarama.(*brokerProducer).run(0xc000130000)
        /Users/slaunay/work/open-source/sarama/async_producer.go:831 +0x127a
github.com/Shopify/sarama.withRecover(0xc000118060)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:691 +0x438

goroutine 54 [chan receive]:
github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1()
        /Users/slaunay/work/open-source/sarama/async_producer.go:695 +0x6f
github.com/Shopify/sarama.withRecover(0xc00012e0c0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:694 +0x5c5

# Invoking the bridge callback from another in flight request
# but blocked on the run goroutine (trying to close the Broker)
goroutine 8 [chan send]:
github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1.1.1(0x0, {0x17bc540, 0xc00029a1e0})
        /Users/slaunay/work/open-source/sarama/async_producer.go:701 +0x126
github.com/Shopify/sarama.(*Broker).AsyncProduce.func1({0x0, 0x0, 0x0}, {0x17bc540, 0xc00029a1e0})
        /Users/slaunay/work/open-source/sarama/broker.go:406 +0x1aa
github.com/Shopify/sarama.(*responsePromise).handle(0xc00028c3c0, {0x0, 0x0, 0x0}, {0x17bc540, 0xc00029a1e0})
        /Users/slaunay/work/open-source/sarama/broker.go:132 +0x1b9
github.com/Shopify/sarama.(*Broker).responseReceiver(0xc0002a0380)
        /Users/slaunay/work/open-source/sarama/broker.go:1038 +0x125
github.com/Shopify/sarama.withRecover(0xc000064160)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x75
created by github.com/Shopify/sarama.(*Broker).Open.func1
        /Users/slaunay/work/open-source/sarama/broker.go:244 +0x110b

goroutine 39 [IO wait]:
internal/poll.runtime_pollWait(0xa3802b8, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc000292098, 0xc000294bac, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0xbd
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Read(0xc000292080, {0xc000294bac, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:167 +0x419
net.(*netFD).Read(0xc000292080, {0xc000294bac, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_posix.go:56 +0x51
net.(*conn).Read(0xc000296080, {0xc000294bac, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/net.go:183 +0xb1
io.ReadAtLeast({0x17bc580, 0xc000296080}, {0xc000294bac, 0x4, 0x4}, 0x4)
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:328 +0xde
io.ReadFull({0x17bc580, 0xc000296080}, {0xc000294bac, 0x0, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:347 +0x58
github.com/Shopify/sarama.(*MockBroker).readToBytes(0xc000200270, {0x17bc580, 0xc000296080})
        /Users/slaunay/work/open-source/sarama/mockbroker.go:193 +0x8f
github.com/Shopify/sarama.(*MockBroker).handleRequests(0xc000200200, {0xa3250d8, 0xc000296080}, 0x0, 0xc000122014)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:243 +0x4c9
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:176 +0x1d3

goroutine 55 [select]:
github.com/Shopify/sarama.(*MockBroker).handleRequests.func2()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:233 +0xda
created by github.com/Shopify/sarama.(*MockBroker).handleRequests
        /Users/slaunay/work/open-source/sarama/mockbroker.go:232 +0x41e
exit status 2
FAIL    github.com/Shopify/sarama       5.395s

And from 59dd565 (proposed fix):

=== RUN   TestAsyncProducerMultipleRetriesWithConcurrentRequests
[sarama] 2022/02/08 11:20:28 *** mockbroker/1 listening on 127.0.0.1:61929
[sarama] 2022/02/08 11:20:28 *** mockbroker/2 listening on 127.0.0.1:61930
[sarama] 2022/02/08 11:20:28 Initializing new client
[sarama] 2022/02/08 11:20:28 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:20:28 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:20:28 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:20:28 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:20:28 client/metadata fetching metadata for all topics from broker 127.0.0.1:61929
[sarama] 2022/02/08 11:20:28 *** mockbroker/1/0: connection opened
[sarama] 2022/02/08 11:20:28 Connected to broker at 127.0.0.1:61929 (unregistered)
[sarama] 2022/02/08 11:20:28 *** mockbroker/1/0: replied to *sarama.MetadataRequest with *sarama.MetadataResponse
-> (*sarama.MetadataRequest){Version:(int16)0 Topics:([]string)<nil> AllowAutoTopicCreation:(bool)false}
-> (*sarama.MetadataResponse){Version:(int16)0 ThrottleTimeMs:(int32)0 Brokers:([]*sarama.Broker)[<max>] ClusterID:(*string)<nil> ControllerID:(int32)0 Topics:([]*sarama.TopicMetadata)[<max>]}
[sarama] 2022/02/08 11:20:28 client/brokers registered new broker #2 at 127.0.0.1:61930
[sarama] 2022/02/08 11:20:28 Successfully initialized new client
[sarama] 2022/02/08 11:20:28 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:20:28 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:20:28 producer/broker/2 starting up
[sarama] 2022/02/08 11:20:28 producer/broker/2 state change to [open] on my_topic/0
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/0: connection opened
[sarama] 2022/02/08 11:20:28 Connected to broker at 127.0.0.1:61930 (registered as #2)
[sarama] 2022/02/08 11:20:28 producer/broker/2 state change to [closing] because read tcp 127.0.0.1:61932->127.0.0.1:61930: i/o timeout
[sarama] 2022/02/08 11:20:28 producer/broker/2 state change to [closing] because read tcp 127.0.0.1:61932->127.0.0.1:61930: i/o timeout
[sarama] 2022/02/08 11:20:28 producer/leader/my_topic/0 state change to [retrying-1]
[sarama] 2022/02/08 11:20:28 producer/leader/my_topic/0 abandoning broker 2
[sarama] 2022/02/08 11:20:28 Closed connection to broker 127.0.0.1:61930
[sarama] 2022/02/08 11:20:28 producer/broker/2 closing done
[sarama] 2022/02/08 11:20:28 producer/broker/2 input chan closed
[sarama] 2022/02/08 11:20:28 producer/broker/2 shut down
[sarama] 2022/02/08 11:20:28 client/metadata fetching metadata for [my_topic] from broker 127.0.0.1:61929
[sarama] 2022/02/08 11:20:28 *** mockbroker/1/0: replied to *sarama.MetadataRequest with *sarama.MetadataResponse
-> (*sarama.MetadataRequest){Version:(int16)0 Topics:([]string)[<max>] AllowAutoTopicCreation:(bool)false}
-> (*sarama.MetadataResponse){Version:(int16)0 ThrottleTimeMs:(int32)0 Brokers:([]*sarama.Broker)[<max>] ClusterID:(*string)<nil> ControllerID:(int32)0 Topics:([]*sarama.TopicMetadata)[<max>]}
[sarama] 2022/02/08 11:20:28 Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.
[sarama] 2022/02/08 11:20:28 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/08 11:20:28 producer/broker/2 starting up
[sarama] 2022/02/08 11:20:28 producer/broker/2 state change to [open] on my_topic/0
[sarama] 2022/02/08 11:20:28 producer/leader/my_topic/0 selected broker 2
[sarama] 2022/02/08 11:20:28 producer/leader/my_topic/0 state change to [flushing-1]
[sarama] 2022/02/08 11:20:28 producer/leader/my_topic/0 state change to [normal]
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/1: connection opened
[sarama] 2022/02/08 11:20:28 Connected to broker at 127.0.0.1:61930 (registered as #2)
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/08 11:20:28 producer/broker/2 state change to [closing] because read tcp 127.0.0.1:61933->127.0.0.1:61930: i/o timeout
[sarama] 2022/02/08 11:20:28 producer/broker/2 state change to [closing] because read tcp 127.0.0.1:61933->127.0.0.1:61930: i/o timeout
[sarama] 2022/02/08 11:20:28 Closed connection to broker 127.0.0.1:61930
[sarama] 2022/02/08 11:20:28 producer/broker/2 closing done
[sarama] 2022/02/08 11:20:28 *** mockbroker/1/0: invalid request: err=read tcp 127.0.0.1:61929->127.0.0.1:61931: use of closed network connection, ([]uint8) <nil>
[sarama] 2022/02/08 11:20:28 *** mockbroker/1/0: connection closed, err=<nil>
[sarama] 2022/02/08 11:20:28 *** mockbroker/1: listener closed, err=accept tcp 127.0.0.1:61929: use of closed network connection
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/0: connection closed, err=<nil>
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/1: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/08 11:20:28 *** mockbroker/2/1: connection closed, err=<nil>
[sarama] 2022/02/08 11:20:28 *** mockbroker/2: listener closed, err=accept tcp 127.0.0.1:61930: use of closed network connection
[sarama] 2022/02/08 11:20:28 Producer shutting down.
[sarama] 2022/02/08 11:20:28 Closing Client
[sarama] 2022/02/08 11:20:28 Closed connection to broker 127.0.0.1:61929
[sarama] 2022/02/08 11:20:28 producer/broker/2 input chan closed
--- PASS: TestAsyncProducerMultipleRetriesWithConcurrentRequests (0.64s)
[sarama] 2022/02/08 11:20:28 producer/broker/2 shut down
PASS
coverage: 13.5% of statements
ok      github.com/Shopify/sarama       1.139s

I was able to get the functional tests to pass as well with 100% of the new code covered.

But because concurrency is hard, it would be great to have @eafzali and @hxiaodon apply the fix and confirm that it is working for them too.
Just remember that the AsyncProducer can take a while to recover with all the retry logic.
I believe the following dependency can be used to test it out in a project:

require (
	github.com/Shopify/sarama v1.31.1
)

replace (
	github.com/Shopify/sarama => github.com/slaunay/sarama v1.31.2-0.20220208202506-59dd56590fd2
)

- add unit test to reproduce the deadlock by simulating a network error
- document possible deadlock when closing the Broker from an AsyncProduce
  callback when handling a response error
- add closeBroker goroutine and channel to asynchronously close a Broker
  once
- reuse the stopchan channel to signal that the closeBroker goroutine is
  done
- update TestBrokerProducerShutdown to check goroutine leak by closing
  the input vs the stopchan channel
- fixes IBM#2129
@slaunay slaunay requested a review from bai as a code owner February 8, 2022 20:28
WARNING: DATA RACE
Write at 0x00c0003421f0 by goroutine 71:
  runtime.closechan()
      runtime/chan.go:355 +0x0
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1()
      github.com/Shopify/sarama/async_producer.go:725 +0x1c4
  github.com/Shopify/sarama.withRecover()
      github.com/Shopify/sarama/utils.go:43 +0x74
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer·dwrap·15()
      github.com/Shopify/sarama/async_producer.go:695 +0x39

Previous read at 0x00c0003421f0 by goroutine 58:
  runtime.chansend()
      runtime/chan.go:158 +0x0
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1.1.1()
      github.com/Shopify/sarama/async_producer.go:702 +0x125
  github.com/Shopify/sarama.(*Broker).AsyncProduce.func1()
      github.com/Shopify/sarama/broker.go:408 +0x1a9
  github.com/Shopify/sarama.(*responsePromise).handle()
      github.com/Shopify/sarama/broker.go:132 +0x1b8
  github.com/Shopify/sarama.(*Broker).responseReceiver()
      github.com/Shopify/sarama/broker.go:1040 +0x124
  github.com/Shopify/sarama.(*Broker).responseReceiver-fm()
      github.com/Shopify/sarama/broker.go:1032 +0x39
  github.com/Shopify/sarama.withRecover()
      github.com/Shopify/sarama/utils.go:43 +0x74
  github.com/Shopify/sarama.(*Broker).Open.func1·dwrap·22()
      github.com/Shopify/sarama/broker.go:244 +0x39
@slaunay
Copy link
Contributor Author

slaunay commented Feb 9, 2022

The race detector found a data race that is not directly linked to the deadlock but can lead to a panic when sending to a closed channel.
So I added an additional fix using a sync.WaitGroup as it is possible for the bridge goroutine to be done receiving requests (bridge/output channel closed) while still having pending callbacks.

Here is the details of the data race:

WARNING: DATA RACE
Write at 0x00c0003421f0 by goroutine 71:
  runtime.closechan()
      runtime/chan.go:355 +0x0
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1()
      github.com/Shopify/sarama/async_producer.go:725 +0x1c4
  github.com/Shopify/sarama.withRecover()
      github.com/Shopify/sarama/utils.go:43 +0x74
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer·dwrap·15()
      github.com/Shopify/sarama/async_producer.go:695 +0x39

Previous read at 0x00c0003421f0 by goroutine 58:
  runtime.chansend()
      runtime/chan.go:158 +0x0
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1.1.1()
      github.com/Shopify/sarama/async_producer.go:702 +0x125
  github.com/Shopify/sarama.(*Broker).AsyncProduce.func1()
      github.com/Shopify/sarama/broker.go:408 +0x1a9
  github.com/Shopify/sarama.(*responsePromise).handle()
      github.com/Shopify/sarama/broker.go132 +0x1b8
  github.com/Shopify/sarama.(*Broker).responseReceiver()
      github.com/Shopify/sarama/broker.go:1040 +0x124
  github.com/Shopify/sarama.(*Broker).responseReceiver-fm()
      github.com/Shopify/sarama/broker.go:1032 +0x39
  github.com/Shopify/sarama.withRecover()
      github.com/Shopify/sarama/utils.go:43 +0x74
  github.com/Shopify/sarama.(*Broker).Open.func0·dwrap·22()
      github.com/Shopify/sarama/broker.go:244 +0x39

Goroutine 71 (running) created at:
  github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer()
      github.com/Shopify/sarama/async_producer.go:695 +0x645
  github.com/Shopify/sarama.(*asyncProducer).getBrokerProducer()
      github.com/Shopify/sarama/async_producer.go:1169 +0x14f
  github.com/Shopify/sarama.(*partitionProducer).dispatch()
      github.com/Shopify/sarama/async_producer.go:535 +0x1bd
  github.com/Shopify/sarama.(*partitionProducer).dispatch-fm()
      github.com/Shopify/sarama/async_producer.go:530 +0x39
  github.com/Shopify/sarama.withRecover()
      github.com/Shopify/sarama/utils.go:43 +0x74
  github.com/Shopify/sarama.(*asyncProducer).newPartitionProducer·dwrap·13()
      github.com/Shopify/sarama/async_producer.go:513 +0x39

Goroutine 58 (running) created at:
  github.com/Shopify/sarama.(*Broker).Open.func1()
      github.com/Shopify/sarama/broker.go:244 +0x110a
  github.com/Shopify/sarama.withRecover()
      github.com/Shopify/sarama/utils.go:43 +0x74
  github.com/Shopify/sarama.(*Broker).Open·dwrap·21()
      github.com/Shopify/sarama/broker.go:172 +0x39:

And the new commit/version to test again:

require (
	github.com/Shopify/sarama v1.31.1
)

replace (
	github.com/Shopify/sarama => github.com/slaunay/sarama v1.31.2-0.20220209005248-6c70a6c230a4
)

Two tests are failing randomly but they might be flaky:
https://github.com/Shopify/sarama/runs/5118582592?check_suite_focus=true#step:7:166

=== RUN   TestAsyncProducerBrokerBounce
    async_producer_test.go:51: kafka: client has run out of available brokers to talk to (Is your cluster reachable?)
    async_producer_test.go:65: Unexpected successes 1 or errors -1
--- FAIL: TestAsyncProducerBrokerBounce (362.29s)

https://github.com/Shopify/sarama/runs/5118582681?check_suite_focus=true#step:8:30402

=== RUN   TestReadOnlyAndAllCommittedMessages
[sarama] 2022/02/09 01:10:59 Initializing new client
...
[sarama] 2022/02/09 01:11:01 consumer/broker/3 added subscription to uncommitted-topic-test-4/0
    functional_consumer_test.go:278: Received uncommitted message 1 from uncommitted-topic-test-4-0 at offset 0
    functional_consumer_test.go:279: 
        	Error Trace:	functional_consumer_test.go:279
        	Error:      	Not equal: 
        	            	expected: "Committed 1"
        	            	actual  : "uncommitted message 1"
        	            	
        	            	Diff:
        	            	--- Expected
        	            	+++ Actual
        	            	@@ -1 +1 @@
        	            	-Committed 1
        	            	+uncommitted message 1
        	Test:       	TestReadOnlyAndAllCommittedMessages
[sarama] 2022/02/09 01:11:01 Closing Client
[sarama] 2022/02/09 01:11:01 Producer shutting down.
[sarama] 2022/02/09 01:11:01 Closed connection to broker 127.0.0.1:29091
[sarama] 2022/02/09 01:11:01 Closing Client
[sarama] 2022/02/09 01:11:01 Closed connection to broker localhost:29092
[sarama] 2022/02/09 01:11:01 Closed connection to broker 127.0.0.1:29095
[sarama] 2022/02/09 01:11:01 producer/broker/3 input chan closed
[sarama] 2022/02/09 01:11:01 producer/broker/3 shut down
[sarama] 2022/02/09 01:11:01 Closing Client
[sarama] 2022/02/09 01:11:01 Closed connection to broker localhost:29093
--- FAIL: TestReadOnlyAndAllCommittedMessages (2.13s)

@hxiaodon
Copy link

hxiaodon commented Feb 9, 2022

@slaunay, Thank you for the quick fix:)
We have one of our instance rebuild with your pseudo-version v1.31.2-0.20220208202506-59dd56590fd2, and have deployed it in production env.
We had also tried another tough fix for a couple of days, and seems it works, but I'm not sure if it will cause any other issue(e.g goroutine leak)

@@ -675,7 +675,7 @@ 
func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {
        var (
                input     = make(chan *ProducerMessage)
                bridge    = make(chan *produceSet)
-               responses = make(chan *brokerProducerResponse)
+               responses = make(chan *brokerProducerResponse, 1)
        )
 
        bp := &brokerProducer

We will keep monitoring our instances and hope that the deadlock issue is resolved completely.
Thanks again~

@bai bai requested a review from dnwe February 9, 2022 05:31
Copy link
Contributor

@bai bai left a comment

Choose a reason for hiding this comment

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

LGTM, will let @dnwe take a look too.

As always — excellent writeup! Thanks for contributing ❤️

@eafzali
Copy link

eafzali commented Feb 9, 2022

I highly recommend adding a retract for 1.31.0 and 1.31.1 in your go.mod file. Because it already caused 2 incidents in our production environment, so can happen to other users as well.

@slaunay
Copy link
Contributor Author

slaunay commented Feb 9, 2022

We had also tried another tough fix for a couple of days, and seems it works, but I'm not sure if it will cause any other issue(e.g goroutine leak)

@@ -675,7 +675,7 @@ 
func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {
        var (
                input     = make(chan *ProducerMessage)
                bridge    = make(chan *produceSet)
-               responses = make(chan *brokerProducerResponse)
+               responses = make(chan *brokerProducerResponse, 1)
        )
 
        bp := &brokerProducer

We will keep monitoring our instances and hope that the deadlock issue is resolved completely. Thanks again~

Thanks for testing the fix that fast, not sure if production is the best environment to test such changes 😬 but it will be useful to have real functional testing.

Your proposed fix does workaround the deadlock indeed when config.Net.MaxOpenRequests = 1 because the buffering would prevent blocking the responseReceiver goroutine and allow for closing the Broker.
Now if there are more than 1 in-flight request, the deadlock would unfortunately still occur as the callback would be blocked trying to send an extra response to the run goroutine.

Here is the updated test case from commit 8f92872 (using Producer.Flush.MaxMessages vs Producer.Flush.Messages for deterministic batching and the default config.Net.MaxOpenRequests = 5) run with the buffered channel fix:

$ go test -v -timeout 5s -run=TestAsyncProducerMultipleRetriesWithConcurrentRequests -count 1
=== RUN   TestAsyncProducerMultipleRetriesWithConcurrentRequests
[sarama] 2022/02/09 10:29:07 *** mockbroker/1 listening on 127.0.0.1:57889
[sarama] 2022/02/09 10:29:07 *** mockbroker/2 listening on 127.0.0.1:57890
[sarama] 2022/02/09 10:29:07 Initializing new client
[sarama] 2022/02/09 10:29:07 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/09 10:29:07 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/09 10:29:07 client/metadata fetching metadata for all topics from broker 127.0.0.1:57889
[sarama] 2022/02/09 10:29:07 *** mockbroker/1/0: connection opened
[sarama] 2022/02/09 10:29:07 Connected to broker at 127.0.0.1:57889 (unregistered)
[sarama] 2022/02/09 10:29:07 *** mockbroker/1/0: replied to *sarama.MetadataRequest with *sarama.MetadataResponse
-> (*sarama.MetadataRequest){Version:(int16)0 Topics:([]string)<nil> AllowAutoTopicCreation:(bool)false}
-> (*sarama.MetadataResponse){Version:(int16)0 ThrottleTimeMs:(int32)0 Brokers:([]*sarama.Broker)[<max>] ClusterID:(*string)<nil> ControllerID:(int32)0 Topics:([]*sarama.TopicMetadata)[<max>]}
[sarama] 2022/02/09 10:29:07 client/brokers registered new broker #2 at 127.0.0.1:57890
[sarama] 2022/02/09 10:29:07 Successfully initialized new client
[sarama] 2022/02/09 10:29:07 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/09 10:29:07 producer/broker/2 starting up
[sarama] 2022/02/09 10:29:07 producer/broker/2 state change to [open] on my_topic/0
[sarama] 2022/02/09 10:29:07 producer/broker/2 maximum request accumulated, waiting for space
[sarama] 2022/02/09 10:29:07 producer/broker/2 maximum request accumulated, waiting for space
[sarama] 2022/02/09 10:29:07 *** mockbroker/2/0: connection opened
[sarama] 2022/02/09 10:29:07 Connected to broker at 127.0.0.1:57890 (registered as #2)
[sarama] 2022/02/09 10:29:07 producer/broker/2 maximum request accumulated, waiting for space
[sarama] 2022/02/09 10:29:07 producer/broker/2 maximum request accumulated, waiting for space
[sarama] 2022/02/09 10:29:07 producer/broker/2 maximum request accumulated, waiting for space
[sarama] 2022/02/09 10:29:07 producer/broker/2 state change to [closing] because read tcp 127.0.0.1:57892->127.0.0.1:57890: i/o timeout
[sarama] 2022/02/09 10:29:07 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/09 10:29:07 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/09 10:29:07 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/09 10:29:07 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/09 10:29:08 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/09 10:29:08 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
panic: test timed out after 5s

goroutine 20 [running]:
testing.(*M).startAlarm.func1()
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1788 +0x8e
created by time.goFunc
        /usr/local/Cellar/go/1.17.5/libexec/src/time/sleep.go:180 +0x31

goroutine 1 [chan receive]:
testing.(*T).Run(0xc00011d520, {0x14a09cf, 0x195301add50e2}, 0x14ac3f8)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1307 +0x375
testing.runTests.func1(0xc000113680)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1598 +0x6e
testing.tRunner(0xc00011d520, 0xc000123d18)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1259 +0x102
testing.runTests(0xc00017e000, {0x17bc380, 0x178, 0x178}, {0x107ee2d, 0x487dd0, 0x17c07c0})
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1596 +0x43f
testing.(*M).Run(0xc00017e000)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1504 +0x51d
main.main()
        _testmain.go:803 +0x14b

goroutine 4 [select]:
github.com/Shopify/sarama.expectResults(0xc00011d6c0, {0x1531a98, 0xc00015ae70}, 0x1486b6c, 0xc00004f720)
        /Users/slaunay/work/open-source/sarama/async_producer_test.go:43 +0x118
github.com/Shopify/sarama.TestAsyncProducerMultipleRetriesWithConcurrentRequests(0xc00011d6c0)
        /Users/slaunay/work/open-source/sarama/async_producer_test.go:689 +0x2d9
testing.tRunner(0xc00011d6c0, 0x14ac3f8)
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1259 +0x102
created by testing.(*T).Run
        /usr/local/Cellar/go/1.17.5/libexec/src/testing/testing.go:1306 +0x35a

goroutine 7 [IO wait]:
internal/poll.runtime_pollWait(0x1e66468, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc00017e100, 0x4, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0x32
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Accept(0xc00017e100)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:402 +0x22c
net.(*netFD).accept(0xc00017e100)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_unix.go:173 +0x35
net.(*TCPListener).accept(0xc00000e750)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock_posix.go:140 +0x28
net.(*TCPListener).Accept(0xc00000e750)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock.go:262 +0x3d
github.com/Shopify/sarama.(*MockBroker).serverLoop(0xc00017e180)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:174 +0x113
created by github.com/Shopify/sarama.NewMockBrokerListener
        /Users/slaunay/work/open-source/sarama/mockbroker.go:419 +0x42f

goroutine 10 [IO wait]:
internal/poll.runtime_pollWait(0x1e66380, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc00017e200, 0x4, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0x32
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Accept(0xc00017e200)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:402 +0x22c
net.(*netFD).accept(0xc00017e200)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_unix.go:173 +0x35
net.(*TCPListener).accept(0xc00000e780)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock_posix.go:140 +0x28
net.(*TCPListener).Accept(0xc00000e780)
        /usr/local/Cellar/go/1.17.5/libexec/src/net/tcpsock.go:262 +0x3d
github.com/Shopify/sarama.(*MockBroker).serverLoop(0xc00017e280)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:174 +0x113
created by github.com/Shopify/sarama.NewMockBrokerListener
        /Users/slaunay/work/open-source/sarama/mockbroker.go:419 +0x42f

goroutine 14 [select]:
github.com/Shopify/sarama.(*client).backgroundMetadataUpdater(0xc00014a5a0)
        /Users/slaunay/work/open-source/sarama/client.go:824 +0x12e
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.NewClient
        /Users/slaunay/work/open-source/sarama/client.go:183 +0x3ef

goroutine 34 [chan receive]:
github.com/Shopify/sarama.(*MockBroker).serverLoop.func1()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:165 +0x2c
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:164 +0xc5

goroutine 35 [chan receive]:
github.com/Shopify/sarama.(*MockBroker).serverLoop.func1()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:165 +0x2c
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:164 +0xc5

goroutine 12 [runnable]:
github.com/rcrowley/go-metrics.(*meterArbiter).tick(0x17c0500)
        /Users/slaunay/go/pkg/mod/github.com/rcrowley/[email protected]/meter.go:239 +0x2a
created by github.com/rcrowley/go-metrics.NewMeter
        /Users/slaunay/go/pkg/mod/github.com/rcrowley/[email protected]/meter.go:46 +0xd8

goroutine 51 [IO wait]:
internal/poll.runtime_pollWait(0x1e661b0, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc00028c100, 0xc000028ad0, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0x32
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Read(0xc00028c100, {0xc000028ad0, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:167 +0x25a
net.(*netFD).Read(0xc00028c100, {0xc000028ad0, 0x100e634, 0x3d})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_posix.go:56 +0x29
net.(*conn).Read(0xc000286008, {0xc000028ad0, 0x18b1108, 0x10})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/net.go:183 +0x45
io.ReadAtLeast({0x1526f40, 0xc000286008}, {0xc000028ad0, 0x4, 0x4}, 0x4)
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:328 +0x9a
io.ReadFull(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:347
github.com/Shopify/sarama.(*MockBroker).readToBytes(0xc000286008, {0x1526f40, 0xc000286008})
        /Users/slaunay/work/open-source/sarama/mockbroker.go:193 +0x72
github.com/Shopify/sarama.(*MockBroker).handleRequests(0xc00017e180, {0x97040e0, 0xc000286008}, 0x0, 0xc00017e280)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:243 +0x305
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:176 +0xfd

goroutine 52 [select]:
github.com/Shopify/sarama.(*MockBroker).handleRequests.func2()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:233 +0x72
created by github.com/Shopify/sarama.(*MockBroker).handleRequests
        /Users/slaunay/work/open-source/sarama/mockbroker.go:232 +0x2c5

goroutine 13 [chan receive]:
github.com/Shopify/sarama.(*Broker).responseReceiver(0xc00015f500)
        /Users/slaunay/work/open-source/sarama/broker.go:1033 +0x74
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.(*Broker).Open.func1
        /Users/slaunay/work/open-source/sarama/broker.go:244 +0xb65

goroutine 15 [chan receive]:
github.com/Shopify/sarama.(*asyncProducer).dispatcher(0xc00015ae70)
        /Users/slaunay/work/open-source/sarama/async_producer.go:331 +0xd2
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.newAsyncProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:167 +0x287

goroutine 16 [chan receive]:
github.com/Shopify/sarama.(*asyncProducer).retryHandler(0xc00015ae70)
        /Users/slaunay/work/open-source/sarama/async_producer.go:1052 +0x19f
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.newAsyncProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:168 +0x307

goroutine 66 [chan receive]:
github.com/Shopify/sarama.(*topicProducer).dispatch(0xc000035f40)
        /Users/slaunay/work/open-source/sarama/async_producer.go:413 +0x54
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.(*asyncProducer).newTopicProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:408 +0x1fb

goroutine 36 [chan send]:
github.com/Shopify/sarama.(*partitionProducer).dispatch(0xc00020e240)
        /Users/slaunay/work/open-source/sarama/async_producer.go:606 +0x6f8
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.(*asyncProducer).newPartitionProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:513 +0x1fb

goroutine 38 [chan receive]:
github.com/Shopify/sarama.(*Broker).Close(0xc00015fc00)
        /Users/slaunay/work/open-source/sarama/broker.go:287 +0xcb
github.com/Shopify/sarama.(*brokerProducer).handleError(0xc000222070, 0x1d, {0x1526f00, 0xc0001087d0})
        /Users/slaunay/work/open-source/sarama/async_producer.go:1031 +0x185
github.com/Shopify/sarama.(*brokerProducer).handleResponse(0xc000222070, 0xc00005ee18)
        /Users/slaunay/work/open-source/sarama/async_producer.go:898 +0x2e
github.com/Shopify/sarama.(*brokerProducer).waitForSpace(0xc000222070, 0xc00013edc0, 0x0)
        /Users/slaunay/work/open-source/sarama/async_producer.go:876 +0xc5
github.com/Shopify/sarama.(*brokerProducer).run(0xc000222070)
        /Users/slaunay/work/open-source/sarama/async_producer.go:803 +0x50c
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:691 +0x272

goroutine 39 [chan receive]:
github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1()
        /Users/slaunay/work/open-source/sarama/async_producer.go:695 +0x56
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer
        /Users/slaunay/work/open-source/sarama/async_producer.go:694 +0x347

goroutine 53 [IO wait]:
internal/poll.runtime_pollWait(0x1e65fe0, 0x72)
        /usr/local/Cellar/go/1.17.5/libexec/src/runtime/netpoll.go:234 +0x89
internal/poll.(*pollDesc).wait(0xc00028c180, 0xc0000be500, 0x0)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:84 +0x32
internal/poll.(*pollDesc).waitRead(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_poll_runtime.go:89
internal/poll.(*FD).Read(0xc00028c180, {0xc0000be500, 0x4, 0x4})
        /usr/local/Cellar/go/1.17.5/libexec/src/internal/poll/fd_unix.go:167 +0x25a
net.(*netFD).Read(0xc00028c180, {0xc0000be500, 0x100e634, 0x20})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/fd_posix.go:56 +0x29
net.(*conn).Read(0xc000286030, {0xc0000be500, 0x18b15b8, 0x10})
        /usr/local/Cellar/go/1.17.5/libexec/src/net/net.go:183 +0x45
io.ReadAtLeast({0x1526f40, 0xc000286030}, {0xc0000be500, 0x4, 0x4}, 0x4)
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:328 +0x9a
io.ReadFull(...)
        /usr/local/Cellar/go/1.17.5/libexec/src/io/io.go:347
github.com/Shopify/sarama.(*MockBroker).readToBytes(0xc000286030, {0x1526f40, 0xc000286030})
        /Users/slaunay/work/open-source/sarama/mockbroker.go:193 +0x72
github.com/Shopify/sarama.(*MockBroker).handleRequests(0xc00017e280, {0x97040e0, 0xc000286030}, 0x12ca7a6, 0x0)
        /Users/slaunay/work/open-source/sarama/mockbroker.go:243 +0x305
created by github.com/Shopify/sarama.(*MockBroker).serverLoop
        /Users/slaunay/work/open-source/sarama/mockbroker.go:176 +0xfd

goroutine 54 [select]:
github.com/Shopify/sarama.(*MockBroker).handleRequests.func2()
        /Users/slaunay/work/open-source/sarama/mockbroker.go:233 +0x72
created by github.com/Shopify/sarama.(*MockBroker).handleRequests
        /Users/slaunay/work/open-source/sarama/mockbroker.go:232 +0x2c5

goroutine 55 [chan send]:
github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1.2(0x0, {0x1526f00, 0xc0001087d0})
        /Users/slaunay/work/open-source/sarama/async_producer.go:701 +0xc5
github.com/Shopify/sarama.(*Broker).AsyncProduce.func1({0x0, 0xc000209678, 0x121b7c2}, {0x1526f00, 0xc0001087d0})
        /Users/slaunay/work/open-source/sarama/broker.go:406 +0xc3
github.com/Shopify/sarama.(*responsePromise).handle(0xc00015fc00, {0x0, 0x17c07c0, 0x8}, {0x1526f00, 0xc0001087d0})
        /Users/slaunay/work/open-source/sarama/broker.go:132 +0x99
github.com/Shopify/sarama.(*Broker).responseReceiver(0xc00015fc00)
        /Users/slaunay/work/open-source/sarama/broker.go:1038 +0x40e
github.com/Shopify/sarama.withRecover(0x0)
        /Users/slaunay/work/open-source/sarama/utils.go:43 +0x3e
created by github.com/Shopify/sarama.(*Broker).Open.func1
        /Users/slaunay/work/open-source/sarama/broker.go:244 +0xb65
exit status 2
FAIL    github.com/Shopify/sarama       5.360s

A better fix would then be:

@@ -675,7 +675,7 @@ 
func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {
        var (
                input     = make(chan *ProducerMessage)
                bridge    = make(chan *produceSet)
-               responses = make(chan *brokerProducerResponse)
+               responses = make(chan *brokerProducerResponse, p.conf.Net.MaxOpenRequests)
        )
 
        bp := &brokerProducer

but using this approach we are leaking some internal from the Broker logic into the AsyncProducer on how many in-flight requests are supported.
There is still some odd behaviour on where the semaphore is used to limit the number of in-flight requests and if it changes down the road then it could impact the AsyncProducer if we rely on channel buffering.

I highly recommend adding a retract for 1.31.0 and 1.31.1 in your go.mod file. Because it already caused 2 incidents in our production environment, so can happen to other users as well.

I have never used retract but that sounds like a good idea once 1.31.2 is released.

@dnwe dnwe added the fix label Feb 9, 2022
@dnwe
Copy link
Collaborator

dnwe commented Feb 9, 2022

@slaunay it does look like TestAsyncProducerRetryWithReferenceOpen has become flakey when run on this branch

Against main I can re-run go test -run '^TestAsyncProducerRetryWithReferenceOpen$' -count=4096 multiple times without a single failure, but against bugfix/broker-producer-callback-deadlock I do get not all expectations were satisfied errors on the mockbrokers with the producer Errors chan having returned i/o timeout or ErrNotConnected.

It may just be that the old test was relying on some timing constraints that are no longer valid and it should have always been able to cope with these errors on the chan, but we should dig into it just to make sure

@hxiaodon
Copy link

@slaunay
The producer hang bug is reproduced at our prod env and I think the hang point is changed to another place

goroutine profile: total 332
43 @ 0x4383b6 0x40640c 0x405e78 0x74d266 0x7a639e 0x468c21
#	0x74d265	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x1a5	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:546
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

24 @ 0x4383b6 0x40640c 0x405e78 0x768c3a 0x7a639e 0x468c21
#	0x768c39	github.com/Shopify/sarama.(*partitionConsumer).dispatcher+0x39	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer.go:436
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

24 @ 0x4383b6 0x40640c 0x405e78 0x770994 0x468c21
#	0x770993	github.com/Shopify/sarama.newConsumerGroupSession.func1+0x93	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:677

24 @ 0x4383b6 0x40640c 0x405e78 0x77123c 0x468c21
#	0x77123b	github.com/Shopify/sarama.(*consumerGroupSession).consume.func1+0x9b	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:763

24 @ 0x4383b6 0x40640c 0x405e78 0x772018 0x468c21
#	0x772017	github.com/Shopify/sarama.newConsumerGroupClaim.func1+0x97	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:951

24 @ 0x4383b6 0x448052 0x77115e 0x468c21
#	0x77115d	github.com/Shopify/sarama.(*consumerGroupSession).consume.func2+0x7d	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:770


14 @ 0x4383b6 0x448052 0x76989f 0x7a639e 0x468c21
#	0x76989e	github.com/Shopify/sarama.(*partitionConsumer).responseFeeder+0x41e	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer.go:589
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

10 @ 0x4383b6 0x40640c 0x405e78 0x769512 0x7a639e 0x468c21
#	0x769511	github.com/Shopify/sarama.(*partitionConsumer).responseFeeder+0x91	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer.go:566
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

10 @ 0x4383b6 0x448052 0xcd5d73 0x468c21
#	0xcd5d72	database/sql.(*DB).connectionOpener+0x92	/Users/tiger/.go/go1.17.3/src/database/sql/sql.go:1196

8 @ 0x4383b6 0x40640c 0x405e78 0x74e8bb 0x7a639e 0x468c21
#	0x74e8ba	github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func2+0x3a	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:740
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

8 @ 0x4383b6 0x40640c 0x405e78 0x74eb45 0x7a639e 0x468c21
#	0x74eb44	github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1+0x64	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:697
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43


7 @ 0x4383b6 0x40640c 0x405e78 0x757934 0x7a639e 0x468c21
#	0x757933	github.com/Shopify/sarama.(*Broker).responseReceiver+0x73	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1035
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43


5 @ 0x4383b6 0x448052 0x74f014 0x7a639e 0x468c21
#	0x74f013	github.com/Shopify/sarama.(*brokerProducer).run+0x193	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:790
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

4 @ 0x4383b6 0x448052 0x76150e 0x7a639e 0x468c21
#	0x76150d	github.com/Shopify/sarama.(*client).backgroundMetadataUpdater+0x12d	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/client.go:824
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x405565 0x40511d 0x74e349 0x5064fd 0x5063f4 0x74e1b6 0x74e175 0x74d5ed 0x7a639e 0x468c21
#	0x74e348	github.com/Shopify/sarama.(*partitionProducer).updateLeader.func1+0x168	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:667
#	0x5064fc	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork.func1+0x5c	/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:85
#	0x5063f3	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork+0x33		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:86
#	0x74e1b5	github.com/eapache/go-resiliency/breaker.(*Breaker).Run+0x55		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:55
#	0x74e174	github.com/Shopify/sarama.(*partitionProducer).updateLeader+0x14	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:656
#	0x74d5ec	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x52c		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:589
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x405565 0x40511d 0x751d94 0x750baa 0x79bfe2 0x74ff4f 0x74fe7f 0x74f03f 0x7a639e 0x468c21
#	0x751d93	github.com/Shopify/sarama.(*asyncProducer).returnSuccesses+0xb3		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1148
#	0x750ba9	github.com/Shopify/sarama.(*brokerProducer).handleSuccess.func1+0x469	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:962
#	0x79bfe1	github.com/Shopify/sarama.(*produceSet).eachPartition+0x101		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/produce_set.go:211
#	0x74ff4e	github.com/Shopify/sarama.(*brokerProducer).handleSuccess+0x8e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:938
#	0x74fe7e	github.com/Shopify/sarama.(*brokerProducer).handleResponse+0x3e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:926
#	0x74f03e	github.com/Shopify/sarama.(*brokerProducer).run+0x1be			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:856
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x40640c 0x405e38 0x75175f 0x7a639e 0x468c21
#	0x75175e	github.com/Shopify/sarama.(*asyncProducer).retryHandler+0x19e	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1083
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x40640c 0x405e78 0x74c1b2 0x7a639e 0x468c21
#	0x74c1b1	github.com/Shopify/sarama.(*asyncProducer).dispatcher+0xd1	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:331
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x40640c 0x405e78 0x7a4f65 0x7a639e 0x468c21
#	0x7a4f64	github.com/Shopify/sarama.(*syncProducer).handleErrors+0xa4	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:138
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x430973 0x463209 0x4d4532 0x4d589a 0x4d5888 0x5a29e9 0x5b4d25 0x560fb4 0x7a6745 0x4cc8ba 0x756934 0x756905 0x7579b8 0x7a639e 0x468c21
#	0x463208	internal/poll.runtime_pollWait+0x88				/Users/tiger/.go/go1.17.3/src/runtime/netpoll.go:234
#	0x4d4531	internal/poll.(*pollDesc).wait+0x31				/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:84
#	0x4d5899	internal/poll.(*pollDesc).waitRead+0x259			/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:89
#	0x4d5887	internal/poll.(*FD).Read+0x247					/Users/tiger/.go/go1.17.3/src/internal/poll/fd_unix.go:167
#	0x5a29e8	net.(*netFD).Read+0x28						/Users/tiger/.go/go1.17.3/src/net/fd_posix.go:56
#	0x5b4d24	net.(*conn).Read+0x44						/Users/tiger/.go/go1.17.3/src/net/net.go:183
#	0x560fb3	bufio.(*Reader).Read+0x1b3					/Users/tiger/.go/go1.17.3/src/bufio/bufio.go:227
#	0x7a6744	github.com/Shopify/sarama.(*bufConn).Read+0x24			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:107
#	0x4cc8b9	io.ReadAtLeast+0x99						/Users/tiger/.go/go1.17.3/src/io/io.go:328
#	0x756933	io.ReadFull+0xb3						/Users/tiger/.go/go1.17.3/src/io/io.go:347
#	0x756904	github.com/Shopify/sarama.(*Broker).readFull+0x84		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:871
#	0x7579b7	github.com/Shopify/sarama.(*Broker).responseReceiver+0xf7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1047
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x448052 0x757332 0x75538f 0x76cc87 0x76b605 0x7a639e 0x468c21
#	0x757331	github.com/Shopify/sarama.(*Broker).sendAndReceive+0x131		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:963
#	0x75538e	github.com/Shopify/sarama.(*Broker).Fetch+0x4e				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:454
#	0x76cc86	github.com/Shopify/sarama.(*brokerConsumer).fetchNewMessages+0x686	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer.go:1077
#	0x76b604	github.com/Shopify/sarama.(*brokerConsumer).subscriptionConsumer+0xe4	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer.go:929
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x448052 0x76b3e5 0x7a639e 0x468c21
#	0x76b3e4	github.com/Shopify/sarama.(*brokerConsumer).subscriptionManager+0x264	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer.go:896
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

2 @ 0x4383b6 0x405565 0x40511d 0x74d7b8 0x7a639e 0x468c21
#	0x74d7b7	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x6f7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:606
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43



2 @ 0x4383b6 0x40640c 0x405e78 0x7a4dfc 0x7a639e 0x468c21
#	0x7a4dfb	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x9b	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:130
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43


1 @ 0x4383b6 0x405236 0x40511d 0x7a4ded 0x7a639e 0x468c21
#	0x7a4dec	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x8c	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:132
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43


1 @ 0x4383b6 0x40640c 0x405e38 0x76d59f 0xd9c617 0x468c21
#	0x76d59e	github.com/Shopify/sarama.(*consumerGroup).Consume+0x27e			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:205
#	0xd9c616	git.tigerbrokers.net/astro/campaign/pkg/xkafka.ListenTopicByGroup.func1+0x176	/Users/tiger/data/code/campaign/pkg/xkafka/consumer.go:65


1 @ 0x4383b6 0x40640c 0x405e78 0x74c934 0x7a639e 0x468c21
#	0x74c933	github.com/Shopify/sarama.(*topicProducer).dispatch+0x53	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:413
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43


1 @ 0x4383b6 0x448052 0x76fa06 0x468c21
#	0x76fa05	github.com/Shopify/sarama.(*consumerGroup).loopCheckPartitionNumbers+0x225	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:547

1 @ 0x4383b6 0x448052 0x77199a 0x468c21
#	0x771999	github.com/Shopify/sarama.(*consumerGroupSession).heartbeatLoop+0x459	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/consumer_group.go:879

1 @ 0x4383b6 0x448052 0x7934d8 0x7a639e 0x468c21
#	0x7934d7	github.com/Shopify/sarama.(*offsetManager).mainLoop+0xf7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/offset_manager.go:236
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

Below is the sarama log
sarama-hang.txt

@dnwe
Copy link
Collaborator

dnwe commented Feb 10, 2022

@hxiaodon aside, unrelated to this issue, but from your log you might want to tune your consumer config (e.g., MaxProcessingTime) as it looks like your consumers are frequently taking too long to read from the Messages channel and your subscriptions are being abandoned

@dnwe
Copy link
Collaborator

dnwe commented Feb 10, 2022

@hxiaodon also, is that sarama-hang.txt merged output from multiple processes, or just a single application?

I'm just trying to narrow things down for a recreate attempt, but I noticed from the output that it seems to suggest you're connecting to three different 3-broker clusters here?

client/brokers registered new broker #1 at 10.7.12.101:9092
client/brokers registered new broker #2 at 10.7.12.102:9092
client/brokers registered new broker #3 at 10.7.12.103:9092

client/brokers registered new broker #1 at 10.7.22.79:9092
client/brokers registered new broker #2 at 10.7.22.80:9092
client/brokers registered new broker #3 at 10.7.22.81:9092

client/brokers registered new broker #1 at 172.17.133.191:9092
client/brokers registered new broker #2 at 172.17.133.192:9092
client/brokers registered new broker #3 at 172.17.133.193:9092

@hxiaodon
Copy link

hxiaodon commented Feb 10, 2022

aside

Yeah~, thanks for reminding, it's another story which I will solve

@hxiaodon also, is that sarama-hang.txt merged output from multiple processes, or just a single application?

I'm just trying to narrow things down for a recreate attempt, but I noticed from the output that it seems to suggest you're connecting to three different 3-broker clusters here?

client/brokers registered new broker #1 at 10.7.12.101:9092 client/brokers registered new broker #2 at 10.7.12.102:9092 client/brokers registered new broker #3 at 10.7.12.103:9092

client/brokers registered new broker #1 at 10.7.22.79:9092 client/brokers registered new broker #2 at 10.7.22.80:9092 client/brokers registered new broker #3 at 10.7.22.81:9092

client/brokers registered new broker #1 at 172.17.133.191:9092 client/brokers registered new broker #2 at 172.17.133.192:9092 client/brokers registered new broker #3 at 172.17.133.193:9092

It's a single application, my program try to access different kafka clusters, and I think the following cluster lead to the hang problem
[sarama] 2022/02/10 13:56:00 producer/broker/2 state change to [closing] because write tcp 10.7.12.21:58712->10.7.12.102:9092: write: broken pipe
[sarama] 2022/02/10 13:16:01 producer/broker/3 state change to [closing] because write tcp 10.7.12.21:35342->10.7.12.103:9092: write: broken pipe
[sarama] 2022/02/10 13:16:02 producer/broker/1 state change to [closing] because write tcp 10.7.12.21:49300->10.7.12.101:9092: write: broken pipe

My understanding is that the sarama producer side has no heartbeat(periodically metadata refresh) , and the broker connection will be closed after default 10 mins with no TCP data

@dnwe
Copy link
Collaborator

dnwe commented Feb 10, 2022

Thanks, we can simulate that in the functional tests using toxiproxy

eafzali added a commit to eafzali/sarama that referenced this pull request Feb 10, 2022
This should be released as v1.32.2 to avoid users to upgrade to these faulty versions until IBM#2133 is merged.
Closing the broker asynchronously fixes the deadlock but leads to a
race condition between opening the broker in client updateLeader.
This might result in a closed broker used by the new brokerProducer
and all produce requests will fail with ErrNotConnected.
@slaunay
Copy link
Contributor Author

slaunay commented Feb 11, 2022

Against main I can re-run go test -run '^TestAsyncProducerRetryWithReferenceOpen$' -count=4096 multiple times without a single failure, but against bugfix/broker-producer-callback-deadlock I do get not all expectations were satisfied errors on the mockbrokers with the producer Errors chan having returned i/o timeout or ErrNotConnected.

@dnwe I was able to reproduce the failure (when running a high number of count) and I was able to find the cause.
I believe the proposed fix of closing the broker asynchronously does fix the deadlock but introducing a race condition.
That is, the broker.Close() might happen after a new brokerProducer is in place (because of the retry logic is triggered just after requesting a close of the broker) and more importantly possibly after updateLeader is called.
Because updateLeader does reopen the (same) broker, if it gets closed after then all Produce requests will fail with ErrNotConnected and there is no logic to open it again (till the next updateLeader).

Here is the output of the TestAsyncProducerRetryWithReferenceOpen when it fails that way.
We get the expected EOF error when reading from the socket but instead of a successful retry, we get kafka: broker not connected:

=== RUN   TestAsyncProducerRetryWithReferenceOpen
[sarama] 2022/02/10 21:50:26 *** mockbroker/1 listening on 127.0.0.1:49507
[sarama] 2022/02/10 21:50:26 *** mockbroker/2 listening on 127.0.0.1:49510
[sarama] 2022/02/10 21:50:26 Initializing new client
[sarama] 2022/02/10 21:50:26 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/10 21:50:26 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/10 21:50:26 client/metadata fetching metadata for all topics from broker 127.0.0.1:49507
[sarama] 2022/02/10 21:50:26 *** mockbroker/1/0: connection opened
[sarama] 2022/02/10 21:50:26 Connected to broker at 127.0.0.1:49507 (unregistered)
[sarama] 2022/02/10 21:50:26 *** mockbroker/1/0: replied to *sarama.MetadataRequest with *sarama.MetadataResponse
-> (*sarama.MetadataRequest){Version:(int16)0 Topics:([]string)<nil> AllowAutoTopicCreation:(bool)false}
-> (*sarama.MetadataResponse){Version:(int16)0 ThrottleTimeMs:(int32)0 Brokers:([]*sarama.Broker)[<max>] ClusterID:(*string)<nil> ControllerID:(int32)0 Topics:([]*sarama.TopicMetadata)[<max>]}
[sarama] 2022/02/10 21:50:26 client/brokers registered new broker #2 at 127.0.0.1:49510
[sarama] 2022/02/10 21:50:26 Successfully initialized new client
[sarama] 2022/02/10 21:50:26 ClientID is the default of 'sarama', you should consider setting it to something application-specific.
[sarama] 2022/02/10 21:50:26 producer/broker/2 starting up
[sarama] 2022/02/10 21:50:26 producer/broker/2 state change to [open] on my_topic/0
[sarama] 2022/02/10 21:50:26 *** mockbroker/2/0: connection opened
[sarama] 2022/02/10 21:50:26 Connected to broker at 127.0.0.1:49510 (registered as #2)
[sarama] 2022/02/10 21:50:26 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/10 21:50:26 producer/broker/2 state change to [open] on my_topic/1
[sarama] 2022/02/10 21:50:26 *** mockbroker/2/0: replied to *sarama.ProduceRequest with *sarama.ProduceResponse
-> (*sarama.ProduceRequest){TransactionalID:(*string)<nil> RequiredAcks:(sarama.RequiredAcks)1 Timeout:(int32)10000 Version:(int16)0 records:(map[string]map[int32]sarama.Records)map[<max>]}
-> (*sarama.ProduceResponse){Blocks:(map[string]map[int32]*sarama.ProduceResponseBlock)map[<max>] Version:(int16)0 ThrottleTime:(time.Duration)0s}
[sarama] 2022/02/10 21:50:26 *** mockbroker/2/0: invalid request: err=read tcp 127.0.0.1:49510->127.0.0.1:49512: use of closed network connection, ([]uint8) <nil>
[sarama] 2022/02/10 21:50:26 *** mockbroker/2/0: connection closed, err=<nil>
[sarama] 2022/02/10 21:50:26 *** mockbroker/2: listener closed, err=accept tcp 127.0.0.1:49510: use of closed network connection
[sarama] 2022/02/10 21:50:26 *** mockbroker/2 listening on 127.0.0.1:49510
[sarama] 2022/02/10 21:50:26 producer/broker/2 state change to [closing] because EOF
[sarama] 2022/02/10 21:50:26 producer/leader/my_topic/0 state change to [retrying-1]
[sarama] 2022/02/10 21:50:26 producer/leader/my_topic/0 abandoning broker 2
[sarama] 2022/02/10 21:50:26 client/metadata fetching metadata for [my_topic] from broker 127.0.0.1:49507
[sarama] 2022/02/10 21:50:26 *** mockbroker/1/0: replied to *sarama.MetadataRequest with *sarama.MetadataResponse
-> (*sarama.MetadataRequest){Version:(int16)0 Topics:([]string)[<max>] AllowAutoTopicCreation:(bool)false}
-> (*sarama.MetadataResponse){Version:(int16)0 ThrottleTimeMs:(int32)0 Brokers:([]*sarama.Broker)[<max>] ClusterID:(*string)<nil> ControllerID:(int32)0 Topics:([]*sarama.TopicMetadata)[<max>]}
[sarama] 2022/02/10 21:50:26 Closed connection to broker 127.0.0.1:49510
[sarama] 2022/02/10 21:50:26 producer/broker/2 starting up
[sarama] 2022/02/10 21:50:26 producer/broker/2 state change to [open] on my_topic/0
[sarama] 2022/02/10 21:50:26 producer/broker/2 closing done
[sarama] 2022/02/10 21:50:26 producer/leader/my_topic/0 selected broker 2
[sarama] 2022/02/10 21:50:26 producer/leader/my_topic/0 state change to [flushing-1]
[sarama] 2022/02/10 21:50:26 producer/leader/my_topic/0 state change to [normal]
[sarama] 2022/02/10 21:50:26 producer/broker/2 state change to [closing] because kafka: broker not connected
[sarama] 2022/02/10 21:50:26 producer/broker/2 unable to close broker: kafka: broker not connected
[sarama] 2022/02/10 21:50:26 Producer shutting down.
[sarama] 2022/02/10 21:50:26 Closing Client
[sarama] 2022/02/10 21:50:26 producer/broker/2 input chan closed
[sarama] 2022/02/10 21:50:26 Closed connection to broker 127.0.0.1:49507
[sarama] 2022/02/10 21:50:26 producer/broker/2 input chan closed
[sarama] 2022/02/10 21:50:26 *** mockbroker/1/0: invalid request: err=read tcp 127.0.0.1:49507->127.0.0.1:49511: use of closed network connection, ([]uint8) <nil>
[sarama] 2022/02/10 21:50:26 *** mockbroker/1/0: connection closed, err=<nil>
[sarama] 2022/02/10 21:50:26 *** mockbroker/1: listener closed, err=accept tcp 127.0.0.1:49507: use of closed network connection
[sarama] 2022/02/10 21:50:26 producer/broker/2 shut down
[sarama] 2022/02/10 21:50:26 producer/broker/2 shut down
[sarama] 2022/02/10 21:50:26 *** mockbroker/2: listener closed, err=accept tcp 127.0.0.1:49510: use of closed network connection
--- FAIL: TestAsyncProducerRetryWithReferenceOpen (0.00s)
    async_producer_test.go:51: kafka: broker not connected
    async_producer_test.go:65: Unexpected successes 1 or errors -1
    mockbroker.go:144: mockbroker/2: not all expectations were satisfied! Still waiting on:
        (*sarama.ProduceResponse)(0xc0005b3410)({
         Blocks: (map[string]map[int32]*sarama.ProduceResponseBlock) (len=1) {
          (string) (len=8) "my_topic": (map[int32]*sarama.ProduceResponseBlock) (len=1) {
           (int32) 0: (*sarama.ProduceResponseBlock)(0xc000cfd170)({
            Err: (sarama.KError) kafka server: Not an error, why are you printing me?,
            Offset: (int64) 0,
            Timestamp: (time.Time) 0001-01-01 00:00:00 +0000 UTC,
            StartOffset: (int64) 0
           })
          }
         },
         Version: (int16) 0,
         ThrottleTime: (time.Duration) 0s
        })
        
FAIL
exit status 1
FAIL	github.com/Shopify/sarama	17.077s

So I believe that closing the broker synchronously is actually necessary to keep the existing behaviour.
And to prevent the deadlock, I changed the bridge goroutine logic to buffer responses similar to how the retryHandler works:
https://github.com/Shopify/sarama/blob/6693712f54b76066ea239255c30585832983947d/async_producer.go#L1046-L1068

The result is similar to buffering the responses channel but it does not depend on how many in flight requests are configured.

The producer hang bug is reproduced at our prod env and I think the hang point is changed to another place

@hxiaodon It is hard to tell but I don't see a broker responseReceiver goroutine blocked in your stack traces to show that the deadlock is still happening (there is one blocked on a socket read with Brocker.readFull).
What you could get (in case you hit the race condition described above) is lots of kafka: broker not connected in your applicative logs.

My understanding is that the sarama producer side has no heartbeat(periodically metadata refresh) , and the broker connection will be closed after default 10 mins with no TCP data

That's right, a broker can drop the TCP connection if it has been idle for a while (connections.max.idle.ms) or even during a simple restart.
The fastest way to detect it is to reduce the timeout so that the next write/read on the socket will notice the connection is closed.

The Client does have a backgroundMetadataUpdater goroutine that will send a request every 10 minutes by default, you can reduce the frequency with config.Metadata.RefreshFrequency.
This will maintain the connection (or prevent it from being idle for too long) but to a single broker unfortunately.
In our custom producer, we actually send an empty Metadata request to opened Brokers in order to trick the (real) broker in keeping the connection opened (just to prevent such retries that generally happen at the time you try to produce a new record).
That being said the AsyncProducer is supposed to reconnect properly through the retry logic so you should not have to worry about that.

Would you be able to test the last commit, I believe the following dependency can be used:

require (
	github.com/Shopify/sarama v1.31.1
)

replace (
	github.com/Shopify/sarama => github.com/slaunay/sarama v1.31.2-0.20220211051606-f1bc44e541ee
)

@hxiaodon
Copy link

hxiaodon commented Feb 11, 2022

@slaunay Sure, I will have a try~
I just glanced at the new hang problem when it occur before adding comments here, and did not go deep into it 😳

3 @ 0x4383b6 0x405565 0x40511d 0x74e349 0x5064fd 0x5063f4 0x74e1b6 0x74e175 0x74d5ed 0x7a639e 0x468c21
#	0x74e348	github.com/Shopify/sarama.(*partitionProducer).updateLeader.func1+0x168	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:667
#	0x5064fc	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork.func1+0x5c	/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:85
#	0x5063f3	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork+0x33		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:86
#	0x74e1b5	github.com/eapache/go-resiliency/breaker.(*Breaker).Run+0x55		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:55
#	0x74e174	github.com/Shopify/sarama.(*partitionProducer).updateLeader+0x14	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:656
#	0x74d5ec	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x52c		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:589
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x405565 0x40511d 0x751d94 0x750baa 0x79bfe2 0x74ff4f 0x74fe7f 0x74f03f 0x7a639e 0x468c21
#	0x751d93	github.com/Shopify/sarama.(*asyncProducer).returnSuccesses+0xb3		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1148
#	0x750ba9	github.com/Shopify/sarama.(*brokerProducer).handleSuccess.func1+0x469	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:962
#	0x79bfe1	github.com/Shopify/sarama.(*produceSet).eachPartition+0x101		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/produce_set.go:211
#	0x74ff4e	github.com/Shopify/sarama.(*brokerProducer).handleSuccess+0x8e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:938
#	0x74fe7e	github.com/Shopify/sarama.(*brokerProducer).handleResponse+0x3e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:926
#	0x74f03e	github.com/Shopify/sarama.(*brokerProducer).run+0x1be			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:856
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x430973 0x463209 0x4d4532 0x4d589a 0x4d5888 0x5a29e9 0x5b4d25 0x560fb4 0x7a6745 0x4cc8ba 0x756934 0x756905 0x7579b8 0x7a639e 0x468c21
#	0x463208	internal/poll.runtime_pollWait+0x88				/Users/tiger/.go/go1.17.3/src/runtime/netpoll.go:234
#	0x4d4531	internal/poll.(*pollDesc).wait+0x31				/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:84
#	0x4d5899	internal/poll.(*pollDesc).waitRead+0x259			/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:89
#	0x4d5887	internal/poll.(*FD).Read+0x247					/Users/tiger/.go/go1.17.3/src/internal/poll/fd_unix.go:167
#	0x5a29e8	net.(*netFD).Read+0x28						/Users/tiger/.go/go1.17.3/src/net/fd_posix.go:56
#	0x5b4d24	net.(*conn).Read+0x44						/Users/tiger/.go/go1.17.3/src/net/net.go:183
#	0x560fb3	bufio.(*Reader).Read+0x1b3					/Users/tiger/.go/go1.17.3/src/bufio/bufio.go:227
#	0x7a6744	github.com/Shopify/sarama.(*bufConn).Read+0x24			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:107
#	0x4cc8b9	io.ReadAtLeast+0x99						/Users/tiger/.go/go1.17.3/src/io/io.go:328
#	0x756933	io.ReadFull+0xb3						/Users/tiger/.go/go1.17.3/src/io/io.go:347
#	0x756904	github.com/Shopify/sarama.(*Broker).readFull+0x84		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:871
#	0x7579b7	github.com/Shopify/sarama.(*Broker).responseReceiver+0xf7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1047
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

2 @ 0x4383b6 0x40640c 0x405e78 0x7a4dfc 0x7a639e 0x468c21
#	0x7a4dfb	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x9b	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:130
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x4383b6 0x405236 0x40511d 0x7a4ded 0x7a639e 0x468c21
#	0x7a4dec	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x8c	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:132
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

And I' m also curious about the backtrace at that time. Anyway, I will monitor my application with your latest fix again

@hxiaodon
Copy link

hxiaodon commented Feb 12, 2022

@slaunay . After running about half a day, my application with your latest fix hang again, although I'm confused by the following goroutine, because net.Conn readtimeout is only 30s (Does it mean that the broker's readFull func is repeatedly invoked every 30s, responseReceiver's other code branch will never be executed?)

3 @ 0x4383b6 0x430973 0x463209 0x4d4532 0x4d589a 0x4d5888 0x5a4ae9 0x5b6e25 0x5630b4 0x83c745 0x4cc8ba 0x7ec934 0x7ec905 0x7ed9b8 0x83c39e 0x468c21
#	0x463208	internal/poll.runtime_pollWait+0x88				/Users/tiger/.go/go1.17.3/src/runtime/netpoll.go:234
#	0x4d4531	internal/poll.(*pollDesc).wait+0x31				/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:84
#	0x4d5899	internal/poll.(*pollDesc).waitRead+0x259			/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:89
#	0x4d5887	internal/poll.(*FD).Read+0x247					/Users/tiger/.go/go1.17.3/src/internal/poll/fd_unix.go:167
#	0x5a4ae8	net.(*netFD).Read+0x28						/Users/tiger/.go/go1.17.3/src/net/fd_posix.go:56
#	0x5b6e24	net.(*conn).Read+0x44						/Users/tiger/.go/go1.17.3/src/net/net.go:183
#	0x5630b3	bufio.(*Reader).Read+0x1b3					/Users/tiger/.go/go1.17.3/src/bufio/bufio.go:227
#	0x83c744	github.com/Shopify/sarama.(*bufConn).Read+0x24			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:107
#	0x4cc8b9	io.ReadAtLeast+0x99						/Users/tiger/.go/go1.17.3/src/io/io.go:328
#	0x7ec933	io.ReadFull+0xb3						/Users/tiger/.go/go1.17.3/src/io/io.go:347
#	0x7ec904	github.com/Shopify/sarama.(*Broker).readFull+0x84		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:871
#	0x7ed9b7	github.com/Shopify/sarama.(*Broker).responseReceiver+0xf7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1047
#	0x83c39d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

FYI, I removed my application code's backtrace from the goroutine stack, and keep other goroutines there, if you want to full goroutine include my application code , I will attach it.

hang-goroutine.log
sarama.txt

Below is another backtrace of the above application, we restarted another app with the workaround fix(it keep working and does not hang) in same consumer group, and the rebalance is triggered, another app with the workaround fix now consume all topic partitions
hang-application after restarting another app in same consumer group and trigger rebalance.txt

@slaunay
Copy link
Contributor Author

slaunay commented Feb 12, 2022

@slaunay Sure, I will have a try~ I just glanced at the new hang problem when it occur before adding comments here, and did not go deep into it 😳

3 @ 0x4383b6 0x405565 0x40511d 0x74e349 0x5064fd 0x5063f4 0x74e1b6 0x74e175 0x74d5ed 0x7a639e 0x468c21
#	0x74e348	github.com/Shopify/sarama.(*partitionProducer).updateLeader.func1+0x168	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:667
#	0x5064fc	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork.func1+0x5c	/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:85
#	0x5063f3	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork+0x33		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:86
#	0x74e1b5	github.com/eapache/go-resiliency/breaker.(*Breaker).Run+0x55		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:55
#	0x74e174	github.com/Shopify/sarama.(*partitionProducer).updateLeader+0x14	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:656
#	0x74d5ec	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x52c		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:589
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x405565 0x40511d 0x751d94 0x750baa 0x79bfe2 0x74ff4f 0x74fe7f 0x74f03f 0x7a639e 0x468c21
#	0x751d93	github.com/Shopify/sarama.(*asyncProducer).returnSuccesses+0xb3		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1148
#	0x750ba9	github.com/Shopify/sarama.(*brokerProducer).handleSuccess.func1+0x469	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:962
#	0x79bfe1	github.com/Shopify/sarama.(*produceSet).eachPartition+0x101		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/produce_set.go:211
#	0x74ff4e	github.com/Shopify/sarama.(*brokerProducer).handleSuccess+0x8e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:938
#	0x74fe7e	github.com/Shopify/sarama.(*brokerProducer).handleResponse+0x3e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:926
#	0x74f03e	github.com/Shopify/sarama.(*brokerProducer).run+0x1be			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:856
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x4383b6 0x430973 0x463209 0x4d4532 0x4d589a 0x4d5888 0x5a29e9 0x5b4d25 0x560fb4 0x7a6745 0x4cc8ba 0x756934 0x756905 0x7579b8 0x7a639e 0x468c21
#	0x463208	internal/poll.runtime_pollWait+0x88				/Users/tiger/.go/go1.17.3/src/runtime/netpoll.go:234
#	0x4d4531	internal/poll.(*pollDesc).wait+0x31				/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:84
#	0x4d5899	internal/poll.(*pollDesc).waitRead+0x259			/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:89
#	0x4d5887	internal/poll.(*FD).Read+0x247					/Users/tiger/.go/go1.17.3/src/internal/poll/fd_unix.go:167
#	0x5a29e8	net.(*netFD).Read+0x28						/Users/tiger/.go/go1.17.3/src/net/fd_posix.go:56
#	0x5b4d24	net.(*conn).Read+0x44						/Users/tiger/.go/go1.17.3/src/net/net.go:183
#	0x560fb3	bufio.(*Reader).Read+0x1b3					/Users/tiger/.go/go1.17.3/src/bufio/bufio.go:227
#	0x7a6744	github.com/Shopify/sarama.(*bufConn).Read+0x24			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:107
#	0x4cc8b9	io.ReadAtLeast+0x99						/Users/tiger/.go/go1.17.3/src/io/io.go:328
#	0x756933	io.ReadFull+0xb3						/Users/tiger/.go/go1.17.3/src/io/io.go:347
#	0x756904	github.com/Shopify/sarama.(*Broker).readFull+0x84		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:871
#	0x7579b7	github.com/Shopify/sarama.(*Broker).responseReceiver+0xf7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1047
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

2 @ 0x4383b6 0x40640c 0x405e78 0x7a4dfc 0x7a639e 0x468c21
#	0x7a4dfb	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x9b	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:130
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x4383b6 0x405236 0x40511d 0x7a4ded 0x7a639e 0x468c21
#	0x7a4dec	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x8c	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:132
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

@hxiaodon Those stack traces do not show the original issue caused by a deadlock.
There is a one Broker blocked reading from a socket which is often slow (and therefore would be captured) even if it can block for a while if the TCP connection is closed (or broken).

1 @ 0x4383b6 0x405236 0x40511d 0x7a4ded 0x7a639e 0x468c21
#	0x7a4dec	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x8c	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:132
#	0x7a639d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

suggests that a success is trying to be forwarded to an invocation of SyncerProducer.SendMessage[s] (not present in the stack traces):
https://github.com/slaunay/sarama/blob/6c70a6c230a42afe4408e54ecdf73722e415945e/sync_producer.go#L132
Again not necessarily a problem as long as successes keeps getting forward to your application.

@slaunay . After running about half a day, my application with your latest fix hang again, although I'm confused by the following goroutine, because net.Conn readtimeout is only 30s (Does it mean that the broker's readFull func is repeatedly invoked every 30s, responseReceiver's other code branch will never be executed?)

3 @ 0x4383b6 0x430973 0x463209 0x4d4532 0x4d589a 0x4d5888 0x5a4ae9 0x5b6e25 0x5630b4 0x83c745 0x4cc8ba 0x7ec934 0x7ec905 0x7ed9b8 0x83c39e 0x468c21
#	0x463208	internal/poll.runtime_pollWait+0x88				/Users/tiger/.go/go1.17.3/src/runtime/netpoll.go:234
#	0x4d4531	internal/poll.(*pollDesc).wait+0x31				/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:84
#	0x4d5899	internal/poll.(*pollDesc).waitRead+0x259			/Users/tiger/.go/go1.17.3/src/internal/poll/fd_poll_runtime.go:89
#	0x4d5887	internal/poll.(*FD).Read+0x247					/Users/tiger/.go/go1.17.3/src/internal/poll/fd_unix.go:167
#	0x5a4ae8	net.(*netFD).Read+0x28						/Users/tiger/.go/go1.17.3/src/net/fd_posix.go:56
#	0x5b6e24	net.(*conn).Read+0x44						/Users/tiger/.go/go1.17.3/src/net/net.go:183
#	0x5630b3	bufio.(*Reader).Read+0x1b3					/Users/tiger/.go/go1.17.3/src/bufio/bufio.go:227
#	0x83c744	github.com/Shopify/sarama.(*bufConn).Read+0x24			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:107
#	0x4cc8b9	io.ReadAtLeast+0x99						/Users/tiger/.go/go1.17.3/src/io/io.go:328
#	0x7ec933	io.ReadFull+0xb3						/Users/tiger/.go/go1.17.3/src/io/io.go:347
#	0x7ec904	github.com/Shopify/sarama.(*Broker).readFull+0x84		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:871
#	0x7ed9b7	github.com/Shopify/sarama.(*Broker).responseReceiver+0xf7	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1047
#	0x83c39d	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

This looks like another legitimate blocking read from the socket.
The way the (Sarama) Broker works is by having a responseReceiver goroutine that will read from the socket when we are waiting for a response (i.e. after a request has been sent to the socket).
When a responsePromise is consumed, the Broker reads from the socket for up to config.Net.ReadTimeout (30s by default) but once an error occurs the goroutine will terminate (after sending that same error to remaining responsePromise(s)).
If we were not relying on read deadline, the responseReceiver goroutine could seat for a very long time reading from a broken socket.

So reading is not something that will happens every 30 seconds per se, except in the odd case where you end up sending one Produce request to a Kafka broker exactly every 30 seconds and the response is received without latency.

Looking at the Sarama applicative logs, it seems like the retry logic works as expected and new brokerProducer after getting different types of network errors

$ grep -e producer/broker/1 -e  10.7.12.101 /Users/slaunay/Downloads/sarama.txt | grep -v "client/meta"
[sarama] 2022/02/12 05:10:19 producer/broker/1 state change to [open] on realtime/24
[sarama] 2022/02/12 05:53:34 producer/broker/1 input chan closed
[sarama] 2022/02/12 05:53:34 producer/broker/1 shut down
[sarama] 2022/02/12 05:53:34 producer/broker/1 state change to [open] on realtime/42
[sarama] 2022/02/12 05:53:34 producer/broker/1 state change to [closing] because EOF
[sarama] 2022/02/12 05:53:34 Closed connection to broker 10.7.12.101:9092
[sarama] 2022/02/12 05:53:34 producer/broker/1 state change to [closing] because EOF
[sarama] 2022/02/12 05:53:34 producer/broker/1 starting up
[sarama] 2022/02/12 05:53:34 producer/broker/1 state change to [open] on realtime/42
[sarama] 2022/02/12 05:53:34 Connected to broker at 10.7.12.101:9092 (registered as #1)
[sarama] 2022/02/12 05:55:49 producer/broker/1 state change to [open] on realtime/6
[sarama] 2022/02/12 07:26:00 producer/broker/1 state change to [open] on realtime/12
[sarama] 2022/02/12 07:26:00 producer/broker/1 state change to [closing] because write tcp 10.7.12.20:50365->10.7.12.101:9092: write: broken pipe
[sarama] 2022/02/12 07:26:00 Closed connection to broker 10.7.12.101:9092
[sarama] 2022/02/12 07:26:00 producer/broker/1 starting up
[sarama] 2022/02/12 07:26:00 producer/broker/1 state change to [open] on realtime/12
[sarama] 2022/02/12 07:26:00 Connected to broker at 10.7.12.101:9092 (registered as #1)
$ grep -e producer/broker/2 -e  10.7.12.102 /Users/slaunay/Downloads/sarama.txt | grep -v "client/meta"
[sarama] 2022/02/12 05:06:47 producer/broker/2 state change to [open] on realtime/37
[sarama] 2022/02/12 06:36:02 producer/broker/2 state change to [closing] because write tcp 10.7.12.20:56716->10.7.12.102:9092: write: broken pipe
[sarama] 2022/02/12 06:36:02 Closed connection to broker 10.7.12.102:9092
[sarama] 2022/02/12 06:36:02 producer/broker/2 starting up
[sarama] 2022/02/12 06:36:02 producer/broker/2 state change to [open] on realtime/19
[sarama] 2022/02/12 06:36:02 Connected to broker at 10.7.12.102:9092 (registered as #2)
$ grep -e producer/broker/3 -e  10.7.12.103 /Users/slaunay/Downloads/sarama.txt | grep -v "client/meta"
[sarama] 2022/02/12 05:14:35 producer/broker/3 state change to [open] on realtime/47
[sarama] 2022/02/12 05:46:29 producer/broker/3 state change to [open] on realtime/41
[sarama] 2022/02/12 05:46:29 producer/broker/3 state change to [closing] because write tcp 10.7.12.20:43544->10.7.12.103:9092: write: broken pipe
[sarama] 2022/02/12 05:46:29 Closed connection to broker 10.7.12.103:9092
[sarama] 2022/02/12 05:46:29 producer/broker/3 starting up
[sarama] 2022/02/12 05:46:29 producer/broker/3 state change to [open] on realtime/41
[sarama] 2022/02/12 05:46:29 Connected to broker at 10.7.12.103:9092 (registered as #3)
[sarama] 2022/02/12 06:09:23 producer/broker/3 state change to [open] on realtime/2
[sarama] 2022/02/12 06:09:23 producer/broker/3 state change to [closing] because write tcp 10.7.12.20:33872->10.7.12.103:9092: write: broken pipe
[sarama] 2022/02/12 06:09:23 Closed connection to broker 10.7.12.103:9092
[sarama] 2022/02/12 06:09:23 producer/broker/3 state change to [closing] because kafka: broker not connected
[sarama] 2022/02/12 06:09:23 producer/broker/3 starting up
[sarama] 2022/02/12 06:09:23 producer/broker/3 state change to [open] on realtime/2
[sarama] 2022/02/12 06:09:23 producer/broker/3 input chan closed
[sarama] 2022/02/12 06:09:23 Connected to broker at 10.7.12.103:9092 (registered as #3)
license:[sarama] 2022/02/12 06:09:23 producer/broker/3 shut down
[sarama] 2022/02/12 06:09:24 producer/broker/3 starting up
[sarama] 2022/02/12 06:09:24 producer/broker/3 state change to [open] on realtime/2

Below is another backtrace of the above application, we restarted another app with the workaround fix(it keep working and does not hang) in same consumer group, and the rebalance is triggered, another app with the workaround fix now consume all topic partitions hang-application after restarting another app in same consumer group and trigger rebalance.txt

It's not clear to me if what you are saying is that the issue does not occur anymore with the latest fix applied (f1bc44e).
It seems like you are saying that the same application on a different node (and part of the same consumer group) is working and not hanging and also consuming from all topic partitions.
Yet you provide a stack trace with hang in the name hence the confusion 😉.

But here is what I can see from the capture (similar to the stack trace above):

One partitionProducer is waiting on a syn message to be sent to a new brokerProducer:

5 @ 0x4383b6 0x405565 0x40511d 0x7e43a9 0x6cd65d 0x6cd554 0x7e4216 0x7e41d5 0x7e364d 0x83c39e 0x468c21
#	0x7e43a8	github.com/Shopify/sarama.(*partitionProducer).updateLeader.func1+0x168	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:667
#	0x6cd65c	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork.func1+0x5c	/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:85
#	0x6cd553	github.com/eapache/go-resiliency/breaker.(*Breaker).doWork+0x33		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:86
#	0x7e4215	github.com/eapache/go-resiliency/breaker.(*Breaker).Run+0x55		/Users/tiger/go/pkg/mod/github.com/eapache/[email protected]/breaker/breaker.go:55
#	0x7e41d4	github.com/Shopify/sarama.(*partitionProducer).updateLeader+0x14	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:656
#	0x7e364c	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x52c		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:589
#	0x83c39d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

One brokerProducer possibly the same one is trying to send a success:

3 @ 0x4383b6 0x405565 0x40511d 0x7e7d94 0x7e6baa 0x831fe2 0x7e5f4f 0x7e5e7f 0x7e507f 0x83c39e 0x468c21
#	0x7e7d93	github.com/Shopify/sarama.(*asyncProducer).returnSuccesses+0xb3		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1155
#	0x7e6ba9	github.com/Shopify/sarama.(*brokerProducer).handleSuccess.func1+0x469	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:974
#	0x831fe1	github.com/Shopify/sarama.(*produceSet).eachPartition+0x101		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/produce_set.go:211
#	0x7e5f4e	github.com/Shopify/sarama.(*brokerProducer).handleSuccess+0x8e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:950
#	0x7e5e7e	github.com/Shopify/sarama.(*brokerProducer).handleResponse+0x3e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:938
#	0x7e507e	github.com/Shopify/sarama.(*brokerProducer).run+0x1be			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:872
#	0x83c39d	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

This synchronization would explain why the syn message is not yet received by the brokerProducer run goroutine as it is busy sending a success (but once this is done, it will be back to the select).
Because we are capturing the stack trace at that particular point it time, it might just be part the regular flow and not an issue.

It's really difficult to know for sure if what you are experiencing is linked to the deadlock we are trying to fix as the AsyncProducer logic is fairly complex.
Plus with network errors, concurrency and the (slow) retry logic, it's tricky to interpret those stack traces.
One way to know if what you are seeing now is cause by the application would be to test with v1.30.1 and see if what you see happens with that version too.

@eafzali Would you be able to test the fix in your application to know if your producer recovers properly when you run the scenario described in #2129 now?

The same replace should work:

replace (
	github.com/Shopify/sarama => github.com/slaunay/sarama v1.31.2-0.20220211051606-f1bc44e541ee
)

@hxiaodon
Copy link

Because we are capturing the stack trace at that particular point it time, it might just be part the regular flow and not an issue.

Yeah, I think we did not reach an agreement on it 😅 , and your detailed explanation for my confusion sounds reasonable. I will have a try with 1.30.1 at my prod env and with v1.31.2-0.20220211051606-f1bc44e541e fix at my local env.

@dnwe
Copy link
Collaborator

dnwe commented Feb 13, 2022

@slaunay I'd suggest we go ahead and merge this PR now and we can follow-up any further problems on the individual issues? Thanks again for all your hard work tracking down the problems

@dnwe dnwe merged commit 06513c1 into IBM:main Feb 13, 2022
@eafzali
Copy link

eafzali commented Feb 14, 2022

Thank you @slaunay, I can confirm that it has fixed the issue I reported.

@hxiaodon
Copy link

hxiaodon commented Feb 15, 2022

@slaunay
with my local laptop(mac OS 10.15.7), I think I produced the hang problem again :). I suppose that your fix do work and cover many scenarios which make hang problem's occurring more and more harder
Reproducing Steps:
Install kafka cluster(kafka 2.8.1) locally(3 zk nodes and 3 broker nodes), the critical config I changed is:

connections.max.idle.ms=120000

Full kafka config is here

server_conf.zip

kafka starting command is:

bin/kafka-server-start.sh config/server-1.properties
bin/kafka-server-start.sh config/server-2.properties
bin/kafka-server-start.sh config/server-3.properties

create a topic with 48 partitions

bin/kafka-topics.sh --create --bootstrap-server localhost:9092 --replication-factor 3 --partitions 48 --topic my-test-hang-topic

Now all infra things are ready, and it's time to prepare the producer code, it's also very simple, core logic is no more than 20 lines(feel free to ignore the Chinese comments since I copy and paste some of them roughly from the online application code )
producer-hxiaodon.zip
I think the critical config is in producer.go

	cfg.Producer.Return.Successes = true
	cfg.Metadata.RefreshFrequency = 2 * time.Minute

I started the program at 2022/02/14 19:16:29 ,and the producer hang at 2022-02-14 20:54:23 because I could not get any message with following command:

bin/kafka-console-consumer.sh --bootstrap-server localhost:9092  --property print.key=true --topic my-test-hang-topic --from-beginning 

This is the full program log which include the sarama output
program.log.zip

At the hang point, I found that the kafka shell get a {null null} value

QugFBShlicKZabRvLlcFUsaGgDoNjkQJ	"testing 0 at 2022-02-14 20:54:23.138817 +0800 CST m=+5700.010782811 for routine 2"
null	null
SKBZRYxjublXKHlouWaZkggfQbNxcpLI	"testing 0 at 2022-02-14 20:54:23.138742 +0800 CST m=+5700.010708020 for routine 0"

Full logs are here
topic content.log.zip

And this time the full backtrace is very short

goroutine profile: total 90
48 @ 0x1038d16 0x10077cc 0x1007238 0x12fca91 0x132e9de 0x10688a1
#	0x12fca90	github.com/Shopify/sarama.(*partitionProducer).dispatch+0x1b0	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:546
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

8 @ 0x1038d16 0x10077cc 0x10071f8 0x132d910 0x1334d50 0x1334a0c 0x10688a1
#	0x132d90f	github.com/Shopify/sarama.(*syncProducer).SendMessage+0x8f	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:96
#	0x1334d4f	main.(*SyncProducer).SendMessage+0x22f				/Users/tiger/Downloads/working/kafka-cluster/producer/producer.go:41
#	0x1334a0b	main.main.func3+0x1cb						/Users/tiger/Downloads/working/kafka-cluster/producer/main.go:59

6 @ 0x1038d16 0x10077cc 0x1007238 0x12fe365 0x132e9de 0x10688a1
#	0x12fe364	github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func1+0x64	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:698
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

4 @ 0x1038d16 0x10077cc 0x1007238 0x12fe12c 0x132e9de 0x10688a1
#	0x12fe12b	github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func2+0xcb	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:744
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

4 @ 0x1038d16 0x10077cc 0x1007238 0x1304bd4 0x132e9de 0x10688a1
#	0x1304bd3	github.com/Shopify/sarama.(*Broker).responseReceiver+0x73	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/broker.go:1035
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x1038d16 0x1006925 0x10064dd 0x1301574 0x130038a 0x1326142 0x12ff72f 0x12ff65f 0x12fe85f 0x132e9de 0x10688a1
#	0x1301573	github.com/Shopify/sarama.(*asyncProducer).returnSuccesses+0xb3		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1155
#	0x1300389	github.com/Shopify/sarama.(*brokerProducer).handleSuccess.func1+0x469	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:974
#	0x1326141	github.com/Shopify/sarama.(*produceSet).eachPartition+0x101		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/produce_set.go:211
#	0x12ff72e	github.com/Shopify/sarama.(*brokerProducer).handleSuccess+0x8e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:950
#	0x12ff65e	github.com/Shopify/sarama.(*brokerProducer).handleResponse+0x3e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:938
#	0x12fe85e	github.com/Shopify/sarama.(*brokerProducer).run+0x1be			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:872
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

3 @ 0x1038d16 0x1048812 0x12fe834 0x132e9de 0x10688a1
#	0x12fe833	github.com/Shopify/sarama.(*brokerProducer).run+0x193	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:806
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

2 @ 0x1038d16 0x1048812 0x12fe206 0x132e9de 0x10688a1
#	0x12fe205	github.com/Shopify/sarama.(*asyncProducer).newBrokerProducer.func2+0x1a5	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:755
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d					/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x10065f6 0x10064dd 0x132da0d 0x132e9de 0x10688a1
#	0x132da0c	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x8c	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:132
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x10077cc 0x10071f8 0x12623ea 0x10688a1
#	0x12623e9	github.com/rcrowley/go-metrics.(*meterArbiter).tick+0x29	/Users/tiger/go/pkg/mod/github.com/rcrowley/[email protected]/meter.go:239

1 @ 0x1038d16 0x10077cc 0x10071f8 0x1300f3f 0x132e9de 0x10688a1
#	0x1300f3e	github.com/Shopify/sarama.(*asyncProducer).retryHandler+0x19e	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1090
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x10077cc 0x1007238 0x12fb9d2 0x132e9de 0x10688a1
#	0x12fb9d1	github.com/Shopify/sarama.(*asyncProducer).dispatcher+0xd1	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:331
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x10077cc 0x1007238 0x12fc154 0x132e9de 0x10688a1
#	0x12fc153	github.com/Shopify/sarama.(*topicProducer).dispatch+0x53	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:413
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x10077cc 0x1007238 0x132db85 0x132e9de 0x10688a1
#	0x132db84	github.com/Shopify/sarama.(*syncProducer).handleErrors+0xa4	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:138
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x1031873 0x10630a9 0x10af952 0x10b04fa 0x10b04e8 0x114ee69 0x115b005 0x11f9f6d 0x1116063 0x1116bcf 0x1116e27 0x11b1bd9 0x11f6b59 0x11f6b5a 0x11fb325 0x11ff645 0x10688a1
#	0x10630a8	internal/poll.runtime_pollWait+0x88		/usr/local/go/src/runtime/netpoll.go:234
#	0x10af951	internal/poll.(*pollDesc).wait+0x31		/usr/local/go/src/internal/poll/fd_poll_runtime.go:84
#	0x10b04f9	internal/poll.(*pollDesc).waitRead+0x259	/usr/local/go/src/internal/poll/fd_poll_runtime.go:89
#	0x10b04e7	internal/poll.(*FD).Read+0x247			/usr/local/go/src/internal/poll/fd_unix.go:167
#	0x114ee68	net.(*netFD).Read+0x28				/usr/local/go/src/net/fd_posix.go:56
#	0x115b004	net.(*conn).Read+0x44				/usr/local/go/src/net/net.go:183
#	0x11f9f6c	net/http.(*connReader).Read+0x16c		/usr/local/go/src/net/http/server.go:780
#	0x1116062	bufio.(*Reader).fill+0x102			/usr/local/go/src/bufio/bufio.go:101
#	0x1116bce	bufio.(*Reader).ReadSlice+0x2e			/usr/local/go/src/bufio/bufio.go:360
#	0x1116e26	bufio.(*Reader).ReadLine+0x26			/usr/local/go/src/bufio/bufio.go:389
#	0x11b1bd8	net/textproto.(*Reader).readLineSlice+0x98	/usr/local/go/src/net/textproto/reader.go:57
#	0x11f6b58	net/textproto.(*Reader).ReadLine+0x78		/usr/local/go/src/net/textproto/reader.go:38
#	0x11f6b59	net/http.readRequest+0x79			/usr/local/go/src/net/http/request.go:1029
#	0x11fb324	net/http.(*conn).readRequest+0x224		/usr/local/go/src/net/http/server.go:966
#	0x11ff644	net/http.(*conn).serve+0x864			/usr/local/go/src/net/http/server.go:1856

1 @ 0x1038d16 0x1031873 0x10630a9 0x10af952 0x10b13cc 0x10b13b9 0x1150215 0x1160b08 0x115fe9d 0x1203734 0x120335d 0x1334fc5 0x1334f95 0x10688a1
#	0x10630a8	internal/poll.runtime_pollWait+0x88		/usr/local/go/src/runtime/netpoll.go:234
#	0x10af951	internal/poll.(*pollDesc).wait+0x31		/usr/local/go/src/internal/poll/fd_poll_runtime.go:84
#	0x10b13cb	internal/poll.(*pollDesc).waitRead+0x22b	/usr/local/go/src/internal/poll/fd_poll_runtime.go:89
#	0x10b13b8	internal/poll.(*FD).Accept+0x218		/usr/local/go/src/internal/poll/fd_unix.go:402
#	0x1150214	net.(*netFD).accept+0x34			/usr/local/go/src/net/fd_unix.go:173
#	0x1160b07	net.(*TCPListener).accept+0x27			/usr/local/go/src/net/tcpsock_posix.go:140
#	0x115fe9c	net.(*TCPListener).Accept+0x3c			/usr/local/go/src/net/tcpsock.go:262
#	0x1203733	net/http.(*Server).Serve+0x393			/usr/local/go/src/net/http/server.go:3002
#	0x120335c	net/http.(*Server).ListenAndServe+0x7c		/usr/local/go/src/net/http/server.go:2931
#	0x1334fc4	net/http.ListenAndServe+0x44			/usr/local/go/src/net/http/server.go:3185
#	0x1334f94	main.main.func1+0x14				/Users/tiger/Downloads/working/kafka-cluster/producer/main.go:30

1 @ 0x1038d16 0x1048812 0x130c38e 0x132e9de 0x10688a1
#	0x130c38d	github.com/Shopify/sarama.(*client).backgroundMetadataUpdater+0x12d	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/client.go:824
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

1 @ 0x1038d16 0x104990c 0x10498e6 0x10645c5 0x1070a31 0x13345e8 0x1038947 0x10688a1
#	0x10645c4	sync.runtime_Semacquire+0x24	/usr/local/go/src/runtime/sema.go:56
#	0x1070a30	sync.(*WaitGroup).Wait+0x70	/usr/local/go/src/sync/waitgroup.go:130
#	0x13345e7	main.main+0x1c7			/Users/tiger/Downloads/working/kafka-cluster/producer/main.go:66
#	0x1038946	runtime.main+0x226		/usr/local/go/src/runtime/proc.go:255

1 @ 0x1062ce5 0x1245095 0x1244ead 0x124202b 0x1250c3a 0x12517ee 0x120058f 0x1201e89 0x12031bb 0x11ff8e8 0x10688a1
#	0x1062ce4	runtime/pprof.runtime_goroutineProfileWithLabels+0x24	/usr/local/go/src/runtime/mprof.go:746
#	0x1245094	runtime/pprof.writeRuntimeProfile+0xb4			/usr/local/go/src/runtime/pprof/pprof.go:724
#	0x1244eac	runtime/pprof.writeGoroutine+0x4c			/usr/local/go/src/runtime/pprof/pprof.go:684
#	0x124202a	runtime/pprof.(*Profile).WriteTo+0x14a			/usr/local/go/src/runtime/pprof/pprof.go:331
#	0x1250c39	net/http/pprof.handler.ServeHTTP+0x499			/usr/local/go/src/net/http/pprof/pprof.go:253
#	0x12517ed	net/http/pprof.Index+0x12d				/usr/local/go/src/net/http/pprof/pprof.go:371
#	0x120058e	net/http.HandlerFunc.ServeHTTP+0x2e			/usr/local/go/src/net/http/server.go:2047
#	0x1201e88	net/http.(*ServeMux).ServeHTTP+0x148			/usr/local/go/src/net/http/server.go:2425
#	0x12031ba	net/http.serverHandler.ServeHTTP+0x43a			/usr/local/go/src/net/http/server.go:2879
#	0x11ff8e7	net/http.(*conn).serve+0xb07				/usr/local/go/src/net/http/server.go:1930

1 @ 0x10688a1

Hope for your reply, thank!

@slaunay
Copy link
Contributor Author

slaunay commented Feb 15, 2022

Thank you @slaunay, I can confirm that it has fixed the issue I reported.

Thanks a lot @eafzali for testing the fix.

@slaunay
with my local laptop(mac OS 10.15.7), I think I produced the hang problem again :). I suppose that your fix do work and cover many scenarios which make hang problem's occurring more and more harder

Thanks for providing those details.
I am not sure if this is linked to the current issue (deadlock regression) but something is not right indeed and we should probably create another issue for that particular scenario.

Here are some things I found:

  • the brokerProducer is blocked sending a success:
3 @ 0x1038d16 0x1006925 0x10064dd 0x1301574 0x130038a 0x1326142 0x12ff72f 0x12ff65f 0x12fe85f 0x132e9de 0x10688a1
#	0x1301573	github.com/Shopify/sarama.(*asyncProducer).returnSuccesses+0xb3		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:1155
#	0x1300389	github.com/Shopify/sarama.(*brokerProducer).handleSuccess.func1+0x469	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:974
#	0x1326141	github.com/Shopify/sarama.(*produceSet).eachPartition+0x101		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/produce_set.go:211
#	0x12ff72e	github.com/Shopify/sarama.(*brokerProducer).handleSuccess+0x8e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:950
#	0x12ff65e	github.com/Shopify/sarama.(*brokerProducer).handleResponse+0x3e		/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:938
#	0x12fe85e	github.com/Shopify/sarama.(*brokerProducer).run+0x1be			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/async_producer.go:872
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d				/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

https://github.com/Shopify/sarama/blob/f1bc44e541eecf45f935b97db6a457740aaa073e/async_producer.go#L1155

  • the syncProducer successes goroutine is blocked forwarding a success (and therefore blocking the brokerProducer):
1 @ 0x1038d16 0x10065f6 0x10064dd 0x132da0d 0x132e9de 0x10688a1
#	0x132da0c	github.com/Shopify/sarama.(*syncProducer).handleSuccesses+0x8c	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:132
#	0x132e9dd	github.com/Shopify/sarama.withRecover+0x3d			/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/utils.go:43

https://github.com/Shopify/sarama/blob/f1bc44e541eecf45f935b97db6a457740aaa073e/sync_producer.go#L132

  • one producer goroutine is blocked waiting for the current success/error:
8 @ 0x1038d16 0x10077cc 0x10071f8 0x132d910 0x1334d50 0x1334a0c 0x10688a1
#	0x132d90f	github.com/Shopify/sarama.(*syncProducer).SendMessage+0x8f	/Users/tiger/go/pkg/mod/github.com/slaunay/[email protected]/sync_producer.go:96
#	0x1334d4f	main.(*SyncProducer).SendMessage+0x22f				/Users/tiger/Downloads/working/kafka-cluster/producer/producer.go:41
#	0x1334a0b	main.main.func3+0x1cb						/Users/tiger/Downloads/working/kafka-cluster/producer/main.go:59

https://github.com/Shopify/sarama/blob/f1bc44e541eecf45f935b97db6a457740aaa073e/sync_producer.go#L96

Now what is really interesting is that the expectation field on a ProducerMessage used by the syncProducer is a channel that is always buffered with a capacity of 1.
So it should never block the the syncProducer successes goroutine in theory but it seems to be the case.

The null key null value record you see in the topic makes me think that:

  • a mostly "empty" ProducerMessage ends up being sent to the remote broker
  • the same message with a nil expectation field ends up being sent as a success to syncProducer successes goroutine
  • sending to a nil channel blocks forever therefore blocking the brokerProducer and preventing more records to be produced

Now such "empty" ProducerMessage used by the AsyncProducer can be:

As fin messages are used during retries, it might the root cause of the hanging if somehow they escape the AsyncProducer and ends up on the broker and in a success channel.

It would be great to confirm this is the case and ideally have a simple unit test for that scenario.
If you reduce connections.max.idle.ms and Net.ReadTimeout you might be able to reproduce it faster.

@hxiaodon Would you mind creating another issue?

@hxiaodon
Copy link

@slaunay Sure, I will create another issue to track this problem
Yesterday I planed to reproduce the hang problem again with binary created by build option "-gcflags='-N -l'"(since first time when attaching the hang process with dlv tool, I found that some goroutines' local var could not be monitored), and the new application now has been running for about 20 hours without any problem. I'm wondering maybe the compiler's optimization lead to the hang problem's happening somehow easier

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants