github.com/m3db/m3@v1.5.0/src/aggregator/client/queue.go (about) 1 // Copyright (c) 2018 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package client 22 23 import ( 24 "errors" 25 "fmt" 26 "io" 27 "math" 28 "strings" 29 "sync" 30 31 "github.com/m3db/m3/src/cluster/placement" 32 "github.com/m3db/m3/src/metrics/encoding/protobuf" 33 34 "github.com/uber-go/tally" 35 "go.uber.org/atomic" 36 "go.uber.org/zap" 37 ) 38 39 const ( 40 _queueMinWriteBufSize = 65536 41 _queueMaxWriteBufSize = 8 * _queueMinWriteBufSize 42 ) 43 44 var ( 45 errInstanceQueueClosed = errors.New("instance queue is closed") 46 errWriterQueueFull = errors.New("writer queue is full") 47 errInvalidDropType = errors.New("invalid queue drop type") 48 49 _queueConnWriteBufPool = sync.Pool{New: func() interface{} { 50 b := make([]byte, 0, _queueMinWriteBufSize) 51 return &b 52 }} 53 ) 54 55 // DropType determines which metrics should be dropped when the queue is full. 56 type DropType int 57 58 const ( 59 // DropOldest signifies that the oldest metrics in the queue should be dropped. 60 DropOldest DropType = iota 61 62 // DropCurrent signifies that the current metrics in the queue should be dropped. 63 DropCurrent 64 ) 65 66 var ( 67 validDropTypes = []DropType{ 68 DropOldest, 69 DropCurrent, 70 } 71 ) 72 73 func (t DropType) String() string { 74 switch t { 75 case DropOldest: 76 return "oldest" 77 case DropCurrent: 78 return "current" 79 } 80 return "unknown" 81 } 82 83 // UnmarshalYAML unmarshals a DropType into a valid type from string. 84 func (t *DropType) UnmarshalYAML(unmarshal func(interface{}) error) error { 85 var str string 86 if err := unmarshal(&str); err != nil { 87 return err 88 } 89 if str == "" { 90 *t = defaultDropType 91 return nil 92 } 93 strs := make([]string, 0, len(validDropTypes)) 94 for _, valid := range validDropTypes { 95 if str == valid.String() { 96 *t = valid 97 return nil 98 } 99 strs = append(strs, "'"+valid.String()+"'") 100 } 101 return fmt.Errorf( 102 "invalid DropType '%s' valid types are: %s", str, strings.Join(strs, ", "), 103 ) 104 } 105 106 // instanceQueue processes write requests for given instance. 107 type instanceQueue interface { 108 // Enqueue enqueues a data buffer. 109 Enqueue(buf protobuf.Buffer) error 110 111 // Size returns the number of items in the queue. 112 Size() int 113 114 // Close closes the queue, it blocks until the queue is drained. 115 Close() error 116 117 // Flush flushes the queue, it blocks until the queue is drained. 118 Flush() 119 } 120 121 type writeFn func([]byte) error 122 123 type queue struct { 124 metrics queueMetrics 125 instance placement.Instance 126 conn *connection 127 log *zap.Logger 128 writeFn writeFn 129 buf qbuf 130 dropType DropType 131 closed atomic.Bool 132 mtx sync.Mutex 133 } 134 135 func newInstanceQueue(instance placement.Instance, opts Options) instanceQueue { 136 var ( 137 instrumentOpts = opts.InstrumentOptions() 138 scope = instrumentOpts.MetricsScope() 139 connInstrumentOpts = instrumentOpts.SetMetricsScope(scope.SubScope("connection")) 140 connOpts = opts.ConnectionOptions(). 141 SetInstrumentOptions(connInstrumentOpts). 142 SetRWOptions(opts.RWOptions()) 143 conn = newConnection(instance.Endpoint(), connOpts) 144 iOpts = opts.InstrumentOptions() 145 queueSize = opts.InstanceQueueSize() 146 ) 147 148 // Round up queue size to power of 2. 149 // buf is a ring buffer of byte buffers, so it should definitely be many orders of magnitude 150 // below max uint32. 151 qsize := uint32(roundUpToPowerOfTwo(queueSize)) 152 153 q := &queue{ 154 dropType: opts.QueueDropType(), 155 log: iOpts.Logger(), 156 metrics: newQueueMetrics(iOpts.MetricsScope()), 157 instance: instance, 158 conn: conn, 159 buf: qbuf{ 160 b: make([]protobuf.Buffer, int(qsize)), 161 }, 162 } 163 q.writeFn = q.conn.Write 164 165 return q 166 } 167 168 func (q *queue) Enqueue(buf protobuf.Buffer) error { 169 if q.closed.Load() { 170 q.metrics.enqueueClosedErrors.Inc(1) 171 return errInstanceQueueClosed 172 } 173 174 if len(buf.Bytes()) == 0 { 175 return nil 176 } 177 178 q.mtx.Lock() 179 defer q.mtx.Unlock() 180 181 if full := q.buf.full(); full { 182 switch q.dropType { 183 case DropCurrent: 184 // Close the current buffer so it's resources are freed. 185 buf.Close() 186 q.metrics.enqueueCurrentDropped.Inc(1) 187 return errWriterQueueFull 188 case DropOldest: 189 // Consume oldest buffer instead. 190 oldest := q.buf.shift() 191 oldest.Close() 192 q.metrics.enqueueOldestDropped.Inc(1) 193 default: 194 return errInvalidDropType 195 } 196 } 197 198 q.buf.push(buf) 199 q.metrics.enqueueSuccesses.Inc(1) 200 return nil 201 } 202 203 func (q *queue) Close() error { 204 if !q.closed.CAS(false, true) { 205 return errInstanceQueueClosed 206 } 207 208 return nil 209 } 210 211 func (q *queue) Flush() { 212 var ( 213 buf = _queueConnWriteBufPool.Get().(*[]byte) 214 n int 215 err error 216 ) 217 218 for err == nil { 219 // flush everything in batches, to make sure no single payload is too large, 220 // to prevent a) allocs and b) timeouts due to big buffer IO taking too long. 221 var processed int 222 processed, err = q.flush(buf) 223 n += processed 224 } 225 226 if err != nil && !errors.Is(err, io.EOF) { 227 q.log.Error("error writing data", 228 zap.String("target_instance_id", q.instance.ID()), 229 zap.String("target_instance", q.instance.Endpoint()), 230 zap.Int("bytes_processed", n), 231 zap.Error(err), 232 ) 233 } 234 235 // Check buffer capacity, not length, to make sure we're not pooling slices that are too large. 236 // Otherwise, it could result in multi-megabyte slices hanging around, in case we get a single massive write. 237 if cap(*buf) <= _queueMaxWriteBufSize { 238 *buf = (*buf)[:0] 239 _queueConnWriteBufPool.Put(buf) 240 } 241 } 242 243 func (q *queue) flush(tmpWriteBuf *[]byte) (int, error) { 244 var n int 245 246 q.mtx.Lock() 247 248 if q.buf.size() == 0 { 249 q.mtx.Unlock() 250 return n, io.EOF 251 } 252 253 *tmpWriteBuf = (*tmpWriteBuf)[:0] 254 for q.buf.size() > 0 { 255 protoBuffer := q.buf.peek() 256 bytes := protoBuffer.Bytes() 257 258 if n > 0 && len(bytes)+len(*tmpWriteBuf) >= _queueMaxWriteBufSize { 259 // only merge buffers that are smaller than _queueMaxWriteBufSize bytes 260 break 261 } 262 _ = q.buf.shift() 263 264 if len(bytes) == 0 { 265 continue 266 } 267 268 *tmpWriteBuf = append(*tmpWriteBuf, bytes...) 269 n += len(bytes) 270 protoBuffer.Close() 271 } 272 273 // mutex is not held while doing IO 274 q.mtx.Unlock() 275 276 if n == 0 { 277 return n, io.EOF 278 } 279 280 if err := q.writeFn(*tmpWriteBuf); err != nil { 281 q.metrics.connWriteErrors.Inc(1) 282 return n, err 283 } 284 285 q.metrics.connWriteSuccesses.Inc(1) 286 287 return n, nil 288 } 289 290 func (q *queue) Size() int { 291 return int(q.buf.size()) 292 } 293 294 type queueMetrics struct { 295 enqueueSuccesses tally.Counter 296 enqueueOldestDropped tally.Counter 297 enqueueCurrentDropped tally.Counter 298 enqueueClosedErrors tally.Counter 299 connWriteSuccesses tally.Counter 300 connWriteErrors tally.Counter 301 } 302 303 func newQueueMetrics(s tally.Scope) queueMetrics { 304 enqueueScope := s.Tagged(map[string]string{"action": "enqueue"}) 305 connWriteScope := s.Tagged(map[string]string{"action": "conn-write"}) 306 return queueMetrics{ 307 enqueueSuccesses: enqueueScope.Counter("successes"), 308 enqueueOldestDropped: enqueueScope.Tagged(map[string]string{"drop-type": "oldest"}). 309 Counter("dropped"), 310 enqueueCurrentDropped: enqueueScope.Tagged(map[string]string{"drop-type": "current"}). 311 Counter("dropped"), 312 enqueueClosedErrors: enqueueScope.Tagged(map[string]string{"error-type": "queue-closed"}). 313 Counter("errors"), 314 connWriteSuccesses: connWriteScope.Counter("successes"), 315 connWriteErrors: connWriteScope.Counter("errors"), 316 } 317 } 318 319 // qbuf is a specialized ring buffer for proto payloads 320 type qbuf struct { 321 b []protobuf.Buffer 322 // buffer cursors 323 r uint32 324 w uint32 325 } 326 327 func (q *qbuf) size() uint32 { 328 return q.w - q.r 329 } 330 331 func (q *qbuf) full() bool { 332 return q.size() == uint32(cap(q.b)) 333 } 334 335 func (q *qbuf) mask(idx uint32) uint32 { 336 return idx & (uint32(cap(q.b)) - 1) 337 } 338 339 func (q *qbuf) push(buf protobuf.Buffer) { 340 q.w++ 341 idx := q.mask(q.w) 342 q.b[idx].Close() 343 q.b[idx] = buf 344 } 345 346 func (q *qbuf) shift() protobuf.Buffer { 347 q.r++ 348 idx := q.mask(q.r) 349 val := q.b[idx] 350 q.b[idx] = protobuf.Buffer{} 351 return val 352 } 353 354 func (q *qbuf) peek() protobuf.Buffer { 355 idx := q.mask(q.r + 1) 356 return q.b[idx] 357 } 358 359 func roundUpToPowerOfTwo(val int) int { 360 return int(math.Pow(2, math.Ceil(math.Log2(float64(val))))) 361 }