github.com/Jeffail/benthos/v3@v3.65.0/lib/output/switch_deprecated.go (about)

     1  package output
     2  
     3  import (
     4  	"encoding/json"
     5  	"sync"
     6  
     7  	"github.com/Jeffail/benthos/v3/internal/shutdown"
     8  	"github.com/Jeffail/benthos/v3/lib/condition"
     9  	"github.com/Jeffail/benthos/v3/lib/response"
    10  	"github.com/Jeffail/benthos/v3/lib/types"
    11  	"github.com/Jeffail/benthos/v3/lib/util/throttle"
    12  	"golang.org/x/sync/errgroup"
    13  )
    14  
    15  //------------------------------------------------------------------------------
    16  
    17  // SwitchConfigOutput contains configuration fields per output of a switch type.
    18  type SwitchConfigOutput struct {
    19  	Condition   condition.Config `json:"condition" yaml:"condition"`
    20  	Fallthrough bool             `json:"fallthrough" yaml:"fallthrough"`
    21  	Output      Config           `json:"output" yaml:"output"`
    22  }
    23  
    24  // NewSwitchConfigOutput creates a new switch output config with default values.
    25  func NewSwitchConfigOutput() SwitchConfigOutput {
    26  	cond := condition.NewConfig()
    27  	cond.Type = condition.TypeStatic
    28  	cond.Static = true
    29  
    30  	return SwitchConfigOutput{
    31  		Condition:   cond,
    32  		Fallthrough: false,
    33  		Output:      NewConfig(),
    34  	}
    35  }
    36  
    37  // UnmarshalJSON ensures that when parsing configs that are in a map or slice
    38  // the default values are still applied.
    39  func (s *SwitchConfigOutput) UnmarshalJSON(bytes []byte) error {
    40  	type confAlias SwitchConfigOutput
    41  	aliased := confAlias(NewSwitchConfigOutput())
    42  
    43  	if err := json.Unmarshal(bytes, &aliased); err != nil {
    44  		return err
    45  	}
    46  
    47  	*s = SwitchConfigOutput(aliased)
    48  	return nil
    49  }
    50  
    51  // UnmarshalYAML ensures that when parsing configs that are in a map or slice
    52  // the default values are still applied.
    53  func (s *SwitchConfigOutput) UnmarshalYAML(unmarshal func(interface{}) error) error {
    54  	type confAlias SwitchConfigOutput
    55  	aliased := confAlias(NewSwitchConfigOutput())
    56  
    57  	if err := unmarshal(&aliased); err != nil {
    58  		return err
    59  	}
    60  
    61  	*s = SwitchConfigOutput(aliased)
    62  	return nil
    63  }
    64  
    65  //------------------------------------------------------------------------------
    66  
    67  func (o *Switch) loopDeprecated() {
    68  	var (
    69  		wg         = sync.WaitGroup{}
    70  		mMsgRcvd   = o.stats.GetCounter("switch.messages.received")
    71  		mMsgSnt    = o.stats.GetCounter("switch.messages.sent")
    72  		mOutputErr = o.stats.GetCounter("switch.output.error")
    73  	)
    74  
    75  	defer func() {
    76  		wg.Wait()
    77  		for i, output := range o.outputs {
    78  			output.CloseAsync()
    79  			close(o.outputTSChans[i])
    80  		}
    81  		for _, output := range o.outputs {
    82  			_ = output.WaitForClose(shutdown.MaximumShutdownWait())
    83  		}
    84  		close(o.closedChan)
    85  	}()
    86  
    87  	sendLoop := func() {
    88  		defer wg.Done()
    89  		for {
    90  			var ts types.Transaction
    91  			var open bool
    92  
    93  			select {
    94  			case ts, open = <-o.transactions:
    95  				if !open {
    96  					return
    97  				}
    98  			case <-o.ctx.Done():
    99  				return
   100  			}
   101  			mMsgRcvd.Incr(1)
   102  
   103  			var outputTargets []int
   104  			for i, oCond := range o.conditions {
   105  				if oCond.Check(ts.Payload) {
   106  					outputTargets = append(outputTargets, i)
   107  					if !o.fallthroughs[i] {
   108  						break
   109  					}
   110  				}
   111  			}
   112  
   113  			if o.strictMode && len(outputTargets) == 0 {
   114  				select {
   115  				case ts.ResponseChan <- response.NewError(ErrSwitchNoConditionMet):
   116  				case <-o.ctx.Done():
   117  					return
   118  				}
   119  				continue
   120  			}
   121  
   122  			var owg errgroup.Group
   123  			for _, target := range outputTargets {
   124  				msgCopy, i := ts.Payload.Copy(), target
   125  				owg.Go(func() error {
   126  					throt := throttle.New(throttle.OptCloseChan(o.ctx.Done()))
   127  					resChan := make(chan types.Response)
   128  
   129  					// Try until success or shutdown.
   130  					for {
   131  						select {
   132  						case o.outputTSChans[i] <- types.NewTransaction(msgCopy, resChan):
   133  						case <-o.ctx.Done():
   134  							return types.ErrTypeClosed
   135  						}
   136  						select {
   137  						case res := <-resChan:
   138  							if res.Error() != nil {
   139  								if o.retryUntilSuccess {
   140  									o.logger.Errorf("Failed to dispatch switch message: %v\n", res.Error())
   141  									mOutputErr.Incr(1)
   142  									if !throt.Retry() {
   143  										return types.ErrTypeClosed
   144  									}
   145  								} else {
   146  									return res.Error()
   147  								}
   148  							} else {
   149  								mMsgSnt.Incr(1)
   150  								return nil
   151  							}
   152  						case <-o.ctx.Done():
   153  							return types.ErrTypeClosed
   154  						}
   155  					}
   156  				})
   157  			}
   158  
   159  			var oResponse types.Response = response.NewAck()
   160  			if resErr := owg.Wait(); resErr != nil {
   161  				oResponse = response.NewError(resErr)
   162  			}
   163  			select {
   164  			case ts.ResponseChan <- oResponse:
   165  			case <-o.ctx.Done():
   166  				return
   167  			}
   168  		}
   169  	}
   170  
   171  	// Max in flight
   172  	for i := 0; i < o.maxInFlight; i++ {
   173  		wg.Add(1)
   174  		go sendLoop()
   175  	}
   176  }
   177  
   178  //------------------------------------------------------------------------------