Skip to content

Commit

Permalink
sql: add crdb_internal.reset_sql_statistics() builtin
Browse files Browse the repository at this point in the history
Previously, there was no mechanism to immediately clear SQL statistics. Users
would have to wait until the reset interval expires. This commit creates a
builtin to immediately clears SQL stats.

Release note (sql change): SQL stats can now be cleared using
crdb_internal.reset_sql_statistics()
  • Loading branch information
Azhng committed Mar 18, 2021
1 parent 3267fc5 commit 6b178cd
Show file tree
Hide file tree
Showing 15 changed files with 1,209 additions and 507 deletions.
46 changes: 45 additions & 1 deletion docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -1369,7 +1369,7 @@ Support status: [reserved](#support-status)



Request object for issing a query cancel request.
Request object for issuing a query cancel request.


| Field | Type | Label | Description | Support status |
Expand Down Expand Up @@ -2834,6 +2834,50 @@ Support status: [reserved](#support-status)



## SQLStatisticsReset

`POST /_status/sqlstatisticsreset`



Support status: [reserved](#support-status)

#### Request Parameters




Request object for issuing a SQL stats reset request.


| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [string](#cockroach.server.serverpb.SQLStatisticsResetRequest-string) | | | [reserved](#support-status) |







#### Response Parameters




Response object returned by SQLStatisticsReset.


| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| num_of_nodes_resetted | [int32](#cockroach.server.serverpb.SQLStatisticsResetResponse-int32) | | Number of nodes that have resetted their SQL stats. | [reserved](#support-status) |







## Users

`GET /_admin/v1/users`
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2714,6 +2714,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr>
<tr><td><a name="crdb_internal.range_stats"></a><code>crdb_internal.range_stats(key: <a href="bytes.html">bytes</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>This function is used to retrieve range statistics information as a JSON object.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.reset_sql_statistics"></a><code>crdb_internal.reset_sql_statistics() &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function can be used to clear the collected SQL-related statistics.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.round_decimal_values"></a><code>crdb_internal.round_decimal_values(val: <a href="decimal.html">decimal</a>, scale: <a href="int.html">int</a>) &rarr; <a href="decimal.html">decimal</a></code></td><td><span class="funcdesc"><p>This function is used internally to round decimal values during mutations.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.round_decimal_values"></a><code>crdb_internal.round_decimal_values(val: <a href="decimal.html">decimal</a>[], scale: <a href="int.html">int</a>) &rarr; <a href="decimal.html">decimal</a>[]</code></td><td><span class="funcdesc"><p>This function is used internally to round decimal array values during mutations.</p>
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ go_library(
"server_systemlog_gc.go",
"settings_cache.go",
"settingsworker.go",
"sql_stats.go",
"statement_diagnostics_requests.go",
"statements.go",
"status.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/server/serverpb/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ type SQLStatusServer interface {
CancelSession(context.Context, *CancelSessionRequest) (*CancelSessionResponse, error)
ListContentionEvents(context.Context, *ListContentionEventsRequest) (*ListContentionEventsResponse, error)
ListLocalContentionEvents(context.Context, *ListContentionEventsRequest) (*ListContentionEventsResponse, error)
SQLStatisticsReset(ctx context.Context, request *SQLStatisticsResetRequest) (*SQLStatisticsResetResponse, error)
}

// OptionalNodesStatusServer is a StatusServer that is only optionally present
Expand Down
1,331 changes: 830 additions & 501 deletions pkg/server/serverpb/status.pb.go

Large diffs are not rendered by default.

78 changes: 78 additions & 0 deletions pkg/server/serverpb/status.pb.gw.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

20 changes: 19 additions & 1 deletion pkg/server/serverpb/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -600,7 +600,7 @@ message ListSessionsResponse {
repeated ListSessionsError errors = 2 [ (gogoproto.nullable) = false ];
}

// Request object for issing a query cancel request.
// Request object for issuing a query cancel request.
message CancelQueryRequest {
// ID of gateway node for the query to be canceled.
//
Expand Down Expand Up @@ -1010,6 +1010,17 @@ message JobStatusResponse {
cockroach.sql.jobs.jobspb.Job job = 1;
}

// Request object for issuing a SQL stats reset request.
message SQLStatisticsResetRequest {
string node_id = 1 [(gogoproto.customname) = "NodeID"];
}

// Response object returned by SQLStatisticsReset.
message SQLStatisticsResetResponse {
// Number of nodes that have resetted their SQL stats.
int32 num_of_nodes_resetted = 1;
}

service Status {
// Certificates retrieves a copy of the TLS certificates.
rpc Certificates(CertificatesRequest) returns (CertificatesResponse) {
Expand Down Expand Up @@ -1295,4 +1306,11 @@ service Status {
get : "/_status/job/{job_id}"
};
}

rpc SQLStatisticsReset(SQLStatisticsResetRequest) returns (SQLStatisticsResetResponse) {
option (google.api.http) = {
post: "/_status/sqlstatisticsreset"
body: "*"
};
}
}
78 changes: 78 additions & 0 deletions pkg/server/sql_stats.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
// Copyright 2021 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 server

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)

func (s *statusServer) SQLStatisticsReset(
ctx context.Context, req *serverpb.SQLStatisticsResetRequest,
) (*serverpb.SQLStatisticsResetResponse, error) {
ctx = propagateGatewayMetadata(ctx)
ctx = s.AnnotateCtx(ctx)

if _, err := s.privilegeChecker.requireAdminUser(ctx); err != nil {
return nil, err
}

response := &serverpb.SQLStatisticsResetResponse{}

localReq := &serverpb.SQLStatisticsResetRequest{
NodeID: "local",
}

if len(req.NodeID) > 0 {
requestedNodeID, local, err := s.parseNodeID(req.NodeID)
if err != nil {
return nil, status.Errorf(codes.InvalidArgument, err.Error())
}
if local {
s.admin.server.sqlServer.pgServer.SQLServer.ResetSQLStats(ctx)
response.NumOfNodesResetted = 1
return response, nil
}
status, err := s.dialNode(ctx, requestedNodeID)
if err != nil {
return nil, err
}
return status.SQLStatisticsReset(ctx, localReq)
}

dialFn := func(ctx context.Context, nodeID roachpb.NodeID) (interface{}, error) {
client, err := s.dialNode(ctx, nodeID)
return client, err
}

resetSQLStats := func(ctx context.Context, client interface{}, _ roachpb.NodeID) (interface{}, error) {
status := client.(serverpb.StatusClient)
return status.SQLStatisticsReset(ctx, localReq)
}

if err := s.iterateNodes(ctx, fmt.Sprintf("reset SQL statistics for node %s", req.NodeID),
dialFn,
resetSQLStats,
func(nodeID roachpb.NodeID, resp interface{}) {
response.NumOfNodesResetted++
},
func(nodeID roachpb.NodeID, err error) {
},
); err != nil {
return nil, err
}
return response, nil
}
Loading

0 comments on commit 6b178cd

Please sign in to comment.