github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/nats_stream.go (about) 1 package reader 2 3 import ( 4 "context" 5 "crypto/tls" 6 "fmt" 7 "strconv" 8 "strings" 9 "sync" 10 "time" 11 12 "github.com/Jeffail/benthos/v3/internal/impl/nats/auth" 13 btls "github.com/Jeffail/benthos/v3/lib/util/tls" 14 "github.com/nats-io/nats.go" 15 16 "github.com/Jeffail/benthos/v3/lib/log" 17 "github.com/Jeffail/benthos/v3/lib/message" 18 "github.com/Jeffail/benthos/v3/lib/message/batch" 19 "github.com/Jeffail/benthos/v3/lib/metrics" 20 "github.com/Jeffail/benthos/v3/lib/types" 21 "github.com/gofrs/uuid" 22 "github.com/nats-io/stan.go" 23 ) 24 25 //------------------------------------------------------------------------------ 26 27 // NATSStreamConfig contains configuration fields for the NATSStream input type. 28 type NATSStreamConfig struct { 29 URLs []string `json:"urls" yaml:"urls"` 30 ClusterID string `json:"cluster_id" yaml:"cluster_id"` 31 ClientID string `json:"client_id" yaml:"client_id"` 32 QueueID string `json:"queue" yaml:"queue"` 33 DurableName string `json:"durable_name" yaml:"durable_name"` 34 UnsubOnClose bool `json:"unsubscribe_on_close" yaml:"unsubscribe_on_close"` 35 StartFromOldest bool `json:"start_from_oldest" yaml:"start_from_oldest"` 36 Subject string `json:"subject" yaml:"subject"` 37 MaxInflight int `json:"max_inflight" yaml:"max_inflight"` 38 AckWait string `json:"ack_wait" yaml:"ack_wait"` 39 TLS btls.Config `json:"tls" yaml:"tls"` 40 Auth auth.Config `json:"auth" yaml:"auth"` 41 42 // TODO: V4 remove this. 43 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 44 } 45 46 // NewNATSStreamConfig creates a new NATSStreamConfig with default values. 47 func NewNATSStreamConfig() NATSStreamConfig { 48 return NATSStreamConfig{ 49 URLs: []string{stan.DefaultNatsURL}, 50 ClusterID: "test-cluster", 51 ClientID: "benthos_client", 52 QueueID: "benthos_queue", 53 DurableName: "benthos_offset", 54 UnsubOnClose: false, 55 StartFromOldest: true, 56 Subject: "benthos_messages", 57 MaxInflight: 1024, 58 AckWait: "30s", 59 Batching: batch.NewPolicyConfig(), 60 TLS: btls.NewConfig(), 61 Auth: auth.New(), 62 } 63 } 64 65 //------------------------------------------------------------------------------ 66 67 // NATSStream is an input type that receives NATSStream messages. 68 type NATSStream struct { 69 urls string 70 conf NATSStreamConfig 71 ackWait time.Duration 72 73 stats metrics.Type 74 log log.Modular 75 76 unAckMsgs []*stan.Msg 77 78 stanConn stan.Conn 79 natsConn *nats.Conn 80 natsSub stan.Subscription 81 cMut sync.Mutex 82 83 msgChan chan *stan.Msg 84 interruptChan chan struct{} 85 tlsConf *tls.Config 86 } 87 88 // NewNATSStream creates a new NATSStream input type. 89 func NewNATSStream(conf NATSStreamConfig, log log.Modular, stats metrics.Type) (*NATSStream, error) { 90 if conf.ClientID == "" { 91 u4, err := uuid.NewV4() 92 if err != nil { 93 return nil, err 94 } 95 conf.ClientID = u4.String() 96 } 97 98 var ackWait time.Duration 99 if tout := conf.AckWait; len(tout) > 0 { 100 var err error 101 if ackWait, err = time.ParseDuration(tout); err != nil { 102 return nil, fmt.Errorf("failed to parse ack_wait string: %v", err) 103 } 104 } 105 106 n := NATSStream{ 107 conf: conf, 108 ackWait: ackWait, 109 stats: stats, 110 log: log, 111 msgChan: make(chan *stan.Msg), 112 interruptChan: make(chan struct{}), 113 } 114 115 close(n.msgChan) 116 n.urls = strings.Join(conf.URLs, ",") 117 var err error 118 if conf.TLS.Enabled { 119 if n.tlsConf, err = conf.TLS.Get(); err != nil { 120 return nil, err 121 } 122 } 123 124 return &n, nil 125 } 126 127 //------------------------------------------------------------------------------ 128 129 func (n *NATSStream) disconnect() { 130 n.cMut.Lock() 131 defer n.cMut.Unlock() 132 133 if n.natsSub != nil { 134 if n.conf.UnsubOnClose { 135 n.natsSub.Unsubscribe() 136 } 137 n.natsConn.Close() 138 n.stanConn.Close() 139 140 n.natsSub = nil 141 n.natsConn = nil 142 n.stanConn = nil 143 } 144 } 145 146 // Connect attempts to establish a connection to a NATS streaming server. 147 func (n *NATSStream) Connect() error { 148 return n.ConnectWithContext(context.Background()) 149 } 150 151 // ConnectWithContext attempts to establish a connection to a NATS streaming 152 // server. 153 func (n *NATSStream) ConnectWithContext(ctx context.Context) error { 154 n.cMut.Lock() 155 defer n.cMut.Unlock() 156 157 if n.natsSub != nil { 158 return nil 159 } 160 161 var opts []nats.Option 162 if n.tlsConf != nil { 163 opts = append(opts, nats.Secure(n.tlsConf)) 164 } 165 166 opts = append(opts, auth.GetOptions(n.conf.Auth)...) 167 168 natsConn, err := nats.Connect(n.urls, opts...) 169 if err != nil { 170 return err 171 } 172 173 newMsgChan := make(chan *stan.Msg) 174 handler := func(m *stan.Msg) { 175 select { 176 case newMsgChan <- m: 177 case <-n.interruptChan: 178 n.disconnect() 179 } 180 } 181 dcHandler := func() { 182 if newMsgChan == nil { 183 return 184 } 185 close(newMsgChan) 186 newMsgChan = nil 187 n.disconnect() 188 } 189 190 stanConn, err := stan.Connect( 191 n.conf.ClusterID, 192 n.conf.ClientID, 193 stan.NatsConn(natsConn), 194 stan.SetConnectionLostHandler(func(_ stan.Conn, reason error) { 195 n.log.Errorf("Connection lost: %v", reason) 196 dcHandler() 197 }), 198 ) 199 if err != nil { 200 return err 201 } 202 203 options := []stan.SubscriptionOption{ 204 stan.SetManualAckMode(), 205 } 206 if len(n.conf.DurableName) > 0 { 207 options = append(options, stan.DurableName(n.conf.DurableName)) 208 } 209 if n.conf.StartFromOldest { 210 options = append(options, stan.DeliverAllAvailable()) 211 } else { 212 options = append(options, stan.StartWithLastReceived()) 213 } 214 if n.conf.MaxInflight != 0 { 215 options = append(options, stan.MaxInflight(n.conf.MaxInflight)) 216 } 217 if n.ackWait > 0 { 218 options = append(options, stan.AckWait(n.ackWait)) 219 } 220 221 var natsSub stan.Subscription 222 if len(n.conf.QueueID) > 0 { 223 natsSub, err = stanConn.QueueSubscribe( 224 n.conf.Subject, 225 n.conf.QueueID, 226 handler, 227 options..., 228 ) 229 } else { 230 natsSub, err = stanConn.Subscribe( 231 n.conf.Subject, 232 handler, 233 options..., 234 ) 235 } 236 if err != nil { 237 natsConn.Close() 238 return err 239 } 240 241 n.natsConn = natsConn 242 n.stanConn = stanConn 243 n.natsSub = natsSub 244 n.msgChan = newMsgChan 245 n.log.Infof("Receiving NATS Streaming messages from subject: %v\n", n.conf.Subject) 246 return nil 247 } 248 249 func (n *NATSStream) read(ctx context.Context) (*stan.Msg, error) { 250 var msg *stan.Msg 251 var open bool 252 select { 253 case msg, open = <-n.msgChan: 254 if !open { 255 return nil, types.ErrNotConnected 256 } 257 case <-ctx.Done(): 258 return nil, types.ErrTimeout 259 case <-n.interruptChan: 260 n.unAckMsgs = nil 261 n.disconnect() 262 return nil, types.ErrTypeClosed 263 } 264 return msg, nil 265 } 266 267 // ReadWithContext attempts to read a new message from the NATS streaming 268 // server. 269 func (n *NATSStream) ReadWithContext(ctx context.Context) (types.Message, AsyncAckFn, error) { 270 msg, err := n.read(ctx) 271 if err != nil { 272 return nil, nil, err 273 } 274 275 bmsg := message.New([][]byte{msg.Data}) 276 bmsg.Get(0).Metadata().Set("nats_stream_subject", msg.Subject) 277 bmsg.Get(0).Metadata().Set("nats_stream_sequence", strconv.FormatUint(msg.Sequence, 10)) 278 279 return bmsg, func(rctx context.Context, res types.Response) error { 280 if res.Error() == nil { 281 return msg.Ack() 282 } 283 return nil 284 }, nil 285 } 286 287 // Read attempts to read a new message from the NATS streaming server. 288 func (n *NATSStream) Read() (types.Message, error) { 289 msg, err := n.read(context.Background()) 290 if err != nil { 291 return nil, err 292 } 293 n.unAckMsgs = append(n.unAckMsgs, msg) 294 295 bmsg := message.New([][]byte{msg.Data}) 296 bmsg.Get(0).Metadata().Set("nats_stream_subject", msg.Subject) 297 bmsg.Get(0).Metadata().Set("nats_stream_sequence", strconv.FormatUint(msg.Sequence, 10)) 298 299 return bmsg, nil 300 } 301 302 // Acknowledge instructs whether unacknowledged messages have been successfully 303 // propagated. 304 func (n *NATSStream) Acknowledge(err error) error { 305 if err == nil { 306 for _, m := range n.unAckMsgs { 307 m.Ack() 308 } 309 } 310 n.unAckMsgs = nil 311 return nil 312 } 313 314 // CloseAsync shuts down the NATSStream input and stops processing requests. 315 func (n *NATSStream) CloseAsync() { 316 close(n.interruptChan) 317 } 318 319 // WaitForClose blocks until the NATSStream input has closed down. 320 func (n *NATSStream) WaitForClose(timeout time.Duration) error { 321 n.disconnect() 322 return nil 323 } 324 325 //------------------------------------------------------------------------------