github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/nsq.go (about) 1 package reader 2 3 import ( 4 "context" 5 "crypto/tls" 6 "io" 7 llog "log" 8 "strings" 9 "sync" 10 "time" 11 12 "github.com/Jeffail/benthos/v3/lib/log" 13 "github.com/Jeffail/benthos/v3/lib/message" 14 "github.com/Jeffail/benthos/v3/lib/message/batch" 15 "github.com/Jeffail/benthos/v3/lib/metrics" 16 "github.com/Jeffail/benthos/v3/lib/types" 17 btls "github.com/Jeffail/benthos/v3/lib/util/tls" 18 nsq "github.com/nsqio/go-nsq" 19 ) 20 21 //------------------------------------------------------------------------------ 22 23 // NSQConfig contains configuration fields for the NSQ input type. 24 type NSQConfig struct { 25 Addresses []string `json:"nsqd_tcp_addresses" yaml:"nsqd_tcp_addresses"` 26 LookupAddresses []string `json:"lookupd_http_addresses" yaml:"lookupd_http_addresses"` 27 Topic string `json:"topic" yaml:"topic"` 28 Channel string `json:"channel" yaml:"channel"` 29 UserAgent string `json:"user_agent" yaml:"user_agent"` 30 TLS btls.Config `json:"tls" yaml:"tls"` 31 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 32 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 33 } 34 35 // NewNSQConfig creates a new NSQConfig with default values. 36 func NewNSQConfig() NSQConfig { 37 return NSQConfig{ 38 Addresses: []string{"localhost:4150"}, 39 LookupAddresses: []string{"localhost:4161"}, 40 Topic: "benthos_messages", 41 Channel: "benthos_stream", 42 UserAgent: "benthos_consumer", 43 TLS: btls.NewConfig(), 44 MaxInFlight: 100, 45 Batching: batch.NewPolicyConfig(), 46 } 47 } 48 49 //------------------------------------------------------------------------------ 50 51 // NSQ is an input type that receives NSQ messages. 52 type NSQ struct { 53 consumer *nsq.Consumer 54 cMut sync.Mutex 55 56 unAckMsgs []*nsq.Message 57 58 tlsConf *tls.Config 59 addresses []string 60 lookupAddresses []string 61 conf NSQConfig 62 stats metrics.Type 63 log log.Modular 64 65 internalMessages chan *nsq.Message 66 interruptChan chan struct{} 67 } 68 69 // NewNSQ creates a new NSQ input type. 70 func NewNSQ(conf NSQConfig, log log.Modular, stats metrics.Type) (*NSQ, error) { 71 n := NSQ{ 72 conf: conf, 73 stats: stats, 74 log: log, 75 internalMessages: make(chan *nsq.Message), 76 interruptChan: make(chan struct{}), 77 } 78 for _, addr := range conf.Addresses { 79 for _, splitAddr := range strings.Split(addr, ",") { 80 if len(splitAddr) > 0 { 81 n.addresses = append(n.addresses, splitAddr) 82 } 83 } 84 } 85 for _, addr := range conf.LookupAddresses { 86 for _, splitAddr := range strings.Split(addr, ",") { 87 if len(splitAddr) > 0 { 88 n.lookupAddresses = append(n.lookupAddresses, splitAddr) 89 } 90 } 91 } 92 if conf.TLS.Enabled { 93 var err error 94 if n.tlsConf, err = conf.TLS.Get(); err != nil { 95 return nil, err 96 } 97 } 98 return &n, nil 99 } 100 101 //------------------------------------------------------------------------------ 102 103 // HandleMessage handles an NSQ message. 104 func (n *NSQ) HandleMessage(message *nsq.Message) error { 105 message.DisableAutoResponse() 106 select { 107 case n.internalMessages <- message: 108 case <-n.interruptChan: 109 message.Requeue(-1) 110 message.Finish() 111 } 112 return nil 113 } 114 115 //------------------------------------------------------------------------------ 116 117 // Connect establishes a connection to an NSQ server. 118 func (n *NSQ) Connect() (err error) { 119 return n.ConnectWithContext(context.Background()) 120 } 121 122 // ConnectWithContext establishes a connection to an NSQ server. 123 func (n *NSQ) ConnectWithContext(ctx context.Context) (err error) { 124 n.cMut.Lock() 125 defer n.cMut.Unlock() 126 127 if n.consumer != nil { 128 return nil 129 } 130 131 cfg := nsq.NewConfig() 132 cfg.UserAgent = n.conf.UserAgent 133 cfg.MaxInFlight = n.conf.MaxInFlight 134 if n.tlsConf != nil { 135 cfg.TlsV1 = true 136 cfg.TlsConfig = n.tlsConf 137 } 138 139 var consumer *nsq.Consumer 140 if consumer, err = nsq.NewConsumer(n.conf.Topic, n.conf.Channel, cfg); err != nil { 141 return 142 } 143 144 consumer.SetLogger(llog.New(io.Discard, "", llog.Flags()), nsq.LogLevelError) 145 consumer.AddHandler(n) 146 147 if err = consumer.ConnectToNSQDs(n.addresses); err != nil { 148 consumer.Stop() 149 return 150 } 151 if err = consumer.ConnectToNSQLookupds(n.lookupAddresses); err != nil { 152 consumer.Stop() 153 return 154 } 155 156 n.consumer = consumer 157 n.log.Infof("Receiving NSQ messages from addresses: %s\n", n.addresses) 158 return 159 } 160 161 // disconnect safely closes a connection to an NSQ server. 162 func (n *NSQ) disconnect() error { 163 n.cMut.Lock() 164 defer n.cMut.Unlock() 165 166 if n.consumer != nil { 167 n.consumer.Stop() 168 n.consumer = nil 169 } 170 return nil 171 } 172 173 //------------------------------------------------------------------------------ 174 175 func (n *NSQ) read(ctx context.Context) (*nsq.Message, error) { 176 var msg *nsq.Message 177 select { 178 case msg = <-n.internalMessages: 179 return msg, nil 180 case <-ctx.Done(): 181 case <-n.interruptChan: 182 for _, m := range n.unAckMsgs { 183 m.Requeue(-1) 184 m.Finish() 185 } 186 n.unAckMsgs = nil 187 n.disconnect() 188 return nil, types.ErrTypeClosed 189 } 190 return nil, types.ErrTimeout 191 } 192 193 // ReadWithContext attempts to read a new message from NSQ. 194 func (n *NSQ) ReadWithContext(ctx context.Context) (types.Message, AsyncAckFn, error) { 195 msg, err := n.read(ctx) 196 if err != nil { 197 return nil, nil, err 198 } 199 n.unAckMsgs = append(n.unAckMsgs, msg) 200 return message.New([][]byte{msg.Body}), func(rctx context.Context, res types.Response) error { 201 if res.Error() != nil { 202 msg.Requeue(-1) 203 } 204 msg.Finish() 205 return nil 206 }, nil 207 } 208 209 // Read attempts to read a new message from NSQ. 210 func (n *NSQ) Read() (types.Message, error) { 211 msg, err := n.read(context.Background()) 212 if err != nil { 213 return nil, err 214 } 215 n.unAckMsgs = append(n.unAckMsgs, msg) 216 return message.New([][]byte{msg.Body}), nil 217 } 218 219 // Acknowledge instructs whether unacknowledged messages have been successfully 220 // propagated. 221 func (n *NSQ) Acknowledge(err error) error { 222 if err != nil { 223 for _, m := range n.unAckMsgs { 224 m.Requeue(-1) 225 } 226 } 227 for _, m := range n.unAckMsgs { 228 m.Finish() 229 } 230 n.unAckMsgs = nil 231 return nil 232 } 233 234 // CloseAsync shuts down the NSQ input and stops processing requests. 235 func (n *NSQ) CloseAsync() { 236 close(n.interruptChan) 237 } 238 239 // WaitForClose blocks until the NSQ input has closed down. 240 func (n *NSQ) WaitForClose(timeout time.Duration) error { 241 n.disconnect() 242 return nil 243 } 244 245 //------------------------------------------------------------------------------