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

     1  package output
     2  
     3  import (
     4  	"bytes"
     5  	"fmt"
     6  	"io"
     7  	"sync/atomic"
     8  	"time"
     9  
    10  	"github.com/Jeffail/benthos/v3/internal/batch"
    11  	"github.com/Jeffail/benthos/v3/internal/tracing"
    12  	"github.com/Jeffail/benthos/v3/lib/log"
    13  	"github.com/Jeffail/benthos/v3/lib/message"
    14  	"github.com/Jeffail/benthos/v3/lib/metrics"
    15  	"github.com/Jeffail/benthos/v3/lib/response"
    16  	"github.com/Jeffail/benthos/v3/lib/types"
    17  )
    18  
    19  //------------------------------------------------------------------------------
    20  
    21  // LineWriter is an output type that writes messages to an io.WriterCloser type
    22  // as lines.
    23  type LineWriter struct {
    24  	running int32
    25  
    26  	typeStr string
    27  	log     log.Modular
    28  	stats   metrics.Type
    29  
    30  	customDelim []byte
    31  
    32  	transactions <-chan types.Transaction
    33  
    34  	handle      io.WriteCloser
    35  	closeOnExit bool
    36  
    37  	closeChan  chan struct{}
    38  	closedChan chan struct{}
    39  }
    40  
    41  // NewLineWriter creates a new LineWriter output type.
    42  func NewLineWriter(
    43  	handle io.WriteCloser,
    44  	closeOnExit bool,
    45  	customDelimiter []byte,
    46  	typeStr string,
    47  	log log.Modular,
    48  	stats metrics.Type,
    49  ) (Type, error) {
    50  	return &LineWriter{
    51  		running:     1,
    52  		typeStr:     typeStr,
    53  		log:         log,
    54  		stats:       stats,
    55  		customDelim: customDelimiter,
    56  		handle:      handle,
    57  		closeOnExit: closeOnExit,
    58  		closeChan:   make(chan struct{}),
    59  		closedChan:  make(chan struct{}),
    60  	}, nil
    61  }
    62  
    63  //------------------------------------------------------------------------------
    64  
    65  // loop is an internal loop that brokers incoming messages to output pipe.
    66  func (w *LineWriter) loop() {
    67  	// Metrics paths
    68  	var (
    69  		mCount     = w.stats.GetCounter("count")
    70  		mPartsSent = w.stats.GetCounter("sent")
    71  		mSent      = w.stats.GetCounter("batch.sent")
    72  		mBytesSent = w.stats.GetCounter("batch.bytes")
    73  		mLatency   = w.stats.GetTimer("batch.latency")
    74  	)
    75  
    76  	defer func() {
    77  		if w.closeOnExit {
    78  			w.handle.Close()
    79  		}
    80  		close(w.closedChan)
    81  	}()
    82  
    83  	delim := []byte("\n")
    84  	if len(w.customDelim) > 0 {
    85  		delim = w.customDelim
    86  	}
    87  
    88  	for atomic.LoadInt32(&w.running) == 1 {
    89  		var ts types.Transaction
    90  		var open bool
    91  		select {
    92  		case ts, open = <-w.transactions:
    93  			if !open {
    94  				return
    95  			}
    96  			mCount.Incr(1)
    97  		case <-w.closeChan:
    98  			return
    99  		}
   100  
   101  		w.log.Tracef("Attempting to write %v messages to '%v'.\n", ts.Payload.Len(), w.typeStr)
   102  		spans := tracing.CreateChildSpans("output_"+w.typeStr, ts.Payload)
   103  
   104  		var err error
   105  		t0 := time.Now()
   106  		if ts.Payload.Len() == 1 {
   107  			_, err = fmt.Fprintf(w.handle, "%s%s", ts.Payload.Get(0).Get(), delim)
   108  		} else {
   109  			_, err = fmt.Fprintf(w.handle, "%s%s%s", bytes.Join(message.GetAllBytes(ts.Payload), delim), delim, delim)
   110  		}
   111  		latency := time.Since(t0).Nanoseconds()
   112  		if err == nil {
   113  			mSent.Incr(1)
   114  			mPartsSent.Incr(int64(batch.MessageCollapsedCount(ts.Payload)))
   115  			mBytesSent.Incr(int64(message.GetAllBytesLen(ts.Payload)))
   116  			w.log.Tracef("Successfully wrote %v messages to '%v'.\n", ts.Payload.Len(), w.typeStr)
   117  			mLatency.Timing(latency)
   118  		}
   119  
   120  		for _, s := range spans {
   121  			s.Finish()
   122  		}
   123  
   124  		select {
   125  		case ts.ResponseChan <- response.NewError(err):
   126  		case <-w.closeChan:
   127  			return
   128  		}
   129  	}
   130  }
   131  
   132  // Connected returns a boolean indicating whether this output is currently
   133  // connected to its target.
   134  func (w *LineWriter) Connected() bool {
   135  	return true
   136  }
   137  
   138  // Consume assigns a messages channel for the output to read.
   139  func (w *LineWriter) Consume(ts <-chan types.Transaction) error {
   140  	if w.transactions != nil {
   141  		return types.ErrAlreadyStarted
   142  	}
   143  	w.transactions = ts
   144  	go w.loop()
   145  	return nil
   146  }
   147  
   148  // CloseAsync shuts down the File output and stops processing messages.
   149  func (w *LineWriter) CloseAsync() {
   150  	if atomic.CompareAndSwapInt32(&w.running, 1, 0) {
   151  		close(w.closeChan)
   152  	}
   153  }
   154  
   155  // WaitForClose blocks until the File output has closed down.
   156  func (w *LineWriter) WaitForClose(timeout time.Duration) error {
   157  	select {
   158  	case <-w.closedChan:
   159  	case <-time.After(timeout):
   160  		return types.ErrTimeout
   161  	}
   162  	return nil
   163  }
   164  
   165  //------------------------------------------------------------------------------