github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/async_batcher_test.go (about) 1 package reader 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "testing" 8 "time" 9 10 "github.com/Jeffail/benthos/v3/lib/log" 11 "github.com/Jeffail/benthos/v3/lib/message" 12 "github.com/Jeffail/benthos/v3/lib/message/batch" 13 "github.com/Jeffail/benthos/v3/lib/metrics" 14 "github.com/Jeffail/benthos/v3/lib/response" 15 "github.com/Jeffail/benthos/v3/lib/types" 16 "github.com/stretchr/testify/assert" 17 "github.com/stretchr/testify/require" 18 ) 19 20 func TestAsyncBatcherZero(t *testing.T) { 21 rdr := newMockAsyncReader() 22 conf := batch.NewPolicyConfig() 23 conf.Count = 1 24 res, err := NewAsyncBatcher(conf, rdr, nil, log.Noop(), metrics.Noop()) 25 if err != nil { 26 t.Fatal(err) 27 } 28 if res != rdr { 29 t.Error("Underlying reader was not passed through") 30 } 31 } 32 33 func TestAsyncBatcherHappy(t *testing.T) { 34 ctx, done := context.WithTimeout(context.Background(), time.Second*10) 35 defer done() 36 37 testMsgs := []string{} 38 for i := 0; i < 10; i++ { 39 testMsgs = append(testMsgs, fmt.Sprintf("test %v", i)) 40 } 41 rdr := newMockAsyncReader() 42 for _, str := range testMsgs { 43 rdr.msgsToSnd = append(rdr.msgsToSnd, message.New([][]byte{[]byte(str)})) 44 } 45 46 conf := batch.NewPolicyConfig() 47 conf.Count = 5 48 batcher, err := NewAsyncBatcher(conf, rdr, nil, log.Noop(), metrics.Noop()) 49 if err != nil { 50 t.Fatal(err) 51 } 52 defer func() { 53 batcher.CloseAsync() 54 deadline, _ := ctx.Deadline() 55 if err = batcher.WaitForClose(time.Until(deadline)); err != nil { 56 t.Error(err) 57 } 58 }() 59 60 lastErr := errors.New("test error") 61 go func() { 62 rdr.connChan <- nil 63 for i := 0; i < 5; i++ { 64 rdr.readChan <- nil 65 } 66 for i := 0; i < 5; i++ { 67 rdr.ackChan <- nil 68 } 69 for i := 0; i < 5; i++ { 70 rdr.readChan <- nil 71 } 72 for i := 0; i < 4; i++ { 73 rdr.ackChan <- nil 74 } 75 rdr.ackChan <- lastErr 76 rdr.closeAsyncChan <- struct{}{} 77 rdr.waitForCloseChan <- nil 78 }() 79 80 if err = batcher.ConnectWithContext(ctx); err != nil { 81 t.Fatal(err) 82 } 83 84 msg, ackFn, err := batcher.ReadWithContext(ctx) 85 if err != nil { 86 t.Fatal(err) 87 } 88 if msg.Len() != 5 { 89 t.Errorf("Wrong batch count: %v", msg.Len()) 90 } 91 msg.Iter(func(i int, part types.Part) error { 92 if exp, act := fmt.Sprintf("test %v", i), string(part.Get()); exp != act { 93 t.Errorf("Wrong message contents: %v != %v", act, exp) 94 } 95 return nil 96 }) 97 if err = ackFn(ctx, response.NewAck()); err != nil { 98 t.Error(err) 99 } 100 101 if msg, ackFn, err = batcher.ReadWithContext(ctx); err != nil { 102 t.Fatal(err) 103 } 104 if msg.Len() != 5 { 105 t.Errorf("Wrong batch count: %v", msg.Len()) 106 } 107 msg.Iter(func(i int, part types.Part) error { 108 if exp, act := fmt.Sprintf("test %v", i+5), string(part.Get()); exp != act { 109 t.Errorf("Wrong message contents: %v != %v", act, exp) 110 } 111 return nil 112 }) 113 if err = ackFn(ctx, response.NewAck()); err != lastErr { 114 t.Errorf("Expected '%v', received: %v", lastErr, err) 115 } 116 } 117 118 func TestAsyncBatcherSadThenHappy(t *testing.T) { 119 ctx, done := context.WithTimeout(context.Background(), time.Second*10) 120 defer done() 121 122 testMsgs := []string{} 123 for i := 0; i < 10; i++ { 124 testMsgs = append(testMsgs, fmt.Sprintf("test %v", i)) 125 } 126 rdr := newMockAsyncReader() 127 for _, str := range testMsgs { 128 rdr.msgsToSnd = append(rdr.msgsToSnd, message.New([][]byte{[]byte(str)})) 129 } 130 131 conf := batch.NewPolicyConfig() 132 conf.Count = 5 133 batcher, err := NewAsyncBatcher(conf, rdr, nil, log.Noop(), metrics.Noop()) 134 if err != nil { 135 t.Fatal(err) 136 } 137 defer func() { 138 batcher.CloseAsync() 139 deadline, _ := ctx.Deadline() 140 if err = batcher.WaitForClose(time.Until(deadline)); err != nil { 141 t.Error(err) 142 } 143 }() 144 145 firstReadErr := errors.New("reading failed 1") 146 secondReadErr := errors.New("reading failed 2") 147 go func() { 148 rdr.connChan <- nil 149 rdr.readChan <- firstReadErr 150 for i := 0; i < 5; i++ { 151 rdr.readChan <- nil 152 } 153 for i := 0; i < 5; i++ { 154 rdr.ackChan <- nil 155 } 156 for i := 0; i < 2; i++ { 157 rdr.readChan <- nil 158 } 159 rdr.readChan <- secondReadErr 160 for i := 0; i < 3; i++ { 161 rdr.readChan <- nil 162 } 163 for i := 0; i < 5; i++ { 164 rdr.ackChan <- nil 165 } 166 rdr.closeAsyncChan <- struct{}{} 167 rdr.waitForCloseChan <- nil 168 }() 169 170 if err = batcher.ConnectWithContext(ctx); err != nil { 171 t.Fatal(err) 172 } 173 174 if _, _, err = batcher.ReadWithContext(ctx); err != firstReadErr { 175 t.Fatalf("Expected '%v', received: %v", firstReadErr, err) 176 } 177 178 msg, ackFn, err := batcher.ReadWithContext(ctx) 179 if err != nil { 180 t.Fatal(err) 181 } 182 if msg.Len() != 5 { 183 t.Errorf("Wrong batch count: %v", msg.Len()) 184 } 185 msg.Iter(func(i int, part types.Part) error { 186 if exp, act := fmt.Sprintf("test %v", i), string(part.Get()); exp != act { 187 t.Errorf("Wrong message contents: %v != %v", act, exp) 188 } 189 return nil 190 }) 191 if err = ackFn(ctx, response.NewAck()); err != nil { 192 t.Error(err) 193 } 194 195 if _, _, err = batcher.ReadWithContext(ctx); err != secondReadErr { 196 t.Fatalf("Expected '%v', received: %v", secondReadErr, err) 197 } 198 199 if msg, ackFn, err = batcher.ReadWithContext(ctx); err != nil { 200 t.Fatal(err) 201 } 202 if msg.Len() != 5 { 203 t.Errorf("Wrong batch count: %v", msg.Len()) 204 } 205 msg.Iter(func(i int, part types.Part) error { 206 if exp, act := fmt.Sprintf("test %v", i+5), string(part.Get()); exp != act { 207 t.Errorf("Wrong message contents: %v != %v", act, exp) 208 } 209 return nil 210 }) 211 if err = ackFn(ctx, response.NewAck()); err != nil { 212 t.Error(err) 213 } 214 } 215 216 func TestAsyncBatcherTimeout(t *testing.T) { 217 rdr := newMockAsyncReader() 218 219 conf := batch.NewPolicyConfig() 220 conf.Count = 5 221 batcher, err := NewAsyncBatcher(conf, rdr, nil, log.Noop(), metrics.Noop()) 222 require.NoError(t, err) 223 224 defer func() { 225 batcher.CloseAsync() 226 assert.NoError(t, batcher.WaitForClose(time.Second)) 227 }() 228 229 go func() { 230 rdr.connChan <- nil 231 rdr.readChan <- types.ErrTimeout 232 rdr.closeAsyncChan <- struct{}{} 233 rdr.waitForCloseChan <- nil 234 }() 235 236 require.NoError(t, batcher.ConnectWithContext(context.Background())) 237 238 ctx, done := context.WithTimeout(context.Background(), time.Millisecond) 239 defer done() 240 _, _, err = batcher.ReadWithContext(ctx) 241 assert.EqualError(t, err, types.ErrTimeout.Error()) 242 } 243 244 func TestAsyncBatcherTimedBatches(t *testing.T) { 245 ctx, done := context.WithTimeout(context.Background(), time.Second*10) 246 defer done() 247 248 testMsgs := []string{} 249 for i := 0; i < 10; i++ { 250 testMsgs = append(testMsgs, fmt.Sprintf("test %v", i)) 251 } 252 rdr := newMockAsyncReader() 253 for _, str := range testMsgs { 254 rdr.msgsToSnd = append(rdr.msgsToSnd, message.New([][]byte{[]byte(str)})) 255 } 256 257 conf := batch.NewPolicyConfig() 258 conf.Count = 8 259 conf.Period = "500ms" 260 batcher, err := NewAsyncBatcher(conf, rdr, nil, log.Noop(), metrics.Noop()) 261 if err != nil { 262 t.Fatal(err) 263 } 264 defer func() { 265 batcher.CloseAsync() 266 deadline, _ := ctx.Deadline() 267 if err = batcher.WaitForClose(time.Until(deadline)); err != nil { 268 t.Error(err) 269 } 270 }() 271 272 go func() { 273 rdr.connChan <- nil 274 // Only send two messages through. 275 for i := 0; i < 2; i++ { 276 rdr.readChan <- nil 277 } 278 rdr.readChan <- types.ErrTimeout 279 for i := 0; i < 2; i++ { 280 rdr.ackChan <- nil 281 } 282 for i := 0; i < 8; i++ { 283 rdr.readChan <- nil 284 } 285 for i := 0; i < 8; i++ { 286 rdr.ackChan <- nil 287 } 288 rdr.closeAsyncChan <- struct{}{} 289 rdr.waitForCloseChan <- nil 290 }() 291 292 if err = batcher.ConnectWithContext(ctx); err != nil { 293 t.Fatal(err) 294 } 295 296 msg, ackFn, err := batcher.ReadWithContext(ctx) 297 if err != nil { 298 t.Fatal(err) 299 } 300 if msg.Len() != 2 { 301 t.Errorf("Wrong batch count: %v", msg.Len()) 302 } 303 msg.Iter(func(i int, part types.Part) error { 304 if exp, act := fmt.Sprintf("test %v", i), string(part.Get()); exp != act { 305 t.Errorf("Wrong message contents: %v != %v", act, exp) 306 } 307 return nil 308 }) 309 if err = ackFn(ctx, response.NewAck()); err != nil { 310 t.Error(err) 311 } 312 313 if msg, ackFn, err = batcher.ReadWithContext(ctx); err != nil { 314 t.Fatal(err) 315 } 316 if msg.Len() != 8 { 317 t.Errorf("Wrong batch count: %v", msg.Len()) 318 } 319 msg.Iter(func(i int, part types.Part) error { 320 if exp, act := fmt.Sprintf("test %v", i+2), string(part.Get()); exp != act { 321 t.Errorf("Wrong message contents: %v != %v", act, exp) 322 } 323 return nil 324 }) 325 if err = ackFn(ctx, response.NewAck()); err != nil { 326 t.Error(err) 327 } 328 }