github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/rowexec/joinreader_test.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 // This file doesn't live next to execinfra/joinreader.go in order to avoid 12 // the import cycle with distsqlutils. 13 14 package rowexec 15 16 import ( 17 "context" 18 "fmt" 19 "math" 20 "sort" 21 "strings" 22 "testing" 23 24 "github.com/cockroachdb/cockroach/pkg/base" 25 "github.com/cockroachdb/cockroach/pkg/keys" 26 "github.com/cockroachdb/cockroach/pkg/kv" 27 "github.com/cockroachdb/cockroach/pkg/settings/cluster" 28 "github.com/cockroachdb/cockroach/pkg/sql/execinfra" 29 "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" 30 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 31 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 32 "github.com/cockroachdb/cockroach/pkg/sql/types" 33 "github.com/cockroachdb/cockroach/pkg/storage" 34 "github.com/cockroachdb/cockroach/pkg/testutils" 35 "github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils" 36 "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" 37 "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" 38 "github.com/cockroachdb/cockroach/pkg/util/leaktest" 39 "github.com/cockroachdb/cockroach/pkg/util/log" 40 "github.com/cockroachdb/cockroach/pkg/util/mon" 41 "github.com/cockroachdb/cockroach/pkg/util/tracing" 42 "github.com/cockroachdb/errors" 43 "github.com/stretchr/testify/require" 44 ) 45 46 func TestJoinReader(t *testing.T) { 47 defer leaktest.AfterTest(t)() 48 ctx := context.Background() 49 50 s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) 51 defer s.Stopper().Stop(ctx) 52 53 // Create a table where each row is: 54 // 55 // | a | b | sum | s | 56 // |-----------------------------------------------------------------| 57 // | rowId/10 | rowId%10 | rowId/10 + rowId%10 | IntToEnglish(rowId) | 58 59 aFn := func(row int) tree.Datum { 60 return tree.NewDInt(tree.DInt(row / 10)) 61 } 62 bFn := func(row int) tree.Datum { 63 return tree.NewDInt(tree.DInt(row % 10)) 64 } 65 sumFn := func(row int) tree.Datum { 66 return tree.NewDInt(tree.DInt(row/10 + row%10)) 67 } 68 69 sqlutils.CreateTable(t, sqlDB, "t", 70 "a INT, b INT, sum INT, s STRING, PRIMARY KEY (a,b), INDEX bs (b,s)", 71 99, 72 sqlutils.ToRowFn(aFn, bFn, sumFn, sqlutils.RowEnglishFn)) 73 74 // Insert a row for NULL testing. 75 if _, err := sqlDB.Exec("INSERT INTO test.t VALUES (10, 0, NULL, NULL)"); err != nil { 76 t.Fatal(err) 77 } 78 79 tdSecondary := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") 80 81 sqlutils.CreateTable(t, sqlDB, "t2", 82 "a INT, b INT, sum INT, s STRING, PRIMARY KEY (a,b), FAMILY f1 (a, b), FAMILY f2 (s), FAMILY f3 (sum), INDEX bs (b,s)", 83 99, 84 sqlutils.ToRowFn(aFn, bFn, sumFn, sqlutils.RowEnglishFn)) 85 86 tdFamily := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t2") 87 88 sqlutils.CreateTable(t, sqlDB, "t3parent", 89 "a INT PRIMARY KEY", 90 0, 91 sqlutils.ToRowFn(aFn)) 92 93 sqlutils.CreateTableInterleaved(t, sqlDB, "t3", 94 "a INT, b INT, sum INT, s STRING, PRIMARY KEY (a,b), INDEX bs (b,s)", 95 "t3parent(a)", 96 99, 97 sqlutils.ToRowFn(aFn, bFn, sumFn, sqlutils.RowEnglishFn)) 98 tdInterleaved := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t3") 99 100 testCases := []struct { 101 description string 102 indexIdx uint32 103 post execinfrapb.PostProcessSpec 104 onExpr string 105 input [][]tree.Datum 106 lookupCols []uint32 107 joinType sqlbase.JoinType 108 inputTypes []*types.T 109 outputTypes []*types.T 110 expected string 111 }{ 112 { 113 description: "Test selecting columns from second table", 114 post: execinfrapb.PostProcessSpec{ 115 Projection: true, 116 OutputColumns: []uint32{0, 1, 4}, 117 }, 118 input: [][]tree.Datum{ 119 {aFn(2), bFn(2)}, 120 {aFn(5), bFn(5)}, 121 {aFn(10), bFn(10)}, 122 {aFn(15), bFn(15)}, 123 }, 124 lookupCols: []uint32{0, 1}, 125 inputTypes: sqlbase.TwoIntCols, 126 outputTypes: sqlbase.ThreeIntCols, 127 expected: "[[0 2 2] [0 5 5] [1 0 1] [1 5 6]]", 128 }, 129 { 130 description: "Test duplicates in the input of lookup joins", 131 post: execinfrapb.PostProcessSpec{ 132 Projection: true, 133 OutputColumns: []uint32{0, 1, 3}, 134 }, 135 input: [][]tree.Datum{ 136 {aFn(2), bFn(2)}, 137 {aFn(2), bFn(2)}, 138 {aFn(5), bFn(5)}, 139 {aFn(10), bFn(10)}, 140 {aFn(15), bFn(15)}, 141 }, 142 lookupCols: []uint32{0, 1}, 143 inputTypes: sqlbase.TwoIntCols, 144 outputTypes: sqlbase.ThreeIntCols, 145 expected: "[[0 2 2] [0 2 2] [0 5 5] [1 0 0] [1 5 5]]", 146 }, 147 { 148 description: "Test lookup join queries with separate families", 149 post: execinfrapb.PostProcessSpec{ 150 Projection: true, 151 OutputColumns: []uint32{0, 1, 3, 4}, 152 }, 153 input: [][]tree.Datum{ 154 {aFn(2), bFn(2)}, 155 {aFn(5), bFn(5)}, 156 {aFn(10), bFn(10)}, 157 {aFn(15), bFn(15)}, 158 }, 159 lookupCols: []uint32{0, 1}, 160 inputTypes: sqlbase.TwoIntCols, 161 outputTypes: sqlbase.FourIntCols, 162 expected: "[[0 2 2 2] [0 5 5 5] [1 0 0 1] [1 5 5 6]]", 163 }, 164 { 165 description: "Test lookup joins preserve order of left input", 166 post: execinfrapb.PostProcessSpec{ 167 Projection: true, 168 OutputColumns: []uint32{0, 1, 3}, 169 }, 170 input: [][]tree.Datum{ 171 {aFn(2), bFn(2)}, 172 {aFn(5), bFn(5)}, 173 {aFn(2), bFn(2)}, 174 {aFn(10), bFn(10)}, 175 {aFn(15), bFn(15)}, 176 }, 177 lookupCols: []uint32{0, 1}, 178 inputTypes: sqlbase.TwoIntCols, 179 outputTypes: sqlbase.ThreeIntCols, 180 expected: "[[0 2 2] [0 5 5] [0 2 2] [1 0 0] [1 5 5]]", 181 }, 182 { 183 description: "Test lookup join with onExpr", 184 post: execinfrapb.PostProcessSpec{ 185 Projection: true, 186 OutputColumns: []uint32{0, 1, 4}, 187 }, 188 input: [][]tree.Datum{ 189 {aFn(2), bFn(2)}, 190 {aFn(5), bFn(5)}, 191 {aFn(10), bFn(10)}, 192 {aFn(15), bFn(15)}, 193 }, 194 lookupCols: []uint32{0, 1}, 195 inputTypes: sqlbase.TwoIntCols, 196 outputTypes: sqlbase.ThreeIntCols, 197 onExpr: "@2 < @5", 198 expected: "[[1 0 1] [1 5 6]]", 199 }, 200 { 201 description: "Test left outer lookup join on primary index", 202 post: execinfrapb.PostProcessSpec{ 203 Projection: true, 204 OutputColumns: []uint32{0, 1, 4}, 205 }, 206 input: [][]tree.Datum{ 207 {aFn(100), bFn(100)}, 208 {aFn(2), bFn(2)}, 209 }, 210 lookupCols: []uint32{0, 1}, 211 joinType: sqlbase.LeftOuterJoin, 212 inputTypes: sqlbase.TwoIntCols, 213 outputTypes: sqlbase.ThreeIntCols, 214 expected: "[[10 0 NULL] [0 2 2]]", 215 }, 216 { 217 description: "Test lookup join on secondary index with NULL lookup value", 218 indexIdx: 1, 219 post: execinfrapb.PostProcessSpec{ 220 Projection: true, 221 OutputColumns: []uint32{0}, 222 }, 223 input: [][]tree.Datum{ 224 {tree.NewDInt(0), tree.DNull}, 225 }, 226 lookupCols: []uint32{0, 1}, 227 inputTypes: sqlbase.TwoIntCols, 228 outputTypes: sqlbase.OneIntCol, 229 expected: "[]", 230 }, 231 { 232 description: "Test left outer lookup join on secondary index with NULL lookup value", 233 indexIdx: 1, 234 post: execinfrapb.PostProcessSpec{ 235 Projection: true, 236 OutputColumns: []uint32{0, 2}, 237 }, 238 input: [][]tree.Datum{ 239 {tree.NewDInt(0), tree.DNull}, 240 }, 241 lookupCols: []uint32{0, 1}, 242 joinType: sqlbase.LeftOuterJoin, 243 inputTypes: sqlbase.TwoIntCols, 244 outputTypes: sqlbase.TwoIntCols, 245 expected: "[[0 NULL]]", 246 }, 247 { 248 description: "Test lookup join on secondary index with an implicit key column", 249 indexIdx: 1, 250 post: execinfrapb.PostProcessSpec{ 251 Projection: true, 252 OutputColumns: []uint32{2}, 253 }, 254 input: [][]tree.Datum{ 255 {aFn(2), bFn(2), sqlutils.RowEnglishFn(2)}, 256 }, 257 lookupCols: []uint32{1, 2, 0}, 258 inputTypes: []*types.T{types.Int, types.Int, types.String}, 259 outputTypes: sqlbase.OneIntCol, 260 expected: "[['two']]", 261 }, 262 { 263 description: "Test left semi lookup join", 264 indexIdx: 1, 265 post: execinfrapb.PostProcessSpec{ 266 Projection: true, 267 OutputColumns: []uint32{0, 1}, 268 }, 269 input: [][]tree.Datum{ 270 {tree.NewDInt(tree.DInt(1)), sqlutils.RowEnglishFn(2)}, 271 {tree.NewDInt(tree.DInt(1)), sqlutils.RowEnglishFn(2)}, 272 {tree.NewDInt(tree.DInt(1234)), sqlutils.RowEnglishFn(2)}, 273 {tree.NewDInt(tree.DInt(6)), sqlutils.RowEnglishFn(2)}, 274 {tree.NewDInt(tree.DInt(7)), sqlutils.RowEnglishFn(2)}, 275 {tree.NewDInt(tree.DInt(1)), sqlutils.RowEnglishFn(2)}, 276 }, 277 lookupCols: []uint32{0}, 278 joinType: sqlbase.LeftSemiJoin, 279 inputTypes: []*types.T{types.Int, types.String}, 280 outputTypes: sqlbase.TwoIntCols, 281 expected: "[[1 'two'] [1 'two'] [6 'two'] [7 'two'] [1 'two']]", 282 }, 283 { 284 description: "Test left semi lookup join on secondary index with NULL lookup value", 285 indexIdx: 1, 286 post: execinfrapb.PostProcessSpec{ 287 Projection: true, 288 OutputColumns: []uint32{0}, 289 }, 290 input: [][]tree.Datum{ 291 {tree.NewDInt(0), tree.DNull}, 292 }, 293 lookupCols: []uint32{0, 1}, 294 joinType: sqlbase.LeftSemiJoin, 295 inputTypes: sqlbase.TwoIntCols, 296 outputTypes: sqlbase.OneIntCol, 297 expected: "[]", 298 }, 299 { 300 description: "Test left semi lookup join with onExpr", 301 indexIdx: 1, 302 post: execinfrapb.PostProcessSpec{ 303 Projection: true, 304 OutputColumns: []uint32{0, 1}, 305 }, 306 input: [][]tree.Datum{ 307 {tree.NewDInt(tree.DInt(1)), bFn(3)}, 308 {tree.NewDInt(tree.DInt(1)), bFn(2)}, 309 {tree.NewDInt(tree.DInt(1234)), bFn(2)}, 310 {tree.NewDInt(tree.DInt(6)), bFn(2)}, 311 {tree.NewDInt(tree.DInt(7)), bFn(3)}, 312 {tree.NewDInt(tree.DInt(1)), bFn(2)}, 313 }, 314 lookupCols: []uint32{0}, 315 joinType: sqlbase.LeftSemiJoin, 316 onExpr: "@2 > 2", 317 inputTypes: sqlbase.TwoIntCols, 318 outputTypes: sqlbase.TwoIntCols, 319 expected: "[[1 3] [7 3]]", 320 }, 321 { 322 description: "Test left anti lookup join", 323 indexIdx: 1, 324 post: execinfrapb.PostProcessSpec{ 325 Projection: true, 326 OutputColumns: []uint32{0, 1}, 327 }, 328 input: [][]tree.Datum{ 329 {tree.NewDInt(tree.DInt(1234)), tree.NewDInt(tree.DInt(1234))}, 330 }, 331 lookupCols: []uint32{0}, 332 joinType: sqlbase.LeftAntiJoin, 333 inputTypes: sqlbase.TwoIntCols, 334 outputTypes: sqlbase.TwoIntCols, 335 expected: "[[1234 1234]]", 336 }, 337 { 338 description: "Test left anti lookup join with onExpr", 339 indexIdx: 1, 340 post: execinfrapb.PostProcessSpec{ 341 Projection: true, 342 OutputColumns: []uint32{0, 1}, 343 }, 344 input: [][]tree.Datum{ 345 {tree.NewDInt(tree.DInt(1)), bFn(3)}, 346 {tree.NewDInt(tree.DInt(1)), bFn(2)}, 347 {tree.NewDInt(tree.DInt(6)), bFn(2)}, 348 {tree.NewDInt(tree.DInt(7)), bFn(3)}, 349 {tree.NewDInt(tree.DInt(1)), bFn(2)}, 350 }, 351 lookupCols: []uint32{0}, 352 joinType: sqlbase.LeftAntiJoin, 353 onExpr: "@2 > 2", 354 inputTypes: sqlbase.TwoIntCols, 355 outputTypes: sqlbase.TwoIntCols, 356 expected: "[[1 2] [6 2] [1 2]]", 357 }, 358 { 359 description: "Test left anti lookup join with match", 360 indexIdx: 1, 361 post: execinfrapb.PostProcessSpec{ 362 Projection: true, 363 OutputColumns: []uint32{0, 1}, 364 }, 365 input: [][]tree.Datum{ 366 {aFn(10), tree.NewDInt(tree.DInt(1234))}, 367 }, 368 lookupCols: []uint32{0}, 369 joinType: sqlbase.LeftAntiJoin, 370 inputTypes: sqlbase.TwoIntCols, 371 outputTypes: sqlbase.OneIntCol, 372 expected: "[]", 373 }, 374 { 375 description: "Test left anti lookup join on secondary index with NULL lookup value", 376 indexIdx: 1, 377 post: execinfrapb.PostProcessSpec{ 378 Projection: true, 379 OutputColumns: []uint32{0, 1}, 380 }, 381 input: [][]tree.Datum{ 382 {tree.NewDInt(0), tree.DNull}, 383 }, 384 lookupCols: []uint32{0, 1}, 385 joinType: sqlbase.LeftAntiJoin, 386 inputTypes: sqlbase.TwoIntCols, 387 outputTypes: sqlbase.TwoIntCols, 388 expected: "[[0 NULL]]", 389 }, 390 } 391 st := cluster.MakeTestingClusterSettings() 392 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec) 393 if err != nil { 394 t.Fatal(err) 395 } 396 defer tempEngine.Close() 397 diskMonitor := mon.MakeMonitor( 398 "test-disk", 399 mon.DiskResource, 400 nil, /* curCount */ 401 nil, /* maxHist */ 402 -1, /* increment: use default block size */ 403 math.MaxInt64, 404 st, 405 ) 406 diskMonitor.Start(ctx, nil /* pool */, mon.MakeStandaloneBudget(math.MaxInt64)) 407 defer diskMonitor.Stop(ctx) 408 for i, td := range []*sqlbase.TableDescriptor{tdSecondary, tdFamily, tdInterleaved} { 409 for _, c := range testCases { 410 for _, reqOrdering := range []bool{true, false} { 411 t.Run(fmt.Sprintf("%d/reqOrdering=%t/%s", i, reqOrdering, c.description), func(t *testing.T) { 412 evalCtx := tree.MakeTestingEvalContext(st) 413 defer evalCtx.Stop(ctx) 414 flowCtx := execinfra.FlowCtx{ 415 EvalCtx: &evalCtx, 416 Cfg: &execinfra.ServerConfig{ 417 Settings: st, 418 TempStorage: tempEngine, 419 DiskMonitor: &diskMonitor, 420 }, 421 Txn: kv.NewTxn(ctx, s.DB(), s.NodeID()), 422 } 423 encRows := make(sqlbase.EncDatumRows, len(c.input)) 424 for rowIdx, row := range c.input { 425 encRow := make(sqlbase.EncDatumRow, len(row)) 426 for i, d := range row { 427 encRow[i] = sqlbase.DatumToEncDatum(c.inputTypes[i], d) 428 } 429 encRows[rowIdx] = encRow 430 } 431 in := distsqlutils.NewRowBuffer(c.inputTypes, encRows, distsqlutils.RowBufferArgs{}) 432 433 out := &distsqlutils.RowBuffer{} 434 jr, err := newJoinReader( 435 &flowCtx, 436 0, /* processorID */ 437 &execinfrapb.JoinReaderSpec{ 438 Table: *td, 439 IndexIdx: c.indexIdx, 440 LookupColumns: c.lookupCols, 441 OnExpr: execinfrapb.Expression{Expr: c.onExpr}, 442 Type: c.joinType, 443 MaintainOrdering: reqOrdering, 444 }, 445 in, 446 &c.post, 447 out, 448 ) 449 if err != nil { 450 t.Fatal(err) 451 } 452 453 // Set a lower batch size to force multiple batches. 454 jr.(*joinReader).SetBatchSizeBytes(int64(encRows[0].Size() * 3)) 455 456 jr.Run(ctx) 457 458 if !in.Done { 459 t.Fatal("joinReader didn't consume all the rows") 460 } 461 if !out.ProducerClosed() { 462 t.Fatalf("output RowReceiver not closed") 463 } 464 465 var res sqlbase.EncDatumRows 466 for { 467 row := out.NextNoMeta(t) 468 if row == nil { 469 break 470 } 471 res = append(res, row) 472 } 473 474 // processOutputRows is a helper function that takes a stringified 475 // EncDatumRows output (e.g. [[1 2] [3 1]]) and returns a slice of 476 // stringified rows without brackets (e.g. []string{"1 2", "3 1"}). 477 processOutputRows := func(output string) []string { 478 // Comma-separate the rows. 479 output = strings.ReplaceAll(output, "] [", ",") 480 // Remove leading and trailing bracket. 481 output = strings.Trim(output, "[]") 482 // Split on the commas that were introduced and return that. 483 return strings.Split(output, ",") 484 } 485 486 result := processOutputRows(res.String(c.outputTypes)) 487 expected := processOutputRows(c.expected) 488 489 if !reqOrdering { 490 // An ordering was not required, so sort both the result and 491 // expected slice to reuse equality comparison. 492 sort.Strings(result) 493 sort.Strings(expected) 494 } 495 496 require.Equal(t, expected, result) 497 }) 498 } 499 } 500 } 501 } 502 503 func TestJoinReaderDiskSpill(t *testing.T) { 504 defer leaktest.AfterTest(t)() 505 ctx := context.Background() 506 507 s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) 508 defer s.Stopper().Stop(ctx) 509 510 // Create our lookup table, with a single key which has enough rows to trigger 511 // a disk spill. 512 key := 0 513 stringColVal := "0123456789" 514 numRows := 100 515 if _, err := sqlDB.Exec(` 516 CREATE DATABASE test; 517 CREATE TABLE test.t (a INT, s STRING, INDEX (a, s))`); err != nil { 518 t.Fatal(err) 519 } 520 if _, err := sqlDB.Exec( 521 `INSERT INTO test.t SELECT $1, $2 FROM generate_series(1, $3)`, 522 key, stringColVal, numRows); err != nil { 523 t.Fatal(err) 524 } 525 td := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") 526 527 st := cluster.MakeTestingClusterSettings() 528 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec) 529 if err != nil { 530 t.Fatal(err) 531 } 532 defer tempEngine.Close() 533 534 evalCtx := tree.MakeTestingEvalContext(st) 535 defer evalCtx.Stop(ctx) 536 diskMonitor := mon.MakeMonitor( 537 "test-disk", 538 mon.DiskResource, 539 nil, /* curCount */ 540 nil, /* maxHist */ 541 -1, /* increment: use default block size */ 542 math.MaxInt64, 543 st, 544 ) 545 diskMonitor.Start(ctx, nil /* pool */, mon.MakeStandaloneBudget(math.MaxInt64)) 546 defer diskMonitor.Stop(ctx) 547 flowCtx := execinfra.FlowCtx{ 548 EvalCtx: &evalCtx, 549 Cfg: &execinfra.ServerConfig{ 550 Settings: st, 551 TempStorage: tempEngine, 552 DiskMonitor: &diskMonitor, 553 }, 554 Txn: kv.NewTxn(ctx, s.DB(), s.NodeID()), 555 } 556 // Set the memory limit to the minimum allocation size so that the row 557 // container can buffer some rows in memory before spilling to disk. This 558 // also means we don't need to disable caching in the 559 // DiskBackedIndexedRowContainer. 560 flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = mon.DefaultPoolAllocationSize 561 562 // Input row is just a single 0. 563 inputRows := sqlbase.EncDatumRows{ 564 sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDInt(tree.DInt(key))}}, 565 } 566 567 out := &distsqlutils.RowBuffer{} 568 jr, err := newJoinReader( 569 &flowCtx, 570 0, /* processorID */ 571 &execinfrapb.JoinReaderSpec{ 572 Table: *td, 573 IndexIdx: 1, 574 LookupColumns: []uint32{0}, 575 Type: sqlbase.InnerJoin, 576 // Disk storage is only used when the input ordering must be maintained. 577 MaintainOrdering: true, 578 }, 579 distsqlutils.NewRowBuffer(sqlbase.OneIntCol, inputRows, distsqlutils.RowBufferArgs{}), 580 &execinfrapb.PostProcessSpec{ 581 Projection: true, 582 OutputColumns: []uint32{2}, 583 }, 584 out, 585 ) 586 if err != nil { 587 t.Fatal(err) 588 } 589 jr.Run(ctx) 590 591 count := 0 592 for { 593 row := out.NextNoMeta(t) 594 if row == nil { 595 break 596 } 597 expected := fmt.Sprintf("['%s']", stringColVal) 598 actual := row.String([]*types.T{types.String}) 599 require.Equal(t, expected, actual) 600 count++ 601 } 602 require.Equal(t, numRows, count) 603 require.True(t, jr.(*joinReader).Spilled()) 604 } 605 606 // TestJoinReaderDrain tests various scenarios in which a joinReader's consumer 607 // is closed. 608 func TestJoinReaderDrain(t *testing.T) { 609 defer leaktest.AfterTest(t)() 610 611 s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) 612 defer s.Stopper().Stop(context.Background()) 613 614 sqlutils.CreateTable( 615 t, 616 sqlDB, 617 "t", 618 "a INT, PRIMARY KEY (a)", 619 1, /* numRows */ 620 sqlutils.ToRowFn(sqlutils.RowIdxFn), 621 ) 622 td := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") 623 624 st := s.ClusterSettings() 625 tempEngine, _, err := storage.NewTempEngine(context.Background(), storage.DefaultStorageEngine, base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec) 626 if err != nil { 627 t.Fatal(err) 628 } 629 defer tempEngine.Close() 630 631 // Run the flow in a snowball trace so that we can test for tracing info. 632 tracer := tracing.NewTracer() 633 ctx, sp := tracing.StartSnowballTrace(context.Background(), tracer, "test flow ctx") 634 defer sp.Finish() 635 636 evalCtx := tree.MakeTestingEvalContext(st) 637 defer evalCtx.Stop(context.Background()) 638 diskMonitor := mon.MakeMonitor( 639 "test-disk", 640 mon.DiskResource, 641 nil, /* curCount */ 642 nil, /* maxHist */ 643 -1, /* increment: use default block size */ 644 math.MaxInt64, 645 st, 646 ) 647 diskMonitor.Start(ctx, nil /* pool */, mon.MakeStandaloneBudget(math.MaxInt64)) 648 defer diskMonitor.Stop(ctx) 649 650 rootTxn := kv.NewTxn(ctx, s.DB(), s.NodeID()) 651 leafInputState := rootTxn.GetLeafTxnInputState(ctx) 652 leafTxn := kv.NewLeafTxn(ctx, s.DB(), s.NodeID(), &leafInputState) 653 654 flowCtx := execinfra.FlowCtx{ 655 EvalCtx: &evalCtx, 656 Cfg: &execinfra.ServerConfig{ 657 Settings: st, 658 TempStorage: tempEngine, 659 DiskMonitor: &diskMonitor, 660 }, 661 Txn: leafTxn, 662 } 663 664 encRow := make(sqlbase.EncDatumRow, 1) 665 encRow[0] = sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(1)) 666 667 // ConsumerClosed verifies that when a joinReader's consumer is closed, the 668 // joinReader finishes gracefully. 669 t.Run("ConsumerClosed", func(t *testing.T) { 670 in := distsqlutils.NewRowBuffer(sqlbase.OneIntCol, sqlbase.EncDatumRows{encRow}, distsqlutils.RowBufferArgs{}) 671 672 out := &distsqlutils.RowBuffer{} 673 out.ConsumerClosed() 674 jr, err := newJoinReader( 675 &flowCtx, 0 /* processorID */, &execinfrapb.JoinReaderSpec{Table: *td}, in, &execinfrapb.PostProcessSpec{}, out, 676 ) 677 if err != nil { 678 t.Fatal(err) 679 } 680 jr.Run(ctx) 681 }) 682 683 // ConsumerDone verifies that the producer drains properly by checking that 684 // metadata coming from the producer is still read when ConsumerDone is 685 // called on the consumer. 686 t.Run("ConsumerDone", func(t *testing.T) { 687 expectedMetaErr := errors.New("dummy") 688 in := distsqlutils.NewRowBuffer(sqlbase.OneIntCol, nil /* rows */, distsqlutils.RowBufferArgs{}) 689 if status := in.Push(encRow, &execinfrapb.ProducerMetadata{Err: expectedMetaErr}); status != execinfra.NeedMoreRows { 690 t.Fatalf("unexpected response: %d", status) 691 } 692 693 out := &distsqlutils.RowBuffer{} 694 out.ConsumerDone() 695 jr, err := newJoinReader( 696 &flowCtx, 0 /* processorID */, &execinfrapb.JoinReaderSpec{Table: *td}, in, &execinfrapb.PostProcessSpec{}, out, 697 ) 698 if err != nil { 699 t.Fatal(err) 700 } 701 jr.Run(ctx) 702 row, meta := out.Next() 703 if row != nil { 704 t.Fatalf("row was pushed unexpectedly: %s", row.String(sqlbase.OneIntCol)) 705 } 706 if !errors.Is(meta.Err, expectedMetaErr) { 707 t.Fatalf("unexpected error in metadata: %v", meta.Err) 708 } 709 710 // Check for trailing metadata. 711 var traceSeen, txnFinalStateSeen bool 712 for { 713 row, meta = out.Next() 714 if row != nil { 715 t.Fatalf("row was pushed unexpectedly: %s", row.String(sqlbase.OneIntCol)) 716 } 717 if meta == nil { 718 break 719 } 720 if meta.TraceData != nil { 721 traceSeen = true 722 } 723 if meta.LeafTxnFinalState != nil { 724 txnFinalStateSeen = true 725 } 726 } 727 if !traceSeen { 728 t.Fatal("missing tracing trailing metadata") 729 } 730 if !txnFinalStateSeen { 731 t.Fatal("missing txn final state") 732 } 733 }) 734 } 735 736 // BenchmarkJoinReader benchmarks different lookup join match ratios against a 737 // table with half a million rows. A match ratio specifies how many rows are 738 // returned for a single lookup row. Some cases will cause the join reader to 739 // spill to disk, in which case the benchmark logs that the join spilled. 740 func BenchmarkJoinReader(b *testing.B) { 741 if testing.Short() { 742 b.Skip() 743 } 744 745 // Create an *on-disk* store spec for the primary store and temp engine to 746 // reflect the real costs of lookups and spilling. 747 primaryStoragePath, cleanupPrimaryDir := testutils.TempDir(b) 748 defer cleanupPrimaryDir() 749 storeSpec, err := base.NewStoreSpec(fmt.Sprintf("path=%s", primaryStoragePath)) 750 require.NoError(b, err) 751 752 var ( 753 logScope = log.Scope(b) 754 ctx = context.Background() 755 s, sqlDB, kvDB = serverutils.StartServer(b, base.TestServerArgs{ 756 StoreSpecs: []base.StoreSpec{storeSpec}, 757 }) 758 st = s.ClusterSettings() 759 evalCtx = tree.MakeTestingEvalContext(st) 760 diskMonitor = execinfra.NewTestDiskMonitor(ctx, st) 761 flowCtx = execinfra.FlowCtx{ 762 EvalCtx: &evalCtx, 763 Cfg: &execinfra.ServerConfig{ 764 DiskMonitor: diskMonitor, 765 Settings: st, 766 }, 767 } 768 ) 769 defer logScope.Close(b) 770 defer s.Stopper().Stop(ctx) 771 defer evalCtx.Stop(ctx) 772 defer diskMonitor.Stop(ctx) 773 774 tempStoragePath, cleanupTempDir := testutils.TempDir(b) 775 defer cleanupTempDir() 776 tempStoreSpec, err := base.NewStoreSpec(fmt.Sprintf("path=%s", tempStoragePath)) 777 require.NoError(b, err) 778 tempEngine, _, err := storage.NewTempEngine(ctx, storage.DefaultStorageEngine, base.TempStorageConfig{Path: tempStoragePath, Mon: diskMonitor}, tempStoreSpec) 779 require.NoError(b, err) 780 defer tempEngine.Close() 781 flowCtx.Cfg.TempStorage = tempEngine 782 783 // rightSideColumnDef is the definition of a column in the table that is being 784 // looked up. 785 type rightSideColumnDef struct { 786 // name is the name of the column. 787 name string 788 // matchesPerLookupRow is the number of rows with the same column value. 789 matchesPerLookupRow int 790 } 791 rightSideColumnDefs := []rightSideColumnDef{ 792 {name: "one", matchesPerLookupRow: 1}, 793 {name: "four", matchesPerLookupRow: 4}, 794 {name: "sixteen", matchesPerLookupRow: 16}, 795 {name: "thirtytwo", matchesPerLookupRow: 32}, 796 {name: "sixtyfour", matchesPerLookupRow: 64}, 797 } 798 tableSizeToName := func(sz int) string { 799 return fmt.Sprintf("t%d", sz) 800 } 801 802 createRightSideTable := func(sz int) { 803 colDefs := make([]string, 0, len(rightSideColumnDefs)) 804 indexDefs := make([]string, 0, len(rightSideColumnDefs)) 805 genValueFns := make([]sqlutils.GenValueFn, 0, len(rightSideColumnDefs)) 806 for _, columnDef := range rightSideColumnDefs { 807 if columnDef.matchesPerLookupRow > sz { 808 continue 809 } 810 colDefs = append(colDefs, fmt.Sprintf("%s INT", columnDef.name)) 811 indexDefs = append(indexDefs, fmt.Sprintf("INDEX (%s)", columnDef.name)) 812 813 curValue := -1 814 // Capture matchesPerLookupRow for use in the generating function later 815 // on. 816 matchesPerLookupRow := columnDef.matchesPerLookupRow 817 genValueFns = append(genValueFns, func(row int) tree.Datum { 818 idx := row - 1 819 if idx%matchesPerLookupRow == 0 { 820 // Increment curValue every columnDef.matchesPerLookupRow values. The 821 // first value will be 0. 822 curValue++ 823 } 824 return tree.NewDInt(tree.DInt(curValue)) 825 }) 826 } 827 tableName := tableSizeToName(sz) 828 829 sqlutils.CreateTable( 830 b, sqlDB, tableName, strings.Join(append(colDefs, indexDefs...), ", "), sz, 831 sqlutils.ToRowFn(genValueFns...), 832 ) 833 } 834 835 rightSz := 1 << 19 /* 524,288 rows */ 836 createRightSideTable(rightSz) 837 // Create a new txn after the table has been created. 838 flowCtx.Txn = kv.NewTxn(ctx, s.DB(), s.NodeID()) 839 for _, reqOrdering := range []bool{true, false} { 840 for columnIdx, columnDef := range rightSideColumnDefs { 841 for _, numLookupRows := range []int{1, 1 << 4 /* 16 */, 1 << 8 /* 256 */, 1 << 10 /* 1024 */, 1 << 12 /* 4096 */, 1 << 13 /* 8192 */, 1 << 14 /* 16384 */, 1 << 15 /* 32768 */, 1 << 16 /* 65,536 */, 1 << 19 /* 524,288 */} { 842 for _, memoryLimit := range []int64{100 << 10, math.MaxInt64} { 843 memoryLimitStr := "mem=unlimited" 844 if memoryLimit != math.MaxInt64 { 845 if !reqOrdering { 846 // Smaller memory limit is not relevant when there is no ordering. 847 continue 848 } 849 memoryLimitStr = fmt.Sprintf("mem=%dKB", memoryLimit/(1<<10)) 850 // The benchmark workloads are such that each right row never joins 851 // with more than one left row. And the access pattern of right rows 852 // accessed across all the left rows is monotonically increasing. So 853 // once spilled to disk, the reads will always need to get from disk 854 // (caching cannot improve performance). 855 // 856 // TODO(sumeer): add workload that can benefit from caching. 857 } 858 if rightSz/columnDef.matchesPerLookupRow < numLookupRows { 859 // This case does not make sense since we won't have distinct lookup 860 // rows. We don't currently merge spans which could make this an 861 // interesting case to benchmark, but we probably should. 862 continue 863 } 864 865 eqColsAreKey := []bool{false} 866 if numLookupRows == 1 { 867 // For this case, execute the parallel lookup case as well. 868 eqColsAreKey = []bool{true, false} 869 } 870 for _, parallel := range eqColsAreKey { 871 benchmarkName := fmt.Sprintf("reqOrdering=%t/matchratio=oneto%s/lookuprows=%d/%s", 872 reqOrdering, columnDef.name, numLookupRows, memoryLimitStr) 873 if parallel { 874 benchmarkName += "/parallel=true" 875 } 876 b.Run(benchmarkName, func(b *testing.B) { 877 tableName := tableSizeToName(rightSz) 878 879 // Get the table descriptor and find the index that will provide us with 880 // the expected match ratio. 881 tableDesc := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", tableName) 882 indexIdx := uint32(0) 883 for i := range tableDesc.Indexes { 884 require.Equal(b, 1, len(tableDesc.Indexes[i].ColumnNames), "all indexes created in this benchmark should only contain one column") 885 if tableDesc.Indexes[i].ColumnNames[0] == columnDef.name { 886 // Found indexIdx. 887 indexIdx = uint32(i + 1) 888 break 889 } 890 } 891 if indexIdx == 0 { 892 b.Fatalf("failed to find secondary index for column %s", columnDef.name) 893 } 894 input := newRowGeneratingSource(sqlbase.OneIntCol, sqlutils.ToRowFn(func(rowIdx int) tree.Datum { 895 // Convert to 0-based. 896 return tree.NewDInt(tree.DInt(rowIdx - 1)) 897 }), numLookupRows) 898 output := rowDisposer{} 899 900 spec := execinfrapb.JoinReaderSpec{ 901 Table: *tableDesc, 902 LookupColumns: []uint32{0}, 903 LookupColumnsAreKey: parallel, 904 IndexIdx: indexIdx, 905 MaintainOrdering: reqOrdering, 906 } 907 // Post specifies that only the columns contained in the secondary index 908 // need to be output. 909 post := execinfrapb.PostProcessSpec{ 910 Projection: true, 911 OutputColumns: []uint32{uint32(columnIdx + 1)}, 912 } 913 914 expectedNumOutputRows := numLookupRows * columnDef.matchesPerLookupRow 915 b.ResetTimer() 916 // The number of bytes processed in this benchmark is the number of 917 // lookup bytes processed + the number of result bytes. We only look 918 // up using a single int column and the request only a single int column 919 // contained in the index. 920 b.SetBytes(int64((numLookupRows * 8) + (expectedNumOutputRows * 8))) 921 922 spilled := false 923 for i := 0; i < b.N; i++ { 924 flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = memoryLimit 925 jr, err := newJoinReader(&flowCtx, 0 /* processorID */, &spec, input, &post, &output) 926 if err != nil { 927 b.Fatal(err) 928 } 929 jr.Run(ctx) 930 if !spilled && jr.(*joinReader).Spilled() { 931 spilled = true 932 } 933 meta := output.DrainMeta(ctx) 934 if meta != nil { 935 b.Fatalf("unexpected metadata: %v", meta) 936 } 937 if output.NumRowsDisposed() != expectedNumOutputRows { 938 b.Fatalf("got %d output rows, expected %d", output.NumRowsDisposed(), expectedNumOutputRows) 939 } 940 output.ResetNumRowsDisposed() 941 input.Reset() 942 } 943 if spilled { 944 b.Log("joinReader spilled to disk in at least one of the benchmark iterations") 945 } 946 }) 947 } 948 } 949 } 950 } 951 } 952 }