Skip to content

Commit

Permalink
This commit provides a new implementation of systemStatusServer.SpanS…
Browse files Browse the repository at this point in the history
…tats,

proposed in #84105. The goal is to provide accurate table/index stats since
the assumption that 1 range can contain at most 1 table/index no longer holds.

Part of: https://cockroachlabs.atlassian.net/browse/CRDB-22711
Release note: None

add interface and abstract away access to KV

rename accessor implementation

dial nodes that have replicas

fix tests

remove newlines

add Accessor implementation for secondary tenants

revert ui changes
  • Loading branch information
Zach Lite committed Feb 7, 2023
1 parent 0723521 commit 3baa165
Show file tree
Hide file tree
Showing 28 changed files with 677 additions and 241 deletions.
2 changes: 1 addition & 1 deletion docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -2936,7 +2936,7 @@ Support status: [reserved](#support-status)

| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [string](#cockroach.server.serverpb.SpanStatsRequest-string) | | | [reserved](#support-status) |
| node_id | [int32](#cockroach.server.serverpb.SpanStatsRequest-int32) | | | [reserved](#support-status) |
| start_key | [bytes](#cockroach.server.serverpb.SpanStatsRequest-bytes) | | | [reserved](#support-status) |
| end_key | [bytes](#cockroach.server.serverpb.SpanStatsRequest-bytes) | | | [reserved](#support-status) |

Expand Down
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1159,6 +1159,8 @@ GO_TARGETS = [
"//pkg/kv/kvclient/rangefeed:rangefeed",
"//pkg/kv/kvclient/rangefeed:rangefeed_test",
"//pkg/kv/kvclient/rangestats:rangestats",
"//pkg/kv/kvclient/spanstats/spanstatsaccessor:spanstatsaccessor",
"//pkg/kv/kvclient/spanstats:spanstats",
"//pkg/kv/kvclient:kvclient",
"//pkg/kv/kvnemesis/kvnemesisutil:kvnemesisutil",
"//pkg/kv/kvnemesis:kvnemesis",
Expand Down Expand Up @@ -2581,6 +2583,8 @@ GET_X_DATA_TARGETS = [
"//pkg/kv/kvclient/rangefeed/rangefeedbuffer:get_x_data",
"//pkg/kv/kvclient/rangefeed/rangefeedcache:get_x_data",
"//pkg/kv/kvclient/rangestats:get_x_data",
"//pkg/kv/kvclient/spanstats:get_x_data",
"//pkg/kv/kvclient/spanstats/spanstatsaccessor:get_x_data",
"//pkg/kv/kvnemesis:get_x_data",
"//pkg/kv/kvnemesis/kvnemesisutil:get_x_data",
"//pkg/kv/kvprober:get_x_data",
Expand Down
22 changes: 22 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -648,6 +648,28 @@ func (c *Connector) SpanConfigConformance(
return report, nil
}

func (c *Connector) SpanStats(
ctx context.Context,
startKey,
endKey roachpb.Key,
nodeID roachpb.NodeID,
) (*serverpb.InternalSpanStatsResponse, error) {
var response *serverpb.InternalSpanStatsResponse
err := c.withClient(ctx, func(ctx context.Context, c *client) error {
stats, err := c.SpanStats(ctx, &serverpb.SpanStatsRequest{
NodeID: nodeID,
StartKey: roachpb.RKey(startKey),
EndKey: roachpb.RKey(endKey),
})
if err != nil {
return err
}
response = (*serverpb.InternalSpanStatsResponse)(stats)
return nil
})
return response, err
}

// GetAllSystemSpanConfigsThatApply implements the spanconfig.KVAccessor
// interface.
func (c *Connector) GetAllSystemSpanConfigsThatApply(
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/kvtenant/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
"//pkg/keys",
"//pkg/kv/kvclient/kvcoord",
"//pkg/kv/kvclient/rangecache",
"//pkg/kv/kvclient/spanstats",
"//pkg/roachpb",
"//pkg/rpc",
"//pkg/rpc/nodedialer",
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvclient/kvtenant/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ package kvtenant

import (
"context"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/spanstats"
"net"

"github.com/cockroachdb/cockroach/pkg/config"
Expand Down Expand Up @@ -91,6 +92,9 @@ type Connector interface {
// applicable to secondary tenants.
spanconfig.KVAccessor

// Accessor provides access to span stats.
spanstats.Accessor

// Reporter provides access to conformance reports, i.e. whether ranges
// backing queried keyspans conform the span configs that apply to them.
spanconfig.Reporter
Expand Down
15 changes: 15 additions & 0 deletions pkg/kv/kvclient/spanstats/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "spanstats",
srcs = ["span_stats.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient/spanstats",
visibility = ["//visibility:public"],
deps = [
"//pkg/roachpb",
"//pkg/server/serverpb",
],
)

get_x_data(name = "get_x_data")
33 changes: 33 additions & 0 deletions pkg/kv/kvclient/spanstats/span_stats.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// Copyright 2023 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 spanstats

import (
"context"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
)

// Accessor mediates access to SpanStats by consumers across the KV/Tenant
// boundary.
type Accessor interface {
// SpanStats returns the span stats between startKey and endKey.
// The caller can request span stats from across the cluster by setting
// nodeId to 0. The caller can request span stats from a specific node by
// setting the value of nodeID accordingly.
SpanStats(
ctx context.Context,
startKey,
endKey roachpb.Key,
nodeID roachpb.NodeID,
) (*serverpb.InternalSpanStatsResponse, error)
}
15 changes: 15 additions & 0 deletions pkg/kv/kvclient/spanstats/spanstatsaccessor/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "spanstatsaccessor",
srcs = ["accessor.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient/spanstats/spanstatsaccessor",
visibility = ["//visibility:public"],
deps = [
"//pkg/roachpb",
"//pkg/server/serverpb",
],
)

get_x_data(name = "get_x_data")
41 changes: 41 additions & 0 deletions pkg/kv/kvclient/spanstats/spanstatsaccessor/accessor.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
package spanstatsaccessor

import (
"context"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
)

// LocalAccessor is an implementation of spanstats.Accessor that is meant
// to provide access to span stats to servers that are co-located on the KV
// node.
type LocalAccessor struct {
spanStatsServer serverpb.SpanStatsServer
}

// New returns a new instance of AccessorImpl.
func New(server serverpb.SpanStatsServer) *LocalAccessor {
return &LocalAccessor{spanStatsServer: server}
}

// SpanStats implements the spanstats.Accessor interface.
func (a *LocalAccessor) SpanStats(
ctx context.Context, startKey,
endKey roachpb.Key, nodeID roachpb.NodeID,
) (*serverpb.InternalSpanStatsResponse, error) {
res, err := a.spanStatsServer.GetSpanStats(ctx,
&serverpb.InternalSpanStatsRequest{
Span: roachpb.Span{
Key: startKey,
EndKey: endKey,
},
NodeID: nodeID,
})

if err != nil {
return nil, err
}

return res, nil
}
1 change: 0 additions & 1 deletion pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,6 @@ go_test(
"client_spanconfigs_test.go",
"client_split_burst_test.go",
"client_split_test.go",
"client_status_test.go",
"client_store_test.go",
"client_tenant_test.go",
"client_test.go",
Expand Down
28 changes: 0 additions & 28 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
Expand Down Expand Up @@ -3138,33 +3137,6 @@ func mapToHotReplicasInfo(repls []CandidateReplica) []HotReplicaInfo {
return hotRepls
}

// StoreKeySpanStats carries the result of a stats computation over a key range.
type StoreKeySpanStats struct {
ReplicaCount int
MVCC enginepb.MVCCStats
ApproximateDiskBytes uint64
}

// ComputeStatsForKeySpan computes the aggregated MVCCStats for all replicas on
// this store which contain any keys in the supplied range.
func (s *Store) ComputeStatsForKeySpan(startKey, endKey roachpb.RKey) (StoreKeySpanStats, error) {
var result StoreKeySpanStats

newStoreReplicaVisitor(s).UndefinedOrder().Visit(func(repl *Replica) bool {
desc := repl.Desc()
if bytes.Compare(startKey, desc.EndKey) >= 0 || bytes.Compare(desc.StartKey, endKey) >= 0 {
return true // continue
}
result.MVCC.Add(repl.GetMVCCStats())
result.ReplicaCount++
return true
})

var err error
result.ApproximateDiskBytes, err = s.engine.ApproximateDiskBytes(startKey.AsRawKey(), endKey.AsRawKey())
return result, err
}

// ReplicateQueueDryRun runs the given replica through the replicate queue
// (using the allocator) without actually carrying out any changes, returning
// all trace messages collected along the way.
Expand Down
3 changes: 3 additions & 0 deletions pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,9 @@ func (a tenantAuthorizer) authorize(
case "/cockroach.server.serverpb.Status/TransactionContentionEvents":
return a.authTenant(tenID)

case "/cockroach.server.serverpb.Status/SpanStats":
return a.authTenant(tenID)

case "/cockroach.roachpb.Internal/GetSpanConfigs":
return a.authGetSpanConfigs(tenID, req.(*roachpb.GetSpanConfigsRequest))

Expand Down
4 changes: 4 additions & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ go_library(
"init.go",
"init_handshake.go",
"initial_sql.go",
"key_visualizer_server.go",
"listen_and_update_addrs.go",
"load_endpoint.go",
"loss_of_quorum.go",
Expand Down Expand Up @@ -113,6 +114,8 @@ go_library(
"//pkg/kv/kvclient/kvtenant",
"//pkg/kv/kvclient/rangefeed",
"//pkg/kv/kvclient/rangestats",
"//pkg/kv/kvclient/spanstats",
"//pkg/kv/kvclient/spanstats/spanstatsaccessor",
"//pkg/kv/kvprober",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/allocator/allocatorimpl",
Expand Down Expand Up @@ -410,6 +413,7 @@ go_test(
"server_test.go",
"settings_cache_test.go",
"settings_test.go",
"span_stats_test.go",
"statements_test.go",
"stats_test.go",
"status_ext_test.go",
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -1360,7 +1360,7 @@ func (s *adminServer) statsForSpan(
req := serverpb.SpanStatsRequest{
StartKey: rSpan.Key,
EndKey: rSpan.EndKey,
NodeID: nodeID.String(),
NodeID: nodeID,
}
spanResponse, err = client.SpanStats(ctx, &req)
}
Expand Down
Loading

0 comments on commit 3baa165

Please sign in to comment.