github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/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 // MarshalYAML marshals a DropType. 84 func (t *DropType) MarshalYAML() (interface{}, error) { 85 return t.String(), nil 86 } 87 88 // UnmarshalYAML unmarshals a DropType into a valid type from string. 89 func (t *DropType) UnmarshalYAML(unmarshal func(interface{}) error) error { 90 var str string 91 if err := unmarshal(&str); err != nil { 92 return err 93 } 94 if str == "" { 95 *t = defaultDropType 96 return nil 97 } 98 strs := make([]string, 0, len(validDropTypes)) 99 for _, valid := range validDropTypes { 100 if str == valid.String() { 101 *t = valid 102 return nil 103 } 104 strs = append(strs, "'"+valid.String()+"'") 105 } 106 return fmt.Errorf( 107 "invalid DropType '%s' valid types are: %s", str, strings.Join(strs, ", "), 108 ) 109 } 110 111 // instanceQueue processes write requests for given instance. 112 type instanceQueue interface { 113 // Enqueue enqueues a data buffer. 114 Enqueue(buf protobuf.Buffer) error 115 116 // Size returns the number of items in the queue. 117 Size() int 118 119 // Close closes the queue, it blocks until the queue is drained. 120 Close() error 121 122 // Flush flushes the queue, it blocks until the queue is drained. 123 Flush() 124 } 125 126 type writeFn func([]byte) error 127 128 type queue struct { 129 metrics queueMetrics 130 instance placement.Instance 131 conn *connection 132 log *zap.Logger 133 writeFn writeFn 134 buf qbuf 135 dropType DropType 136 closed atomic.Bool 137 mtx sync.Mutex 138 } 139 140 func newInstanceQueue(instance placement.Instance, opts Options) instanceQueue { 141 var ( 142 instrumentOpts = opts.InstrumentOptions() 143 scope = instrumentOpts.MetricsScope() 144 connInstrumentOpts = instrumentOpts.SetMetricsScope(scope.SubScope("connection")) 145 connOpts = opts.ConnectionOptions(). 146 SetInstrumentOptions(connInstrumentOpts). 147 SetRWOptions(opts.RWOptions()) 148 conn = newConnection(instance.Endpoint(), connOpts) 149 iOpts = opts.InstrumentOptions() 150 queueSize = opts.InstanceQueueSize() 151 ) 152 153 // Round up queue size to power of 2. 154 // buf is a ring buffer of byte buffers, so it should definitely be many orders of magnitude 155 // below max uint32. 156 qsize := uint32(roundUpToPowerOfTwo(queueSize)) 157 158 q := &queue{ 159 dropType: opts.QueueDropType(), 160 log: iOpts.Logger(), 161 metrics: newQueueMetrics(iOpts.MetricsScope()), 162 instance: instance, 163 conn: conn, 164 buf: qbuf{ 165 b: make([]protobuf.Buffer, int(qsize)), 166 }, 167 } 168 q.writeFn = q.conn.Write 169 170 return q 171 } 172 173 func (q *queue) Enqueue(buf protobuf.Buffer) error { 174 if q.closed.Load() { 175 q.metrics.enqueueClosedErrors.Inc(1) 176 return errInstanceQueueClosed 177 } 178 179 if len(buf.Bytes()) == 0 { 180 return nil 181 } 182 183 q.mtx.Lock() 184 defer q.mtx.Unlock() 185 186 if full := q.buf.full(); full { 187 switch q.dropType { 188 case DropCurrent: 189 // Close the current buffer so it's resources are freed. 190 buf.Close() 191 q.metrics.enqueueCurrentDropped.Inc(1) 192 return errWriterQueueFull 193 case DropOldest: 194 // Consume oldest buffer instead. 195 oldest := q.buf.shift() 196 oldest.Close() 197 q.metrics.enqueueOldestDropped.Inc(1) 198 default: 199 return errInvalidDropType 200 } 201 } 202 203 q.buf.push(buf) 204 q.metrics.enqueueSuccesses.Inc(1) 205 return nil 206 } 207 208 func (q *queue) Close() error { 209 if !q.closed.CAS(false, true) { 210 return errInstanceQueueClosed 211 } 212 213 return nil 214 } 215 216 func (q *queue) Flush() { 217 var ( 218 buf = _queueConnWriteBufPool.Get().(*[]byte) 219 n int 220 err error 221 ) 222 223 for err == nil { 224 // flush everything in batches, to make sure no single payload is too large, 225 // to prevent a) allocs and b) timeouts due to big buffer IO taking too long. 226 var processed int 227 processed, err = q.flush(buf) 228 n += processed 229 } 230 231 if err != nil && !errors.Is(err, io.EOF) { 232 q.log.Error("error writing data", 233 zap.String("target_instance_id", q.instance.ID()), 234 zap.String("target_instance", q.instance.Endpoint()), 235 zap.Int("bytes_processed", n), 236 zap.Error(err), 237 ) 238 } 239 240 // Check buffer capacity, not length, to make sure we're not pooling slices that are too large. 241 // Otherwise, it could result in multi-megabyte slices hanging around, in case we get a single massive write. 242 if cap(*buf) <= _queueMaxWriteBufSize { 243 *buf = (*buf)[:0] 244 _queueConnWriteBufPool.Put(buf) 245 } 246 } 247 248 func (q *queue) flush(tmpWriteBuf *[]byte) (int, error) { 249 var n int 250 251 q.mtx.Lock() 252 253 if q.buf.size() == 0 { 254 q.mtx.Unlock() 255 return n, io.EOF 256 } 257 258 *tmpWriteBuf = (*tmpWriteBuf)[:0] 259 for q.buf.size() > 0 { 260 protoBuffer := q.buf.peek() 261 bytes := protoBuffer.Bytes() 262 263 if n > 0 && len(bytes)+len(*tmpWriteBuf) >= _queueMaxWriteBufSize { 264 // only merge buffers that are smaller than _queueMaxWriteBufSize bytes 265 break 266 } 267 _ = q.buf.shift() 268 269 if len(bytes) == 0 { 270 continue 271 } 272 273 *tmpWriteBuf = append(*tmpWriteBuf, bytes...) 274 n += len(bytes) 275 protoBuffer.Close() 276 } 277 278 // mutex is not held while doing IO 279 q.mtx.Unlock() 280 281 if n == 0 { 282 return n, io.EOF 283 } 284 285 if err := q.writeFn(*tmpWriteBuf); err != nil { 286 q.metrics.connWriteErrors.Inc(1) 287 return n, err 288 } 289 290 q.metrics.connWriteSuccesses.Inc(1) 291 292 return n, nil 293 } 294 295 func (q *queue) Size() int { 296 return int(q.buf.size()) 297 } 298 299 type queueMetrics struct { 300 enqueueSuccesses tally.Counter 301 enqueueOldestDropped tally.Counter 302 enqueueCurrentDropped tally.Counter 303 enqueueClosedErrors tally.Counter 304 connWriteSuccesses tally.Counter 305 connWriteErrors tally.Counter 306 } 307 308 func newQueueMetrics(s tally.Scope) queueMetrics { 309 enqueueScope := s.Tagged(map[string]string{"action": "enqueue"}) 310 connWriteScope := s.Tagged(map[string]string{"action": "conn-write"}) 311 return queueMetrics{ 312 enqueueSuccesses: enqueueScope.Counter("successes"), 313 enqueueOldestDropped: enqueueScope.Tagged(map[string]string{"drop-type": "oldest"}). 314 Counter("dropped"), 315 enqueueCurrentDropped: enqueueScope.Tagged(map[string]string{"drop-type": "current"}). 316 Counter("dropped"), 317 enqueueClosedErrors: enqueueScope.Tagged(map[string]string{"error-type": "queue-closed"}). 318 Counter("errors"), 319 connWriteSuccesses: connWriteScope.Counter("successes"), 320 connWriteErrors: connWriteScope.Counter("errors"), 321 } 322 } 323 324 // qbuf is a specialized ring buffer for proto payloads 325 type qbuf struct { 326 b []protobuf.Buffer 327 // buffer cursors 328 r uint32 329 w uint32 330 } 331 332 func (q *qbuf) size() uint32 { 333 return q.w - q.r 334 } 335 336 func (q *qbuf) full() bool { 337 return q.size() == uint32(cap(q.b)) 338 } 339 340 func (q *qbuf) mask(idx uint32) uint32 { 341 return idx & (uint32(cap(q.b)) - 1) 342 } 343 344 func (q *qbuf) push(buf protobuf.Buffer) { 345 q.w++ 346 idx := q.mask(q.w) 347 q.b[idx].Close() 348 q.b[idx] = buf 349 } 350 351 func (q *qbuf) shift() protobuf.Buffer { 352 q.r++ 353 idx := q.mask(q.r) 354 val := q.b[idx] 355 q.b[idx] = protobuf.Buffer{} 356 return val 357 } 358 359 func (q *qbuf) peek() protobuf.Buffer { 360 idx := q.mask(q.r + 1) 361 return q.b[idx] 362 } 363 364 func roundUpToPowerOfTwo(val int) int { 365 return int(math.Pow(2, math.Ceil(math.Log2(float64(val))))) 366 }