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

     1  package processor
     2  
     3  import (
     4  	"fmt"
     5  	"sync"
     6  	"time"
     7  
     8  	"github.com/Jeffail/benthos/v3/internal/docs"
     9  	"github.com/Jeffail/benthos/v3/internal/tracing"
    10  	"github.com/Jeffail/benthos/v3/lib/log"
    11  	"github.com/Jeffail/benthos/v3/lib/metrics"
    12  	"github.com/Jeffail/benthos/v3/lib/types"
    13  )
    14  
    15  //------------------------------------------------------------------------------
    16  
    17  func init() {
    18  	Constructors[TypeThrottle] = TypeSpec{
    19  		constructor: NewThrottle,
    20  		Status:      docs.StatusDeprecated,
    21  		Categories: []Category{
    22  			CategoryUtility,
    23  		},
    24  		Summary: `
    25  Throttles the throughput of a pipeline to a maximum of one message batch per
    26  period. This throttle is per processing pipeline, and therefore four threads
    27  each with a throttle would result in four times the rate specified.`,
    28  		Description: `
    29  The period should be specified as a time duration string. For example, '1s'
    30  would be 1 second, '10ms' would be 10 milliseconds, etc.
    31  
    32  ### Alternatives
    33  
    34  It's recommended that you use the ` + "[`rate_limit` processor](/docs/components/processors/rate_limit)" + ` instead.`,
    35  		FieldSpecs: docs.FieldSpecs{
    36  			docs.FieldCommon("period", "The period to throttle to."),
    37  		},
    38  	}
    39  }
    40  
    41  //------------------------------------------------------------------------------
    42  
    43  // ThrottleConfig contains configuration fields for the Throttle processor.
    44  type ThrottleConfig struct {
    45  	Period string `json:"period" yaml:"period"`
    46  }
    47  
    48  // NewThrottleConfig returns a ThrottleConfig with default values.
    49  func NewThrottleConfig() ThrottleConfig {
    50  	return ThrottleConfig{
    51  		Period: "100us",
    52  	}
    53  }
    54  
    55  //------------------------------------------------------------------------------
    56  
    57  // Throttle is a processor that limits the stream of a pipeline to one message
    58  // batch per period specified.
    59  type Throttle struct {
    60  	conf  Config
    61  	log   log.Modular
    62  	stats metrics.Type
    63  
    64  	duration  time.Duration
    65  	lastBatch time.Time
    66  
    67  	mut sync.Mutex
    68  
    69  	mCount     metrics.StatCounter
    70  	mSent      metrics.StatCounter
    71  	mBatchSent metrics.StatCounter
    72  }
    73  
    74  // NewThrottle returns a Throttle processor.
    75  func NewThrottle(
    76  	conf Config, mgr types.Manager, log log.Modular, stats metrics.Type,
    77  ) (Type, error) {
    78  	t := &Throttle{
    79  		conf:  conf,
    80  		log:   log,
    81  		stats: stats,
    82  
    83  		mCount:     stats.GetCounter("count"),
    84  		mSent:      stats.GetCounter("sent"),
    85  		mBatchSent: stats.GetCounter("batch.sent"),
    86  	}
    87  
    88  	var err error
    89  	if t.duration, err = time.ParseDuration(conf.Throttle.Period); err != nil {
    90  		return nil, fmt.Errorf("failed to parse period: %v", err)
    91  	}
    92  
    93  	return t, nil
    94  }
    95  
    96  //------------------------------------------------------------------------------
    97  
    98  // ProcessMessage applies the processor to a message, either creating >0
    99  // resulting messages or a response to be sent back to the message source.
   100  func (m *Throttle) ProcessMessage(msg types.Message) ([]types.Message, types.Response) {
   101  	m.mCount.Incr(1)
   102  	m.mut.Lock()
   103  	defer m.mut.Unlock()
   104  
   105  	spans := tracing.CreateChildSpans(TypeThrottle, msg)
   106  
   107  	var throttleFor time.Duration
   108  	if since := time.Since(m.lastBatch); m.duration > since {
   109  		throttleFor = m.duration - since
   110  		time.Sleep(throttleFor)
   111  	}
   112  
   113  	for _, s := range spans {
   114  		s.SetTag("throttled_for", throttleFor.String())
   115  		s.Finish()
   116  	}
   117  
   118  	m.lastBatch = time.Now()
   119  
   120  	m.mBatchSent.Incr(1)
   121  	m.mSent.Incr(int64(msg.Len()))
   122  	msgs := [1]types.Message{msg}
   123  	return msgs[:], nil
   124  }
   125  
   126  // CloseAsync shuts down the processor and stops processing requests.
   127  func (m *Throttle) CloseAsync() {
   128  }
   129  
   130  // WaitForClose blocks until the processor has closed down.
   131  func (m *Throttle) WaitForClose(timeout time.Duration) error {
   132  	return nil
   133  }
   134  
   135  //------------------------------------------------------------------------------