Skip to content

Commit

Permalink
Merge #98077 #98815
Browse files Browse the repository at this point in the history
98077: ts: add support for multitenant timeseries metrics r=aadityasondhi a=aadityasondhi

This patch allows tsdb to support tenant-sperated timeseries metrics.
The changes can be summarized as:

(1) Changes in the write path:
In `pkg/status` we now store metrics corresponding to non-system tenants
with their tenant IDs.  This makes the `source` component of the ts
key includes both node IDs and tenant IDs:
```
.../[NodeID]-[TenantID]
```
Note: the system tenant does not include the tenant ID to allow for
backwards compatibility.

(2) Changes in the query path:
(2.1) as system tenant:
If the request to ts server comes annotated as system tenant, we
aggregate timeseries across all tenants in that metric name's keyspace.

(2.2) as secondary tenant:
When a secondary tenant queries the server, it is routed through tenant
connect and checked for having tenant capabilites for viewing tsdb data.
During this stage that request is annotated with the Tenant ID. When the
system tenant ts server recevies this query with a Tenant ID set:
- If no sources are requested as part of the query, it will aggregate
  across all sources that match the tenant ID.
- If specific sources are specified along with the tenant ID, it will
  scan the keyspace for the NodeID and then filter out results that do
not match with the TenantID.

These changes ensure that the system tenant is able to have a view of
metrics across all tenants and application tenants have access to their
own metrics. These changes are all done entirely server-side so no
client changes are needed. The client will be automatically served the
timeseries data it has access to based on its tenant capability.

Fixes: #96438.

Release note (ui change): Timeseries metrics in db concole will show
tenant-specific metrics. For the system tenant, these timeseries will be
aggregated across all tenants. For a secondary tenant, only metrics
belonging to that particular tenant will be shown.

98815: server, ui: add sort and limit to sql activity pages, switch sql activity pages to read only from system table r=maryliag a=xinhaoz

See individual commits.

DB-Console: https://www.loom.com/share/73248244baa54cdf80389d0eec788447
DB-Console Details pages: https://www.loom.com/share/2d74c6cf082f4dcfb5e7f690166c2baf

Co-authored-by: Aaditya Sondhi <20070511+aadityasondhi@users.noreply.github.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: maryliag <marylia@cockroachlabs.com>
  • Loading branch information
