github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/distsql_physical_planner_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 package sql 12 13 import ( 14 "context" 15 gosql "database/sql" 16 "fmt" 17 "net/url" 18 "reflect" 19 "strconv" 20 "strings" 21 "sync" 22 "testing" 23 24 "github.com/cockroachdb/cockroach/pkg/base" 25 "github.com/cockroachdb/cockroach/pkg/config/zonepb" 26 "github.com/cockroachdb/cockroach/pkg/gossip" 27 "github.com/cockroachdb/cockroach/pkg/keys" 28 "github.com/cockroachdb/cockroach/pkg/kv" 29 "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" 30 "github.com/cockroachdb/cockroach/pkg/roachpb" 31 "github.com/cockroachdb/cockroach/pkg/rpc" 32 "github.com/cockroachdb/cockroach/pkg/security" 33 "github.com/cockroachdb/cockroach/pkg/settings/cluster" 34 "github.com/cockroachdb/cockroach/pkg/sql/distsql" 35 "github.com/cockroachdb/cockroach/pkg/sql/execinfra" 36 "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" 37 "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" 38 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 39 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 40 "github.com/cockroachdb/cockroach/pkg/testutils" 41 "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" 42 "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" 43 "github.com/cockroachdb/cockroach/pkg/util" 44 "github.com/cockroachdb/cockroach/pkg/util/leaktest" 45 "github.com/cockroachdb/cockroach/pkg/util/log" 46 "github.com/cockroachdb/cockroach/pkg/util/metric" 47 "github.com/cockroachdb/cockroach/pkg/util/randutil" 48 "github.com/cockroachdb/cockroach/pkg/util/stop" 49 "github.com/cockroachdb/errors" 50 ) 51 52 // SplitTable splits a range in the table, creates a replica for the right 53 // side of the split on TargetNodeIdx, and moves the lease for the right 54 // side of the split to TargetNodeIdx for each SplitPoint. This forces the 55 // querying against the table to be distributed. 56 // 57 // TODO(radu): SplitTable or its equivalent should be added to TestCluster. 58 // 59 // TODO(radu): we should verify that the queries in tests using SplitTable 60 // are indeed distributed as intended. 61 func SplitTable( 62 t *testing.T, 63 tc serverutils.TestClusterInterface, 64 desc *sqlbase.TableDescriptor, 65 sps []SplitPoint, 66 ) { 67 if tc.ReplicationMode() != base.ReplicationManual { 68 t.Fatal("SplitTable called on a test cluster that was not in manual replication mode") 69 } 70 71 rkts := make(map[roachpb.RangeID]rangeAndKT) 72 for _, sp := range sps { 73 pik, err := sqlbase.TestingMakePrimaryIndexKey(desc, sp.Vals...) 74 if err != nil { 75 t.Fatal(err) 76 } 77 78 _, rightRange, err := tc.Server(0).SplitRange(pik) 79 if err != nil { 80 t.Fatal(err) 81 } 82 83 rightRangeStartKey := rightRange.StartKey.AsRawKey() 84 target := tc.Target(sp.TargetNodeIdx) 85 86 rkts[rightRange.RangeID] = rangeAndKT{ 87 rightRange, 88 serverutils.KeyAndTargets{StartKey: rightRangeStartKey, Targets: []roachpb.ReplicationTarget{target}}} 89 } 90 91 var kts []serverutils.KeyAndTargets 92 for _, rkt := range rkts { 93 kts = append(kts, rkt.KT) 94 } 95 descs, errs := tc.AddReplicasMulti(kts...) 96 for _, err := range errs { 97 if err != nil && !testutils.IsError(err, "is already present") { 98 t.Fatal(err) 99 } 100 } 101 102 for _, desc := range descs { 103 rkt, ok := rkts[desc.RangeID] 104 if !ok { 105 continue 106 } 107 108 for _, target := range rkt.KT.Targets { 109 if err := tc.TransferRangeLease(desc, target); err != nil { 110 t.Fatal(err) 111 } 112 } 113 } 114 } 115 116 // SplitPoint describes a split point that is passed to SplitTable. 117 type SplitPoint struct { 118 // TargetNodeIdx is the node that will have the lease for the new range. 119 TargetNodeIdx int 120 // Vals is list of values forming a primary key for the table. 121 Vals []interface{} 122 } 123 124 type rangeAndKT struct { 125 Range roachpb.RangeDescriptor 126 KT serverutils.KeyAndTargets 127 } 128 129 // TestPlanningDuringSplits verifies that table reader planning (resolving 130 // spans) tolerates concurrent splits and merges. 131 func TestPlanningDuringSplitsAndMerges(t *testing.T) { 132 defer leaktest.AfterTest(t)() 133 134 const n = 100 135 const numNodes = 1 136 tc := serverutils.StartTestCluster(t, numNodes, base.TestClusterArgs{ 137 ServerArgs: base.TestServerArgs{UseDatabase: "test"}, 138 }) 139 140 defer tc.Stopper().Stop(context.Background()) 141 142 sqlutils.CreateTable( 143 t, tc.ServerConn(0), "t", "x INT PRIMARY KEY, xsquared INT", 144 n, 145 sqlutils.ToRowFn(sqlutils.RowIdxFn, func(row int) tree.Datum { 146 return tree.NewDInt(tree.DInt(row * row)) 147 }), 148 ) 149 150 // Start a worker that continuously performs splits in the background. 151 tc.Stopper().RunWorker(context.Background(), func(ctx context.Context) { 152 rng, _ := randutil.NewPseudoRand() 153 cdb := tc.Server(0).DB() 154 for { 155 select { 156 case <-tc.Stopper().ShouldStop(): 157 return 158 default: 159 // Split the table at a random row. 160 desc := sqlbase.GetTableDescriptor(cdb, keys.SystemSQLCodec, "test", "t") 161 162 val := rng.Intn(n) 163 t.Logf("splitting at %d", val) 164 pik, err := sqlbase.TestingMakePrimaryIndexKey(desc, val) 165 if err != nil { 166 panic(err) 167 } 168 169 if _, _, err := tc.Server(0).SplitRange(pik); err != nil { 170 panic(err) 171 } 172 } 173 } 174 }) 175 176 sumX, sumXSquared := 0, 0 177 for x := 1; x <= n; x++ { 178 sumX += x 179 sumXSquared += x * x 180 } 181 182 // Run queries continuously in parallel workers. We need more than one worker 183 // because some queries result in cache updates, and we want to verify 184 // race conditions when planning during cache updates (see #15249). 185 const numQueriers = 4 186 187 var wg sync.WaitGroup 188 wg.Add(numQueriers) 189 190 for i := 0; i < numQueriers; i++ { 191 go func(idx int) { 192 defer wg.Done() 193 194 // Create a gosql.DB for this worker. 195 pgURL, cleanupGoDB := sqlutils.PGUrl( 196 t, tc.Server(0).ServingSQLAddr(), fmt.Sprintf("%d", idx), url.User(security.RootUser), 197 ) 198 defer cleanupGoDB() 199 200 pgURL.Path = "test" 201 goDB, err := gosql.Open("postgres", pgURL.String()) 202 if err != nil { 203 t.Error(err) 204 return 205 } 206 207 defer func() { 208 if err := goDB.Close(); err != nil { 209 t.Error(err) 210 } 211 }() 212 213 // Limit to 1 connection because we set a session variable. 214 goDB.SetMaxOpenConns(1) 215 if _, err := goDB.Exec("SET DISTSQL = ALWAYS"); err != nil { 216 t.Error(err) 217 return 218 } 219 220 for run := 0; run < 20; run++ { 221 t.Logf("querier %d run %d", idx, run) 222 rows, err := goDB.Query("SELECT sum(x), sum(xsquared) FROM t") 223 if err != nil { 224 t.Error(err) 225 return 226 } 227 if !rows.Next() { 228 t.Errorf("no rows") 229 return 230 } 231 var sum, sumSq int 232 if err := rows.Scan(&sum, &sumSq); err != nil { 233 t.Error(err) 234 return 235 } 236 if sum != sumX || sumXSquared != sumSq { 237 t.Errorf("invalid results: expected %d, %d got %d, %d", sumX, sumXSquared, sum, sumSq) 238 return 239 } 240 if rows.Next() { 241 t.Errorf("more than one row") 242 return 243 } 244 } 245 }(i) 246 } 247 wg.Wait() 248 } 249 250 // Test that DistSQLReceiver uses inbound metadata to update the 251 // RangeDescriptorCache and the LeaseHolderCache. 252 func TestDistSQLReceiverUpdatesCaches(t *testing.T) { 253 defer leaktest.AfterTest(t)() 254 255 size := func() int64 { return 2 << 10 } 256 st := cluster.MakeTestingClusterSettings() 257 rangeCache := kvcoord.NewRangeDescriptorCache(st, nil /* db */, size, stop.NewStopper()) 258 leaseCache := kvcoord.NewLeaseHolderCache(size) 259 r := MakeDistSQLReceiver( 260 context.Background(), nil /* resultWriter */, tree.Rows, 261 rangeCache, leaseCache, nil /* txn */, nil /* updateClock */, &SessionTracing{}) 262 263 descs := []roachpb.RangeDescriptor{ 264 {RangeID: 1, StartKey: roachpb.RKey("a"), EndKey: roachpb.RKey("c")}, 265 {RangeID: 2, StartKey: roachpb.RKey("c"), EndKey: roachpb.RKey("e")}, 266 {RangeID: 3, StartKey: roachpb.RKey("g"), EndKey: roachpb.RKey("z")}, 267 } 268 269 // Push some metadata and check that the caches are updated with it. 270 status := r.Push(nil /* row */, &execinfrapb.ProducerMetadata{ 271 Ranges: []roachpb.RangeInfo{ 272 { 273 Desc: descs[0], 274 Lease: roachpb.Lease{Replica: roachpb.ReplicaDescriptor{ 275 NodeID: 1, StoreID: 1, ReplicaID: 1}}, 276 }, 277 { 278 Desc: descs[1], 279 Lease: roachpb.Lease{Replica: roachpb.ReplicaDescriptor{ 280 NodeID: 2, StoreID: 2, ReplicaID: 2}}, 281 }, 282 }}) 283 if status != execinfra.NeedMoreRows { 284 t.Fatalf("expected status NeedMoreRows, got: %d", status) 285 } 286 status = r.Push(nil /* row */, &execinfrapb.ProducerMetadata{ 287 Ranges: []roachpb.RangeInfo{ 288 { 289 Desc: descs[2], 290 Lease: roachpb.Lease{Replica: roachpb.ReplicaDescriptor{ 291 NodeID: 3, StoreID: 3, ReplicaID: 3}}, 292 }, 293 }}) 294 if status != execinfra.NeedMoreRows { 295 t.Fatalf("expected status NeedMoreRows, got: %d", status) 296 } 297 298 for i := range descs { 299 desc := rangeCache.GetCachedRangeDescriptor(descs[i].StartKey, false /* inclusive */) 300 if desc == nil { 301 t.Fatalf("failed to find range for key: %s", descs[i].StartKey) 302 } 303 if !desc.Equal(descs[i]) { 304 t.Fatalf("expected: %+v, got: %+v", descs[i], desc) 305 } 306 307 _, ok := leaseCache.Lookup(context.Background(), descs[i].RangeID) 308 if !ok { 309 t.Fatalf("didn't find lease for RangeID: %d", descs[i].RangeID) 310 } 311 } 312 } 313 314 // Test that a gateway improves the physical plans that it generates as a result 315 // of running a badly-planned query and receiving range information in response; 316 // this range information is used to update caches on the gateway. 317 func TestDistSQLRangeCachesIntegrationTest(t *testing.T) { 318 defer leaktest.AfterTest(t)() 319 320 // We're going to setup a cluster with 4 nodes. The last one will not be a 321 // target of any replication so that its caches stay virgin. 322 323 tc := serverutils.StartTestCluster(t, 4, /* numNodes */ 324 base.TestClusterArgs{ 325 ReplicationMode: base.ReplicationManual, 326 ServerArgs: base.TestServerArgs{ 327 UseDatabase: "test", 328 }, 329 }) 330 defer tc.Stopper().Stop(context.Background()) 331 332 db0 := tc.ServerConn(0) 333 sqlutils.CreateTable(t, db0, "left", 334 "num INT PRIMARY KEY", 335 3, /* numRows */ 336 sqlutils.ToRowFn(sqlutils.RowIdxFn)) 337 sqlutils.CreateTable(t, db0, "right", 338 "num INT PRIMARY KEY", 339 3, /* numRows */ 340 sqlutils.ToRowFn(sqlutils.RowIdxFn)) 341 342 // Disable eviction of the first range from the range cache on node 4 because 343 // the unpredictable nature of those updates interferes with the expectations 344 // of this test below. 345 // 346 // TODO(andrei): This is super hacky. What this test really wants to do is to 347 // precisely control the contents of the range cache on node 4. 348 tc.Server(3).DistSenderI().(*kvcoord.DistSender).DisableFirstRangeUpdates() 349 db3 := tc.ServerConn(3) 350 // Do a query on node 4 so that it populates the its cache with an initial 351 // descriptor containing all the SQL key space. If we don't do this, the state 352 // of the cache is left at the whim of gossiping the first descriptor done 353 // during cluster startup - it can happen that the cache remains empty, which 354 // is not what this test wants. 355 _, err := db3.Exec(`SELECT * FROM "left"`) 356 if err != nil { 357 t.Fatal(err) 358 } 359 360 // We're going to split one of the tables, but node 4 is unaware of this. 361 _, err = db0.Exec(fmt.Sprintf(` 362 ALTER TABLE "right" SPLIT AT VALUES (1), (2), (3); 363 ALTER TABLE "right" EXPERIMENTAL_RELOCATE VALUES (ARRAY[%d], 1), (ARRAY[%d], 2), (ARRAY[%d], 3); 364 `, 365 tc.Server(1).GetFirstStoreID(), 366 tc.Server(0).GetFirstStoreID(), 367 tc.Server(2).GetFirstStoreID())) 368 if err != nil { 369 t.Fatal(err) 370 } 371 372 // Ensure that the range cache is populated (see #31235). 373 _, err = db0.Exec(`SHOW RANGES FROM TABLE "right"`) 374 if err != nil { 375 t.Fatal(err) 376 } 377 378 // Run everything in a transaction, so we're bound on a connection on which we 379 // force DistSQL. 380 txn, err := db3.BeginTx(context.Background(), nil /* opts */) 381 if err != nil { 382 t.Fatal(err) 383 } 384 if _, err := txn.Exec("SET DISTSQL = ALWAYS"); err != nil { 385 t.Fatal(err) 386 } 387 388 // Check that the initial planning is suboptimal: the cache on db3 is unaware 389 // of the splits and still holds the state after the first dummy query at the 390 // beginning of the test, which had everything on the first node. 391 query := `SELECT count(1) FROM "left" INNER JOIN "right" USING (num)` 392 row := db3.QueryRow(fmt.Sprintf(`SELECT json FROM [EXPLAIN (DISTSQL) %v]`, query)) 393 var json string 394 if err := row.Scan(&json); err != nil { 395 t.Fatal(err) 396 } 397 exp := `"nodeNames":["1","4"]` 398 if !strings.Contains(json, exp) { 399 t.Fatalf("expected json to contain %s, but json is: %s", exp, json) 400 } 401 402 // Run a non-trivial query to force the "wrong range" metadata to flow through 403 // a number of components. 404 row = txn.QueryRowContext(context.Background(), query) 405 var cnt int 406 if err := row.Scan(&cnt); err != nil { 407 t.Fatal(err) 408 } 409 if cnt != 3 { 410 t.Fatalf("expected 3, got: %d", cnt) 411 } 412 if err := txn.Rollback(); err != nil { 413 t.Fatal(err) 414 } 415 416 // Now assert that new plans correctly contain all the nodes. This is expected 417 // to be a result of the caches having been updated on the gateway by the 418 // previous query. 419 row = db3.QueryRow(fmt.Sprintf(`SELECT json FROM [EXPLAIN (DISTSQL) %v]`, query)) 420 if err := row.Scan(&json); err != nil { 421 t.Fatal(err) 422 } 423 exp = `"nodeNames":["1","2","3","4"]` 424 if !strings.Contains(json, exp) { 425 t.Fatalf("expected json to contain %s, but json is: %s", exp, json) 426 } 427 } 428 429 func TestDistSQLDeadHosts(t *testing.T) { 430 defer leaktest.AfterTest(t)() 431 432 t.Skip("test is too slow; we need to tweak timeouts so connections die faster (see #14376)") 433 434 const n = 100 435 const numNodes = 5 436 437 tc := serverutils.StartTestCluster(t, numNodes, base.TestClusterArgs{ 438 ReplicationMode: base.ReplicationManual, 439 ServerArgs: base.TestServerArgs{UseDatabase: "test"}, 440 }) 441 defer tc.Stopper().Stop(context.Background()) 442 443 db := tc.ServerConn(0) 444 db.SetMaxOpenConns(1) 445 r := sqlutils.MakeSQLRunner(db) 446 r.Exec(t, "CREATE DATABASE test") 447 448 r.Exec(t, "CREATE TABLE t (x INT PRIMARY KEY, xsquared INT)") 449 450 for i := 0; i < numNodes; i++ { 451 r.Exec(t, fmt.Sprintf("ALTER TABLE t SPLIT AT VALUES (%d)", n*i/5)) 452 } 453 454 for i := 0; i < numNodes; i++ { 455 r.Exec(t, fmt.Sprintf( 456 "ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[%d,%d,%d], %d)", 457 i+1, (i+1)%5+1, (i+2)%5+1, n*i/5, 458 )) 459 } 460 461 r.Exec(t, "SHOW RANGES FROM TABLE t") 462 463 r.Exec(t, fmt.Sprintf("INSERT INTO t SELECT i, i*i FROM generate_series(1, %d) AS g(i)", n)) 464 465 r.Exec(t, "SET DISTSQL = ON") 466 467 // Run a query that uses the entire table and is easy to verify. 468 runQuery := func() error { 469 log.Infof(context.Background(), "running test query") 470 var res int 471 if err := db.QueryRow("SELECT sum(xsquared) FROM t").Scan(&res); err != nil { 472 return err 473 } 474 if exp := (n * (n + 1) * (2*n + 1)) / 6; res != exp { 475 t.Fatalf("incorrect result %d, expected %d", res, exp) 476 } 477 log.Infof(context.Background(), "test query OK") 478 return nil 479 } 480 if err := runQuery(); err != nil { 481 t.Error(err) 482 } 483 484 // Verify the plan (should include all 5 nodes). 485 r.CheckQueryResults(t, 486 "SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(xsquared) FROM t]", 487 [][]string{{"https://cockroachdb.github.io/distsqlplan/decode.html#eJy8k09LwzAYxu9-CnlOCu9h7bo5e5rHHXQy9SQ91OalFLamJCkoo99d1iDaIskgo8f8-T2_PG1yRC0FP-UH1kjfEYEQgzAHIQFhgYzQKFmw1lKdtlhgIz6RzghV3bTmNJ0RCqkY6RGmMntGitf8Y887zgUrEASbvNr3kkZVh1x9rQ0I29ak1-sYWUeQrflJ6-h8z0NZKi5zI0eal7fHm3V0e3b0b2JbSyVYsRgEZt2F5dFE38_jCakQT1TB4wmpMJ-ogscTUiGZqILHc6mH-E_0jnUja82jBznMywgsSrZvWctWFfysZNGH2-G2391PCNbGrkZ2sKnt0ulYf-HICccDOBrDsdvsUc-ddOKGk5BzL5zw0m1ehpjvnPDKbV6FmO_d_2rmuSbuSzZ2Z93VdwAAAP__XTV6BQ=="}}, 488 ) 489 490 // Stop node 5. 491 tc.StopServer(4) 492 493 testutils.SucceedsSoon(t, runQuery) 494 495 r.CheckQueryResults(t, 496 "SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(xsquared) FROM t]", 497 [][]string{{"https://cockroachdb.github.io/distsqlplan/decode.html#eJy8k8FK7DAYhff3KS5npZCF6dRx7KouZ6Ejo64ki9j8lEKnKUkKytB3lzaItkg60qHL5M93vpySHlFpRQ_yQBbJKzgYIjCswBBDMNRGZ2StNt3YH96qdyRXDEVVN67bFgyZNoTkCFe4kpDgWb6VtCepyIBBkZNF2QtqUxyk-UgdGHaNS_6nEUTLoBv3lday0z13eW4ol06PNE8v9xcpvzw5-juxqbRRZEgNAkV7Zjlf6PtNeOZUiBaqMOGZU2G1UIUJz7le8S_Re7K1riyNXvMwTzCQysn_CFY3JqNHo7M-3C93_el-Q5F1fsr9Ylv5UXetnzAPwtEA5mM4CsK3YfMqCMdhOJ5z7esgvA6b13PMN0F4EzZv_mQW7b_PAAAA__-DuA-E"}}, 498 ) 499 500 // Stop node 2; note that no range had replicas on both 2 and 5. 501 tc.StopServer(1) 502 503 testutils.SucceedsSoon(t, runQuery) 504 505 r.CheckQueryResults(t, 506 "SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(xsquared) FROM t]", 507 [][]string{{"https://cockroachdb.github.io/distsqlplan/decode.html#eJy8kkFLwzAUx-9-CvmfFHIwXZ3QUz3uoJOpJ8khNo9S6JrykoIy-t2lDaItkk02dkxe_r_fe-Ht0FhDj3pLDtkbJAQWEEihBFq2BTlneSiFhyvzgexGoGrazg_XSqCwTMh28JWvCRle9HtNG9KGGAKGvK7qEd5ytdX8mXsIrDufXeYJVC9gO_9N68XhnvuyZCq1tzPN8-vDVS6vD0b_ELvGsiEmMwGq_sRyeab_2-M5ZoTkTCPs8ZxqBf5Ab8i1tnE0W4UpTwmQKSlskbMdF_TEthjh4bgeX48XhpwPVRkOqyaUhrZ-h2U0nEzCch5OouG7uHkRDafxcHpM27fR8DJuXv7LrPqLrwAAAP__vMyldA=="}}, 508 ) 509 } 510 511 func TestDistSQLDrainingHosts(t *testing.T) { 512 defer leaktest.AfterTest(t)() 513 514 const numNodes = 2 515 tc := serverutils.StartTestCluster( 516 t, 517 numNodes, 518 base.TestClusterArgs{ 519 ReplicationMode: base.ReplicationManual, 520 ServerArgs: base.TestServerArgs{Knobs: base.TestingKnobs{DistSQL: &execinfra.TestingKnobs{DrainFast: true}}, UseDatabase: "test"}, 521 }, 522 ) 523 ctx := context.Background() 524 defer tc.Stopper().Stop(ctx) 525 526 conn := tc.ServerConn(0) 527 sqlutils.CreateTable( 528 t, 529 conn, 530 "nums", 531 "num INT", 532 numNodes, /* numRows */ 533 sqlutils.ToRowFn(sqlutils.RowIdxFn), 534 ) 535 536 db := tc.ServerConn(0) 537 db.SetMaxOpenConns(1) 538 r := sqlutils.MakeSQLRunner(db) 539 540 // Force the query to be distributed. 541 r.Exec(t, "SET DISTSQL = ON") 542 543 // Shortly after starting a cluster, the first server's StorePool may not be 544 // fully initialized and ready to do rebalancing yet, so wrap this in a 545 // SucceedsSoon. 546 testutils.SucceedsSoon(t, func() error { 547 _, err := db.Exec( 548 fmt.Sprintf(`ALTER TABLE nums SPLIT AT VALUES (1); 549 ALTER TABLE nums EXPERIMENTAL_RELOCATE VALUES (ARRAY[%d], 1);`, 550 tc.Server(1).GetFirstStoreID(), 551 ), 552 ) 553 return err 554 }) 555 556 // Ensure that the range cache is populated (see #31235). 557 r.Exec(t, "SHOW RANGES FROM TABLE nums") 558 559 const query = "SELECT count(*) FROM NUMS" 560 expectPlan := func(expectedPlan [][]string) { 561 planQuery := fmt.Sprintf(`SELECT url FROM [EXPLAIN (DISTSQL) %s]`, query) 562 testutils.SucceedsSoon(t, func() error { 563 resultPlan := r.QueryStr(t, planQuery) 564 if !reflect.DeepEqual(resultPlan, expectedPlan) { 565 return errors.Errorf("\nexpected:%v\ngot:%v", expectedPlan, resultPlan) 566 } 567 return nil 568 }) 569 } 570 571 // Verify distribution. 572 expectPlan([][]string{{"https://cockroachdb.github.io/distsqlplan/decode.html#eJyskd-Lm0AQx9_7V8g8mbKHWZO-7NMd1xSEnF7Vo4UgYesOIphduz-gJfi_F7WQGBKblj46O9_5fJw5gvneAINss908557TjfcpTV683ebr6_Ypij3_Y5Tl2eftwvvdUionrf9-MfZJdzAFEJBKYMwPaIDtgAKBEAoCrVYlGqN0Xz4OTZH4AWxJoJats325IFAqjcCOYGvbIDDI-bcGU-QCdbAEAgItr5thdKvrA9c_H3suEMhaLg3zHoIemTjLvFhJhKIjoJw9EYzlFQKjHbnf4qmqNFbcKh2EU4nn5C3O92nyJfMXN1nhTdYJ4aTSAjWKyfyim7dZT22yt5d9FOf-I70ts5rI0PvXT-9Zf0Af_mH9f7A4--HVf13_FVaKplXS4MUZrk9e9udBUeF4S6OcLvFVq3LAjJ_JkBsKAo0dX-n4EcnxqRc8D9PZcDgJ08twOBv-ME9ezYbX8-H1X2kX3btfAQAA__9aiHOO"}}) 573 574 // Drain the second node and expect the query to be planned on only the 575 // first node. 576 distServer := tc.Server(1).DistSQLServer().(*distsql.ServerImpl) 577 distServer.Drain(ctx, 0 /* flowDrainWait */, nil /* reporter */) 578 579 expectPlan([][]string{{"https://cockroachdb.github.io/distsqlplan/decode.html#eJyUkM9Kw0AYxO8-xTKnVlba9LgnS60QqElNIgolyJp8hEC6G_cPKCHvLkkErVDR4843M79hO9jXBgLpdrfdZMybht0m8R07bJ_2u3UYsdlNmGbp_W7OPi2F9srNLueTT_mjzcGhdEmRPJKFOCBAztEaXZC12gxSNxrC8g1iyVGr1rtBzjkKbQiig6tdQxDI5EtDCcmSzGIJjpKcrJuxtjX1UZr364EJjrSVygp2BY7YO8EirQh5z6G9--q3TlYEEfT87xvWVWWokk6bRXA6YRM_RNlzEj-ms_lZ1uo_rIRsq5WlE8655mWfc1BZ0fSnVntT0N7oYsRMz3jMjUJJ1k3XYHqEajoNA7-Hg1_Dqx_hvL_4CAAA__-ln7ge"}}) 580 581 // Verify correctness. 582 var res int 583 if err := db.QueryRow(query).Scan(&res); err != nil { 584 t.Fatal(err) 585 } 586 if res != numNodes { 587 t.Fatalf("expected %d rows but got %d", numNodes, res) 588 } 589 } 590 591 // testSpanResolverRange describes a range in a test. The ranges are specified 592 // in order, so only the start key is needed. 593 type testSpanResolverRange struct { 594 startKey string 595 node int 596 } 597 598 // testSpanResolver is a SpanResolver that uses a fixed set of ranges. 599 type testSpanResolver struct { 600 nodes []*roachpb.NodeDescriptor 601 602 ranges []testSpanResolverRange 603 } 604 605 // NewSpanResolverIterator is part of the SpanResolver interface. 606 func (tsr *testSpanResolver) NewSpanResolverIterator(_ *kv.Txn) physicalplan.SpanResolverIterator { 607 return &testSpanResolverIterator{tsr: tsr} 608 } 609 610 type testSpanResolverIterator struct { 611 tsr *testSpanResolver 612 curRangeIdx int 613 endKey string 614 } 615 616 var _ physicalplan.SpanResolverIterator = &testSpanResolverIterator{} 617 618 // Seek is part of the SpanResolverIterator interface. 619 func (it *testSpanResolverIterator) Seek( 620 ctx context.Context, span roachpb.Span, scanDir kvcoord.ScanDirection, 621 ) { 622 if scanDir != kvcoord.Ascending { 623 panic("descending not implemented") 624 } 625 it.endKey = string(span.EndKey) 626 key := string(span.Key) 627 i := 0 628 for ; i < len(it.tsr.ranges)-1; i++ { 629 if key < it.tsr.ranges[i+1].startKey { 630 break 631 } 632 } 633 it.curRangeIdx = i 634 } 635 636 // Valid is part of the SpanResolverIterator interface. 637 func (*testSpanResolverIterator) Valid() bool { 638 return true 639 } 640 641 // Error is part of the SpanResolverIterator interface. 642 func (*testSpanResolverIterator) Error() error { 643 return nil 644 } 645 646 // NeedAnother is part of the SpanResolverIterator interface. 647 func (it *testSpanResolverIterator) NeedAnother() bool { 648 return it.curRangeIdx < len(it.tsr.ranges)-1 && 649 it.tsr.ranges[it.curRangeIdx+1].startKey < it.endKey 650 } 651 652 // Next is part of the SpanResolverIterator interface. 653 func (it *testSpanResolverIterator) Next(_ context.Context) { 654 if !it.NeedAnother() { 655 panic("Next called with NeedAnother false") 656 } 657 it.curRangeIdx++ 658 } 659 660 // Desc is part of the SpanResolverIterator interface. 661 func (it *testSpanResolverIterator) Desc() roachpb.RangeDescriptor { 662 endKey := roachpb.RKeyMax 663 if it.curRangeIdx < len(it.tsr.ranges)-1 { 664 endKey = roachpb.RKey(it.tsr.ranges[it.curRangeIdx+1].startKey) 665 } 666 return roachpb.RangeDescriptor{ 667 StartKey: roachpb.RKey(it.tsr.ranges[it.curRangeIdx].startKey), 668 EndKey: endKey, 669 } 670 } 671 672 // ReplicaInfo is part of the SpanResolverIterator interface. 673 func (it *testSpanResolverIterator) ReplicaInfo( 674 _ context.Context, 675 ) (roachpb.ReplicaDescriptor, error) { 676 n := it.tsr.nodes[it.tsr.ranges[it.curRangeIdx].node-1] 677 return roachpb.ReplicaDescriptor{NodeID: n.NodeID}, nil 678 } 679 680 func TestPartitionSpans(t *testing.T) { 681 defer leaktest.AfterTest(t)() 682 683 testCases := []struct { 684 ranges []testSpanResolverRange 685 deadNodes []int 686 687 gatewayNode int 688 689 // spans to be passed to PartitionSpans 690 spans [][2]string 691 692 // expected result: a map of node to list of spans. 693 partitions map[int][][2]string 694 }{ 695 { 696 ranges: []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}, {"D", 3}}, 697 gatewayNode: 1, 698 699 spans: [][2]string{{"A1", "C1"}, {"D1", "X"}}, 700 701 partitions: map[int][][2]string{ 702 1: {{"A1", "B"}, {"C", "C1"}}, 703 2: {{"B", "C"}}, 704 3: {{"D1", "X"}}, 705 }, 706 }, 707 708 { 709 ranges: []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}, {"D", 3}}, 710 deadNodes: []int{1}, // The health status of the gateway node shouldn't matter. 711 gatewayNode: 1, 712 713 spans: [][2]string{{"A1", "C1"}, {"D1", "X"}}, 714 715 partitions: map[int][][2]string{ 716 1: {{"A1", "B"}, {"C", "C1"}}, 717 2: {{"B", "C"}}, 718 3: {{"D1", "X"}}, 719 }, 720 }, 721 722 { 723 ranges: []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}, {"D", 3}}, 724 deadNodes: []int{2}, 725 gatewayNode: 1, 726 727 spans: [][2]string{{"A1", "C1"}, {"D1", "X"}}, 728 729 partitions: map[int][][2]string{ 730 1: {{"A1", "C1"}}, 731 3: {{"D1", "X"}}, 732 }, 733 }, 734 735 { 736 ranges: []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}, {"D", 3}}, 737 deadNodes: []int{3}, 738 gatewayNode: 1, 739 740 spans: [][2]string{{"A1", "C1"}, {"D1", "X"}}, 741 742 partitions: map[int][][2]string{ 743 1: {{"A1", "B"}, {"C", "C1"}, {"D1", "X"}}, 744 2: {{"B", "C"}}, 745 }, 746 }, 747 748 { 749 ranges: []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}, {"D", 3}}, 750 deadNodes: []int{1}, 751 gatewayNode: 2, 752 753 spans: [][2]string{{"A1", "C1"}, {"D1", "X"}}, 754 755 partitions: map[int][][2]string{ 756 2: {{"A1", "C1"}}, 757 3: {{"D1", "X"}}, 758 }, 759 }, 760 761 { 762 ranges: []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}, {"D", 3}}, 763 deadNodes: []int{1}, 764 gatewayNode: 3, 765 766 spans: [][2]string{{"A1", "C1"}, {"D1", "X"}}, 767 768 partitions: map[int][][2]string{ 769 2: {{"B", "C"}}, 770 3: {{"A1", "B"}, {"C", "C1"}, {"D1", "X"}}, 771 }, 772 }, 773 } 774 775 // We need a mock Gossip to contain addresses for the nodes. Otherwise the 776 // DistSQLPlanner will not plan flows on them. 777 testStopper := stop.NewStopper() 778 defer testStopper.Stop(context.Background()) 779 mockGossip := gossip.NewTest(roachpb.NodeID(1), nil /* rpcContext */, nil, /* grpcServer */ 780 testStopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) 781 var nodeDescs []*roachpb.NodeDescriptor 782 for i := 1; i <= 10; i++ { 783 nodeID := roachpb.NodeID(i) 784 desc := &roachpb.NodeDescriptor{ 785 NodeID: nodeID, 786 Address: util.UnresolvedAddr{AddressField: fmt.Sprintf("addr%d", i)}, 787 } 788 if err := mockGossip.SetNodeDescriptor(desc); err != nil { 789 t.Fatal(err) 790 } 791 if err := mockGossip.AddInfoProto( 792 gossip.MakeDistSQLNodeVersionKey(nodeID), 793 &execinfrapb.DistSQLVersionGossipInfo{ 794 MinAcceptedVersion: execinfra.MinAcceptedVersion, 795 Version: execinfra.Version, 796 }, 797 0, // ttl - no expiration 798 ); err != nil { 799 t.Fatal(err) 800 } 801 802 nodeDescs = append(nodeDescs, desc) 803 } 804 805 for testIdx, tc := range testCases { 806 t.Run(strconv.Itoa(testIdx), func(t *testing.T) { 807 stopper := stop.NewStopper() 808 defer stopper.Stop(context.Background()) 809 810 tsp := &testSpanResolver{ 811 nodes: nodeDescs, 812 ranges: tc.ranges, 813 } 814 815 gw := gossip.MakeExposedGossip(mockGossip) 816 dsp := DistSQLPlanner{ 817 planVersion: execinfra.Version, 818 st: cluster.MakeTestingClusterSettings(), 819 nodeDesc: *tsp.nodes[tc.gatewayNode-1], 820 stopper: stopper, 821 spanResolver: tsp, 822 gossip: gw, 823 nodeHealth: distSQLNodeHealth{ 824 gossip: gw, 825 connHealth: func(node roachpb.NodeID, _ rpc.ConnectionClass) error { 826 for _, n := range tc.deadNodes { 827 if int(node) == n { 828 return fmt.Errorf("test node is unhealthy") 829 } 830 } 831 return nil 832 }, 833 isLive: func(nodeID roachpb.NodeID) (bool, error) { 834 return true, nil 835 }, 836 }, 837 } 838 839 planCtx := dsp.NewPlanningCtx(context.Background(), nil /* evalCtx */, nil /* txn */, true /* distribute */) 840 var spans []roachpb.Span 841 for _, s := range tc.spans { 842 spans = append(spans, roachpb.Span{Key: roachpb.Key(s[0]), EndKey: roachpb.Key(s[1])}) 843 } 844 845 partitions, err := dsp.PartitionSpans(planCtx, spans) 846 if err != nil { 847 t.Fatal(err) 848 } 849 850 resMap := make(map[int][][2]string) 851 for _, p := range partitions { 852 if _, ok := resMap[int(p.Node)]; ok { 853 t.Fatalf("node %d shows up in multiple partitions", p) 854 } 855 var spans [][2]string 856 for _, s := range p.Spans { 857 spans = append(spans, [2]string{string(s.Key), string(s.EndKey)}) 858 } 859 resMap[int(p.Node)] = spans 860 } 861 862 if !reflect.DeepEqual(resMap, tc.partitions) { 863 t.Errorf("expected partitions:\n %v\ngot:\n %v", tc.partitions, resMap) 864 } 865 }) 866 } 867 } 868 869 // Test that span partitioning takes into account the advertised acceptable 870 // versions of each node. Spans for which the owner node doesn't support our 871 // plan's version will be planned on the gateway. 872 func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { 873 defer leaktest.AfterTest(t)() 874 875 // The spans that we're going to plan for. 876 span := roachpb.Span{Key: roachpb.Key("A"), EndKey: roachpb.Key("Z")} 877 gatewayNode := roachpb.NodeID(2) 878 ranges := []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}} 879 880 testCases := []struct { 881 // the test's name 882 name string 883 884 // planVersion is the DistSQL version that this plan is targeting. 885 // We'll play with this version and expect nodes to be skipped because of 886 // this. 887 planVersion execinfrapb.DistSQLVersion 888 889 // The versions accepted by each node. 890 nodeVersions map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo 891 892 // nodesNotAdvertisingDistSQLVersion is the set of nodes for which gossip is 893 // not going to have information about the supported DistSQL version. This 894 // is to simulate CRDB 1.0 nodes which don't advertise this information. 895 nodesNotAdvertisingDistSQLVersion map[roachpb.NodeID]struct{} 896 897 // expected result: a map of node to list of spans. 898 partitions map[roachpb.NodeID][][2]string 899 }{ 900 { 901 // In the first test, all nodes are compatible. 902 name: "current_version", 903 planVersion: 2, 904 nodeVersions: map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo{ 905 1: { 906 MinAcceptedVersion: 1, 907 Version: 2, 908 }, 909 2: { 910 MinAcceptedVersion: 1, 911 Version: 2, 912 }, 913 }, 914 partitions: map[roachpb.NodeID][][2]string{ 915 1: {{"A", "B"}, {"C", "Z"}}, 916 2: {{"B", "C"}}, 917 }, 918 }, 919 { 920 // Plan version is incompatible with node 1. We expect everything to be 921 // assigned to the gateway. 922 // Remember that the gateway is node 2. 923 name: "next_version", 924 planVersion: 3, 925 nodeVersions: map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo{ 926 1: { 927 MinAcceptedVersion: 1, 928 Version: 2, 929 }, 930 2: { 931 MinAcceptedVersion: 3, 932 Version: 3, 933 }, 934 }, 935 partitions: map[roachpb.NodeID][][2]string{ 936 2: {{"A", "Z"}}, 937 }, 938 }, 939 { 940 // Like the above, except node 1 is not gossiping its version (simulating 941 // a crdb 1.0 node). 942 name: "crdb_1.0", 943 planVersion: 3, 944 nodeVersions: map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo{ 945 2: { 946 MinAcceptedVersion: 3, 947 Version: 3, 948 }, 949 }, 950 nodesNotAdvertisingDistSQLVersion: map[roachpb.NodeID]struct{}{ 951 1: {}, 952 }, 953 partitions: map[roachpb.NodeID][][2]string{ 954 2: {{"A", "Z"}}, 955 }, 956 }, 957 } 958 959 for _, tc := range testCases { 960 t.Run(tc.name, func(t *testing.T) { 961 962 stopper := stop.NewStopper() 963 defer stopper.Stop(context.Background()) 964 965 // We need a mock Gossip to contain addresses for the nodes. Otherwise the 966 // DistSQLPlanner will not plan flows on them. This Gossip will also 967 // reflect tc.nodesNotAdvertisingDistSQLVersion. 968 testStopper := stop.NewStopper() 969 defer testStopper.Stop(context.Background()) 970 mockGossip := gossip.NewTest(roachpb.NodeID(1), nil /* rpcContext */, nil, /* grpcServer */ 971 testStopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) 972 var nodeDescs []*roachpb.NodeDescriptor 973 for i := 1; i <= 2; i++ { 974 nodeID := roachpb.NodeID(i) 975 desc := &roachpb.NodeDescriptor{ 976 NodeID: nodeID, 977 Address: util.UnresolvedAddr{AddressField: fmt.Sprintf("addr%d", i)}, 978 } 979 if err := mockGossip.SetNodeDescriptor(desc); err != nil { 980 t.Fatal(err) 981 } 982 if _, ok := tc.nodesNotAdvertisingDistSQLVersion[nodeID]; !ok { 983 verInfo := tc.nodeVersions[nodeID] 984 if err := mockGossip.AddInfoProto( 985 gossip.MakeDistSQLNodeVersionKey(nodeID), 986 &verInfo, 987 0, // ttl - no expiration 988 ); err != nil { 989 t.Fatal(err) 990 } 991 } 992 993 nodeDescs = append(nodeDescs, desc) 994 } 995 tsp := &testSpanResolver{ 996 nodes: nodeDescs, 997 ranges: ranges, 998 } 999 1000 gw := gossip.MakeExposedGossip(mockGossip) 1001 dsp := DistSQLPlanner{ 1002 planVersion: tc.planVersion, 1003 st: cluster.MakeTestingClusterSettings(), 1004 nodeDesc: *tsp.nodes[gatewayNode-1], 1005 stopper: stopper, 1006 spanResolver: tsp, 1007 gossip: gw, 1008 nodeHealth: distSQLNodeHealth{ 1009 gossip: gw, 1010 connHealth: func(roachpb.NodeID, rpc.ConnectionClass) error { 1011 // All the nodes are healthy. 1012 return nil 1013 }, 1014 isLive: func(roachpb.NodeID) (bool, error) { 1015 return true, nil 1016 }, 1017 }, 1018 } 1019 1020 planCtx := dsp.NewPlanningCtx(context.Background(), nil /* evalCtx */, nil /* txn */, true /* distribute */) 1021 partitions, err := dsp.PartitionSpans(planCtx, roachpb.Spans{span}) 1022 if err != nil { 1023 t.Fatal(err) 1024 } 1025 1026 resMap := make(map[roachpb.NodeID][][2]string) 1027 for _, p := range partitions { 1028 if _, ok := resMap[p.Node]; ok { 1029 t.Fatalf("node %d shows up in multiple partitions", p) 1030 } 1031 var spans [][2]string 1032 for _, s := range p.Spans { 1033 spans = append(spans, [2]string{string(s.Key), string(s.EndKey)}) 1034 } 1035 resMap[p.Node] = spans 1036 } 1037 1038 if !reflect.DeepEqual(resMap, tc.partitions) { 1039 t.Errorf("expected partitions:\n %v\ngot:\n %v", tc.partitions, resMap) 1040 } 1041 }) 1042 } 1043 } 1044 1045 // Test that a node whose descriptor info is not accessible through gossip is 1046 // not used. This is to simulate nodes that have been decomisioned and also 1047 // nodes that have been "replaced" by another node at the same address (which, I 1048 // guess, is also a type of decomissioning). 1049 func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { 1050 defer leaktest.AfterTest(t)() 1051 1052 // The spans that we're going to plan for. 1053 span := roachpb.Span{Key: roachpb.Key("A"), EndKey: roachpb.Key("Z")} 1054 gatewayNode := roachpb.NodeID(2) 1055 ranges := []testSpanResolverRange{{"A", 1}, {"B", 2}, {"C", 1}} 1056 1057 stopper := stop.NewStopper() 1058 defer stopper.Stop(context.Background()) 1059 1060 mockGossip := gossip.NewTest(roachpb.NodeID(1), nil /* rpcContext */, nil, /* grpcServer */ 1061 stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) 1062 var nodeDescs []*roachpb.NodeDescriptor 1063 for i := 1; i <= 2; i++ { 1064 nodeID := roachpb.NodeID(i) 1065 desc := &roachpb.NodeDescriptor{ 1066 NodeID: nodeID, 1067 Address: util.UnresolvedAddr{AddressField: fmt.Sprintf("addr%d", i)}, 1068 } 1069 if i == 2 { 1070 if err := mockGossip.SetNodeDescriptor(desc); err != nil { 1071 t.Fatal(err) 1072 } 1073 } 1074 // All the nodes advertise their DistSQL versions. This is to simulate the 1075 // "node overridden by another node at the same address" case mentioned in 1076 // the test comment - for such a node, the descriptor would be taken out of 1077 // the gossip data, but other datums it advertised are left in place. 1078 if err := mockGossip.AddInfoProto( 1079 gossip.MakeDistSQLNodeVersionKey(nodeID), 1080 &execinfrapb.DistSQLVersionGossipInfo{ 1081 MinAcceptedVersion: execinfra.MinAcceptedVersion, 1082 Version: execinfra.Version, 1083 }, 1084 0, // ttl - no expiration 1085 ); err != nil { 1086 t.Fatal(err) 1087 } 1088 1089 nodeDescs = append(nodeDescs, desc) 1090 } 1091 tsp := &testSpanResolver{ 1092 nodes: nodeDescs, 1093 ranges: ranges, 1094 } 1095 1096 gw := gossip.MakeExposedGossip(mockGossip) 1097 dsp := DistSQLPlanner{ 1098 planVersion: execinfra.Version, 1099 st: cluster.MakeTestingClusterSettings(), 1100 nodeDesc: *tsp.nodes[gatewayNode-1], 1101 stopper: stopper, 1102 spanResolver: tsp, 1103 gossip: gw, 1104 nodeHealth: distSQLNodeHealth{ 1105 gossip: gw, 1106 connHealth: func(node roachpb.NodeID, _ rpc.ConnectionClass) error { 1107 _, err := mockGossip.GetNodeIDAddress(node) 1108 return err 1109 }, 1110 isLive: func(roachpb.NodeID) (bool, error) { 1111 return true, nil 1112 }, 1113 }, 1114 } 1115 1116 planCtx := dsp.NewPlanningCtx(context.Background(), nil /* evalCtx */, nil /* txn */, true /* distribute */) 1117 partitions, err := dsp.PartitionSpans(planCtx, roachpb.Spans{span}) 1118 if err != nil { 1119 t.Fatal(err) 1120 } 1121 1122 resMap := make(map[roachpb.NodeID][][2]string) 1123 for _, p := range partitions { 1124 if _, ok := resMap[p.Node]; ok { 1125 t.Fatalf("node %d shows up in multiple partitions", p) 1126 } 1127 var spans [][2]string 1128 for _, s := range p.Spans { 1129 spans = append(spans, [2]string{string(s.Key), string(s.EndKey)}) 1130 } 1131 resMap[p.Node] = spans 1132 } 1133 1134 expectedPartitions := 1135 map[roachpb.NodeID][][2]string{ 1136 2: {{"A", "Z"}}, 1137 } 1138 if !reflect.DeepEqual(resMap, expectedPartitions) { 1139 t.Errorf("expected partitions:\n %v\ngot:\n %v", expectedPartitions, resMap) 1140 } 1141 } 1142 1143 func TestCheckNodeHealth(t *testing.T) { 1144 defer leaktest.AfterTest(t)() 1145 1146 stopper := stop.NewStopper() 1147 defer stopper.Stop(context.Background()) 1148 1149 const nodeID = roachpb.NodeID(5) 1150 1151 mockGossip := gossip.NewTest(nodeID, nil /* rpcContext */, nil, /* grpcServer */ 1152 stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) 1153 1154 desc := &roachpb.NodeDescriptor{ 1155 NodeID: nodeID, 1156 Address: util.UnresolvedAddr{NetworkField: "tcp", AddressField: "testaddr"}, 1157 } 1158 if err := mockGossip.SetNodeDescriptor(desc); err != nil { 1159 t.Fatal(err) 1160 } 1161 if err := mockGossip.AddInfoProto( 1162 gossip.MakeDistSQLNodeVersionKey(nodeID), 1163 &execinfrapb.DistSQLVersionGossipInfo{ 1164 MinAcceptedVersion: execinfra.MinAcceptedVersion, 1165 Version: execinfra.Version, 1166 }, 1167 0, // ttl - no expiration 1168 ); err != nil { 1169 t.Fatal(err) 1170 } 1171 1172 errLive := func(roachpb.NodeID) (bool, error) { 1173 return false, errors.New("injected liveness error") 1174 } 1175 notLive := func(roachpb.NodeID) (bool, error) { 1176 return false, nil 1177 } 1178 live := func(roachpb.NodeID) (bool, error) { 1179 return true, nil 1180 } 1181 1182 connHealthy := func(roachpb.NodeID, rpc.ConnectionClass) error { 1183 return nil 1184 } 1185 connUnhealthy := func(roachpb.NodeID, rpc.ConnectionClass) error { 1186 return errors.New("injected conn health error") 1187 } 1188 _ = connUnhealthy 1189 1190 livenessTests := []struct { 1191 isLive func(roachpb.NodeID) (bool, error) 1192 exp string 1193 }{ 1194 {live, ""}, 1195 {errLive, "not using n5 due to liveness: injected liveness error"}, 1196 {notLive, "not using n5 due to liveness: node n5 is not live"}, 1197 } 1198 1199 gw := gossip.MakeExposedGossip(mockGossip) 1200 for _, test := range livenessTests { 1201 t.Run("liveness", func(t *testing.T) { 1202 h := distSQLNodeHealth{ 1203 gossip: gw, 1204 connHealth: connHealthy, 1205 isLive: test.isLive, 1206 } 1207 if err := h.check(context.Background(), nodeID); !testutils.IsError(err, test.exp) { 1208 t.Fatalf("expected %v, got %v", test.exp, err) 1209 } 1210 }) 1211 } 1212 1213 connHealthTests := []struct { 1214 connHealth func(roachpb.NodeID, rpc.ConnectionClass) error 1215 exp string 1216 }{ 1217 {connHealthy, ""}, 1218 {connUnhealthy, "injected conn health error"}, 1219 } 1220 1221 for _, test := range connHealthTests { 1222 t.Run("connHealth", func(t *testing.T) { 1223 h := distSQLNodeHealth{ 1224 gossip: gw, 1225 connHealth: test.connHealth, 1226 isLive: live, 1227 } 1228 if err := h.check(context.Background(), nodeID); !testutils.IsError(err, test.exp) { 1229 t.Fatalf("expected %v, got %v", test.exp, err) 1230 } 1231 }) 1232 } 1233 }