github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/series/reader.go (about) 1 // Copyright (c) 2017 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 series 22 23 import ( 24 "errors" 25 "fmt" 26 "time" 27 28 "github.com/m3db/m3/src/dbnode/namespace" 29 "github.com/m3db/m3/src/dbnode/retention" 30 "github.com/m3db/m3/src/dbnode/storage/block" 31 "github.com/m3db/m3/src/dbnode/x/xio" 32 "github.com/m3db/m3/src/x/context" 33 xerrors "github.com/m3db/m3/src/x/errors" 34 "github.com/m3db/m3/src/x/ident" 35 xtime "github.com/m3db/m3/src/x/time" 36 ) 37 38 var ( 39 errSeriesReadInvalidRange = errors.New( 40 "series invalid time range read argument specified") 41 ) 42 43 // Reader reads results from a series, or a series block 44 // retriever or both. 45 // It is implemented as a struct so it can be allocated on 46 // the stack. 47 type Reader struct { 48 opts Options 49 id ident.ID 50 retriever QueryableBlockRetriever 51 onRetrieve block.OnRetrieveBlock 52 onRead block.OnReadBlock 53 } 54 55 // NewReaderUsingRetriever returns a reader for a series 56 // block retriever, it will use the block retriever as the 57 // source to read blocks from. 58 func NewReaderUsingRetriever( 59 id ident.ID, 60 retriever QueryableBlockRetriever, 61 onRetrieveBlock block.OnRetrieveBlock, 62 onReadBlock block.OnReadBlock, 63 opts Options, 64 ) Reader { 65 return Reader{ 66 opts: opts, 67 id: id, 68 retriever: retriever, 69 onRetrieve: onRetrieveBlock, 70 onRead: onReadBlock, 71 } 72 } 73 74 // ReadEncoded reads encoded blocks using just a block retriever. 75 func (r *Reader) ReadEncoded( 76 ctx context.Context, 77 start, end xtime.UnixNano, 78 nsCtx namespace.Context, 79 ) (BlockReaderIter, error) { 80 return r.readersWithBlocksMapAndBuffer(ctx, start, end, nil, nil, nsCtx) 81 } 82 83 func (r *Reader) readersWithBlocksMapAndBuffer( 84 ctx context.Context, 85 start, end xtime.UnixNano, 86 seriesBlocks block.DatabaseSeriesBlocks, 87 seriesBuffer databaseBuffer, 88 nsCtx namespace.Context, 89 ) (BlockReaderIter, error) { 90 if end.Before(start) { 91 return nil, xerrors.NewInvalidParamsError(errSeriesReadInvalidRange) 92 } 93 94 var ( 95 nowFn = r.opts.ClockOptions().NowFn() 96 now = xtime.ToUnixNano(nowFn()) 97 ropts = r.opts.RetentionOptions() 98 size = ropts.BlockSize() 99 alignedStart = start.Truncate(size) 100 alignedEnd = end.Truncate(size) 101 ) 102 103 if alignedEnd.Equal(end) { 104 // Move back to make range [start, end) 105 alignedEnd = alignedEnd.Add(-1 * size) 106 } 107 108 // Squeeze the lookup window by what's available to make range queries like [0, infinity) possible 109 earliest := retention.FlushTimeStart(ropts, now) 110 if alignedStart.Before(earliest) { 111 alignedStart = earliest 112 } 113 latest := now.Add(ropts.BufferFuture()).Truncate(size) 114 if alignedEnd.After(latest) { 115 alignedEnd = latest 116 } 117 118 return r.readersWithBlocksMapAndBufferAligned(ctx, alignedStart, alignedEnd, 119 seriesBlocks, seriesBuffer, nsCtx) 120 } 121 122 // BlockReaderIter provides an Iterator interface to a collection of BlockReaders. 123 // 124 // The Iterator allows disk read to be lazily requested when Next() is called. This allows the system to limit an 125 // expensive Query before it issues all its disk reads. 126 // 127 // The Iterator does not hold any locks so it's safe to pause or even abandon iterating through the results. Any 128 // in-memory blocks (cache, series buffer) are eagerly loaded to avoid holding read locks while iterating. 129 // 130 // This iterator is not thread safe and should be only be used by a single go routine. 131 type BlockReaderIter interface { 132 // Next requests the next BlockReaders, returning if any more existed. 133 // The BlockReaders are available with Current. 134 // If an error occurs, false is returned and it is available with Err. 135 Next(ctx context.Context) bool 136 137 // Current returns the current set of BlockReaders for a given blockStart time, guaranteed to be not empty. 138 // Results are returned in block start time asc ordering. Within the same block start time there is no guaranteed 139 // ordering. 140 Current() []xio.BlockReader 141 142 // Err is non-nil if an error occurred when calling Next. 143 Err() error 144 145 // ToSlices eagerly loads all BlockReaders into the legacy slices of slices for backward compatibility. 146 // TODO: remove this and convert everything to the iterator pattern. 147 ToSlices(ctx context.Context) ([][]xio.BlockReader, error) 148 } 149 150 type blockReaderIterOpts struct { 151 start xtime.UnixNano 152 end xtime.UnixNano 153 blockSize time.Duration 154 reader *Reader 155 nsCtx namespace.Context 156 cached []xio.BlockReader 157 buffer [][]xio.BlockReader 158 } 159 160 type blockReaderIter struct { 161 blockReaderIterOpts 162 163 blockAt xtime.UnixNano 164 curr []xio.BlockReader 165 err error 166 } 167 168 func (i *blockReaderIter) Err() error { 169 return i.err 170 } 171 172 func (i *blockReaderIter) Current() []xio.BlockReader { 173 return i.curr 174 } 175 176 // nolint: gocyclo 177 func (i *blockReaderIter) Next(ctx context.Context) bool { 178 if i.blockAt.IsZero() { 179 i.blockAt = i.start 180 } 181 i.curr = make([]xio.BlockReader, 0) 182 for !i.blockAt.After(i.end) && len(i.curr) == 0 { 183 // first checks the blocks from the series buffer at this time. 184 // blocks are sorted by start time so we only need to look at the head. 185 for len(i.buffer) > 0 && len(i.buffer[0]) > 0 && i.buffer[0][0].Start.Equal(i.blockAt) { 186 i.curr = append(i.curr, i.buffer[0][0]) 187 i.buffer[0] = i.buffer[0][1:] 188 if len(i.buffer[0]) == 0 { 189 i.buffer = i.buffer[1:] 190 } 191 } 192 193 // next check for the disk block at this time. 194 // check the cache first. 195 // blocks are sorted by start time so we only need to look at the head. 196 if len(i.cached) > 0 && i.cached[0].Start.Equal(i.blockAt) { 197 i.curr = append(i.curr, i.cached[0]) 198 i.cached = i.cached[1:] 199 } else { 200 // if not in the cache, request a load from disk. 201 blockReader, found, err := i.reader.streamBlock(ctx, i.blockAt, i.reader.onRetrieve, i.nsCtx) 202 if err != nil { 203 i.err = err 204 return false 205 } 206 if found { 207 i.curr = append(i.curr, blockReader) 208 } 209 } 210 i.blockAt = i.blockAt.Add(i.blockSize) 211 } 212 return len(i.curr) != 0 213 } 214 215 func (i *blockReaderIter) ToSlices(ctx context.Context) ([][]xio.BlockReader, error) { 216 var results [][]xio.BlockReader 217 for i.Next(ctx) { 218 results = append(results, i.Current()) 219 } 220 if i.Err() != nil { 221 return nil, i.Err() 222 } 223 return results, nil 224 } 225 226 func (r *Reader) readersWithBlocksMapAndBufferAligned( 227 ctx context.Context, 228 start, end xtime.UnixNano, 229 seriesBlocks block.DatabaseSeriesBlocks, 230 seriesBuffer databaseBuffer, 231 nsCtx namespace.Context, 232 ) (BlockReaderIter, error) { 233 var ( 234 nowFn = r.opts.ClockOptions().NowFn() 235 now = xtime.ToUnixNano(nowFn()) 236 ropts = r.opts.RetentionOptions() 237 blockSize = ropts.BlockSize() 238 readerCount = end.Sub(start) / blockSize 239 buffer [][]xio.BlockReader 240 cached []xio.BlockReader 241 ) 242 243 if readerCount < 0 { 244 readerCount = 0 245 } 246 247 for blockAt := start; !blockAt.After(end); blockAt = blockAt.Add(blockSize) { 248 // Eagerly load the readers from the series buffer and disk cache to avoid holding locks in the iterator. 249 // 250 // It is important to look for data in the series buffer one block at 251 // a time within this loop so that the returned results contain data 252 // from blocks in chronological order. Failure to do this will result 253 // in an out of order error in the MultiReaderIterator on query. 254 255 blockReader, blk, found, err := retrieveCached(ctx, blockAt, seriesBlocks) 256 if err != nil { 257 return nil, err 258 } 259 260 if found { 261 // NB(r): Mark this block as read now 262 blk.SetLastReadTime(now) 263 if r.onRead != nil { 264 r.onRead.OnReadBlock(blk) 265 } 266 cached = append(cached, blockReader) 267 } 268 269 if seriesBuffer != nil { 270 var bufferReaders []xio.BlockReader 271 bufferResults, err := seriesBuffer.ReadEncoded(ctx, blockAt, blockAt.Add(blockSize), nsCtx) 272 if err != nil { 273 return nil, err 274 } 275 276 // Multiple block results may be returned here (for the same block 277 // start) - one for warm writes and another for cold writes. 278 for _, bufferRes := range bufferResults { 279 bufferReaders = append(bufferReaders, bufferRes...) 280 } 281 if len(bufferReaders) > 0 { 282 buffer = append(buffer, bufferReaders) 283 } 284 } 285 } 286 287 return &blockReaderIter{ 288 blockReaderIterOpts: blockReaderIterOpts{ 289 start: start, 290 end: end, 291 blockSize: blockSize, 292 reader: r, 293 nsCtx: nsCtx, 294 buffer: buffer, 295 cached: cached, 296 }, 297 }, nil 298 } 299 300 // FetchBlocks returns data blocks given a list of block start times using 301 // just a block retriever. 302 func (r *Reader) FetchBlocks( 303 ctx context.Context, 304 starts []xtime.UnixNano, 305 nsCtx namespace.Context, 306 ) ([]block.FetchBlockResult, error) { 307 return r.fetchBlocksWithBlocksMapAndBuffer(ctx, starts, nil, nil, nsCtx) 308 } 309 310 func (r *Reader) fetchBlocksWithBlocksMapAndBuffer( 311 ctx context.Context, 312 starts []xtime.UnixNano, 313 seriesBlocks block.DatabaseSeriesBlocks, 314 seriesBuffer databaseBuffer, 315 nsCtx namespace.Context, 316 ) ([]block.FetchBlockResult, error) { 317 res := r.resolveBlockResults(ctx, starts, seriesBlocks, nsCtx) 318 if seriesBuffer != nil && !seriesBuffer.IsEmpty() { 319 bufferResults := seriesBuffer.FetchBlocks(ctx, starts, nsCtx) 320 321 // Ensure both slices are sorted before merging as two sorted lists. 322 block.SortFetchBlockResultByTimeAscending(res) 323 block.SortFetchBlockResultByTimeAscending(bufferResults) 324 bufferIdx := 0 325 for i := range res { 326 blockResult := res[i] 327 if !(bufferIdx < len(bufferResults)) { 328 break 329 } 330 331 currBufferResult := bufferResults[bufferIdx] 332 if blockResult.Start.Equal(currBufferResult.Start) { 333 if currBufferResult.Err != nil { 334 res[i].Err = currBufferResult.Err 335 } else { 336 res[i].Blocks = append(res[i].Blocks, currBufferResult.Blocks...) 337 } 338 bufferIdx++ 339 continue 340 } 341 } 342 343 // Add any buffer results for which there was no existing blockstart 344 // to the end. 345 if bufferIdx < len(bufferResults) { 346 res = append(res, bufferResults[bufferIdx:]...) 347 } 348 } 349 350 // Should still be sorted but do it again for sanity. 351 block.SortFetchBlockResultByTimeAscending(res) 352 return res, nil 353 } 354 355 func (r *Reader) resolveBlockResults( 356 ctx context.Context, 357 starts []xtime.UnixNano, 358 seriesBlocks block.DatabaseSeriesBlocks, 359 nsCtx namespace.Context, 360 ) []block.FetchBlockResult { 361 // Two-dimensional slice (each block.FetchBlockResult has a []xio.BlockReader internally) 362 // such that the first dimension is unique by blockstart and the second dimension is blocks 363 // of data for that blockstart (not necessarily in chronological order). 364 // 365 // ex. (querying 2P.M -> 6P.M with a 2-hour blocksize): 366 // []block.FetchBlockResult{ 367 // block.FetchBlockResult{ 368 // Start: 2P.M, 369 // Blocks: []xio.BlockReader{block0, block1, block2}, 370 // }, 371 // block.FetchBlockResult{ 372 // Start: 4P.M, 373 // Blocks: []xio.BlockReader{block0}, 374 // }, 375 // } 376 res := make([]block.FetchBlockResult, 0, len(starts)) 377 for _, start := range starts { 378 // Slice of xio.BlockReader such that all data belong to the same blockstart. 379 var blockReaders []xio.BlockReader 380 381 blockReader, _, found, err := retrieveCached(ctx, start, seriesBlocks) 382 if err != nil { 383 // Short-circuit this entire blockstart if an error was encountered. 384 r := block.NewFetchBlockResult(start, nil, 385 fmt.Errorf("unable to retrieve block stream for series %s time %v: %w", 386 r.id.String(), start, err)) 387 res = append(res, r) 388 continue 389 } 390 391 if !found { 392 // NB(r): Always use nil for OnRetrieveBlock so we don't cache the 393 // series after fetching it from disk, the fetch blocks API is called 394 // during streaming so to cache it in memory would mean we would 395 // eventually cache all series in memory when we stream results to a 396 // peer. 397 blockReader, found, err = r.streamBlock(ctx, start, nil, nsCtx) 398 if err != nil { 399 // Short-circuit this entire blockstart if an error was encountered. 400 r := block.NewFetchBlockResult(start, nil, 401 fmt.Errorf("unable to retrieve block stream for series %s time %v: %w", 402 r.id.String(), start, err)) 403 res = append(res, r) 404 continue 405 } 406 } 407 408 if found { 409 blockReaders = append(blockReaders, blockReader) 410 } 411 412 if len(blockReaders) > 0 { 413 res = append(res, block.NewFetchBlockResult(start, blockReaders, nil)) 414 } 415 } 416 417 return res 418 } 419 420 func retrieveCached( 421 ctx context.Context, 422 start xtime.UnixNano, 423 seriesBlocks block.DatabaseSeriesBlocks, 424 ) (xio.BlockReader, block.DatabaseBlock, bool, error) { 425 if seriesBlocks != nil { 426 if b, exists := seriesBlocks.BlockAt(start); exists { 427 streamedBlock, err := b.Stream(ctx) 428 if err != nil { 429 return xio.BlockReader{}, b, false, err 430 } 431 432 if streamedBlock.IsNotEmpty() { 433 return streamedBlock, b, true, nil 434 } 435 } 436 } 437 438 return xio.BlockReader{}, nil, false, nil 439 } 440 441 func (r *Reader) streamBlock( 442 ctx context.Context, 443 start xtime.UnixNano, 444 onRetrieve block.OnRetrieveBlock, 445 nsCtx namespace.Context, 446 ) (xio.BlockReader, bool, error) { 447 cachePolicy := r.opts.CachePolicy() 448 switch { 449 case cachePolicy == CacheAll: 450 // No-op, block metadata should have been in-memory 451 case r.retriever != nil: 452 // Try to stream from disk 453 isRetrievable, err := r.retriever.IsBlockRetrievable(start) 454 if err != nil { 455 return xio.BlockReader{}, false, err 456 } 457 458 if isRetrievable { 459 streamedBlock, err := r.retriever.Stream(ctx, r.id, start, onRetrieve, nsCtx) 460 if err != nil { 461 // Short-circuit this entire blockstart if an error was encountered. 462 return xio.BlockReader{}, false, err 463 } 464 465 if streamedBlock.IsNotEmpty() { 466 return streamedBlock, true, nil 467 } 468 } 469 } 470 471 return xio.BlockReader{}, false, nil 472 }