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 producer unable register when cnx closed #761

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 6 additions & 4 deletions pulsar/internal/connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,8 @@ type TLSOptions struct {
}

var (
errConnectionClosed = errors.New("connection closed")
errConnectionClosed = errors.New("connection closed")
errUnableRegisterListener = errors.New("unable register listener when con closed")
)

// ConnectionListener is a user of a connection (eg. a producer or
Expand All @@ -72,7 +73,7 @@ type Connection interface {
SendRequest(requestID uint64, req *pb.BaseCommand, callback func(*pb.BaseCommand, error))
SendRequestNoWait(req *pb.BaseCommand) error
WriteData(data Buffer)
RegisterListener(id uint64, listener ConnectionListener)
RegisterListener(id uint64, listener ConnectionListener) error
UnregisterListener(id uint64)
AddConsumeHandler(id uint64, handler ConsumerHandler)
DeleteConsumeHandler(id uint64)
Expand Down Expand Up @@ -847,17 +848,18 @@ func (c *connection) handleCloseProducer(closeProducer *pb.CommandCloseProducer)
}
}

func (c *connection) RegisterListener(id uint64, listener ConnectionListener) {
func (c *connection) RegisterListener(id uint64, listener ConnectionListener) error {
// do not add if connection is closed
if c.closed() {
c.log.Warnf("Connection closed unable register listener id=%+v", id)
return
return errUnableRegisterListener
}

c.listenersLock.Lock()
defer c.listenersLock.Unlock()

c.listeners[id] = listener
return nil
}

func (c *connection) UnregisterListener(id uint64) {
Expand Down
7 changes: 5 additions & 2 deletions pulsar/producer_partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -262,11 +262,14 @@ func (p *partitionProducer) grabCnx() error {
p.sequenceIDGenerator = &nextSequenceID
}
p._setConn(res.Cnx)
p._getConn().RegisterListener(p.producerID, p)
err = p._getConn().RegisterListener(p.producerID, p)
if err != nil {
return err
}
p.log.WithFields(log.Fields{
"cnx": res.Cnx.ID(),
"epoch": atomic.LoadUint64(&p.epoch),
}).Debug("Connected producer")
}).Info("Connected producer")

pendingItems := p.pendingQueue.ReadableSlice()
viewSize := len(pendingItems)
Expand Down