Skip to content

Commit

Permalink
server: extend span statistics endpoint
Browse files Browse the repository at this point in the history
Extends: #96223

This PR extends the implementation of our SpanStats RPC endpoint to fetch stats
for multiple spans at once. By extending the endpoint, we amortize the cost of
the RPC's node fanout across all requested spans, whereas previously, we were
issuing a fanout per span requested.  Additionally, this change batches KV
layer requests for ranges fully contained by the span, instead of issuing a
request per fully contained range.

Release note (sql change): span statistics are unavailable on mixed-version
clusters
  • Loading branch information
Thomas Hardy committed Mar 13, 2023
1 parent 66bb3e9 commit 7109b18
Show file tree
Hide file tree
Showing 18 changed files with 871 additions and 157 deletions.
120 changes: 105 additions & 15 deletions pkg/ccl/serverccl/statusccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,9 +149,8 @@ func testTenantSpanStats(ctx context.Context, t *testing.T, helper serverccl.Ten

t.Run("test tenant permissioning", func(t *testing.T) {
req := roachpb.SpanStatsRequest{
NodeID: "0",
StartKey: roachpb.RKey(aSpan.Key),
EndKey: roachpb.RKey(aSpan.EndKey),
NodeID: "0",
Spans: []roachpb.Span{aSpan},
}
resp := roachpb.SpanStatsResponse{}

Expand All @@ -162,19 +161,44 @@ func testTenantSpanStats(ctx context.Context, t *testing.T, helper serverccl.Ten

adminClient := helper.TestCluster().TenantHTTPClient(t, 1, true)
adminClient.PostJSON("/_status/span", &req, &resp)
require.Greaterf(t, resp.RangeCount, int32(0), "postive range count")
require.Greaterf(t, resp.SpanToStats[aSpan.String()].RangeCount, int32(0), "positive range count")
})

t.Run("test tenant isolation", func(t *testing.T) {
_, err := tenantA.TenantStatusSrv().(serverpb.TenantStatusServer).SpanStats(ctx,
&roachpb.SpanStatsRequest{
NodeID: "0", // 0 indicates we want stats from all nodes.
StartKey: roachpb.RKey(bSpan.Key),
EndKey: roachpb.RKey(bSpan.EndKey),
NodeID: "0", // 0 indicates we want stats from all nodes.
Spans: []roachpb.Span{bSpan},
})
require.Error(t, err)
})

t.Run("test invalid request payload", func(t *testing.T) {
_, err := tenantA.TenantStatusSrv().(serverpb.TenantStatusServer).SpanStats(ctx,
&roachpb.SpanStatsRequest{
NodeID: "0", // 0 indicates we want stats from all nodes.
StartKey: roachpb.RKey(aSpan.Key),
EndKey: roachpb.RKey(aSpan.EndKey),
})
require.ErrorContains(t, err, `span stats request - unexpected populated legacy fields (StartKey, EndKey)`)
})

t.Run("test exceed span request limit", func(t *testing.T) {
// Set the span batch limit to 1.
_, err := helper.HostCluster().ServerConn(0).Exec(`SET CLUSTER SETTING server.span_stats.span_batch_limit = 1`)
require.NoError(t, err)
_, err = tenantA.TenantStatusSrv().(serverpb.TenantStatusServer).SpanStats(ctx,
&roachpb.SpanStatsRequest{
NodeID: "0", // 0 indicates we want stats from all nodes.
Spans: []roachpb.Span{aSpan, aSpan},
})
require.ErrorContains(t, err, `error getting span statistics - number of spans in request payload (2) exceeds`+
` 'server.span_stats.span_batch_limit' cluster setting limit (1)`)
// Reset the span batch limit to default.
_, err = helper.HostCluster().ServerConn(0).Exec(`SET CLUSTER SETTING server.span_stats.span_batch_limit = $1`, roachpb.DefaultSpanStatsSpanLimit)
require.NoError(t, err)
})

t.Run("test KV node fan-out", func(t *testing.T) {
_, tID, err := keys.DecodeTenantPrefix(aSpan.Key)
require.NoError(t, err)
Expand All @@ -186,9 +210,8 @@ func testTenantSpanStats(ctx context.Context, t *testing.T, helper serverccl.Ten

controlStats, err := tenantA.TenantStatusSrv().(serverpb.TenantStatusServer).SpanStats(ctx,
&roachpb.SpanStatsRequest{
NodeID: "0", // 0 indicates we want stats from all nodes.
StartKey: roachpb.RKey(aSpan.Key),
EndKey: roachpb.RKey(aSpan.EndKey),
NodeID: "0", // 0 indicates we want stats from all nodes.
Spans: []roachpb.Span{aSpan},
})
require.NoError(t, err)

Expand All @@ -212,14 +235,81 @@ func testTenantSpanStats(ctx context.Context, t *testing.T, helper serverccl.Ten

stats, err := tenantA.TenantStatusSrv().(serverpb.TenantStatusServer).SpanStats(ctx,
&roachpb.SpanStatsRequest{
NodeID: "0", // 0 indicates we want stats from all nodes.
StartKey: roachpb.RKey(aSpan.Key),
EndKey: roachpb.RKey(aSpan.EndKey),
NodeID: "0", // 0 indicates we want stats from all nodes.
Spans: []roachpb.Span{aSpan},
})

require.NoError(t, err)
require.Equal(t, controlStats.RangeCount+1, stats.RangeCount)
require.Equal(t, controlStats.TotalStats.LiveCount+int64(len(incKeys)), stats.TotalStats.LiveCount)

controlSpanStats := controlStats.SpanToStats[aSpan.String()]
testSpanStats := stats.SpanToStats[aSpan.String()]
require.Equal(t, controlSpanStats.RangeCount+1, testSpanStats.RangeCount)
require.Equal(t, controlSpanStats.TotalStats.LiveCount+int64(len(incKeys)), testSpanStats.TotalStats.LiveCount)

// Make a multi-span call
type spanCase struct {
span roachpb.Span
expectedRangeCount int32
expectedLiveCount int64
}
spanCases := []spanCase{
{
// "a", "b" - single range, single key
span: roachpb.Span{
Key: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[0])),
EndKey: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[1])),
},
expectedRangeCount: 1,
expectedLiveCount: 1,
},
{
// "d", "f" - single range, multiple keys
span: roachpb.Span{
Key: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[3])),
EndKey: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[5])),
},
expectedRangeCount: 1,
expectedLiveCount: 2,
},
{
// "bb", "e" - multiple ranges, multiple keys
span: roachpb.Span{
Key: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[2])),
EndKey: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[4])),
},
expectedRangeCount: 2,
expectedLiveCount: 2,
},

