github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/redis_streams.go (about) 1 package reader 2 3 import ( 4 "context" 5 "fmt" 6 "strings" 7 "sync" 8 "time" 9 10 bredis "github.com/Jeffail/benthos/v3/internal/impl/redis" 11 "github.com/Jeffail/benthos/v3/lib/log" 12 "github.com/Jeffail/benthos/v3/lib/message" 13 "github.com/Jeffail/benthos/v3/lib/message/batch" 14 "github.com/Jeffail/benthos/v3/lib/metrics" 15 "github.com/Jeffail/benthos/v3/lib/response" 16 "github.com/Jeffail/benthos/v3/lib/types" 17 "github.com/go-redis/redis/v7" 18 ) 19 20 //------------------------------------------------------------------------------ 21 22 // RedisStreamsConfig contains configuration fields for the RedisStreams input 23 // type. 24 type RedisStreamsConfig struct { 25 bredis.Config `json:",inline" yaml:",inline"` 26 BodyKey string `json:"body_key" yaml:"body_key"` 27 Streams []string `json:"streams" yaml:"streams"` 28 CreateStreams bool `json:"create_streams" yaml:"create_streams"` 29 ConsumerGroup string `json:"consumer_group" yaml:"consumer_group"` 30 ClientID string `json:"client_id" yaml:"client_id"` 31 Limit int64 `json:"limit" yaml:"limit"` 32 StartFromOldest bool `json:"start_from_oldest" yaml:"start_from_oldest"` 33 CommitPeriod string `json:"commit_period" yaml:"commit_period"` 34 Timeout string `json:"timeout" yaml:"timeout"` 35 36 // TODO: V4 remove this. 37 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 38 } 39 40 // NewRedisStreamsConfig creates a new RedisStreamsConfig with default values. 41 func NewRedisStreamsConfig() RedisStreamsConfig { 42 return RedisStreamsConfig{ 43 Config: bredis.NewConfig(), 44 BodyKey: "body", 45 Streams: []string{"benthos_stream"}, 46 CreateStreams: true, 47 ConsumerGroup: "benthos_group", 48 ClientID: "benthos_consumer", 49 Limit: 10, 50 Batching: batch.NewPolicyConfig(), 51 StartFromOldest: true, 52 CommitPeriod: "1s", 53 Timeout: "1s", 54 } 55 } 56 57 //------------------------------------------------------------------------------ 58 59 type pendingRedisStreamMsg struct { 60 payload types.Message 61 stream string 62 id string 63 } 64 65 // RedisStreams is an input type that reads Redis Streams messages. 66 type RedisStreams struct { 67 client redis.UniversalClient 68 cMut sync.Mutex 69 pendingMsgs []pendingRedisStreamMsg 70 pendingMsgsMut sync.Mutex 71 72 timeout time.Duration 73 commitPeriod time.Duration 74 75 conf RedisStreamsConfig 76 77 backlogs map[string]string 78 79 aMut sync.Mutex 80 ackSend map[string][]string // Acks that can be sent 81 82 deprecatedAckFns []AsyncAckFn 83 84 stats metrics.Type 85 log log.Modular 86 87 closeChan chan struct{} 88 closedChan chan struct{} 89 closeOnce sync.Once 90 } 91 92 // NewRedisStreams creates a new RedisStreams input type. 93 func NewRedisStreams( 94 conf RedisStreamsConfig, log log.Modular, stats metrics.Type, 95 ) (*RedisStreams, error) { 96 r := &RedisStreams{ 97 conf: conf, 98 stats: stats, 99 log: log, 100 backlogs: make(map[string]string, len(conf.Streams)), 101 ackSend: make(map[string][]string, len(conf.Streams)), 102 closeChan: make(chan struct{}), 103 closedChan: make(chan struct{}), 104 } 105 106 for _, str := range conf.Streams { 107 r.backlogs[str] = "0" 108 } 109 110 if _, err := r.conf.Config.Client(); err != nil { 111 return nil, err 112 } 113 114 if tout := conf.Timeout; len(tout) > 0 { 115 var err error 116 if r.timeout, err = time.ParseDuration(tout); err != nil { 117 return nil, fmt.Errorf("failed to parse timeout string: %v", err) 118 } 119 } 120 121 if tout := conf.CommitPeriod; len(tout) > 0 { 122 var err error 123 if r.commitPeriod, err = time.ParseDuration(tout); err != nil { 124 return nil, fmt.Errorf("failed to parse commit period string: %v", err) 125 } 126 } 127 128 go r.loop() 129 return r, nil 130 } 131 132 //------------------------------------------------------------------------------ 133 134 func (r *RedisStreams) loop() { 135 defer func() { 136 var client redis.UniversalClient 137 r.cMut.Lock() 138 client = r.client 139 r.client = nil 140 r.cMut.Unlock() 141 if client != nil { 142 client.Close() 143 } 144 close(r.closedChan) 145 }() 146 commitTimer := time.NewTicker(r.commitPeriod) 147 148 closed := false 149 for !closed { 150 select { 151 case <-commitTimer.C: 152 case <-r.closeChan: 153 closed = true 154 } 155 r.sendAcks() 156 } 157 } 158 159 func (r *RedisStreams) addAsyncAcks(stream string, ids ...string) { 160 r.aMut.Lock() 161 if acks, exists := r.ackSend[stream]; exists { 162 acks = append(acks, ids...) 163 r.ackSend[stream] = acks 164 } else { 165 r.ackSend[stream] = ids 166 } 167 r.aMut.Unlock() 168 } 169 170 func (r *RedisStreams) sendAcks() { 171 var client redis.UniversalClient 172 r.cMut.Lock() 173 client = r.client 174 r.cMut.Unlock() 175 176 if client == nil { 177 return 178 } 179 180 r.aMut.Lock() 181 ackSend := r.ackSend 182 r.ackSend = map[string][]string{} 183 r.aMut.Unlock() 184 185 for str, ids := range ackSend { 186 if len(ids) == 0 { 187 continue 188 } 189 if err := r.client.XAck(str, r.conf.ConsumerGroup, ids...).Err(); err != nil { 190 r.log.Errorf("Failed to ack stream %v: %v\n", str, err) 191 } 192 } 193 } 194 195 //------------------------------------------------------------------------------ 196 197 // Connect establishes a connection to a Redis server. 198 func (r *RedisStreams) Connect() error { 199 return r.ConnectWithContext(context.Background()) 200 } 201 202 // ConnectWithContext establishes a connection to a Redis server. 203 func (r *RedisStreams) ConnectWithContext(ctx context.Context) error { 204 r.cMut.Lock() 205 defer r.cMut.Unlock() 206 207 if r.client != nil { 208 return nil 209 } 210 211 client, err := r.conf.Config.Client() 212 if err != nil { 213 return err 214 } 215 if _, err := client.Ping().Result(); err != nil { 216 return err 217 } 218 219 for _, s := range r.conf.Streams { 220 offset := "$" 221 if r.conf.StartFromOldest { 222 offset = "0" 223 } 224 var err error 225 if r.conf.CreateStreams { 226 err = client.XGroupCreateMkStream(s, r.conf.ConsumerGroup, offset).Err() 227 } else { 228 err = client.XGroupCreate(s, r.conf.ConsumerGroup, offset).Err() 229 } 230 if err != nil && err.Error() != "BUSYGROUP Consumer Group name already exists" { 231 return fmt.Errorf("failed to create group %v for stream %v: %v", r.conf.ConsumerGroup, s, err) 232 } 233 } 234 235 r.log.Infof("Receiving messages from Redis streams: %v\n", r.conf.Streams) 236 237 r.client = client 238 return nil 239 } 240 241 func (r *RedisStreams) read() (pendingRedisStreamMsg, error) { 242 var client redis.UniversalClient 243 var msg pendingRedisStreamMsg 244 245 r.cMut.Lock() 246 client = r.client 247 r.cMut.Unlock() 248 249 if client == nil { 250 return msg, types.ErrNotConnected 251 } 252 253 r.pendingMsgsMut.Lock() 254 defer r.pendingMsgsMut.Unlock() 255 if len(r.pendingMsgs) > 0 { 256 msg = r.pendingMsgs[0] 257 r.pendingMsgs = r.pendingMsgs[1:] 258 return msg, nil 259 } 260 261 strs := make([]string, len(r.conf.Streams)*2) 262 for i, str := range r.conf.Streams { 263 strs[i] = str 264 if bl := r.backlogs[str]; bl != "" { 265 strs[len(r.conf.Streams)+i] = bl 266 } else { 267 strs[len(r.conf.Streams)+i] = ">" 268 } 269 } 270 271 res, err := client.XReadGroup(&redis.XReadGroupArgs{ 272 Block: r.timeout, 273 Consumer: r.conf.ClientID, 274 Group: r.conf.ConsumerGroup, 275 Streams: strs, 276 Count: r.conf.Limit, 277 }).Result() 278 279 if err != nil && err != redis.Nil { 280 if strings.Contains(err.Error(), "i/o timeout") { 281 return msg, types.ErrTimeout 282 } 283 r.disconnect() 284 r.log.Errorf("Error from redis: %v\n", err) 285 return msg, types.ErrNotConnected 286 } 287 288 pendingMsgs := []pendingRedisStreamMsg{} 289 for _, strRes := range res { 290 if _, exists := r.backlogs[strRes.Stream]; exists { 291 if len(strRes.Messages) > 0 { 292 r.backlogs[strRes.Stream] = strRes.Messages[len(strRes.Messages)-1].ID 293 } else { 294 delete(r.backlogs, strRes.Stream) 295 } 296 } 297 for _, xmsg := range strRes.Messages { 298 body, exists := xmsg.Values[r.conf.BodyKey] 299 if !exists { 300 continue 301 } 302 delete(xmsg.Values, r.conf.BodyKey) 303 304 var bodyBytes []byte 305 switch t := body.(type) { 306 case string: 307 bodyBytes = []byte(t) 308 case []byte: 309 bodyBytes = t 310 } 311 if bodyBytes == nil { 312 continue 313 } 314 315 part := message.NewPart(bodyBytes) 316 part.Metadata().Set("redis_stream", xmsg.ID) 317 for k, v := range xmsg.Values { 318 part.Metadata().Set(k, fmt.Sprintf("%v", v)) 319 } 320 321 nextMsg := pendingRedisStreamMsg{ 322 payload: message.New(nil), 323 stream: strRes.Stream, 324 id: xmsg.ID, 325 } 326 nextMsg.payload.Append(part) 327 if msg.payload == nil { 328 msg = nextMsg 329 } else { 330 pendingMsgs = append(pendingMsgs, nextMsg) 331 } 332 } 333 } 334 335 r.pendingMsgs = pendingMsgs 336 if msg.payload == nil { 337 return msg, types.ErrTimeout 338 } 339 return msg, nil 340 } 341 342 // ReadWithContext attempts to pop a message from a Redis list. 343 func (r *RedisStreams) ReadWithContext(ctx context.Context) (types.Message, AsyncAckFn, error) { 344 msg, err := r.read() 345 if err != nil { 346 if err == types.ErrTimeout { 347 // Allow for one more attempt in case we asked for backlog. 348 select { 349 case <-ctx.Done(): 350 default: 351 msg, err = r.read() 352 } 353 } 354 if err != nil { 355 return nil, nil, err 356 } 357 } 358 return msg.payload, func(rctx context.Context, res types.Response) error { 359 if res.Error() != nil { 360 r.pendingMsgsMut.Lock() 361 r.pendingMsgs = append(r.pendingMsgs, msg) 362 r.pendingMsgsMut.Unlock() 363 } else { 364 r.addAsyncAcks(msg.stream, msg.id) 365 } 366 return nil 367 }, nil 368 } 369 370 // Read attempts to pop a message from a Redis list. 371 func (r *RedisStreams) Read() (types.Message, error) { 372 msg, ackFn, err := r.ReadWithContext(context.Background()) 373 if err != nil { 374 return nil, err 375 } 376 r.deprecatedAckFns = append(r.deprecatedAckFns, ackFn) 377 return msg, nil 378 } 379 380 // Acknowledge is a noop since Redis Lists do not support acknowledgements. 381 func (r *RedisStreams) Acknowledge(err error) error { 382 res := response.NewError(err) 383 for _, p := range r.deprecatedAckFns { 384 _ = p(context.Background(), res) 385 } 386 r.deprecatedAckFns = nil 387 return nil 388 } 389 390 // disconnect safely closes a connection to an RedisStreams server. 391 func (r *RedisStreams) disconnect() error { 392 r.sendAcks() 393 394 r.cMut.Lock() 395 defer r.cMut.Unlock() 396 397 var err error 398 if r.client != nil { 399 err = r.client.Close() 400 r.client = nil 401 } 402 return err 403 } 404 405 // CloseAsync shuts down the RedisStreams input and stops processing requests. 406 func (r *RedisStreams) CloseAsync() { 407 r.closeOnce.Do(func() { 408 close(r.closeChan) 409 }) 410 } 411 412 // WaitForClose blocks until the RedisStreams input has closed down. 413 func (r *RedisStreams) WaitForClose(timeout time.Duration) error { 414 select { 415 case <-r.closedChan: 416 case <-time.After(timeout): 417 return types.ErrTimeout 418 } 419 return nil 420 } 421 422 //------------------------------------------------------------------------------