github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader.go (about) 1 package input 2 3 import ( 4 "sync" 5 "sync/atomic" 6 "time" 7 8 "github.com/Jeffail/benthos/v3/internal/shutdown" 9 "github.com/Jeffail/benthos/v3/internal/tracing" 10 "github.com/Jeffail/benthos/v3/lib/input/reader" 11 "github.com/Jeffail/benthos/v3/lib/log" 12 "github.com/Jeffail/benthos/v3/lib/metrics" 13 "github.com/Jeffail/benthos/v3/lib/types" 14 "github.com/Jeffail/benthos/v3/lib/util/throttle" 15 ) 16 17 //------------------------------------------------------------------------------ 18 19 // Reader is an input implementation that reads messages from a reader.Type. 20 type Reader struct { 21 running int32 22 connected int32 23 24 typeStr string 25 reader reader.Type 26 27 stats metrics.Type 28 log log.Modular 29 30 connThrot *throttle.Type 31 32 transactions chan types.Transaction 33 responses chan types.Response 34 35 closeChan chan struct{} 36 fullyCloseChan chan struct{} 37 fullyCloseOnce sync.Once 38 39 closedChan chan struct{} 40 } 41 42 // NewReader creates a new Reader input type. 43 func NewReader( 44 typeStr string, 45 r reader.Type, 46 log log.Modular, 47 stats metrics.Type, 48 ) (Type, error) { 49 rdr := &Reader{ 50 running: 1, 51 typeStr: typeStr, 52 reader: r, 53 log: log, 54 stats: stats, 55 transactions: make(chan types.Transaction), 56 responses: make(chan types.Response), 57 closeChan: make(chan struct{}), 58 fullyCloseChan: make(chan struct{}), 59 closedChan: make(chan struct{}), 60 } 61 62 rdr.connThrot = throttle.New(throttle.OptCloseChan(rdr.closeChan)) 63 64 go rdr.loop() 65 return rdr, nil 66 } 67 68 //------------------------------------------------------------------------------ 69 70 func (r *Reader) loop() { 71 // Metrics paths 72 var ( 73 mRunning = r.stats.GetGauge("running") 74 mCount = r.stats.GetCounter("count") 75 mRcvd = r.stats.GetCounter("batch.received") 76 mPartsRcvd = r.stats.GetCounter("received") 77 mConn = r.stats.GetCounter("connection.up") 78 mFailedConn = r.stats.GetCounter("connection.failed") 79 mLostConn = r.stats.GetCounter("connection.lost") 80 mLatency = r.stats.GetTimer("latency") 81 ) 82 83 defer func() { 84 _ = r.reader.WaitForClose(shutdown.MaximumShutdownWait()) 85 mRunning.Decr(1) 86 atomic.StoreInt32(&r.connected, 0) 87 88 close(r.transactions) 89 close(r.closedChan) 90 }() 91 mRunning.Incr(1) 92 93 for { 94 if err := r.reader.Connect(); err != nil { 95 if err == types.ErrTypeClosed { 96 return 97 } 98 r.log.Errorf("Failed to connect to %v: %v\n", r.typeStr, err) 99 mFailedConn.Incr(1) 100 if !r.connThrot.Retry() { 101 return 102 } 103 } else { 104 r.connThrot.Reset() 105 break 106 } 107 } 108 mConn.Incr(1) 109 atomic.StoreInt32(&r.connected, 1) 110 111 for atomic.LoadInt32(&r.running) == 1 { 112 msg, err := r.reader.Read() 113 114 // If our reader says it is not connected. 115 if err == types.ErrNotConnected { 116 mLostConn.Incr(1) 117 atomic.StoreInt32(&r.connected, 0) 118 119 // Continue to try to reconnect while still active. 120 for atomic.LoadInt32(&r.running) == 1 { 121 if err = r.reader.Connect(); err != nil { 122 // Close immediately if our reader is closed. 123 if err == types.ErrTypeClosed { 124 return 125 } 126 127 r.log.Errorf("Failed to reconnect to %v: %v\n", r.typeStr, err) 128 mFailedConn.Incr(1) 129 } else if msg, err = r.reader.Read(); err != types.ErrNotConnected { 130 mConn.Incr(1) 131 atomic.StoreInt32(&r.connected, 1) 132 r.connThrot.Reset() 133 break 134 } 135 if !r.connThrot.Retry() { 136 return 137 } 138 } 139 } 140 141 // Close immediately if our reader is closed. 142 if err == types.ErrTypeClosed { 143 return 144 } 145 146 if err != nil || msg == nil { 147 if err != types.ErrTimeout && err != types.ErrNotConnected { 148 r.log.Errorf("Failed to read message: %v\n", err) 149 } 150 if !r.connThrot.Retry() { 151 return 152 } 153 continue 154 } else { 155 r.connThrot.Reset() 156 mCount.Incr(1) 157 mPartsRcvd.Incr(int64(msg.Len())) 158 mRcvd.Incr(1) 159 r.log.Tracef("Consumed %v messages from '%v'.\n", msg.Len(), r.typeStr) 160 } 161 162 tracing.InitSpans("input_"+r.typeStr, msg) 163 select { 164 case r.transactions <- types.NewTransaction(msg, r.responses): 165 case <-r.closeChan: 166 return 167 } 168 169 var res types.Response 170 var open bool 171 select { 172 case res, open = <-r.responses: 173 case <-r.closeChan: 174 // The pipeline is terminating but we still want to attempt to 175 // propagate an acknowledgement from in-transit messages. 176 select { 177 case res, open = <-r.responses: 178 case <-r.fullyCloseChan: 179 return 180 } 181 } 182 if !open { 183 return 184 } 185 if res.Error() != nil || !res.SkipAck() { 186 if err = r.reader.Acknowledge(res.Error()); err != nil { 187 r.log.Errorf("Failed to acknowledge message: %v\n", err) 188 } 189 tTaken := time.Since(msg.CreatedAt()).Nanoseconds() 190 mLatency.Timing(tTaken) 191 } 192 tracing.FinishSpans(msg) 193 } 194 } 195 196 // TransactionChan returns a transactions channel for consuming messages from 197 // this input type. 198 func (r *Reader) TransactionChan() <-chan types.Transaction { 199 return r.transactions 200 } 201 202 // Connected returns a boolean indicating whether this input is currently 203 // connected to its target. 204 func (r *Reader) Connected() bool { 205 return atomic.LoadInt32(&r.connected) == 1 206 } 207 208 // CloseAsync shuts down the Reader input and stops processing requests. 209 func (r *Reader) CloseAsync() { 210 if atomic.CompareAndSwapInt32(&r.running, 1, 0) { 211 close(r.closeChan) 212 r.reader.CloseAsync() 213 } 214 } 215 216 // WaitForClose blocks until the Reader input has closed down. 217 func (r *Reader) WaitForClose(timeout time.Duration) error { 218 go r.fullyCloseOnce.Do(func() { 219 <-time.After(timeout - time.Second) 220 close(r.fullyCloseChan) 221 }) 222 select { 223 case <-r.closedChan: 224 case <-time.After(timeout): 225 return types.ErrTimeout 226 } 227 return nil 228 } 229 230 //------------------------------------------------------------------------------