4 people committed Mar 21, 2023
3 parents b806772 + d6fb82d + ae9c259 commit b89fa2c
Show file tree
Hide file tree
Showing 83 changed files with 2,207 additions and 590 deletions.
20 changes: 20 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -4098,6 +4098,8 @@ tenant pods.
| last_reset | [google.protobuf.Timestamp](#cockroach.server.serverpb.StatementsResponse-google.protobuf.Timestamp) | | Timestamp of the last stats reset. | [reserved](#support-status) |
| internal_app_name_prefix | [string](#cockroach.server.serverpb.StatementsResponse-string) | | If set and non-empty, indicates the prefix to application_name used for statements/queries issued internally by CockroachDB. | [reserved](#support-status) |
| transactions | [StatementsResponse.ExtendedCollectedTransactionStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedCollectedTransactionStatistics) | repeated | Transactions is transaction-level statistics for the collection of statements in this response. | [reserved](#support-status) |
| stmts_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) |
| txns_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) |



Expand Down Expand Up @@ -4170,12 +4172,28 @@ Support status: [reserved](#support-status)
| ----- | ---- | ----- | ----------- | -------------- |
| start | [int64](#cockroach.server.serverpb.CombinedStatementsStatsRequest-int64) | | Unix time range for aggregated statements. | [reserved](#support-status) |
| end | [int64](#cockroach.server.serverpb.CombinedStatementsStatsRequest-int64) | | | [reserved](#support-status) |
| fetch_mode | [CombinedStatementsStatsRequest.FetchMode](#cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.CombinedStatementsStatsRequest.FetchMode) | | Note that if fetch_mode is set to transactions only, we will also include the statement statistics for the stmts in the transactions response. This is more of a hack-y method to get the complete stats for txns, because in the client we need to fill in some txn stats info from its stmt stats, such as the query string.<br><br>We prefer this hackier method right now to reduce surface area for backporting these changes, but in the future we will introduce more endpoints to properly organize these differing requests. TODO (xinhaoz) - Split this API into stmts and txns properly instead of using this param. | [reserved](#support-status) |
| limit | [int64](#cockroach.server.serverpb.CombinedStatementsStatsRequest-int64) | | | [reserved](#support-status) |






<a name="cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.CombinedStatementsStatsRequest.FetchMode"></a>
#### CombinedStatementsStatsRequest.FetchMode



| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| stats_type | [CombinedStatementsStatsRequest.StatsType](#cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.CombinedStatementsStatsRequest.StatsType) | | | [reserved](#support-status) |
| sort | [StatsSortOptions](#cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.StatsSortOptions) | | | [reserved](#support-status) |






#### Response Parameters

Expand All @@ -4191,6 +4209,8 @@ Support status: [reserved](#support-status)
| last_reset | [google.protobuf.Timestamp](#cockroach.server.serverpb.StatementsResponse-google.protobuf.Timestamp) | | Timestamp of the last stats reset. | [reserved](#support-status) |
| internal_app_name_prefix | [string](#cockroach.server.serverpb.StatementsResponse-string) | | If set and non-empty, indicates the prefix to application_name used for statements/queries issued internally by CockroachDB. | [reserved](#support-status) |
| transactions | [StatementsResponse.ExtendedCollectedTransactionStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedCollectedTransactionStatistics) | repeated | Transactions is transaction-level statistics for the collection of statements in this response. | [reserved](#support-status) |
| stmts_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) |
| txns_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) |



Expand Down
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2060,6 +2060,7 @@ GO_TARGETS = [
"//pkg/ts/testmodel:testmodel",
"//pkg/ts/testmodel:testmodel_test",
"//pkg/ts/tspb:tspb",
"//pkg/ts/tsutil:tsutil",
"//pkg/ts:ts",
"//pkg/ts:ts_test",
"//pkg/ui:ui",
Expand Down Expand Up @@ -3194,6 +3195,7 @@ GET_X_DATA_TARGETS = [
"//pkg/ts/catalog:get_x_data",
"//pkg/ts/testmodel:get_x_data",
"//pkg/ts/tspb:get_x_data",
"//pkg/ts/tsutil:get_x_data",
"//pkg/ui:get_x_data",
"//pkg/upgrade:get_x_data",
"//pkg/upgrade/migrationstable:get_x_data",
Expand Down
53 changes: 16 additions & 37 deletions pkg/ccl/serverccl/statusccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -343,11 +343,9 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
require.NoError(t, err)

request := &serverpb.StatementsRequest{}
combinedStatsRequest := &serverpb.CombinedStatementsStatsRequest{}
var tenantStats *serverpb.StatementsResponse
var tenantCombinedStats *serverpb.StatementsResponse

// Populate `tenantStats` and `tenantCombinedStats`. The tenant server
// Populate `tenantStats`. The tenant server
// `Statements` and `CombinedStatements` methods are backed by the
// sqlinstance system which uses a cache populated through rangefeed
// for keeping track of SQL pod data. We use `SucceedsSoon` to eliminate
Expand All @@ -362,10 +360,6 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
return errors.New("tenant statements are unexpectedly empty")
}

tenantCombinedStats, err = tenantStatusServer.CombinedStatementStats(ctx, combinedStatsRequest)
if tenantCombinedStats == nil || len(tenantCombinedStats.Statements) == 0 {
return errors.New("tenant combined statements are unexpectedly empty")
}
return nil
})

Expand All @@ -374,11 +368,6 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
err = serverutils.GetJSONProto(nonTenant, path, &nonTenantStats)
require.NoError(t, err)

path = "/_status/combinedstmts"
var nonTenantCombinedStats serverpb.StatementsResponse
err = serverutils.GetJSONProto(nonTenant, path, &nonTenantCombinedStats)
require.NoError(t, err)

checkStatements := func(t *testing.T, tc []testCase, actual *serverpb.StatementsResponse) {
t.Helper()
var expectedStatements []string
Expand Down Expand Up @@ -414,13 +403,11 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
// First we verify that we have expected stats from tenants.
t.Run("tenant-stats", func(t *testing.T) {
checkStatements(t, testCaseTenant, tenantStats)
checkStatements(t, testCaseTenant, tenantCombinedStats)
})

// Now we verify the non tenant stats are what we expected.
t.Run("non-tenant-stats", func(t *testing.T) {
checkStatements(t, testCaseNonTenant, &nonTenantStats)
checkStatements(t, testCaseNonTenant, &nonTenantCombinedStats)
})

// Now we verify that tenant and non-tenant have no visibility into each other's stats.
Expand All @@ -437,17 +424,6 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
}
}

for _, tenantStmt := range tenantCombinedStats.Statements {
for _, nonTenantStmt := range nonTenantCombinedStats.Statements {
require.NotEqual(t, tenantStmt, nonTenantStmt, "expected tenant to have no visibility to non-tenant's statement stats, but found:", nonTenantStmt)
}
}

for _, tenantTxn := range tenantCombinedStats.Transactions {
for _, nonTenantTxn := range nonTenantCombinedStats.Transactions {
require.NotEqual(t, tenantTxn, nonTenantTxn, "expected tenant to have no visibility to non-tenant's transaction stats, but found:", nonTenantTxn)
}
}
})
}

Expand All @@ -463,43 +439,46 @@ func testResetSQLStatsRPCForTenant(
testCluster := testHelper.TestCluster()
controlCluster := testHelper.ControlCluster()

// Disable automatic flush to ensure tests are deterministic.
// Set automatic flush to some long duration we'll never hit to
// ensure tests are deterministic.
testCluster.TenantConn(0 /* idx */).
Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = false")
Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '24h'")
controlCluster.TenantConn(0 /* idx */).
Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = false")
Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '24h'")

defer func() {
// Cleanup
testCluster.TenantConn(0 /* idx */).
Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = true")
Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '10m'")
controlCluster.TenantConn(0 /* idx */).
Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = true")
Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '10m'")

}()

for _, flushed := range []bool{false, true} {
testTenant := testCluster.Tenant(serverccl.RandomServer)
testTenantConn := testTenant.GetTenantConn()
t.Run(fmt.Sprintf("flushed=%t", flushed), func(t *testing.T) {
// Clears the SQL Stats at the end of each test via builtin.
defer func() {
testCluster.TenantConn(serverccl.RandomServer).Exec(t, "SELECT crdb_internal.reset_sql_stats()")
testTenantConn.Exec(t, "SELECT crdb_internal.reset_sql_stats()")
controlCluster.TenantConn(serverccl.RandomServer).Exec(t, "SELECT crdb_internal.reset_sql_stats()")
}()

for _, stmt := range stmts {
testCluster.TenantConn(serverccl.RandomServer).Exec(t, stmt)
testTenantConn.Exec(t, stmt)
controlCluster.TenantConn(serverccl.RandomServer).Exec(t, stmt)
}

if flushed {
testCluster.TenantSQLStats(serverccl.RandomServer).Flush(ctx)
testTenant.TenantSQLStats().Flush(ctx)
controlCluster.TenantSQLStats(serverccl.RandomServer).Flush(ctx)
}

status := testCluster.TenantStatusSrv(serverccl.RandomServer)
status := testTenant.TenantStatusSrv()

statsPreReset, err := status.Statements(ctx, &serverpb.StatementsRequest{
Combined: true,
Combined: flushed,
})
require.NoError(t, err)

Expand All @@ -513,7 +492,7 @@ func testResetSQLStatsRPCForTenant(
require.NoError(t, err)

statsPostReset, err := status.Statements(ctx, &serverpb.StatementsRequest{
Combined: true,
Combined: flushed,
})
require.NoError(t, err)

Expand All @@ -538,7 +517,7 @@ func testResetSQLStatsRPCForTenant(
// Ensures that sql stats reset is isolated by tenant boundary.
statsFromControlCluster, err :=
controlCluster.TenantStatusSrv(serverccl.RandomServer).Statements(ctx, &serverpb.StatementsRequest{
Combined: true,
Combined: flushed,
})
require.NoError(t, err)

Expand Down
1 change: 1 addition & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,7 @@ go_library(
"//pkg/testutils/serverutils",
"//pkg/ts",
"//pkg/ts/tspb",
"//pkg/ts/tsutil",
"//pkg/upgrade/upgrades",
"//pkg/util",
"//pkg/util/cgroups",
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/decode_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ func TestTryDecodeValue(t *testing.T) {
s: `\u0008\u0080ì¿ùÛ\u008bù\u0083\u0017\u0010\u0080¬¢ÿ¾ôù\u0083\u0017\u001a \n\u0018cr.node.sql.select.count\u0010\u0001\u0018\u0002 \u0002\u001a \n\u0018cr.node.sql.update.count\u0010\u0001\u0018\u0002 \u0002\u001a \n\u0018cr.node.sql.insert.count\u0010\u0001\u0018\u0002 \u0002\u001a \n\u0018cr.node.sql.delete.count\u0010\u0001\u0018\u0002 \u0002\u001a*\n\u001fcr.node.sql.service.latency-p99\u0010\u0003\u0018\u0002 \u0000*\u00011\u001a3\n+cr.node.sql.distsql.contended_queries.count\u0010\u0001\u0018\u0002 \u0002\u001a\u001c\n\u0011cr.store.replicas\u0010\u0001\u0018\u0002 \u0000*\u00011\u001a\u0019\n\u0011cr.store.capacity\u0010\u0001\u0018\u0002 \u0000\u001a#\n\u001bcr.store.capacity.available\u0010\u0001\u0018\u0002 \u0000\u001a\u001e\n\u0016cr.store.capacity.used\u0010\u0001\u0018\u0002 \u0000 \u0080Ø\u008eáo`,
wantOK: true,
protoType: "cockroach.ts.tspb.TimeSeriesQueryRequest",
wantVal: `{"endNanos": "1659549679000000000", "queries": [{"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.select.count", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.update.count", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.insert.count", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.delete.count", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NONE", "downsampler": "MAX", "name": "cr.node.sql.service.latency-p99", "sourceAggregator": "SUM", "sources": ["1"]}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.distsql.contended_queries.count", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.replicas", "sourceAggregator": "SUM", "sources": ["1"]}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.capacity", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.capacity.available", "sourceAggregator": "SUM", "sources": []}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.capacity.used", "sourceAggregator": "SUM", "sources": []}], "sampleNanos": "30000000000", "startNanos": "1659546079000000000"}`,
wantVal: `{"endNanos": "1659549679000000000", "queries": [{"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.select.count", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.update.count", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.insert.count", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.delete.count", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NONE", "downsampler": "MAX", "name": "cr.node.sql.service.latency-p99", "sourceAggregator": "SUM", "sources": ["1"], "tenantId": {"id": "0"}}, {"derivative": "NON_NEGATIVE_DERIVATIVE", "downsampler": "AVG", "name": "cr.node.sql.distsql.contended_queries.count", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.replicas", "sourceAggregator": "SUM", "sources": ["1"], "tenantId": {"id": "0"}}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.capacity", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.capacity.available", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}, {"derivative": "NONE", "downsampler": "AVG", "name": "cr.store.capacity.used", "sourceAggregator": "SUM", "sources": [], "tenantId": {"id": "0"}}], "sampleNanos": "30000000000", "startNanos": "1659546079000000000"}`,
},
}
for _, tt := range tests {
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/tsdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/cli/clierrorplus"
"github.com/cockroachdb/cockroach/pkg/ts"
"github.com/cockroachdb/cockroach/pkg/ts/tspb"
"github.com/cockroachdb/cockroach/pkg/ts/tsutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
Expand Down Expand Up @@ -74,7 +74,7 @@ output.
// Buffer the writes to os.Stdout since we're going to
// be writing potentially a lot of data to it.
w := bufio.NewWriter(os.Stdout)
if err := ts.DumpRawTo(stream, w); err != nil {
if err := tsutil.DumpRawTo(stream, w); err != nil {
return err
}
return w.Flush()
Expand Down
2 changes: 2 additions & 0 deletions pkg/rpc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ go_library(
"//pkg/security/username",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/ts/tspb",
"//pkg/util",
"//pkg/util/buildutil",
"//pkg/util/contextutil",
Expand Down Expand Up @@ -116,6 +117,7 @@ go_test(
"//pkg/spanconfig",
"//pkg/testutils",
"//pkg/testutils/skip",
"//pkg/ts/tspb",
"//pkg/util",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
Expand Down
23 changes: 22 additions & 1 deletion pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/ts/tspb"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
Expand Down Expand Up @@ -155,7 +156,7 @@ func (a tenantAuthorizer) authorize(
return a.capabilitiesAuthorizer.HasNodeStatusCapability(ctx, tenID)

case "/cockroach.ts.tspb.TimeSeries/Query":
return a.capabilitiesAuthorizer.HasTSDBQueryCapability(ctx, tenID)
return a.authTSDBQuery(ctx, tenID, req.(*tspb.TimeSeriesQueryRequest))

default:
return authErrorf("unknown method %q", fullMethod)
Expand Down Expand Up @@ -379,6 +380,26 @@ func (a tenantAuthorizer) authSpanConfigConformance(
return nil
}

// authTSDBQuery authorizes the provided tenant to invoke the TSDB Query RPC
// with the provided args. A non-system tenant is only allowed to query its own
// time series.
func (a tenantAuthorizer) authTSDBQuery(
ctx context.Context, id roachpb.TenantID, request *tspb.TimeSeriesQueryRequest,
) error {
for _, query := range request.Queries {
if !query.TenantID.IsSet() {
return authError("tsdb query with unspecified tenant not permitted")
}
if !query.TenantID.Equal(id) {
return authErrorf("tsdb query with invalid tenant not permitted")
}
}
if err := a.capabilitiesAuthorizer.HasTSDBQueryCapability(ctx, id); err != nil {
return authError(err.Error())
}
return nil
}

// validateSpanConfigTarget validates that the tenant is authorized to interact
// with the supplied span config target. In particular, span targets must be
// wholly contained within the tenant keyspace and system span config targets
Expand Down
Loading

0 comments on commit b89fa2c

Please sign in to comment.