github.com/Jeffail/benthos/v3@v3.65.0/public/service/stream_builder_test.go (about)

     1  package service_test
     2  
     3  import (
     4  	"bytes"
     5  	"context"
     6  	"fmt"
     7  	"net/http"
     8  	"os"
     9  	"path/filepath"
    10  	"strings"
    11  	"sync"
    12  	"testing"
    13  	"time"
    14  
    15  	"github.com/Jeffail/benthos/v3/public/service"
    16  	"github.com/stretchr/testify/assert"
    17  	"github.com/stretchr/testify/require"
    18  )
    19  
    20  func TestStreamBuilderDefault(t *testing.T) {
    21  	b := service.NewStreamBuilder()
    22  
    23  	act, err := b.AsYAML()
    24  	require.NoError(t, err)
    25  
    26  	exp := []string{
    27  		`input:
    28      label: ""
    29      stdin:`,
    30  		`buffer:
    31      none: {}`,
    32  		`pipeline:
    33      threads: 0
    34      processors: []`,
    35  		`output:
    36      label: ""
    37      stdout:`,
    38  		`logger:
    39      level: INFO`,
    40  		`metrics:
    41      http_server:`,
    42  	}
    43  
    44  	for _, str := range exp {
    45  		assert.Contains(t, act, str)
    46  	}
    47  }
    48  
    49  func TestStreamBuilderProducerFunc(t *testing.T) {
    50  	tmpDir := t.TempDir()
    51  
    52  	outFilePath := filepath.Join(tmpDir, "out.txt")
    53  
    54  	b := service.NewStreamBuilder()
    55  	require.NoError(t, b.SetLoggerYAML("level: NONE"))
    56  	require.NoError(t, b.AddProcessorYAML(`bloblang: 'root = content().uppercase()'`))
    57  	require.NoError(t, b.AddOutputYAML(fmt.Sprintf(`
    58  file:
    59    codec: lines
    60    path: %v`, outFilePath)))
    61  
    62  	pushFn, err := b.AddProducerFunc()
    63  	require.NoError(t, err)
    64  
    65  	// Fails on second call.
    66  	_, err = b.AddProducerFunc()
    67  	require.Error(t, err)
    68  
    69  	// Don't allow input overrides now.
    70  	err = b.SetYAML(`input: {}`)
    71  	require.Error(t, err)
    72  
    73  	strm, err := b.Build()
    74  	require.NoError(t, err)
    75  
    76  	wg := sync.WaitGroup{}
    77  	wg.Add(1)
    78  	go func() {
    79  		defer wg.Done()
    80  
    81  		ctx, done := context.WithTimeout(context.Background(), time.Second*10)
    82  		defer done()
    83  
    84  		require.NoError(t, pushFn(ctx, service.NewMessage([]byte("hello world 1"))))
    85  		require.NoError(t, pushFn(ctx, service.NewMessage([]byte("hello world 2"))))
    86  		require.NoError(t, pushFn(ctx, service.NewMessage([]byte("hello world 3"))))
    87  
    88  		require.NoError(t, strm.StopWithin(time.Second*5))
    89  	}()
    90  
    91  	require.NoError(t, strm.Run(context.Background()))
    92  	wg.Wait()
    93  
    94  	outBytes, err := os.ReadFile(outFilePath)
    95  	require.NoError(t, err)
    96  
    97  	assert.Equal(t, "HELLO WORLD 1\nHELLO WORLD 2\nHELLO WORLD 3\n", string(outBytes))
    98  }
    99  
   100  func TestStreamBuilderBatchProducerFunc(t *testing.T) {
   101  	tmpDir := t.TempDir()
   102  
   103  	outFilePath := filepath.Join(tmpDir, "out.txt")
   104  
   105  	b := service.NewStreamBuilder()
   106  	require.NoError(t, b.SetLoggerYAML("level: NONE"))
   107  	require.NoError(t, b.AddProcessorYAML(`bloblang: 'root = content().uppercase()'`))
   108  	require.NoError(t, b.AddOutputYAML(fmt.Sprintf(`
   109  file:
   110    codec: lines
   111    path: %v`, outFilePath)))
   112  
   113  	pushFn, err := b.AddBatchProducerFunc()
   114  	require.NoError(t, err)
   115  
   116  	// Fails on second call.
   117  	_, err = b.AddProducerFunc()
   118  	require.Error(t, err)
   119  
   120  	// Don't allow input overrides now.
   121  	err = b.SetYAML(`input: {}`)
   122  	require.Error(t, err)
   123  
   124  	strm, err := b.Build()
   125  	require.NoError(t, err)
   126  
   127  	wg := sync.WaitGroup{}
   128  	wg.Add(1)
   129  	go func() {
   130  		defer wg.Done()
   131  
   132  		ctx, done := context.WithTimeout(context.Background(), time.Second*10)
   133  		defer done()
   134  
   135  		require.NoError(t, pushFn(ctx, service.MessageBatch{
   136  			service.NewMessage([]byte("hello world 1")),
   137  			service.NewMessage([]byte("hello world 2")),
   138  		}))
   139  		require.NoError(t, pushFn(ctx, service.MessageBatch{
   140  			service.NewMessage([]byte("hello world 3")),
   141  			service.NewMessage([]byte("hello world 4")),
   142  		}))
   143  		require.NoError(t, pushFn(ctx, service.MessageBatch{
   144  			service.NewMessage([]byte("hello world 5")),
   145  			service.NewMessage([]byte("hello world 6")),
   146  		}))
   147  
   148  		require.NoError(t, strm.StopWithin(time.Second*5))
   149  	}()
   150  
   151  	require.NoError(t, strm.Run(context.Background()))
   152  	wg.Wait()
   153  
   154  	outBytes, err := os.ReadFile(outFilePath)
   155  	require.NoError(t, err)
   156  
   157  	assert.Equal(t, "HELLO WORLD 1\nHELLO WORLD 2\n\nHELLO WORLD 3\nHELLO WORLD 4\n\nHELLO WORLD 5\nHELLO WORLD 6\n\n", string(outBytes))
   158  }
   159  
   160  func TestStreamBuilderEnvVarInterpolation(t *testing.T) {
   161  	os.Setenv("BENTHOS_TEST_ONE", "foo")
   162  	os.Setenv("BENTHOS_TEST_TWO", "bar")
   163  
   164  	b := service.NewStreamBuilder()
   165  	require.NoError(t, b.AddInputYAML(`
   166  kafka:
   167    topics: [ ${BENTHOS_TEST_ONE} ]
   168  `))
   169  
   170  	require.NoError(t, b.SetLoggerYAML(`level: ${BENTHOS_TEST_TWO}`))
   171  
   172  	act, err := b.AsYAML()
   173  	require.NoError(t, err)
   174  
   175  	exp := []string{
   176  		` topics:
   177              - foo`,
   178  		`level: bar`,
   179  	}
   180  
   181  	for _, str := range exp {
   182  		assert.Contains(t, act, str)
   183  	}
   184  
   185  	b = service.NewStreamBuilder()
   186  	require.NoError(t, b.SetYAML(`
   187  input:
   188    kafka:
   189      topics: [ ${BENTHOS_TEST_ONE} ]
   190  logger:
   191    level: ${BENTHOS_TEST_TWO}
   192  `))
   193  
   194  	act, err = b.AsYAML()
   195  	require.NoError(t, err)
   196  
   197  	for _, str := range exp {
   198  		assert.Contains(t, act, str)
   199  	}
   200  }
   201  
   202  func TestStreamBuilderConsumerFunc(t *testing.T) {
   203  	tmpDir := t.TempDir()
   204  
   205  	inFilePath := filepath.Join(tmpDir, "in.txt")
   206  	require.NoError(t, os.WriteFile(inFilePath, []byte(`HELLO WORLD 1
   207  HELLO WORLD 2
   208  HELLO WORLD 3`), 0o755))
   209  
   210  	b := service.NewStreamBuilder()
   211  	require.NoError(t, b.SetLoggerYAML("level: NONE"))
   212  	require.NoError(t, b.AddInputYAML(fmt.Sprintf(`
   213  file:
   214    codec: lines
   215    paths: [ %v ]`, inFilePath)))
   216  	require.NoError(t, b.AddProcessorYAML(`bloblang: 'root = content().lowercase()'`))
   217  
   218  	outMsgs := map[string]struct{}{}
   219  	var outMut sync.Mutex
   220  	handler := func(_ context.Context, m *service.Message) error {
   221  		outMut.Lock()
   222  		defer outMut.Unlock()
   223  
   224  		b, err := m.AsBytes()
   225  		assert.NoError(t, err)
   226  
   227  		outMsgs[string(b)] = struct{}{}
   228  		return nil
   229  	}
   230  	require.NoError(t, b.AddConsumerFunc(handler))
   231  
   232  	// Fails on second call.
   233  	require.Error(t, b.AddConsumerFunc(handler))
   234  
   235  	// Don't allow output overrides now.
   236  	err := b.SetYAML(`output: {}`)
   237  	require.Error(t, err)
   238  
   239  	strm, err := b.Build()
   240  	require.NoError(t, err)
   241  
   242  	require.NoError(t, strm.Run(context.Background()))
   243  
   244  	outMut.Lock()
   245  	assert.Equal(t, map[string]struct{}{
   246  		"hello world 1": {},
   247  		"hello world 2": {},
   248  		"hello world 3": {},
   249  	}, outMsgs)
   250  	outMut.Unlock()
   251  }
   252  
   253  func TestStreamBuilderBatchConsumerFunc(t *testing.T) {
   254  	tmpDir := t.TempDir()
   255  
   256  	inFilePath := filepath.Join(tmpDir, "in.txt")
   257  	require.NoError(t, os.WriteFile(inFilePath, []byte(`HELLO WORLD 1
   258  HELLO WORLD 2
   259  
   260  HELLO WORLD 3
   261  HELLO WORLD 4
   262  
   263  HELLO WORLD 5
   264  HELLO WORLD 6
   265  `), 0o755))
   266  
   267  	b := service.NewStreamBuilder()
   268  	require.NoError(t, b.SetLoggerYAML("level: NONE"))
   269  	require.NoError(t, b.AddInputYAML(fmt.Sprintf(`
   270  file:
   271    codec: lines/multipart
   272    paths: [ %v ]`, inFilePath)))
   273  	require.NoError(t, b.AddProcessorYAML(`bloblang: 'root = content().lowercase()'`))
   274  
   275  	outBatches := map[string]struct{}{}
   276  	var outMut sync.Mutex
   277  	handler := func(_ context.Context, mb service.MessageBatch) error {
   278  		outMut.Lock()
   279  		defer outMut.Unlock()
   280  
   281  		outMsgs := []string{}
   282  		for _, m := range mb {
   283  			b, err := m.AsBytes()
   284  			assert.NoError(t, err)
   285  			outMsgs = append(outMsgs, string(b))
   286  		}
   287  
   288  		outBatches[strings.Join(outMsgs, ",")] = struct{}{}
   289  		return nil
   290  	}
   291  	require.NoError(t, b.AddBatchConsumerFunc(handler))
   292  
   293  	// Fails on second call.
   294  	require.Error(t, b.AddBatchConsumerFunc(handler))
   295  
   296  	// Don't allow output overrides now.
   297  	err := b.SetYAML(`output: {}`)
   298  	require.Error(t, err)
   299  
   300  	strm, err := b.Build()
   301  	require.NoError(t, err)
   302  
   303  	require.NoError(t, strm.Run(context.Background()))
   304  
   305  	outMut.Lock()
   306  	assert.Equal(t, map[string]struct{}{
   307  		"hello world 1,hello world 2": {},
   308  		"hello world 3,hello world 4": {},
   309  		"hello world 5,hello world 6": {},
   310  	}, outBatches)
   311  	outMut.Unlock()
   312  }
   313  
   314  func TestStreamBuilderCustomLogger(t *testing.T) {
   315  	b := service.NewStreamBuilder()
   316  	b.SetPrintLogger(nil)
   317  
   318  	act, err := b.AsYAML()
   319  	require.NoError(t, err)
   320  
   321  	exp := `logger:
   322      level: INFO`
   323  
   324  	assert.NotContains(t, act, exp)
   325  }
   326  
   327  func TestStreamBuilderSetYAML(t *testing.T) {
   328  	b := service.NewStreamBuilder()
   329  	b.SetThreads(10)
   330  	require.NoError(t, b.AddCacheYAML(`label: foocache
   331  type: memory`))
   332  	require.NoError(t, b.AddInputYAML(`type: kafka`))
   333  	require.NoError(t, b.AddOutputYAML(`type: nats`))
   334  	require.NoError(t, b.AddProcessorYAML(`type: bloblang`))
   335  	require.NoError(t, b.AddProcessorYAML(`type: jmespath`))
   336  	require.NoError(t, b.AddRateLimitYAML(`label: foorl
   337  type: local`))
   338  	require.NoError(t, b.SetMetricsYAML(`type: prometheus`))
   339  	require.NoError(t, b.SetLoggerYAML(`level: DEBUG`))
   340  	require.NoError(t, b.SetBufferYAML(`type: memory`))
   341  
   342  	act, err := b.AsYAML()
   343  	require.NoError(t, err)
   344  
   345  	exp := []string{
   346  		`input:
   347      label: ""
   348      kafka:`,
   349  		`buffer:
   350      memory:
   351          limit`,
   352  		`pipeline:
   353      threads: 10
   354      processors:`,
   355  		`
   356          - label: ""
   357            bloblang: ""`,
   358  		`
   359          - label: ""
   360            jmespath:
   361              query: ""`,
   362  		`output:
   363      label: ""
   364      nats:`,
   365  		`metrics:
   366      prometheus:`,
   367  		`cache_resources:
   368      - label: foocache
   369        memory:`,
   370  		`rate_limit_resources:
   371      - label: foorl
   372        local:`,
   373  		`  level: DEBUG`,
   374  	}
   375  
   376  	for _, str := range exp {
   377  		assert.Contains(t, act, str)
   378  	}
   379  }
   380  
   381  func TestStreamBuilderSetResourcesYAML(t *testing.T) {
   382  	b := service.NewStreamBuilder()
   383  	require.NoError(t, b.AddResourcesYAML(`
   384  cache_resources:
   385    - label: foocache
   386      type: memory
   387  
   388  rate_limit_resources:
   389    - label: foorl
   390      type: local
   391  
   392  processor_resources:
   393    - label: fooproc1
   394      type: bloblang
   395    - label: fooproc2
   396      type: jmespath
   397  
   398  input_resources:
   399    - label: fooinput
   400      type: kafka
   401  
   402  output_resources:
   403    - label: foooutput
   404      type: nats
   405  `))
   406  
   407  	act, err := b.AsYAML()
   408  	require.NoError(t, err)
   409  
   410  	exp := []string{
   411  		`cache_resources:
   412      - label: foocache
   413        memory:`,
   414  		`rate_limit_resources:
   415      - label: foorl
   416        local:`,
   417  		`processor_resources:
   418      - label: fooproc1
   419        bloblang:`,
   420  		`    - label: fooproc2
   421        jmespath:`,
   422  		`input_resources:
   423      - label: fooinput
   424        kafka:`,
   425  		`output_resources:
   426      - label: foooutput
   427        nats:`,
   428  	}
   429  
   430  	for _, str := range exp {
   431  		assert.Contains(t, act, str)
   432  	}
   433  }
   434  
   435  func TestStreamBuilderSetYAMLBrokers(t *testing.T) {
   436  	b := service.NewStreamBuilder()
   437  	b.SetThreads(10)
   438  	require.NoError(t, b.AddInputYAML(`type: kafka`))
   439  	require.NoError(t, b.AddInputYAML(`type: amqp_0_9`))
   440  	require.NoError(t, b.AddOutputYAML(`type: nats`))
   441  	require.NoError(t, b.AddOutputYAML(`type: file`))
   442  
   443  	act, err := b.AsYAML()
   444  	require.NoError(t, err)
   445  
   446  	exp := []string{
   447  		`input:
   448      label: ""
   449      broker:
   450          copies: 1
   451          inputs:`,
   452  		`            - label: ""
   453                kafka:`,
   454  		`            - label: ""
   455                amqp_0_9:`,
   456  		`output:
   457      label: ""
   458      broker:
   459          copies: 1
   460          pattern: fan_out
   461          max_in_flight: 1
   462          outputs:`,
   463  		`            - label: ""
   464                nats:`,
   465  		`            - label: ""
   466                file:`,
   467  	}
   468  
   469  	for _, str := range exp {
   470  		assert.Contains(t, act, str)
   471  	}
   472  }
   473  
   474  func TestStreamBuilderYAMLErrors(t *testing.T) {
   475  	b := service.NewStreamBuilder()
   476  
   477  	err := b.AddCacheYAML(`{ label: "", type: memory }`)
   478  	require.Error(t, err)
   479  	assert.EqualError(t, err, "a label must be specified for cache resources")
   480  
   481  	err = b.AddInputYAML(`not valid ! yaml 34324`)
   482  	require.Error(t, err)
   483  	assert.Contains(t, err.Error(), "unmarshal errors")
   484  
   485  	err = b.SetYAML(`not valid ! yaml 34324`)
   486  	require.Error(t, err)
   487  	assert.Contains(t, err.Error(), "expected object value")
   488  
   489  	err = b.SetYAML(`input: { foo: nope }`)
   490  	require.Error(t, err)
   491  	assert.Contains(t, err.Error(), "unable to infer")
   492  
   493  	err = b.SetYAML(`input: { kafka: { not_a_field: nope } }`)
   494  	require.Error(t, err)
   495  	assert.Contains(t, err.Error(), "field not_a_field not recognised")
   496  
   497  	err = b.AddInputYAML(`not_a_field: nah`)
   498  	require.Error(t, err)
   499  	assert.Contains(t, err.Error(), "unable to infer")
   500  
   501  	err = b.AddInputYAML(`kafka: { not_a_field: nah }`)
   502  	require.Error(t, err)
   503  	assert.Contains(t, err.Error(), "field not_a_field not recognised")
   504  
   505  	err = b.SetLoggerYAML(`not_a_field: nah`)
   506  	require.Error(t, err)
   507  	assert.Contains(t, err.Error(), "field not_a_field not recognised")
   508  
   509  	err = b.AddRateLimitYAML(`{ label: "", local: {} }`)
   510  	require.Error(t, err)
   511  	assert.EqualError(t, err, "a label must be specified for rate limit resources")
   512  }
   513  
   514  func TestStreamBuilderSetFields(t *testing.T) {
   515  	tests := []struct {
   516  		name        string
   517  		input       string
   518  		args        []interface{}
   519  		output      string
   520  		errContains string
   521  	}{
   522  		{
   523  			name:  "odd number of args",
   524  			input: `{}`,
   525  			args: []interface{}{
   526  				"just a field",
   527  			},
   528  			errContains: "odd number of pathValues",
   529  		},
   530  		{
   531  			name:  "a path isnt a string",
   532  			input: `{}`,
   533  			args: []interface{}{
   534  				10, "hello world",
   535  			},
   536  			errContains: "should be a string",
   537  		},
   538  		{
   539  			name: "unknown field error",
   540  			input: `
   541  input:
   542    kafka:
   543      topics: [ foo, bar ]
   544  `,
   545  			args: []interface{}{
   546  				"input.kafka.unknown_field", "baz",
   547  			},
   548  			errContains: "field not recognised",
   549  		},
   550  		{
   551  			name: "create lint error",
   552  			input: `
   553  input:
   554    kafka:
   555      topics: [ foo, bar ]
   556  `,
   557  			args: []interface{}{
   558  				"input.label", "foo",
   559  				"output.label", "foo",
   560  			},
   561  			errContains: "collides with a previously",
   562  		},
   563  		{
   564  			name: "set kafka input topics",
   565  			input: `
   566  input:
   567    kafka:
   568      topics: [ foo, bar ]
   569  `,
   570  			args: []interface{}{
   571  				"input.kafka.topics.1", "baz",
   572  			},
   573  			output: `
   574  input:
   575    kafka:
   576      topics: [ foo, baz ]
   577  `,
   578  		},
   579  		{
   580  			name: "append kafka input topics",
   581  			input: `
   582  input:
   583    kafka:
   584      topics: [ foo, bar ]
   585  `,
   586  			args: []interface{}{
   587  				"input.kafka.topics.-", "baz",
   588  				"input.kafka.topics.-", "buz",
   589  				"input.kafka.topics.-", "bev",
   590  			},
   591  			output: `
   592  input:
   593    kafka:
   594      topics: [ foo, bar, baz, buz, bev ]
   595  `,
   596  		},
   597  		{
   598  			name: "add a processor",
   599  			input: `
   600  input:
   601    kafka:
   602      topics: [ foo, bar ]
   603  `,
   604  			args: []interface{}{
   605  				"pipeline.processors.-.bloblang", `root = "meow"`,
   606  			},
   607  			output: `
   608  input:
   609    kafka:
   610      topics: [ foo, bar ]
   611  pipeline:
   612    processors:
   613      - bloblang: 'root = "meow"'
   614  `,
   615  		},
   616  	}
   617  
   618  	for _, test := range tests {
   619  		t.Run(test.name, func(t *testing.T) {
   620  			b := service.NewStreamBuilder()
   621  			require.NoError(t, b.SetYAML(test.input))
   622  			err := b.SetFields(test.args...)
   623  			if test.errContains != "" {
   624  				require.Error(t, err)
   625  				assert.Contains(t, err.Error(), test.errContains)
   626  			} else {
   627  				require.NoError(t, err)
   628  
   629  				b2 := service.NewStreamBuilder()
   630  				require.NoError(t, b2.SetYAML(test.output))
   631  
   632  				bAsYAML, err := b.AsYAML()
   633  				require.NoError(t, err)
   634  
   635  				b2AsYAML, err := b2.AsYAML()
   636  				require.NoError(t, err)
   637  
   638  				assert.YAMLEq(t, b2AsYAML, bAsYAML)
   639  			}
   640  		})
   641  	}
   642  }
   643  
   644  func TestStreamBuilderSetCoreYAML(t *testing.T) {
   645  	b := service.NewStreamBuilder()
   646  	b.SetThreads(10)
   647  	require.NoError(t, b.SetYAML(`
   648  input:
   649    kafka: {}
   650  
   651  pipeline:
   652    threads: 5
   653    processors:
   654      - type: bloblang
   655      - type: jmespath
   656  
   657  output:
   658    nats: {}
   659  `))
   660  
   661  	act, err := b.AsYAML()
   662  	require.NoError(t, err)
   663  
   664  	exp := []string{
   665  		`input:
   666      label: ""
   667      kafka:`,
   668  		`buffer:
   669      none: {}`,
   670  		`pipeline:
   671      threads: 5
   672      processors:`,
   673  		`
   674          - label: ""
   675            bloblang: ""`,
   676  		`
   677          - label: ""
   678            jmespath:
   679              query: ""`,
   680  		`output:
   681      label: ""
   682      nats:`,
   683  	}
   684  
   685  	for _, str := range exp {
   686  		assert.Contains(t, act, str)
   687  	}
   688  }
   689  
   690  func TestStreamBuilderDisabledLinting(t *testing.T) {
   691  	lintingErrorConfig := `
   692  input:
   693    kafka: {}
   694    meow: ignore this field
   695  
   696  output:
   697    nats:
   698      another: linting error
   699  `
   700  	b := service.NewStreamBuilder()
   701  	require.Error(t, b.SetYAML(lintingErrorConfig))
   702  
   703  	b = service.NewStreamBuilder()
   704  	b.DisableLinting()
   705  	require.NoError(t, b.SetYAML(lintingErrorConfig))
   706  }
   707  
   708  type disabledMux struct{}
   709  
   710  func (d disabledMux) HandleFunc(pattern string, handler func(http.ResponseWriter, *http.Request)) {
   711  }
   712  
   713  func BenchmarkStreamRun(b *testing.B) {
   714  	config := `
   715  input:
   716    generate:
   717      count: 5
   718      interval: ""
   719      mapping: |
   720        root.id = uuid_v4()
   721  
   722  pipeline:
   723    processors:
   724      - bloblang: 'root = this'
   725  
   726  output:
   727    drop: {}
   728  
   729  logger:
   730    level: OFF
   731  `
   732  
   733  	strmBuilder := service.NewStreamBuilder()
   734  	strmBuilder.SetHTTPMux(disabledMux{})
   735  	require.NoError(b, strmBuilder.SetYAML(config))
   736  
   737  	b.ResetTimer()
   738  	b.ReportAllocs()
   739  
   740  	for i := 0; i < b.N; i++ {
   741  		strm, err := strmBuilder.Build()
   742  		require.NoError(b, err)
   743  
   744  		require.NoError(b, strm.Run(context.Background()))
   745  	}
   746  }
   747  
   748  func BenchmarkStreamRunOutputN1(b *testing.B) {
   749  	benchmarkStreamRunOutputNX(b, 1)
   750  }
   751  
   752  func BenchmarkStreamRunOutputN10(b *testing.B) {
   753  	benchmarkStreamRunOutputNX(b, 10)
   754  }
   755  
   756  func BenchmarkStreamRunOutputN100(b *testing.B) {
   757  	benchmarkStreamRunOutputNX(b, 100)
   758  }
   759  
   760  type noopOutput struct{}
   761  
   762  func (n *noopOutput) Connect(ctx context.Context) error {
   763  	return nil
   764  }
   765  
   766  func (n *noopOutput) Write(ctx context.Context, msg *service.Message) error {
   767  	return nil
   768  }
   769  
   770  func (n *noopOutput) WriteBatch(ctx context.Context, b service.MessageBatch) error {
   771  	return nil
   772  }
   773  
   774  func (n *noopOutput) Close(ctx context.Context) error {
   775  	return nil
   776  }
   777  
   778  func benchmarkStreamRunOutputNX(b *testing.B, size int) {
   779  	var outputsBuf bytes.Buffer
   780  	for i := 0; i < size; i++ {
   781  		outputsBuf.WriteString("      - custom: {}\n")
   782  	}
   783  
   784  	config := fmt.Sprintf(`
   785  input:
   786    generate:
   787      count: 5
   788      interval: ""
   789      mapping: |
   790        root.id = uuid_v4()
   791  
   792  pipeline:
   793    processors:
   794      - bloblang: 'root = this'
   795  
   796  output:
   797    broker:
   798      outputs:
   799  %v
   800  
   801  logger:
   802    level: OFF
   803  `, outputsBuf.String())
   804  
   805  	env := service.NewEnvironment()
   806  	require.NoError(b, env.RegisterOutput(
   807  		"custom",
   808  		service.NewConfigSpec(),
   809  		func(conf *service.ParsedConfig, mgr *service.Resources) (out service.Output, maxInFlight int, err error) {
   810  			return &noopOutput{}, 1, nil
   811  		},
   812  	))
   813  
   814  	strmBuilder := env.NewStreamBuilder()
   815  	strmBuilder.SetHTTPMux(disabledMux{})
   816  	require.NoError(b, strmBuilder.SetYAML(config))
   817  
   818  	b.ResetTimer()
   819  	b.ReportAllocs()
   820  
   821  	for i := 0; i < b.N; i++ {
   822  		strm, err := strmBuilder.Build()
   823  		require.NoError(b, err)
   824  
   825  		require.NoError(b, strm.Run(context.Background()))
   826  	}
   827  }