diff --git a/pkg/ccl/serverccl/BUILD.bazel b/pkg/ccl/serverccl/BUILD.bazel index 874b2d505e45..3bf36d6e366d 100644 --- a/pkg/ccl/serverccl/BUILD.bazel +++ b/pkg/ccl/serverccl/BUILD.bazel @@ -33,6 +33,7 @@ go_test( "server_sql_test.go", "tenant_grpc_test.go", "tenant_status_test.go", + "tenant_vars_test.go", ], embed = [":serverccl"], deps = [ @@ -62,7 +63,10 @@ go_test( "//pkg/util/log", "//pkg/util/randutil", "//pkg/util/timeutil", + "@com_github_elastic_gosigar//:gosigar", "@com_github_lib_pq//:pq", + "@com_github_prometheus_client_model//go", + "@com_github_prometheus_common//expfmt", "@com_github_stretchr_testify//require", "@org_golang_x_crypto//bcrypt", ], diff --git a/pkg/ccl/serverccl/tenant_vars_test.go b/pkg/ccl/serverccl/tenant_vars_test.go new file mode 100644 index 000000000000..eb7bba78b632 --- /dev/null +++ b/pkg/ccl/serverccl/tenant_vars_test.go @@ -0,0 +1,110 @@ +// Copyright 2021 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package serverccl + +import ( + "context" + "crypto/tls" + "net/http" + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/elastic/gosigar" + io_prometheus_client "github.com/prometheus/client_model/go" + "github.com/prometheus/common/expfmt" + "github.com/stretchr/testify/require" +) + +func TestTenantVars(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + serverParams, _ := tests.CreateTestServerParams() + testCluster := serverutils.StartNewTestCluster(t, 1 /* numNodes */, base.TestClusterArgs{ + ServerArgs: serverParams, + }) + defer testCluster.Stopper().Stop(ctx) + + server := testCluster.Server(0 /* idx */) + + tenant, _ := serverutils.StartTenant(t, server, base.TestTenantArgs{ + TenantID: roachpb.MakeTenantID(10 /* id */), + }) + + url := "https://" + tenant.HTTPAddr() + "/_status/load" + client := http.Client{ + Transport: &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + }, + } + resp, err := client.Get(url) + require.NoError(t, err) + defer resp.Body.Close() + require.Equal(t, 200, resp.StatusCode, + "invalid non-200 status code %v from tenant", resp.StatusCode) + + var parser expfmt.TextParser + metrics, err := parser.TextToMetricFamilies(resp.Body) + require.NoError(t, err) + + userCPU, found := metrics["sys_cpu_user_ns"] + require.True(t, found) + require.Len(t, userCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, userCPU.GetType()) + cpuUserNS := userCPU.Metric[0].GetGauge().GetValue() + + sysCPU, found := metrics["sys_cpu_sys_ns"] + require.True(t, found) + require.True(t, found) + require.Len(t, sysCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, sysCPU.GetType()) + cpuSysNS := sysCPU.Metric[0].GetGauge().GetValue() + + // The values are between zero and whatever User/Sys time is observed after the get. + require.Positive(t, cpuUserNS) + require.Positive(t, cpuSysNS) + cpuTime := gosigar.ProcTime{} + require.NoError(t, cpuTime.Get(os.Getpid())) + require.LessOrEqual(t, cpuUserNS, float64(cpuTime.User)*1e6) + require.LessOrEqual(t, cpuSysNS, float64(cpuTime.Sys)*1e6) + + resp, err = client.Get(url) + require.NoError(t, err) + defer resp.Body.Close() + require.Equal(t, 200, resp.StatusCode, + "invalid non-200 status code %v from tenant", resp.StatusCode) + + metrics, err = parser.TextToMetricFamilies(resp.Body) + require.NoError(t, err) + + userCPU, found = metrics["sys_cpu_user_ns"] + require.True(t, found) + require.Len(t, userCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, userCPU.GetType()) + cpuUserNS2 := userCPU.Metric[0].GetGauge().GetValue() + + sysCPU, found = metrics["sys_cpu_sys_ns"] + require.True(t, found) + require.True(t, found) + require.Len(t, sysCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, sysCPU.GetType()) + cpuSysNS2 := sysCPU.Metric[0].GetGauge().GetValue() + + require.LessOrEqual(t, float64(cpuTime.User)*1e6, cpuUserNS2) + require.LessOrEqual(t, float64(cpuTime.Sys)*1e6, cpuSysNS2) +} diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 2e03724a0454..1553c9574dbe 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -203,6 +203,7 @@ go_library( "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_redact//:redact", "@com_github_cockroachdb_sentry_go//:sentry-go", + "@com_github_elastic_gosigar//:gosigar", "@com_github_gogo_protobuf//proto", "@com_github_gorilla_mux//:mux", "@com_github_grpc_ecosystem_grpc_gateway//runtime:go_default_library", diff --git a/pkg/server/status.go b/pkg/server/status.go index 70cf78680954..b95a81563783 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -87,6 +87,9 @@ const ( // statusVars exposes prometheus metrics for monitoring consumption. statusVars = statusPrefix + "vars" + // loadStatusVars exposes prometheus metrics for instant monitoring of CPU load. + loadStatusVars = statusPrefix + "load" + // raftStateDormant is used when there is no known raft state. raftStateDormant = "StateDormant" diff --git a/pkg/server/status/runtime.go b/pkg/server/status/runtime.go index 6982dd699e0a..51fae9c021ef 100644 --- a/pkg/server/status/runtime.go +++ b/pkg/server/status/runtime.go @@ -445,8 +445,8 @@ func (rsr *RuntimeStatSampler) SampleEnvironment( if err := mem.Get(pid); err != nil { log.Ops.Errorf(ctx, "unable to get mem usage: %v", err) } - cpuTime := gosigar.ProcTime{} - if err := cpuTime.Get(pid); err != nil { + userTimeMS, sysTimeMS, err := GetCPUTime(ctx) + if err != nil { log.Ops.Errorf(ctx, "unable to get cpu usage: %v", err) } cgroupCPU, _ := cgroups.GetCgroupCPU() @@ -507,8 +507,8 @@ func (rsr *RuntimeStatSampler) SampleEnvironment( now := rsr.clock.PhysicalNow() dur := float64(now - rsr.last.now) // cpuTime.{User,Sys} are in milliseconds, convert to nanoseconds. - utime := int64(cpuTime.User) * 1e6 - stime := int64(cpuTime.Sys) * 1e6 + utime := userTimeMS * 1e6 + stime := sysTimeMS * 1e6 urate := float64(utime-rsr.last.utime) / dur srate := float64(stime-rsr.last.stime) / dur combinedNormalizedPerc := (srate + urate) / cpuShare @@ -690,14 +690,13 @@ func subtractNetworkCounters(from *net.IOCountersStat, sub net.IOCountersStat) { from.PacketsSent -= sub.PacketsSent } -// GetUserCPUSeconds returns the cumulative User CPU time for this process, in -// seconds. -func GetUserCPUSeconds(ctx context.Context) float64 { +// GetCPUTime returns the cumulative user/system time (in ms) since the process start. +func GetCPUTime(ctx context.Context) (userTimeMS, sysTimeMS int64, err error) { pid := os.Getpid() cpuTime := gosigar.ProcTime{} if err := cpuTime.Get(pid); err != nil { log.Ops.Errorf(ctx, "unable to get cpu usage: %v", err) + return 0, 0, err } - // cpuTime.User is in milliseconds; convert to seconds. - return float64(cpuTime.User) * 1e-3 + return int64(cpuTime.User), int64(cpuTime.Sys), nil } diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 10f807118fd9..ed2920405ef9 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -14,6 +14,7 @@ import ( "context" "crypto/tls" "net/http" + "os" "time" "github.com/cockroachdb/cockroach/pkg/base" @@ -52,6 +53,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" + "github.com/elastic/gosigar" ) // StartTenant starts a stand-alone SQL server against a KV backend. @@ -219,6 +221,8 @@ func StartTenant( }) f := varsHandler{metricSource: args.recorder, st: args.Settings}.handleVars mux.Handle(statusVars, http.HandlerFunc(f)) + ff := loadVarsHandler(ctx, args.runtime) + mux.Handle(loadStatusVars, http.HandlerFunc(ff)) tlsConnManager := netutil.MakeServer( args.stopper, @@ -283,9 +287,17 @@ func StartTenant( s.SQLInstanceID(), ) + getUserCPUSec := func(ctx context.Context) float64 { + userTimeMS, _, err := status.GetCPUTime(ctx) + if err != nil { + log.Ops.Errorf(ctx, "unable to get cpu usage: %v", err) + } + return float64(userTimeMS) * 1e-3 + } + if err := args.costController.Start( ctx, args.stopper, s.SQLInstanceID(), s.sqlLivenessSessionID, - status.GetUserCPUSeconds, nextLiveInstanceIDFn, + getUserCPUSec, nextLiveInstanceIDFn, ); err != nil { return nil, "", "", err } @@ -301,6 +313,39 @@ func StartTenant( return s, pgLAddr, httpLAddr, nil } +// Construct a handler responsible for serving the instant values of selected +// load metrics. These include user and system CPU time currently. +func loadVarsHandler( + ctx context.Context, rsr *status.RuntimeStatSampler, +) func(http.ResponseWriter, *http.Request) { + cpuTime := gosigar.ProcTime{} + cpuUserNS := metric.NewGauge(rsr.CPUUserNS.GetMetadata()) + cpuSysNS := metric.NewGauge(rsr.CPUSysNS.GetMetadata()) + registry := metric.NewRegistry() + registry.AddMetric(cpuUserNS) + registry.AddMetric(cpuSysNS) + + return func(w http.ResponseWriter, r *http.Request) { + if err := cpuTime.Get(os.Getpid()); err != nil { + log.Ops.Errorf(ctx, "unable to get cpu usage: %v", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + utime := int64(cpuTime.User) * 1e6 + stime := int64(cpuTime.Sys) * 1e6 + cpuUserNS.Update(utime) + cpuSysNS.Update(stime) + + exporter := metric.MakePrometheusExporter() + exporter.ScrapeRegistry(registry, true) + if err := exporter.PrintAsText(w); err != nil { + log.Errorf(r.Context(), "%v", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + } +} + func makeTenantSQLServerArgs( stopper *stop.Stopper, kvClusterName string, baseCfg BaseConfig, sqlCfg SQLConfig, ) (sqlServerArgs, error) {