github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/physicalplan/span_resolver_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 physicalplan_test
    12  
    13  import (
    14  	"context"
    15  	gosql "database/sql"
    16  	"fmt"
    17  	"testing"
    18  
    19  	"github.com/cockroachdb/cockroach/pkg/base"
    20  	"github.com/cockroachdb/cockroach/pkg/gossip"
    21  	"github.com/cockroachdb/cockroach/pkg/keys"
    22  	"github.com/cockroachdb/cockroach/pkg/kv"
    23  	"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
    24  	"github.com/cockroachdb/cockroach/pkg/roachpb"
    25  	"github.com/cockroachdb/cockroach/pkg/server"
    26  	"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
    27  	"github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle"
    28  	"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
    29  	"github.com/cockroachdb/cockroach/pkg/testutils"
    30  	"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
    31  	"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
    32  	"github.com/cockroachdb/cockroach/pkg/util/leaktest"
    33  	"github.com/cockroachdb/errors"
    34  )
    35  
    36  // Test that resolving spans uses a node's range cache and lease holder cache.
    37  // The idea is to test that resolving is not random, but predictable given the
    38  // state of caches.
    39  func TestSpanResolverUsesCaches(t *testing.T) {
    40  	defer leaktest.AfterTest(t)()
    41  	tc := testcluster.StartTestCluster(t, 4,
    42  		base.TestClusterArgs{
    43  			ReplicationMode: base.ReplicationManual,
    44  			ServerArgs: base.TestServerArgs{
    45  				UseDatabase: "t",
    46  			},
    47  		})
    48  	defer tc.Stopper().Stop(context.Background())
    49  
    50  	rowRanges, _ := setupRanges(
    51  		tc.Conns[0], tc.Servers[0], tc.Servers[0].DB(), t)
    52  
    53  	// Replicate the row ranges on all of the first 3 nodes. Save the 4th node in
    54  	// a pristine state, with empty caches.
    55  	for i := 0; i < 3; i++ {
    56  		rowRanges[i] = tc.AddReplicasOrFatal(
    57  			t, rowRanges[i].StartKey.AsRawKey(), tc.Target(1), tc.Target(2))
    58  	}
    59  
    60  	// Scatter the leases around; node i gets range i.
    61  	for i := 0; i < 3; i++ {
    62  		if err := tc.TransferRangeLease(rowRanges[i], tc.Target(i)); err != nil {
    63  			t.Fatal(err)
    64  		}
    65  		// Wait for everybody to apply the new lease, so that we can rely on the
    66  		// lease discovery done later by the SpanResolver to be up to date.
    67  		testutils.SucceedsSoon(t, func() error {
    68  			for j := 0; j < 3; j++ {
    69  				target := tc.Target(j)
    70  				rt, err := tc.FindRangeLeaseHolder(rowRanges[i], &target)
    71  				if err != nil {
    72  					return err
    73  				}
    74  				if rt != tc.Target(i) {
    75  					return errors.Errorf("node %d hasn't applied the lease yet", j)
    76  				}
    77  			}
    78  			return nil
    79  		})
    80  	}
    81  
    82  	// Create a SpanResolver using the 4th node, with empty caches.
    83  	s3 := tc.Servers[3]
    84  
    85  	lr := physicalplan.NewSpanResolver(
    86  		s3.Cfg.Settings,
    87  		s3.DistSenderI().(*kvcoord.DistSender),
    88  		gossip.MakeExposedGossip(s3.Gossip()),
    89  		s3.GetNode().Descriptor, nil,
    90  		replicaoracle.BinPackingChoice)
    91  
    92  	var spans []spanWithDir
    93  	for i := 0; i < 3; i++ {
    94  		spans = append(
    95  			spans,
    96  			spanWithDir{
    97  				Span: roachpb.Span{
    98  					Key:    rowRanges[i].StartKey.AsRawKey(),
    99  					EndKey: rowRanges[i].EndKey.AsRawKey(),
   100  				},
   101  				dir: kvcoord.Ascending,
   102  			})
   103  	}
   104  
   105  	// Resolve the spans. Since the LeaseHolderCache is empty, all the ranges
   106  	// should be grouped and "assigned" to replica 0.
   107  	replicas, err := resolveSpans(context.Background(), lr.NewSpanResolverIterator(nil), spans...)
   108  	if err != nil {
   109  		t.Fatal(err)
   110  	}
   111  	if len(replicas) != 3 {
   112  		t.Fatalf("expected replies for 3 spans, got %d", len(replicas))
   113  	}
   114  	si := tc.Servers[0]
   115  
   116  	storeID := si.GetFirstStoreID()
   117  	for i := 0; i < 3; i++ {
   118  		if len(replicas[i]) != 1 {
   119  			t.Fatalf("expected 1 range for span %s, got %d",
   120  				spans[i].Span, len(replicas[i]))
   121  		}
   122  		rd := replicas[i][0].ReplicaDescriptor
   123  		if rd.StoreID != storeID {
   124  			t.Fatalf("expected span %s to be on replica (%d) but was on %s",
   125  				spans[i].Span, storeID, rd)
   126  		}
   127  	}
   128  
   129  	// Now populate the cached on node 4 and query again. This time, we expect to see
   130  	// each span on its own range.
   131  	if err := populateCache(tc.Conns[3], 3 /* expectedNumRows */); err != nil {
   132  		t.Fatal(err)
   133  	}
   134  	replicas, err = resolveSpans(context.Background(), lr.NewSpanResolverIterator(nil), spans...)
   135  	if err != nil {
   136  		t.Fatal(err)
   137  	}
   138  
   139  	var expected [][]rngInfo
   140  	for i := 0; i < 3; i++ {
   141  		expected = append(expected, []rngInfo{selectReplica(tc.Servers[i].NodeID(), rowRanges[i])})
   142  	}
   143  	if err = expectResolved(replicas, expected...); err != nil {
   144  		t.Fatal(err)
   145  	}
   146  }
   147  
   148  // populateCache runs a scan over a whole table to populate the range cache and
   149  // the lease holder cache of the server to which db is connected.
   150  func populateCache(db *gosql.DB, expectedNumRows int) error {
   151  	var numRows int
   152  	err := db.QueryRow(`SELECT count(1) FROM test`).Scan(&numRows)
   153  	if err != nil {
   154  		return err
   155  	}
   156  	if numRows != expectedNumRows {
   157  		return errors.Errorf("expected %d rows, got %d", expectedNumRows, numRows)
   158  	}
   159  	return nil
   160  }
   161  
   162  // splitRangeAtVal splits the range for a table with schema
   163  // `CREATE TABLE test (k INT PRIMARY KEY)` at row with value pk (the row will be
   164  // the first on the right of the split).
   165  func splitRangeAtVal(
   166  	ts *server.TestServer, tableDesc *sqlbase.TableDescriptor, pk int,
   167  ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) {
   168  	if len(tableDesc.Indexes) != 0 {
   169  		return roachpb.RangeDescriptor{}, roachpb.RangeDescriptor{},
   170  			errors.AssertionFailedf("expected table with just a PK, got: %+v", tableDesc)
   171  	}
   172  	pik, err := sqlbase.TestingMakePrimaryIndexKey(tableDesc, pk)
   173  	if err != nil {
   174  		return roachpb.RangeDescriptor{}, roachpb.RangeDescriptor{}, err
   175  	}
   176  
   177  	leftRange, rightRange, err := ts.SplitRange(pik)
   178  	if err != nil {
   179  		return roachpb.RangeDescriptor{}, roachpb.RangeDescriptor{},
   180  			errors.Wrapf(err, "failed to split at row: %d", pk)
   181  	}
   182  	return leftRange, rightRange, nil
   183  }
   184  
   185  func TestSpanResolver(t *testing.T) {
   186  	defer leaktest.AfterTest(t)()
   187  	s, db, cdb := serverutils.StartServer(t, base.TestServerArgs{
   188  		UseDatabase: "t",
   189  	})
   190  	defer s.Stopper().Stop(context.Background())
   191  
   192  	rowRanges, tableDesc := setupRanges(db, s.(*server.TestServer), cdb, t)
   193  	lr := physicalplan.NewSpanResolver(
   194  		s.(*server.TestServer).Cfg.Settings,
   195  		s.DistSenderI().(*kvcoord.DistSender),
   196  		gossip.MakeExposedGossip(s.GossipI().(*gossip.Gossip)),
   197  		s.(*server.TestServer).GetNode().Descriptor, nil,
   198  		replicaoracle.BinPackingChoice)
   199  
   200  	ctx := context.Background()
   201  	it := lr.NewSpanResolverIterator(nil)
   202  
   203  	testCases := []struct {
   204  		spans    []roachpb.Span
   205  		expected [][]rngInfo
   206  	}{
   207  		{
   208  			[]roachpb.Span{makeSpan(tableDesc, 0, 10000)},
   209  			[][]rngInfo{{
   210  				onlyReplica(rowRanges[0]),
   211  				onlyReplica(rowRanges[1]),
   212  				onlyReplica(rowRanges[2])}},
   213  		},
   214  		{
   215  			[]roachpb.Span{
   216  				makeSpan(tableDesc, 0, 9),
   217  				makeSpan(tableDesc, 11, 19),
   218  				makeSpan(tableDesc, 21, 29),
   219  			},
   220  			[][]rngInfo{
   221  				{onlyReplica(rowRanges[0])},
   222  				{onlyReplica(rowRanges[1])},
   223  				{onlyReplica(rowRanges[2])},
   224  			},
   225  		},
   226  		{
   227  			[]roachpb.Span{
   228  				makeSpan(tableDesc, 0, 20),
   229  				makeSpan(tableDesc, 20, 29),
   230  			},
   231  			[][]rngInfo{
   232  				{onlyReplica(rowRanges[0]), onlyReplica(rowRanges[1])},
   233  				{onlyReplica(rowRanges[2])},
   234  			},
   235  		},
   236  		{
   237  			[]roachpb.Span{
   238  				makeSpan(tableDesc, 0, 1),
   239  				makeSpan(tableDesc, 1, 2),
   240  				makeSpan(tableDesc, 2, 3),
   241  				makeSpan(tableDesc, 3, 4),
   242  				makeSpan(tableDesc, 5, 11),
   243  				makeSpan(tableDesc, 20, 29),
   244  			},
   245  			[][]rngInfo{
   246  				{onlyReplica(rowRanges[0])},
   247  				{onlyReplica(rowRanges[0])},
   248  				{onlyReplica(rowRanges[0])},
   249  				{onlyReplica(rowRanges[0])},
   250  				{onlyReplica(rowRanges[0]), onlyReplica(rowRanges[1])},
   251  				{onlyReplica(rowRanges[2])},
   252  			},
   253  		},
   254  	}
   255  	for i, tc := range testCases {
   256  		for _, dir := range []kvcoord.ScanDirection{kvcoord.Ascending, kvcoord.Descending} {
   257  			t.Run(fmt.Sprintf("%d-direction:%d", i, dir), func(t *testing.T) {
   258  				replicas, err := resolveSpans(ctx, it, orient(dir, tc.spans...)...)
   259  				if err != nil {
   260  					t.Fatal(err)
   261  				}
   262  				if dir == kvcoord.Descending {
   263  					// When testing Descending resolving, reverse the expected results.
   264  					for i, j := 0, len(tc.expected)-1; i <= j; i, j = i+1, j-1 {
   265  						reverse(tc.expected[i])
   266  						if i != j {
   267  							reverse(tc.expected[j])
   268  						}
   269  						tc.expected[i], tc.expected[j] = tc.expected[j], tc.expected[i]
   270  					}
   271  				}
   272  				if err = expectResolved(replicas, tc.expected...); err != nil {
   273  					t.Fatal(err)
   274  				}
   275  			})
   276  		}
   277  	}
   278  }
   279  
   280  func TestMixedDirections(t *testing.T) {
   281  	defer leaktest.AfterTest(t)()
   282  	s, db, cdb := serverutils.StartServer(t, base.TestServerArgs{
   283  		UseDatabase: "t",
   284  	})
   285  	defer s.Stopper().Stop(context.Background())
   286  
   287  	rowRanges, tableDesc := setupRanges(db, s.(*server.TestServer), cdb, t)
   288  	lr := physicalplan.NewSpanResolver(
   289  		s.(*server.TestServer).Cfg.Settings,
   290  		s.DistSenderI().(*kvcoord.DistSender),
   291  		gossip.MakeExposedGossip(s.GossipI().(*gossip.Gossip)),
   292  		s.(*server.TestServer).GetNode().Descriptor,
   293  		nil,
   294  		replicaoracle.BinPackingChoice)
   295  
   296  	ctx := context.Background()
   297  	it := lr.NewSpanResolverIterator(nil)
   298  
   299  	spans := []spanWithDir{
   300  		orient(kvcoord.Ascending, makeSpan(tableDesc, 11, 15))[0],
   301  		orient(kvcoord.Descending, makeSpan(tableDesc, 1, 14))[0],
   302  	}
   303  	replicas, err := resolveSpans(ctx, it, spans...)
   304  	if err != nil {
   305  		t.Fatal(err)
   306  	}
   307  	expected := [][]rngInfo{
   308  		{onlyReplica(rowRanges[1])},
   309  		{onlyReplica(rowRanges[1]), onlyReplica(rowRanges[0])},
   310  	}
   311  	if err = expectResolved(replicas, expected...); err != nil {
   312  		t.Fatal(err)
   313  	}
   314  }
   315  
   316  func setupRanges(
   317  	db *gosql.DB, s *server.TestServer, cdb *kv.DB, t *testing.T,
   318  ) ([]roachpb.RangeDescriptor, *sqlbase.TableDescriptor) {
   319  	if _, err := db.Exec(`CREATE DATABASE t`); err != nil {
   320  		t.Fatal(err)
   321  	}
   322  	if _, err := db.Exec(`CREATE TABLE test (k INT PRIMARY KEY)`); err != nil {
   323  		t.Fatal(err)
   324  	}
   325  	values := []int{0, 10, 20}
   326  	for _, val := range values {
   327  		// Multiply by 10 to space out the values so we can easily construct keys
   328  		// that fall within the range.
   329  		if _, err := db.Exec("INSERT INTO test VALUES ($1)", val*10); err != nil {
   330  			t.Fatal(err)
   331  		}
   332  	}
   333  
   334  	tableDesc := sqlbase.GetTableDescriptor(cdb, keys.SystemSQLCodec, "t", "test")
   335  	// Split every SQL row to its own range.
   336  	rowRanges := make([]roachpb.RangeDescriptor, len(values))
   337  	for i, val := range values {
   338  		var err error
   339  		var l roachpb.RangeDescriptor
   340  		l, rowRanges[i], err = splitRangeAtVal(s, tableDesc, val)
   341  		if err != nil {
   342  			t.Fatal(err)
   343  		}
   344  		if i > 0 {
   345  			rowRanges[i-1] = l
   346  		}
   347  	}
   348  
   349  	// Run a select across the whole table to populate the caches with all the
   350  	// ranges.
   351  	if _, err := db.Exec(`SELECT count(1) from test`); err != nil {
   352  		t.Fatal(err)
   353  	}
   354  
   355  	return rowRanges, tableDesc
   356  }
   357  
   358  type spanWithDir struct {
   359  	roachpb.Span
   360  	dir kvcoord.ScanDirection
   361  }
   362  
   363  func orient(dir kvcoord.ScanDirection, spans ...roachpb.Span) []spanWithDir {
   364  	res := make([]spanWithDir, 0, len(spans))
   365  	for _, span := range spans {
   366  		res = append(res, spanWithDir{span, dir})
   367  	}
   368  	if dir == kvcoord.Descending {
   369  		for i, j := 0, len(res)-1; i < j; i, j = i+1, j-1 {
   370  			res[i], res[j] = res[j], res[i]
   371  		}
   372  	}
   373  	return res
   374  }
   375  
   376  type rngInfo struct {
   377  	roachpb.ReplicaDescriptor
   378  	rngDesc roachpb.RangeDescriptor
   379  }
   380  
   381  func resolveSpans(
   382  	ctx context.Context, it physicalplan.SpanResolverIterator, spans ...spanWithDir,
   383  ) ([][]rngInfo, error) {
   384  	res := make([][]rngInfo, 0)
   385  	for _, span := range spans {
   386  		repls := make([]rngInfo, 0)
   387  		for it.Seek(ctx, span.Span, span.dir); ; it.Next(ctx) {
   388  			if !it.Valid() {
   389  				return nil, it.Error()
   390  			}
   391  			repl, err := it.ReplicaInfo(ctx)
   392  			if err != nil {
   393  				return nil, err
   394  			}
   395  			repls = append(repls, rngInfo{
   396  				ReplicaDescriptor: repl,
   397  				rngDesc:           it.Desc(),
   398  			})
   399  			if !it.NeedAnother() {
   400  				break
   401  			}
   402  		}
   403  		res = append(res, repls)
   404  	}
   405  	return res, nil
   406  }
   407  
   408  func onlyReplica(rng roachpb.RangeDescriptor) rngInfo {
   409  	if len(rng.InternalReplicas) != 1 {
   410  		panic(fmt.Sprintf("expected one replica in %+v", rng))
   411  	}
   412  	return rngInfo{ReplicaDescriptor: rng.InternalReplicas[0], rngDesc: rng}
   413  }
   414  
   415  func selectReplica(nodeID roachpb.NodeID, rng roachpb.RangeDescriptor) rngInfo {
   416  	for _, rep := range rng.InternalReplicas {
   417  		if rep.NodeID == nodeID {
   418  			return rngInfo{ReplicaDescriptor: rep, rngDesc: rng}
   419  		}
   420  	}
   421  	panic(fmt.Sprintf("no replica on node %d in: %s", nodeID, &rng))
   422  }
   423  
   424  func expectResolved(actual [][]rngInfo, expected ...[]rngInfo) error {
   425  	if len(actual) != len(expected) {
   426  		return errors.Errorf(
   427  			"expected %d ranges, got %d", len(expected), len(actual))
   428  	}
   429  	for i, exp := range expected {
   430  		act := actual[i]
   431  		if len(exp) != len(act) {
   432  			return errors.Errorf("expected %d ranges, got %d",
   433  				len(exp), len(act))
   434  		}
   435  		for i, e := range exp {
   436  			a := act[i]
   437  			if e.ReplicaDescriptor.StoreID != a.ReplicaDescriptor.StoreID || e.rngDesc.RangeID != a.rngDesc.RangeID {
   438  				return errors.Errorf(
   439  					"expected replica (%d,%d) but got: (%d,%d)",
   440  					e.ReplicaDescriptor.StoreID, e.rngDesc.RangeID,
   441  					a.ReplicaDescriptor.StoreID, a.rngDesc.RangeID)
   442  			}
   443  		}
   444  	}
   445  	return nil
   446  }
   447  
   448  func makeSpan(tableDesc *sqlbase.TableDescriptor, i, j int) roachpb.Span {
   449  	makeKey := func(val int) roachpb.Key {
   450  		key, err := sqlbase.TestingMakePrimaryIndexKey(tableDesc, val)
   451  		if err != nil {
   452  			panic(err)
   453  		}
   454  		return key
   455  	}
   456  	return roachpb.Span{
   457  		Key:    makeKey(i),
   458  		EndKey: makeKey(j),
   459  	}
   460  }
   461  
   462  func reverse(r []rngInfo) {
   463  	for i, j := 0, len(r)-1; i < j; i, j = i+1, j-1 {
   464  		r[i], r[j] = r[j], r[i]
   465  	}
   466  }