Skip to content

Commit

Permalink
multitenant: NewIterator connector infinite retry loop
Browse files Browse the repository at this point in the history
Fixes #98822

This change fixes an infinite retry loop in `Connector.NewIterator` that would
occur when the `GetRangeDescriptors` stream returned an auth error. An example
trigger would be passing in a span that was outside of the calling tenant's
keyspace.

Now `NewIterator` correctly propagates auth errors up to the caller.

Release note: None
  • Loading branch information
ecwall committed Mar 22, 2023
1 parent 4be5775 commit f3b8baf
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 11 deletions.
7 changes: 5 additions & 2 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -536,9 +536,12 @@ func (c *Connector) NewIterator(
curIdx: 0,
}, nil
}
// TODO(arul): We probably don't want to treat all errors here as "soft".
// Soft RPC error. Drop client and retry.
log.Warningf(ctx, "error consuming GetRangeDescriptors RPC: %v", err)
if grpcutil.IsAuthError(err) {
// Authentication or authorization error. Propagate.
return nil, err
}
// Soft RPC error. Drop client and retry.
c.tryForgetClient(ctx, client)
break
}
Expand Down
36 changes: 27 additions & 9 deletions pkg/ccl/kvccl/kvtenantccl/tenant_scan_range_descriptors_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,16 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/stretchr/testify/require"
)

// TestScanRangeDescriptors is an integration test to ensure that tenants can
// scan range descriptors iff they correspond to tenant owned ranges.
func TestScanRangeDescriptors(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
func setup(
t *testing.T, ctx context.Context,
) (*testcluster.TestCluster, serverutils.TestTenantInterface, rangedesc.IteratorFactory) {
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
DisableDefaultTestTenant: true, // we're going to manually add tenants
Expand All @@ -38,16 +36,26 @@ func TestScanRangeDescriptors(t *testing.T) {
},
},
})
defer tc.Stopper().Stop(ctx)

ten2ID := roachpb.MustMakeTenantID(2)
tenant2, err := tc.Server(0).StartTenant(ctx, base.TestTenantArgs{
TenantID: ten2ID,
})
require.NoError(t, err)
return tc, tenant2, tenant2.RangeDescIteratorFactory().(rangedesc.IteratorFactory)
}

// TestScanRangeDescriptors is an integration test to ensure that tenants can
// scan range descriptors iff they correspond to tenant owned ranges.
func TestScanRangeDescriptors(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
tc, tenant2, iteratorFactory := setup(t, ctx)
defer tc.Stopper().Stop(ctx)

// Split some ranges within tenant2 that we'll scan over.
ten2Codec := keys.MakeSQLCodec(ten2ID)
ten2Codec := tenant2.Codec()
ten2Split1 := append(ten2Codec.TenantPrefix(), 'a')
ten2Split2 := append(ten2Codec.TenantPrefix(), 'b')
{
Expand All @@ -56,7 +64,6 @@ func TestScanRangeDescriptors(t *testing.T) {
tc.SplitRangeOrFatal(t, ten2Codec.TenantPrefix().PrefixEnd()) // Last range
}

iteratorFactory := tenant2.RangeDescIteratorFactory().(rangedesc.IteratorFactory)
iter, err := iteratorFactory.NewIterator(ctx, ten2Codec.TenantSpan())
require.NoError(t, err)

Expand Down Expand Up @@ -112,3 +119,14 @@ func TestScanRangeDescriptors(t *testing.T) {
rangeDescs[numRanges-1].StartKey,
)
}

func TestScanRangeDescriptorsOutsideTenantKeyspace(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
tc, _, iteratorFactory := setup(t, ctx)
defer tc.Stopper().Stop(ctx)

_, err := iteratorFactory.NewIterator(ctx, keys.EverythingSpan)
require.ErrorContains(t, err, "requested key span /M{in-ax} not fully contained in tenant keyspace /Tenant/{2-3}")
}

0 comments on commit f3b8baf

Please sign in to comment.