{
// "a", "d" - multiple ranges, multiple keys
span: roachpb.Span{
Key: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[0])),
EndKey: makeKey(keys.MakeTenantPrefix(tID), []byte(incKeys[3])),
},
expectedRangeCount: 2,
expectedLiveCount: 3,
},
}

var spans []roachpb.Span
for _, sc := range spanCases {
spans = append(spans, sc.span)
}

stats, err = tenantA.TenantStatusSrv().(serverpb.TenantStatusServer).SpanStats(ctx,
&roachpb.SpanStatsRequest{
NodeID: "0", // 0 indicates we want stats from all nodes.
Spans: spans,
})

require.NoError(t, err)
// Check each span has their expected values.
for _, sc := range spanCases {
spanStats := stats.SpanToStats[sc.span.String()]
require.Equal(t, spanStats.RangeCount, sc.expectedRangeCount, fmt.Sprintf("mismatch on expected range count for span case with span %v", sc.span.String()))
require.Equal(t, spanStats.TotalStats.LiveCount, sc.expectedLiveCount, fmt.Sprintf("mismatch on expected live count for span case with span %v", sc.span.String()))
}
})

}
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/registry/owners.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,4 +30,5 @@ const (
OwnerTestEng Owner = `test-eng`
OwnerDevInf Owner = `dev-inf`
OwnerMultiTenant Owner = `multi-tenant`
OwnerClusterObs Owner = `cluster-observability`
)
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ go_library(
"mixed_version_job_compatibility_in_declarative_schema_changer.go",
"mixed_version_jobs.go",
"mixed_version_schemachange.go",
"mixed_version_tenant_span_stats.go",
"multitenant.go",
"multitenant_distsql.go",
"multitenant_shared_process.go",
Expand Down
Loading

0 comments on commit 7109b18

Please sign in to comment.