Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
35517: build: Upgrade base image for deployment dockerfile r=mberhault a=bdarnell

Debian 8.x has unpatched security vulnerabilities in the glibc
package (even though it's supposed to be in its LTS period?), so
upgrade to the current 9.x.

https://security-tracker.debian.org/tracker/source-package/glibc

Release note (build change): Release Docker images are now built on
Debian 9.8.

35647: sqltelemetry: make SQL telemetry more uniform, w/ linter r=knz a=knz

This patch groups the telemetry counters for the SQL layer into a
single package `sqltelemetry`. The package aims to document the
telemetry counter in a way that's easily understandable / discoverable
without having to read code.

This also introduces the pattern required by @dt: that telemetry
counter names generated dynamically to be generated in a way that's
regular, subject to later formalization via a new API
`telemetry.GetCounterf()` (so that we can automate the inventorization
of all telemetry counters). This patch ensures that all the dynamic
counter names go to `GetCounter(fmt.Sprintf(...))`, which we'll be
able to re-structure at a later point.

Finally it introduces the prefix `sql.plan`, in anticipation of a
further split between `sql.parse`, `sql.plan`, `sql.exec`.

Release note: None

35656: sql: add telemetry for SERIAL expansions r=knz a=knz

First commit from  #35647.
Informs https://github.com/cockroachlabs/registration/issues/203

This adds telemetry for the SERIAL expansion strategies.

Counter format: `sql.schema.serial.<normtype>.<coltype>`

For example: `sql.schema.serial.rowid.SERIAL2`

Release note (sql change): CockroachDB will now report how the SERIAL
pseudo-type is expanded in table column definitions, when telemetry is
enabled.


Co-authored-by: Ben Darnell <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Mar 12, 2019
4 parents 93da31a + d845f3b + df6bba0 + 558c7be commit 895659a
Show file tree
Hide file tree
Showing 19 changed files with 367 additions and 117 deletions.
11 changes: 7 additions & 4 deletions build/deploy/Dockerfile
Original file line number Diff line number Diff line change
@@ -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
Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/docgen/funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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":
Expand Down
12 changes: 12 additions & 0 deletions pkg/server/telemetry/features.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
25 changes: 18 additions & 7 deletions pkg/server/updates_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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__)`,
Expand Down Expand Up @@ -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__)`,
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/pgwire/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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 {
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/pgwire/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/sem/tree/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/sem/tree/function_definition.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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]
}
Expand Down
93 changes: 12 additions & 81 deletions pkg/sql/sem/tree/operators.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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.<kind>.<lhstype> <opname> <rhstype>.
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.<fromtype>::<totype>.
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
Expand All @@ -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] == "" {
Expand All @@ -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())
}
}

Expand All @@ -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)
}
}

Expand All @@ -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)
}
}
}
Expand All @@ -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
}
Loading

0 comments on commit 895659a

Please sign in to comment.