github.com/Jeffail/benthos/v3@v3.65.0/lib/buffer/parallel/memory.go (about)

     1  package parallel
     2  
     3  import (
     4  	"sync"
     5  
     6  	"github.com/Jeffail/benthos/v3/lib/types"
     7  )
     8  
     9  //------------------------------------------------------------------------------
    10  
    11  // Memory is a parallel buffer implementation that allows multiple parallel
    12  // consumers to read and purge messages from the buffer asynchronously.
    13  type Memory struct {
    14  	messages     []types.Message
    15  	bytes        int
    16  	pendingBytes int
    17  
    18  	cap  int
    19  	cond *sync.Cond
    20  
    21  	closed bool
    22  }
    23  
    24  // NewMemory creates a memory based parallel buffer.
    25  func NewMemory(capacity int) *Memory {
    26  	return &Memory{
    27  		bytes: 0,
    28  		cap:   capacity,
    29  		cond:  sync.NewCond(&sync.Mutex{}),
    30  	}
    31  }
    32  
    33  //------------------------------------------------------------------------------
    34  
    35  // NextMessage reads the next oldest message, the message is preserved until the
    36  // returned AckFunc is called.
    37  func (m *Memory) NextMessage() (types.Message, AckFunc, error) {
    38  	m.cond.L.Lock()
    39  	for len(m.messages) == 0 && !m.closed {
    40  		m.cond.Wait()
    41  	}
    42  
    43  	if m.closed {
    44  		m.cond.L.Unlock()
    45  		return nil, nil, types.ErrTypeClosed
    46  	}
    47  
    48  	msg := m.messages[0]
    49  
    50  	m.messages[0] = nil
    51  	m.messages = m.messages[1:]
    52  
    53  	messageSize := 0
    54  	msg.Iter(func(i int, b types.Part) error {
    55  		messageSize += len(b.Get())
    56  		return nil
    57  	})
    58  	m.pendingBytes += messageSize
    59  
    60  	m.cond.Broadcast()
    61  	m.cond.L.Unlock()
    62  
    63  	return msg, func(ack bool) (int, error) {
    64  		m.cond.L.Lock()
    65  		if m.closed {
    66  			m.cond.L.Unlock()
    67  			return 0, types.ErrTypeClosed
    68  		}
    69  		m.pendingBytes -= messageSize
    70  		if ack {
    71  			m.bytes -= messageSize
    72  		} else {
    73  			m.messages = append([]types.Message{msg}, m.messages...)
    74  		}
    75  		m.cond.Broadcast()
    76  
    77  		backlog := m.bytes
    78  		m.cond.L.Unlock()
    79  
    80  		return backlog, nil
    81  	}, nil
    82  }
    83  
    84  // PushMessage adds a new message to the stack. Returns the backlog in bytes.
    85  func (m *Memory) PushMessage(msg types.Message) (int, error) {
    86  	extraBytes := 0
    87  	msg.Iter(func(i int, b types.Part) error {
    88  		extraBytes += len(b.Get())
    89  		return nil
    90  	})
    91  
    92  	if extraBytes > m.cap {
    93  		return 0, types.ErrMessageTooLarge
    94  	}
    95  
    96  	m.cond.L.Lock()
    97  
    98  	if m.closed {
    99  		m.cond.L.Unlock()
   100  		return 0, types.ErrTypeClosed
   101  	}
   102  
   103  	for (m.bytes + extraBytes) > m.cap {
   104  		m.cond.Wait()
   105  		if m.closed {
   106  			m.cond.L.Unlock()
   107  			return 0, types.ErrTypeClosed
   108  		}
   109  	}
   110  
   111  	m.messages = append(m.messages, msg.DeepCopy())
   112  	m.bytes += extraBytes
   113  
   114  	backlog := m.bytes
   115  
   116  	m.cond.Broadcast()
   117  	m.cond.L.Unlock()
   118  
   119  	return backlog, nil
   120  }
   121  
   122  // CloseOnceEmpty closes the Buffer once the buffer has been emptied, this is a
   123  // way for a writer to signal to a reader that it is finished writing messages,
   124  // and therefore the reader can close once it is caught up. This call blocks
   125  // until the close is completed.
   126  func (m *Memory) CloseOnceEmpty() {
   127  	m.cond.L.Lock()
   128  	// TODO: We include pendingBytes here even though they're not acked, which
   129  	// means if those pending messages fail we have message loss. However, if we
   130  	// don't count them then we don't have any way to signal to a batcher at the
   131  	// upper level that it should flush the final batch. We need a cleaner
   132  	// mechanism here.
   133  	for (m.bytes-m.pendingBytes > 0) && !m.closed {
   134  		m.cond.Wait()
   135  	}
   136  	if !m.closed {
   137  		m.closed = true
   138  		m.cond.Broadcast()
   139  	}
   140  	m.cond.L.Unlock()
   141  }
   142  
   143  // Close closes the Buffer so that blocked readers or writers become
   144  // unblocked.
   145  func (m *Memory) Close() {
   146  	m.cond.L.Lock()
   147  	m.closed = true
   148  	m.cond.Broadcast()
   149  	m.cond.L.Unlock()
   150  }
   151  
   152  //------------------------------------------------------------------------------