github.com/Jeffail/benthos/v3@v3.65.0/lib/broker/fan_in_test.go (about) 1 package broker 2 3 import ( 4 "bytes" 5 "errors" 6 "fmt" 7 "sync" 8 "testing" 9 "time" 10 11 "github.com/Jeffail/benthos/v3/lib/message" 12 "github.com/Jeffail/benthos/v3/lib/metrics" 13 "github.com/Jeffail/benthos/v3/lib/response" 14 "github.com/Jeffail/benthos/v3/lib/types" 15 ) 16 17 var _ types.Producer = &FanIn{} 18 var _ types.Closable = &FanIn{} 19 20 //------------------------------------------------------------------------------ 21 22 func TestBasicFanIn(t *testing.T) { 23 nInputs, nMsgs := 10, 1000 24 25 Inputs := []types.Producer{} 26 mockInputs := []*MockInputType{} 27 28 resChan := make(chan types.Response) 29 30 for i := 0; i < nInputs; i++ { 31 mockInputs = append(mockInputs, &MockInputType{ 32 TChan: make(chan types.Transaction), 33 }) 34 Inputs = append(Inputs, mockInputs[i]) 35 } 36 37 fanIn, err := NewFanIn(Inputs, metrics.Noop()) 38 if err != nil { 39 t.Error(err) 40 return 41 } 42 43 for i := 0; i < nMsgs; i++ { 44 for j := 0; j < nInputs; j++ { 45 content := [][]byte{[]byte(fmt.Sprintf("hello world %v", i))} 46 select { 47 case mockInputs[j].TChan <- types.NewTransaction(message.New(content), resChan): 48 case <-time.After(time.Second * 5): 49 t.Errorf("Timed out waiting for broker send: %v, %v", i, j) 50 return 51 } 52 go func() { 53 var ts types.Transaction 54 select { 55 case ts = <-fanIn.TransactionChan(): 56 if !bytes.Equal(ts.Payload.Get(0).Get(), content[0]) { 57 t.Errorf("Wrong content returned %s != %s", ts.Payload.Get(0).Get(), content[0]) 58 } 59 case <-time.After(time.Second * 5): 60 t.Errorf("Timed out waiting for broker propagate: %v, %v", i, j) 61 } 62 select { 63 case ts.ResponseChan <- response.NewAck(): 64 case <-time.After(time.Second * 5): 65 t.Errorf("Timed out waiting for response to broker: %v, %v", i, j) 66 } 67 }() 68 select { 69 case <-resChan: 70 case <-time.After(time.Second * 5): 71 t.Errorf("Timed out waiting for response to input: %v, %v", i, j) 72 return 73 } 74 } 75 } 76 77 fanIn.CloseAsync() 78 79 if err := fanIn.WaitForClose(time.Second * 10); err != nil { 80 t.Error(err) 81 } 82 } 83 84 func TestFanInShutdown(t *testing.T) { 85 nInputs := 10 86 87 Inputs := []types.Producer{} 88 mockInputs := []*MockInputType{} 89 90 for i := 0; i < nInputs; i++ { 91 mockInputs = append(mockInputs, &MockInputType{ 92 TChan: make(chan types.Transaction), 93 }) 94 Inputs = append(Inputs, mockInputs[i]) 95 } 96 97 fanIn, err := NewFanIn(Inputs, metrics.Noop()) 98 if err != nil { 99 t.Error(err) 100 return 101 } 102 103 for _, mockIn := range mockInputs { 104 select { 105 case _, open := <-mockIn.TransactionChan(): 106 if !open { 107 t.Error("fan in closed early") 108 } else { 109 t.Error("fan in sent unexpected message") 110 } 111 default: 112 } 113 close(mockIn.TChan) 114 } 115 116 select { 117 case <-fanIn.TransactionChan(): 118 case <-time.After(time.Second * 5): 119 t.Error("fan in failed to close") 120 } 121 } 122 123 func TestFanInAsync(t *testing.T) { 124 nInputs, nMsgs := 10, 1000 125 126 Inputs := []types.Producer{} 127 mockInputs := []*MockInputType{} 128 129 for i := 0; i < nInputs; i++ { 130 mockInputs = append(mockInputs, &MockInputType{ 131 TChan: make(chan types.Transaction), 132 }) 133 Inputs = append(Inputs, mockInputs[i]) 134 } 135 136 fanIn, err := NewFanIn(Inputs, metrics.Noop()) 137 if err != nil { 138 t.Error(err) 139 return 140 } 141 142 wg := sync.WaitGroup{} 143 wg.Add(nInputs) 144 145 for j := 0; j < nInputs; j++ { 146 go func(index int) { 147 rChan := make(chan types.Response) 148 for i := 0; i < nMsgs; i++ { 149 content := [][]byte{[]byte(fmt.Sprintf("hello world %v %v", i, index))} 150 select { 151 case mockInputs[index].TChan <- types.NewTransaction(message.New(content), rChan): 152 case <-time.After(time.Second * 5): 153 t.Errorf("Timed out waiting for broker send: %v, %v", i, index) 154 return 155 } 156 select { 157 case res := <-rChan: 158 if expected, actual := string(content[0]), res.Error().Error(); expected != actual { 159 t.Errorf("Wrong response: %v != %v", expected, actual) 160 } 161 case <-time.After(time.Second * 5): 162 t.Errorf("Timed out waiting for response to input: %v, %v", i, index) 163 return 164 } 165 } 166 wg.Done() 167 }(j) 168 } 169 170 for i := 0; i < nMsgs*nInputs; i++ { 171 var ts types.Transaction 172 select { 173 case ts = <-fanIn.TransactionChan(): 174 case <-time.After(time.Second * 5): 175 t.Errorf("Timed out waiting for broker propagate: %v", i) 176 return 177 } 178 select { 179 case ts.ResponseChan <- response.NewError(errors.New(string(ts.Payload.Get(0).Get()))): 180 case <-time.After(time.Second * 5): 181 t.Errorf("Timed out waiting for response to broker: %v", i) 182 return 183 } 184 } 185 186 wg.Wait() 187 } 188 189 func BenchmarkBasicFanIn(b *testing.B) { 190 nInputs := 10 191 192 Inputs := []types.Producer{} 193 mockInputs := []*MockInputType{} 194 resChan := make(chan types.Response) 195 196 for i := 0; i < nInputs; i++ { 197 mockInputs = append(mockInputs, &MockInputType{ 198 TChan: make(chan types.Transaction), 199 }) 200 Inputs = append(Inputs, mockInputs[i]) 201 } 202 203 fanIn, err := NewFanIn(Inputs, metrics.Noop()) 204 if err != nil { 205 b.Error(err) 206 return 207 } 208 209 defer func() { 210 fanIn.CloseAsync() 211 fanIn.WaitForClose(time.Second) 212 }() 213 214 b.ResetTimer() 215 216 for i := 0; i < b.N; i++ { 217 for j := 0; j < nInputs; j++ { 218 content := [][]byte{[]byte(fmt.Sprintf("hello world %v", i))} 219 select { 220 case mockInputs[j].TChan <- types.NewTransaction(message.New(content), resChan): 221 case <-time.After(time.Second * 5): 222 b.Errorf("Timed out waiting for broker send: %v, %v", i, j) 223 return 224 } 225 var ts types.Transaction 226 select { 227 case ts = <-fanIn.TransactionChan(): 228 if !bytes.Equal(ts.Payload.Get(0).Get(), content[0]) { 229 b.Errorf("Wrong content returned %s != %s", ts.Payload.Get(0).Get(), content[0]) 230 } 231 case <-time.After(time.Second * 5): 232 b.Errorf("Timed out waiting for broker propagate: %v, %v", i, j) 233 return 234 } 235 select { 236 case ts.ResponseChan <- response.NewAck(): 237 case <-time.After(time.Second * 5): 238 b.Errorf("Timed out waiting for response to broker: %v, %v", i, j) 239 return 240 } 241 select { 242 case <-resChan: 243 case <-time.After(time.Second * 5): 244 b.Errorf("Timed out waiting for response to input: %v, %v", i, j) 245 return 246 } 247 } 248 } 249 250 b.StopTimer() 251 } 252 253 //------------------------------------------------------------------------------