github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/rowcontainer/hash_row_container.go (about) 1 // Copyright 2017 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 "bytes" 15 "context" 16 "unsafe" 17 18 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/diskmap" 19 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 20 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 21 "github.com/cockroachdb/cockroach/pkg/sql/types" 22 "github.com/cockroachdb/cockroach/pkg/util/encoding" 23 "github.com/cockroachdb/cockroach/pkg/util/log" 24 "github.com/cockroachdb/cockroach/pkg/util/mon" 25 "github.com/cockroachdb/errors" 26 ) 27 28 type columns []uint32 29 30 // RowMarkerIterator is a RowIterator that can be used to mark rows. 31 type RowMarkerIterator interface { 32 RowIterator 33 // Reset resets this iterator to point at a bucket that matches the given 34 // row. This will cause RowIterator.Rewind to rewind to the front of the 35 // input row's bucket. 36 Reset(ctx context.Context, row sqlbase.EncDatumRow) error 37 Mark(ctx context.Context, mark bool) error 38 IsMarked(ctx context.Context) bool 39 } 40 41 // HashRowContainer is a container used to store rows according to an encoding 42 // of given equality columns. The stored rows can then be probed to return a 43 // bucket of matching rows. Additionally, each stored row can be marked and all 44 // rows that are unmarked can be iterated over. An example of where this is 45 // useful is in full/outer joins. The caller can mark all matched rows and 46 // iterate over the unmarked rows to produce a result. 47 type HashRowContainer interface { 48 // Init initializes the HashRowContainer with the given equality columns. 49 // - shouldMark specifies whether the caller cares about marking rows. If 50 // not, the HashRowContainer will not perform any row marking logic. This 51 // is meant to optimize space usage and runtime. 52 // - types is the schema of rows that will be added to this container. 53 // - storedEqCols are the equality columns of rows stored in this 54 // container. 55 // i.e. when adding a row, the columns specified by storedEqCols are used 56 // to get the bucket that the row should be added to. 57 // - encodeNull indicates whether rows with NULL equality columns should be 58 // stored or skipped. 59 Init( 60 ctx context.Context, shouldMark bool, types []*types.T, storedEqCols columns, 61 encodeNull bool, 62 ) error 63 AddRow(context.Context, sqlbase.EncDatumRow) error 64 65 // NewBucketIterator returns a RowMarkerIterator that iterates over a bucket 66 // of rows that match the given row on equality columns. This iterator can 67 // also be used to mark rows. 68 // Rows are marked because of the use of this interface by the hashJoiner. 69 // Given a row, the hashJoiner does not necessarily want to emit all rows 70 // that match on equality columns. There is an additional `ON` clause that 71 // specifies an arbitrary expression that matching rows must pass to be 72 // emitted. For full/outer joins, this is tracked through marking rows if 73 // they match and then iterating over all unmarked rows to emit those that 74 // did not match. 75 // - probeEqCols are the equality columns of the given row that are used to 76 // get the bucket of matching rows. 77 NewBucketIterator( 78 ctx context.Context, row sqlbase.EncDatumRow, probeEqCols columns, 79 ) (RowMarkerIterator, error) 80 81 // NewUnmarkedIterator returns a RowIterator that iterates over unmarked 82 // rows. If shouldMark was false in Init(), this iterator iterates over all 83 // rows. 84 NewUnmarkedIterator(context.Context) RowIterator 85 86 // Close frees up resources held by the HashRowContainer. 87 Close(context.Context) 88 } 89 90 // columnEncoder is a utility struct used by implementations of HashRowContainer 91 // to encode equality columns, the result of which is used as a key to a bucket. 92 type columnEncoder struct { 93 scratch []byte 94 // types for the "key" columns (equality columns) 95 keyTypes []*types.T 96 datumAlloc sqlbase.DatumAlloc 97 encodeNull bool 98 } 99 100 func (e *columnEncoder) init(typs []*types.T, keyCols columns, encodeNull bool) { 101 e.keyTypes = make([]*types.T, len(keyCols)) 102 for i, c := range keyCols { 103 e.keyTypes[i] = typs[c] 104 } 105 e.encodeNull = encodeNull 106 } 107 108 // encodeColumnsOfRow returns the encoding for the grouping columns. This is 109 // then used as our group key to determine which bucket to add to. 110 // If the row contains any NULLs and encodeNull is false, hasNull is true and 111 // no encoding is returned. If encodeNull is true, hasNull is never set. 112 func encodeColumnsOfRow( 113 da *sqlbase.DatumAlloc, 114 appendTo []byte, 115 row sqlbase.EncDatumRow, 116 cols columns, 117 colTypes []*types.T, 118 encodeNull bool, 119 ) (encoding []byte, hasNull bool, err error) { 120 for i, colIdx := range cols { 121 if row[colIdx].IsNull() && !encodeNull { 122 return nil, true, nil 123 } 124 appendTo, err = row[colIdx].Fingerprint(colTypes[i], da, appendTo) 125 if err != nil { 126 return appendTo, false, err 127 } 128 } 129 return appendTo, false, nil 130 } 131 132 // encodeEqualityCols returns the encoding of the specified columns of the given 133 // row. The returned byte slice is only valid until the next call to 134 // encodeEqualityColumns(). 135 func (e *columnEncoder) encodeEqualityCols( 136 ctx context.Context, row sqlbase.EncDatumRow, eqCols columns, 137 ) ([]byte, error) { 138 encoded, hasNull, err := encodeColumnsOfRow( 139 &e.datumAlloc, e.scratch, row, eqCols, e.keyTypes, e.encodeNull, 140 ) 141 if err != nil { 142 return nil, err 143 } 144 e.scratch = encoded[:0] 145 if hasNull { 146 log.Fatal(ctx, "cannot process rows with NULL in an equality column") 147 } 148 return encoded, nil 149 } 150 151 const sizeOfBucket = int64(unsafe.Sizeof([]int{})) 152 const sizeOfRowIdx = int64(unsafe.Sizeof(int(0))) 153 const sizeOfBoolSlice = int64(unsafe.Sizeof([]bool{})) 154 const sizeOfBool = int64(unsafe.Sizeof(false)) 155 156 // HashMemRowContainer is an in-memory implementation of a HashRowContainer. 157 // The rows are stored in an underlying MemRowContainer and an accompanying 158 // map stores the mapping from equality column encodings to indices in the 159 // MemRowContainer corresponding to matching rows. 160 // NOTE: Once a row is marked, adding more rows to the HashMemRowContainer 161 // results in undefined behavior. It is not necessary to do otherwise for the 162 // current usage of HashMemRowContainer. 163 type HashMemRowContainer struct { 164 *MemRowContainer 165 columnEncoder 166 167 // shouldMark specifies whether the caller cares about marking rows. If not, 168 // marked is never initialized. 169 shouldMark bool 170 171 // marked specifies for each row in MemRowContainer whether that row has 172 // been marked. Used for iterating over unmarked rows. 173 marked []bool 174 175 // markMemoryReserved specifies whether the HashMemRowContainer's memory 176 // account already accounts for the memory needed to mark the rows in the 177 // HashMemRowContainer. 178 markMemoryReserved bool 179 180 // buckets contains the indices into MemRowContainer for a given group 181 // key (which is the encoding of storedEqCols). 182 buckets map[string][]int 183 // bucketsAcc is the memory account for the buckets. The datums themselves 184 // are all in the MemRowContainer. 185 bucketsAcc mon.BoundAccount 186 187 // storedEqCols contains the indices of the columns of a row that are 188 // encoded and used as a key into buckets when adding a row. 189 storedEqCols columns 190 } 191 192 var _ HashRowContainer = &HashMemRowContainer{} 193 194 // MakeHashMemRowContainer creates a HashMemRowContainer from the given 195 // rowContainer. This rowContainer must still be Close()d by the caller. 196 func MakeHashMemRowContainer(rowContainer *MemRowContainer) HashMemRowContainer { 197 return HashMemRowContainer{ 198 MemRowContainer: rowContainer, 199 buckets: make(map[string][]int), 200 bucketsAcc: rowContainer.evalCtx.Mon.MakeBoundAccount(), 201 } 202 } 203 204 // Init implements the HashRowContainer interface. types is ignored because the 205 // schema is inferred from the MemRowContainer. 206 func (h *HashMemRowContainer) Init( 207 ctx context.Context, shouldMark bool, _ []*types.T, storedEqCols columns, encodeNull bool, 208 ) error { 209 if h.storedEqCols != nil { 210 return errors.New("HashMemRowContainer has already been initialized") 211 } 212 h.columnEncoder.init(h.MemRowContainer.types, storedEqCols, encodeNull) 213 h.shouldMark = shouldMark 214 h.storedEqCols = storedEqCols 215 216 // Build buckets from the rowContainer. 217 for rowIdx := 0; rowIdx < h.Len(); rowIdx++ { 218 if err := h.addRowToBucket(ctx, h.EncRow(rowIdx), rowIdx); err != nil { 219 return err 220 } 221 } 222 return nil 223 } 224 225 // AddRow adds a row to the HashMemRowContainer. This row is unmarked by default. 226 func (h *HashMemRowContainer) AddRow(ctx context.Context, row sqlbase.EncDatumRow) error { 227 rowIdx := h.Len() 228 if err := h.MemRowContainer.AddRow(ctx, row); err != nil { 229 return err 230 } 231 return h.addRowToBucket(ctx, row, rowIdx) 232 } 233 234 // Close implements the HashRowContainer interface. 235 func (h *HashMemRowContainer) Close(ctx context.Context) { 236 h.MemRowContainer.Close(ctx) 237 h.bucketsAcc.Close(ctx) 238 } 239 240 // addRowToBucket is a helper function that encodes the equality columns of the 241 // given row and appends the rowIdx to the matching bucket. 242 func (h *HashMemRowContainer) addRowToBucket( 243 ctx context.Context, row sqlbase.EncDatumRow, rowIdx int, 244 ) error { 245 encoded, err := h.encodeEqualityCols(ctx, row, h.storedEqCols) 246 if err != nil { 247 return err 248 } 249 250 bucket, ok := h.buckets[string(encoded)] 251 252 usage := sizeOfRowIdx 253 if !ok { 254 usage += int64(len(encoded)) 255 usage += sizeOfBucket 256 } 257 258 if err := h.bucketsAcc.Grow(ctx, usage); err != nil { 259 return err 260 } 261 262 h.buckets[string(encoded)] = append(bucket, rowIdx) 263 return nil 264 } 265 266 // ReserveMarkMemoryMaybe is a utility function to grow the 267 // HashMemRowContainer's memory account by the memory needed to mark all rows. 268 // It is a noop if h.markMemoryReserved is true. 269 func (h *HashMemRowContainer) ReserveMarkMemoryMaybe(ctx context.Context) error { 270 if h.markMemoryReserved { 271 return nil 272 } 273 if err := h.bucketsAcc.Grow(ctx, sizeOfBoolSlice+(sizeOfBool*int64(h.Len()))); err != nil { 274 return err 275 } 276 h.markMemoryReserved = true 277 return nil 278 } 279 280 // hashMemRowBucketIterator iterates over the rows in a bucket. 281 type hashMemRowBucketIterator struct { 282 *HashMemRowContainer 283 probeEqCols columns 284 // rowIdxs are the indices of rows in the bucket. 285 rowIdxs []int 286 curIdx int 287 } 288 289 var _ RowMarkerIterator = &hashMemRowBucketIterator{} 290 291 // NewBucketIterator implements the HashRowContainer interface. 292 func (h *HashMemRowContainer) NewBucketIterator( 293 ctx context.Context, row sqlbase.EncDatumRow, probeEqCols columns, 294 ) (RowMarkerIterator, error) { 295 ret := &hashMemRowBucketIterator{ 296 HashMemRowContainer: h, 297 probeEqCols: probeEqCols, 298 } 299 300 if err := ret.Reset(ctx, row); err != nil { 301 return nil, err 302 } 303 return ret, nil 304 } 305 306 // Rewind implements the RowIterator interface. 307 func (i *hashMemRowBucketIterator) Rewind() { 308 i.curIdx = 0 309 } 310 311 // Valid implements the RowIterator interface. 312 func (i *hashMemRowBucketIterator) Valid() (bool, error) { 313 return i.curIdx < len(i.rowIdxs), nil 314 } 315 316 // Next implements the RowIterator interface. 317 func (i *hashMemRowBucketIterator) Next() { 318 i.curIdx++ 319 } 320 321 // Row implements the RowIterator interface. 322 func (i *hashMemRowBucketIterator) Row() (sqlbase.EncDatumRow, error) { 323 return i.EncRow(i.rowIdxs[i.curIdx]), nil 324 } 325 326 // IsMarked implements the RowMarkerIterator interface. 327 func (i *hashMemRowBucketIterator) IsMarked(ctx context.Context) bool { 328 if !i.shouldMark { 329 log.Fatal(ctx, "hash mem row container not set up for marking") 330 } 331 if i.marked == nil { 332 return false 333 } 334 335 return i.marked[i.rowIdxs[i.curIdx]] 336 } 337 338 // Mark implements the RowMarkerIterator interface. 339 func (i *hashMemRowBucketIterator) Mark(ctx context.Context, mark bool) error { 340 if !i.shouldMark { 341 log.Fatal(ctx, "hash mem row container not set up for marking") 342 } 343 if i.marked == nil { 344 if !i.markMemoryReserved { 345 panic("mark memory should have been reserved already") 346 } 347 i.marked = make([]bool, i.Len()) 348 } 349 350 i.marked[i.rowIdxs[i.curIdx]] = mark 351 return nil 352 } 353 354 func (i *hashMemRowBucketIterator) Reset(ctx context.Context, row sqlbase.EncDatumRow) error { 355 encoded, err := i.encodeEqualityCols(ctx, row, i.probeEqCols) 356 if err != nil { 357 return err 358 } 359 i.rowIdxs = i.buckets[string(encoded)] 360 return nil 361 } 362 363 // Close implements the RowIterator interface. 364 func (i *hashMemRowBucketIterator) Close() {} 365 366 // hashMemRowIterator iterates over all unmarked rows in a HashMemRowContainer. 367 type hashMemRowIterator struct { 368 *HashMemRowContainer 369 curIdx int 370 371 // curKey contains the key that would be assigned to the current row if it 372 // were to be put on disk. It is needed to optimize the recreation of the 373 // iterators when HashDiskBackedRowContainer spills to disk and is computed 374 // once, right before the spilling occurs. 375 curKey []byte 376 } 377 378 var _ RowIterator = &hashMemRowIterator{} 379 380 // NewUnmarkedIterator implements the HashRowContainer interface. 381 func (h *HashMemRowContainer) NewUnmarkedIterator(ctx context.Context) RowIterator { 382 return &hashMemRowIterator{HashMemRowContainer: h} 383 } 384 385 // Rewind implements the RowIterator interface. 386 func (i *hashMemRowIterator) Rewind() { 387 i.curIdx = -1 388 // Next will advance curIdx to the first unmarked row. 389 i.Next() 390 } 391 392 // Valid implements the RowIterator interface. 393 func (i *hashMemRowIterator) Valid() (bool, error) { 394 return i.curIdx < i.Len(), nil 395 } 396 397 // computeKey calculates the key for the current row as if the row is put on 398 // disk. This method must be kept in sync with AddRow() of DiskRowContainer. 399 func (i *hashMemRowIterator) computeKey() error { 400 valid, err := i.Valid() 401 if err != nil { 402 return err 403 } 404 405 var row sqlbase.EncDatumRow 406 if valid { 407 row = i.EncRow(i.curIdx) 408 } else { 409 if i.curIdx == 0 { 410 // There are no rows in the container, so the key corresponding to the 411 // "current" row is nil. 412 i.curKey = nil 413 return nil 414 } 415 // The iterator points at right after all the rows in the container, so we 416 // will "simulate" the key corresponding to the non-existent row as the key 417 // to the last existing row plus one (plus one part is done below where we 418 // append the index of the row to curKey). 419 row = i.EncRow(i.curIdx - 1) 420 } 421 422 i.curKey = i.curKey[:0] 423 for _, col := range i.storedEqCols { 424 var err error 425 i.curKey, err = row[col].Fingerprint(i.types[col], &i.columnEncoder.datumAlloc, i.curKey) 426 if err != nil { 427 return err 428 } 429 } 430 i.curKey = encoding.EncodeUvarintAscending(i.curKey, uint64(i.curIdx)) 431 return nil 432 } 433 434 // Next implements the RowIterator interface. 435 func (i *hashMemRowIterator) Next() { 436 // Move the curIdx to the next unmarked row. 437 i.curIdx++ 438 if i.marked != nil { 439 for ; i.curIdx < len(i.marked) && i.marked[i.curIdx]; i.curIdx++ { 440 } 441 } 442 } 443 444 // Row implements the RowIterator interface. 445 func (i *hashMemRowIterator) Row() (sqlbase.EncDatumRow, error) { 446 return i.EncRow(i.curIdx), nil 447 } 448 449 // Close implements the RowIterator interface. 450 func (i *hashMemRowIterator) Close() {} 451 452 // HashDiskRowContainer is an on-disk implementation of a HashRowContainer. 453 // The rows are stored in an underlying DiskRowContainer with an extra boolean 454 // column to keep track of that row's mark. 455 type HashDiskRowContainer struct { 456 DiskRowContainer 457 columnEncoder 458 459 diskMonitor *mon.BytesMonitor 460 // shouldMark specifies whether the caller cares about marking rows. If not, 461 // rows are stored with one less column (which usually specifies that row's 462 // mark). 463 shouldMark bool 464 engine diskmap.Factory 465 scratchEncRow sqlbase.EncDatumRow 466 } 467 468 var _ HashRowContainer = &HashDiskRowContainer{} 469 470 var ( 471 encodedTrue = encoding.EncodeBoolValue(nil, encoding.NoColumnID, true) 472 encodedFalse = encoding.EncodeBoolValue(nil, encoding.NoColumnID, false) 473 ) 474 475 // MakeHashDiskRowContainer creates a HashDiskRowContainer with the given engine 476 // as the underlying store that rows are stored on. shouldMark specifies whether 477 // the HashDiskRowContainer should set itself up to mark rows. 478 func MakeHashDiskRowContainer( 479 diskMonitor *mon.BytesMonitor, e diskmap.Factory, 480 ) HashDiskRowContainer { 481 return HashDiskRowContainer{ 482 diskMonitor: diskMonitor, 483 engine: e, 484 } 485 } 486 487 // Init implements the HashRowContainer interface. 488 func (h *HashDiskRowContainer) Init( 489 _ context.Context, shouldMark bool, typs []*types.T, storedEqCols columns, encodeNull bool, 490 ) error { 491 h.columnEncoder.init(typs, storedEqCols, encodeNull) 492 // Provide the DiskRowContainer with an ordering on the equality columns of 493 // the rows that we will store. This will result in rows with the 494 // same equality columns occurring contiguously in the keyspace. 495 ordering := make(sqlbase.ColumnOrdering, len(storedEqCols)) 496 for i := range ordering { 497 ordering[i] = sqlbase.ColumnOrderInfo{ 498 ColIdx: int(storedEqCols[i]), 499 Direction: encoding.Ascending, 500 } 501 } 502 503 h.shouldMark = shouldMark 504 505 storedTypes := typs 506 if h.shouldMark { 507 // Add a boolean column to the end of the rows to implement marking rows. 508 storedTypes = make([]*types.T, len(typs)+1) 509 copy(storedTypes, typs) 510 storedTypes[len(storedTypes)-1] = types.Bool 511 512 h.scratchEncRow = make(sqlbase.EncDatumRow, len(storedTypes)) 513 // Initialize the last column of the scratch row we use in AddRow() to 514 // be unmarked. 515 h.scratchEncRow[len(h.scratchEncRow)-1] = sqlbase.DatumToEncDatum( 516 types.Bool, 517 tree.MakeDBool(false), 518 ) 519 } 520 521 h.DiskRowContainer = MakeDiskRowContainer(h.diskMonitor, storedTypes, ordering, h.engine) 522 return nil 523 } 524 525 // AddRow adds a row to the HashDiskRowContainer. This row is unmarked by 526 // default. 527 func (h *HashDiskRowContainer) AddRow(ctx context.Context, row sqlbase.EncDatumRow) error { 528 var err error 529 if h.shouldMark { 530 // len(h.scratchEncRow) == len(row) + 1 if h.shouldMark == true. The 531 // last column has been initialized to a false mark in Init(). 532 copy(h.scratchEncRow, row) 533 err = h.DiskRowContainer.AddRow(ctx, h.scratchEncRow) 534 } else { 535 err = h.DiskRowContainer.AddRow(ctx, row) 536 } 537 return err 538 } 539 540 // hashDiskRowBucketIterator iterates over the rows in a bucket. 541 type hashDiskRowBucketIterator struct { 542 *diskRowIterator 543 *HashDiskRowContainer 544 probeEqCols columns 545 // haveMarkedRows returns true if we've marked rows since the last time we 546 // recreated our underlying diskRowIterator. 547 haveMarkedRows bool 548 // encodedEqCols is the encoding of the equality columns of the rows in the 549 // bucket that this iterator iterates over. 550 encodedEqCols []byte 551 // Temporary buffer used for constructed marked values. 552 tmpBuf []byte 553 } 554 555 var _ RowMarkerIterator = &hashDiskRowBucketIterator{} 556 557 // NewBucketIterator implements the HashRowContainer interface. 558 func (h *HashDiskRowContainer) NewBucketIterator( 559 ctx context.Context, row sqlbase.EncDatumRow, probeEqCols columns, 560 ) (RowMarkerIterator, error) { 561 ret := &hashDiskRowBucketIterator{ 562 HashDiskRowContainer: h, 563 probeEqCols: probeEqCols, 564 diskRowIterator: h.NewIterator(ctx).(*diskRowIterator), 565 } 566 if err := ret.Reset(ctx, row); err != nil { 567 ret.Close() 568 return nil, err 569 } 570 return ret, nil 571 } 572 573 // Rewind implements the RowIterator interface. 574 func (i *hashDiskRowBucketIterator) Rewind() { 575 i.SeekGE(i.encodedEqCols) 576 } 577 578 // Valid implements the RowIterator interface. 579 func (i *hashDiskRowBucketIterator) Valid() (bool, error) { 580 ok, err := i.diskRowIterator.Valid() 581 if !ok || err != nil { 582 return ok, err 583 } 584 // Since the underlying map is sorted, once the key prefix does not equal 585 // the encoded equality columns, we have gone past the end of the bucket. 586 return bytes.HasPrefix(i.UnsafeKey(), i.encodedEqCols), nil 587 } 588 589 // Row implements the RowIterator interface. 590 func (i *hashDiskRowBucketIterator) Row() (sqlbase.EncDatumRow, error) { 591 row, err := i.diskRowIterator.Row() 592 if err != nil { 593 return nil, err 594 } 595 596 // Remove the mark from the end of the row. 597 if i.HashDiskRowContainer.shouldMark { 598 row = row[:len(row)-1] 599 } 600 return row, nil 601 } 602 603 func (i *hashDiskRowBucketIterator) Reset(ctx context.Context, row sqlbase.EncDatumRow) error { 604 encoded, err := i.HashDiskRowContainer.encodeEqualityCols(ctx, row, i.probeEqCols) 605 if err != nil { 606 return err 607 } 608 i.encodedEqCols = append(i.encodedEqCols[:0], encoded...) 609 if i.haveMarkedRows { 610 // We have to recreate our iterator if we need to flush marks to disk. 611 // TODO(jordan): do this less by keeping a cache of written marks. 612 i.haveMarkedRows = false 613 i.diskRowIterator.Close() 614 i.diskRowIterator = i.HashDiskRowContainer.NewIterator(ctx).(*diskRowIterator) 615 } 616 return nil 617 } 618 619 // IsMarked implements the RowMarkerIterator interface. 620 func (i *hashDiskRowBucketIterator) IsMarked(ctx context.Context) bool { 621 if !i.HashDiskRowContainer.shouldMark { 622 log.Fatal(ctx, "hash disk row container not set up for marking") 623 } 624 ok, err := i.diskRowIterator.Valid() 625 if !ok || err != nil { 626 return false 627 } 628 629 rowVal := i.UnsafeValue() 630 return bytes.Equal(rowVal[len(rowVal)-len(encodedTrue):], encodedTrue) 631 } 632 633 // Mark implements the RowMarkerIterator interface. 634 func (i *hashDiskRowBucketIterator) Mark(ctx context.Context, mark bool) error { 635 if !i.HashDiskRowContainer.shouldMark { 636 log.Fatal(ctx, "hash disk row container not set up for marking") 637 } 638 i.haveMarkedRows = true 639 markBytes := encodedFalse 640 if mark { 641 markBytes = encodedTrue 642 } 643 // rowVal are the non-equality encoded columns, the last of which is the 644 // column we use to mark a row. 645 rowVal := append(i.tmpBuf[:0], i.UnsafeValue()...) 646 originalLen := len(rowVal) 647 rowVal = append(rowVal, markBytes...) 648 649 // Write the new encoding of mark over the old encoding of mark and truncate 650 // the extra bytes. 651 copy(rowVal[originalLen-len(markBytes):], rowVal[originalLen:]) 652 rowVal = rowVal[:originalLen] 653 i.tmpBuf = rowVal 654 655 // These marks only matter when using a hashDiskRowIterator to iterate over 656 // unmarked rows. The writes are flushed when creating a NewIterator() in 657 // NewUnmarkedIterator(). 658 return i.HashDiskRowContainer.bufferedRows.Put(i.UnsafeKey(), rowVal) 659 } 660 661 // hashDiskRowIterator iterates over all unmarked rows in a 662 // HashDiskRowContainer. 663 type hashDiskRowIterator struct { 664 *diskRowIterator 665 } 666 667 var _ RowIterator = &hashDiskRowIterator{} 668 669 // NewUnmarkedIterator implements the HashRowContainer interface. 670 func (h *HashDiskRowContainer) NewUnmarkedIterator(ctx context.Context) RowIterator { 671 if h.shouldMark { 672 return &hashDiskRowIterator{ 673 diskRowIterator: h.NewIterator(ctx).(*diskRowIterator), 674 } 675 } 676 return h.NewIterator(ctx) 677 } 678 679 // Rewind implements the RowIterator interface. 680 func (i *hashDiskRowIterator) Rewind() { 681 i.diskRowIterator.Rewind() 682 // If the current row is marked, move the iterator to the next unmarked row. 683 if i.isRowMarked() { 684 i.Next() 685 } 686 } 687 688 // Next implements the RowIterator interface. 689 func (i *hashDiskRowIterator) Next() { 690 i.diskRowIterator.Next() 691 for i.isRowMarked() { 692 i.diskRowIterator.Next() 693 } 694 } 695 696 // Row implements the RowIterator interface. 697 func (i *hashDiskRowIterator) Row() (sqlbase.EncDatumRow, error) { 698 row, err := i.diskRowIterator.Row() 699 if err != nil { 700 return nil, err 701 } 702 703 // Remove the mark from the end of the row. 704 row = row[:len(row)-1] 705 return row, nil 706 } 707 708 // isRowMarked returns true if the current row is marked or false if it wasn't 709 // marked or there was an error establishing the row's validity. Subsequent 710 // calls to Valid() will uncover this error. 711 func (i *hashDiskRowIterator) isRowMarked() bool { 712 // isRowMarked is not necessarily called after Valid(). 713 ok, err := i.diskRowIterator.Valid() 714 if !ok || err != nil { 715 return false 716 } 717 718 rowVal := i.UnsafeValue() 719 return bytes.Equal(rowVal[len(rowVal)-len(encodedTrue):], encodedTrue) 720 } 721 722 // HashDiskBackedRowContainer is a hashRowContainer that uses a 723 // HashMemRowContainer to store rows and spills to disk automatically if memory 724 // usage exceeds a given budget. When spilled to disk, the rows are stored with 725 // an extra boolean column to keep track of that row's mark. 726 type HashDiskBackedRowContainer struct { 727 // src is the current hashRowContainer that is being used to store rows. 728 // All the hashRowContainer methods are redefined rather than delegated 729 // to an embedded struct because of how defer works: 730 // rc.init(...) 731 // defer rc.Close(ctx) 732 // Close will call HashMemRowContainer.Close(ctx) even after spilling to 733 // disk. 734 src HashRowContainer 735 736 hmrc *HashMemRowContainer 737 hdrc *HashDiskRowContainer 738 739 // shouldMark specifies whether the caller cares about marking rows. 740 shouldMark bool 741 types []*types.T 742 storedEqCols columns 743 encodeNull bool 744 745 // mrc is used to build HashMemRowContainer upon. 746 mrc *MemRowContainer 747 748 evalCtx *tree.EvalContext 749 memoryMonitor *mon.BytesMonitor 750 diskMonitor *mon.BytesMonitor 751 engine diskmap.Factory 752 scratchEncRow sqlbase.EncDatumRow 753 754 // allRowsIterators keeps track of all iterators created via 755 // NewAllRowsIterator(). If the container spills to disk, these become 756 // invalid, so the container actively recreates the iterators, advances them 757 // to appropriate positions, and updates each iterator in-place. 758 allRowsIterators []*AllRowsIterator 759 } 760 761 var _ HashRowContainer = &HashDiskBackedRowContainer{} 762 763 // NewHashDiskBackedRowContainer makes a HashDiskBackedRowContainer. 764 // mrc (the first argument) can either be nil (in which case 765 // HashMemRowContainer will be built upon an empty MemRowContainer) or non-nil 766 // (in which case mrc is used as underlying MemRowContainer under 767 // HashMemRowContainer). The latter case is used by the hashJoiner since when 768 // initializing HashDiskBackedRowContainer it will have accumulated rows from 769 // both sides of the join in MemRowContainers, and we can reuse one of them. 770 func NewHashDiskBackedRowContainer( 771 mrc *MemRowContainer, 772 evalCtx *tree.EvalContext, 773 memoryMonitor *mon.BytesMonitor, 774 diskMonitor *mon.BytesMonitor, 775 engine diskmap.Factory, 776 ) *HashDiskBackedRowContainer { 777 return &HashDiskBackedRowContainer{ 778 mrc: mrc, 779 evalCtx: evalCtx, 780 memoryMonitor: memoryMonitor, 781 diskMonitor: diskMonitor, 782 engine: engine, 783 allRowsIterators: make([]*AllRowsIterator, 0, 1), 784 } 785 } 786 787 // Init implements the hashRowContainer interface. 788 func (h *HashDiskBackedRowContainer) Init( 789 ctx context.Context, shouldMark bool, types []*types.T, storedEqCols columns, encodeNull bool, 790 ) error { 791 h.shouldMark = shouldMark 792 h.types = types 793 h.storedEqCols = storedEqCols 794 h.encodeNull = encodeNull 795 if shouldMark { 796 // We might need to preserve the marks when spilling to disk which requires 797 // adding an extra boolean column to the row when read from memory. 798 h.scratchEncRow = make(sqlbase.EncDatumRow, len(types)+1) 799 } 800 801 // Provide the MemRowContainer with an ordering on the equality columns of 802 // the rows that we will store. This will result in rows with the 803 // same equality columns occurring contiguously in the keyspace. 804 ordering := make(sqlbase.ColumnOrdering, len(storedEqCols)) 805 for i := range ordering { 806 ordering[i] = sqlbase.ColumnOrderInfo{ 807 ColIdx: int(storedEqCols[i]), 808 Direction: encoding.Ascending, 809 } 810 } 811 if h.mrc == nil { 812 h.mrc = &MemRowContainer{} 813 h.mrc.InitWithMon(ordering, types, h.evalCtx, h.memoryMonitor, 0 /* rowCapacity */) 814 } 815 hmrc := MakeHashMemRowContainer(h.mrc) 816 h.hmrc = &hmrc 817 h.src = h.hmrc 818 if err := h.hmrc.Init(ctx, shouldMark, types, storedEqCols, encodeNull); err != nil { 819 if spilled, spillErr := h.spillIfMemErr(ctx, err); !spilled && spillErr == nil { 820 // The error was not an out of memory error. 821 return err 822 } else if spillErr != nil { 823 // A disk spill was attempted but there was an error in doing so. 824 return spillErr 825 } 826 } 827 828 return nil 829 } 830 831 // AddRow adds a row to the HashDiskBackedRowContainer. This row is unmarked by default. 832 func (h *HashDiskBackedRowContainer) AddRow(ctx context.Context, row sqlbase.EncDatumRow) error { 833 if err := h.src.AddRow(ctx, row); err != nil { 834 if spilled, spillErr := h.spillIfMemErr(ctx, err); !spilled && spillErr == nil { 835 // The error was not an out of memory error. 836 return err 837 } else if spillErr != nil { 838 // A disk spill was attempted but there was an error in doing so. 839 return spillErr 840 } 841 // Add the row that caused the memory error. 842 return h.src.AddRow(ctx, row) 843 } 844 return nil 845 } 846 847 // Close implements the hashRowContainer interface. 848 func (h *HashDiskBackedRowContainer) Close(ctx context.Context) { 849 if h.hdrc != nil { 850 h.hdrc.Close(ctx) 851 } 852 h.hmrc.Close(ctx) 853 } 854 855 // UsingDisk returns whether or not the HashDiskBackedRowContainer is currently 856 // using disk. 857 func (h *HashDiskBackedRowContainer) UsingDisk() bool { 858 return h.hdrc != nil 859 } 860 861 // ReserveMarkMemoryMaybe attempts to reserve memory for marks if we're using 862 // an in-memory container at the moment. If there is not enough memory left, it 863 // spills to disk. 864 func (h *HashDiskBackedRowContainer) ReserveMarkMemoryMaybe(ctx context.Context) error { 865 if !h.UsingDisk() { 866 // We're assuming that the disk space is infinite, so we only need to 867 // reserve the memory for marks if we're using in-memory container. 868 if err := h.hmrc.ReserveMarkMemoryMaybe(ctx); err != nil { 869 return h.SpillToDisk(ctx) 870 } 871 } 872 return nil 873 } 874 875 // spillIfMemErr checks err and calls SpillToDisk if the given err is an out of 876 // memory error. Returns whether the HashDiskBackedRowContainer spilled to disk 877 // and an error if one occurred while doing so. 878 func (h *HashDiskBackedRowContainer) spillIfMemErr(ctx context.Context, err error) (bool, error) { 879 if !sqlbase.IsOutOfMemoryError(err) { 880 return false, nil 881 } 882 if spillErr := h.SpillToDisk(ctx); spillErr != nil { 883 return false, spillErr 884 } 885 log.VEventf(ctx, 2, "spilled to disk: %v", err) 886 return true, nil 887 } 888 889 // SpillToDisk creates a disk row container, injects all the data from the 890 // in-memory container into it, and clears the in-memory one afterwards. 891 func (h *HashDiskBackedRowContainer) SpillToDisk(ctx context.Context) error { 892 if h.UsingDisk() { 893 return errors.New("already using disk") 894 } 895 hdrc := MakeHashDiskRowContainer(h.diskMonitor, h.engine) 896 if err := hdrc.Init(ctx, h.shouldMark, h.types, h.storedEqCols, h.encodeNull); err != nil { 897 return err 898 } 899 900 // We compute the "current" keys of the iterators that will need to be 901 // recreated. 902 if err := h.computeKeysForAllRowsIterators(); err != nil { 903 return err 904 } 905 906 // rowIdx is used to look up the mark on the row and is only updated and used 907 // if marks are present. 908 rowIdx := 0 909 i := h.hmrc.NewFinalIterator(ctx) 910 defer i.Close() 911 for i.Rewind(); ; i.Next() { 912 if ok, err := i.Valid(); err != nil { 913 return err 914 } else if !ok { 915 break 916 } 917 row, err := i.Row() 918 if err != nil { 919 return err 920 } 921 if h.shouldMark && h.hmrc.marked != nil { 922 // We need to preserve the mark on this row. 923 copy(h.scratchEncRow, row) 924 h.scratchEncRow[len(h.types)] = sqlbase.EncDatum{Datum: tree.MakeDBool(tree.DBool(h.hmrc.marked[rowIdx]))} 925 row = h.scratchEncRow 926 rowIdx++ 927 } 928 if err := hdrc.AddRow(ctx, row); err != nil { 929 return err 930 } 931 } 932 h.hmrc.Clear(ctx) 933 934 h.src = &hdrc 935 h.hdrc = &hdrc 936 937 return h.recreateAllRowsIterators(ctx) 938 } 939 940 // NewBucketIterator implements the hashRowContainer interface. 941 func (h *HashDiskBackedRowContainer) NewBucketIterator( 942 ctx context.Context, row sqlbase.EncDatumRow, probeEqCols columns, 943 ) (RowMarkerIterator, error) { 944 return h.src.NewBucketIterator(ctx, row, probeEqCols) 945 } 946 947 // NewUnmarkedIterator implements the hashRowContainer interface. 948 func (h *HashDiskBackedRowContainer) NewUnmarkedIterator(ctx context.Context) RowIterator { 949 return h.src.NewUnmarkedIterator(ctx) 950 } 951 952 // UnsafeReset resets the container for reuse. The HashDiskBackedRowContainer 953 // will reset to using memory if it is using disk. 954 func (h *HashDiskBackedRowContainer) UnsafeReset(ctx context.Context) error { 955 h.allRowsIterators = h.allRowsIterators[:0] 956 if h.hdrc != nil { 957 h.hdrc.Close(ctx) 958 h.src = h.hmrc 959 h.hdrc = nil 960 return nil 961 } 962 return h.hmrc.UnsafeReset(ctx) 963 } 964 965 // Sort sorts the underlying row container based on stored equality columns 966 // which forces all rows from the same hash bucket to be contiguous. 967 func (h *HashDiskBackedRowContainer) Sort(ctx context.Context) { 968 if !h.UsingDisk() && len(h.storedEqCols) > 0 { 969 // We need to explicitly sort only if we're using in-memory container since 970 // if we're using disk, the underlying sortedDiskMap will be sorted 971 // already. 972 h.hmrc.Sort(ctx) 973 } 974 } 975 976 // AllRowsIterator iterates over all rows in HashDiskBackedRowContainer which 977 // should be initialized to not do marking. This iterator will be recreated 978 // in-place if the container spills to disk. 979 type AllRowsIterator struct { 980 RowIterator 981 982 container *HashDiskBackedRowContainer 983 } 984 985 // Close implements RowIterator interface. 986 func (i *AllRowsIterator) Close() { 987 i.RowIterator.Close() 988 for j, iterator := range i.container.allRowsIterators { 989 if i == iterator { 990 i.container.allRowsIterators = append(i.container.allRowsIterators[:j], i.container.allRowsIterators[j+1:]...) 991 return 992 } 993 } 994 } 995 996 // NewAllRowsIterator creates AllRowsIterator that can iterate over all rows 997 // (equivalent to an unmarked iterator when the container doesn't do marking) 998 // and will be recreated if the spilling to disk occurs. 999 func (h *HashDiskBackedRowContainer) NewAllRowsIterator( 1000 ctx context.Context, 1001 ) (*AllRowsIterator, error) { 1002 if h.shouldMark { 1003 return nil, errors.Errorf("AllRowsIterator can only be created when the container doesn't do marking") 1004 } 1005 i := AllRowsIterator{h.src.NewUnmarkedIterator(ctx), h} 1006 h.allRowsIterators = append(h.allRowsIterators, &i) 1007 return &i, nil 1008 } 1009 1010 func (h *HashDiskBackedRowContainer) computeKeysForAllRowsIterators() error { 1011 var oldIterator *hashMemRowIterator 1012 var ok bool 1013 for _, iterator := range h.allRowsIterators { 1014 if oldIterator, ok = (*iterator).RowIterator.(*hashMemRowIterator); !ok { 1015 return errors.Errorf("the iterator is unexpectedly not hashMemRowIterator") 1016 } 1017 if err := oldIterator.computeKey(); err != nil { 1018 return err 1019 } 1020 } 1021 return nil 1022 } 1023 1024 func (h *HashDiskBackedRowContainer) recreateAllRowsIterators(ctx context.Context) error { 1025 var oldIterator *hashMemRowIterator 1026 var ok bool 1027 for _, iterator := range h.allRowsIterators { 1028 if oldIterator, ok = (*iterator).RowIterator.(*hashMemRowIterator); !ok { 1029 return errors.Errorf("the iterator is unexpectedly not hashMemRowIterator") 1030 } 1031 newIterator := h.NewUnmarkedIterator(ctx) 1032 newIterator.(*diskRowIterator).SeekGE(oldIterator.curKey) 1033 (*iterator).RowIterator.Close() 1034 iterator.RowIterator = newIterator 1035 } 1036 return nil 1037 }