mirror of
https://github.com/redis/go-redis.git
synced 2025-07-28 06:42:00 +03:00
Rework ReceiveMessage
This commit is contained in:
219
pubsub.go
219
pubsub.go
@ -2,7 +2,6 @@ package redis
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -11,11 +10,11 @@ import (
|
||||
)
|
||||
|
||||
// PubSub implements Pub/Sub commands as described in
|
||||
// http://redis.io/topics/pubsub. It's NOT safe for concurrent use by
|
||||
// multiple goroutines.
|
||||
// http://redis.io/topics/pubsub. Message receiving is NOT safe
|
||||
// for concurrent use by multiple goroutines.
|
||||
//
|
||||
// PubSub automatically resubscribes to the channels and patterns
|
||||
// when Redis becomes unavailable.
|
||||
// PubSub automatically reconnects to Redis Server and resubscribes
|
||||
// to the channels in case of network errors.
|
||||
type PubSub struct {
|
||||
opt *Options
|
||||
|
||||
@ -27,13 +26,21 @@ type PubSub struct {
|
||||
channels map[string]struct{}
|
||||
patterns map[string]struct{}
|
||||
closed bool
|
||||
exit chan struct{}
|
||||
|
||||
cmd *Cmd
|
||||
|
||||
pingOnce sync.Once
|
||||
ping chan struct{}
|
||||
|
||||
chOnce sync.Once
|
||||
ch chan *Message
|
||||
}
|
||||
|
||||
func (c *PubSub) init() {
|
||||
c.exit = make(chan struct{})
|
||||
}
|
||||
|
||||
func (c *PubSub) conn() (*pool.Conn, error) {
|
||||
c.mu.Lock()
|
||||
cn, err := c._conn(nil)
|
||||
@ -66,31 +73,36 @@ func (c *PubSub) _conn(channels []string) (*pool.Conn, error) {
|
||||
|
||||
func (c *PubSub) resubscribe(cn *pool.Conn) error {
|
||||
var firstErr error
|
||||
|
||||
if len(c.channels) > 0 {
|
||||
channels := make([]string, len(c.channels))
|
||||
i := 0
|
||||
for channel := range c.channels {
|
||||
channels[i] = channel
|
||||
i++
|
||||
}
|
||||
if err := c._subscribe(cn, "subscribe", channels...); err != nil && firstErr == nil {
|
||||
channels := mapKeys(c.channels)
|
||||
err := c._subscribe(cn, "subscribe", channels...)
|
||||
if err != nil && firstErr == nil {
|
||||
firstErr = err
|
||||
}
|
||||
}
|
||||
|
||||
if len(c.patterns) > 0 {
|
||||
patterns := make([]string, len(c.patterns))
|
||||
i := 0
|
||||
for pattern := range c.patterns {
|
||||
patterns[i] = pattern
|
||||
i++
|
||||
}
|
||||
if err := c._subscribe(cn, "psubscribe", patterns...); err != nil && firstErr == nil {
|
||||
patterns := mapKeys(c.patterns)
|
||||
err := c._subscribe(cn, "psubscribe", patterns...)
|
||||
if err != nil && firstErr == nil {
|
||||
firstErr = err
|
||||
}
|
||||
}
|
||||
|
||||
return firstErr
|
||||
}
|
||||
|
||||
func mapKeys(m map[string]struct{}) []string {
|
||||
s := make([]string, len(m))
|
||||
i := 0
|
||||
for k := range m {
|
||||
s[i] = k
|
||||
i++
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (c *PubSub) _subscribe(cn *pool.Conn, redisCmd string, channels ...string) error {
|
||||
args := make([]interface{}, 1+len(channels))
|
||||
args[0] = redisCmd
|
||||
@ -114,16 +126,30 @@ func (c *PubSub) _releaseConn(cn *pool.Conn, err error) {
|
||||
return
|
||||
}
|
||||
if internal.IsBadConn(err, true) {
|
||||
_ = c.closeTheCn()
|
||||
c._reconnect()
|
||||
}
|
||||
}
|
||||
|
||||
func (c *PubSub) closeTheCn() error {
|
||||
err := c.closeConn(c.cn)
|
||||
c.cn = nil
|
||||
func (c *PubSub) _closeTheCn() error {
|
||||
var err error
|
||||
if c.cn != nil {
|
||||
err = c.closeConn(c.cn)
|
||||
c.cn = nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *PubSub) reconnect() {
|
||||
c.mu.Lock()
|
||||
c._reconnect()
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
func (c *PubSub) _reconnect() {
|
||||
_ = c._closeTheCn()
|
||||
_, _ = c._conn(nil)
|
||||
}
|
||||
|
||||
func (c *PubSub) Close() error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
@ -132,17 +158,18 @@ func (c *PubSub) Close() error {
|
||||
return pool.ErrClosed
|
||||
}
|
||||
c.closed = true
|
||||
close(c.exit)
|
||||
|
||||
if c.cn != nil {
|
||||
return c.closeTheCn()
|
||||
}
|
||||
return nil
|
||||
err := c._closeTheCn()
|
||||
return err
|
||||
}
|
||||
|
||||
// Subscribe the client to the specified channels. It returns
|
||||
// empty subscription if there are no channels.
|
||||
func (c *PubSub) Subscribe(channels ...string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
err := c.subscribe("subscribe", channels...)
|
||||
if c.channels == nil {
|
||||
c.channels = make(map[string]struct{})
|
||||
@ -150,7 +177,6 @@ func (c *PubSub) Subscribe(channels ...string) error {
|
||||
for _, channel := range channels {
|
||||
c.channels[channel] = struct{}{}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
@ -158,6 +184,8 @@ func (c *PubSub) Subscribe(channels ...string) error {
|
||||
// empty subscription if there are no patterns.
|
||||
func (c *PubSub) PSubscribe(patterns ...string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
err := c.subscribe("psubscribe", patterns...)
|
||||
if c.patterns == nil {
|
||||
c.patterns = make(map[string]struct{})
|
||||
@ -165,7 +193,6 @@ func (c *PubSub) PSubscribe(patterns ...string) error {
|
||||
for _, pattern := range patterns {
|
||||
c.patterns[pattern] = struct{}{}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
@ -173,11 +200,12 @@ func (c *PubSub) PSubscribe(patterns ...string) error {
|
||||
// them if none is given.
|
||||
func (c *PubSub) Unsubscribe(channels ...string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
err := c.subscribe("unsubscribe", channels...)
|
||||
for _, channel := range channels {
|
||||
delete(c.channels, channel)
|
||||
}
|
||||
c.mu.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
@ -185,11 +213,12 @@ func (c *PubSub) Unsubscribe(channels ...string) error {
|
||||
// them if none is given.
|
||||
func (c *PubSub) PUnsubscribe(patterns ...string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
err := c.subscribe("punsubscribe", patterns...)
|
||||
for _, pattern := range patterns {
|
||||
delete(c.patterns, pattern)
|
||||
}
|
||||
c.mu.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
@ -298,7 +327,7 @@ func (c *PubSub) newMessage(reply interface{}) (interface{}, error) {
|
||||
|
||||
// ReceiveTimeout acts like Receive but returns an error if message
|
||||
// is not received in time. This is low-level API and most clients
|
||||
// should use ReceiveMessage.
|
||||
// should use ReceiveMessage instead.
|
||||
func (c *PubSub) ReceiveTimeout(timeout time.Duration) (interface{}, error) {
|
||||
if c.cmd == nil {
|
||||
c.cmd = NewCmd()
|
||||
@ -309,7 +338,7 @@ func (c *PubSub) ReceiveTimeout(timeout time.Duration) (interface{}, error) {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
cn.SetReadTimeout(readTimeout(timeout))
|
||||
cn.SetReadTimeout(timeout)
|
||||
err = c.cmd.readReply(cn)
|
||||
c.releaseConn(cn, err)
|
||||
if err != nil {
|
||||
@ -321,48 +350,28 @@ func (c *PubSub) ReceiveTimeout(timeout time.Duration) (interface{}, error) {
|
||||
|
||||
// Receive returns a message as a Subscription, Message, Pong or error.
|
||||
// See PubSub example for details. This is low-level API and most clients
|
||||
// should use ReceiveMessage.
|
||||
// should use ReceiveMessage instead.
|
||||
func (c *PubSub) Receive() (interface{}, error) {
|
||||
return c.ReceiveTimeout(0)
|
||||
}
|
||||
|
||||
// ReceiveMessage returns a Message or error ignoring Subscription or Pong
|
||||
// messages. It automatically reconnects to Redis Server and resubscribes
|
||||
// to channels in case of network errors.
|
||||
// messages. It periodically sends Ping messages to test connection health.
|
||||
func (c *PubSub) ReceiveMessage() (*Message, error) {
|
||||
return c.receiveMessage(5 * time.Second)
|
||||
}
|
||||
|
||||
func (c *PubSub) receiveMessage(timeout time.Duration) (*Message, error) {
|
||||
var errNum uint
|
||||
c.pingOnce.Do(c.initPing)
|
||||
for {
|
||||
msgi, err := c.ReceiveTimeout(timeout)
|
||||
msg, err := c.Receive()
|
||||
if err != nil {
|
||||
if !internal.IsNetworkError(err) {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
errNum++
|
||||
if errNum < 3 {
|
||||
if netErr, ok := err.(net.Error); ok && netErr.Timeout() {
|
||||
err := c.Ping()
|
||||
if err != nil {
|
||||
internal.Logf("PubSub.Ping failed: %s", err)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// 3 consequent errors - connection is broken or
|
||||
// Redis Server is down.
|
||||
// Sleep to not exceed max number of open connections.
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
continue
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Reset error number, because we received a message.
|
||||
errNum = 0
|
||||
// Any message is as good as a ping.
|
||||
select {
|
||||
case c.ping <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
|
||||
switch msg := msgi.(type) {
|
||||
switch msg := msg.(type) {
|
||||
case *Subscription:
|
||||
// Ignore.
|
||||
case *Pong:
|
||||
@ -370,30 +379,74 @@ func (c *PubSub) receiveMessage(timeout time.Duration) (*Message, error) {
|
||||
case *Message:
|
||||
return msg, nil
|
||||
default:
|
||||
return nil, fmt.Errorf("redis: unknown message: %T", msgi)
|
||||
err := fmt.Errorf("redis: unknown message: %T", msg)
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Channel returns a Go channel for concurrently receiving messages.
|
||||
// The channel is closed with PubSub. Receive or ReceiveMessage APIs
|
||||
// can not be used after channel is created.
|
||||
// The channel is closed with PubSub. Receive* APIs can not be used
|
||||
// after channel is created.
|
||||
func (c *PubSub) Channel() <-chan *Message {
|
||||
c.chOnce.Do(func() {
|
||||
c.ch = make(chan *Message, 100)
|
||||
go func() {
|
||||
for {
|
||||
msg, err := c.ReceiveMessage()
|
||||
if err != nil {
|
||||
if err == pool.ErrClosed {
|
||||
break
|
||||
}
|
||||
continue
|
||||
}
|
||||
c.ch <- msg
|
||||
}
|
||||
close(c.ch)
|
||||
}()
|
||||
})
|
||||
c.chOnce.Do(c.initChannel)
|
||||
return c.ch
|
||||
}
|
||||
|
||||
func (c *PubSub) initChannel() {
|
||||
c.ch = make(chan *Message, 100)
|
||||
go func() {
|
||||
var errCount int
|
||||
for {
|
||||
msg, err := c.ReceiveMessage()
|
||||
if err != nil {
|
||||
if err == pool.ErrClosed {
|
||||
close(c.ch)
|
||||
return
|
||||
}
|
||||
if errCount > 0 {
|
||||
time.Sleep(c.retryBackoff(errCount))
|
||||
}
|
||||
errCount++
|
||||
continue
|
||||
}
|
||||
errCount = 0
|
||||
c.ch <- msg
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (c *PubSub) initPing() {
|
||||
const timeout = 5 * time.Second
|
||||
|
||||
c.ping = make(chan struct{}, 10)
|
||||
go func() {
|
||||
timer := time.NewTimer(timeout)
|
||||
timer.Stop()
|
||||
|
||||
var hasPing bool
|
||||
for {
|
||||
timer.Reset(timeout)
|
||||
select {
|
||||
case <-c.ping:
|
||||
hasPing = true
|
||||
if !timer.Stop() {
|
||||
<-timer.C
|
||||
}
|
||||
case <-timer.C:
|
||||
if hasPing {
|
||||
hasPing = false
|
||||
_ = c.Ping()
|
||||
} else {
|
||||
c.reconnect()
|
||||
}
|
||||
case <-c.exit:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (c *PubSub) retryBackoff(attempt int) time.Duration {
|
||||
return internal.RetryBackoff(attempt, c.opt.MinRetryBackoff, c.opt.MaxRetryBackoff)
|
||||
}
|
||||
|
Reference in New Issue
Block a user