Skip to content

Commit

Permalink
Merge #93127
Browse files Browse the repository at this point in the history
93127: multitenant: handle missing NodeDescriptor in crdb_internal.ranges_no_leases r=arulajmani a=ecwall

Fixes #92915

This change matches the previous behavior of using "" for locality if the NodeDescriptor is not found instead of returning an error when generating crdb_internal.ranges_no_leases.

Release note: None

Co-authored-by: Evan Wall <wall@cockroachlabs.com>
  • Loading branch information
craig[bot] and ecwall committed Dec 8, 2022
2 parents 223f580 + c9f9198 commit 51f951c
Show file tree
Hide file tree
Showing 12 changed files with 67 additions and 13 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/errorutil",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -496,11 +497,11 @@ func (s mockNodeStore) GetNodeDescriptor(id roachpb.NodeID) (*roachpb.NodeDescri
return desc, nil
}
}
return nil, errors.Errorf("unable to look up descriptor for n%d", id)
return nil, errorutil.NewNodeNotFoundError(id)
}

func (s mockNodeStore) GetStoreDescriptor(id roachpb.StoreID) (*roachpb.StoreDescriptor, error) {
return nil, errors.Errorf("unable to look up descriptor for store ID %d", id)
return nil, errorutil.NewStoreNotFoundError(id)
}

// TestOracle tests the Oracle exposed by this package.
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/settings",
"//pkg/spanconfig",
"//pkg/util/contextutil",
"//pkg/util/errorutil",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
"//pkg/util/log",
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -334,7 +335,7 @@ func (c *Connector) GetNodeDescriptor(nodeID roachpb.NodeID) (*roachpb.NodeDescr
defer c.mu.RUnlock()
desc, ok := c.mu.nodeDescs[nodeID]
if !ok {
return nil, errors.Errorf("unable to look up descriptor for n%d", nodeID)
return nil, errorutil.NewNodeNotFoundError(nodeID)
}
return desc, nil
}
Expand All @@ -345,7 +346,7 @@ func (c *Connector) GetStoreDescriptor(storeID roachpb.StoreID) (*roachpb.StoreD
defer c.mu.RUnlock()
desc, ok := c.mu.storeDescs[storeID]
if !ok {
return nil, errors.Errorf("unable to look up descriptor for store ID %d", storeID)
return nil, errorutil.NewStoreNotFoundError(storeID)
}
return desc, nil
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -568,7 +568,7 @@ func (g *Gossip) GetStoreDescriptor(storeID roachpb.StoreID) (*roachpb.StoreDesc
desc := (*roachpb.StoreDescriptor)(value)
return desc, nil
}
return nil, errors.Errorf("unable to look up descriptor for store ID %d", storeID)
return nil, roachpb.NewStoreNotFoundError(storeID)
}

// LogStatus logs the current status of gossip such as the incoming and
Expand Down Expand Up @@ -997,7 +997,7 @@ func (g *Gossip) getNodeDescriptor(
return nodeDescriptor, nil
}

return nil, errors.Errorf("unable to look up descriptor for n%d", nodeID)
return nil, errorutil.NewNodeNotFoundError(nodeID)
}

// getNodeIDAddress looks up the address of the node by ID. The method accepts a
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/kvcoord/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,7 @@ go_test(
"//pkg/util",
"//pkg/util/caller",
"//pkg/util/ctxgroup",
"//pkg/util/errorutil",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
"//pkg/util/leaktest",
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvclient/kvcoord/replica_slice_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/shuffle"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

Expand All @@ -40,14 +40,14 @@ func (ns *mockNodeStore) GetNodeDescriptor(nodeID roachpb.NodeID) (*roachpb.Node
return &nd, nil
}
}
return nil, errors.Errorf("unable to look up descriptor for n%d", nodeID)
return nil, errorutil.NewNodeNotFoundError(nodeID)
}

// GetStoreDescriptor is part of the NodeDescStore interface.
func (ns *mockNodeStore) GetStoreDescriptor(
storeID roachpb.StoreID,
) (*roachpb.StoreDescriptor, error) {
return nil, errors.Errorf("unable to look up descriptor for store ID %d", storeID)
return nil, errorutil.NewStoreNotFoundError(storeID)
}

func TestNewReplicaSlice(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -414,6 +414,7 @@ go_test(
"//pkg/util/contextutil",
"//pkg/util/ctxgroup",
"//pkg/util/encoding",
"//pkg/util/errorutil",
"//pkg/util/hlc",
"//pkg/util/humanizeutil",
"//pkg/util/leaktest",
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/store_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -141,7 +142,7 @@ func (m mockNodeStore) GetNodeDescriptor(id roachpb.NodeID) (*roachpb.NodeDescri
}

func (m mockNodeStore) GetStoreDescriptor(id roachpb.StoreID) (*roachpb.StoreDescriptor, error) {
return nil, errors.Errorf("unable to look up descriptor for store ID %d", id)
return nil, errorutil.NewStoreNotFoundError(id)
}

type dummyFirstRangeProvider struct {
Expand Down
13 changes: 10 additions & 3 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -3713,12 +3713,19 @@ CREATE TABLE crdb_internal.ranges_no_leases (

replicaLocalityArr := tree.NewDArray(types.String)
for _, replica := range votersAndNonVoters {
// The table should still be rendered even if node locality is unavailable,
// so use NULL if nodeDesc is not found.
// See https://github.com/cockroachdb/cockroach/issues/92915.
replicaLocalityDatum := tree.DNull
nodeDesc, err := p.ExecCfg().NodeDescs.GetNodeDescriptor(replica.NodeID)
if err != nil {
return nil, err
if !errorutil.IsDescriptorNotFoundError(err) {
return nil, err
}
} else {
replicaLocalityDatum = tree.NewDString(nodeDesc.Locality.String())
}
replicaLocality := tree.NewDString(nodeDesc.Locality.String())
if err := replicaLocalityArr.Append(replicaLocality); err != nil {
if err := replicaLocalityArr.Append(replicaLocalityDatum); err != nil {
return nil, err
}
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/util/errorutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,15 @@ go_library(
name = "errorutil",
srcs = [
"catch.go",
"descriptor.go",
"error.go",
"tenant.go",
"tenant_deprecated_wrapper.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/util/errorutil",
visibility = ["//visibility:public"],
deps = [
"//pkg/roachpb",
"//pkg/settings",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/log/logcrash",
Expand Down
38 changes: 38 additions & 0 deletions pkg/util/errorutil/descriptor.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package errorutil

import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/errors"
)

type descriptorNotFound struct {
msg string
}

func (e *descriptorNotFound) Error() string {
return e.msg
}

func IsDescriptorNotFoundError(err error) bool {
return errors.HasType(err, (*descriptorNotFound)(nil))
}

func NewNodeNotFoundError(nodeID roachpb.NodeID) error {
return &descriptorNotFound{fmt.Sprintf("unable to look up descriptor for n%d", nodeID)}
}

func NewStoreNotFoundError(storeID roachpb.StoreID) error {
return &descriptorNotFound{fmt.Sprintf("unable to look up descriptor for store ID %d", storeID)}
}

0 comments on commit 51f951c

Please sign in to comment.