github.com/Jeffail/benthos/v3@v3.65.0/lib/buffer/parallel_batcher_test.go (about)

     1  package buffer
     2  
     3  import (
     4  	"fmt"
     5  	"reflect"
     6  	"testing"
     7  	"time"
     8  
     9  	"github.com/Jeffail/benthos/v3/lib/buffer/parallel"
    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  )
    17  
    18  //------------------------------------------------------------------------------
    19  
    20  func TestParallelBatcherBasic(t *testing.T) {
    21  	tChan := make(chan types.Transaction)
    22  	resChan := make(chan types.Response)
    23  
    24  	wrap := NewParallelWrapper(
    25  		NewConfig(), parallel.NewMemory(10000),
    26  		log.Noop(), metrics.Noop(),
    27  	)
    28  	policyConf := batch.NewPolicyConfig()
    29  	policyConf.Count = 4
    30  	batcher, err := batch.NewPolicy(policyConf, nil, log.Noop(), metrics.Noop())
    31  	if err != nil {
    32  		t.Fatal(err)
    33  	}
    34  
    35  	b := NewParallelBatcher(batcher, wrap, log.Noop(), metrics.Noop())
    36  	if err := b.Consume(tChan); err != nil {
    37  		t.Fatal(err)
    38  	}
    39  
    40  	var firstBatchExpected [][]byte
    41  	var secondBatchExpected [][]byte
    42  	var finalBatchExpected [][]byte
    43  	for i := 0; i < 10; i++ {
    44  		inputBytes := []byte(fmt.Sprintf("foo %v", i))
    45  		if i < 4 {
    46  			firstBatchExpected = append(firstBatchExpected, inputBytes)
    47  		} else if i < 8 {
    48  			secondBatchExpected = append(secondBatchExpected, inputBytes)
    49  		} else {
    50  			finalBatchExpected = append(finalBatchExpected, inputBytes)
    51  		}
    52  
    53  		select {
    54  		case tChan <- types.NewTransaction(message.New([][]byte{inputBytes}), resChan):
    55  		case <-time.After(time.Second):
    56  			t.Fatalf("Timed out waiting for message %v send", i)
    57  		}
    58  
    59  		// Instant response from buffer
    60  		select {
    61  		case res := <-resChan:
    62  			if res.Error() != nil {
    63  				t.Error(res.Error())
    64  			}
    65  		case <-time.After(time.Second):
    66  			t.Fatalf("Timed out waiting for unbuffered message %v response", i)
    67  		}
    68  	}
    69  
    70  	// Receive first batch on output
    71  	var outTr types.Transaction
    72  	select {
    73  	case outTr = <-b.TransactionChan():
    74  	case <-time.After(time.Second):
    75  		t.Fatal("Timed out waiting for unbuffered message read")
    76  	}
    77  	if exp, act := firstBatchExpected, message.GetAllBytes(outTr.Payload); !reflect.DeepEqual(exp, act) {
    78  		t.Errorf("Wrong result from batch: %s != %s", act, exp)
    79  	}
    80  
    81  	// Return response
    82  	select {
    83  	case outTr.ResponseChan <- response.NewAck():
    84  	case <-time.After(time.Second):
    85  		t.Fatal("Timed out waiting for unbuffered response send back")
    86  	}
    87  
    88  	// Receive second batch on output
    89  	select {
    90  	case outTr = <-b.TransactionChan():
    91  	case <-time.After(time.Second):
    92  		t.Fatal("Timed out waiting for unbuffered message read")
    93  	}
    94  	if exp, act := secondBatchExpected, message.GetAllBytes(outTr.Payload); !reflect.DeepEqual(exp, act) {
    95  		t.Errorf("Wrong result from batch: %s != %s", act, exp)
    96  	}
    97  
    98  	// Return response
    99  	select {
   100  	case outTr.ResponseChan <- response.NewAck():
   101  	case <-time.After(time.Second):
   102  		t.Fatal("Timed out waiting for unbuffered response send back")
   103  	}
   104  
   105  	// Check for empty buffer
   106  	select {
   107  	case <-b.TransactionChan():
   108  		t.Error("Unexpected batch")
   109  	case <-time.After(time.Millisecond * 100):
   110  	}
   111  
   112  	b.StopConsuming()
   113  
   114  	// Receive final batch on output
   115  	select {
   116  	case outTr = <-b.TransactionChan():
   117  	case <-time.After(time.Second):
   118  		t.Fatal("Timed out waiting for unbuffered message read")
   119  	}
   120  	if exp, act := finalBatchExpected, message.GetAllBytes(outTr.Payload); !reflect.DeepEqual(exp, act) {
   121  		t.Errorf("Wrong result from batch: %s != %s", act, exp)
   122  	}
   123  
   124  	// Return response
   125  	select {
   126  	case outTr.ResponseChan <- response.NewAck():
   127  	case <-time.After(time.Second):
   128  		t.Fatal("Timed out waiting for unbuffered response send back")
   129  	}
   130  
   131  	b.CloseAsync()
   132  	if err = b.WaitForClose(time.Second); err != nil {
   133  		t.Error(err)
   134  	}
   135  
   136  	close(resChan)
   137  	close(tChan)
   138  }
   139  
   140  func TestParallelBatcherTimed(t *testing.T) {
   141  	tChan := make(chan types.Transaction)
   142  	resChan := make(chan types.Response)
   143  
   144  	wrap := NewParallelWrapper(
   145  		NewConfig(), parallel.NewMemory(10000),
   146  		log.Noop(), metrics.Noop(),
   147  	)
   148  	policyConf := batch.NewPolicyConfig()
   149  	policyConf.Period = "100ms"
   150  	batcher, err := batch.NewPolicy(policyConf, nil, log.Noop(), metrics.Noop())
   151  	if err != nil {
   152  		t.Fatal(err)
   153  	}
   154  
   155  	b := NewParallelBatcher(batcher, wrap, log.Noop(), metrics.Noop())
   156  	if err := b.Consume(tChan); err != nil {
   157  		t.Fatal(err)
   158  	}
   159  
   160  	batchExpected := [][]byte{
   161  		[]byte("foo1"),
   162  		[]byte("foo2"),
   163  		[]byte("foo3"),
   164  	}
   165  
   166  	select {
   167  	case tChan <- types.NewTransaction(message.New(batchExpected), resChan):
   168  	case <-time.After(time.Second):
   169  		t.Fatal("Timed out waiting for message send")
   170  	}
   171  
   172  	// Instant response from buffer
   173  	select {
   174  	case res := <-resChan:
   175  		if res.Error() != nil {
   176  			t.Error(res.Error())
   177  		}
   178  	case <-time.After(time.Second):
   179  		t.Fatal("Timed out waiting for unbuffered message response")
   180  	}
   181  
   182  	// Receive first batch on output
   183  	var outTr types.Transaction
   184  	select {
   185  	case outTr = <-b.TransactionChan():
   186  	case <-time.After(time.Second):
   187  		t.Fatal("Timed out waiting for unbuffered message read")
   188  	}
   189  	if exp, act := batchExpected, message.GetAllBytes(outTr.Payload); !reflect.DeepEqual(exp, act) {
   190  		t.Errorf("Wrong result from batch: %s != %s", act, exp)
   191  	}
   192  
   193  	// Return response
   194  	select {
   195  	case outTr.ResponseChan <- response.NewAck():
   196  	case <-time.After(time.Second):
   197  		t.Fatal("Timed out waiting for unbuffered response send back")
   198  	}
   199  
   200  	select {
   201  	case tChan <- types.NewTransaction(message.New(batchExpected), resChan):
   202  	case <-time.After(time.Second):
   203  		t.Fatal("Timed out waiting for message send")
   204  	}
   205  
   206  	b.StopConsuming()
   207  	b.CloseAsync()
   208  	if err = b.WaitForClose(time.Second); err != nil {
   209  		t.Error(err)
   210  	}
   211  
   212  	close(resChan)
   213  	close(tChan)
   214  }
   215  
   216  //------------------------------------------------------------------------------