github.com/Jeffail/benthos/v3@v3.65.0/lib/processor/while.go (about)

     1  package processor
     2  
     3  import (
     4  	"errors"
     5  	"fmt"
     6  	"sync/atomic"
     7  	"time"
     8  
     9  	"github.com/Jeffail/benthos/v3/internal/bloblang/mapping"
    10  	"github.com/Jeffail/benthos/v3/internal/docs"
    11  	"github.com/Jeffail/benthos/v3/internal/interop"
    12  	"github.com/Jeffail/benthos/v3/internal/tracing"
    13  	"github.com/Jeffail/benthos/v3/lib/condition"
    14  	"github.com/Jeffail/benthos/v3/lib/log"
    15  	"github.com/Jeffail/benthos/v3/lib/metrics"
    16  	"github.com/Jeffail/benthos/v3/lib/response"
    17  	"github.com/Jeffail/benthos/v3/lib/types"
    18  	"github.com/google/go-cmp/cmp"
    19  	yaml "gopkg.in/yaml.v3"
    20  )
    21  
    22  //------------------------------------------------------------------------------
    23  
    24  func init() {
    25  	Constructors[TypeWhile] = TypeSpec{
    26  		constructor: NewWhile,
    27  		Categories: []Category{
    28  			CategoryComposition,
    29  		},
    30  		Summary: `
    31  While is a processor that checks a [Bloblang query](/docs/guides/bloblang/about/) against messages and executes child processors on them for as long as the query resolves to true.`,
    32  		Description: `
    33  The field ` + "`at_least_once`" + `, if true, ensures that the child processors are always executed at least one time (like a do .. while loop.)
    34  
    35  The field ` + "`max_loops`" + `, if greater than zero, caps the number of loops for a message batch to this value.
    36  
    37  If following a loop execution the number of messages in a batch is reduced to zero the loop is exited regardless of the condition result. If following a loop execution there are more than 1 message batches the query is checked against the first batch only.`,
    38  		FieldSpecs: docs.FieldSpecs{
    39  			docs.FieldCommon("at_least_once", "Whether to always run the child processors at least one time."),
    40  			docs.FieldAdvanced("max_loops", "An optional maximum number of loops to execute. Helps protect against accidentally creating infinite loops."),
    41  			docs.FieldBloblang(
    42  				"check",
    43  				"A [Bloblang query](/docs/guides/bloblang/about/) that should return a boolean value indicating whether the while loop should execute again.",
    44  				`errored()`,
    45  				`this.urls.unprocessed.length() > 0`,
    46  			).HasDefault(""),
    47  			docs.FieldDeprecated("condition").HasType(docs.FieldTypeCondition).OmitWhen(func(v, _ interface{}) (string, bool) {
    48  				defaultBytes, err := yaml.Marshal(condition.NewConfig())
    49  				if err != nil {
    50  					return "", false
    51  				}
    52  				var iDefault interface{}
    53  				if err = yaml.Unmarshal(defaultBytes, &iDefault); err != nil {
    54  					return "", false
    55  				}
    56  				return "field condition is deprecated in favour of check", cmp.Equal(v, iDefault)
    57  			}),
    58  			docs.FieldCommon("processors", "A list of child processors to execute on each loop.").Array().HasType(docs.FieldTypeProcessor),
    59  		},
    60  	}
    61  }
    62  
    63  //------------------------------------------------------------------------------
    64  
    65  // WhileConfig is a config struct containing fields for the While
    66  // processor.
    67  type WhileConfig struct {
    68  	AtLeastOnce bool             `json:"at_least_once" yaml:"at_least_once"`
    69  	MaxLoops    int              `json:"max_loops" yaml:"max_loops"`
    70  	Check       string           `json:"check" yaml:"check"`
    71  	Condition   condition.Config `json:"condition" yaml:"condition"`
    72  	Processors  []Config         `json:"processors" yaml:"processors"`
    73  }
    74  
    75  // NewWhileConfig returns a default WhileConfig.
    76  func NewWhileConfig() WhileConfig {
    77  	return WhileConfig{
    78  		AtLeastOnce: false,
    79  		MaxLoops:    0,
    80  		Check:       "",
    81  		Condition:   condition.NewConfig(),
    82  		Processors:  []Config{},
    83  	}
    84  }
    85  
    86  //------------------------------------------------------------------------------
    87  
    88  // While is a processor that applies child processors for as long as a child
    89  // condition resolves to true.
    90  type While struct {
    91  	running     int32
    92  	maxLoops    int
    93  	atLeastOnce bool
    94  	cond        condition.Type
    95  	check       *mapping.Executor
    96  	children    []types.Processor
    97  
    98  	log log.Modular
    99  
   100  	mCount      metrics.StatCounter
   101  	mLoop       metrics.StatCounter
   102  	mCondFailed metrics.StatCounter
   103  	mSent       metrics.StatCounter
   104  	mBatchSent  metrics.StatCounter
   105  }
   106  
   107  // NewWhile returns a While processor.
   108  func NewWhile(
   109  	conf Config, mgr types.Manager, log log.Modular, stats metrics.Type,
   110  ) (Type, error) {
   111  	var cond condition.Type
   112  	var check *mapping.Executor
   113  	var err error
   114  
   115  	if !isDefaultGroupCond(conf.While.Condition) {
   116  		cMgr, cLog, cStats := interop.LabelChild("condition", mgr, log, stats)
   117  		if cond, err = condition.New(conf.While.Condition, cMgr, cLog, cStats); err != nil {
   118  			return nil, err
   119  		}
   120  	}
   121  	if len(conf.While.Check) > 0 {
   122  		if check, err = interop.NewBloblangMapping(mgr, conf.While.Check); err != nil {
   123  			return nil, fmt.Errorf("failed to parse check query: %w", err)
   124  		}
   125  	}
   126  
   127  	if cond == nil && check == nil {
   128  		return nil, errors.New("a check query is required")
   129  	}
   130  
   131  	if cond != nil && check != nil {
   132  		return nil, errors.New("cannot specify both a condition and a check query")
   133  	}
   134  
   135  	var children []types.Processor
   136  	for i, pconf := range conf.While.Processors {
   137  		pMgr, pLog, pStats := interop.LabelChild(fmt.Sprintf("while.%v", i), mgr, log, stats)
   138  		var proc Type
   139  		if proc, err = New(pconf, pMgr, pLog, pStats); err != nil {
   140  			return nil, err
   141  		}
   142  		children = append(children, proc)
   143  	}
   144  
   145  	return &While{
   146  		running:     1,
   147  		maxLoops:    conf.While.MaxLoops,
   148  		atLeastOnce: conf.While.AtLeastOnce,
   149  		cond:        cond,
   150  		check:       check,
   151  		children:    children,
   152  
   153  		log: log,
   154  
   155  		mCount:      stats.GetCounter("count"),
   156  		mLoop:       stats.GetCounter("loop"),
   157  		mCondFailed: stats.GetCounter("failed"),
   158  		mSent:       stats.GetCounter("sent"),
   159  		mBatchSent:  stats.GetCounter("batch.sent"),
   160  	}, nil
   161  }
   162  
   163  //------------------------------------------------------------------------------
   164  
   165  func (w *While) checkMsg(msg types.Message) bool {
   166  	if w.cond != nil {
   167  		return w.cond.Check(msg)
   168  	}
   169  	c, err := w.check.QueryPart(0, msg)
   170  	if err != nil {
   171  		c = false
   172  		w.log.Errorf("Query failed for loop: %v\n", err)
   173  	}
   174  	return c
   175  }
   176  
   177  // ProcessMessage applies the processor to a message, either creating >0
   178  // resulting messages or a response to be sent back to the message source.
   179  func (w *While) ProcessMessage(msg types.Message) (msgs []types.Message, res types.Response) {
   180  	w.mCount.Incr(1)
   181  
   182  	spans := tracing.CreateChildSpans(TypeWhile, msg)
   183  	msgs = []types.Message{msg}
   184  
   185  	loops := 0
   186  	condResult := w.atLeastOnce || w.checkMsg(msg)
   187  	for condResult {
   188  		if atomic.LoadInt32(&w.running) != 1 {
   189  			return nil, response.NewError(types.ErrTypeClosed)
   190  		}
   191  		if w.maxLoops > 0 && loops >= w.maxLoops {
   192  			w.log.Traceln("Reached max loops count")
   193  			break
   194  		}
   195  
   196  		w.mLoop.Incr(1)
   197  		w.log.Traceln("Looped")
   198  		for _, s := range spans {
   199  			s.LogKV("event", "loop")
   200  		}
   201  
   202  		msgs, res = ExecuteAll(w.children, msgs...)
   203  		if len(msgs) == 0 {
   204  			return
   205  		}
   206  		condResult = w.checkMsg(msgs[0])
   207  		loops++
   208  	}
   209  
   210  	for _, s := range spans {
   211  		s.SetTag("result", condResult)
   212  		s.Finish()
   213  	}
   214  
   215  	w.mBatchSent.Incr(int64(len(msgs)))
   216  	totalParts := 0
   217  	for _, msg := range msgs {
   218  		totalParts += msg.Len()
   219  	}
   220  	w.mSent.Incr(int64(totalParts))
   221  	return
   222  }
   223  
   224  // CloseAsync shuts down the processor and stops processing requests.
   225  func (w *While) CloseAsync() {
   226  	atomic.StoreInt32(&w.running, 0)
   227  	for _, p := range w.children {
   228  		p.CloseAsync()
   229  	}
   230  }
   231  
   232  // WaitForClose blocks until the processor has closed down.
   233  func (w *While) WaitForClose(timeout time.Duration) error {
   234  	stopBy := time.Now().Add(timeout)
   235  	for _, p := range w.children {
   236  		if err := p.WaitForClose(time.Until(stopBy)); err != nil {
   237  			return err
   238  		}
   239  	}
   240  	return nil
   241  }
   242  
   243  //------------------------------------------------------------------------------