diff --git a/build/deploy/Dockerfile b/build/deploy/Dockerfile index fa858f0c8982..34b067fd2698 100644 --- a/build/deploy/Dockerfile +++ b/build/deploy/Dockerfile @@ -1,10 +1,13 @@ -FROM debian:8.11 +FROM debian:9.8-slim -# Install root CAs so we can make SSL connections to phone home and -# do backups to GCE/AWS/Azure. +# For deployment, we need +# libc6 - dynamically linked by cockroach binary +# ca-certificates - to authenticate TLS connections for telemetry and +# bulk-io with S3/GCS/Azure +# tzdata - for time zone functions RUN apt-get update && \ apt-get -y upgrade && \ - apt-get install -y ca-certificates && \ + apt-get install -y libc6 ca-certificates tzdata && \ rm -rf /var/lib/apt/lists/* RUN mkdir -p /cockroach diff --git a/pkg/cmd/docgen/funcs.go b/pkg/cmd/docgen/funcs.go index affff8d48bbc..d32faf0f3d5f 100644 --- a/pkg/cmd/docgen/funcs.go +++ b/pkg/cmd/docgen/funcs.go @@ -267,12 +267,14 @@ func linkArguments(t string) string { func linkTypeName(s string) string { s = strings.TrimSuffix(s, "{}") + s = strings.TrimSuffix(s, "{*}") name := s switch s { case "timestamptz": s = "timestamp" } s = strings.TrimSuffix(s, "[]") + s = strings.TrimSuffix(s, "*") switch s { case "int", "decimal", "float", "bool", "date", "timestamp", "interval", "string", "bytes", "inet", "uuid", "collatedstring", "time": diff --git a/pkg/server/telemetry/features.go b/pkg/server/telemetry/features.go index c4d0183b5117..86878756e531 100644 --- a/pkg/server/telemetry/features.go +++ b/pkg/server/telemetry/features.go @@ -64,6 +64,18 @@ func Inc(c Counter) { atomic.AddInt32(c, 1) } +// GetCounterOnce returns a counter from the global registry, +// and asserts it didn't exist previously. +func GetCounterOnce(feature string) Counter { + counters.RLock() + _, ok := counters.m[feature] + counters.RUnlock() + if ok { + panic("counter already exists: " + feature) + } + return GetCounter(feature) +} + // GetCounter returns a counter from the global registry. func GetCounter(feature string) Counter { counters.RLock() diff --git a/pkg/server/updates_test.go b/pkg/server/updates_test.go index a4e6934f2c12..875e677e3e88 100644 --- a/pkg/server/updates_test.go +++ b/pkg/server/updates_test.go @@ -275,6 +275,12 @@ func TestReportUsage(t *testing.T) { t.Fatal(err) } + if _, err := db.Exec( + fmt.Sprintf(`CREATE TABLE %[1]s.%[1]s_s (%[1]s SERIAL2)`, elemName), + ); err != nil { + t.Fatal(err) + } + // Run some queries so we have some query statistics collected. for i := 0; i < 10; i++ { // Run some sample queries. Each are passed a string and int by Exec. @@ -388,7 +394,7 @@ func TestReportUsage(t *testing.T) { if err != nil { t.Fatal(err) } - if actual := len(tables); actual != 1 { + if actual := len(tables); actual != 2 { t.Fatalf("unexpected table count %d", actual) } for _, table := range tables { @@ -543,14 +549,17 @@ func TestReportUsage(t *testing.T) { "test.b": 2, "test.c": 3, + // SERIAL normalization. + "sql.schema.serial.rowid.SERIAL2": 1, + // Although the query is executed 10 times, due to plan caching // keyed by the SQL text, the planning only occurs once. - "sql.ops.cast.text::inet": 1, - "sql.ops.bin.jsonb - text": 1, - "sql.builtins.crdb_internal.force_assertion_error(msg: string) -> int": 1, - "sql.ops.array.ind": 1, - "sql.ops.array.cons": 1, - "sql.ops.array.flatten": 1, + "sql.plan.ops.cast.string::inet": 1, + "sql.plan.ops.bin.jsonb - string": 1, + "sql.plan.builtins.crdb_internal.force_assertion_error(msg: string) -> int": 1, + "sql.plan.ops.array.ind": 1, + "sql.plan.ops.array.cons": 1, + "sql.plan.ops.array.flatten": 1, "unimplemented.#33285.json_object_agg": 10, "unimplemented.pg_catalog.pg_stat_wal_receiver": 10, @@ -693,6 +702,7 @@ func TestReportUsage(t *testing.T) { `[false,false,false] SET application_name = $1`, `[false,false,false] SET application_name = DEFAULT`, `[false,false,false] SET application_name = _`, + `[true,false,false] CREATE TABLE _ (_ INT8 NOT NULL DEFAULT unique_rowid())`, `[true,false,false] CREATE TABLE _ (_ INT8, CONSTRAINT _ CHECK (_ > _))`, `[true,false,false] INSERT INTO _ SELECT unnest(ARRAY[_, _, __more2__])`, `[true,false,false] INSERT INTO _ VALUES (_), (__more2__)`, @@ -743,6 +753,7 @@ func TestReportUsage(t *testing.T) { `ALTER TABLE _ CONFIGURE ZONE = _`, `CREATE DATABASE _`, `CREATE TABLE _ (_ INT8, CONSTRAINT _ CHECK (_ > _))`, + `CREATE TABLE _ (_ INT8 NOT NULL DEFAULT unique_rowid())`, `CREATE TABLE _ (_ INT8 PRIMARY KEY, _ INT8, INDEX (_) INTERLEAVE IN PARENT _ (_))`, `INSERT INTO _ VALUES (length($1::STRING)), (__more1__)`, `INSERT INTO _ VALUES (_), (__more2__)`, diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 0ffcedeb5bc9..260719efbbc4 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -478,7 +479,7 @@ func (s *Server) ServeConn(ctx context.Context, conn net.Conn) error { if version != version30 { if version == versionCancel { - telemetry.Count("pgwire.unimplemented.cancel_request") + telemetry.Inc(sqltelemetry.CancelRequestCounter) _ = conn.Close() return nil } @@ -563,7 +564,8 @@ func parseOptions(ctx context.Context, data []byte) (sql.SessionArgs, error) { } else { if !exists { if _, ok := sql.UnsupportedVars[key]; ok { - telemetry.Count("unimplemented.pgwire.parameter." + key) + counter := sqltelemetry.UnimplementedClientStatusParameterCounter(key) + telemetry.Inc(counter) } log.Warningf(ctx, "unknown configuration parameter: %q", key) } else { diff --git a/pkg/sql/pgwire/types.go b/pkg/sql/pgwire/types.go index 559a9b805a73..d03c7afb61d7 100644 --- a/pkg/sql/pgwire/types.go +++ b/pkg/sql/pgwire/types.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -275,7 +276,7 @@ func (b *writeBuffer) writeBinaryDatum( // The above encoding is not correct for Infinity, but since that encoding // doesn't exist in postgres, it's unclear what to do. For now use the NaN // encoding and count it to see if anyone even needs this. - telemetry.Count("pgwire.#32489.binary_decimal_infinity") + telemetry.Inc(sqltelemetry.BinaryDecimalInfinityCounter) } return diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 3b6e3679dd81..56bab3a6410e 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/arith" "github.com/cockroachdb/cockroach/pkg/util/bitarray" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -1339,7 +1340,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ Fn: func(_ *EvalContext, left Datum, right Datum) (Datum, error) { rval := MustBeDInt(right) if rval < 0 || rval >= 64 { - telemetry.Count("sql.large_lshift_argument") + telemetry.Inc(sqltelemetry.LargeLShiftArgumentCounter) return nil, pgerror.NewErrorf(pgerror.CodeInvalidParameterValueError, "shift argument out of range") } return NewDInt(MustBeDInt(left) << uint(rval)), nil @@ -1377,7 +1378,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ Fn: func(_ *EvalContext, left Datum, right Datum) (Datum, error) { rval := MustBeDInt(right) if rval < 0 || rval >= 64 { - telemetry.Count("sql.large_rshift_argument") + telemetry.Inc(sqltelemetry.LargeRShiftArgumentCounter) return nil, pgerror.NewErrorf(pgerror.CodeInvalidParameterValueError, "shift argument out of range") } return NewDInt(MustBeDInt(left) >> uint(rval)), nil diff --git a/pkg/sql/sem/tree/function_definition.go b/pkg/sql/sem/tree/function_definition.go index e4a3ea28fc60..93f1649cedfd 100644 --- a/pkg/sql/sem/tree/function_definition.go +++ b/pkg/sql/sem/tree/function_definition.go @@ -14,7 +14,7 @@ package tree -import "github.com/cockroachdb/cockroach/pkg/server/telemetry" +import "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" // FunctionDefinition implements a reference to the (possibly several) // overloads for a built-in function. @@ -126,7 +126,7 @@ func NewFunctionDefinition( props.AmbiguousReturnType = true } // Produce separate telemetry for each overload. - def[i].counter = telemetry.GetCounter("sql.builtins." + name + def[i].Signature(false)) + def[i].counter = sqltelemetry.BuiltinCounter(name, def[i].Signature(false)) overloads[i] = &def[i] } diff --git a/pkg/sql/sem/tree/operators.go b/pkg/sql/sem/tree/operators.go index ac071df9380e..1f25007cf255 100644 --- a/pkg/sql/sem/tree/operators.go +++ b/pkg/sql/sem/tree/operators.go @@ -16,11 +16,9 @@ package tree import ( "fmt" - "strings" - "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" - "github.com/lib/pq/oid" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" ) // This file implements the generation of unique names for every @@ -29,22 +27,10 @@ import ( // The historical first purpose of generating these names is to be used // as telemetry keys, for feature usage reporting. -// Scalar operators will be counter as sql.ops.. . -const unOpCounterNameFmt = "sql.ops.un.%s %s" -const cmpOpCounterNameFmt = "sql.ops.cmp.%s %s %s" -const binOpCounterNameFmt = "sql.ops.bin.%s %s %s" - -// Cast operators will be counted as sql.ops.cast.::. -const castCounterNameFmt = "sql.ops.cast.%s::%s" - -// All casts that involve arrays will also be counted towards this -// feature counter. -var arrayCastCounter = telemetry.GetCounter("sql.ops.cast.arrays") - // Detailed counter name generation follows. // // We pre-allocate the counter objects upfront here and later use -// Inc(), to avoid the hash map lookup in telemetry.Count() upon type +// Inc(), to avoid the hash map lookup in telemetry.Count upon type // checking every scalar operator node. // The logic that follows is also associated with a related feature in @@ -67,22 +53,7 @@ var arrayCastCounter = telemetry.GetCounter("sql.ops.cast.arrays") // in distsql. // -// makeOidName generates a short name for the given type OID. -func makeOidName(op fmt.Stringer, tOid oid.Oid) (string, bool) { - oidName, ok := oid.TypeName[tOid] - if !ok { - return "", false - } - name := strings.ToLower(oidName) - if strings.HasPrefix(name, "timestamp") { - name = "ts" + name[9:] - } - return name, true -} - func init() { - seen := map[string]struct{}{} - // Label the unary operators. for op, overloads := range UnaryOps { if int(op) >= len(unaryOpName) || unaryOpName[op] == "" { @@ -91,15 +62,7 @@ func init() { opName := unaryOpName[op] for _, impl := range overloads { o := impl.(*UnaryOp) - uname, ok := makeOidName(op, o.Typ.Oid()) - if !ok { - continue - } - name := fmt.Sprintf(unOpCounterNameFmt, opName, uname) - if _, ok := seen[name]; ok { - panic(fmt.Sprintf("duplicate name: %q", name)) - } - o.counter = telemetry.GetCounter(name) + o.counter = sqltelemetry.UnaryOpCounter(opName, o.Typ.String()) } } @@ -111,16 +74,9 @@ func init() { opName := comparisonOpName[op] for _, impl := range overloads { o := impl.(*CmpOp) - lname, lok := makeOidName(op, o.LeftType.Oid()) - rname, rok := makeOidName(op, o.RightType.Oid()) - if !lok || !rok { - continue - } - name := fmt.Sprintf(cmpOpCounterNameFmt, lname, opName, rname) - if _, ok := seen[name]; ok { - panic(fmt.Sprintf("duplicate name: %q", name)) - } - o.counter = telemetry.GetCounter(name) + lname := o.LeftType.String() + rname := o.RightType.String() + o.counter = sqltelemetry.CmpOpCounter(opName, lname, rname) } } @@ -132,16 +88,9 @@ func init() { opName := binaryOpName[op] for _, impl := range overloads { o := impl.(*BinOp) - lname, lok := makeOidName(op, o.LeftType.Oid()) - rname, rok := makeOidName(op, o.RightType.Oid()) - if !lok || !rok { - continue - } - name := fmt.Sprintf(binOpCounterNameFmt, lname, opName, rname) - if _, ok := seen[name]; ok { - panic(fmt.Sprintf("duplicate name: %q", name)) - } - o.counter = telemetry.GetCounter(name) + lname := o.LeftType.String() + rname := o.RightType.String() + o.counter = sqltelemetry.BinOpCounter(opName, lname, rname) } } } @@ -153,29 +102,11 @@ func annotateCast(toType types.T, fromTypes []types.T) []castInfo { for i, fromType := range fromTypes { ci[i].fromT = fromType } - var rname string - if toType.FamilyEqual(types.FamArray) { - rname = "array" - } else { - var rok bool - rname, rok = makeOidName(nil, toType.Oid()) - if !rok { - return ci - } - } + rname := toType.String() for i, fromType := range fromTypes { - var lname string - if fromType.FamilyEqual(types.FamArray) { - lname = "array" - } else { - var lok bool - lname, lok = makeOidName(nil, fromType.Oid()) - if !lok { - continue - } - } - ci[i].counter = telemetry.GetCounter(fmt.Sprintf(castCounterNameFmt, lname, rname)) + lname := fromType.String() + ci[i].counter = sqltelemetry.CastOpCounter(lname, rname) } return ci } diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 492ccbf17f6e..67cb57f1c197 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -413,7 +414,7 @@ func isCastDeepValid(castFrom, castTo types.T) (bool, telemetry.Counter) { if castTo.FamilyEqual(types.FamArray) && castFrom.FamilyEqual(types.FamArray) { ok, c := isCastDeepValid(castFrom.(types.TArray).Typ, castTo.(types.TArray).Typ) if ok { - telemetry.Inc(arrayCastCounter) + telemetry.Inc(sqltelemetry.ArrayCastCounter) } return ok, c } @@ -474,8 +475,6 @@ func (expr *CastExpr) TypeCheck(ctx *SemaContext, _ types.T) (TypedExpr, error) return nil, pgerror.NewErrorf(pgerror.CodeCannotCoerceError, "invalid cast: %s -> %s", castFrom, expr.Type) } -var arraySubscriptTelemetryCounter = telemetry.GetCounter("sql.ops.array.ind") - // TypeCheck implements the Expr interface. func (expr *IndirectionExpr) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, error) { for i, t := range expr.Indirection { @@ -505,7 +504,7 @@ func (expr *IndirectionExpr) TypeCheck(ctx *SemaContext, desired types.T) (Typed expr.Expr = subExpr expr.typ = arrType.Typ - telemetry.Inc(arraySubscriptTelemetryCounter) + telemetry.Inc(sqltelemetry.ArraySubscriptCounter) return expr, nil } @@ -971,8 +970,6 @@ func (f *WindowFrame) TypeCheck(ctx *SemaContext, windowDef *WindowDef) error { return nil } -var ifErrTelemetryCounter = telemetry.GetCounter("sql.ops.iferr") - // TypeCheck implements the Expr interface. func (expr *IfErrExpr) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, error) { var typedCond, typedElse TypedExpr @@ -1006,7 +1003,7 @@ func (expr *IfErrExpr) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, expr.ErrCode = typedErrCode expr.typ = retType - telemetry.Inc(ifErrTelemetryCounter) + telemetry.Inc(sqltelemetry.IfErrCounter) return expr, nil } @@ -1267,8 +1264,6 @@ func (expr *Tuple) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, erro var errAmbiguousArrayType = pgerror.NewErrorf(pgerror.CodeIndeterminateDatatypeError, "cannot determine type of empty array. "+ "Consider annotating with the desired type, for example ARRAY[]:::int[]") -var arrayConstructorTelemetryCounter = telemetry.GetCounter("sql.ops.array.cons") - // TypeCheck implements the Expr interface. func (expr *Array) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, error) { desiredParam := types.Any @@ -1294,12 +1289,10 @@ func (expr *Array) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, erro expr.Exprs[i] = typedSubExprs[i] } - telemetry.Inc(arrayConstructorTelemetryCounter) + telemetry.Inc(sqltelemetry.ArrayConstructorCounter) return expr, nil } -var arrayFlattenTelemetryCounter = telemetry.GetCounter("sql.ops.array.flatten") - // TypeCheck implements the Expr interface. func (expr *ArrayFlatten) TypeCheck(ctx *SemaContext, desired types.T) (TypedExpr, error) { desiredParam := types.Any @@ -1314,7 +1307,7 @@ func (expr *ArrayFlatten) TypeCheck(ctx *SemaContext, desired types.T) (TypedExp expr.Subquery = subqueryTyped expr.typ = types.TArray{Typ: subqueryTyped.ResolvedType()} - telemetry.Inc(arrayFlattenTelemetryCounter) + telemetry.Inc(sqltelemetry.ArrayFlattenCounter) return expr, nil } diff --git a/pkg/sql/sem/types/types.go b/pkg/sql/sem/types/types.go index 45fe80fe7c33..d1d5fa6b00cb 100644 --- a/pkg/sql/sem/types/types.go +++ b/pkg/sql/sem/types/types.go @@ -193,6 +193,10 @@ type TCollatedString struct { // String implements the fmt.Stringer interface. func (t TCollatedString) String() string { + if t.Locale == "" { + // Used in telemetry. + return "collatedstring{*}" + } return fmt.Sprintf("collatedstring{%s}", t.Locale) } @@ -445,7 +449,13 @@ func (TPlaceholder) IsAmbiguous() bool { panic("TPlaceholder.IsAmbiguous() is un // TArray is the type of a DArray. type TArray struct{ Typ T } -func (a TArray) String() string { return a.Typ.String() + "[]" } +func (a TArray) String() string { + if a.Typ == nil { + // Used in telemetry. + return "*[]" + } + return a.Typ.String() + "[]" +} // Equivalent implements the T interface. func (a TArray) Equivalent(other T) bool { diff --git a/pkg/sql/serial.go b/pkg/sql/serial.go index e96696335334..25f33f7eea95 100644 --- a/pkg/sql/serial.go +++ b/pkg/sql/serial.go @@ -18,10 +18,12 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -83,6 +85,8 @@ func (p *planner) processSerialInColumnDef( newSpec.Type = t.TInt } + telemetry.Inc(sqltelemetry.SerialColumnNormalizationCounter(t.String(), serialNormalizationMode.String())) + if serialNormalizationMode == sessiondata.SerialUsesRowID { // We're not constructing a sequence for this SERIAL column. // Use the "old school" CockroachDB default. diff --git a/pkg/sql/sqltelemetry/doc.go b/pkg/sql/sqltelemetry/doc.go new file mode 100644 index 000000000000..8fb0064dab5e --- /dev/null +++ b/pkg/sql/sqltelemetry/doc.go @@ -0,0 +1,30 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +/* +Package sqltelemetry contains telemetry counter definitions +for various SQL features. + +Centralizing the counters in a single place achieves three objectives: + +- the comments that accompany the counters enable non-technical users + to comprehend what is being reported without having to read code. + +- the counters are placed side-by-side, grouped by category, so as to + enable exploratory discovery of available telemetry. + +- the counters are pre-registered and their unicity is asserted, + so that no two features end up using the same counter name. +*/ +package sqltelemetry diff --git a/pkg/sql/sqltelemetry/pgwire.go b/pkg/sql/sqltelemetry/pgwire.go new file mode 100644 index 000000000000..bdb58e33ff76 --- /dev/null +++ b/pkg/sql/sqltelemetry/pgwire.go @@ -0,0 +1,37 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sqltelemetry + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/server/telemetry" +) + +// CancelRequestCounter is to be incremented every time a pgwire-level +// cancel request is received from a client. +var CancelRequestCounter = telemetry.GetCounterOnce("pgwire.unimplemented.cancel_request") + +// UnimplementedClientStatusParameterCounter is to be incremented +// every time a client attempts to configure a status parameter +// that's not supported upon session initialization. +func UnimplementedClientStatusParameterCounter(key string) telemetry.Counter { + return telemetry.GetCounter(fmt.Sprintf("unimplemented.pgwire.parameter.%s", key)) +} + +// BinaryDecimalInfinityCounter is to be incremented every time a +// client requests the binary encoding for a decimal infinity, which +// is not well defined in the pg protocol (#32489). +var BinaryDecimalInfinityCounter = telemetry.GetCounterOnce("pgwire.#32489.binary_decimal_infinity") diff --git a/pkg/sql/sqltelemetry/scalar.go b/pkg/sql/sqltelemetry/scalar.go new file mode 100644 index 000000000000..90bf8e37e075 --- /dev/null +++ b/pkg/sql/sqltelemetry/scalar.go @@ -0,0 +1,81 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sqltelemetry + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/server/telemetry" +) + +// BuiltinCounter creates a telemetry counter for a built-in function. +// This is to be incremented upon type checking of a function application. +func BuiltinCounter(name, signature string) telemetry.Counter { + return telemetry.GetCounterOnce(fmt.Sprintf("sql.plan.builtins.%s%s", name, signature)) +} + +// UnaryOpCounter creates a telemetry counter for a scalar unary operator. +// This is to be incremented upon type checking of this type of scalar operation. +func UnaryOpCounter(op, typ string) telemetry.Counter { + return telemetry.GetCounterOnce(fmt.Sprintf("sql.plan.ops.un.%s %s", op, typ)) +} + +// CmpOpCounter creates a telemetry counter for a scalar comparison operator. +// This is to be incremented upon type checking of this type of scalar operation. +func CmpOpCounter(op, ltyp, rtyp string) telemetry.Counter { + return telemetry.GetCounterOnce(fmt.Sprintf("sql.plan.ops.cmp.%s %s %s", ltyp, op, rtyp)) +} + +// BinOpCounter creates a telemetry counter for a scalar binary operator. +// This is to be incremented upon type checking of this type of scalar operation. +func BinOpCounter(op, ltyp, rtyp string) telemetry.Counter { + return telemetry.GetCounterOnce(fmt.Sprintf("sql.plan.ops.bin.%s %s %s", ltyp, op, rtyp)) +} + +// CastOpCounter creates a telemetry counter for a scalar cast operator. +// This is to be incremented upon type checking of this type of scalar operation. +func CastOpCounter(ftyp, ttyp string) telemetry.Counter { + return telemetry.GetCounterOnce(fmt.Sprintf("sql.plan.ops.cast.%s::%s", ftyp, ttyp)) +} + +// ArrayCastCounter is to be incremented when type checking all casts +// that involve arrays. This separate telemetry counter is needed +// because an inter-array cast lands on `sql.plan.ops.cast` telemetry +// counter for the element type. +var ArrayCastCounter = telemetry.GetCounterOnce("sql.plan.ops.cast.arrays") + +// ArrayConstructorCounter is to be incremented upon type checking +// of ARRAY[...] expressions/ +var ArrayConstructorCounter = telemetry.GetCounterOnce("sql.plan.ops.array.cons") + +// ArrayFlattenCounter is to be incremented upon type checking +// of ARRAY(...) expressions. +var ArrayFlattenCounter = telemetry.GetCounterOnce("sql.plan.ops.array.flatten") + +// ArraySubscriptCounter is to be incremented upon type checking an +// array subscript expression x[...]. +var ArraySubscriptCounter = telemetry.GetCounterOnce("sql.plan.ops.array.ind") + +// IfErrCounter is to be incremented upon type checking an +// IFERROR(...) expression or analogous. +var IfErrCounter = telemetry.GetCounterOnce("sql.plan.ops.iferr") + +// LargeLShiftArgumentCounter is to be incremented upon evaluating a scalar +// expressions a << b when b is larger than 64 or negative. +var LargeLShiftArgumentCounter = telemetry.GetCounterOnce("sql.large_lshift_argument") + +// LargeRShiftArgumentCounter is to be incremented upon evaluating a scalar +// expressions a >> b when b is larger than 64 or negative. +var LargeRShiftArgumentCounter = telemetry.GetCounterOnce("sql.large_rshift_argument") diff --git a/pkg/sql/sqltelemetry/schema.go b/pkg/sql/sqltelemetry/schema.go new file mode 100644 index 000000000000..71872bd3343f --- /dev/null +++ b/pkg/sql/sqltelemetry/schema.go @@ -0,0 +1,29 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sqltelemetry + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/server/telemetry" +) + +// SerialColumnNormalizationCounter is to be incremented every time +// a SERIAL type is processed in a column definition. +// It includes the normalization type, so we can +// estimate usage of the various normalization strategies. +func SerialColumnNormalizationCounter(inputType, normType string) telemetry.Counter { + return telemetry.GetCounter(fmt.Sprintf("sql.schema.serial.%s.%s", normType, inputType)) +} diff --git a/pkg/sql/sqltelemetry/session.go b/pkg/sql/sqltelemetry/session.go new file mode 100644 index 000000000000..7fa77da4cffe --- /dev/null +++ b/pkg/sql/sqltelemetry/session.go @@ -0,0 +1,37 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sqltelemetry + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/server/telemetry" +) + +// DefaultIntSize4Counter is to be incremented every time a client +// change the default_int_size variable to its non-default value 4. +var DefaultIntSize4Counter = telemetry.GetCounterOnce("sql.default_int_size.4") + +// ForceSavepointRestartCounter is to be incremented every time a +// client customizes the session variable force_savepoint_restart +// to a non-empty string. +var ForceSavepointRestartCounter = telemetry.GetCounterOnce("sql.force_savepoint_restart") + +// UnimplementedSessionVarValueCounter is to be incremented every time +// a client attempts to set a compatitibility session var to an +// unsupported value. +func UnimplementedSessionVarValueCounter(varName, val string) telemetry.Counter { + return telemetry.GetCounter(fmt.Sprintf("unimplemented.sql.session_var.%s.%s", varName, val)) +} diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 16a1fd602418..40488790f1b5 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" ) @@ -246,7 +247,7 @@ var varGen = map[string]sessionVar{ // set to int4 by a connection string. // TODO(bob): Change to 8 in v2.3: https://github.com/cockroachdb/cockroach/issues/32534 if i == 4 { - telemetry.Count("sql.default_int_size.4") + telemetry.Inc(sqltelemetry.DefaultIntSize4Counter) } m.SetDefaultIntSize(int(i)) return nil @@ -456,7 +457,7 @@ var varGen = map[string]sessionVar{ return err } if b { - telemetry.Count("sql.force_savepoint_restart") + telemetry.Inc(sqltelemetry.ForceSavepointRestartCounter) } m.SetForceSavepointRestart(b) return nil @@ -763,7 +764,7 @@ func makeCompatBoolVar(varName string, displayValue, anyValAllowed bool) session if anyValAllowed || b == displayValue { return nil } - telemetry.Count(fmt.Sprintf("unimplemented.sql.session_var.%s.%s", varName, s)) + telemetry.Inc(sqltelemetry.UnimplementedSessionVarValueCounter(varName, s)) allowedVals := []string{displayValStr} if anyValAllowed { allowedVals = append(allowedVals, formatBoolAsPostgresSetting(!displayValue)) @@ -799,7 +800,7 @@ func makeCompatStringVar(varName, displayValue string, extraAllowed ...string) s return nil } } - telemetry.Count(fmt.Sprintf("unimplemented.sql.session_var.%s.%s", varName, s)) + telemetry.Inc(sqltelemetry.UnimplementedSessionVarValueCounter(varName, s)) return newVarValueError(varName, s, allowedVals...).SetDetailf( "this parameter is currently recognized only for compatibility and has no effect in CockroachDB.") }, diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 065a27469b89..315358d70f47 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -342,6 +342,71 @@ func TestLint(t *testing.T) { } }) + t.Run("TestSQLTelemetryDirectCount", func(t *testing.T) { + t.Parallel() + cmd, stderr, filter, err := dirCmd( + pkgDir, + "git", + "grep", + "-nE", + `[^[:alnum:]]telemetry\.Count\(`, + "--", + "sql", + ) + if err != nil { + t.Fatal(err) + } + + if err := cmd.Start(); err != nil { + t.Fatal(err) + } + + if err := stream.ForEach(filter, func(s string) { + t.Errorf("\n%s <- forbidden; use 'sqltelemetry.xxxCounter()' / `telemetry.Inc' instead", s) + }); err != nil { + t.Error(err) + } + + if err := cmd.Wait(); err != nil { + if out := stderr.String(); len(out) > 0 { + t.Fatalf("err=%s, stderr=%s", err, out) + } + } + }) + + t.Run("TestSQLTelemetryGetCounter", func(t *testing.T) { + t.Parallel() + cmd, stderr, filter, err := dirCmd( + pkgDir, + "git", + "grep", + "-nE", + `[^[:alnum:]]telemetry\.GetCounter`, + "--", + "sql", + ":!sql/sqltelemetry", + ) + if err != nil { + t.Fatal(err) + } + + if err := cmd.Start(); err != nil { + t.Fatal(err) + } + + if err := stream.ForEach(filter, func(s string) { + t.Errorf("\n%s <- forbidden; use 'sqltelemetry.xxxCounter() instead", s) + }); err != nil { + t.Error(err) + } + + if err := cmd.Wait(); err != nil { + if out := stderr.String(); len(out) > 0 { + t.Fatalf("err=%s, stderr=%s", err, out) + } + } + }) + t.Run("TestTodoStyle", func(t *testing.T) { t.Parallel() // TODO(tamird): enforce presence of name.