github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/rowcontainer/row_container.go (about) 1 // Copyright 2016 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package rowcontainer 12 13 import ( 14 "container/heap" 15 "context" 16 "fmt" 17 "unsafe" 18 19 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/diskmap" 20 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 21 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 22 "github.com/cockroachdb/cockroach/pkg/sql/types" 23 "github.com/cockroachdb/cockroach/pkg/util/log" 24 "github.com/cockroachdb/cockroach/pkg/util/mon" 25 "github.com/cockroachdb/cockroach/pkg/util/ring" 26 "github.com/cockroachdb/errors" 27 ) 28 29 // SortableRowContainer is a container used to store rows and optionally sort 30 // these. 31 type SortableRowContainer interface { 32 Len() int 33 AddRow(context.Context, sqlbase.EncDatumRow) error 34 // Sort sorts the rows according to the current ordering (the one set either 35 // at initialization or by the last call of Reorder() - if the container is 36 // ReorderableRowContainer). 37 Sort(context.Context) 38 // NewIterator returns a RowIterator that can be used to iterate over 39 // the rows. 40 NewIterator(context.Context) RowIterator 41 // NewFinalIterator returns a RowIterator that can be used to iterate over the 42 // rows, possibly freeing resources along the way. Subsequent calls to 43 // NewIterator or NewFinalIterator are not guaranteed to return any rows. 44 NewFinalIterator(context.Context) RowIterator 45 46 // UnsafeReset resets the container, allowing for reuse. It renders all 47 // previously allocated rows unsafe. 48 UnsafeReset(context.Context) error 49 50 // InitTopK enables optimizations in cases where the caller cares only about 51 // the top k rows where k is the size of the SortableRowContainer when 52 // InitTopK is called. Once InitTopK is called, callers should not call 53 // AddRow. Iterators created after calling InitTopK are guaranteed to read the 54 // top k rows only. 55 InitTopK() 56 // MaybeReplaceMax checks whether the given row belongs in the top k rows, 57 // potentially evicting a row in favor of the given row. 58 MaybeReplaceMax(context.Context, sqlbase.EncDatumRow) error 59 60 // Close frees up resources held by the SortableRowContainer. 61 Close(context.Context) 62 } 63 64 // ReorderableRowContainer is a SortableRowContainer that can change the 65 // ordering on which the rows are sorted. 66 type ReorderableRowContainer interface { 67 SortableRowContainer 68 69 // Reorder changes the ordering on which the rows are sorted. In order for 70 // new ordering to take effect, Sort() must be called. It returns an error if 71 // it occurs. 72 Reorder(context.Context, sqlbase.ColumnOrdering) error 73 } 74 75 // IndexedRowContainer is a ReorderableRowContainer which also implements 76 // tree.IndexedRows. It allows retrieving a row at a particular index. 77 type IndexedRowContainer interface { 78 ReorderableRowContainer 79 80 // GetRow returns a row at the given index or an error. 81 GetRow(ctx context.Context, idx int) (tree.IndexedRow, error) 82 } 83 84 // DeDupingRowContainer is a container that de-duplicates rows added to the 85 // container, and assigns them a dense index starting from 0, representing 86 // when that row was first added. It only supports a configuration where all 87 // the columns are encoded into the key -- relaxing this is not hard, but is 88 // not worth adding the code without a use for it. 89 type DeDupingRowContainer interface { 90 // AddRowWithDeDup adds the given row if not already present in the 91 // container. It returns the dense number of when the row is first 92 // added. 93 AddRowWithDeDup(context.Context, sqlbase.EncDatumRow) (int, error) 94 // UnsafeReset resets the container, allowing for reuse. It renders all 95 // previously allocated rows unsafe. 96 UnsafeReset(context.Context) error 97 // Close frees up resources held by the container. 98 Close(context.Context) 99 } 100 101 // RowIterator is a simple iterator used to iterate over sqlbase.EncDatumRows. 102 // Example use: 103 // var i RowIterator 104 // for i.Rewind(); ; i.Next() { 105 // if ok, err := i.Valid(); err != nil { 106 // // Handle error. 107 // } else if !ok { 108 // break 109 // } 110 // row, err := i.Row() 111 // if err != nil { 112 // // Handle error. 113 // } 114 // // Do something. 115 // } 116 // 117 type RowIterator interface { 118 // Rewind seeks to the first row. 119 Rewind() 120 // Valid must be called after any call to Rewind() or Next(). It returns 121 // (true, nil) if the iterator points to a valid row and (false, nil) if the 122 // iterator has moved past the last row. 123 // If an error has occurred, the returned bool is invalid. 124 Valid() (bool, error) 125 // Next advances the iterator to the next row in the iteration. 126 Next() 127 // Row returns the current row. The returned row is only valid until the 128 // next call to Rewind() or Next(). 129 Row() (sqlbase.EncDatumRow, error) 130 131 // Close frees up resources held by the iterator. 132 Close() 133 } 134 135 // MemRowContainer is the wrapper around rowcontainer.RowContainer that 136 // provides more functionality, especially around converting to/from 137 // EncDatumRows and facilitating sorting. 138 type MemRowContainer struct { 139 RowContainer 140 types []*types.T 141 invertSorting bool // Inverts the sorting predicate. 142 ordering sqlbase.ColumnOrdering 143 scratchRow tree.Datums 144 scratchEncRow sqlbase.EncDatumRow 145 146 evalCtx *tree.EvalContext 147 148 datumAlloc sqlbase.DatumAlloc 149 } 150 151 var _ heap.Interface = &MemRowContainer{} 152 var _ IndexedRowContainer = &MemRowContainer{} 153 154 // Init initializes the MemRowContainer. The MemRowContainer uses evalCtx.Mon 155 // to track memory usage. 156 func (mc *MemRowContainer) Init( 157 ordering sqlbase.ColumnOrdering, types []*types.T, evalCtx *tree.EvalContext, 158 ) { 159 mc.InitWithMon(ordering, types, evalCtx, evalCtx.Mon, 0 /* rowCapacity */) 160 } 161 162 // InitWithMon initializes the MemRowContainer with an explicit monitor. Only 163 // use this if the default MemRowContainer.Init() function is insufficient. 164 func (mc *MemRowContainer) InitWithMon( 165 ordering sqlbase.ColumnOrdering, 166 types []*types.T, 167 evalCtx *tree.EvalContext, 168 mon *mon.BytesMonitor, 169 rowCapacity int, 170 ) { 171 acc := mon.MakeBoundAccount() 172 mc.RowContainer.Init(acc, sqlbase.ColTypeInfoFromColTypes(types), rowCapacity) 173 mc.types = types 174 mc.ordering = ordering 175 mc.scratchRow = make(tree.Datums, len(types)) 176 mc.scratchEncRow = make(sqlbase.EncDatumRow, len(types)) 177 mc.evalCtx = evalCtx 178 } 179 180 // Types returns the MemRowContainer's types. 181 func (mc *MemRowContainer) Types() []*types.T { 182 return mc.types 183 } 184 185 // Less is part of heap.Interface and is only meant to be used internally. 186 func (mc *MemRowContainer) Less(i, j int) bool { 187 cmp := sqlbase.CompareDatums(mc.ordering, mc.evalCtx, mc.At(i), mc.At(j)) 188 if mc.invertSorting { 189 cmp = -cmp 190 } 191 return cmp < 0 192 } 193 194 // EncRow returns the idx-th row as an EncDatumRow. The slice itself is reused 195 // so it is only valid until the next call to EncRow. 196 func (mc *MemRowContainer) EncRow(idx int) sqlbase.EncDatumRow { 197 datums := mc.At(idx) 198 for i, d := range datums { 199 mc.scratchEncRow[i] = sqlbase.DatumToEncDatum(mc.types[i], d) 200 } 201 return mc.scratchEncRow 202 } 203 204 // AddRow adds a row to the container. 205 func (mc *MemRowContainer) AddRow(ctx context.Context, row sqlbase.EncDatumRow) error { 206 if len(row) != len(mc.types) { 207 log.Fatalf(ctx, "invalid row length %d, expected %d", len(row), len(mc.types)) 208 } 209 for i := range row { 210 err := row[i].EnsureDecoded(mc.types[i], &mc.datumAlloc) 211 if err != nil { 212 return err 213 } 214 mc.scratchRow[i] = row[i].Datum 215 } 216 _, err := mc.RowContainer.AddRow(ctx, mc.scratchRow) 217 return err 218 } 219 220 // Sort is part of the SortableRowContainer interface. 221 func (mc *MemRowContainer) Sort(ctx context.Context) { 222 mc.invertSorting = false 223 cancelChecker := sqlbase.NewCancelChecker(ctx) 224 sqlbase.Sort(mc, cancelChecker) 225 } 226 227 // Reorder implements ReorderableRowContainer. We don't need to create a new 228 // MemRowContainer and can just change the ordering on-the-fly. 229 func (mc *MemRowContainer) Reorder(_ context.Context, ordering sqlbase.ColumnOrdering) error { 230 mc.ordering = ordering 231 return nil 232 } 233 234 // Push is part of heap.Interface. 235 func (mc *MemRowContainer) Push(_ interface{}) { panic("unimplemented") } 236 237 // Pop is part of heap.Interface. 238 func (mc *MemRowContainer) Pop() interface{} { panic("unimplemented") } 239 240 // MaybeReplaceMax replaces the maximum element with the given row, if it is 241 // smaller. Assumes InitTopK was called. 242 func (mc *MemRowContainer) MaybeReplaceMax(ctx context.Context, row sqlbase.EncDatumRow) error { 243 max := mc.At(0) 244 cmp, err := row.CompareToDatums(mc.types, &mc.datumAlloc, mc.ordering, mc.evalCtx, max) 245 if err != nil { 246 return err 247 } 248 if cmp < 0 { 249 // row is smaller than the max; replace. 250 for i := range row { 251 if err := row[i].EnsureDecoded(mc.types[i], &mc.datumAlloc); err != nil { 252 return err 253 } 254 mc.scratchRow[i] = row[i].Datum 255 } 256 if err := mc.Replace(ctx, 0, mc.scratchRow); err != nil { 257 return err 258 } 259 heap.Fix(mc, 0) 260 } 261 return nil 262 } 263 264 // InitTopK rearranges the rows in the MemRowContainer into a Max-Heap. 265 func (mc *MemRowContainer) InitTopK() { 266 mc.invertSorting = true 267 heap.Init(mc) 268 } 269 270 // memRowIterator is a RowIterator that iterates over a MemRowContainer. This 271 // iterator doesn't iterate over a snapshot of MemRowContainer. 272 type memRowIterator struct { 273 *MemRowContainer 274 curIdx int 275 } 276 277 var _ RowIterator = &memRowIterator{} 278 279 // NewIterator returns an iterator that can be used to iterate over a 280 // MemRowContainer. Note that this iterator doesn't iterate over a snapshot 281 // of MemRowContainer. 282 func (mc *MemRowContainer) NewIterator(_ context.Context) RowIterator { 283 return &memRowIterator{MemRowContainer: mc} 284 } 285 286 // Rewind implements the RowIterator interface. 287 func (i *memRowIterator) Rewind() { 288 i.curIdx = 0 289 } 290 291 // Valid implements the RowIterator interface. 292 func (i *memRowIterator) Valid() (bool, error) { 293 return i.curIdx < i.Len(), nil 294 } 295 296 // Next implements the RowIterator interface. 297 func (i *memRowIterator) Next() { 298 i.curIdx++ 299 } 300 301 // Row implements the RowIterator interface. 302 func (i *memRowIterator) Row() (sqlbase.EncDatumRow, error) { 303 return i.EncRow(i.curIdx), nil 304 } 305 306 // Close implements the RowIterator interface. 307 func (i *memRowIterator) Close() {} 308 309 // memRowFinalIterator is a RowIterator that iterates over a MemRowContainer. 310 // This iterator doesn't iterate over a snapshot of MemRowContainer and deletes 311 // rows as soon as they are iterated over to free up memory eagerly. 312 type memRowFinalIterator struct { 313 *MemRowContainer 314 } 315 316 // NewFinalIterator returns an iterator that can be used to iterate over a 317 // MemRowContainer. Note that this iterator doesn't iterate over a snapshot 318 // of MemRowContainer and that it deletes rows as soon as they are iterated 319 // over. 320 func (mc *MemRowContainer) NewFinalIterator(_ context.Context) RowIterator { 321 return memRowFinalIterator{MemRowContainer: mc} 322 } 323 324 // GetRow implements IndexedRowContainer. 325 func (mc *MemRowContainer) GetRow(ctx context.Context, pos int) (tree.IndexedRow, error) { 326 return IndexedRow{Idx: pos, Row: mc.EncRow(pos)}, nil 327 } 328 329 var _ RowIterator = memRowFinalIterator{} 330 331 // Rewind implements the RowIterator interface. 332 func (i memRowFinalIterator) Rewind() {} 333 334 // Valid implements the RowIterator interface. 335 func (i memRowFinalIterator) Valid() (bool, error) { 336 return i.Len() > 0, nil 337 } 338 339 // Next implements the RowIterator interface. 340 func (i memRowFinalIterator) Next() { 341 i.PopFirst() 342 } 343 344 // Row implements the RowIterator interface. 345 func (i memRowFinalIterator) Row() (sqlbase.EncDatumRow, error) { 346 return i.EncRow(0), nil 347 } 348 349 // Close implements the RowIterator interface. 350 func (i memRowFinalIterator) Close() {} 351 352 // DiskBackedRowContainer is a ReorderableRowContainer that uses a 353 // MemRowContainer to store rows and spills back to disk automatically if 354 // memory usage exceeds a given budget. 355 type DiskBackedRowContainer struct { 356 // src is the current ReorderableRowContainer that is being used to store 357 // rows. All the ReorderableRowContainer methods are redefined rather than 358 // delegated to an embedded struct because of how defer works: 359 // rc.Init(...) 360 // defer rc.Close(ctx) 361 // The Close will call MemRowContainer.Close(ctx) even after spilling to disk. 362 src ReorderableRowContainer 363 364 mrc *MemRowContainer 365 drc *DiskRowContainer 366 367 // See comment in DoDeDuplicate(). 368 deDuplicate bool 369 keyToIndex map[string]int 370 // Encoding helpers for de-duplication: 371 // encodings keeps around the DatumEncoding equivalents of the encoding 372 // directions in ordering to avoid conversions in hot paths. 373 encodings []sqlbase.DatumEncoding 374 datumAlloc sqlbase.DatumAlloc 375 scratchKey []byte 376 377 spilled bool 378 379 // The following fields are used to create a DiskRowContainer when spilling 380 // to disk. 381 engine diskmap.Factory 382 diskMonitor *mon.BytesMonitor 383 } 384 385 var _ ReorderableRowContainer = &DiskBackedRowContainer{} 386 var _ DeDupingRowContainer = &DiskBackedRowContainer{} 387 388 // Init initializes a DiskBackedRowContainer. 389 // Arguments: 390 // - ordering is the output ordering; the order in which rows should be sorted. 391 // - types is the schema of rows that will be added to this container. 392 // - evalCtx defines the context in which to evaluate comparisons, only used 393 // when storing rows in memory. 394 // - engine is the store used for rows when spilling to disk. 395 // - memoryMonitor is used to monitor the DiskBackedRowContainer's memory usage. 396 // If this monitor denies an allocation, the DiskBackedRowContainer will 397 // spill to disk. 398 // - diskMonitor is used to monitor the DiskBackedRowContainer's disk usage if 399 // and when it spills to disk. 400 // - rowCapacity (if not 0) indicates the number of rows that the underlying 401 // in-memory container should be preallocated for. 402 func (f *DiskBackedRowContainer) Init( 403 ordering sqlbase.ColumnOrdering, 404 types []*types.T, 405 evalCtx *tree.EvalContext, 406 engine diskmap.Factory, 407 memoryMonitor *mon.BytesMonitor, 408 diskMonitor *mon.BytesMonitor, 409 rowCapacity int, 410 ) { 411 mrc := MemRowContainer{} 412 mrc.InitWithMon(ordering, types, evalCtx, memoryMonitor, rowCapacity) 413 f.mrc = &mrc 414 f.src = &mrc 415 f.engine = engine 416 f.diskMonitor = diskMonitor 417 f.encodings = make([]sqlbase.DatumEncoding, len(ordering)) 418 for i, orderInfo := range ordering { 419 f.encodings[i] = sqlbase.EncodingDirToDatumEncoding(orderInfo.Direction) 420 } 421 } 422 423 // DoDeDuplicate causes DiskBackedRowContainer to behave as an implementation 424 // of DeDupingRowContainer. It should not be mixed with calls to AddRow(). It 425 // de-duplicates the keys such that only the first row with the given key will 426 // be stored. The index returned in AddRowWithDedup() is a dense index 427 // starting from 0, representing when that key was first added. This feature 428 // does not combine with Sort(), Reorder() etc., and only to be used for 429 // assignment of these dense indexes. The main reason to add this to 430 // DiskBackedRowContainer is to avoid significant code duplication in 431 // constructing another row container. 432 func (f *DiskBackedRowContainer) DoDeDuplicate() { 433 f.deDuplicate = true 434 f.keyToIndex = make(map[string]int) 435 } 436 437 // Len is part of the SortableRowContainer interface. 438 func (f *DiskBackedRowContainer) Len() int { 439 return f.src.Len() 440 } 441 442 // AddRow is part of the SortableRowContainer interface. 443 func (f *DiskBackedRowContainer) AddRow(ctx context.Context, row sqlbase.EncDatumRow) error { 444 if err := f.src.AddRow(ctx, row); err != nil { 445 if spilled, spillErr := f.spillIfMemErr(ctx, err); !spilled && spillErr == nil { 446 // The error was not an out of memory error. 447 return err 448 } else if spillErr != nil { 449 // A disk spill was attempted but there was an error in doing so. 450 return spillErr 451 } 452 // Add the row that caused the memory error. 453 return f.src.AddRow(ctx, row) 454 } 455 return nil 456 } 457 458 // AddRowWithDeDup is part of the DeDupingRowContainer interface. 459 func (f *DiskBackedRowContainer) AddRowWithDeDup( 460 ctx context.Context, row sqlbase.EncDatumRow, 461 ) (int, error) { 462 if !f.UsingDisk() { 463 if err := f.encodeKey(ctx, row); err != nil { 464 return 0, err 465 } 466 encodedStr := string(f.scratchKey) 467 idx, ok := f.keyToIndex[encodedStr] 468 if ok { 469 return idx, nil 470 } 471 idx = f.Len() 472 if err := f.AddRow(ctx, row); err != nil { 473 return 0, err 474 } 475 // AddRow may have spilled and deleted the map. 476 if !f.UsingDisk() { 477 f.keyToIndex[encodedStr] = idx 478 } 479 return idx, nil 480 } 481 // Using disk. 482 return f.drc.AddRowWithDeDup(ctx, row) 483 } 484 485 func (f *DiskBackedRowContainer) encodeKey(ctx context.Context, row sqlbase.EncDatumRow) error { 486 if len(row) != len(f.mrc.types) { 487 log.Fatalf(ctx, "invalid row length %d, expected %d", len(row), len(f.mrc.types)) 488 } 489 f.scratchKey = f.scratchKey[:0] 490 for i, orderInfo := range f.mrc.ordering { 491 col := orderInfo.ColIdx 492 var err error 493 f.scratchKey, err = row[col].Encode(f.mrc.types[col], &f.datumAlloc, f.encodings[i], f.scratchKey) 494 if err != nil { 495 return err 496 } 497 } 498 return nil 499 } 500 501 // Sort is part of the SortableRowContainer interface. 502 func (f *DiskBackedRowContainer) Sort(ctx context.Context) { 503 f.src.Sort(ctx) 504 } 505 506 // Reorder implements ReorderableRowContainer. 507 func (f *DiskBackedRowContainer) Reorder( 508 ctx context.Context, ordering sqlbase.ColumnOrdering, 509 ) error { 510 return f.src.Reorder(ctx, ordering) 511 } 512 513 // InitTopK is part of the SortableRowContainer interface. 514 func (f *DiskBackedRowContainer) InitTopK() { 515 f.src.InitTopK() 516 } 517 518 // MaybeReplaceMax is part of the SortableRowContainer interface. 519 func (f *DiskBackedRowContainer) MaybeReplaceMax( 520 ctx context.Context, row sqlbase.EncDatumRow, 521 ) error { 522 return f.src.MaybeReplaceMax(ctx, row) 523 } 524 525 // NewIterator is part of the SortableRowContainer interface. 526 func (f *DiskBackedRowContainer) NewIterator(ctx context.Context) RowIterator { 527 return f.src.NewIterator(ctx) 528 } 529 530 // NewFinalIterator is part of the SortableRowContainer interface. 531 func (f *DiskBackedRowContainer) NewFinalIterator(ctx context.Context) RowIterator { 532 return f.src.NewFinalIterator(ctx) 533 } 534 535 // UnsafeReset resets the container for reuse. The DiskBackedRowContainer will 536 // reset to use memory if it is using disk. 537 func (f *DiskBackedRowContainer) UnsafeReset(ctx context.Context) error { 538 if f.deDuplicate { 539 f.keyToIndex = make(map[string]int) 540 } 541 if f.drc != nil { 542 f.drc.Close(ctx) 543 f.src = f.mrc 544 f.drc = nil 545 return nil 546 } 547 return f.mrc.UnsafeReset(ctx) 548 } 549 550 // Close is part of the SortableRowContainer interface. 551 func (f *DiskBackedRowContainer) Close(ctx context.Context) { 552 if f.drc != nil { 553 f.drc.Close(ctx) 554 } 555 f.mrc.Close(ctx) 556 if f.deDuplicate { 557 f.keyToIndex = nil 558 } 559 } 560 561 // Spilled returns whether or not the DiskBackedRowContainer spilled to disk 562 // in its lifetime. 563 func (f *DiskBackedRowContainer) Spilled() bool { 564 return f.spilled 565 } 566 567 // UsingDisk returns whether or not the DiskBackedRowContainer is currently 568 // using disk. 569 func (f *DiskBackedRowContainer) UsingDisk() bool { 570 return f.drc != nil 571 } 572 573 // spillIfMemErr checks err and calls SpillToDisk if the given err is an out of 574 // memory error. Returns whether the DiskBackedRowContainer spilled to disk and 575 // an error if one occurred while doing so. 576 func (f *DiskBackedRowContainer) spillIfMemErr(ctx context.Context, err error) (bool, error) { 577 if !sqlbase.IsOutOfMemoryError(err) { 578 return false, nil 579 } 580 if spillErr := f.SpillToDisk(ctx); spillErr != nil { 581 return false, spillErr 582 } 583 log.VEventf(ctx, 2, "spilled to disk: %v", err) 584 return true, nil 585 } 586 587 // SpillToDisk creates a disk row container, injects all the data from the 588 // in-memory container into it, and clears the in-memory one afterwards. 589 func (f *DiskBackedRowContainer) SpillToDisk(ctx context.Context) error { 590 if f.UsingDisk() { 591 return errors.New("already using disk") 592 } 593 drc := MakeDiskRowContainer(f.diskMonitor, f.mrc.types, f.mrc.ordering, f.engine) 594 if f.deDuplicate { 595 drc.DoDeDuplicate() 596 // After spilling to disk we don't need this map to de-duplicate. The 597 // DiskRowContainer will do the de-duplication. Calling AddRow() below 598 // is correct since these rows are already de-duplicated. 599 f.keyToIndex = nil 600 } 601 i := f.mrc.NewFinalIterator(ctx) 602 defer i.Close() 603 for i.Rewind(); ; i.Next() { 604 if ok, err := i.Valid(); err != nil { 605 return err 606 } else if !ok { 607 break 608 } 609 memRow, err := i.Row() 610 if err != nil { 611 return err 612 } 613 if err := drc.AddRow(ctx, memRow); err != nil { 614 return err 615 } 616 } 617 f.mrc.Clear(ctx) 618 619 f.src = &drc 620 f.drc = &drc 621 f.spilled = true 622 return nil 623 } 624 625 // DiskBackedIndexedRowContainer is a wrapper around DiskBackedRowContainer 626 // that adds an index to each row added in the order of addition of those rows 627 // by storing an extra int column at the end of each row. These indices can be 628 // thought of as ordinals of the rows. 629 // 630 // Note: although DiskRowContainer appends unique rowIDs to the keys that the 631 // rows are put at, MemRowContainer doesn't do something like that, so the code 632 // that utilizes internal rowIDs of DiskRowContainer ends up being worse than 633 // having this specialized container. 634 type DiskBackedIndexedRowContainer struct { 635 *DiskBackedRowContainer 636 637 scratchEncRow sqlbase.EncDatumRow 638 storedTypes []*types.T 639 datumAlloc sqlbase.DatumAlloc 640 rowAlloc sqlbase.EncDatumRowAlloc 641 idx uint64 // the index of the next row to be added into the container 642 643 // These fields are for optimizations when container spilled to disk. 644 diskRowIter RowIterator 645 idxRowIter int 646 // nextPosToCache is the index of the row to be cached next. If it is greater 647 // than 0, the cache contains all rows with position in the range 648 // [firstCachedRowPos, nextPosToCache). 649 firstCachedRowPos int 650 nextPosToCache int 651 // indexedRowsCache is the cache of up to maxCacheSize contiguous rows. 652 indexedRowsCache ring.Buffer 653 // maxCacheSize indicates the maximum number of rows to be cached. It is 654 // initialized to maxIndexedRowsCacheSize and dynamically adjusted if OOM 655 // error is encountered. 656 maxCacheSize int 657 cacheMemAcc mon.BoundAccount 658 hitCount int 659 missCount int 660 661 // DisableCache is intended for testing only. It can be set to true to 662 // disable reading and writing from the row cache. 663 DisableCache bool 664 } 665 666 var _ IndexedRowContainer = &DiskBackedIndexedRowContainer{} 667 668 // NewDiskBackedIndexedRowContainer creates a DiskBackedIndexedRowContainer 669 // with the given engine as the underlying store that rows are stored on when 670 // it spills to disk. 671 // Arguments: 672 // - ordering is the output ordering; the order in which rows should be sorted. 673 // - types is the schema of rows that will be added to this container. 674 // - evalCtx defines the context in which to evaluate comparisons, only used 675 // when storing rows in memory. 676 // - engine is the underlying store that rows are stored on when the container 677 // spills to disk. 678 // - memoryMonitor is used to monitor this container's memory usage. 679 // - diskMonitor is used to monitor this container's disk usage. 680 // - rowCapacity (if not 0) specifies the number of rows in-memory container 681 // should be preallocated for. 682 func NewDiskBackedIndexedRowContainer( 683 ordering sqlbase.ColumnOrdering, 684 typs []*types.T, 685 evalCtx *tree.EvalContext, 686 engine diskmap.Factory, 687 memoryMonitor *mon.BytesMonitor, 688 diskMonitor *mon.BytesMonitor, 689 rowCapacity int, 690 ) *DiskBackedIndexedRowContainer { 691 d := DiskBackedIndexedRowContainer{} 692 693 // We will be storing an index of each row as the last INT column. 694 d.storedTypes = make([]*types.T, len(typs)+1) 695 copy(d.storedTypes, typs) 696 d.storedTypes[len(d.storedTypes)-1] = types.Int 697 d.scratchEncRow = make(sqlbase.EncDatumRow, len(d.storedTypes)) 698 d.DiskBackedRowContainer = &DiskBackedRowContainer{} 699 d.DiskBackedRowContainer.Init(ordering, d.storedTypes, evalCtx, engine, memoryMonitor, diskMonitor, rowCapacity) 700 d.maxCacheSize = maxIndexedRowsCacheSize 701 d.cacheMemAcc = memoryMonitor.MakeBoundAccount() 702 return &d 703 } 704 705 // AddRow implements SortableRowContainer. 706 func (f *DiskBackedIndexedRowContainer) AddRow(ctx context.Context, row sqlbase.EncDatumRow) error { 707 copy(f.scratchEncRow, row) 708 f.scratchEncRow[len(f.scratchEncRow)-1] = sqlbase.DatumToEncDatum( 709 types.Int, 710 tree.NewDInt(tree.DInt(f.idx)), 711 ) 712 f.idx++ 713 return f.DiskBackedRowContainer.AddRow(ctx, f.scratchEncRow) 714 } 715 716 // Reorder implements ReorderableRowContainer. 717 func (f *DiskBackedIndexedRowContainer) Reorder( 718 ctx context.Context, ordering sqlbase.ColumnOrdering, 719 ) error { 720 if err := f.DiskBackedRowContainer.Reorder(ctx, ordering); err != nil { 721 return err 722 } 723 f.resetCache(ctx) 724 f.resetIterator() 725 return nil 726 } 727 728 // resetCache resets cache-related fields allowing for reusing the underlying 729 // already allocated memory. Since all rows in the cache are flushed, it also 730 // clears the corresponding memory account. 731 func (f *DiskBackedIndexedRowContainer) resetCache(ctx context.Context) { 732 f.firstCachedRowPos = 0 733 f.nextPosToCache = 0 734 f.indexedRowsCache.Reset() 735 f.cacheMemAcc.Clear(ctx) 736 } 737 738 func (f *DiskBackedIndexedRowContainer) resetIterator() { 739 if f.diskRowIter != nil { 740 f.diskRowIter.Close() 741 f.diskRowIter = nil 742 f.idxRowIter = 0 743 } 744 } 745 746 // UnsafeReset resets the underlying container (if it is using disk, it will be 747 // reset to using memory). 748 func (f *DiskBackedIndexedRowContainer) UnsafeReset(ctx context.Context) error { 749 f.resetCache(ctx) 750 f.resetIterator() 751 f.idx = 0 752 return f.DiskBackedRowContainer.UnsafeReset(ctx) 753 } 754 755 // Close implements SortableRowContainer. 756 func (f *DiskBackedIndexedRowContainer) Close(ctx context.Context) { 757 if f.diskRowIter != nil { 758 f.diskRowIter.Close() 759 } 760 f.cacheMemAcc.Close(ctx) 761 f.DiskBackedRowContainer.Close(ctx) 762 } 763 764 const maxIndexedRowsCacheSize = 4096 765 766 // GetRow implements tree.IndexedRows. 767 // 768 // Getting a row by index is fast from an in-memory row container but is a lot 769 // slower from a disk-backed one. In order to mitigate the impact we add 770 // optimizations of maintaining a cache of tree.IndexedRow's and storing a disk 771 // iterator along with the index of the row it currently points at. 772 func (f *DiskBackedIndexedRowContainer) GetRow( 773 ctx context.Context, pos int, 774 ) (tree.IndexedRow, error) { 775 var rowWithIdx sqlbase.EncDatumRow 776 var err error 777 if f.UsingDisk() { 778 if f.DisableCache { 779 return f.getRowWithoutCache(ctx, pos), nil 780 } 781 // The cache contains all contiguous rows up to the biggest pos requested 782 // so far (even if the rows were not requested explicitly). For example, 783 // if the cache is empty and the request comes for a row at pos 3, the 784 // cache will contain 4 rows at positions 0, 1, 2, and 3. 785 if pos >= f.firstCachedRowPos && pos < f.nextPosToCache { 786 requestedRowCachePos := pos - f.firstCachedRowPos 787 f.hitCount++ 788 return f.indexedRowsCache.Get(requestedRowCachePos).(tree.IndexedRow), nil 789 } 790 f.missCount++ 791 if f.diskRowIter == nil { 792 f.diskRowIter = f.DiskBackedRowContainer.drc.NewIterator(ctx) 793 f.diskRowIter.Rewind() 794 } 795 if f.idxRowIter > pos { 796 // The iterator has been advanced further than we need, so we need to 797 // start iterating from the beginning. 798 log.VEventf(ctx, 1, "rewinding: cache contains indices [%d, %d) but index %d requested", f.firstCachedRowPos, f.nextPosToCache, pos) 799 f.idxRowIter = 0 800 f.diskRowIter.Rewind() 801 f.resetCache(ctx) 802 if pos-maxIndexedRowsCacheSize > f.nextPosToCache { 803 // The requested pos is further away from the beginning of the 804 // container for the cache to hold all the rows up to pos, so we need 805 // to skip exactly pos-maxIndexedRowsCacheSize of them. 806 f.nextPosToCache = pos - maxIndexedRowsCacheSize 807 f.firstCachedRowPos = f.nextPosToCache 808 } 809 } 810 for ; ; f.diskRowIter.Next() { 811 if ok, err := f.diskRowIter.Valid(); err != nil { 812 return nil, err 813 } else if !ok { 814 return nil, errors.Errorf("row at pos %d not found", pos) 815 } 816 if f.idxRowIter == f.nextPosToCache { 817 rowWithIdx, err = f.diskRowIter.Row() 818 if err != nil { 819 return nil, err 820 } 821 for i := range rowWithIdx { 822 if err := rowWithIdx[i].EnsureDecoded(f.storedTypes[i], &f.datumAlloc); err != nil { 823 return nil, err 824 } 825 } 826 row, rowIdx := rowWithIdx[:len(rowWithIdx)-1], rowWithIdx[len(rowWithIdx)-1].Datum 827 if idx, ok := rowIdx.(*tree.DInt); ok { 828 if f.indexedRowsCache.Len() == f.maxCacheSize { 829 // The cache size is capped at f.maxCacheSize, so we reuse the row 830 // with the smallest pos, put it as the last row, and advance 831 // f.firstCachedRowPos. 832 if err := f.reuseFirstRowInCache(ctx, int(*idx), row); err != nil { 833 return nil, err 834 } 835 } else { 836 // We choose to ignore minor details like IndexedRow overhead and 837 // the cache overhead. 838 usage := sizeOfInt + int64(row.Size()) 839 if err := f.cacheMemAcc.Grow(ctx, usage); err != nil { 840 if sqlbase.IsOutOfMemoryError(err) { 841 // We hit the memory limit, so we need to cap the cache size 842 // and reuse the memory underlying first row in the cache. 843 if f.indexedRowsCache.Len() == 0 { 844 // The cache is empty, so there is no memory to be reused. 845 return nil, err 846 } 847 f.maxCacheSize = f.indexedRowsCache.Len() 848 if err := f.reuseFirstRowInCache(ctx, int(*idx), row); err != nil { 849 return nil, err 850 } 851 } else { 852 return nil, err 853 } 854 } else { 855 // We actually need to copy the row into memory. 856 ir := IndexedRow{int(*idx), f.rowAlloc.CopyRow(row)} 857 f.indexedRowsCache.AddLast(ir) 858 } 859 } 860 f.nextPosToCache++ 861 } else { 862 return nil, errors.Errorf("unexpected last column type: should be DInt but found %T", idx) 863 } 864 if f.idxRowIter == pos { 865 return f.indexedRowsCache.GetLast().(tree.IndexedRow), nil 866 } 867 } 868 f.idxRowIter++ 869 } 870 } 871 rowWithIdx = f.DiskBackedRowContainer.mrc.EncRow(pos) 872 row, rowIdx := rowWithIdx[:len(rowWithIdx)-1], rowWithIdx[len(rowWithIdx)-1].Datum 873 if idx, ok := rowIdx.(*tree.DInt); ok { 874 return IndexedRow{int(*idx), row}, nil 875 } 876 return nil, errors.Errorf("unexpected last column type: should be DInt but found %T", rowIdx) 877 } 878 879 // reuseFirstRowInCache reuses the underlying memory of the first row in the 880 // cache to store 'row' and puts it as the last one in the cache. It adjusts 881 // the memory account accordingly and, if necessary, removes some first rows. 882 func (f *DiskBackedIndexedRowContainer) reuseFirstRowInCache( 883 ctx context.Context, idx int, row sqlbase.EncDatumRow, 884 ) error { 885 newRowSize := row.Size() 886 for { 887 if f.indexedRowsCache.Len() == 0 { 888 return errors.Errorf("unexpectedly the cache of DiskBackedIndexedRowContainer contains zero rows") 889 } 890 indexedRowToReuse := f.indexedRowsCache.GetFirst().(IndexedRow) 891 oldRowSize := indexedRowToReuse.Row.Size() 892 delta := int64(newRowSize - oldRowSize) 893 if delta > 0 { 894 // New row takes up more memory than the old one. 895 if err := f.cacheMemAcc.Grow(ctx, delta); err != nil { 896 if sqlbase.IsOutOfMemoryError(err) { 897 // We need to actually reduce the cache size, so we remove the first 898 // row and adjust the memory account, maxCacheSize, and 899 // f.firstCachedRowPos accordingly. 900 f.indexedRowsCache.RemoveFirst() 901 f.cacheMemAcc.Shrink(ctx, int64(oldRowSize)) 902 f.maxCacheSize-- 903 f.firstCachedRowPos++ 904 if f.indexedRowsCache.Len() == 0 { 905 return err 906 } 907 continue 908 } 909 return err 910 } 911 } else if delta < 0 { 912 f.cacheMemAcc.Shrink(ctx, -delta) 913 } 914 indexedRowToReuse.Idx = idx 915 copy(indexedRowToReuse.Row, row) 916 f.indexedRowsCache.RemoveFirst() 917 f.indexedRowsCache.AddLast(indexedRowToReuse) 918 f.firstCachedRowPos++ 919 return nil 920 } 921 } 922 923 // getRowWithoutCache returns the row at requested position without using the 924 // cache. It utilizes the same disk row iterator along multiple consequent 925 // calls and rewinds the iterator only when it has been advanced further than 926 // the position requested. 927 // 928 // NOTE: this method should only be used for testing purposes. 929 func (f *DiskBackedIndexedRowContainer) getRowWithoutCache( 930 ctx context.Context, pos int, 931 ) tree.IndexedRow { 932 if !f.UsingDisk() { 933 panic(errors.Errorf("getRowWithoutCache is called when the container is using memory")) 934 } 935 if f.diskRowIter == nil { 936 f.diskRowIter = f.DiskBackedRowContainer.drc.NewIterator(ctx) 937 f.diskRowIter.Rewind() 938 } 939 if f.idxRowIter > pos { 940 // The iterator has been advanced further than we need, so we need to 941 // start iterating from the beginning. 942 f.idxRowIter = 0 943 f.diskRowIter.Rewind() 944 } 945 for ; ; f.diskRowIter.Next() { 946 if ok, err := f.diskRowIter.Valid(); err != nil { 947 panic(err) 948 } else if !ok { 949 panic(fmt.Sprintf("row at pos %d not found", pos)) 950 } 951 if f.idxRowIter == pos { 952 rowWithIdx, err := f.diskRowIter.Row() 953 if err != nil { 954 panic(err) 955 } 956 for i := range rowWithIdx { 957 if err := rowWithIdx[i].EnsureDecoded(f.storedTypes[i], &f.datumAlloc); err != nil { 958 panic(err) 959 } 960 } 961 row, rowIdx := rowWithIdx[:len(rowWithIdx)-1], rowWithIdx[len(rowWithIdx)-1].Datum 962 if idx, ok := rowIdx.(*tree.DInt); ok { 963 return IndexedRow{int(*idx), f.rowAlloc.CopyRow(row)} 964 } 965 panic(errors.Errorf("unexpected last column type: should be DInt but found %T", rowIdx)) 966 } 967 f.idxRowIter++ 968 } 969 } 970 971 // IndexedRow is a row with a corresponding index. 972 type IndexedRow struct { 973 Idx int 974 Row sqlbase.EncDatumRow 975 } 976 977 // GetIdx implements tree.IndexedRow interface. 978 func (ir IndexedRow) GetIdx() int { 979 return ir.Idx 980 } 981 982 // GetDatum implements tree.IndexedRow interface. 983 func (ir IndexedRow) GetDatum(colIdx int) (tree.Datum, error) { 984 return ir.Row[colIdx].Datum, nil 985 } 986 987 // GetDatums implements tree.IndexedRow interface. 988 func (ir IndexedRow) GetDatums(startColIdx, endColIdx int) (tree.Datums, error) { 989 datums := make(tree.Datums, 0, endColIdx-startColIdx) 990 for idx := startColIdx; idx < endColIdx; idx++ { 991 datums = append(datums, ir.Row[idx].Datum) 992 } 993 return datums, nil 994 } 995 996 const sizeOfInt = int64(unsafe.Sizeof(int(0)))