github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/rowcontainer/row_container_test.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 "context" 15 "fmt" 16 "math" 17 "math/rand" 18 "sort" 19 "testing" 20 21 "github.com/cockroachdb/cockroach/pkg/base" 22 "github.com/cockroachdb/cockroach/pkg/settings/cluster" 23 "github.com/cockroachdb/cockroach/pkg/sql/execinfra" 24 "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" 25 "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" 26 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 27 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 28 "github.com/cockroachdb/cockroach/pkg/sql/types" 29 "github.com/cockroachdb/cockroach/pkg/storage" 30 "github.com/cockroachdb/cockroach/pkg/util/encoding" 31 "github.com/cockroachdb/cockroach/pkg/util/leaktest" 32 "github.com/cockroachdb/cockroach/pkg/util/mon" 33 "github.com/cockroachdb/cockroach/pkg/util/randutil" 34 "github.com/cockroachdb/cockroach/pkg/util/timeutil" 35 "github.com/cockroachdb/errors" 36 "github.com/stretchr/testify/require" 37 ) 38 39 // verifyRows verifies that the rows read with the given RowIterator match up 40 // with the given rows. evalCtx and ordering are used to compare rows. 41 func verifyRows( 42 ctx context.Context, 43 i RowIterator, 44 expectedRows sqlbase.EncDatumRows, 45 evalCtx *tree.EvalContext, 46 ordering sqlbase.ColumnOrdering, 47 ) error { 48 for i.Rewind(); ; i.Next() { 49 if ok, err := i.Valid(); err != nil { 50 return err 51 } else if !ok { 52 break 53 } 54 row, err := i.Row() 55 if err != nil { 56 return err 57 } 58 if cmp, err := compareRows( 59 sqlbase.OneIntCol, row, expectedRows[0], evalCtx, &sqlbase.DatumAlloc{}, ordering, 60 ); err != nil { 61 return err 62 } else if cmp != 0 { 63 return fmt.Errorf("unexpected row %v, expected %v", row, expectedRows[0]) 64 } 65 expectedRows = expectedRows[1:] 66 } 67 if len(expectedRows) != 0 { 68 return fmt.Errorf("iterator missed %d row(s)", len(expectedRows)) 69 } 70 return nil 71 } 72 73 // TestRowContainerReplaceMax verifies that MaybeReplaceMax correctly adjusts 74 // the memory accounting. 75 func TestRowContainerReplaceMax(t *testing.T) { 76 defer leaktest.AfterTest(t)() 77 78 ctx := context.Background() 79 rng, _ := randutil.NewPseudoRand() 80 81 st := cluster.MakeTestingClusterSettings() 82 evalCtx := tree.NewTestingEvalContext(st) 83 defer evalCtx.Stop(ctx) 84 85 makeRow := func(intVal int, strLen int) sqlbase.EncDatumRow { 86 var b []byte 87 for i := 0; i < strLen; i++ { 88 b = append(b, 'a') 89 } 90 return sqlbase.EncDatumRow{ 91 sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(intVal))), 92 sqlbase.DatumToEncDatum(types.String, tree.NewDString(string(b))), 93 } 94 } 95 96 m := mon.MakeUnlimitedMonitor( 97 context.Background(), "test", mon.MemoryResource, nil, nil, math.MaxInt64, st, 98 ) 99 defer m.Stop(ctx) 100 101 var mc MemRowContainer 102 mc.InitWithMon( 103 sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}}, 104 []*types.T{types.Int, types.String}, evalCtx, &m, 0, /* rowCapacity */ 105 ) 106 defer mc.Close(ctx) 107 108 // Initialize the heap with small rows. 109 for i := 0; i < 1000; i++ { 110 err := mc.AddRow(ctx, makeRow(rng.Intn(10000), rng.Intn(10))) 111 if err != nil { 112 t.Fatal(err) 113 } 114 } 115 mc.InitTopK() 116 // Replace some of the rows with large rows. 117 for i := 0; i < 1000; i++ { 118 err := mc.MaybeReplaceMax(ctx, makeRow(rng.Intn(10000), rng.Intn(100))) 119 if err != nil { 120 t.Fatal(err) 121 } 122 } 123 // Now pop the rows, which shrinks the memory account according to the current 124 // row sizes. If we did not account for the larger rows, this will panic. 125 for mc.Len() > 0 { 126 mc.PopFirst() 127 } 128 } 129 130 func TestRowContainerIterators(t *testing.T) { 131 defer leaktest.AfterTest(t)() 132 133 ctx := context.Background() 134 st := cluster.MakeTestingClusterSettings() 135 evalCtx := tree.NewTestingEvalContext(st) 136 defer evalCtx.Stop(ctx) 137 138 const numRows = 10 139 const numCols = 1 140 rows := sqlbase.MakeIntRows(numRows, numCols) 141 ordering := sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} 142 143 var mc MemRowContainer 144 mc.Init( 145 ordering, 146 sqlbase.OneIntCol, 147 evalCtx, 148 ) 149 defer mc.Close(ctx) 150 151 for _, row := range rows { 152 if err := mc.AddRow(ctx, row); err != nil { 153 t.Fatal(err) 154 } 155 } 156 157 // NewIterator verifies that we read the expected rows from the 158 // MemRowContainer and can recreate an iterator. 159 t.Run("NewIterator", func(t *testing.T) { 160 for k := 0; k < 2; k++ { 161 func() { 162 i := mc.NewIterator(ctx) 163 defer i.Close() 164 if err := verifyRows(ctx, i, rows, evalCtx, ordering); err != nil { 165 t.Fatalf("rows mismatch on the run number %d: %s", k+1, err) 166 } 167 }() 168 } 169 }) 170 171 // NewFinalIterator verifies that we read the expected rows from the 172 // MemRowContainer and as we do so, these rows are deleted from the 173 // MemRowContainer. 174 t.Run("NewFinalIterator", func(t *testing.T) { 175 i := mc.NewFinalIterator(ctx) 176 defer i.Close() 177 if err := verifyRows(ctx, i, rows, evalCtx, ordering); err != nil { 178 t.Fatal(err) 179 } 180 if mc.Len() != 0 { 181 t.Fatal("MemRowContainer is not empty") 182 } 183 }) 184 } 185 186 func TestDiskBackedRowContainer(t *testing.T) { 187 defer leaktest.AfterTest(t)() 188 189 ctx := context.Background() 190 st := cluster.MakeTestingClusterSettings() 191 evalCtx := tree.MakeTestingEvalContext(st) 192 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.TempStorageConfig{InMemory: true}, base.DefaultTestStoreSpec) 193 if err != nil { 194 t.Fatal(err) 195 } 196 defer tempEngine.Close() 197 198 // These monitors are started and stopped by subtests. 199 memoryMonitor := mon.MakeMonitor( 200 "test-mem", 201 mon.MemoryResource, 202 nil, /* curCount */ 203 nil, /* maxHist */ 204 -1, /* increment */ 205 math.MaxInt64, /* noteworthy */ 206 st, 207 ) 208 diskMonitor := mon.MakeMonitor( 209 "test-disk", 210 mon.DiskResource, 211 nil, /* curCount */ 212 nil, /* maxHist */ 213 -1, /* increment */ 214 math.MaxInt64, /* noteworthy */ 215 st, 216 ) 217 218 const numRows = 10 219 const numCols = 1 220 rows := sqlbase.MakeIntRows(numRows, numCols) 221 ordering := sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} 222 223 rc := DiskBackedRowContainer{} 224 rc.Init( 225 ordering, 226 sqlbase.OneIntCol, 227 &evalCtx, 228 tempEngine, 229 &memoryMonitor, 230 &diskMonitor, 231 0, /* rowCapacity */ 232 ) 233 defer rc.Close(ctx) 234 235 // NormalRun adds rows to a DiskBackedRowContainer, makes it spill to disk 236 // halfway through, keeps on adding rows, and then verifies that all rows 237 // were properly added to the DiskBackedRowContainer. 238 t.Run("NormalRun", func(t *testing.T) { 239 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 240 defer memoryMonitor.Stop(ctx) 241 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 242 defer diskMonitor.Stop(ctx) 243 244 defer func() { 245 if err := rc.UnsafeReset(ctx); err != nil { 246 t.Fatal(err) 247 } 248 }() 249 250 mid := len(rows) / 2 251 for i := 0; i < mid; i++ { 252 if err := rc.AddRow(ctx, rows[i]); err != nil { 253 t.Fatal(err) 254 } 255 } 256 if rc.Spilled() { 257 t.Fatal("unexpectedly using disk") 258 } 259 func() { 260 i := rc.NewIterator(ctx) 261 defer i.Close() 262 if err := verifyRows(ctx, i, rows[:mid], &evalCtx, ordering); err != nil { 263 t.Fatalf("verifying memory rows failed with: %s", err) 264 } 265 }() 266 if err := rc.SpillToDisk(ctx); err != nil { 267 t.Fatal(err) 268 } 269 if !rc.Spilled() { 270 t.Fatal("unexpectedly using memory") 271 } 272 for i := mid; i < len(rows); i++ { 273 if err := rc.AddRow(ctx, rows[i]); err != nil { 274 t.Fatal(err) 275 } 276 } 277 func() { 278 i := rc.NewIterator(ctx) 279 defer i.Close() 280 if err := verifyRows(ctx, i, rows, &evalCtx, ordering); err != nil { 281 t.Fatalf("verifying disk rows failed with: %s", err) 282 } 283 }() 284 }) 285 286 t.Run("AddRowOutOfMem", func(t *testing.T) { 287 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(1)) 288 defer memoryMonitor.Stop(ctx) 289 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 290 defer diskMonitor.Stop(ctx) 291 292 defer func() { 293 if err := rc.UnsafeReset(ctx); err != nil { 294 t.Fatal(err) 295 } 296 }() 297 298 if err := rc.AddRow(ctx, rows[0]); err != nil { 299 t.Fatal(err) 300 } 301 if !rc.Spilled() { 302 t.Fatal("expected to have spilled to disk") 303 } 304 if diskMonitor.AllocBytes() == 0 { 305 t.Fatal("disk monitor reports no disk usage") 306 } 307 if memoryMonitor.AllocBytes() > 0 { 308 t.Fatal("memory monitor reports unexpected usage") 309 } 310 }) 311 312 t.Run("AddRowOutOfDisk", func(t *testing.T) { 313 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(1)) 314 defer memoryMonitor.Stop(ctx) 315 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(1)) 316 defer diskMonitor.Stop(ctx) 317 318 defer func() { 319 if err := rc.UnsafeReset(ctx); err != nil { 320 t.Fatal(err) 321 } 322 }() 323 324 err := rc.AddRow(ctx, rows[0]) 325 if code := pgerror.GetPGCode(err); code != pgcode.DiskFull { 326 t.Fatalf( 327 "unexpected error %v, expected disk full error %s", err, pgcode.DiskFull, 328 ) 329 } 330 if !rc.Spilled() { 331 t.Fatal("expected to have tried to spill to disk") 332 } 333 if diskMonitor.AllocBytes() != 0 { 334 t.Fatal("disk monitor reports unexpected usage") 335 } 336 if memoryMonitor.AllocBytes() != 0 { 337 t.Fatal("memory monitor reports unexpected usage") 338 } 339 }) 340 } 341 342 func TestDiskBackedRowContainerDeDuping(t *testing.T) { 343 defer leaktest.AfterTest(t)() 344 345 ctx := context.Background() 346 st := cluster.MakeTestingClusterSettings() 347 evalCtx := tree.MakeTestingEvalContext(st) 348 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.TempStorageConfig{InMemory: true}, base.DefaultTestStoreSpec) 349 if err != nil { 350 t.Fatal(err) 351 } 352 defer tempEngine.Close() 353 354 memoryMonitor := mon.MakeMonitor( 355 "test-mem", 356 mon.MemoryResource, 357 nil, /* curCount */ 358 nil, /* maxHist */ 359 -1, /* increment */ 360 math.MaxInt64, /* noteworthy */ 361 st, 362 ) 363 diskMonitor := execinfra.NewTestDiskMonitor(ctx, st) 364 365 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 366 defer memoryMonitor.Stop(ctx) 367 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 368 defer diskMonitor.Stop(ctx) 369 370 numRows := 10 371 // Use 2 columns with both ascending and descending ordering to exercise 372 // all possibilities with the randomly chosen types. 373 const numCols = 2 374 ordering := sqlbase.ColumnOrdering{ 375 sqlbase.ColumnOrderInfo{ 376 ColIdx: 0, 377 Direction: encoding.Ascending, 378 }, 379 sqlbase.ColumnOrderInfo{ 380 ColIdx: 1, 381 Direction: encoding.Descending, 382 }, 383 } 384 rng, _ := randutil.NewPseudoRand() 385 // Use random types and random rows. 386 types := sqlbase.RandSortingTypes(rng, numCols) 387 numRows, rows := makeUniqueRows(t, &evalCtx, rng, numRows, types, ordering) 388 rc := DiskBackedRowContainer{} 389 rc.Init( 390 ordering, 391 types, 392 &evalCtx, 393 tempEngine, 394 &memoryMonitor, 395 diskMonitor, 396 0, /* rowCapacity */ 397 ) 398 defer rc.Close(ctx) 399 rc.DoDeDuplicate() 400 401 for run := 0; run < 2; run++ { 402 // Add rows to a DiskBackedRowContainer, and make it spill to disk halfway 403 // through, and keep adding rows. 404 mid := numRows / 2 405 for i := 0; i < mid; i++ { 406 idx, err := rc.AddRowWithDeDup(ctx, rows[i]) 407 require.NoError(t, err) 408 require.Equal(t, i, idx) 409 } 410 require.Equal(t, false, rc.UsingDisk()) 411 require.NoError(t, rc.SpillToDisk(ctx)) 412 require.Equal(t, true, rc.UsingDisk()) 413 414 for i := mid; i < numRows; i++ { 415 idx, err := rc.AddRowWithDeDup(ctx, rows[i]) 416 require.NoError(t, err) 417 require.Equal(t, i, idx) 418 } 419 // Reset and reorder the rows for the next run. 420 rand.Shuffle(numRows, func(i, j int) { 421 rows[i], rows[j] = rows[j], rows[i] 422 }) 423 require.NoError(t, rc.UnsafeReset(ctx)) 424 } 425 } 426 427 // verifyOrdering checks whether the rows in src are ordered according to 428 // ordering. 429 func verifyOrdering( 430 ctx context.Context, 431 evalCtx *tree.EvalContext, 432 src SortableRowContainer, 433 types []*types.T, 434 ordering sqlbase.ColumnOrdering, 435 ) error { 436 var datumAlloc sqlbase.DatumAlloc 437 var rowAlloc sqlbase.EncDatumRowAlloc 438 var prevRow sqlbase.EncDatumRow 439 i := src.NewIterator(ctx) 440 defer i.Close() 441 for i.Rewind(); ; i.Next() { 442 if ok, err := i.Valid(); err != nil { 443 return err 444 } else if !ok { 445 break 446 } 447 row, err := i.Row() 448 if err != nil { 449 return err 450 } 451 if prevRow != nil { 452 if cmp, err := prevRow.Compare(types, &datumAlloc, ordering, evalCtx, row); err != nil { 453 return err 454 } else if cmp > 0 { 455 return errors.Errorf("rows are not ordered as expected: %s was before %s", prevRow.String(types), row.String(types)) 456 } 457 } 458 prevRow = rowAlloc.CopyRow(row) 459 } 460 return nil 461 } 462 463 func TestDiskBackedIndexedRowContainer(t *testing.T) { 464 defer leaktest.AfterTest(t)() 465 466 ctx := context.Background() 467 st := cluster.MakeTestingClusterSettings() 468 evalCtx := tree.MakeTestingEvalContext(st) 469 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.TempStorageConfig{InMemory: true}, base.DefaultTestStoreSpec) 470 if err != nil { 471 t.Fatal(err) 472 } 473 defer tempEngine.Close() 474 475 memoryMonitor := mon.MakeMonitor( 476 "test-mem", 477 mon.MemoryResource, 478 nil, /* curCount */ 479 nil, /* maxHist */ 480 -1, /* increment */ 481 math.MaxInt64, /* noteworthy */ 482 st, 483 ) 484 diskMonitor := mon.MakeMonitor( 485 "test-disk", 486 mon.DiskResource, 487 nil, /* curCount */ 488 nil, /* maxHist */ 489 -1, /* increment */ 490 math.MaxInt64, /* noteworthy */ 491 st, 492 ) 493 494 const numTestRuns = 10 495 const numRows = 10 496 const numCols = 2 497 ordering := sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} 498 newOrdering := sqlbase.ColumnOrdering{{ColIdx: 1, Direction: encoding.Ascending}} 499 500 rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) 501 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 502 defer memoryMonitor.Stop(ctx) 503 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 504 defer diskMonitor.Stop(ctx) 505 506 // SpillingHalfway adds half of all rows into DiskBackedIndexedRowContainer, 507 // forces it to spill to disk, adds the second half into the container, and 508 // verifies that the rows are read correctly (along with the corresponding 509 // index). 510 t.Run("SpillingHalfway", func(t *testing.T) { 511 for i := 0; i < numTestRuns; i++ { 512 rows := make([]sqlbase.EncDatumRow, numRows) 513 types := sqlbase.RandSortingTypes(rng, numCols) 514 for i := 0; i < numRows; i++ { 515 rows[i] = sqlbase.RandEncDatumRowOfTypes(rng, types) 516 } 517 518 func() { 519 rc := NewDiskBackedIndexedRowContainer(ordering, types, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 520 defer rc.Close(ctx) 521 mid := numRows / 2 522 for i := 0; i < mid; i++ { 523 if err := rc.AddRow(ctx, rows[i]); err != nil { 524 t.Fatal(err) 525 } 526 } 527 if rc.Spilled() { 528 t.Fatal("unexpectedly using disk") 529 } 530 if err := rc.SpillToDisk(ctx); err != nil { 531 t.Fatal(err) 532 } 533 if !rc.Spilled() { 534 t.Fatal("unexpectedly using memory") 535 } 536 for i := mid; i < numRows; i++ { 537 if err := rc.AddRow(ctx, rows[i]); err != nil { 538 t.Fatal(err) 539 } 540 } 541 542 // Check equality of the row we wrote and the row we read. 543 for i := 0; i < numRows; i++ { 544 readRow, err := rc.GetRow(ctx, i) 545 if err != nil { 546 t.Fatalf("unexpected error: %v", err) 547 } 548 writtenRow := rows[readRow.GetIdx()] 549 for col := range writtenRow { 550 datum, err := readRow.GetDatum(col) 551 if err != nil { 552 t.Fatalf("unexpected error: %v", err) 553 } 554 if cmp := datum.Compare(&evalCtx, writtenRow[col].Datum); cmp != 0 { 555 t.Fatalf("read row is not equal to written one") 556 } 557 } 558 } 559 }() 560 } 561 }) 562 563 // TestGetRow adds all rows into DiskBackedIndexedRowContainer, sorts them, 564 // and checks that both the index and the row are what we expect by GetRow() 565 // to be returned. Then, it spills to disk and does the same check again. 566 t.Run("TestGetRow", func(t *testing.T) { 567 for i := 0; i < numTestRuns; i++ { 568 rows := make([]sqlbase.EncDatumRow, numRows) 569 sortedRows := indexedRows{rows: make([]IndexedRow, numRows)} 570 types := sqlbase.RandSortingTypes(rng, numCols) 571 for i := 0; i < numRows; i++ { 572 rows[i] = sqlbase.RandEncDatumRowOfTypes(rng, types) 573 sortedRows.rows[i] = IndexedRow{Idx: i, Row: rows[i]} 574 } 575 576 sorter := rowsSorter{evalCtx: &evalCtx, rows: sortedRows, ordering: ordering} 577 sort.Sort(&sorter) 578 if sorter.err != nil { 579 t.Fatal(sorter.err) 580 } 581 582 func() { 583 rc := NewDiskBackedIndexedRowContainer(ordering, types, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 584 defer rc.Close(ctx) 585 for _, row := range rows { 586 if err := rc.AddRow(ctx, row); err != nil { 587 t.Fatal(err) 588 } 589 } 590 if rc.Spilled() { 591 t.Fatal("unexpectedly using disk") 592 } 593 rc.Sort(ctx) 594 595 // Check that GetRow returns the row we expect at each position. 596 for i := 0; i < numRows; i++ { 597 readRow, err := rc.GetRow(ctx, i) 598 if err != nil { 599 t.Fatalf("unexpected error: %v", err) 600 } 601 expectedRow := sortedRows.rows[i] 602 if readRow.GetIdx() != expectedRow.GetIdx() { 603 t.Fatalf("read row has different idx that what we expect") 604 } 605 for col, expectedDatum := range expectedRow.Row { 606 readDatum, err := readRow.GetDatum(col) 607 if err != nil { 608 t.Fatalf("unexpected error: %v", err) 609 } 610 if cmp := readDatum.Compare(&evalCtx, expectedDatum.Datum); cmp != 0 { 611 t.Fatalf("read row is not equal to expected one") 612 } 613 } 614 } 615 if err := rc.SpillToDisk(ctx); err != nil { 616 t.Fatal(err) 617 } 618 if !rc.Spilled() { 619 t.Fatal("unexpectedly using memory") 620 } 621 622 // Check that GetRow returns the row we expect at each position. 623 for i := 0; i < numRows; i++ { 624 readRow, err := rc.GetRow(ctx, i) 625 if err != nil { 626 t.Fatalf("unexpected error: %v", err) 627 } 628 expectedRow := sortedRows.rows[i] 629 if readRow.GetIdx() != expectedRow.GetIdx() { 630 t.Fatalf("read row has different idx that what we expect") 631 } 632 for col, expectedDatum := range expectedRow.Row { 633 readDatum, err := readRow.GetDatum(col) 634 if err != nil { 635 t.Fatalf("unexpected error: %v", err) 636 } 637 if cmp := readDatum.Compare(&evalCtx, expectedDatum.Datum); cmp != 0 { 638 t.Fatalf("read row is not equal to expected one") 639 } 640 } 641 } 642 }() 643 } 644 }) 645 646 // TestGetRowFromDiskWithLimitedMemory forces the container to spill to disk, 647 // adds all rows to it, sorts them, and checks that both the index and the 648 // row are what we expect by GetRow() to be returned. The goal is to test the 649 // behavior of capping the cache size and reusing the memory of the first 650 // rows in the cache, so we use the memory budget that accommodates only 651 // about half of all rows in the cache. 652 t.Run("TestGetRowWithLimitedMemory", func(t *testing.T) { 653 for i := 0; i < numTestRuns; i++ { 654 budget := int64(10240) 655 memoryUsage := int64(0) 656 rows := make([]sqlbase.EncDatumRow, 0, numRows) 657 sortedRows := indexedRows{rows: make([]IndexedRow, 0, numRows)} 658 types := sqlbase.RandSortingTypes(rng, numCols) 659 for memoryUsage < 2*budget { 660 row := sqlbase.RandEncDatumRowOfTypes(rng, types) 661 memoryUsage += int64(row.Size()) 662 rows = append(rows, row) 663 sortedRows.rows = append(sortedRows.rows, IndexedRow{Idx: len(sortedRows.rows), Row: row}) 664 } 665 666 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(budget)) 667 defer memoryMonitor.Stop(ctx) 668 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 669 defer diskMonitor.Stop(ctx) 670 671 sorter := rowsSorter{evalCtx: &evalCtx, rows: sortedRows, ordering: ordering} 672 sort.Sort(&sorter) 673 if sorter.err != nil { 674 t.Fatal(sorter.err) 675 } 676 677 func() { 678 rc := NewDiskBackedIndexedRowContainer(ordering, types, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 679 defer rc.Close(ctx) 680 if err := rc.SpillToDisk(ctx); err != nil { 681 t.Fatal(err) 682 } 683 for _, row := range rows { 684 if err := rc.AddRow(ctx, row); err != nil { 685 t.Fatal(err) 686 } 687 } 688 if !rc.Spilled() { 689 t.Fatal("unexpectedly using memory") 690 } 691 rc.Sort(ctx) 692 693 // Check that GetRow returns the row we expect at each position. 694 for i := 0; i < len(rows); i++ { 695 readRow, err := rc.GetRow(ctx, i) 696 if err != nil { 697 t.Fatalf("unexpected error: %v", err) 698 } 699 expectedRow := sortedRows.rows[i] 700 readOrderingDatum, err := readRow.GetDatum(ordering[0].ColIdx) 701 if err != nil { 702 t.Fatalf("unexpected error: %v", err) 703 } 704 if readOrderingDatum.Compare(&evalCtx, expectedRow.Row[ordering[0].ColIdx].Datum) != 0 { 705 // We're skipping comparison if both rows are equal on the ordering 706 // column since in this case the order of indexed rows after 707 // sorting is nondeterministic. 708 if readRow.GetIdx() != expectedRow.GetIdx() { 709 t.Fatalf("read row has different idx that what we expect") 710 } 711 for col, expectedDatum := range expectedRow.Row { 712 readDatum, err := readRow.GetDatum(col) 713 if err != nil { 714 t.Fatalf("unexpected error: %v", err) 715 } 716 if cmp := readDatum.Compare(&evalCtx, expectedDatum.Datum); cmp != 0 { 717 t.Fatalf("read row is not equal to expected one") 718 } 719 } 720 } 721 } 722 }() 723 } 724 }) 725 726 // ReorderingInMemory initializes a DiskBackedIndexedRowContainer with one 727 // ordering, adds all rows to it, sorts it and makes sure that the rows are 728 // sorted as expected. Then, it reorders the container to a different 729 // ordering, sorts it and verifies that the rows are in the order we expect. 730 // Only in-memory containers should be used. 731 t.Run("ReorderingInMemory", func(t *testing.T) { 732 for i := 0; i < numTestRuns; i++ { 733 rows := make([]sqlbase.EncDatumRow, numRows) 734 typs := sqlbase.RandSortingTypes(rng, numCols) 735 for i := 0; i < numRows; i++ { 736 rows[i] = sqlbase.RandEncDatumRowOfTypes(rng, typs) 737 } 738 storedTypes := make([]*types.T, len(typs)+1) 739 copy(storedTypes, typs) 740 // The container will add an extra int column for indices. 741 storedTypes[len(typs)] = sqlbase.OneIntCol[0] 742 743 func() { 744 rc := NewDiskBackedIndexedRowContainer(ordering, typs, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 745 defer rc.Close(ctx) 746 for i := 0; i < numRows; i++ { 747 if err := rc.AddRow(ctx, rows[i]); err != nil { 748 t.Fatal(err) 749 } 750 } 751 rc.Sort(ctx) 752 if err := verifyOrdering(ctx, &evalCtx, rc, storedTypes, ordering); err != nil { 753 t.Fatal(err) 754 } 755 756 if err := rc.Reorder(ctx, newOrdering); err != nil { 757 t.Fatal(err) 758 } 759 rc.Sort(ctx) 760 if err := verifyOrdering(ctx, &evalCtx, rc, storedTypes, newOrdering); err != nil { 761 t.Fatal(err) 762 } 763 if rc.Spilled() { 764 t.Fatal("unexpectedly using disk") 765 } 766 }() 767 } 768 }) 769 770 // ReorderingOnDisk is the same as ReorderingInMemory except here the 771 // container is forced to spill to disk right after initialization. 772 t.Run("ReorderingOnDisk", func(t *testing.T) { 773 for i := 0; i < numTestRuns; i++ { 774 rows := make([]sqlbase.EncDatumRow, numRows) 775 typs := sqlbase.RandSortingTypes(rng, numCols) 776 for i := 0; i < numRows; i++ { 777 rows[i] = sqlbase.RandEncDatumRowOfTypes(rng, typs) 778 } 779 storedTypes := make([]*types.T, len(typs)+1) 780 copy(storedTypes, typs) 781 // The container will add an extra int column for indices. 782 storedTypes[len(typs)] = sqlbase.OneIntCol[0] 783 784 func() { 785 d := NewDiskBackedIndexedRowContainer(ordering, typs, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 786 defer d.Close(ctx) 787 if err := d.SpillToDisk(ctx); err != nil { 788 t.Fatal(err) 789 } 790 for i := 0; i < numRows; i++ { 791 if err := d.AddRow(ctx, rows[i]); err != nil { 792 t.Fatal(err) 793 } 794 } 795 d.Sort(ctx) 796 if err := verifyOrdering(ctx, &evalCtx, d, storedTypes, ordering); err != nil { 797 t.Fatal(err) 798 } 799 800 if err := d.Reorder(ctx, newOrdering); err != nil { 801 t.Fatal(err) 802 } 803 d.Sort(ctx) 804 if err := verifyOrdering(ctx, &evalCtx, d, storedTypes, newOrdering); err != nil { 805 t.Fatal(err) 806 } 807 if !d.UsingDisk() { 808 t.Fatal("unexpectedly using memory") 809 } 810 }() 811 } 812 }) 813 } 814 815 // indexedRows are rows with the corresponding indices. 816 type indexedRows struct { 817 rows []IndexedRow 818 } 819 820 // Len implements tree.IndexedRows interface. 821 func (ir indexedRows) Len() int { 822 return len(ir.rows) 823 } 824 825 // TODO(yuzefovich): this is a duplicate of partitionSorter from windower.go. 826 // There are possibly couple of other duplicates as well in other files, so we 827 // should refactor it and probably extract the code into a new package. 828 type rowsSorter struct { 829 evalCtx *tree.EvalContext 830 rows indexedRows 831 ordering sqlbase.ColumnOrdering 832 err error 833 } 834 835 func (n *rowsSorter) Len() int { return n.rows.Len() } 836 837 func (n *rowsSorter) Swap(i, j int) { 838 n.rows.rows[i], n.rows.rows[j] = n.rows.rows[j], n.rows.rows[i] 839 } 840 func (n *rowsSorter) Less(i, j int) bool { 841 if n.err != nil { 842 // An error occurred in previous calls to Less(). We want to be done with 843 // sorting and to propagate that error to the caller of Sort(). 844 return false 845 } 846 cmp, err := n.Compare(i, j) 847 if err != nil { 848 n.err = err 849 return false 850 } 851 return cmp < 0 852 } 853 854 func (n *rowsSorter) Compare(i, j int) (int, error) { 855 ra, rb := n.rows.rows[i], n.rows.rows[j] 856 for _, o := range n.ordering { 857 da, err := ra.GetDatum(o.ColIdx) 858 if err != nil { 859 return 0, err 860 } 861 db, err := rb.GetDatum(o.ColIdx) 862 if err != nil { 863 return 0, err 864 } 865 if c := da.Compare(n.evalCtx, db); c != 0 { 866 if o.Direction != encoding.Ascending { 867 return -c, nil 868 } 869 return c, nil 870 } 871 } 872 return 0, nil 873 } 874 875 // generateAccessPattern populates int slices with position of rows to be 876 // accessed by GetRow(). The goal is to simulate an access pattern that would 877 // resemble a real one that might occur while window functions are computed. 878 func generateAccessPattern(numRows int) []int { 879 rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) 880 const avgPeerGroupSize = 100 881 accessPattern := make([]int, 0, 2*numRows) 882 nextPeerGroupStart := 0 883 for { 884 peerGroupSize := int(rng.NormFloat64()*avgPeerGroupSize) + avgPeerGroupSize 885 if peerGroupSize < 1 { 886 peerGroupSize = 1 887 } 888 if nextPeerGroupStart+peerGroupSize > numRows { 889 peerGroupSize = numRows - nextPeerGroupStart 890 } 891 accessPattern = append(accessPattern, nextPeerGroupStart) 892 for j := 1; j < peerGroupSize; j++ { 893 accessPattern = append(accessPattern, accessPattern[len(accessPattern)-1]+1) 894 } 895 for j := 0; j < peerGroupSize; j++ { 896 accessPattern = append(accessPattern, accessPattern[len(accessPattern)-peerGroupSize]) 897 } 898 nextPeerGroupStart += peerGroupSize 899 if nextPeerGroupStart == numRows { 900 return accessPattern 901 } 902 } 903 } 904 905 func BenchmarkDiskBackedIndexedRowContainer(b *testing.B) { 906 const numCols = 1 907 const numRows = 100000 908 909 ctx := context.Background() 910 st := cluster.MakeTestingClusterSettings() 911 evalCtx := tree.MakeTestingEvalContext(st) 912 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.TempStorageConfig{InMemory: true}, base.DefaultTestStoreSpec) 913 if err != nil { 914 b.Fatal(err) 915 } 916 defer tempEngine.Close() 917 918 memoryMonitor := mon.MakeMonitor( 919 "test-mem", 920 mon.MemoryResource, 921 nil, /* curCount */ 922 nil, /* maxHist */ 923 -1, /* increment */ 924 math.MaxInt64, /* noteworthy */ 925 st, 926 ) 927 diskMonitor := mon.MakeMonitor( 928 "test-disk", 929 mon.DiskResource, 930 nil, /* curCount */ 931 nil, /* maxHist */ 932 -1, /* increment */ 933 math.MaxInt64, /* noteworthy */ 934 st, 935 ) 936 rows := sqlbase.MakeIntRows(numRows, numCols) 937 memoryMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 938 defer memoryMonitor.Stop(ctx) 939 diskMonitor.Start(ctx, nil, mon.MakeStandaloneBudget(math.MaxInt64)) 940 defer diskMonitor.Stop(ctx) 941 942 accessPattern := generateAccessPattern(numRows) 943 944 b.Run("InMemory", func(b *testing.B) { 945 rc := NewDiskBackedIndexedRowContainer(nil, sqlbase.OneIntCol, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 946 defer rc.Close(ctx) 947 for i := 0; i < len(rows); i++ { 948 if err := rc.AddRow(ctx, rows[i]); err != nil { 949 b.Fatal(err) 950 } 951 } 952 if rc.Spilled() { 953 b.Fatal("unexpectedly using disk") 954 } 955 b.SetBytes(int64(8 * numCols)) 956 b.ResetTimer() 957 for i := 0; i < b.N; i++ { 958 pos := accessPattern[i%len(accessPattern)] 959 if _, err := rc.GetRow(ctx, pos); err != nil { 960 b.Fatal(err) 961 } 962 } 963 b.StopTimer() 964 }) 965 966 b.Run("OnDiskWithCache", func(b *testing.B) { 967 rc := NewDiskBackedIndexedRowContainer(nil, sqlbase.OneIntCol, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 968 defer rc.Close(ctx) 969 if err := rc.SpillToDisk(ctx); err != nil { 970 b.Fatal(err) 971 } 972 for i := 0; i < len(rows); i++ { 973 if err := rc.AddRow(ctx, rows[i]); err != nil { 974 b.Fatal(err) 975 } 976 } 977 if !rc.Spilled() { 978 b.Fatal("unexpectedly using memory") 979 } 980 b.SetBytes(int64(8 * numCols)) 981 b.ResetTimer() 982 for i := 0; i < b.N; i++ { 983 pos := accessPattern[i%len(accessPattern)] 984 if _, err := rc.GetRow(ctx, pos); err != nil { 985 b.Fatal(err) 986 } 987 } 988 b.StopTimer() 989 }) 990 991 b.Run("OnDiskWithoutCache", func(b *testing.B) { 992 rc := NewDiskBackedIndexedRowContainer(nil, sqlbase.OneIntCol, &evalCtx, tempEngine, &memoryMonitor, &diskMonitor, 0 /* rowCapacity */) 993 defer rc.Close(ctx) 994 if err := rc.SpillToDisk(ctx); err != nil { 995 b.Fatal(err) 996 } 997 for i := 0; i < len(rows); i++ { 998 if err := rc.AddRow(ctx, rows[i]); err != nil { 999 b.Fatal(err) 1000 } 1001 } 1002 if !rc.Spilled() { 1003 b.Fatal("unexpectedly using memory") 1004 } 1005 b.SetBytes(int64(8 * numCols)) 1006 b.ResetTimer() 1007 for i := 0; i < b.N; i++ { 1008 pos := accessPattern[i%len(accessPattern)] 1009 _ = rc.getRowWithoutCache(ctx, pos) 1010 } 1011 b.StopTimer() 1012 }) 1013 }