From 91abb91957f0b7e955b42945093a57f078562b5e Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Wed, 1 Feb 2023 10:49:31 -0500 Subject: [PATCH 1/8] logictest: skip another flaky test in ranges This commit skips a flaky test that I missed in #96271. Informs #96136 Epic: None Release note: None --- pkg/sql/logictest/testdata/logic_test/ranges | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index b67dc499c8a6..4dce2d6db6f5 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -599,10 +599,12 @@ subtest show_range_index_for_row_columns statement ok CREATE TABLE tbl_with_idx_for_row(i INT, INDEX idx (i)); -query TTIITTTTT colnames -SHOW RANGE FROM INDEX tbl_with_idx_for_row@idx FOR ROW (NULL, 0) ----- -start_key end_key range_id lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas - … 94 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} +# TODO(#96136): Re-enable this test after making it deterministic. +# query TTIITTTTT colnames +# query TTIITTTTT colnames +# SHOW RANGE FROM INDEX tbl_with_idx_for_row@idx FOR ROW (NULL, 0) +# ---- +# start_key end_key range_id lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas +# … 94 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} subtest end From ab4af3e5277d15328a504b40b2ca6788a990c917 Mon Sep 17 00:00:00 2001 From: rharding6373 Date: Thu, 12 Jan 2023 11:29:27 -0800 Subject: [PATCH 2/8] sql: support `*` in udf bodies This change allows `*` usage in UDF bodies. We rewrite UDF ASTs in place to expand `*`s into the columns they reference. Informs: #90080 Epic: CRDB-19496 Release note (sql change): Allow `*` expressions in UDFs. --- .../tests/3node-tenant/generated_test.go | 7 + pkg/sql/logictest/testdata/logic_test/udf | 18 -- .../logictest/testdata/logic_test/udf_star | 222 ++++++++++++++++++ .../tests/fakedist-disk/generated_test.go | 7 + .../tests/fakedist-vec-off/generated_test.go | 7 + .../tests/fakedist/generated_test.go | 7 + .../generated_test.go | 7 + .../tests/local-vec-off/generated_test.go | 7 + .../logictest/tests/local/generated_test.go | 7 + pkg/sql/opt/optbuilder/delete.go | 4 +- pkg/sql/opt/optbuilder/insert.go | 8 +- pkg/sql/opt/optbuilder/mutation_builder.go | 2 +- pkg/sql/opt/optbuilder/project.go | 39 ++- pkg/sql/opt/optbuilder/select.go | 11 +- .../opt/optbuilder/testdata/create_function | 10 +- pkg/sql/opt/optbuilder/testdata/udf | 79 +++++++ pkg/sql/opt/optbuilder/update.go | 4 +- pkg/sql/opt/optbuilder/util.go | 3 - 18 files changed, 407 insertions(+), 42 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/udf_star diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index eed192f6d298..fa92cd05e1a8 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -2019,6 +2019,13 @@ func TestTenantLogic_udf( runLogicTest(t, "udf") } +func TestTenantLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestTenantLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 7685e386689a..45056df03089 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -52,24 +52,6 @@ CREATE FUNCTION err(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT j' statement error pgcode 42703 column \"j\" does not exist CREATE FUNCTION err(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT a FROM ab WHERE a = j' -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS ab LANGUAGE SQL AS 'SELECT * FROM ab' - -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS ab LANGUAGE SQL AS 'SELECT ab.* FROM ab' - -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS ab LANGUAGE SQL AS $$ - SELECT 1; - SELECT * FROM ab; -$$ - -statement error pgcode 0A000 functions do not currently support \* expressions -CREATE FUNCTION err(i INT) RETURNS INT LANGUAGE SQL AS $$ - SELECT * FROM ab; - SELECT 1; -$$ - statement ok CREATE FUNCTION d(i INT2) RETURNS INT4 LANGUAGE SQL AS 'SELECT i' diff --git a/pkg/sql/logictest/testdata/logic_test/udf_star b/pkg/sql/logictest/testdata/logic_test/udf_star new file mode 100644 index 000000000000..3f1f3644bcc3 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/udf_star @@ -0,0 +1,222 @@ +statement ok +CREATE TABLE t_onecol (a INT); +INSERT INTO t_onecol VALUES (1) + +statement ok +CREATE TABLE t_twocol (a INT, b INT); +INSERT INTO t_twocol VALUES (1,2) + +statement ok +CREATE FUNCTION f_unqualified_onecol() RETURNS INT AS +$$ + SELECT * FROM t_onecol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_subquery() RETURNS INT AS +$$ + SELECT * FROM (SELECT a FROM (SELECT * FROM t_onecol) AS foo) AS bar; +$$ LANGUAGE SQL; + +statement error pq: unimplemented: unaliased subquery inside a function definition +CREATE FUNCTION f_subquery_unaliased() RETURNS INT AS +$$ + SELECT * FROM (SELECT a FROM (SELECT * FROM t_onecol)); +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_unqualified_twocol() RETURNS t_twocol AS +$$ + SELECT * FROM t_twocol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_allcolsel() RETURNS t_twocol AS +$$ + SELECT t_twocol.* FROM t_twocol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_allcolsel_alias() RETURNS t_twocol AS +$$ + SELECT t1.* FROM t_twocol AS t1, t_twocol AS t2 WHERE t1.a = t2.a; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_tuplestar() RETURNS t_twocol AS +$$ + SELECT (t_twocol.*).* FROM t_twocol; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_unqualified_multicol() RETURNS INT AS +$$ + SELECT *, a FROM t_onecol; + SELECT 1; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_unqualified_doublestar() RETURNS INT AS +$$ + SELECT *, * FROM t_onecol; + SELECT 1; +$$ LANGUAGE SQL; + +statement ok +CREATE FUNCTION f_exprstar() RETURNS STRING AS +$$ + SELECT word FROM (SELECT (pg_get_keywords()).* ORDER BY word LIMIT 1) AS foo; +$$ LANGUAGE SQL; + +statement error pq: unimplemented: unaliased subquery inside a function definition +CREATE FUNCTION f_ambiguous() RETURNS INT AS +$$ + SELECT * FROM (SELECT a FROM t_onecol) JOIN (SELECT a FROM t_twocol) ON true; + SELECT 1; +$$ LANGUAGE SQL; + +statement error pq: column reference "a" is ambiguous +CREATE FUNCTION f_ambiguous() RETURNS INT AS +$$ + SELECT a FROM (SELECT * FROM (SELECT a FROM t_onecol) AS foo JOIN (SELECT a FROM t_twocol) AS bar ON true) AS baz; + SELECT 1; +$$ LANGUAGE SQL; + +query TTT +SELECT oid, proname, prosrc +FROM pg_catalog.pg_proc WHERE proname LIKE 'f\_%' ORDER BY oid; +---- +100108 f_unqualified_onecol SELECT t_onecol.a FROM test.public.t_onecol; +100109 f_subquery SELECT bar.a FROM (SELECT a FROM (SELECT t_onecol.a FROM test.public.t_onecol) AS foo) AS bar; +100110 f_unqualified_twocol SELECT t_twocol.a, t_twocol.b FROM test.public.t_twocol; +100111 f_allcolsel SELECT t_twocol.a, t_twocol.b FROM test.public.t_twocol; +100112 f_allcolsel_alias SELECT t1.a, t1.b FROM test.public.t_twocol AS t1, test.public.t_twocol AS t2 WHERE t1.a = t2.a; +100113 f_tuplestar SELECT t_twocol.a, t_twocol.b FROM test.public.t_twocol; +100114 f_unqualified_multicol SELECT t_onecol.a, a FROM test.public.t_onecol; + SELECT 1; +100115 f_unqualified_doublestar SELECT t_onecol.a, t_onecol.a FROM test.public.t_onecol; + SELECT 1; +100116 f_exprstar SELECT word FROM (SELECT (pg_get_keywords()).word, (pg_get_keywords()).catcode, (pg_get_keywords()).catdesc ORDER BY word LIMIT 1) AS foo; + + +query TT +SHOW CREATE FUNCTION f_subquery +---- +f_subquery CREATE FUNCTION public.f_subquery() + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT bar.a FROM (SELECT a FROM (SELECT t_onecol.a FROM test.public.t_onecol) AS foo) AS bar; + $$ + +query TT +SHOW CREATE FUNCTION f_allcolsel_alias +---- +f_allcolsel_alias CREATE FUNCTION public.f_allcolsel_alias() + RETURNS T_TWOCOL + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT t1.a, t1.b FROM test.public.t_twocol AS t1, test.public.t_twocol AS t2 WHERE t1.a = t2.a; + $$ + +query I +SELECT f_unqualified_onecol() +---- +1 + +query I +SELECT f_subquery() +---- +1 + +query T +SELECT f_exprstar() +---- +abort + +# Adding a column to a table should not change the UDFs that reference it. +statement ok +ALTER TABLE t_onecol ADD COLUMN b INT DEFAULT 5; + +query I +SELECT f_unqualified_onecol() +---- +1 + +query I +SELECT f_subquery() +---- +1 + +# It's ok to drop a column that was not used by the original UDF. +statement ok +ALTER TABLE t_onecol DROP COLUMN b; + +query T +SELECT f_unqualified_twocol() +---- +(1,2) + +query T +SELECT f_allcolsel() +---- +(1,2) + +query T +SELECT f_allcolsel_alias() +---- +(1,2) + +statement ok +ALTER TABLE t_twocol ADD COLUMN c INT DEFAULT 5; + +# TODO(#95558): With early binding, postgres returns an error after adding a +# column when the table is used as the return type. Note that this behavior is +# ok for late binding. +query T +SELECT f_unqualified_twocol() +---- +(1,2) + +# Altering a column type is not allowed in postgres or CRDB. +statement error pq: cannot alter type of column "b" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol ALTER b TYPE FLOAT; + +# TODO(harding): Postgres allows column renaming when only referenced by UDFs. +statement error pq: cannot rename column "a" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol RENAME COLUMN a TO d; + +# TODO(harding): Postgres allows table renaming when only referenced by UDFs. +statement error pq: cannot rename relation "t_twocol" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol RENAME TO t_twocol_prime; + +# Dropping a column a UDF depends on is not allowed. +statement error pq: cannot drop column "b" because function "f_unqualified_twocol" depends on it +ALTER TABLE t_twocol DROP COLUMN b; + +# Drop all but one of the functions with an implicit record return value. +# TODO(96368): Allow these UDFs to be dropped in the CASCADE when the cross- +# references are fixed instead. +statement ok +DROP FUNCTION f_tuplestar; +DROP FUNCTION f_allcolsel_alias; + +# Dropping a column using CASCADE is ok. +statement ok +ALTER TABLE t_twocol DROP COLUMN b CASCADE; + +statement ok +DROP TABLE t_onecol CASCADE; + +# The only remaining function should not reference the tables. +query TTT +SELECT oid, proname, prosrc +FROM pg_catalog.pg_proc WHERE proname LIKE 'f\_%' ORDER BY oid; +---- +100116 f_exprstar SELECT word FROM (SELECT (pg_get_keywords()).word, (pg_get_keywords()).catcode, (pg_get_keywords()).catdesc ORDER BY word LIMIT 1) AS foo; diff --git a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go index f063f7c1018e..1d7d2263459a 100644 --- a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go @@ -1990,6 +1990,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go index 1ed95c3e9141..bb58d21dbe4e 100644 --- a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go @@ -1997,6 +1997,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist/generated_test.go b/pkg/sql/logictest/tests/fakedist/generated_test.go index aabbee91bed6..0580b4334d2a 100644 --- a/pkg/sql/logictest/tests/fakedist/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist/generated_test.go @@ -2011,6 +2011,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go index aa2ce22a5fb9..f6e62fb6e38f 100644 --- a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go @@ -1976,6 +1976,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-vec-off/generated_test.go b/pkg/sql/logictest/tests/local-vec-off/generated_test.go index 0da606fd62c9..f554fe62764b 100644 --- a/pkg/sql/logictest/tests/local-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/local-vec-off/generated_test.go @@ -2011,6 +2011,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local/generated_test.go b/pkg/sql/logictest/tests/local/generated_test.go index 8569be3004de..87de51ba4005 100644 --- a/pkg/sql/logictest/tests/local/generated_test.go +++ b/pkg/sql/logictest/tests/local/generated_test.go @@ -2200,6 +2200,13 @@ func TestLogic_udf( runLogicTest(t, "udf") } +func TestLogic_udf_star( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "udf_star") +} + func TestLogic_union( t *testing.T, ) { diff --git a/pkg/sql/opt/optbuilder/delete.go b/pkg/sql/opt/optbuilder/delete.go index 834eee850a8c..bcac82ceb73d 100644 --- a/pkg/sql/opt/optbuilder/delete.go +++ b/pkg/sql/opt/optbuilder/delete.go @@ -66,7 +66,7 @@ func (b *Builder) buildDelete(del *tree.Delete, inScope *scope) (outScope *scope // Build the final delete statement, including any returned expressions. if resultsNeeded(del.Returning) { - mb.buildDelete(*del.Returning.(*tree.ReturningExprs)) + mb.buildDelete(del.Returning.(*tree.ReturningExprs)) } else { mb.buildDelete(nil /* returning */) } @@ -76,7 +76,7 @@ func (b *Builder) buildDelete(del *tree.Delete, inScope *scope) (outScope *scope // buildDelete constructs a Delete operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildDelete(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildDelete(returning *tree.ReturningExprs) { mb.buildFKChecksAndCascadesForDelete() // Project partial index DEL boolean columns. diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index 5951ab72482b..99cffb0a42e6 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -281,9 +281,9 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // See mutationBuilder.buildCheckInputScan. mb.insertExpr = mb.outScope.expr - var returning tree.ReturningExprs + var returning *tree.ReturningExprs if resultsNeeded(ins.Returning) { - returning = *ins.Returning.(*tree.ReturningExprs) + returning = ins.Returning.(*tree.ReturningExprs) } switch { @@ -669,7 +669,7 @@ func (mb *mutationBuilder) addSynthesizedColsForInsert() { // buildInsert constructs an Insert operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildInsert(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildInsert(returning *tree.ReturningExprs) { // Disambiguate names so that references in any expressions, such as a // check constraint, refer to the correct columns. mb.disambiguateColumns() @@ -874,7 +874,7 @@ func (mb *mutationBuilder) setUpsertCols(insertCols tree.NameList) { // buildUpsert constructs an Upsert operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildUpsert(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildUpsert(returning *tree.ReturningExprs) { // Merge input insert and update columns using CASE expressions. mb.projectUpsertColumns() diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 5b152eed1da8..101164006ae2 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -1019,7 +1019,7 @@ func (mb *mutationBuilder) mapToReturnColID(tabOrd int) opt.ColumnID { // buildReturning wraps the input expression with a Project operator that // projects the given RETURNING expressions. -func (mb *mutationBuilder) buildReturning(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildReturning(returning *tree.ReturningExprs) { // Handle case of no RETURNING clause. if returning == nil { expr := mb.outScope.expr diff --git a/pkg/sql/opt/optbuilder/project.go b/pkg/sql/opt/optbuilder/project.go index a9c9e7622a9c..a925f9e2e8a2 100644 --- a/pkg/sql/opt/optbuilder/project.go +++ b/pkg/sql/opt/optbuilder/project.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // constructProjectForScope constructs a projection if it will result in a @@ -79,7 +80,7 @@ func (b *Builder) dropOrderingAndExtraCols(s *scope) { // and adds the resulting aliases and typed expressions to outScope. See the // header comment for analyzeSelectList. func (b *Builder) analyzeProjectionList( - selects tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, + selects *tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, ) { // We need to save and restore the previous values of the replaceSRFs field // and the field in semaCtx in case we are recursively called within a @@ -98,7 +99,7 @@ func (b *Builder) analyzeProjectionList( // and adds the resulting aliases and typed expressions to outScope. See the // header comment for analyzeSelectList. func (b *Builder) analyzeReturningList( - returning tree.ReturningExprs, desiredTypes []*types.T, inScope, outScope *scope, + returning *tree.ReturningExprs, desiredTypes []*types.T, inScope, outScope *scope, ) { // We need to save and restore the previous value of the field in // semaCtx in case we are recursively called within a subquery @@ -109,7 +110,7 @@ func (b *Builder) analyzeReturningList( b.semaCtx.Properties.Require(exprKindReturning.String(), tree.RejectSpecial) inScope.context = exprKindReturning - b.analyzeSelectList(tree.SelectExprs(returning), desiredTypes, inScope, outScope) + b.analyzeSelectList((*tree.SelectExprs)(returning), desiredTypes, inScope, outScope) } // analyzeSelectList is a helper function used by analyzeProjectionList and @@ -119,10 +120,15 @@ func (b *Builder) analyzeReturningList( // // As a side-effect, the appropriate scopes are updated with aggregations // (scope.groupby.aggs) +// +// If we are building a function, the `selects` expressions will be overwritten +// with expressions that replace any `*` expressions with their columns. func (b *Builder) analyzeSelectList( - selects tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, + selects *tree.SelectExprs, desiredTypes []*types.T, inScope, outScope *scope, ) { - for i, e := range selects { + var expansions tree.SelectExprs + for i, e := range *selects { + expanded := false // Start with fast path, looking for simple column reference. texpr := b.resolveColRef(e.Expr, inScope) if texpr == nil { @@ -142,8 +148,21 @@ func (b *Builder) analyzeSelectList( } aliases, exprs := b.expandStar(e.Expr, inScope) + if b.insideFuncDef { + expanded = true + for _, expr := range exprs { + switch col := expr.(type) { + case *scopeColumn: + expansions = append(expansions, tree.SelectExpr{Expr: tree.NewColumnItem(&col.table, col.name.ReferenceName())}) + case *tree.ColumnAccessExpr: + expansions = append(expansions, tree.SelectExpr{Expr: col}) + default: + panic(errors.AssertionFailedf("unexpected column type in expansion")) + } + } + } if outScope.cols == nil { - outScope.cols = make([]scopeColumn, 0, len(selects)+len(exprs)-1) + outScope.cols = make([]scopeColumn, 0, len(*selects)+len(exprs)-1) } for j, e := range exprs { outScope.addColumn(scopeColName(tree.Name(aliases[j])), e) @@ -164,10 +183,16 @@ func (b *Builder) analyzeSelectList( // have to determine the output column name before we perform type // checking. if outScope.cols == nil { - outScope.cols = make([]scopeColumn, 0, len(selects)) + outScope.cols = make([]scopeColumn, 0, len(*selects)) } alias := b.getColName(e) outScope.addColumn(scopeColName(tree.Name(alias)), texpr) + if b.insideFuncDef && !expanded { + expansions = append(expansions, e) + } + } + if b.insideFuncDef { + *selects = expansions } } diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 9b68a2d9fea3..e0df84f41ead 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -51,7 +51,7 @@ func (b *Builder) buildDataSource( }(inScope.atRoot) inScope.atRoot = false // NB: The case statements are sorted lexicographically. - switch source := texpr.(type) { + switch source := (texpr).(type) { case *tree.AliasedTableExpr: if source.IndexFlags != nil { telemetry.Inc(sqltelemetry.IndexHintUseCounter) @@ -62,6 +62,12 @@ func (b *Builder) buildDataSource( inScope = inScope.push() inScope.alias = &source.As locking = locking.filter(source.As.Alias) + } else if b.insideFuncDef { + // TODO(96375): Allow non-aliased subexpressions in UDFs after ambiguous + // columns can be correctly identified. + if _, ok := source.Expr.(*tree.Subquery); ok { + panic(unimplemented.New("user-defined functions", "unaliased subquery inside a function definition")) + } } outScope = b.buildDataSource(source.Expr, indexFlags, locking, inScope) @@ -110,7 +116,6 @@ func (b *Builder) buildDataSource( } ds, depName, resName := b.resolveDataSource(tn, privilege.SELECT) - locking = locking.filter(tn.ObjectName) if locking.isSet() { // SELECT ... FOR [KEY] UPDATE/SHARE also requires UPDATE privileges. @@ -1052,7 +1057,7 @@ func (b *Builder) buildSelectClause( // function that refers to variables in fromScope or an ancestor scope, // buildAggregateFunction is called which adds columns to the appropriate // aggInScope and aggOutScope. - b.analyzeProjectionList(sel.Exprs, desiredTypes, fromScope, projectionsScope) + b.analyzeProjectionList(&sel.Exprs, desiredTypes, fromScope, projectionsScope) // Any aggregates in the HAVING, ORDER BY and DISTINCT ON clauses (if they // exist) will be added here. diff --git a/pkg/sql/opt/optbuilder/testdata/create_function b/pkg/sql/opt/optbuilder/testdata/create_function index 5ee1e03191e2..6d17e4731778 100644 --- a/pkg/sql/opt/optbuilder/testdata/create_function +++ b/pkg/sql/opt/optbuilder/testdata/create_function @@ -108,9 +108,15 @@ create-function └── s build -CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ SELECT * FROM ab $$ +CREATE FUNCTION f() RETURNS ab LANGUAGE SQL AS $$ SELECT * FROM ab $$ ---- -error (0A000): unimplemented: functions do not currently support * expressions +create-function + ├── CREATE FUNCTION f() + │ RETURNS ab + │ LANGUAGE SQL + │ AS $$SELECT ab.a, ab.b FROM t.public.ab;$$ + └── dependencies + └── ab [columns: a b] build CREATE FUNCTION f() RETURNS INT LANGUAGE SQL BEGIN ATOMIC SELECT 1; END; diff --git a/pkg/sql/opt/optbuilder/testdata/udf b/pkg/sql/opt/optbuilder/testdata/udf index 4f4bc04dfb1a..97c72c00a056 100644 --- a/pkg/sql/opt/optbuilder/testdata/udf +++ b/pkg/sql/opt/optbuilder/testdata/udf @@ -1166,3 +1166,82 @@ project │ └── projections │ └── variable: i:10 [as=i:13] └── const: 1 + +# -------------------------------------------------- +# UDFs with * expressions. +# -------------------------------------------------- + +exec-ddl +CREATE TABLE tstar ( + a INT +) +---- + +exec-ddl +CREATE FUNCTION fn_star() RETURNS INT LANGUAGE SQL AS 'SELECT * FROM tstar' +---- + +build format=show-scalars +SELECT fn_star() +---- +project + ├── columns: fn_star:5 + ├── values + │ └── tuple + └── projections + └── udf: fn_star [as=fn_star:5] + └── body + └── limit + ├── columns: a:1 + ├── project + │ ├── columns: a:1 + │ └── scan tstar + │ └── columns: a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 tableoid:4 + └── const: 1 + +exec-ddl +CREATE TABLE tstar2 ( + a INT, + b INT +) +---- + +exec-ddl +CREATE FUNCTION fn_star2() RETURNS INT LANGUAGE SQL AS 'SELECT * FROM tstar, tstar2 WHERE tstar.a = tstar2.b' +---- + +build format=show-scalars +SELECT fn_star2() +---- +project + ├── columns: fn_star2:11 + ├── values + │ └── tuple + └── projections + └── udf: fn_star2 [as=fn_star2:11] + └── body + └── project + ├── columns: column10:10 + ├── limit + │ ├── columns: tstar.a:1!null tstar2.a:5 b:6!null + │ ├── project + │ │ ├── columns: tstar.a:1!null tstar2.a:5 b:6!null + │ │ └── select + │ │ ├── columns: tstar.a:1!null tstar.rowid:2!null tstar.crdb_internal_mvcc_timestamp:3 tstar.tableoid:4 tstar2.a:5 b:6!null tstar2.rowid:7!null tstar2.crdb_internal_mvcc_timestamp:8 tstar2.tableoid:9 + │ │ ├── inner-join (cross) + │ │ │ ├── columns: tstar.a:1 tstar.rowid:2!null tstar.crdb_internal_mvcc_timestamp:3 tstar.tableoid:4 tstar2.a:5 b:6 tstar2.rowid:7!null tstar2.crdb_internal_mvcc_timestamp:8 tstar2.tableoid:9 + │ │ │ ├── scan tstar + │ │ │ │ └── columns: tstar.a:1 tstar.rowid:2!null tstar.crdb_internal_mvcc_timestamp:3 tstar.tableoid:4 + │ │ │ ├── scan tstar2 + │ │ │ │ └── columns: tstar2.a:5 b:6 tstar2.rowid:7!null tstar2.crdb_internal_mvcc_timestamp:8 tstar2.tableoid:9 + │ │ │ └── filters (true) + │ │ └── filters + │ │ └── eq + │ │ ├── variable: tstar.a:1 + │ │ └── variable: b:6 + │ └── const: 1 + └── projections + └── tuple [as=column10:10] + ├── variable: tstar.a:1 + ├── variable: tstar2.a:5 + └── variable: b:6 diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index 3ec93d7551f7..502e414de34b 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -108,7 +108,7 @@ func (b *Builder) buildUpdate(upd *tree.Update, inScope *scope) (outScope *scope // Build the final update statement, including any returned expressions. if resultsNeeded(upd.Returning) { - mb.buildUpdate(*upd.Returning.(*tree.ReturningExprs)) + mb.buildUpdate(upd.Returning.(*tree.ReturningExprs)) } else { mb.buildUpdate(nil /* returning */) } @@ -326,7 +326,7 @@ func (mb *mutationBuilder) addSynthesizedColsForUpdate() { // buildUpdate constructs an Update operator, possibly wrapped by a Project // operator that corresponds to the given RETURNING clause. -func (mb *mutationBuilder) buildUpdate(returning tree.ReturningExprs) { +func (mb *mutationBuilder) buildUpdate(returning *tree.ReturningExprs) { // Disambiguate names so that references in any expressions, such as a // check constraint, refer to the correct columns. mb.disambiguateColumns() diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index c3f7f68781a0..c210674d376c 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -64,9 +64,6 @@ func (b *Builder) expandStar( if b.insideViewDef { panic(unimplemented.NewWithIssue(10028, "views do not currently support * expressions")) } - if b.insideFuncDef { - panic(unimplemented.NewWithIssue(90080, "functions do not currently support * expressions")) - } switch t := expr.(type) { case *tree.TupleStar: texpr := inScope.resolveType(t.Expr, types.Any) From c91ee210808c54c715a4e9dc8193086edb831bc5 Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Fri, 16 Dec 2022 15:30:31 -0500 Subject: [PATCH 3/8] sql/physicalplan: allow passing replica oracle to planning This refactors span resolver and the physical planning methods to allow passing a replica-choice oracle per planning instead of using a fixed oracle (either bin-packing or closest depending on server type). The existing API that uses the default oracle is kept as is for all existing callers, while a new WithOracle variant is added for future callers who wish to pass their own oracle. A new oracle that prefers followers over leaseholders is also added here with a test, however it is not used at this time. Release note: none. Epic: none. --- pkg/sql/BUILD.bazel | 1 + pkg/sql/backfill.go | 5 +- pkg/sql/distsql_physical_planner.go | 15 +++++- pkg/sql/distsql_physical_planner_test.go | 5 +- pkg/sql/distsql_plan_bulk.go | 38 ++++++++++---- pkg/sql/physicalplan/fake_span_resolver.go | 5 +- .../physicalplan/fake_span_resolver_test.go | 2 +- pkg/sql/physicalplan/replicaoracle/oracle.go | 41 +++++++++++++++ .../physicalplan/replicaoracle/oracle_test.go | 52 +++++++++++++++++-- pkg/sql/physicalplan/span_resolver.go | 17 ++++-- pkg/sql/physicalplan/span_resolver_test.go | 8 +-- 11 files changed, 162 insertions(+), 27 deletions(-) diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index e1efb737b04a..71ab4f8584fc 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -751,6 +751,7 @@ go_test( "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgwirebase", "//pkg/sql/physicalplan", + "//pkg/sql/physicalplan/replicaoracle", "//pkg/sql/privilege", "//pkg/sql/querycache", "//pkg/sql/randgen", diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index e62264454de4..e2031f3035c4 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" @@ -848,7 +849,7 @@ func NumRangesInSpans( ctx context.Context, db *kv.DB, distSQLPlanner *DistSQLPlanner, spans []roachpb.Span, ) (int, error) { txn := db.NewTxn(ctx, "num-ranges-in-spans") - spanResolver := distSQLPlanner.spanResolver.NewSpanResolverIterator(txn) + spanResolver := distSQLPlanner.spanResolver.NewSpanResolverIterator(txn, physicalplan.DefaultReplicaChooser) rangeIds := make(map[int64]struct{}) for _, span := range spans { // For each span, iterate the spanResolver until it's exhausted, storing @@ -882,7 +883,7 @@ func NumRangesInSpanContainedBy( containedBy []roachpb.Span, ) (total, inContainedBy int, _ error) { txn := db.NewTxn(ctx, "num-ranges-in-spans") - spanResolver := distSQLPlanner.spanResolver.NewSpanResolverIterator(txn) + spanResolver := distSQLPlanner.spanResolver.NewSpanResolverIterator(txn, physicalplan.DefaultReplicaChooser) // For each span, iterate the spanResolver until it's exhausted, storing // the found range ids in the map to de-duplicate them. spanResolver.Seek(ctx, outerSpan, kvcoord.Ascending) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index a3ec32a78526..fd6e754449d1 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -4385,6 +4385,19 @@ func (dsp *DistSQLPlanner) NewPlanningCtx( planner *planner, txn *kv.Txn, distributionType DistributionType, +) *PlanningCtx { + return dsp.NewPlanningCtxWithOracle(ctx, evalCtx, planner, txn, distributionType, physicalplan.DefaultReplicaChooser) +} + +// NewPlanningCtxWithOracle is a variant of NewPlanningCtx that allows passing a +// replica choice oracle as well. +func (dsp *DistSQLPlanner) NewPlanningCtxWithOracle( + ctx context.Context, + evalCtx *extendedEvalContext, + planner *planner, + txn *kv.Txn, + distributionType DistributionType, + oracle replicaoracle.Oracle, ) *PlanningCtx { distribute := distributionType == DistributionTypeAlways || (distributionType == DistributionTypeSystemTenantOnly && evalCtx.Codec.ForSystemTenant()) infra := physicalplan.NewPhysicalInfrastructure(uuid.FastMakeV4(), dsp.gatewaySQLInstanceID) @@ -4419,7 +4432,7 @@ func (dsp *DistSQLPlanner) NewPlanningCtx( // we still need to instantiate a full planning context. planCtx.parallelizeScansIfLocal = true } - planCtx.spanIter = dsp.spanResolver.NewSpanResolverIterator(txn) + planCtx.spanIter = dsp.spanResolver.NewSpanResolverIterator(txn, oracle) planCtx.nodeStatuses = make(map[base.SQLInstanceID]NodeStatus) planCtx.nodeStatuses[dsp.gatewaySQLInstanceID] = NodeOK return planCtx diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 15c208d681bb..29ae77741be0 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle" "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -577,7 +578,9 @@ type testSpanResolver struct { } // NewSpanResolverIterator is part of the SpanResolver interface. -func (tsr *testSpanResolver) NewSpanResolverIterator(_ *kv.Txn) physicalplan.SpanResolverIterator { +func (tsr *testSpanResolver) NewSpanResolverIterator( + txn *kv.Txn, optionalOracle replicaoracle.Oracle, +) physicalplan.SpanResolverIterator { return &testSpanResolverIterator{tsr: tsr} } diff --git a/pkg/sql/distsql_plan_bulk.go b/pkg/sql/distsql_plan_bulk.go index ae5f98fbe54d..8323b9d6e6c1 100644 --- a/pkg/sql/distsql_plan_bulk.go +++ b/pkg/sql/distsql_plan_bulk.go @@ -16,6 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -24,20 +26,34 @@ import ( // map for all nodes. It returns all nodes that can be used for planning. func (dsp *DistSQLPlanner) SetupAllNodesPlanning( ctx context.Context, evalCtx *extendedEvalContext, execCfg *ExecutorConfig, +) (*PlanningCtx, []base.SQLInstanceID, error) { + return dsp.SetupAllNodesPlanningWithOracle(ctx, evalCtx, execCfg, physicalplan.DefaultReplicaChooser) +} + +// SetupAllNodesPlanning creates a planCtx and sets up the planCtx.nodeStatuses +// map for all nodes. It returns all nodes that can be used for planning. +func (dsp *DistSQLPlanner) SetupAllNodesPlanningWithOracle( + ctx context.Context, + evalCtx *extendedEvalContext, + execCfg *ExecutorConfig, + oracle replicaoracle.Oracle, ) (*PlanningCtx, []base.SQLInstanceID, error) { if dsp.codec.ForSystemTenant() { - return dsp.setupAllNodesPlanningSystem(ctx, evalCtx, execCfg) + return dsp.setupAllNodesPlanningSystem(ctx, evalCtx, execCfg, oracle) } - return dsp.setupAllNodesPlanningTenant(ctx, evalCtx, execCfg) + return dsp.setupAllNodesPlanningTenant(ctx, evalCtx, execCfg, oracle) } // setupAllNodesPlanningSystem creates a planCtx and returns all nodes available // in a system tenant. func (dsp *DistSQLPlanner) setupAllNodesPlanningSystem( - ctx context.Context, evalCtx *extendedEvalContext, execCfg *ExecutorConfig, + ctx context.Context, + evalCtx *extendedEvalContext, + execCfg *ExecutorConfig, + oracle replicaoracle.Oracle, ) (*PlanningCtx, []base.SQLInstanceID, error) { - planCtx := dsp.NewPlanningCtx(ctx, evalCtx, nil /* planner */, nil, /* txn */ - DistributionTypeAlways) + planCtx := dsp.NewPlanningCtxWithOracle(ctx, evalCtx, nil /* planner */, nil, /* txn */ + DistributionTypeAlways, oracle) ss, err := execCfg.NodesStatusServer.OptionalNodesStatusServer(47900) if err != nil { @@ -65,13 +81,13 @@ func (dsp *DistSQLPlanner) setupAllNodesPlanningSystem( // setupAllNodesPlanningTenant creates a planCtx and returns all nodes available // in a non-system tenant. func (dsp *DistSQLPlanner) setupAllNodesPlanningTenant( - ctx context.Context, evalCtx *extendedEvalContext, execCfg *ExecutorConfig, + ctx context.Context, + evalCtx *extendedEvalContext, + execCfg *ExecutorConfig, + oracle replicaoracle.Oracle, ) (*PlanningCtx, []base.SQLInstanceID, error) { - if dsp.sqlAddressResolver == nil { - return nil, nil, errors.New("sql instance provider not available in multi-tenant environment") - } - planCtx := dsp.NewPlanningCtx(ctx, evalCtx, nil /* planner */, nil, /* txn */ - DistributionTypeAlways) + planCtx := dsp.NewPlanningCtxWithOracle(ctx, evalCtx, nil /* planner */, nil, /* txn */ + DistributionTypeAlways, oracle) pods, err := dsp.sqlAddressResolver.GetAllInstances(ctx) if err != nil { return nil, nil, err diff --git a/pkg/sql/physicalplan/fake_span_resolver.go b/pkg/sql/physicalplan/fake_span_resolver.go index a4edc005e684..0da34b939410 100644 --- a/pkg/sql/physicalplan/fake_span_resolver.go +++ b/pkg/sql/physicalplan/fake_span_resolver.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) @@ -65,7 +66,9 @@ type fakeSpanResolverIterator struct { } // NewSpanResolverIterator is part of the SpanResolver interface. -func (fsr *fakeSpanResolver) NewSpanResolverIterator(txn *kv.Txn) SpanResolverIterator { +func (fsr *fakeSpanResolver) NewSpanResolverIterator( + txn *kv.Txn, optionalOracle replicaoracle.Oracle, +) SpanResolverIterator { rng, _ := randutil.NewTestRand() return &fakeSpanResolverIterator{fsr: fsr, db: txn.DB(), rng: rng} } diff --git a/pkg/sql/physicalplan/fake_span_resolver_test.go b/pkg/sql/physicalplan/fake_span_resolver_test.go index 153be25e2671..f690c9473c53 100644 --- a/pkg/sql/physicalplan/fake_span_resolver_test.go +++ b/pkg/sql/physicalplan/fake_span_resolver_test.go @@ -54,7 +54,7 @@ func TestFakeSpanResolver(t *testing.T) { db := tc.Server(0).DB() txn := kv.NewTxn(ctx, db, tc.Server(0).NodeID()) - it := resolver.NewSpanResolverIterator(txn) + it := resolver.NewSpanResolverIterator(txn, nil) tableDesc := desctestutils.TestingGetPublicTableDescriptor(db, keys.SystemSQLCodec, "test", "t") primIdxValDirs := catalogkeys.IndexKeyValDirs(tableDesc.GetPrimaryIndex()) diff --git a/pkg/sql/physicalplan/replicaoracle/oracle.go b/pkg/sql/physicalplan/replicaoracle/oracle.go index 929df9b8e6c4..bfea46a30318 100644 --- a/pkg/sql/physicalplan/replicaoracle/oracle.go +++ b/pkg/sql/physicalplan/replicaoracle/oracle.go @@ -16,6 +16,7 @@ import ( "context" "math" "math/rand" + "sort" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" @@ -38,6 +39,8 @@ var ( BinPackingChoice = RegisterPolicy(newBinPackingOracle) // ClosestChoice chooses the node closest to the current node. ClosestChoice = RegisterPolicy(newClosestOracle) + // PreferFollowerChoice prefers choosing followers over leaseholders. + PreferFollowerChoice = RegisterPolicy(newPreferFollowerOracle) ) // Config is used to construct an OracleFactory. @@ -288,3 +291,41 @@ func latencyFunc(rpcCtx *rpc.Context) kvcoord.LatencyFunc { } return nil } + +type preferFollowerOracle struct { + nodeDescs kvcoord.NodeDescStore +} + +func newPreferFollowerOracle(cfg Config) Oracle { + return &preferFollowerOracle{nodeDescs: cfg.NodeDescs} +} + +func (o preferFollowerOracle) ChoosePreferredReplica( + ctx context.Context, + _ *kv.Txn, + desc *roachpb.RangeDescriptor, + _ *roachpb.ReplicaDescriptor, + _ roachpb.RangeClosedTimestampPolicy, + _ QueryState, +) (roachpb.ReplicaDescriptor, error) { + replicas, err := replicaSliceOrErr(ctx, o.nodeDescs, desc, kvcoord.AllExtantReplicas) + if err != nil { + return roachpb.ReplicaDescriptor{}, err + } + + leaseholders, err := replicaSliceOrErr(ctx, o.nodeDescs, desc, kvcoord.OnlyPotentialLeaseholders) + if err != nil { + return roachpb.ReplicaDescriptor{}, err + } + leaseholderNodeIDs := make(map[roachpb.NodeID]bool, len(leaseholders)) + for i := range leaseholders { + leaseholderNodeIDs[leaseholders[i].NodeID] = true + } + + sort.Slice(replicas, func(i, j int) bool { + return !leaseholderNodeIDs[replicas[i].NodeID] && leaseholderNodeIDs[replicas[j].NodeID] + }) + + // TODO: Pick a random replica from replicas[:len(replicas)-len(leaseholders)] + return replicas[0].ReplicaDescriptor, nil +} diff --git a/pkg/sql/physicalplan/replicaoracle/oracle_test.go b/pkg/sql/physicalplan/replicaoracle/oracle_test.go index d52d57f02478..90f36bf98d18 100644 --- a/pkg/sql/physicalplan/replicaoracle/oracle_test.go +++ b/pkg/sql/physicalplan/replicaoracle/oracle_test.go @@ -41,7 +41,7 @@ func TestClosest(t *testing.T) { ctx := context.Background() stopper := stop.NewStopper() defer stopper.Stop(ctx) - g, _ := makeGossip(t, stopper) + g, _ := makeGossip(t, stopper, []int{2, 3}) nd2, err := g.GetNodeDescriptor(2) require.NoError(t, err) o := NewOracle(ClosestChoice, Config{ @@ -82,7 +82,7 @@ func TestClosest(t *testing.T) { }) } -func makeGossip(t *testing.T, stopper *stop.Stopper) (*gossip.Gossip, *hlc.Clock) { +func makeGossip(t *testing.T, stopper *stop.Stopper, nodeIDs []int) (*gossip.Gossip, *hlc.Clock) { clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */) const nodeID = 1 @@ -93,7 +93,8 @@ func makeGossip(t *testing.T, stopper *stop.Stopper) (*gossip.Gossip, *hlc.Clock if err := g.AddInfo(gossip.KeySentinel, nil, time.Hour); err != nil { t.Fatal(err) } - for i := roachpb.NodeID(2); i <= 3; i++ { + for _, id := range nodeIDs { + i := roachpb.NodeID(id) err := g.AddInfoProto(gossip.MakeNodeIDKey(i), newNodeDesc(i), gossip.NodeDescriptorTTL) if err != nil { t.Fatal(err) @@ -116,3 +117,48 @@ func newNodeDesc(nodeID roachpb.NodeID) *roachpb.NodeDescriptor { }, } } + +func TestPreferFollower(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + g, _ := makeGossip(t, stopper, []int{2, 3, 4, 5, 6}) + o := NewOracle(PreferFollowerChoice, Config{ + NodeDescs: g, + }) + internalReplicas := []roachpb.ReplicaDescriptor{ + {ReplicaID: 2, NodeID: 2, StoreID: 2, Type: roachpb.VOTER_FULL}, + {ReplicaID: 3, NodeID: 3, StoreID: 3, Type: roachpb.VOTER_FULL}, + {ReplicaID: 4, NodeID: 4, StoreID: 4, Type: roachpb.VOTER_FULL}, + {ReplicaID: 5, NodeID: 5, StoreID: 5, Type: roachpb.NON_VOTER}, + {ReplicaID: 6, NodeID: 6, StoreID: 6, Type: roachpb.NON_VOTER}, + } + rand.Shuffle(len(internalReplicas), func(i, j int) { + internalReplicas[i], internalReplicas[j] = internalReplicas[j], internalReplicas[i] + }) + info, err := o.ChoosePreferredReplica( + ctx, + nil, /* txn */ + &roachpb.RangeDescriptor{ + InternalReplicas: internalReplicas, + }, + nil, /* leaseHolder */ + roachpb.LAG_BY_CLUSTER_SETTING, + QueryState{}, + ) + if err != nil { + t.Fatalf("Failed to choose follower replica: %v", err) + } + + fullVoters := make(map[roachpb.NodeID]bool) + for _, r := range internalReplicas { + if r.Type == roachpb.VOTER_FULL { + fullVoters[r.NodeID] = true + } + } + + if fullVoters[info.NodeID] { + t.Fatalf("Chose a VOTER_FULL replica: %d", info.NodeID) + } +} diff --git a/pkg/sql/physicalplan/span_resolver.go b/pkg/sql/physicalplan/span_resolver.go index e876940701e6..ca777250bdf2 100644 --- a/pkg/sql/physicalplan/span_resolver.go +++ b/pkg/sql/physicalplan/span_resolver.go @@ -63,9 +63,14 @@ import ( type SpanResolver interface { // NewSpanResolverIterator creates a new SpanResolverIterator. // Txn is used for testing and for determining if follower reads are possible. - NewSpanResolverIterator(txn *kv.Txn) SpanResolverIterator + NewSpanResolverIterator(txn *kv.Txn, optionalOracle replicaoracle.Oracle) SpanResolverIterator } +// DefaultReplicaChooser is a nil replicaoracle.Oracle which can be passed in +// place of a replica oracle to some APIs to indicate they can use their default +// replica oracle. +var DefaultReplicaChooser replicaoracle.Oracle + // SpanResolverIterator is used to iterate over the ranges composing a key span. type SpanResolverIterator interface { // Seek positions the iterator on the start of a span (span.Key or @@ -170,11 +175,17 @@ type spanResolverIterator struct { var _ SpanResolverIterator = &spanResolverIterator{} // NewSpanResolverIterator creates a new SpanResolverIterator. -func (sr *spanResolver) NewSpanResolverIterator(txn *kv.Txn) SpanResolverIterator { +func (sr *spanResolver) NewSpanResolverIterator( + txn *kv.Txn, optionalOracle replicaoracle.Oracle, +) SpanResolverIterator { + oracle := optionalOracle + if optionalOracle == nil { + oracle = sr.oracle + } return &spanResolverIterator{ txn: txn, it: kvcoord.MakeRangeIterator(sr.distSender), - oracle: sr.oracle, + oracle: oracle, queryState: replicaoracle.MakeQueryState(), } } diff --git a/pkg/sql/physicalplan/span_resolver_test.go b/pkg/sql/physicalplan/span_resolver_test.go index 7ab53881ea7b..1f17aa01ae13 100644 --- a/pkg/sql/physicalplan/span_resolver_test.go +++ b/pkg/sql/physicalplan/span_resolver_test.go @@ -111,7 +111,7 @@ func TestSpanResolverUsesCaches(t *testing.T) { // Resolve the spans. Since the range descriptor cache doesn't have any // leases, all the ranges should be grouped and "assigned" to replica 0. - replicas, err := resolveSpans(context.Background(), lr.NewSpanResolverIterator(nil), spans...) + replicas, err := resolveSpans(context.Background(), lr.NewSpanResolverIterator(nil, nil), spans...) if err != nil { t.Fatal(err) } @@ -138,7 +138,7 @@ func TestSpanResolverUsesCaches(t *testing.T) { if err := populateCache(tc.Conns[3], 3 /* expectedNumRows */); err != nil { t.Fatal(err) } - replicas, err = resolveSpans(context.Background(), lr.NewSpanResolverIterator(nil), spans...) + replicas, err = resolveSpans(context.Background(), lr.NewSpanResolverIterator(nil, nil), spans...) if err != nil { t.Fatal(err) } @@ -209,7 +209,7 @@ func TestSpanResolver(t *testing.T) { replicaoracle.BinPackingChoice) ctx := context.Background() - it := lr.NewSpanResolverIterator(nil) + it := lr.NewSpanResolverIterator(nil, nil) testCases := []struct { spans []roachpb.Span @@ -308,7 +308,7 @@ func TestMixedDirections(t *testing.T) { replicaoracle.BinPackingChoice) ctx := context.Background() - it := lr.NewSpanResolverIterator(nil) + it := lr.NewSpanResolverIterator(nil, nil) spans := []spanWithDir{ orient(kvcoord.Ascending, makeSpan(tableDesc, 11, 15))[0], From 1398eb80f6872701eaeca93fea9999d196b5faea Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 19 Jan 2023 10:07:19 +0100 Subject: [PATCH 4/8] storage: add string repr for FullReplicaID Helpful for subsequent unit testing involving this type. NB: this type is in the wrong place. Now's not the time to move it. Epic: none Release note: None --- pkg/storage/BUILD.bazel | 1 + pkg/storage/replicas_storage.go | 13 +++++++++++++ 2 files changed, 14 insertions(+) diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index d5a383a7ce68..d28813935a27 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -91,6 +91,7 @@ go_library( "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_pebble//vfs/atomicfs", "@com_github_cockroachdb_redact//:redact", + "@com_github_cockroachdb_redact//interfaces", "@com_github_dustin_go_humanize//:go-humanize", "@com_github_elastic_gosigar//:gosigar", "@com_github_gogo_protobuf//proto", diff --git a/pkg/storage/replicas_storage.go b/pkg/storage/replicas_storage.go index 17953e84cafc..05d42e38fd4d 100644 --- a/pkg/storage/replicas_storage.go +++ b/pkg/storage/replicas_storage.go @@ -12,6 +12,8 @@ package storage import ( "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/redact" + "github.com/cockroachdb/redact/interfaces" "go.etcd.io/raft/v3/raftpb" ) @@ -540,6 +542,17 @@ type FullReplicaID struct { ReplicaID roachpb.ReplicaID } +// SafeFormat implements redact.SafeFormatter. It prints as +// r/. +func (id FullReplicaID) SafeFormat(s interfaces.SafePrinter, _ rune) { + s.Printf("r%d/%d", id.RangeID, id.ReplicaID) +} + +// String formats a store for debug output. +func (id FullReplicaID) String() string { + return redact.StringWithoutMarkers(id) +} + // ReplicaInfo provides the replica ID and state pair. type ReplicaInfo struct { FullReplicaID From 877fd63ff5126d21c771015758ebd4a24b41a02d Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 18 Jan 2023 15:45:15 +0100 Subject: [PATCH 5/8] kvserver,kvstorage: move read/write methods for cluster versions Same rationale as #95432 - they belong in `kvstorage` and I need them there now as I'm working on a datadriven test. Touches #93310. Epic: CRDB-220 Release note: None --- pkg/BUILD.bazel | 2 + pkg/kv/kvserver/kvstorage/BUILD.bazel | 19 +- pkg/kv/kvserver/kvstorage/cluster_version.go | 188 ++++++++++++++++++ .../kvstorage/cluster_version_test.go | 84 ++++++++ pkg/kv/kvserver/store.go | 51 ----- pkg/kv/kvserver/store_test.go | 4 +- pkg/kv/kvserver/stores.go | 115 ----------- pkg/kv/kvserver/stores_test.go | 82 ++------ pkg/server/init.go | 7 +- pkg/server/migration.go | 5 +- pkg/server/migration_test.go | 3 +- pkg/server/node.go | 4 +- pkg/server/node_test.go | 5 +- pkg/server/node_tombstone_storage_test.go | 3 +- pkg/server/server.go | 5 +- pkg/server/version_cluster_test.go | 5 +- .../localtestcluster/local_test_cluster.go | 2 +- 17 files changed, 327 insertions(+), 257 deletions(-) create mode 100644 pkg/kv/kvserver/kvstorage/cluster_version.go create mode 100644 pkg/kv/kvserver/kvstorage/cluster_version_test.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index f4b8f4df8e0e..d6b6266b4e1c 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -204,6 +204,7 @@ ALL_TESTS = [ "//pkg/kv/kvserver/gc:gc_test", "//pkg/kv/kvserver/idalloc:idalloc_test", "//pkg/kv/kvserver/intentresolver:intentresolver_test", + "//pkg/kv/kvserver/kvstorage:kvstorage_test", "//pkg/kv/kvserver/liveness:liveness_test", "//pkg/kv/kvserver/logstore:logstore_test", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_test", @@ -1219,6 +1220,7 @@ GO_TARGETS = [ "//pkg/kv/kvserver/kvserverbase:kvserverbase", "//pkg/kv/kvserver/kvserverpb:kvserverpb", "//pkg/kv/kvserver/kvstorage:kvstorage", + "//pkg/kv/kvserver/kvstorage:kvstorage_test", "//pkg/kv/kvserver/liveness/livenesspb:livenesspb", "//pkg/kv/kvserver/liveness:liveness", "//pkg/kv/kvserver/liveness:liveness_test", diff --git a/pkg/kv/kvserver/kvstorage/BUILD.bazel b/pkg/kv/kvserver/kvstorage/BUILD.bazel index 110f33cf9416..e1911fe3b31d 100644 --- a/pkg/kv/kvserver/kvstorage/BUILD.bazel +++ b/pkg/kv/kvserver/kvstorage/BUILD.bazel @@ -1,15 +1,17 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "kvstorage", srcs = [ + "cluster_version.go", "doc.go", "init.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/keys", "//pkg/kv/kvserver/logstore", "//pkg/roachpb", @@ -23,4 +25,19 @@ go_library( ], ) +go_test( + name = "kvstorage_test", + srcs = ["cluster_version_test.go"], + args = ["-test.timeout=295s"], + embed = [":kvstorage"], + deps = [ + "//pkg/clusterversion", + "//pkg/roachpb", + "//pkg/storage", + "//pkg/testutils", + "//pkg/util/leaktest", + "//pkg/util/log", + ], +) + get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/kvstorage/cluster_version.go b/pkg/kv/kvserver/kvstorage/cluster_version.go new file mode 100644 index 000000000000..6aa10e9e46d7 --- /dev/null +++ b/pkg/kv/kvserver/kvstorage/cluster_version.go @@ -0,0 +1,188 @@ +// Copyright 2023 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 kvstorage + +import ( + "context" + "fmt" + "math" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// WriteClusterVersion writes the given cluster version to the store-local +// cluster version key. We only accept a raw engine to ensure we're persisting +// the write durably. +func WriteClusterVersion( + ctx context.Context, eng storage.Engine, cv clusterversion.ClusterVersion, +) error { + err := storage.MVCCPutProto( + ctx, + eng, + nil, + keys.StoreClusterVersionKey(), + hlc.Timestamp{}, + hlc.ClockTimestamp{}, + nil, + &cv, + ) + if err != nil { + return err + } + + // The storage engine sometimes must make backwards incompatible + // changes. However, the store cluster version key is a key stored + // within the storage engine, so it's unavailable when the store is + // opened. + // + // The storage engine maintains its own minimum version on disk that + // it may consult it before opening the Engine. This version is + // stored in a separate file on the filesystem. For now, write to + // this file in combination with the store cluster version key. + // + // This parallel version state is a bit of a wart and an eventual + // goal is to replace the store cluster version key with the storage + // engine's flat file. This requires that there are no writes to the + // engine until either bootstrapping or joining an existing cluster. + // Writing the version to this file would happen before opening the + // engine for completing the rest of bootstrapping/joining the + // cluster. + return eng.SetMinVersion(cv.Version) +} + +// ReadClusterVersion reads the cluster version from the store-local version +// key. Returns an empty version if the key is not found. +func ReadClusterVersion( + ctx context.Context, reader storage.Reader, +) (clusterversion.ClusterVersion, error) { + var cv clusterversion.ClusterVersion + _, err := storage.MVCCGetProto(ctx, reader, keys.StoreClusterVersionKey(), hlc.Timestamp{}, + &cv, storage.MVCCGetOptions{}) + return cv, err +} + +// WriteClusterVersionToEngines writes the given version to the given engines, +// Returns nil on success; otherwise returns first error encountered writing to +// the stores. It makes no attempt to validate the supplied version. +// +// At the time of writing this is used during bootstrap, initial server start +// (to perhaps fill into additional stores), and during cluster version bumps. +func WriteClusterVersionToEngines( + ctx context.Context, engines []storage.Engine, cv clusterversion.ClusterVersion, +) error { + for _, eng := range engines { + if err := WriteClusterVersion(ctx, eng, cv); err != nil { + return errors.Wrapf(err, "error writing version to engine %s", eng) + } + } + return nil +} + +// SynthesizeClusterVersionFromEngines returns the cluster version that was read +// from the engines or, if none are initialized, binaryMinSupportedVersion. +// Typically all initialized engines will have the same version persisted, +// though ill-timed crashes can result in situations where this is not the +// case. Then, the largest version seen is returned. +// +// binaryVersion is the version of this binary. An error is returned if +// any engine has a higher version, as this would indicate that this node +// has previously acked the higher cluster version but is now running an +// old binary, which is unsafe. +// +// binaryMinSupportedVersion is the minimum version supported by this binary. An +// error is returned if any engine has a version lower that this. +func SynthesizeClusterVersionFromEngines( + ctx context.Context, + engines []storage.Engine, + binaryVersion, binaryMinSupportedVersion roachpb.Version, +) (clusterversion.ClusterVersion, error) { + // Find the most recent bootstrap info. + type originVersion struct { + roachpb.Version + origin string + } + + maxPossibleVersion := roachpb.Version{Major: math.MaxInt32} // Sort above any real version. + minStoreVersion := originVersion{ + Version: maxPossibleVersion, + origin: "(no store)", + } + + // We run this twice because it's only after having seen all the versions + // that we can decide whether the node catches a version error. However, we + // also want to name at least one engine that violates the version + // constraints, which at the latest the second loop will achieve (because + // then minStoreVersion don't change any more). + for _, eng := range engines { + eng := eng.(storage.Reader) // we're read only + var cv clusterversion.ClusterVersion + cv, err := ReadClusterVersion(ctx, eng) + if err != nil { + return clusterversion.ClusterVersion{}, err + } + if cv.Version == (roachpb.Version{}) { + // This is needed when a node first joins an existing cluster, in + // which case it won't know what version to use until the first + // Gossip update comes in. + cv.Version = binaryMinSupportedVersion + } + + // Avoid running a binary with a store that is too new. For example, + // restarting into 1.1 after having upgraded to 1.2 doesn't work. + if binaryVersion.Less(cv.Version) { + return clusterversion.ClusterVersion{}, errors.Errorf( + "cockroach version v%s is incompatible with data in store %s; use version v%s or later", + binaryVersion, eng, cv.Version) + } + + // Track smallest use version encountered. + if cv.Version.Less(minStoreVersion.Version) { + minStoreVersion.Version = cv.Version + minStoreVersion.origin = fmt.Sprint(eng) + } + } + + // If no use version was found, fall back to our binaryMinSupportedVersion. This + // is the case when a brand new node is joining an existing cluster (which + // may be on any older version this binary supports). + if minStoreVersion.Version == maxPossibleVersion { + minStoreVersion.Version = binaryMinSupportedVersion + } + + cv := clusterversion.ClusterVersion{ + Version: minStoreVersion.Version, + } + log.Eventf(ctx, "read clusterVersion %+v", cv) + + // Avoid running a binary too new for this store. This is what you'd catch + // if, say, you restarted directly from 1.0 into 1.2 (bumping the min + // version) without going through 1.1 first. It would also be what you catch if + // you are starting 1.1 for the first time (after 1.0), but it crashes + // half-way through the startup sequence (so now some stores have 1.1, but + // some 1.0), in which case you are expected to run 1.1 again (hopefully + // without the crash this time) which would then rewrite all the stores. + // + // We only verify this now because as we iterate through the stores, we + // may not yet have picked up the final versions we're actually planning + // to use. + if minStoreVersion.Version.Less(binaryMinSupportedVersion) { + return clusterversion.ClusterVersion{}, errors.Errorf("store %s, last used with cockroach version v%s, "+ + "is too old for running version v%s (which requires data from v%s or later)", + minStoreVersion.origin, minStoreVersion.Version, binaryVersion, binaryMinSupportedVersion) + } + return cv, nil +} diff --git a/pkg/kv/kvserver/kvstorage/cluster_version_test.go b/pkg/kv/kvserver/kvstorage/cluster_version_test.go new file mode 100644 index 000000000000..c800907915d5 --- /dev/null +++ b/pkg/kv/kvserver/kvstorage/cluster_version_test.go @@ -0,0 +1,84 @@ +// Copyright 2023 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 kvstorage + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// TestStoresClusterVersionIncompatible verifies an error occurs when +// setting up the cluster version from stores that are incompatible with the +// running binary. +func TestStoresClusterVersionIncompatible(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + vOneDashOne := roachpb.Version{Major: 1, Internal: 1} + vOne := roachpb.Version{Major: 1} + + type testCase struct { + binV, minV roachpb.Version // binary version and min supported version + engV roachpb.Version // version found on engine in test + expErr string + } + for name, tc := range map[string]testCase{ + "StoreTooNew": { + // This is what the node is running. + binV: vOneDashOne, + // This is what the running node requires from its stores. + minV: vOne, + // Version is way too high for this node. + engV: roachpb.Version{Major: 9}, + expErr: `cockroach version v1\.0-1 is incompatible with data in store =; use version v9\.0 or later`, + }, + "StoreTooOldVersion": { + // This is what the node is running. + binV: roachpb.Version{Major: 9}, + // This is what the running node requires from its stores. + minV: roachpb.Version{Major: 5}, + // Version is way too low. + engV: roachpb.Version{Major: 4}, + expErr: `store =, last used with cockroach version v4\.0, is too old for running version v9\.0 \(which requires data from v5\.0 or later\)`, + }, + "StoreTooOldMinVersion": { + // Like the previous test case, but this time cv.MinimumVersion is the culprit. + binV: roachpb.Version{Major: 9}, + minV: roachpb.Version{Major: 5}, + engV: roachpb.Version{Major: 4}, + expErr: `store =, last used with cockroach version v4\.0, is too old for running version v9\.0 \(which requires data from v5\.0 or later\)`, + }, + } { + t.Run(name, func(t *testing.T) { + engs := []storage.Engine{storage.NewDefaultInMemForTesting()} + defer engs[0].Close() + // Configure versions and write. + cv := clusterversion.ClusterVersion{Version: tc.engV} + if err := WriteClusterVersionToEngines(ctx, engs, cv); err != nil { + t.Fatal(err) + } + if cv, err := SynthesizeClusterVersionFromEngines( + ctx, engs, tc.binV, tc.minV, + ); !testutils.IsError(err, tc.expErr) { + t.Fatalf("unexpected error: %+v, got version %v", err, cv) + } + }) + } +} diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index e9c90106b427..07739c9b45d7 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -3440,57 +3440,6 @@ func (s *storeForTruncatorImpl) getEngine() storage.Engine { return (*Store)(s).engine } -// WriteClusterVersion writes the given cluster version to the store-local -// cluster version key. We only accept a raw engine to ensure we're persisting -// the write durably. -func WriteClusterVersion( - ctx context.Context, eng storage.Engine, cv clusterversion.ClusterVersion, -) error { - err := storage.MVCCPutProto( - ctx, - eng, - nil, - keys.StoreClusterVersionKey(), - hlc.Timestamp{}, - hlc.ClockTimestamp{}, - nil, - &cv, - ) - if err != nil { - return err - } - - // The storage engine sometimes must make backwards incompatible - // changes. However, the store cluster version key is a key stored - // within the storage engine, so it's unavailable when the store is - // opened. - // - // The storage engine maintains its own minimum version on disk that - // it may consult it before opening the Engine. This version is - // stored in a separate file on the filesystem. For now, write to - // this file in combination with the store cluster version key. - // - // This parallel version state is a bit of a wart and an eventual - // goal is to replace the store cluster version key with the storage - // engine's flat file. This requires that there are no writes to the - // engine until either bootstrapping or joining an existing cluster. - // Writing the version to this file would happen before opening the - // engine for completing the rest of bootstrapping/joining the - // cluster. - return eng.SetMinVersion(cv.Version) -} - -// ReadClusterVersion reads the cluster version from the store-local version -// key. Returns an empty version if the key is not found. -func ReadClusterVersion( - ctx context.Context, reader storage.Reader, -) (clusterversion.ClusterVersion, error) { - var cv clusterversion.ClusterVersion - _, err := storage.MVCCGetProto(ctx, reader, keys.StoreClusterVersionKey(), hlc.Timestamp{}, - &cv, storage.MVCCGetOptions{}) - return cv, err -} - func init() { tracing.RegisterTagRemapping("s", "store") } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 31ce2b1b5071..b1c46587a7ff 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -263,7 +263,7 @@ func createTestStoreWithoutStart( if opts.bootstrapVersion != (roachpb.Version{}) { cv = clusterversion.ClusterVersion{Version: opts.bootstrapVersion} } - require.NoError(t, WriteClusterVersion(ctx, eng, cv)) + require.NoError(t, kvstorage.WriteClusterVersion(ctx, eng, cv)) if err := kvstorage.InitEngine( ctx, eng, storeIdent, ); err != nil { @@ -493,7 +493,7 @@ func TestInitializeEngineErrors(t *testing.T) { err := kvstorage.InitEngine(ctx, eng, testIdent) require.ErrorContains(t, err, "no cluster version") - require.NoError(t, WriteClusterVersion(ctx, eng, clusterversion.TestingClusterVersion)) + require.NoError(t, kvstorage.WriteClusterVersion(ctx, eng, clusterversion.TestingClusterVersion)) // Put some random garbage into the engine. require.NoError(t, eng.PutUnversioned(roachpb.Key("foo"), []byte("bar"))) diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index 13f9bbff62ae..4b7310bfbd1f 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -13,10 +13,8 @@ package kvserver import ( "context" "fmt" - math "math" "unsafe" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -28,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/errors" ) // Stores provides methods to access a collection of stores. There's @@ -300,118 +297,6 @@ func (ls *Stores) updateBootstrapInfoLocked(bi *gossip.BootstrapInfo) error { return err } -// WriteClusterVersionToEngines writes the given version to the given engines, -// Returns nil on success; otherwise returns first error encountered writing to -// the stores. It makes no attempt to validate the supplied version. -// -// At the time of writing this is used during bootstrap, initial server start -// (to perhaps fill into additional stores), and during cluster version bumps. -func WriteClusterVersionToEngines( - ctx context.Context, engines []storage.Engine, cv clusterversion.ClusterVersion, -) error { - for _, eng := range engines { - if err := WriteClusterVersion(ctx, eng, cv); err != nil { - return errors.Wrapf(err, "error writing version to engine %s", eng) - } - } - return nil -} - -// SynthesizeClusterVersionFromEngines returns the cluster version that was read -// from the engines or, if none are initialized, binaryMinSupportedVersion. -// Typically all initialized engines will have the same version persisted, -// though ill-timed crashes can result in situations where this is not the -// case. Then, the largest version seen is returned. -// -// binaryVersion is the version of this binary. An error is returned if -// any engine has a higher version, as this would indicate that this node -// has previously acked the higher cluster version but is now running an -// old binary, which is unsafe. -// -// binaryMinSupportedVersion is the minimum version supported by this binary. An -// error is returned if any engine has a version lower that this. -func SynthesizeClusterVersionFromEngines( - ctx context.Context, - engines []storage.Engine, - binaryVersion, binaryMinSupportedVersion roachpb.Version, -) (clusterversion.ClusterVersion, error) { - // Find the most recent bootstrap info. - type originVersion struct { - roachpb.Version - origin string - } - - maxPossibleVersion := roachpb.Version{Major: math.MaxInt32} // Sort above any real version. - minStoreVersion := originVersion{ - Version: maxPossibleVersion, - origin: "(no store)", - } - - // We run this twice because it's only after having seen all the versions - // that we can decide whether the node catches a version error. However, we - // also want to name at least one engine that violates the version - // constraints, which at the latest the second loop will achieve (because - // then minStoreVersion don't change any more). - for _, eng := range engines { - eng := eng.(storage.Reader) // we're read only - var cv clusterversion.ClusterVersion - cv, err := ReadClusterVersion(ctx, eng) - if err != nil { - return clusterversion.ClusterVersion{}, err - } - if cv.Version == (roachpb.Version{}) { - // This is needed when a node first joins an existing cluster, in - // which case it won't know what version to use until the first - // Gossip update comes in. - cv.Version = binaryMinSupportedVersion - } - - // Avoid running a binary with a store that is too new. For example, - // restarting into 1.1 after having upgraded to 1.2 doesn't work. - if binaryVersion.Less(cv.Version) { - return clusterversion.ClusterVersion{}, errors.Errorf( - "cockroach version v%s is incompatible with data in store %s; use version v%s or later", - binaryVersion, eng, cv.Version) - } - - // Track smallest use version encountered. - if cv.Version.Less(minStoreVersion.Version) { - minStoreVersion.Version = cv.Version - minStoreVersion.origin = fmt.Sprint(eng) - } - } - - // If no use version was found, fall back to our binaryMinSupportedVersion. This - // is the case when a brand new node is joining an existing cluster (which - // may be on any older version this binary supports). - if minStoreVersion.Version == maxPossibleVersion { - minStoreVersion.Version = binaryMinSupportedVersion - } - - cv := clusterversion.ClusterVersion{ - Version: minStoreVersion.Version, - } - log.Eventf(ctx, "read clusterVersion %+v", cv) - - // Avoid running a binary too new for this store. This is what you'd catch - // if, say, you restarted directly from 1.0 into 1.2 (bumping the min - // version) without going through 1.1 first. It would also be what you catch if - // you are starting 1.1 for the first time (after 1.0), but it crashes - // half-way through the startup sequence (so now some stores have 1.1, but - // some 1.0), in which case you are expected to run 1.1 again (hopefully - // without the crash this time) which would then rewrite all the stores. - // - // We only verify this now because as we iterate through the stores, we - // may not yet have picked up the final versions we're actually planning - // to use. - if minStoreVersion.Version.Less(binaryMinSupportedVersion) { - return clusterversion.ClusterVersion{}, errors.Errorf("store %s, last used with cockroach version v%s, "+ - "is too old for running version v%s (which requires data from v%s or later)", - minStoreVersion.origin, minStoreVersion.Version, binaryVersion, binaryMinSupportedVersion) - } - return cv, nil -} - func (ls *Stores) engines() []storage.Engine { var engines []storage.Engine ls.storeMap.Range(func(_ int64, v unsafe.Pointer) bool { diff --git a/pkg/kv/kvserver/stores_test.go b/pkg/kv/kvserver/stores_test.go index d26d001042b0..c7c90ad16e2a 100644 --- a/pkg/kv/kvserver/stores_test.go +++ b/pkg/kv/kvserver/stores_test.go @@ -18,10 +18,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -344,6 +344,9 @@ func TestStoresGossipStorageReadLatest(t *testing.T) { // TestStoresClusterVersionWriteSynthesize verifies that the cluster version is // written to all stores and that missing versions are filled in appropriately. +// +// TODO(tbg): if this test were a little more principled about only creating +// what it needs, it could move closer to SynthesizeClusterVersionFromEngines. func TestClusterVersionWriteSynthesize(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -366,7 +369,7 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { // If there are no stores, default to binaryMinSupportedVersion // (v1_0 in this test) - if initialCV, err := SynthesizeClusterVersionFromEngines(ctx, ls0.engines(), binV, minV); err != nil { + if initialCV, err := kvstorage.SynthesizeClusterVersionFromEngines(ctx, ls0.engines(), binV, minV); err != nil { t.Fatal(err) } else { expCV := clusterversion.ClusterVersion{ @@ -385,7 +388,7 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { // Verify that the initial read of an empty store synthesizes v1.0-0. This // is the code path that runs after starting the 1.1 binary for the first // time after the rolling upgrade from 1.0. - if initialCV, err := SynthesizeClusterVersionFromEngines(ctx, ls0.engines(), binV, minV); err != nil { + if initialCV, err := kvstorage.SynthesizeClusterVersionFromEngines(ctx, ls0.engines(), binV, minV); err != nil { t.Fatal(err) } else { expCV := clusterversion.ClusterVersion{ @@ -402,12 +405,12 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { cv := clusterversion.ClusterVersion{ Version: versionB, } - if err := WriteClusterVersionToEngines(ctx, ls0.engines(), cv); err != nil { + if err := kvstorage.WriteClusterVersionToEngines(ctx, ls0.engines(), cv); err != nil { t.Fatal(err) } // Verify the same thing comes back on read. - if newCV, err := SynthesizeClusterVersionFromEngines(ctx, ls0.engines(), binV, minV); err != nil { + if newCV, err := kvstorage.SynthesizeClusterVersionFromEngines(ctx, ls0.engines(), binV, minV); err != nil { t.Fatal(err) } else { expCV := cv @@ -427,7 +430,7 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { expCV := clusterversion.ClusterVersion{ Version: v1_0, } - if cv, err := SynthesizeClusterVersionFromEngines(ctx, ls01.engines(), binV, minV); err != nil { + if cv, err := kvstorage.SynthesizeClusterVersionFromEngines(ctx, ls01.engines(), binV, minV); err != nil { t.Fatal(err) } else if !reflect.DeepEqual(cv, expCV) { t.Fatalf("expected %+v, got %+v", expCV, cv) @@ -437,7 +440,7 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { cv := clusterversion.ClusterVersion{ Version: versionB, } - if err := WriteClusterVersionToEngines(ctx, ls01.engines(), cv); err != nil { + if err := kvstorage.WriteClusterVersionToEngines(ctx, ls01.engines(), cv); err != nil { t.Fatal(err) } } @@ -450,7 +453,7 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { { ls3 := makeStores() ls3.AddStore(stores[2]) - if err := WriteClusterVersionToEngines(ctx, ls3.engines(), cv); err != nil { + if err := kvstorage.WriteClusterVersionToEngines(ctx, ls3.engines(), cv); err != nil { t.Fatal(err) } } @@ -465,70 +468,9 @@ func TestClusterVersionWriteSynthesize(t *testing.T) { expCV := clusterversion.ClusterVersion{ Version: versionA, } - if cv, err := SynthesizeClusterVersionFromEngines(ctx, ls012.engines(), binV, minV); err != nil { + if cv, err := kvstorage.SynthesizeClusterVersionFromEngines(ctx, ls012.engines(), binV, minV); err != nil { t.Fatal(err) } else if !reflect.DeepEqual(cv, expCV) { t.Fatalf("expected %+v, got %+v", expCV, cv) } } - -// TestStoresClusterVersionIncompatible verifies an error occurs when -// setting up the cluster version from stores that are incompatible with the -// running binary. -func TestStoresClusterVersionIncompatible(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - - vOneDashOne := roachpb.Version{Major: 1, Internal: 1} - vOne := roachpb.Version{Major: 1} - - type testCase struct { - binV, minV roachpb.Version // binary version and min supported version - engV roachpb.Version // version found on engine in test - expErr string - } - for name, tc := range map[string]testCase{ - "StoreTooNew": { - // This is what the node is running. - binV: vOneDashOne, - // This is what the running node requires from its stores. - minV: vOne, - // Version is way too high for this node. - engV: roachpb.Version{Major: 9}, - expErr: `cockroach version v1\.0-1 is incompatible with data in store =; use version v9\.0 or later`, - }, - "StoreTooOldVersion": { - // This is what the node is running. - binV: roachpb.Version{Major: 9}, - // This is what the running node requires from its stores. - minV: roachpb.Version{Major: 5}, - // Version is way too low. - engV: roachpb.Version{Major: 4}, - expErr: `store =, last used with cockroach version v4\.0, is too old for running version v9\.0 \(which requires data from v5\.0 or later\)`, - }, - "StoreTooOldMinVersion": { - // Like the previous test case, but this time cv.MinimumVersion is the culprit. - binV: roachpb.Version{Major: 9}, - minV: roachpb.Version{Major: 5}, - engV: roachpb.Version{Major: 4}, - expErr: `store =, last used with cockroach version v4\.0, is too old for running version v9\.0 \(which requires data from v5\.0 or later\)`, - }, - } { - t.Run(name, func(t *testing.T) { - engs := []storage.Engine{storage.NewDefaultInMemForTesting()} - defer engs[0].Close() - // Configure versions and write. - cv := clusterversion.ClusterVersion{Version: tc.engV} - if err := WriteClusterVersionToEngines(ctx, engs, cv); err != nil { - t.Fatal(err) - } - if cv, err := SynthesizeClusterVersionFromEngines( - ctx, engs, tc.binV, tc.minV, - ); !testutils.IsError(err, tc.expErr) { - t.Fatalf("unexpected error: %+v, got version %v", err, cv) - } - }) - } -} diff --git a/pkg/server/init.go b/pkg/server/init.go index 016a7e8cbd20..958ebc366a53 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -510,7 +509,7 @@ func (s *initServer) tryBootstrap(ctx context.Context) (*initState, error) { // We use our binary version to bootstrap the cluster. cv := clusterversion.ClusterVersion{Version: s.config.binaryVersion} - if err := kvserver.WriteClusterVersionToEngines(ctx, s.inspectedDiskState.uninitializedEngines, cv); err != nil { + if err := kvstorage.WriteClusterVersionToEngines(ctx, s.inspectedDiskState.uninitializedEngines, cv); err != nil { return nil, err } @@ -542,7 +541,7 @@ func (s *initServer) initializeFirstStoreAfterJoin( firstEngine := s.inspectedDiskState.uninitializedEngines[0] clusterVersion := clusterversion.ClusterVersion{Version: *resp.ActiveVersion} - if err := kvserver.WriteClusterVersion(ctx, firstEngine, clusterVersion); err != nil { + if err := kvstorage.WriteClusterVersion(ctx, firstEngine, clusterVersion); err != nil { return nil, err } @@ -702,7 +701,7 @@ func inspectEngines( initializedEngines = append(initializedEngines, eng) } - clusterVersion, err := kvserver.SynthesizeClusterVersionFromEngines( + clusterVersion, err := kvstorage.SynthesizeClusterVersionFromEngines( ctx, initializedEngines, binaryVersion, binaryMinSupportedVersion, ) if err != nil { diff --git a/pkg/server/migration.go b/pkg/server/migration.go index defa536fef6e..ee93d4638fa6 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -110,7 +111,7 @@ func bumpClusterVersion( ) error { versionSetting := st.Version - prevCV, err := kvserver.SynthesizeClusterVersionFromEngines( + prevCV, err := kvstorage.SynthesizeClusterVersionFromEngines( ctx, engines, versionSetting.BinaryVersion(), versionSetting.BinaryMinSupportedVersion(), ) @@ -130,7 +131,7 @@ func bumpClusterVersion( // Whenever the version changes, we want to persist that update to // wherever the CRDB process retrieved the initial version from // (typically a collection of storage.Engines). - if err := kvserver.WriteClusterVersionToEngines(ctx, engines, newCV); err != nil { + if err := kvstorage.WriteClusterVersionToEngines(ctx, engines, newCV); err != nil { return err } diff --git a/pkg/server/migration_test.go b/pkg/server/migration_test.go index 0f34ab9b0ec3..67977d3533fa 100644 --- a/pkg/server/migration_test.go +++ b/pkg/server/migration_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -191,7 +192,7 @@ func TestBumpClusterVersion(t *testing.T) { } // Check to see that our bumped cluster version was persisted to disk. - synthesizedCV, err := kvserver.SynthesizeClusterVersionFromEngines( + synthesizedCV, err := kvstorage.SynthesizeClusterVersionFromEngines( ctx, s.Engines(), test.binaryVersion, test.activeClusterVersion.Version, ) diff --git a/pkg/server/node.go b/pkg/server/node.go index 76019595ca04..4ca4b7207bee 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -308,7 +308,7 @@ func bootstrapCluster( // other than the first one, and let regular node startup code deal with them. var bootstrapVersion clusterversion.ClusterVersion for i, eng := range engines { - cv, err := kvserver.ReadClusterVersion(ctx, eng) + cv, err := kvstorage.ReadClusterVersion(ctx, eng) if err != nil { return nil, errors.Wrapf(err, "reading cluster version of %s", eng) } else if cv.Major == 0 { @@ -626,7 +626,7 @@ func (n *Node) SetHLCUpperBound(ctx context.Context, hlcUpperBound int64) error } func (n *Node) addStore(ctx context.Context, store *kvserver.Store) { - cv, err := kvserver.ReadClusterVersion(context.TODO(), store.Engine()) + cv, err := kvstorage.ReadClusterVersion(context.TODO(), store.Engine()) if err != nil { log.Fatalf(ctx, "%v", err) } diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 16e8c799aa0f..41b0657c4770 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" @@ -63,7 +64,7 @@ func TestBootstrapCluster(t *testing.T) { ctx := context.Background() e := storage.NewDefaultInMemForTesting() defer e.Close() - require.NoError(t, kvserver.WriteClusterVersion(ctx, e, clusterversion.TestingClusterVersion)) + require.NoError(t, kvstorage.WriteClusterVersion(ctx, e, clusterversion.TestingClusterVersion)) initCfg := initServerCfg{ binaryMinSupportedVersion: clusterversion.TestingBinaryMinSupportedVersion, @@ -248,7 +249,7 @@ func TestCorruptedClusterID(t *testing.T) { defer e.Close() cv := clusterversion.TestingClusterVersion - require.NoError(t, kvserver.WriteClusterVersion(ctx, e, cv)) + require.NoError(t, kvstorage.WriteClusterVersion(ctx, e, cv)) initCfg := initServerCfg{ binaryMinSupportedVersion: clusterversion.TestingBinaryMinSupportedVersion, diff --git a/pkg/server/node_tombstone_storage_test.go b/pkg/server/node_tombstone_storage_test.go index 384e184c554c..906671263f46 100644 --- a/pkg/server/node_tombstone_storage_test.go +++ b/pkg/server/node_tombstone_storage_test.go @@ -16,7 +16,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -41,7 +40,7 @@ func TestNodeTombstoneStorage(t *testing.T) { id, err := uuid.NewV4() require.NoError(t, err) for i := range engs { - require.NoError(t, kvserver.WriteClusterVersion(ctx, engs[i], clusterversion.TestingClusterVersion)) + require.NoError(t, kvstorage.WriteClusterVersion(ctx, engs[i], clusterversion.TestingClusterVersion)) require.NoError(t, kvstorage.InitEngine(ctx, engs[i], roachpb.StoreIdent{ ClusterID: id, NodeID: 1, diff --git a/pkg/server/server.go b/pkg/server/server.go index 8ac117989798..155e61f379b2 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/loqrecovery" @@ -1307,7 +1308,7 @@ func (s *Server) PreStart(ctx context.Context) error { // now, or if the process crashed earlier half-way through the callback, // that version won't be on all engines. For that reason, we backfill // once. - if err := kvserver.WriteClusterVersionToEngines( + if err := kvstorage.WriteClusterVersionToEngines( ctx, s.engines, initialDiskClusterVersion, ); err != nil { return err @@ -1513,7 +1514,7 @@ func (s *Server) PreStart(ctx context.Context) error { // Either way, we'll do so by first persisting the cluster version // itself, and then informing the version setting about it (an invariant // we must up hold whenever setting a new active version). - if err := kvserver.WriteClusterVersionToEngines( + if err := kvstorage.WriteClusterVersionToEngines( ctx, s.engines, state.clusterVersion, ); err != nil { return err diff --git a/pkg/server/version_cluster_test.go b/pkg/server/version_cluster_test.go index a146612fa09d..fe7aad80c99e 100644 --- a/pkg/server/version_cluster_test.go +++ b/pkg/server/version_cluster_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -192,7 +193,7 @@ func TestClusterVersionPersistedOnJoin(t *testing.T) { for i := 0; i < len(tc.TestCluster.Servers); i++ { for _, engine := range tc.TestCluster.Servers[i].Engines() { - cv, err := kvserver.ReadClusterVersion(ctx, engine) + cv, err := kvstorage.ReadClusterVersion(ctx, engine) if err != nil { t.Fatal(err) } @@ -323,7 +324,7 @@ func TestClusterVersionUpgrade(t *testing.T) { // Since the wrapped version setting exposes the new versions, it must // definitely be present on all stores on the first try. if err := tc.Servers[1].GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { - cv, err := kvserver.ReadClusterVersion(ctx, s.Engine()) + cv, err := kvstorage.ReadClusterVersion(ctx, s.Engine()) if err != nil { return err } diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 2a7111651f1b..4a4671ef69e5 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -209,7 +209,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto cfg.Transport = transport cfg.ClosedTimestampReceiver = sidetransport.NewReceiver(nc, ltc.stopper, ltc.Stores, nil /* testingKnobs */) - if err := kvserver.WriteClusterVersion(ctx, ltc.Eng, clusterversion.TestingClusterVersion); err != nil { + if err := kvstorage.WriteClusterVersion(ctx, ltc.Eng, clusterversion.TestingClusterVersion); err != nil { t.Fatalf("unable to write cluster version: %s", err) } if err := kvstorage.InitEngine( From a28aa6cbb4b0de3ff92b4dcc74e319d2e2a220c2 Mon Sep 17 00:00:00 2001 From: David Hartunian Date: Thu, 26 Jan 2023 12:35:30 -0500 Subject: [PATCH 6/8] pkg/util/metric: optionally reintroduce legacy hdrhistogram model Addresses https://github.com/cockroachdb/cockroach/issues/95833 This patch reeintroduces the old HdrHistogram model to optionally be enabled in favor of the new Prometheus model, gated behind an environment variable called `COCKROACH_ENABLE_HDR_HISTOGRAMS`, allowing users a means to "fall back" to the old model in the event that the new model does not adequately serve their needs (think of this as an "insurance policy" to protect against this from happening again with no real mitigation - ideally, this environment variable should never have to be used). Note: some histograms were introduced *after* the new Prometheus histograms were added to CockroachDB. In this case, we use the `ForceUsePrometheus` option in the `HistogramOptions` struct to ignore the value of the env var, since there never was a time where these specific histograms used the HdrHistogram model. Release note (ops change): Histogram metrics can now optionally use the legacy HdrHistogram model by setting the environment var `COCKROACH_ENABLE_HDR_HISTOGRAMS=true` on CockroachDB nodes. **Note that this is not recommended** unless users are having difficulties with the newer Prometheus-backed histogram model. Enabling can cause performance issues with timeseries databases like Prometheus, as processing and storing the increased number of buckets is taxing on both CPU and storage. Note that the HdrHistogram model is slated for full deprecation in upcoming releases. --- pkg/ccl/changefeedccl/metrics.go | 96 +++++-- pkg/ccl/sqlproxyccl/connector.go | 2 +- pkg/ccl/sqlproxyccl/connector_test.go | 27 +- pkg/ccl/sqlproxyccl/metrics.go | 61 +++-- pkg/ccl/streamingccl/streamingest/metrics.go | 42 ++- pkg/kv/bulk/bulk_metrics.go | 14 +- pkg/kv/kvclient/kvcoord/txn_metrics.go | 45 ++-- pkg/kv/kvprober/kvprober.go | 22 +- pkg/kv/kvserver/liveness/liveness.go | 11 +- pkg/kv/kvserver/logstore/logstore.go | 2 +- .../kvserver/logstore/logstore_bench_test.go | 7 +- pkg/kv/kvserver/metrics.go | 89 ++++--- pkg/kv/kvserver/scheduler.go | 2 +- pkg/kv/kvserver/txnwait/metrics.go | 28 +- pkg/rpc/clock_offset.go | 11 +- pkg/server/node.go | 11 +- pkg/server/status/recorder_test.go | 7 +- pkg/sql/conn_executor.go | 87 +++--- pkg/sql/execinfra/metrics.go | 35 ++- pkg/sql/executor_statement_metrics.go | 18 +- pkg/sql/mem_metrics.go | 48 +++- pkg/sql/pgwire/pre_serve.go | 11 +- pkg/sql/pgwire/server.go | 11 +- .../sqlstats/persistedsqlstats/provider.go | 2 +- pkg/sql/sqlstats/sslocal/sql_stats.go | 2 +- pkg/sql/sqlstats/sslocal/sslocal_provider.go | 2 +- pkg/sql/ttl/ttljob/ttljob_metrics.go | 37 +-- pkg/util/admission/work_queue.go | 11 +- pkg/util/metric/BUILD.bazel | 4 + pkg/util/metric/aggmetric/BUILD.bazel | 1 - pkg/util/metric/aggmetric/agg_metric.go | 7 +- pkg/util/metric/aggmetric/agg_metric_test.go | 43 ++- pkg/util/metric/aggmetric/histogram.go | 27 +- .../testdata/add_after_destroy_hdr.txt | 23 ++ .../metric/aggmetric/testdata/basic_hdr.txt | 24 ++ .../metric/aggmetric/testdata/destroy_hdr.txt | 17 ++ pkg/util/metric/hdrhistogram.go | 248 ++++++++++++++++++ pkg/util/metric/metric.go | 101 ++++++- pkg/util/metric/metric_ext_test.go | 7 +- pkg/util/metric/metric_test.go | 28 +- pkg/util/metric/registry_test.go | 37 ++- pkg/util/mon/bytes_usage.go | 10 +- .../scheduler_latency_test.go | 7 +- 43 files changed, 1017 insertions(+), 308 deletions(-) create mode 100644 pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt create mode 100644 pkg/util/metric/aggmetric/testdata/basic_hdr.txt create mode 100644 pkg/util/metric/aggmetric/testdata/destroy_hdr.txt create mode 100644 pkg/util/metric/hdrhistogram.go diff --git a/pkg/ccl/changefeedccl/metrics.go b/pkg/ccl/changefeedccl/metrics.go index 4b042273df8a..45578bc6645e 100644 --- a/pkg/ccl/changefeedccl/metrics.go +++ b/pkg/ccl/changefeedccl/metrics.go @@ -28,6 +28,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) +const ( + changefeedCheckpointHistMaxLatency = 30 * time.Second + changefeedBatchHistMaxLatency = 30 * time.Second + changefeedFlushHistMaxLatency = 1 * time.Minute + admitLatencyMaxValue = 1 * time.Minute + commitLatencyMaxValue = 10 * time.Minute +) + // max length for the scope name. const maxSLIScopeNameLen = 128 @@ -488,16 +496,46 @@ func newAggregateMetrics(histogramWindow time.Duration) *AggMetrics { ErrorRetries: b.Counter(metaChangefeedErrorRetries), EmittedMessages: b.Counter(metaChangefeedEmittedMessages), FilteredMessages: b.Counter(metaChangefeedFilteredMessages), - MessageSize: b.Histogram(metaMessageSize, histogramWindow, metric.DataSize16MBBuckets), + MessageSize: b.Histogram(metric.HistogramOptions{ + Metadata: metaMessageSize, + Duration: histogramWindow, + MaxVal: 10 << 20, /* 10MB max message size */ + SigFigs: 1, + Buckets: metric.DataSize16MBBuckets, + }), EmittedBytes: b.Counter(metaChangefeedEmittedBytes), FlushedBytes: b.Counter(metaChangefeedFlushedBytes), Flushes: b.Counter(metaChangefeedFlushes), SizeBasedFlushes: b.Counter(metaSizeBasedFlushes), - BatchHistNanos: b.Histogram(metaChangefeedBatchHistNanos, histogramWindow, metric.BatchProcessLatencyBuckets), - FlushHistNanos: b.Histogram(metaChangefeedFlushHistNanos, histogramWindow, metric.BatchProcessLatencyBuckets), - CommitLatency: b.Histogram(metaCommitLatency, histogramWindow, metric.BatchProcessLatencyBuckets), - AdmitLatency: b.Histogram(metaAdmitLatency, histogramWindow, metric.BatchProcessLatencyBuckets), + BatchHistNanos: b.Histogram(metric.HistogramOptions{ + Metadata: metaChangefeedBatchHistNanos, + Duration: histogramWindow, + MaxVal: changefeedBatchHistMaxLatency.Nanoseconds(), + SigFigs: 1, + Buckets: metric.BatchProcessLatencyBuckets, + }), + FlushHistNanos: b.Histogram(metric.HistogramOptions{ + Metadata: metaChangefeedFlushHistNanos, + Duration: histogramWindow, + MaxVal: changefeedFlushHistMaxLatency.Nanoseconds(), + SigFigs: 2, + Buckets: metric.BatchProcessLatencyBuckets, + }), + CommitLatency: b.Histogram(metric.HistogramOptions{ + Metadata: metaCommitLatency, + Duration: histogramWindow, + MaxVal: commitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Buckets: metric.BatchProcessLatencyBuckets, + }), + AdmitLatency: b.Histogram(metric.HistogramOptions{ + Metadata: metaAdmitLatency, + Duration: histogramWindow, + MaxVal: admitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Buckets: metric.BatchProcessLatencyBuckets, + }), BackfillCount: b.Gauge(metaChangefeedBackfillCount), BackfillPendingRanges: b.Gauge(metaChangefeedBackfillPendingRanges), RunningCount: b.Gauge(metaChangefeedRunning), @@ -572,12 +610,12 @@ type Metrics struct { Failures *metric.Counter ResolvedMessages *metric.Counter QueueTimeNanos *metric.Counter - CheckpointHistNanos *metric.Histogram + CheckpointHistNanos metric.IHistogram FrontierUpdates *metric.Counter ThrottleMetrics cdcutils.Metrics ReplanCount *metric.Counter - ParallelConsumerFlushNanos *metric.Histogram - ParallelConsumerConsumeNanos *metric.Histogram + ParallelConsumerFlushNanos metric.IHistogram + ParallelConsumerConsumeNanos metric.IHistogram ParallelConsumerInFlightEvents *metric.Gauge mu struct { @@ -599,18 +637,36 @@ func (m *Metrics) getSLIMetrics(scope string) (*sliMetrics, error) { // MakeMetrics makes the metrics for changefeed monitoring. func MakeMetrics(histogramWindow time.Duration) metric.Struct { m := &Metrics{ - AggMetrics: newAggregateMetrics(histogramWindow), - KVFeedMetrics: kvevent.MakeMetrics(histogramWindow), - SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow), - ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages), - Failures: metric.NewCounter(metaChangefeedFailures), - QueueTimeNanos: metric.NewCounter(metaEventQueueTime), - CheckpointHistNanos: metric.NewHistogram(metaChangefeedCheckpointHistNanos, histogramWindow, metric.IOLatencyBuckets), - FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates), - ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow), - ReplanCount: metric.NewCounter(metaChangefeedReplanCount), - ParallelConsumerFlushNanos: metric.NewHistogram(metaChangefeedEventConsumerFlushNanos, histogramWindow, metric.IOLatencyBuckets), - ParallelConsumerConsumeNanos: metric.NewHistogram(metaChangefeedEventConsumerConsumeNanos, histogramWindow, metric.IOLatencyBuckets), + AggMetrics: newAggregateMetrics(histogramWindow), + KVFeedMetrics: kvevent.MakeMetrics(histogramWindow), + SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow), + ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages), + Failures: metric.NewCounter(metaChangefeedFailures), + QueueTimeNanos: metric.NewCounter(metaEventQueueTime), + CheckpointHistNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaChangefeedCheckpointHistNanos, + Duration: histogramWindow, + MaxVal: changefeedCheckpointHistMaxLatency.Nanoseconds(), + SigFigs: 2, + Buckets: metric.IOLatencyBuckets, + }), + FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates), + ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow), + ReplanCount: metric.NewCounter(metaChangefeedReplanCount), + // Below two metrics were never implemented using the hdr histogram. Set ForceUsePrometheus + // to true. + ParallelConsumerFlushNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaChangefeedEventConsumerFlushNanos, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + Mode: metric.HistogramModePrometheus, + }), + ParallelConsumerConsumeNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaChangefeedEventConsumerConsumeNanos, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + Mode: metric.HistogramModePrometheus, + }), ParallelConsumerInFlightEvents: metric.NewGauge(metaChangefeedEventConsumerInFlightEvents), } diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index 8329efecc8ae..6a891a7ab53f 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -77,7 +77,7 @@ type connector struct { // DialTenantLatency tracks how long it takes to retrieve the address for // a tenant and set up a tcp connection to the address. - DialTenantLatency *metric.Histogram + DialTenantLatency metric.IHistogram // DialTenantRetries counts how often dialing a tenant is retried. DialTenantRetries *metric.Counter diff --git a/pkg/ccl/sqlproxyccl/connector_test.go b/pkg/ccl/sqlproxyccl/connector_test.go index 8a28955b71a6..6fa21c56afe3 100644 --- a/pkg/ccl/sqlproxyccl/connector_test.go +++ b/pkg/ccl/sqlproxyccl/connector_test.go @@ -380,9 +380,12 @@ func TestConnector_dialTenantCluster(t *testing.T) { c := &connector{ TenantID: roachpb.MustMakeTenantID(42), - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets, - ), + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metaDialTenantLatency, + Duration: time.Millisecond, + Buckets: metric.NetworkLatencyBuckets, + }), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), } dc := &testTenantDirectoryCache{} @@ -460,9 +463,12 @@ func TestConnector_dialTenantCluster(t *testing.T) { defer cancel() c := &connector{ - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets, - ), + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDialTenantLatency, + Duration: time.Millisecond, + Buckets: metric.NetworkLatencyBuckets, + }), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), } c.testingKnobs.lookupAddr = func(ctx context.Context) (string, error) { @@ -491,9 +497,12 @@ func TestConnector_dialTenantCluster(t *testing.T) { var reportFailureFnCount int c := &connector{ TenantID: roachpb.MustMakeTenantID(42), - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets, - ), + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDialTenantLatency, + Duration: time.Millisecond, + Buckets: metric.NetworkLatencyBuckets, + }), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), } c.DirectoryCache = &testTenantDirectoryCache{ diff --git a/pkg/ccl/sqlproxyccl/metrics.go b/pkg/ccl/sqlproxyccl/metrics.go index 2fe0d8e16131..2432f9b1f668 100644 --- a/pkg/ccl/sqlproxyccl/metrics.go +++ b/pkg/ccl/sqlproxyccl/metrics.go @@ -23,19 +23,19 @@ type metrics struct { RoutingErrCount *metric.Counter RefusedConnCount *metric.Counter SuccessfulConnCount *metric.Counter - ConnectionLatency *metric.Histogram + ConnectionLatency metric.IHistogram AuthFailedCount *metric.Counter ExpiredClientConnCount *metric.Counter - DialTenantLatency *metric.Histogram + DialTenantLatency metric.IHistogram DialTenantRetries *metric.Counter ConnMigrationSuccessCount *metric.Counter ConnMigrationErrorFatalCount *metric.Counter ConnMigrationErrorRecoverableCount *metric.Counter ConnMigrationAttemptedCount *metric.Counter - ConnMigrationAttemptedLatency *metric.Histogram - ConnMigrationTransferResponseMessageSize *metric.Histogram + ConnMigrationAttemptedLatency metric.IHistogram + ConnMigrationTransferResponseMessageSize metric.IHistogram QueryCancelReceivedPGWire *metric.Counter QueryCancelReceivedHTTP *metric.Counter @@ -49,6 +49,16 @@ func (metrics) MetricStruct() {} var _ metric.Struct = metrics{} +const ( + // maxExpectedTransferResponseMessageSize corresponds to maximum expected + // response message size for the SHOW TRANSFER STATE query. We choose 16MB + // here to match the defaultMaxReadBufferSize used for ingesting SQL + // statements in the SQL server (see pkg/sql/pgwire/pgwirebase/encoding.go). + // + // This will be used to tune sql.session_transfer.max_session_size. + maxExpectedTransferResponseMessageSize = 1 << 24 // 16MB +) + var ( metaCurConnCount = metric.Metadata{ Name: "proxy.sql.conns", @@ -213,18 +223,20 @@ func makeProxyMetrics() metrics { RoutingErrCount: metric.NewCounter(metaRoutingErrCount), RefusedConnCount: metric.NewCounter(metaRefusedConnCount), SuccessfulConnCount: metric.NewCounter(metaSuccessfulConnCount), - ConnectionLatency: metric.NewHistogram( - metaConnMigrationAttemptedCount, - base.DefaultHistogramWindowInterval(), - metric.NetworkLatencyBuckets, - ), + ConnectionLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaConnMigrationAttemptedCount, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.NetworkLatencyBuckets, + }), AuthFailedCount: metric.NewCounter(metaAuthFailedCount), ExpiredClientConnCount: metric.NewCounter(metaExpiredClientConnCount), // Connector metrics. - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, - base.DefaultHistogramWindowInterval(), - metric.NetworkLatencyBuckets, + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDialTenantLatency, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.NetworkLatencyBuckets}, ), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), // Connection migration metrics. @@ -232,16 +244,19 @@ func makeProxyMetrics() metrics { ConnMigrationErrorFatalCount: metric.NewCounter(metaConnMigrationErrorFatalCount), ConnMigrationErrorRecoverableCount: metric.NewCounter(metaConnMigrationErrorRecoverableCount), ConnMigrationAttemptedCount: metric.NewCounter(metaConnMigrationAttemptedCount), - ConnMigrationAttemptedLatency: metric.NewHistogram( - metaConnMigrationAttemptedLatency, - base.DefaultHistogramWindowInterval(), - metric.NetworkLatencyBuckets, - ), - ConnMigrationTransferResponseMessageSize: metric.NewHistogram( - metaConnMigrationTransferResponseMessageSize, - base.DefaultHistogramWindowInterval(), - metric.DataSize16MBBuckets, - ), + ConnMigrationAttemptedLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaConnMigrationAttemptedLatency, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.NetworkLatencyBuckets, + }), + ConnMigrationTransferResponseMessageSize: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaConnMigrationTransferResponseMessageSize, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.DataSize16MBBuckets, + MaxVal: maxExpectedTransferResponseMessageSize, + SigFigs: 1, + }), QueryCancelReceivedPGWire: metric.NewCounter(metaQueryCancelReceivedPGWire), QueryCancelReceivedHTTP: metric.NewCounter(metaQueryCancelReceivedHTTP), QueryCancelIgnored: metric.NewCounter(metaQueryCancelIgnored), diff --git a/pkg/ccl/streamingccl/streamingest/metrics.go b/pkg/ccl/streamingccl/streamingest/metrics.go index f9f28f38a247..c2858cd5ad36 100644 --- a/pkg/ccl/streamingccl/streamingest/metrics.go +++ b/pkg/ccl/streamingccl/streamingest/metrics.go @@ -15,6 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" ) +const ( + streamingFlushHistMaxLatency = 1 * time.Minute + streamingAdmitLatencyMaxValue = 3 * time.Minute + streamingCommitLatencyMaxValue = 10 * time.Minute +) + var ( metaReplicationEventsIngested = metric.Metadata{ Name: "replication.events_ingested", @@ -120,9 +126,9 @@ type Metrics struct { Flushes *metric.Counter JobProgressUpdates *metric.Counter ResolvedEvents *metric.Counter - FlushHistNanos *metric.Histogram - CommitLatency *metric.Histogram - AdmitLatency *metric.Histogram + FlushHistNanos metric.IHistogram + CommitLatency metric.IHistogram + AdmitLatency metric.IHistogram RunningCount *metric.Gauge EarliestDataCheckpointSpan *metric.Gauge LatestDataCheckpointSpan *metric.Gauge @@ -143,12 +149,30 @@ func MakeMetrics(histogramWindow time.Duration) metric.Struct { Flushes: metric.NewCounter(metaReplicationFlushes), ResolvedEvents: metric.NewCounter(metaReplicationResolvedEventsIngested), JobProgressUpdates: metric.NewCounter(metaJobProgressUpdates), - FlushHistNanos: metric.NewHistogram(metaReplicationFlushHistNanos, - histogramWindow, metric.BatchProcessLatencyBuckets), - CommitLatency: metric.NewHistogram(metaReplicationCommitLatency, - histogramWindow, metric.BatchProcessLatencyBuckets), - AdmitLatency: metric.NewHistogram(metaReplicationAdmitLatency, - histogramWindow, metric.BatchProcessLatencyBuckets), + FlushHistNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaReplicationFlushHistNanos, + Duration: histogramWindow, + Buckets: metric.BatchProcessLatencyBuckets, + MaxVal: streamingFlushHistMaxLatency.Nanoseconds(), + SigFigs: 1, + Mode: metric.HistogramModePreferHdrLatency, + }), + CommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaReplicationCommitLatency, + Duration: histogramWindow, + Buckets: metric.BatchProcessLatencyBuckets, + MaxVal: streamingCommitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Mode: metric.HistogramModePreferHdrLatency, + }), + AdmitLatency: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaReplicationAdmitLatency, + Duration: histogramWindow, + Buckets: metric.BatchProcessLatencyBuckets, + MaxVal: streamingAdmitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Mode: metric.HistogramModePreferHdrLatency, + }), RunningCount: metric.NewGauge(metaStreamsRunning), EarliestDataCheckpointSpan: metric.NewGauge(metaEarliestDataCheckpointSpan), LatestDataCheckpointSpan: metric.NewGauge(metaLatestDataCheckpointSpan), diff --git a/pkg/kv/bulk/bulk_metrics.go b/pkg/kv/bulk/bulk_metrics.go index f3390d54733e..7cbbc748a20b 100644 --- a/pkg/kv/bulk/bulk_metrics.go +++ b/pkg/kv/bulk/bulk_metrics.go @@ -20,7 +20,7 @@ import ( // Metrics contains pointers to the metrics for // monitoring bulk operations. type Metrics struct { - MaxBytesHist *metric.Histogram + MaxBytesHist metric.IHistogram CurBytesCount *metric.Gauge } @@ -44,10 +44,20 @@ var ( } ) +// See pkg/sql/mem_metrics.go +// log10int64times1000 = log10(math.MaxInt64) * 1000, rounded up somewhat +const log10int64times1000 = 19 * 1000 + // MakeBulkMetrics instantiates the metrics holder for bulk operation monitoring. func MakeBulkMetrics(histogramWindow time.Duration) Metrics { return Metrics{ - MaxBytesHist: metric.NewHistogram(metaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), + MaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaMemMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), CurBytesCount: metric.NewGauge(metaMemCurBytes), } } diff --git a/pkg/kv/kvclient/kvcoord/txn_metrics.go b/pkg/kv/kvclient/kvcoord/txn_metrics.go index eb6313012717..fcca64aa74ae 100644 --- a/pkg/kv/kvclient/kvcoord/txn_metrics.go +++ b/pkg/kv/kvclient/kvcoord/txn_metrics.go @@ -31,14 +31,14 @@ type TxnMetrics struct { RefreshMemoryLimitExceeded *metric.Counter RefreshAutoRetries *metric.Counter - Durations *metric.Histogram + Durations metric.IHistogram TxnsWithCondensedIntents *metric.Counter TxnsWithCondensedIntentsGauge *metric.Gauge TxnsRejectedByLockSpanBudget *metric.Counter // Restarts is the number of times we had to restart the transaction. - Restarts *metric.Histogram + Restarts metric.IHistogram // Counts of restart types. RestartsWriteTooOld telemetry.CounterWithMetric @@ -264,21 +264,32 @@ var ( // windowed portions retain data for approximately histogramWindow. func MakeTxnMetrics(histogramWindow time.Duration) TxnMetrics { return TxnMetrics{ - Aborts: metric.NewCounter(metaAbortsRates), - Commits: metric.NewCounter(metaCommitsRates), - Commits1PC: metric.NewCounter(metaCommits1PCRates), - ParallelCommits: metric.NewCounter(metaParallelCommitsRates), - CommitWaits: metric.NewCounter(metaCommitWaitCount), - RefreshSuccess: metric.NewCounter(metaRefreshSuccess), - RefreshFail: metric.NewCounter(metaRefreshFail), - RefreshFailWithCondensedSpans: metric.NewCounter(metaRefreshFailWithCondensedSpans), - RefreshMemoryLimitExceeded: metric.NewCounter(metaRefreshMemoryLimitExceeded), - RefreshAutoRetries: metric.NewCounter(metaRefreshAutoRetries), - Durations: metric.NewHistogram(metaDurationsHistograms, histogramWindow, metric.IOLatencyBuckets), - TxnsWithCondensedIntents: metric.NewCounter(metaTxnsWithCondensedIntentSpans), - TxnsWithCondensedIntentsGauge: metric.NewGauge(metaTxnsWithCondensedIntentSpansGauge), - TxnsRejectedByLockSpanBudget: metric.NewCounter(metaTxnsRejectedByLockSpanBudget), - Restarts: metric.NewHistogram(metaRestartsHistogram, histogramWindow, metric.Count1KBuckets), + Aborts: metric.NewCounter(metaAbortsRates), + Commits: metric.NewCounter(metaCommitsRates), + Commits1PC: metric.NewCounter(metaCommits1PCRates), + ParallelCommits: metric.NewCounter(metaParallelCommitsRates), + CommitWaits: metric.NewCounter(metaCommitWaitCount), + RefreshSuccess: metric.NewCounter(metaRefreshSuccess), + RefreshFail: metric.NewCounter(metaRefreshFail), + RefreshFailWithCondensedSpans: metric.NewCounter(metaRefreshFailWithCondensedSpans), + RefreshMemoryLimitExceeded: metric.NewCounter(metaRefreshMemoryLimitExceeded), + RefreshAutoRetries: metric.NewCounter(metaRefreshAutoRetries), + Durations: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDurationsHistograms, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + TxnsWithCondensedIntents: metric.NewCounter(metaTxnsWithCondensedIntentSpans), + TxnsWithCondensedIntentsGauge: metric.NewGauge(metaTxnsWithCondensedIntentSpansGauge), + TxnsRejectedByLockSpanBudget: metric.NewCounter(metaTxnsRejectedByLockSpanBudget), + Restarts: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaRestartsHistogram, + Duration: histogramWindow, + MaxVal: 100, + SigFigs: 3, + Buckets: metric.Count1KBuckets, + }), RestartsWriteTooOld: telemetry.NewCounterWithMetric(metaRestartsWriteTooOld), RestartsWriteTooOldMulti: telemetry.NewCounterWithMetric(metaRestartsWriteTooOldMulti), RestartsSerializable: telemetry.NewCounterWithMetric(metaRestartsSerializable), diff --git a/pkg/kv/kvprober/kvprober.go b/pkg/kv/kvprober/kvprober.go index 2e0d84529fc9..0ec07006efe1 100644 --- a/pkg/kv/kvprober/kvprober.go +++ b/pkg/kv/kvprober/kvprober.go @@ -140,10 +140,10 @@ var ( type Metrics struct { ReadProbeAttempts *metric.Counter ReadProbeFailures *metric.Counter - ReadProbeLatency *metric.Histogram + ReadProbeLatency metric.IHistogram WriteProbeAttempts *metric.Counter WriteProbeFailures *metric.Counter - WriteProbeLatency *metric.Histogram + WriteProbeLatency metric.IHistogram WriteProbeQuarantineOldestDuration *metric.Gauge ProbePlanAttempts *metric.Counter ProbePlanFailures *metric.Counter @@ -229,14 +229,20 @@ func NewProber(opts Opts) *Prober { metrics: Metrics{ ReadProbeAttempts: metric.NewCounter(metaReadProbeAttempts), ReadProbeFailures: metric.NewCounter(metaReadProbeFailures), - ReadProbeLatency: metric.NewHistogram( - metaReadProbeLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets, - ), + ReadProbeLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaReadProbeLatency, + Duration: opts.HistogramWindowInterval, + Buckets: metric.NetworkLatencyBuckets, + }), WriteProbeAttempts: metric.NewCounter(metaWriteProbeAttempts), WriteProbeFailures: metric.NewCounter(metaWriteProbeFailures), - WriteProbeLatency: metric.NewHistogram( - metaWriteProbeLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets, - ), + WriteProbeLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaWriteProbeLatency, + Duration: opts.HistogramWindowInterval, + Buckets: metric.NetworkLatencyBuckets, + }), WriteProbeQuarantineOldestDuration: metric.NewFunctionalGauge( metaWriteProbeQuarantineOldestDuration, func() int64 { return qPool.oldestDuration().Nanoseconds() }, diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index 3411ea1bbd78..a11f3abc4114 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -144,7 +144,7 @@ type Metrics struct { HeartbeatSuccesses *metric.Counter HeartbeatFailures telemetry.CounterWithMetric EpochIncrements telemetry.CounterWithMetric - HeartbeatLatency *metric.Histogram + HeartbeatLatency metric.IHistogram } // IsLiveCallback is invoked when a node's IsLive state changes to true. @@ -310,9 +310,12 @@ func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness { HeartbeatSuccesses: metric.NewCounter(metaHeartbeatSuccesses), HeartbeatFailures: telemetry.NewCounterWithMetric(metaHeartbeatFailures), EpochIncrements: telemetry.NewCounterWithMetric(metaEpochIncrements), - HeartbeatLatency: metric.NewHistogram( - metaHeartbeatLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets, - ), + HeartbeatLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaHeartbeatLatency, + Duration: opts.HistogramWindowInterval, + Buckets: metric.NetworkLatencyBuckets, + }), } nl.mu.nodes = make(map[roachpb.NodeID]Record) nl.heartbeatToken <- struct{}{} diff --git a/pkg/kv/kvserver/logstore/logstore.go b/pkg/kv/kvserver/logstore/logstore.go index f1784431b0f5..e1cd72d34483 100644 --- a/pkg/kv/kvserver/logstore/logstore.go +++ b/pkg/kv/kvserver/logstore/logstore.go @@ -90,7 +90,7 @@ type AppendStats struct { // Metrics contains metrics specific to the log storage. type Metrics struct { - RaftLogCommitLatency *metric.Histogram + RaftLogCommitLatency metric.IHistogram } // LogStore is a stub of a separated Raft log storage. diff --git a/pkg/kv/kvserver/logstore/logstore_bench_test.go b/pkg/kv/kvserver/logstore/logstore_bench_test.go index 0c3f8e0473e2..f09472e27385 100644 --- a/pkg/kv/kvserver/logstore/logstore_bench_test.go +++ b/pkg/kv/kvserver/logstore/logstore_bench_test.go @@ -60,7 +60,12 @@ func runBenchmarkLogStore_StoreEntries(b *testing.B, bytes int64) { EntryCache: ec, Settings: cluster.MakeTestingClusterSettings(), Metrics: Metrics{ - RaftLogCommitLatency: metric.NewHistogram(metric.Metadata{}, 10*time.Second, metric.IOLatencyBuckets), + RaftLogCommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metric.Metadata{}, + Duration: 10 * time.Second, + Buckets: metric.IOLatencyBuckets, + }), }, } diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 38953afe5d50..5476a6c17e9a 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -1850,15 +1850,15 @@ type StoreMetrics struct { // Raft processing metrics. RaftTicks *metric.Counter - RaftQuotaPoolPercentUsed *metric.Histogram + RaftQuotaPoolPercentUsed metric.IHistogram RaftWorkingDurationNanos *metric.Counter RaftTickingDurationNanos *metric.Counter RaftCommandsApplied *metric.Counter - RaftLogCommitLatency *metric.Histogram - RaftCommandCommitLatency *metric.Histogram - RaftHandleReadyLatency *metric.Histogram - RaftApplyCommittedLatency *metric.Histogram - RaftSchedulerLatency *metric.Histogram + RaftLogCommitLatency metric.IHistogram + RaftCommandCommitLatency metric.IHistogram + RaftHandleReadyLatency metric.IHistogram + RaftApplyCommittedLatency metric.IHistogram + RaftSchedulerLatency metric.IHistogram RaftTimeoutCampaign *metric.Counter // Raft message metrics. @@ -1990,8 +1990,8 @@ type StoreMetrics struct { ReplicaCircuitBreakerCumTripped *metric.Counter // Replica batch evaluation metrics. - ReplicaReadBatchEvaluationLatency *metric.Histogram - ReplicaWriteBatchEvaluationLatency *metric.Histogram + ReplicaReadBatchEvaluationLatency metric.IHistogram + ReplicaWriteBatchEvaluationLatency metric.IHistogram ReplicaReadBatchDroppedLatchesBeforeEval *metric.Counter ReplicaReadBatchWithoutInterleavingIter *metric.Counter @@ -2377,27 +2377,46 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { // Raft processing metrics. RaftTicks: metric.NewCounter(metaRaftTicks), - RaftQuotaPoolPercentUsed: metric.NewHistogram( - metaRaftQuotaPoolPercentUsed, histogramWindow, metric.Percent100Buckets, - ), + RaftQuotaPoolPercentUsed: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaRaftQuotaPoolPercentUsed, + Duration: histogramWindow, + MaxVal: 100, + SigFigs: 1, + Buckets: metric.Percent100Buckets, + }), RaftWorkingDurationNanos: metric.NewCounter(metaRaftWorkingDurationNanos), RaftTickingDurationNanos: metric.NewCounter(metaRaftTickingDurationNanos), RaftCommandsApplied: metric.NewCounter(metaRaftCommandsApplied), - RaftLogCommitLatency: metric.NewHistogram( - metaRaftLogCommitLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftCommandCommitLatency: metric.NewHistogram( - metaRaftCommandCommitLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftHandleReadyLatency: metric.NewHistogram( - metaRaftHandleReadyLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftApplyCommittedLatency: metric.NewHistogram( - metaRaftApplyCommittedLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftSchedulerLatency: metric.NewHistogram( - metaRaftSchedulerLatency, histogramWindow, metric.IOLatencyBuckets, - ), + RaftLogCommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftLogCommitLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftCommandCommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftCommandCommitLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftHandleReadyLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftHandleReadyLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftApplyCommittedLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftApplyCommittedLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftSchedulerLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftSchedulerLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), RaftTimeoutCampaign: metric.NewCounter(metaRaftTimeoutCampaign), // Raft message metrics. @@ -2538,12 +2557,18 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { ReplicaCircuitBreakerCumTripped: metric.NewCounter(metaReplicaCircuitBreakerCumTripped), // Replica batch evaluation. - ReplicaReadBatchEvaluationLatency: metric.NewHistogram( - metaReplicaReadBatchEvaluationLatency, histogramWindow, metric.IOLatencyBuckets, - ), - ReplicaWriteBatchEvaluationLatency: metric.NewHistogram( - metaReplicaWriteBatchEvaluationLatency, histogramWindow, metric.IOLatencyBuckets, - ), + ReplicaReadBatchEvaluationLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaReplicaReadBatchEvaluationLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + ReplicaWriteBatchEvaluationLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaReplicaWriteBatchEvaluationLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), FlushUtilization: metric.NewGaugeFloat64(metaStorageFlushUtilization), FsyncLatency: metric.NewManualWindowHistogram(metaStorageFsyncLatency, pebble.FsyncLatencyBuckets), diff --git a/pkg/kv/kvserver/scheduler.go b/pkg/kv/kvserver/scheduler.go index 85db3cefa1fb..062bb4562843 100644 --- a/pkg/kv/kvserver/scheduler.go +++ b/pkg/kv/kvserver/scheduler.go @@ -181,7 +181,7 @@ type raftScheduleState struct { type raftScheduler struct { ambientContext log.AmbientContext processor raftProcessor - latency *metric.Histogram + latency metric.IHistogram numWorkers int maxTicks int diff --git a/pkg/kv/kvserver/txnwait/metrics.go b/pkg/kv/kvserver/txnwait/metrics.go index 2e9d1d2a2055..4610fd8e375c 100644 --- a/pkg/kv/kvserver/txnwait/metrics.go +++ b/pkg/kv/kvserver/txnwait/metrics.go @@ -22,8 +22,8 @@ type Metrics struct { PusherWaiting *metric.Gauge QueryWaiting *metric.Gauge PusherSlow *metric.Gauge - PusherWaitTime *metric.Histogram - QueryWaitTime *metric.Histogram + PusherWaitTime metric.IHistogram + QueryWaitTime metric.IHistogram DeadlocksTotal *metric.Counter } @@ -66,27 +66,31 @@ func NewMetrics(histogramWindowInterval time.Duration) *Metrics { }, ), - PusherWaitTime: metric.NewHistogram( - metric.Metadata{ + PusherWaitTime: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "txnwaitqueue.pusher.wait_time", Help: "Histogram of durations spent in queue by pushers", Measurement: "Pusher wait time", Unit: metric.Unit_NANOSECONDS, }, - histogramWindowInterval, - metric.LongRunning60mLatencyBuckets, - ), + MaxVal: time.Hour.Nanoseconds(), + SigFigs: 1, + Duration: histogramWindowInterval, + Buckets: metric.LongRunning60mLatencyBuckets, + }), - QueryWaitTime: metric.NewHistogram( - metric.Metadata{ + QueryWaitTime: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "txnwaitqueue.query.wait_time", Help: "Histogram of durations spent in queue by queries", Measurement: "Query wait time", Unit: metric.Unit_NANOSECONDS, }, - histogramWindowInterval, - metric.LongRunning60mLatencyBuckets, - ), + MaxVal: time.Hour.Nanoseconds(), + SigFigs: 1, + Duration: histogramWindowInterval, + Buckets: metric.LongRunning60mLatencyBuckets, + }), DeadlocksTotal: metric.NewCounter( metric.Metadata{ diff --git a/pkg/rpc/clock_offset.go b/pkg/rpc/clock_offset.go index bd5877f26b6b..99daad030a72 100644 --- a/pkg/rpc/clock_offset.go +++ b/pkg/rpc/clock_offset.go @@ -29,7 +29,7 @@ import ( type RemoteClockMetrics struct { ClockOffsetMeanNanos *metric.Gauge ClockOffsetStdDevNanos *metric.Gauge - LatencyHistogramNanos *metric.Histogram + LatencyHistogramNanos metric.IHistogram } // avgLatencyMeasurementAge determines how to exponentially weight the @@ -136,9 +136,12 @@ func newRemoteClockMonitor( r.metrics = RemoteClockMetrics{ ClockOffsetMeanNanos: metric.NewGauge(metaClockOffsetMeanNanos), ClockOffsetStdDevNanos: metric.NewGauge(metaClockOffsetStdDevNanos), - LatencyHistogramNanos: metric.NewHistogram( - metaLatencyHistogramNanos, histogramWindowInterval, metric.IOLatencyBuckets, - ), + LatencyHistogramNanos: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaLatencyHistogramNanos, + Duration: histogramWindowInterval, + Buckets: metric.IOLatencyBuckets, + }), } return &r } diff --git a/pkg/server/node.go b/pkg/server/node.go index 76019595ca04..2259a512bd75 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -167,7 +167,7 @@ var ( ) type nodeMetrics struct { - Latency *metric.Histogram + Latency metric.IHistogram Success *metric.Counter Err *metric.Counter DiskStalls *metric.Counter @@ -178,9 +178,12 @@ type nodeMetrics struct { func makeNodeMetrics(reg *metric.Registry, histogramWindow time.Duration) nodeMetrics { nm := nodeMetrics{ - Latency: metric.NewHistogram( - metaExecLatency, histogramWindow, metric.IOLatencyBuckets, - ), + Latency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaExecLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), Success: metric.NewCounter(metaExecSuccess), Err: metric.NewCounter(metaExecError), DiskStalls: metric.NewCounter(metaDiskStalls), diff --git a/pkg/server/status/recorder_test.go b/pkg/server/status/recorder_test.go index 82016bd8902c..da77954343af 100644 --- a/pkg/server/status/recorder_test.go +++ b/pkg/server/status/recorder_test.go @@ -385,7 +385,12 @@ func TestMetricsRecorder(t *testing.T) { c.Inc((data.val)) addExpected(reg.prefix, data.name, reg.source, 100, data.val, reg.isNode) case "histogram": - h := metric.NewHistogram(metric.Metadata{Name: reg.prefix + data.name}, time.Second, []float64{1.0, 10.0, 100.0, 1000.0}) + h := metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{Name: reg.prefix + data.name}, + Duration: time.Second, + Buckets: []float64{1.0, 10.0, 100.0, 1000.0}, + Mode: metric.HistogramModePrometheus, + }) reg.reg.AddMetric(h) h.RecordValue(data.val) for _, q := range recordHistogramQuantiles { diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index b593acf1f863..ae3dffb05c35 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -452,21 +452,36 @@ func makeMetrics(internal bool) Metrics { SQLOptPlanCacheHits: metric.NewCounter(getMetricMeta(MetaSQLOptPlanCacheHits, internal)), SQLOptPlanCacheMisses: metric.NewCounter(getMetricMeta(MetaSQLOptPlanCacheMisses, internal)), // TODO(mrtracy): See HistogramWindowInterval in server/config.go for the 6x factor. - DistSQLExecLatency: metric.NewHistogram( - getMetricMeta(MetaDistSQLExecLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - SQLExecLatency: metric.NewHistogram( - getMetricMeta(MetaSQLExecLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - DistSQLServiceLatency: metric.NewHistogram( - getMetricMeta(MetaDistSQLServiceLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - SQLServiceLatency: metric.NewHistogram( - getMetricMeta(MetaSQLServiceLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - SQLTxnLatency: metric.NewHistogram( - getMetricMeta(MetaSQLTxnLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), + DistSQLExecLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaDistSQLExecLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + SQLExecLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaSQLExecLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + DistSQLServiceLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaDistSQLServiceLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + SQLServiceLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaSQLServiceLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + SQLTxnLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaSQLTxnLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), SQLTxnsOpen: metric.NewGauge(getMetricMeta(MetaSQLTxnsOpen, internal)), SQLActiveStatements: metric.NewGauge(getMetricMeta(MetaSQLActiveQueries, internal)), SQLContendedTxns: metric.NewCounter(getMetricMeta(MetaSQLTxnContended, internal)), @@ -490,28 +505,38 @@ func makeMetrics(internal bool) Metrics { func makeServerMetrics(cfg *ExecutorConfig) ServerMetrics { return ServerMetrics{ StatsMetrics: StatsMetrics{ - SQLStatsMemoryMaxBytesHist: metric.NewHistogram( - MetaSQLStatsMemMaxBytes, - cfg.HistogramWindowInterval, - metric.MemoryUsage64MBBuckets, - ), + SQLStatsMemoryMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaSQLStatsMemMaxBytes, + Duration: cfg.HistogramWindowInterval, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), SQLStatsMemoryCurBytesCount: metric.NewGauge(MetaSQLStatsMemCurBytes), - ReportedSQLStatsMemoryMaxBytesHist: metric.NewHistogram( - MetaReportedSQLStatsMemMaxBytes, - cfg.HistogramWindowInterval, - metric.MemoryUsage64MBBuckets, - ), + ReportedSQLStatsMemoryMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaReportedSQLStatsMemMaxBytes, + Duration: cfg.HistogramWindowInterval, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), ReportedSQLStatsMemoryCurBytesCount: metric.NewGauge(MetaReportedSQLStatsMemCurBytes), DiscardedStatsCount: metric.NewCounter(MetaDiscardedSQLStats), SQLStatsFlushStarted: metric.NewCounter(MetaSQLStatsFlushStarted), SQLStatsFlushFailure: metric.NewCounter(MetaSQLStatsFlushFailure), - SQLStatsFlushDuration: metric.NewHistogram( - MetaSQLStatsFlushDuration, 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), + SQLStatsFlushDuration: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: MetaSQLStatsFlushDuration, + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), SQLStatsRemovedRows: metric.NewCounter(MetaSQLStatsRemovedRows), - SQLTxnStatsCollectionOverhead: metric.NewHistogram( - MetaSQLTxnStatsCollectionOverhead, 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), + SQLTxnStatsCollectionOverhead: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: MetaSQLTxnStatsCollectionOverhead, + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), }, ContentionSubsystemMetrics: txnidcache.NewMetrics(), InsightsMetrics: insights.NewMetrics(), diff --git a/pkg/sql/execinfra/metrics.go b/pkg/sql/execinfra/metrics.go index 17ad207d88db..30123ce27ee6 100644 --- a/pkg/sql/execinfra/metrics.go +++ b/pkg/sql/execinfra/metrics.go @@ -24,11 +24,11 @@ type DistSQLMetrics struct { ContendedQueriesCount *metric.Counter FlowsActive *metric.Gauge FlowsTotal *metric.Counter - MaxBytesHist *metric.Histogram + MaxBytesHist metric.IHistogram CurBytesCount *metric.Gauge VecOpenFDs *metric.Gauge CurDiskBytesCount *metric.Gauge - MaxDiskBytesHist *metric.Histogram + MaxDiskBytesHist metric.IHistogram QueriesSpilled *metric.Counter SpilledBytesWritten *metric.Counter SpilledBytesRead *metric.Counter @@ -120,6 +120,10 @@ var ( } ) +// See pkg/sql/mem_metrics.go +// log10int64times1000 = log10(math.MaxInt64) * 1000, rounded up somewhat +const log10int64times1000 = 19 * 1000 + // MakeDistSQLMetrics instantiates the metrics holder for DistSQL monitoring. func MakeDistSQLMetrics(histogramWindow time.Duration) DistSQLMetrics { return DistSQLMetrics{ @@ -128,14 +132,25 @@ func MakeDistSQLMetrics(histogramWindow time.Duration) DistSQLMetrics { ContendedQueriesCount: metric.NewCounter(metaContendedQueriesCount), FlowsActive: metric.NewGauge(metaFlowsActive), FlowsTotal: metric.NewCounter(metaFlowsTotal), - MaxBytesHist: metric.NewHistogram(metaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - CurBytesCount: metric.NewGauge(metaMemCurBytes), - VecOpenFDs: metric.NewGauge(metaVecOpenFDs), - CurDiskBytesCount: metric.NewGauge(metaDiskCurBytes), - MaxDiskBytesHist: metric.NewHistogram(metaDiskMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - QueriesSpilled: metric.NewCounter(metaQueriesSpilled), - SpilledBytesWritten: metric.NewCounter(metaSpilledBytesWritten), - SpilledBytesRead: metric.NewCounter(metaSpilledBytesRead), + MaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaMemMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), + CurBytesCount: metric.NewGauge(metaMemCurBytes), + VecOpenFDs: metric.NewGauge(metaVecOpenFDs), + CurDiskBytesCount: metric.NewGauge(metaDiskCurBytes), + MaxDiskBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaDiskMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets}), + QueriesSpilled: metric.NewCounter(metaQueriesSpilled), + SpilledBytesWritten: metric.NewCounter(metaSpilledBytesWritten), + SpilledBytesRead: metric.NewCounter(metaSpilledBytesRead), } } diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index e9cfcdf12c95..3eb34a83eb75 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -37,11 +37,11 @@ type EngineMetrics struct { SQLOptPlanCacheHits *metric.Counter SQLOptPlanCacheMisses *metric.Counter - DistSQLExecLatency *metric.Histogram - SQLExecLatency *metric.Histogram - DistSQLServiceLatency *metric.Histogram - SQLServiceLatency *metric.Histogram - SQLTxnLatency *metric.Histogram + DistSQLExecLatency metric.IHistogram + SQLExecLatency metric.IHistogram + DistSQLServiceLatency metric.IHistogram + SQLServiceLatency metric.IHistogram + SQLTxnLatency metric.IHistogram SQLTxnsOpen *metric.Gauge SQLActiveStatements *metric.Gauge SQLContendedTxns *metric.Counter @@ -70,20 +70,20 @@ func (EngineMetrics) MetricStruct() {} // StatsMetrics groups metrics related to SQL Stats collection. type StatsMetrics struct { - SQLStatsMemoryMaxBytesHist *metric.Histogram + SQLStatsMemoryMaxBytesHist metric.IHistogram SQLStatsMemoryCurBytesCount *metric.Gauge - ReportedSQLStatsMemoryMaxBytesHist *metric.Histogram + ReportedSQLStatsMemoryMaxBytesHist metric.IHistogram ReportedSQLStatsMemoryCurBytesCount *metric.Gauge DiscardedStatsCount *metric.Counter SQLStatsFlushStarted *metric.Counter SQLStatsFlushFailure *metric.Counter - SQLStatsFlushDuration *metric.Histogram + SQLStatsFlushDuration metric.IHistogram SQLStatsRemovedRows *metric.Counter - SQLTxnStatsCollectionOverhead *metric.Histogram + SQLTxnStatsCollectionOverhead metric.IHistogram } // StatsMetrics is part of the metric.Struct interface. diff --git a/pkg/sql/mem_metrics.go b/pkg/sql/mem_metrics.go index db2198d6bfd8..248aff4ddc88 100644 --- a/pkg/sql/mem_metrics.go +++ b/pkg/sql/mem_metrics.go @@ -19,7 +19,7 @@ import ( // BaseMemoryMetrics contains a max histogram and a current count of the // bytes allocated by a sql endpoint. type BaseMemoryMetrics struct { - MaxBytesHist *metric.Histogram + MaxBytesHist metric.IHistogram CurBytesCount *metric.Gauge } @@ -30,9 +30,9 @@ type BaseMemoryMetrics struct { // - "internal" for activities related to leases, schema changes, etc. type MemoryMetrics struct { BaseMemoryMetrics - TxnMaxBytesHist *metric.Histogram + TxnMaxBytesHist metric.IHistogram TxnCurBytesCount *metric.Gauge - SessionMaxBytesHist *metric.Histogram + SessionMaxBytesHist metric.IHistogram SessionCurBytesCount *metric.Gauge } @@ -41,6 +41,22 @@ func (MemoryMetrics) MetricStruct() {} var _ metric.Struct = MemoryMetrics{} +// TODO(knz): Until #10014 is addressed, the UI graphs don't have a +// log scale on the Y axis and the histograms are thus displayed using +// a manual log scale: we store the logarithm in the value in the DB +// and plot that logarithm in the UI. +// +// We could, but do not, store the full value in the DB and compute +// the log in the UI, because the current histogram implementation +// does not deal well with large maxima (#10015). +// +// Since the DB stores an integer, we scale the values by 1000 so that +// a modicum of precision is restored when exponentiating the value. +// + +// log10int64times1000 = log10(math.MaxInt64) * 1000, rounded up somewhat +const log10int64times1000 = 19 * 1000 + func makeMemMetricMetadata(name, help string) metric.Metadata { return metric.Metadata{ Name: name, @@ -57,7 +73,13 @@ func MakeBaseMemMetrics(endpoint string, histogramWindow time.Duration) BaseMemo MetaMemMaxBytes := makeMemMetricMetadata(prefix+".max", "Memory usage per sql statement for "+endpoint) MetaMemCurBytes := makeMemMetricMetadata(prefix+".current", "Current sql statement memory usage for "+endpoint) return BaseMemoryMetrics{ - MaxBytesHist: metric.NewHistogram(MetaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), + MaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaMemMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), CurBytesCount: metric.NewGauge(MetaMemCurBytes), } } @@ -71,10 +93,20 @@ func MakeMemMetrics(endpoint string, histogramWindow time.Duration) MemoryMetric MetaMemMaxSessionBytes := makeMemMetricMetadata(prefix+".session.max", "Memory usage per sql session for "+endpoint) MetaMemSessionCurBytes := makeMemMetricMetadata(prefix+".session.current", "Current sql session memory usage for "+endpoint) return MemoryMetrics{ - BaseMemoryMetrics: base, - TxnMaxBytesHist: metric.NewHistogram(MetaMemMaxTxnBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - TxnCurBytesCount: metric.NewGauge(MetaMemTxnCurBytes), - SessionMaxBytesHist: metric.NewHistogram(MetaMemMaxSessionBytes, histogramWindow, metric.MemoryUsage64MBBuckets), + BaseMemoryMetrics: base, + TxnMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaMemMaxTxnBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets}), + TxnCurBytesCount: metric.NewGauge(MetaMemTxnCurBytes), + SessionMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaMemMaxSessionBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets}), SessionCurBytesCount: metric.NewGauge(MetaMemSessionCurBytes), } diff --git a/pkg/sql/pgwire/pre_serve.go b/pkg/sql/pgwire/pre_serve.go index e9d2a975c29e..b87b61ad406b 100644 --- a/pkg/sql/pgwire/pre_serve.go +++ b/pkg/sql/pgwire/pre_serve.go @@ -183,7 +183,7 @@ type tenantIndependentMetrics struct { PreServeBytesOutCount *metric.Counter PreServeConnFailures *metric.Counter PreServeNewConns *metric.Counter - PreServeMaxBytes *metric.Histogram + PreServeMaxBytes metric.IHistogram PreServeCurBytes *metric.Gauge } @@ -193,8 +193,13 @@ func makeTenantIndependentMetrics(histogramWindow time.Duration) tenantIndepende PreServeBytesOutCount: metric.NewCounter(MetaPreServeBytesOut), PreServeNewConns: metric.NewCounter(MetaPreServeNewConns), PreServeConnFailures: metric.NewCounter(MetaPreServeConnFailures), - PreServeMaxBytes: metric.NewHistogram(MetaPreServeMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - PreServeCurBytes: metric.NewGauge(MetaPreServeCurBytes), + PreServeMaxBytes: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaPreServeMaxBytes, + Duration: histogramWindow, + Buckets: metric.MemoryUsage64MBBuckets, + Mode: metric.HistogramModePrometheus, + }), + PreServeCurBytes: metric.NewGauge(MetaPreServeCurBytes), } } diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 1e07fc426b36..56d9a0b14ccc 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -256,7 +256,7 @@ type tenantSpecificMetrics struct { BytesOutCount *metric.Counter Conns *metric.Gauge NewConns *metric.Counter - ConnLatency *metric.Histogram + ConnLatency metric.IHistogram ConnFailures *metric.Counter PGWireCancelTotalCount *metric.Counter PGWireCancelIgnoredCount *metric.Counter @@ -273,9 +273,12 @@ func makeTenantSpecificMetrics( BytesOutCount: metric.NewCounter(MetaBytesOut), Conns: metric.NewGauge(MetaConns), NewConns: metric.NewCounter(MetaNewConns), - ConnLatency: metric.NewHistogram( - MetaConnLatency, histogramWindow, metric.IOLatencyBuckets, - ), + ConnLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: MetaConnLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), ConnFailures: metric.NewCounter(MetaConnFailures), PGWireCancelTotalCount: metric.NewCounter(MetaPGWireCancelTotal), PGWireCancelIgnoredCount: metric.NewCounter(MetaPGWireCancelIgnored), diff --git a/pkg/sql/sqlstats/persistedsqlstats/provider.go b/pkg/sql/sqlstats/persistedsqlstats/provider.go index f6581e68eac1..3a95761a8499 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/provider.go +++ b/pkg/sql/sqlstats/persistedsqlstats/provider.go @@ -43,7 +43,7 @@ type Config struct { // Metrics. FlushCounter *metric.Counter - FlushDuration *metric.Histogram + FlushDuration metric.IHistogram FailureCounter *metric.Counter // Testing knobs. diff --git a/pkg/sql/sqlstats/sslocal/sql_stats.go b/pkg/sql/sqlstats/sslocal/sql_stats.go index 90c3350961eb..92fe6dcf3904 100644 --- a/pkg/sql/sqlstats/sslocal/sql_stats.go +++ b/pkg/sql/sqlstats/sslocal/sql_stats.go @@ -75,7 +75,7 @@ func newSQLStats( uniqueStmtFingerprintLimit *settings.IntSetting, uniqueTxnFingerprintLimit *settings.IntSetting, curMemBytesCount *metric.Gauge, - maxMemBytesHist *metric.Histogram, + maxMemBytesHist metric.IHistogram, insightsWriter insights.WriterProvider, parentMon *mon.BytesMonitor, flushTarget Sink, diff --git a/pkg/sql/sqlstats/sslocal/sslocal_provider.go b/pkg/sql/sqlstats/sslocal/sslocal_provider.go index b508533ada09..e375c9c00e27 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_provider.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_provider.go @@ -35,7 +35,7 @@ func New( maxStmtFingerprints *settings.IntSetting, maxTxnFingerprints *settings.IntSetting, curMemoryBytesCount *metric.Gauge, - maxMemoryBytesHist *metric.Histogram, + maxMemoryBytesHist metric.IHistogram, insightsWriter insights.WriterProvider, pool *mon.BytesMonitor, reportingSink Sink, diff --git a/pkg/sql/ttl/ttljob/ttljob_metrics.go b/pkg/sql/ttl/ttljob/ttljob_metrics.go index d9b450a8d481..ae526f574226 100644 --- a/pkg/sql/ttl/ttljob/ttljob_metrics.go +++ b/pkg/sql/ttl/ttljob/ttljob_metrics.go @@ -96,41 +96,48 @@ func (m *RowLevelTTLAggMetrics) loadMetrics(labelMetrics bool, relation string) } func makeRowLevelTTLAggMetrics(histogramWindowInterval time.Duration) metric.Struct { + sigFigs := 2 b := aggmetric.MakeBuilder("relation") ret := &RowLevelTTLAggMetrics{ - SpanTotalDuration: b.Histogram( - metric.Metadata{ + SpanTotalDuration: b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "jobs.row_level_ttl.span_total_duration", Help: "Duration for processing a span during row level TTL.", Measurement: "nanoseconds", Unit: metric.Unit_NANOSECONDS, MetricType: io_prometheus_client.MetricType_HISTOGRAM, }, - histogramWindowInterval, - metric.LongRunning60mLatencyBuckets, - ), - SelectDuration: b.Histogram( - metric.Metadata{ + MaxVal: time.Hour.Nanoseconds(), + SigFigs: sigFigs, + Duration: histogramWindowInterval, + Buckets: metric.LongRunning60mLatencyBuckets, + }), + SelectDuration: b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "jobs.row_level_ttl.select_duration", Help: "Duration for select requests during row level TTL.", Measurement: "nanoseconds", Unit: metric.Unit_NANOSECONDS, MetricType: io_prometheus_client.MetricType_HISTOGRAM, }, - histogramWindowInterval, - metric.BatchProcessLatencyBuckets, - ), - DeleteDuration: b.Histogram( - metric.Metadata{ + MaxVal: time.Minute.Nanoseconds(), + SigFigs: sigFigs, + Duration: histogramWindowInterval, + Buckets: metric.BatchProcessLatencyBuckets, + }), + DeleteDuration: b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "jobs.row_level_ttl.delete_duration", Help: "Duration for delete requests during row level TTL.", Measurement: "nanoseconds", Unit: metric.Unit_NANOSECONDS, MetricType: io_prometheus_client.MetricType_HISTOGRAM, }, - histogramWindowInterval, - metric.BatchProcessLatencyBuckets, - ), + MaxVal: time.Minute.Nanoseconds(), + SigFigs: sigFigs, + Duration: histogramWindowInterval, + Buckets: metric.BatchProcessLatencyBuckets, + }), RowSelections: b.Counter( metric.Metadata{ Name: "jobs.row_level_ttl.rows_selected", diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index be9b4fabc3d4..d6211bee01ab 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -1561,7 +1561,7 @@ type workQueueMetricsSingle struct { Requested *metric.Counter Admitted *metric.Counter Errored *metric.Counter - WaitDurations *metric.Histogram + WaitDurations metric.IHistogram WaitQueueLength *metric.Gauge } @@ -1622,9 +1622,12 @@ func makeWorkQueueMetricsSingle(name string) workQueueMetricsSingle { Requested: metric.NewCounter(addName(name, requestedMeta)), Admitted: metric.NewCounter(addName(name, admittedMeta)), Errored: metric.NewCounter(addName(name, erroredMeta)), - WaitDurations: metric.NewHistogram( - addName(name, waitDurationsMeta), base.DefaultHistogramWindowInterval(), metric.IOLatencyBuckets, - ), + WaitDurations: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: addName(name, waitDurationsMeta), + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.IOLatencyBuckets, + }), WaitQueueLength: metric.NewGauge(addName(name, waitQueueLengthMeta)), } } diff --git a/pkg/util/metric/BUILD.bazel b/pkg/util/metric/BUILD.bazel index 7d0473c33b40..e94462787b40 100644 --- a/pkg/util/metric/BUILD.bazel +++ b/pkg/util/metric/BUILD.bazel @@ -8,6 +8,7 @@ go_library( srcs = [ "doc.go", "graphite_exporter.go", + "hdrhistogram.go", "histogram_buckets.go", "metric.go", "prometheus_exporter.go", @@ -22,10 +23,13 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/util/metric", visibility = ["//visibility:public"], deps = [ + "//pkg/util", + "//pkg/util/envutil", "//pkg/util/log", "//pkg/util/syncutil", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", + "@com_github_codahale_hdrhistogram//:hdrhistogram", "@com_github_gogo_protobuf//proto", "@com_github_prometheus_client_golang//prometheus", "@com_github_prometheus_client_golang//prometheus/graphite", diff --git a/pkg/util/metric/aggmetric/BUILD.bazel b/pkg/util/metric/aggmetric/BUILD.bazel index daf77ba5d5ff..5984de7e5154 100644 --- a/pkg/util/metric/aggmetric/BUILD.bazel +++ b/pkg/util/metric/aggmetric/BUILD.bazel @@ -17,7 +17,6 @@ go_library( "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//proto", "@com_github_google_btree//:btree", - "@com_github_prometheus_client_golang//prometheus", "@com_github_prometheus_client_model//go", ], ) diff --git a/pkg/util/metric/aggmetric/agg_metric.go b/pkg/util/metric/aggmetric/agg_metric.go index c9afb965d64f..ab5ad03ce5b6 100644 --- a/pkg/util/metric/aggmetric/agg_metric.go +++ b/pkg/util/metric/aggmetric/agg_metric.go @@ -15,7 +15,6 @@ package aggmetric import ( "strings" - "time" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -50,10 +49,8 @@ func (b Builder) Counter(metadata metric.Metadata) *AggCounter { } // Histogram constructs a new AggHistogram with the Builder's labels. -func (b Builder) Histogram( - metadata metric.Metadata, duration time.Duration, buckets []float64, -) *AggHistogram { - return NewHistogram(metadata, duration, buckets, b.labels...) +func (b Builder) Histogram(opts metric.HistogramOptions) *AggHistogram { + return NewHistogram(opts, b.labels...) } type childSet struct { diff --git a/pkg/util/metric/aggmetric/agg_metric_test.go b/pkg/util/metric/aggmetric/agg_metric_test.go index 842a30504889..a139347d38db 100644 --- a/pkg/util/metric/aggmetric/agg_metric_test.go +++ b/pkg/util/metric/aggmetric/agg_metric_test.go @@ -13,6 +13,7 @@ package aggmetric_test import ( "bufio" "bytes" + "fmt" "sort" "strings" "testing" @@ -63,10 +64,15 @@ func TestAggMetric(t *testing.T) { Name: "baz_gauge", }, "tenant_id") r.AddMetric(f) - - h := aggmetric.NewHistogram(metric.Metadata{ - Name: "histo_gram", - }, base.DefaultHistogramWindowInterval(), metric.Count1KBuckets, "tenant_id") + h := aggmetric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ + Name: "histo_gram", + }, + Duration: base.DefaultHistogramWindowInterval(), + MaxVal: 100, + SigFigs: 1, + Buckets: metric.Count1KBuckets, + }, "tenant_id") r.AddMetric(h) tenant2 := roachpb.MustMakeTenantID(2) @@ -87,18 +93,28 @@ func TestAggMetric(t *testing.T) { g3.Inc(3) g3.Dec(1) f2.Update(1.5) + fmt.Println(r) f3.Update(2.5) h2.RecordValue(10) h3.RecordValue(90) - echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, "basic.txt")) + testFile := "basic.txt" + if metric.HdrEnabled() { + testFile = "basic_hdr.txt" + } + echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile)) }) t.Run("destroy", func(t *testing.T) { + fmt.Println(r) g3.Unlink() c2.Unlink() f3.Unlink() h3.Unlink() - echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, "destroy.txt")) + testFile := "destroy.txt" + if metric.HdrEnabled() { + testFile = "destroy_hdr.txt" + } + echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile)) }) t.Run("panic on already exists", func(t *testing.T) { @@ -119,7 +135,11 @@ func TestAggMetric(t *testing.T) { c2 = c.AddChild(tenant2.String()) f3 = f.AddChild(tenant3.String()) h3 = h.AddChild(tenant3.String()) - echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, "add_after_destroy.txt")) + testFile := "add_after_destroy.txt" + if metric.HdrEnabled() { + testFile = "add_after_destroy_hdr.txt" + } + echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile)) }) t.Run("panic on label length mismatch", func(t *testing.T) { @@ -135,8 +155,13 @@ func TestAggMetricBuilder(t *testing.T) { c := b.Counter(metric.Metadata{Name: "foo_counter"}) g := b.Gauge(metric.Metadata{Name: "bar_gauge"}) f := b.GaugeFloat64(metric.Metadata{Name: "baz_gauge"}) - h := b.Histogram(metric.Metadata{Name: "histo_gram"}, - base.DefaultHistogramWindowInterval(), metric.Count1KBuckets) + h := b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{Name: "histo_gram"}, + Duration: base.DefaultHistogramWindowInterval(), + MaxVal: 100, + SigFigs: 1, + Buckets: metric.Count1KBuckets, + }) for i := 5; i < 10; i++ { tenantLabel := roachpb.MustMakeTenantID(uint64(i)).String() diff --git a/pkg/util/metric/aggmetric/histogram.go b/pkg/util/metric/aggmetric/histogram.go index 68d30fab17a2..1c07cc1ef3d3 100644 --- a/pkg/util/metric/aggmetric/histogram.go +++ b/pkg/util/metric/aggmetric/histogram.go @@ -11,10 +11,7 @@ package aggmetric import ( - "time" - "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/prometheus/client_golang/prometheus" io_prometheus_client "github.com/prometheus/client_model/go" ) @@ -23,8 +20,8 @@ import ( // children, while its children are additionally exported to prometheus via the // PrometheusIterable interface. type AggHistogram struct { - h metric.Histogram - create func() *metric.Histogram + h metric.IHistogram + create func() metric.IHistogram childSet } @@ -34,14 +31,12 @@ var _ metric.PrometheusExportable = (*AggHistogram)(nil) var _ metric.WindowedHistogram = (*AggHistogram)(nil) // NewHistogram constructs a new AggHistogram. -func NewHistogram( - metadata metric.Metadata, duration time.Duration, buckets []float64, childLabels ...string, -) *AggHistogram { - create := func() *metric.Histogram { - return metric.NewHistogram(metadata, duration, buckets) +func NewHistogram(opts metric.HistogramOptions, childLabels ...string) *AggHistogram { + create := func() metric.IHistogram { + return metric.NewHistogram(opts) } a := &AggHistogram{ - h: *create(), + h: create(), create: create, } a.init(childLabels) @@ -96,19 +91,13 @@ func (a *AggHistogram) ToPrometheusMetric() *io_prometheus_client.Metric { return a.h.ToPrometheusMetric() } -// Windowed returns a copy of the current windowed histogram data and its -// rotation interval. -func (a *AggHistogram) Windowed() prometheus.Histogram { - return a.h.Windowed() -} - // AddChild adds a Counter to this AggCounter. This method panics if a Counter // already exists for this set of labelVals. func (a *AggHistogram) AddChild(labelVals ...string) *Histogram { child := &Histogram{ parent: a, labelValuesSlice: labelValuesSlice(labelVals), - h: *a.create(), + h: a.create(), } a.add(child) return child @@ -121,7 +110,7 @@ func (a *AggHistogram) AddChild(labelVals ...string) *Histogram { type Histogram struct { parent *AggHistogram labelValuesSlice - h metric.Histogram + h metric.IHistogram } // ToPrometheusMetric constructs a prometheus metric for this Histogram. diff --git a/pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt b/pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt new file mode 100644 index 000000000000..ffcbd4571a33 --- /dev/null +++ b/pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt @@ -0,0 +1,23 @@ +echo +---- +bar_gauge 4 +bar_gauge{tenant_id="2"} 2 +bar_gauge{tenant_id="3"} 0 +baz_gauge 4 +baz_gauge{tenant_id="2"} 1.5 +baz_gauge{tenant_id="3"} 0 +foo_counter 6 +foo_counter{tenant_id="2"} 0 +foo_counter{tenant_id="3"} 4 +histo_gram_bucket{le="+Inf"} 2 +histo_gram_bucket{le="10"} 1 +histo_gram_bucket{le="91"} 2 +histo_gram_bucket{tenant_id="2",le="+Inf"} 1 +histo_gram_bucket{tenant_id="2",le="10"} 1 +histo_gram_bucket{tenant_id="3",le="+Inf"} 0 +histo_gram_count 2 +histo_gram_count{tenant_id="2"} 1 +histo_gram_count{tenant_id="3"} 0 +histo_gram_sum 101 +histo_gram_sum{tenant_id="2"} 10 +histo_gram_sum{tenant_id="3"} 0 \ No newline at end of file diff --git a/pkg/util/metric/aggmetric/testdata/basic_hdr.txt b/pkg/util/metric/aggmetric/testdata/basic_hdr.txt new file mode 100644 index 000000000000..a796b8ef3406 --- /dev/null +++ b/pkg/util/metric/aggmetric/testdata/basic_hdr.txt @@ -0,0 +1,24 @@ +echo +---- +bar_gauge 4 +bar_gauge{tenant_id="2"} 2 +bar_gauge{tenant_id="3"} 2 +baz_gauge 4 +baz_gauge{tenant_id="2"} 1.5 +baz_gauge{tenant_id="3"} 2.5 +foo_counter 6 +foo_counter{tenant_id="2"} 2 +foo_counter{tenant_id="3"} 4 +histo_gram_bucket{le="+Inf"} 2 +histo_gram_bucket{le="10"} 1 +histo_gram_bucket{le="91"} 2 +histo_gram_bucket{tenant_id="2",le="+Inf"} 1 +histo_gram_bucket{tenant_id="2",le="10"} 1 +histo_gram_bucket{tenant_id="3",le="+Inf"} 1 +histo_gram_bucket{tenant_id="3",le="91"} 1 +histo_gram_count 2 +histo_gram_count{tenant_id="2"} 1 +histo_gram_count{tenant_id="3"} 1 +histo_gram_sum 101 +histo_gram_sum{tenant_id="2"} 10 +histo_gram_sum{tenant_id="3"} 91 \ No newline at end of file diff --git a/pkg/util/metric/aggmetric/testdata/destroy_hdr.txt b/pkg/util/metric/aggmetric/testdata/destroy_hdr.txt new file mode 100644 index 000000000000..dd17b7aae5bb --- /dev/null +++ b/pkg/util/metric/aggmetric/testdata/destroy_hdr.txt @@ -0,0 +1,17 @@ +echo +---- +bar_gauge 4 +bar_gauge{tenant_id="2"} 2 +baz_gauge 4 +baz_gauge{tenant_id="2"} 1.5 +foo_counter 6 +foo_counter{tenant_id="3"} 4 +histo_gram_bucket{le="+Inf"} 2 +histo_gram_bucket{le="10"} 1 +histo_gram_bucket{le="91"} 2 +histo_gram_bucket{tenant_id="2",le="+Inf"} 1 +histo_gram_bucket{tenant_id="2",le="10"} 1 +histo_gram_count 2 +histo_gram_count{tenant_id="2"} 1 +histo_gram_sum 101 +histo_gram_sum{tenant_id="2"} 10 \ No newline at end of file diff --git a/pkg/util/metric/hdrhistogram.go b/pkg/util/metric/hdrhistogram.go new file mode 100644 index 000000000000..e33582b9c2e3 --- /dev/null +++ b/pkg/util/metric/hdrhistogram.go @@ -0,0 +1,248 @@ +// Copyright 2023 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 metric + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/codahale/hdrhistogram" + prometheusgo "github.com/prometheus/client_model/go" +) + +const ( + // HdrHistogramMaxLatency is the maximum value tracked in latency histograms. Higher + // values will be recorded as this value instead. + HdrHistogramMaxLatency = 10 * time.Second + + // The number of histograms to keep in rolling window. + hdrHistogramHistWrapNum = 2 // TestSampleInterval is passed to histograms during tests which don't +) + +// A HdrHistogram collects observed values by keeping bucketed counts. For +// convenience, internally two sets of buckets are kept: A cumulative set (i.e. +// data is never evicted) and a windowed set (which keeps only recently +// collected samples). +// +// Top-level methods generally apply to the cumulative buckets; the windowed +// variant is exposed through the Windowed method. +// +// TODO(#96357): remove HdrHistogram model entirely once the Prometheus +// backed histogram and its bucket boundaries have been reliably proven in +// production. +type HdrHistogram struct { + Metadata + maxVal int64 + mu struct { + syncutil.Mutex + cumulative *hdrhistogram.Histogram + *tickHelper + sliding *hdrhistogram.WindowedHistogram + } +} + +var _ IHistogram = &HdrHistogram{} +var _ PrometheusExportable = &HdrHistogram{} +var _ Iterable = &HdrHistogram{} + +// NewHdrHistogram initializes a given Histogram. The contained windowed histogram +// rotates every 'duration'; both the windowed and the cumulative histogram +// track nonnegative values up to 'maxVal' with 'sigFigs' decimal points of +// precision. +func NewHdrHistogram( + metadata Metadata, duration time.Duration, maxVal int64, sigFigs int, +) *HdrHistogram { + h := &HdrHistogram{ + Metadata: metadata, + maxVal: maxVal, + } + wHist := hdrhistogram.NewWindowed(hdrHistogramHistWrapNum, 0, maxVal, sigFigs) + h.mu.cumulative = hdrhistogram.New(0, maxVal, sigFigs) + h.mu.sliding = wHist + h.mu.tickHelper = &tickHelper{ + nextT: now(), + tickInterval: duration / hdrHistogramHistWrapNum, + onTick: func() { + wHist.Rotate() + }, + } + return h +} + +// NewHdrLatency is a convenience function which returns a histogram with +// suitable defaults for latency tracking. Values are expressed in ns, +// are truncated into the interval [0, HdrHistogramMaxLatency] and are recorded +// with one digit of precision (i.e. errors of <10ms at 100ms, <6s at 60s). +// +// The windowed portion of the Histogram retains values for approximately +// histogramWindow. +func NewHdrLatency(metadata Metadata, histogramWindow time.Duration) *HdrHistogram { + return NewHdrHistogram( + metadata, histogramWindow, HdrHistogramMaxLatency.Nanoseconds(), 1, + ) +} + +// RecordValue adds the given value to the histogram. Recording a value in +// excess of the configured maximum value for that histogram results in +// recording the maximum value instead. +func (h *HdrHistogram) RecordValue(v int64) { + h.mu.Lock() + defer h.mu.Unlock() + + if h.mu.sliding.Current.RecordValue(v) != nil { + _ = h.mu.sliding.Current.RecordValue(h.maxVal) + } + if h.mu.cumulative.RecordValue(v) != nil { + _ = h.mu.cumulative.RecordValue(h.maxVal) + } +} + +// TotalCount returns the (cumulative) number of samples. +func (h *HdrHistogram) TotalCount() int64 { + h.mu.Lock() + defer h.mu.Unlock() + return h.mu.cumulative.TotalCount() +} + +// Min returns the minimum. +func (h *HdrHistogram) Min() int64 { + h.mu.Lock() + defer h.mu.Unlock() + return h.mu.cumulative.Min() +} + +// Inspect calls the closure with the empty string and the receiver. +func (h *HdrHistogram) Inspect(f func(interface{})) { + h.mu.Lock() + maybeTick(h.mu.tickHelper) + h.mu.Unlock() + f(h) +} + +// GetType returns the prometheus type enum for this metric. +func (h *HdrHistogram) GetType() *prometheusgo.MetricType { + return prometheusgo.MetricType_HISTOGRAM.Enum() +} + +// ToPrometheusMetric returns a filled-in prometheus metric of the right type. +func (h *HdrHistogram) ToPrometheusMetric() *prometheusgo.Metric { + hist := &prometheusgo.Histogram{} + + h.mu.Lock() + maybeTick(h.mu.tickHelper) + bars := h.mu.cumulative.Distribution() + hist.Bucket = make([]*prometheusgo.Bucket, 0, len(bars)) + + var cumCount uint64 + var sum float64 + for _, bar := range bars { + if bar.Count == 0 { + // No need to expose trivial buckets. + continue + } + upperBound := float64(bar.To) + sum += upperBound * float64(bar.Count) + + cumCount += uint64(bar.Count) + curCumCount := cumCount // need a new alloc thanks to bad proto code + + hist.Bucket = append(hist.Bucket, &prometheusgo.Bucket{ + CumulativeCount: &curCumCount, + UpperBound: &upperBound, + }) + } + hist.SampleCount = &cumCount + hist.SampleSum = &sum // can do better here; we approximate in the loop + h.mu.Unlock() + + return &prometheusgo.Metric{ + Histogram: hist, + } +} + +// TotalCountWindowed implements the WindowedHistogram interface. +func (h *HdrHistogram) TotalCountWindowed() int64 { + return int64(h.ToPrometheusMetricWindowed().Histogram.GetSampleCount()) +} + +// TotalSumWindowed implements the WindowedHistogram interface. +func (h *HdrHistogram) TotalSumWindowed() float64 { + return h.ToPrometheusMetricWindowed().Histogram.GetSampleSum() +} + +func (h *HdrHistogram) toPrometheusMetricWindowedLocked() *prometheusgo.Metric { + hist := &prometheusgo.Histogram{} + + maybeTick(h.mu.tickHelper) + bars := h.mu.sliding.Current.Distribution() + hist.Bucket = make([]*prometheusgo.Bucket, 0, len(bars)) + + var cumCount uint64 + var sum float64 + for _, bar := range bars { + if bar.Count == 0 { + // No need to expose trivial buckets. + continue + } + upperBound := float64(bar.To) + sum += upperBound * float64(bar.Count) + + cumCount += uint64(bar.Count) + curCumCount := cumCount // need a new alloc thanks to bad proto code + + hist.Bucket = append(hist.Bucket, &prometheusgo.Bucket{ + CumulativeCount: &curCumCount, + UpperBound: &upperBound, + }) + } + hist.SampleCount = &cumCount + hist.SampleSum = &sum // can do better here; we approximate in the loop + + return &prometheusgo.Metric{ + Histogram: hist, + } +} + +// ToPrometheusMetricWindowed returns a filled-in prometheus metric of the right type. +func (h *HdrHistogram) ToPrometheusMetricWindowed() *prometheusgo.Metric { + h.mu.Lock() + defer h.mu.Unlock() + return h.toPrometheusMetricWindowedLocked() +} + +// GetMetadata returns the metric's metadata including the Prometheus +// MetricType. +func (h *HdrHistogram) GetMetadata() Metadata { + baseMetadata := h.Metadata + baseMetadata.MetricType = prometheusgo.MetricType_HISTOGRAM + return baseMetadata +} + +func (h *HdrHistogram) ValueAtQuantileWindowed(q float64) float64 { + h.mu.Lock() + defer h.mu.Unlock() + + return ValueAtQuantileWindowed(h.toPrometheusMetricWindowedLocked().Histogram, q) +} + +func (h *HdrHistogram) Mean() float64 { + h.mu.Lock() + defer h.mu.Unlock() + + return h.mu.cumulative.Mean() +} + +func (h *HdrHistogram) TotalSum() float64 { + h.mu.Lock() + defer h.mu.Unlock() + + return h.ToPrometheusMetric().GetSummary().GetSampleSum() +} diff --git a/pkg/util/metric/metric.go b/pkg/util/metric/metric.go index a6b794422e4b..afd19cf83638 100644 --- a/pkg/util/metric/metric.go +++ b/pkg/util/metric/metric.go @@ -17,6 +17,8 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/gogo/protobuf/proto" @@ -25,11 +27,9 @@ import ( "github.com/rcrowley/go-metrics" ) -const ( - // TestSampleInterval is passed to histograms during tests which don't - // want to concern themselves with supplying a "correct" interval. - TestSampleInterval = time.Duration(math.MaxInt64) -) +// TestSampleInterval is passed to histograms during tests which don't +// want to concern themselves with supplying a "correct" interval. +const TestSampleInterval = time.Duration(math.MaxInt64) // Iterable provides a method for synchronized access to interior objects. type Iterable interface { @@ -176,10 +176,81 @@ func maybeTick(m periodic) { } } +// useHdrHistogramsEnvVar can be used to switch all histograms to use the +// legacy HDR histograms (except for those that explicitly force the use +// of the newer Prometheus via HistogramModePrometheus). HDR Histograms +// dynamically generate bucket boundaries, which can lead to hundreds of +// buckets. This can cause performance issues with timeseries databases +// like Prometheus. +const useHdrHistogramsEnvVar = "COCKROACH_ENABLE_HDR_HISTOGRAMS" + +var hdrEnabled = util.ConstantWithMetamorphicTestBool(useHdrHistogramsEnvVar, envutil.EnvOrDefaultBool(useHdrHistogramsEnvVar, false)) + +// HdrEnabled returns whether or not the HdrHistogram model is enabled +// in the metric package. Primarily useful in tests where we want to validate +// different outputs depending on whether or not HDR is enabled. +func HdrEnabled() bool { + return hdrEnabled +} + +type HistogramMode byte + +const ( + // HistogramModePrometheus will force the constructed histogram to use + // the Prometheus histogram model, regardless of the value of + // useHdrHistogramsEnvVar. This option should be used for all + // newly defined histograms moving forward. + // + // NB: If neither this mode nor the HistogramModePreferHdrLatency mode + // is set, MaxVal and SigFigs must be defined to maintain backwards + // compatibility with the legacy HdrHistogram model. + HistogramModePrometheus HistogramMode = iota + 1 + // HistogramModePreferHdrLatency will cause the returned histogram to + // use the HdrHistgoram model and be configured with suitable defaults + // for latency tracking iff useHdrHistogramsEnvVar is enabled. + // + // NB: If this option is set, no MaxVal or SigFigs are required in the + // HistogramOptions to maintain backwards compatibility with the legacy + // HdrHistogram model, since suitable defaults are used for both. + HistogramModePreferHdrLatency +) + +type HistogramOptions struct { + // Metadata is the metric Metadata associated with the histogram. + Metadata Metadata + // Duration is the histogram's window duration. + Duration time.Duration + // MaxVal is only relevant to the HdrHistogram, and represents the + // highest trackable value in the resulting histogram buckets. + MaxVal int64 + // SigFigs is only relevant to the HdrHistogram, and represents + // the number of significant figures to be used to determine the + // degree of accuracy used in measurements. + SigFigs int + // Buckets are only relevant to Prometheus histograms, and represent + // the pre-defined histogram bucket boundaries to be used. + Buckets []float64 + // Mode defines the type of histogram to be used. See individual + // comments on each HistogramMode value for details. + Mode HistogramMode +} + +func NewHistogram(opt HistogramOptions) IHistogram { + if hdrEnabled && opt.Mode != HistogramModePrometheus { + if opt.Mode == HistogramModePreferHdrLatency { + return NewHdrLatency(opt.Metadata, opt.Duration) + } else { + return NewHdrHistogram(opt.Metadata, opt.Duration, opt.MaxVal, opt.SigFigs) + } + } else { + return newHistogram(opt.Metadata, opt.Duration, opt.Buckets) + } +} + // NewHistogram is a prometheus-backed histogram. Depending on the value of // opts.Buckets, this is suitable for recording any kind of quantity. Common // sensible choices are {IO,Network}LatencyBuckets. -func NewHistogram(meta Metadata, windowDuration time.Duration, buckets []float64) *Histogram { +func newHistogram(meta Metadata, windowDuration time.Duration, buckets []float64) *Histogram { // TODO(obs-inf): prometheus supports labeled histograms but they require more // plumbing and don't fit into the PrometheusObservable interface any more. opts := prometheus.HistogramOpts{ @@ -236,6 +307,21 @@ type Histogram struct { } } +type IHistogram interface { + Iterable + PrometheusExportable + WindowedHistogram + + RecordValue(n int64) + TotalCount() int64 + TotalSum() float64 + TotalCountWindowed() int64 + TotalSumWindowed() float64 + Mean() float64 +} + +var _ IHistogram = &Histogram{} + func (h *Histogram) nextTick() time.Time { h.windowed.RLock() defer h.windowed.RUnlock() @@ -326,7 +412,8 @@ func (h *Histogram) TotalSumWindowed() float64 { // Mean returns the (cumulative) mean of samples. func (h *Histogram) Mean() float64 { - return h.TotalSum() / float64(h.TotalCount()) + pm := h.ToPrometheusMetric() + return pm.Histogram.GetSampleSum() / float64(pm.Histogram.GetSampleCount()) } // ValueAtQuantileWindowed implements the WindowedHistogram interface. diff --git a/pkg/util/metric/metric_ext_test.go b/pkg/util/metric/metric_ext_test.go index a06cdfc0cb63..60c14ba25bc5 100644 --- a/pkg/util/metric/metric_ext_test.go +++ b/pkg/util/metric/metric_ext_test.go @@ -25,7 +25,12 @@ func TestHistogramPrometheus(t *testing.T) { // Regression test against https://github.com/cockroachdb/cockroach/pull/88331. // The output includes buckets for which the upper bound equals the previous // bucket's upper bound. - h := metric.NewHistogram(metric.Metadata{}, time.Second, []float64{1, 2, 3, 4, 5, 6, 10, 20, 30}) + h := metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metric.Metadata{}, + Duration: time.Second, + Buckets: []float64{1, 2, 3, 4, 5, 6, 10, 20, 30}, + }) h.RecordValue(1) h.RecordValue(5) h.RecordValue(5) diff --git a/pkg/util/metric/metric_test.go b/pkg/util/metric/metric_test.go index 2a273bc7fa85..d8a451ec06ad 100644 --- a/pkg/util/metric/metric_test.go +++ b/pkg/util/metric/metric_test.go @@ -112,17 +112,18 @@ func TestHistogram(t *testing.T) { return &n } - h := NewHistogram( - Metadata{}, - time.Hour, - []float64{ + h := NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{}, + Duration: time.Hour, + Buckets: []float64{ 1.0, 5.0, 10.0, 25.0, 100.0, }, - ) + }) // should return 0 if no observations are made require.Equal(t, 0.0, h.ValueAtQuantileWindowed(0)) @@ -236,23 +237,24 @@ func TestNewHistogramRotate(t *testing.T) { defer TestingSetNow(nil)() setNow(0) - h := NewHistogram(emptyMetadata, 10*time.Second, nil) + h := NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: emptyMetadata, + Duration: 10 * time.Second, + Buckets: nil, + }) for i := 0; i < 4; i++ { // Windowed histogram is initially empty. h.Inspect(func(interface{}) {}) // triggers ticking - var m prometheusgo.Metric - require.NoError(t, h.Windowed().Write(&m)) - require.Zero(t, *m.Histogram.SampleSum) + require.Zero(t, h.TotalSumWindowed()) // But cumulative histogram has history (if i > 0). - require.EqualValues(t, i, *h.ToPrometheusMetric().Histogram.SampleCount) + require.EqualValues(t, i, h.TotalCount()) // Add a measurement and verify it's there. { h.RecordValue(12345) f := float64(12345) - var m prometheusgo.Metric - require.NoError(t, h.Windowed().Write(&m)) - require.Equal(t, *m.Histogram.SampleSum, f) + require.Equal(t, h.TotalSumWindowed(), f) } // Tick. This rotates the histogram. setNow(time.Duration(i+1) * 10 * time.Second) diff --git a/pkg/util/metric/registry_test.go b/pkg/util/metric/registry_test.go index 5d2b2a6c4e88..48f4aba216bd 100644 --- a/pkg/util/metric/registry_test.go +++ b/pkg/util/metric/registry_test.go @@ -76,14 +76,19 @@ func TestRegistry(t *testing.T) { topCounter := NewCounter(Metadata{Name: "top.counter"}) r.AddMetric(topCounter) - r.AddMetric(NewHistogram(Metadata{Name: "top.histogram"}, time.Minute, Count1KBuckets)) + r.AddMetric(NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{Name: "top.histogram"}, + Duration: time.Minute, + Buckets: Count1KBuckets, + })) r.AddMetric(NewGauge(Metadata{Name: "bottom.gauge"})) ms := &struct { StructGauge *Gauge StructGauge64 *GaugeFloat64 StructCounter *Counter - StructHistogram *Histogram + StructHistogram IHistogram NestedStructGauge NestedStruct ArrayStructCounters [4]*Counter // Ensure that nil struct values in arrays are safe. @@ -92,7 +97,7 @@ func TestRegistry(t *testing.T) { privateStructGauge *Gauge privateStructGauge64 *GaugeFloat64 privateStructCounter *Counter - privateStructHistogram *Histogram + privateStructHistogram IHistogram privateNestedStructGauge NestedStruct privateArrayStructCounters [2]*Counter NotAMetric int @@ -100,10 +105,15 @@ func TestRegistry(t *testing.T) { ReallyNotAMetric *Registry DefinitelyNotAnArrayOfMetrics [2]int }{ - StructGauge: NewGauge(Metadata{Name: "struct.gauge"}), - StructGauge64: NewGaugeFloat64(Metadata{Name: "struct.gauge64"}), - StructCounter: NewCounter(Metadata{Name: "struct.counter"}), - StructHistogram: NewHistogram(Metadata{Name: "struct.histogram"}, time.Minute, Count1KBuckets), + StructGauge: NewGauge(Metadata{Name: "struct.gauge"}), + StructGauge64: NewGaugeFloat64(Metadata{Name: "struct.gauge64"}), + StructCounter: NewCounter(Metadata{Name: "struct.counter"}), + StructHistogram: NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{Name: "struct.histogram"}, + Duration: time.Minute, + Buckets: Count1KBuckets, + }), NestedStructGauge: NestedStruct{ NestedStructGauge: NewGauge(Metadata{Name: "nested.struct.gauge"}), }, @@ -119,10 +129,15 @@ func TestRegistry(t *testing.T) { NestedStructGauge: NewGauge(Metadata{Name: "nested.struct.array.1.gauge"}), }, }, - privateStructGauge: NewGauge(Metadata{Name: "private.struct.gauge"}), - privateStructGauge64: NewGaugeFloat64(Metadata{Name: "private.struct.gauge64"}), - privateStructCounter: NewCounter(Metadata{Name: "private.struct.counter"}), - privateStructHistogram: NewHistogram(Metadata{Name: "private.struct.histogram"}, time.Minute, Count1KBuckets), + privateStructGauge: NewGauge(Metadata{Name: "private.struct.gauge"}), + privateStructGauge64: NewGaugeFloat64(Metadata{Name: "private.struct.gauge64"}), + privateStructCounter: NewCounter(Metadata{Name: "private.struct.counter"}), + privateStructHistogram: NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{Name: "private.struct.histogram"}, + Duration: time.Minute, + Buckets: Count1KBuckets, + }), privateNestedStructGauge: NestedStruct{ NestedStructGauge: NewGauge(Metadata{Name: "private.nested.struct.gauge"}), }, diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index 4ea0489afa20..5ba2493730dd 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -195,7 +195,7 @@ type BytesMonitor struct { // maxBytesHist is the metric object used to track the high watermark of bytes // allocated by the monitor during its lifetime. - maxBytesHist *metric.Histogram + maxBytesHist metric.IHistogram } // name identifies this monitor in logging messages. @@ -273,7 +273,7 @@ func NewMonitor( name redact.RedactableString, res Resource, curCount *metric.Gauge, - maxHist *metric.Histogram, + maxHist metric.IHistogram, increment int64, noteworthy int64, settings *cluster.Settings, @@ -289,7 +289,7 @@ func NewMonitorWithLimit( res Resource, limit int64, curCount *metric.Gauge, - maxHist *metric.Histogram, + maxHist metric.IHistogram, increment int64, noteworthy int64, settings *cluster.Settings, @@ -386,7 +386,7 @@ func NewUnlimitedMonitor( name redact.RedactableString, res Resource, curCount *metric.Gauge, - maxHist *metric.Histogram, + maxHist metric.IHistogram, noteworthy int64, settings *cluster.Settings, ) *BytesMonitor { @@ -485,7 +485,7 @@ func (mm *BytesMonitor) AllocBytes() int64 { } // SetMetrics sets the metric objects for the monitor. -func (mm *BytesMonitor) SetMetrics(curCount *metric.Gauge, maxHist *metric.Histogram) { +func (mm *BytesMonitor) SetMetrics(curCount *metric.Gauge, maxHist metric.IHistogram) { mm.mu.Lock() defer mm.mu.Unlock() mm.mu.curBytesCount = curCount diff --git a/pkg/util/schedulerlatency/scheduler_latency_test.go b/pkg/util/schedulerlatency/scheduler_latency_test.go index 1ec15ed46987..2539e6f62adb 100644 --- a/pkg/util/schedulerlatency/scheduler_latency_test.go +++ b/pkg/util/schedulerlatency/scheduler_latency_test.go @@ -170,7 +170,12 @@ func TestComputeSchedulerPercentileAgainstPrometheus(t *testing.T) { } // Compare values against metric.Histogram (prometheus-based implementation) - promhist := metric.NewHistogram(metric.Metadata{}, time.Hour, hist.Buckets) + promhist := metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metric.Metadata{}, + Duration: time.Hour, + Buckets: hist.Buckets, + }) for i := 0; i < len(hist.Counts); i++ { for j := 0; j < int(hist.Counts[i]); j++ { // Since the scheduler buckets are non-inclusive of Upper Bound and prometheus From 4b32a98cbdd44243297847e2291f731554f2fe1d Mon Sep 17 00:00:00 2001 From: Alex Barganier Date: Mon, 30 Jan 2023 16:23:56 -0400 Subject: [PATCH 7/8] pkg/util/metric: increase bucket counts for Prometheus histograms This patch increases the fidelity of the histogram buckets for the new Prometheus model. This is primarily done by increasing the bucket counts for all latency buckets, but may also be manually tweaked according to feedback from various engineering teams for their own use cases. Release note (ops change): Prometheus histograms will now export more buckets across the board to improve precision & fidelity of information reported by histogram metrics, such as quantiles. This will lead to an increase in storage requirements to process these histogram metrics in downstream systems like Prometheus, but should still be a marked improvement when compared to the legacy HdrHistogram model. If users have issues with the precision of these bucket boundaries, they can set the environment variable `COCKROACH_ENABLE_HDR_HISTOGRAMS=true` to revert to using the legacy HdrHistogram model instead, although this is not recommended otherwise as the HdrHistogram strains systems like Prometheus with excessive numbers of histogram buckets. Note that HdrHistograms are slated for full deprecation in upcoming releases. --- pkg/util/metric/histogram_buckets.go | 294 +++++++++++++++++----- pkg/util/metric/histogram_buckets_test.go | 8 +- 2 files changed, 241 insertions(+), 61 deletions(-) diff --git a/pkg/util/metric/histogram_buckets.go b/pkg/util/metric/histogram_buckets.go index 8ce3c9dce884..bf9f70579afb 100644 --- a/pkg/util/metric/histogram_buckets.go +++ b/pkg/util/metric/histogram_buckets.go @@ -17,20 +17,65 @@ package metric var IOLatencyBuckets = []float64{ // Generated via TestHistogramBuckets/IOLatencyBuckets. 10000.000000, // 10µs - 26826.957953, // 26.826µs - 71968.567300, // 71.968µs - 193069.772888, // 193.069µs - 517947.467923, // 517.947µs - 1389495.494373, // 1.389495ms - 3727593.720315, // 3.727593ms - 10000000.000000, // 9.999999ms - 26826957.952797, // 26.826957ms - 71968567.300115, // 71.968567ms - 193069772.888325, // 193.069772ms - 517947467.923120, // 517.947467ms - 1389495494.373135, // 1.389495494s - 3727593720.314933, // 3.72759372s - 9999999999.999981, // 9.999999999s + 12638.482029, // 12.638µs + 15973.122801, // 15.973µs + 20187.602547, // 20.187µs + 25514.065200, // 25.514µs + 32245.905453, // 32.245µs + 40753.929659, // 40.753µs + 51506.780762, // 51.506µs + 65096.752305, // 65.096µs + 82272.413417, // 82.272µs + 103979.841848, // 103.979µs + 131414.736261, // 131.414µs + 166088.278263, // 166.088µs + 209910.372011, // 209.91µs + 265294.846443, // 265.294µs + 335292.414925, // 335.292µs + 423758.716060, // 423.758µs + 535566.691771, // 535.566µs + 676875.000946, // 676.875µs + 855467.253557, // 855.467µs + 1081180.751077, // 1.08118ms + 1366448.349295, // 1.366448ms + 1726983.290659, // 1.726983ms + 2182644.728397, // 2.182644ms + 2758531.617629, // 2.758531ms + 3486365.227678, // 3.486365ms + 4406236.427774, // 4.406236ms + 5568813.990945, // 5.568813ms + 7038135.554932, // 7.038135ms + 8895134.973108, // 8.895134ms + 11242100.350621, // 11.2421ms + 14208308.325339, // 14.208308ms + 17957144.943716, // 17.957144ms + 22695105.366947, // 22.695105ms + 28683168.133420, // 28.683168ms + 36251170.499885, // 36.25117ms + 45815976.690545, // 45.815976ms + 57904439.806025, // 57.904439ms + 73182422.190762, // 73.182422ms + 92491472.772173, // 92.491472ms + 116895181.649858, // 116.895181ms + 147737765.259851, // 147.737765ms + 186718109.129192, // 186.718109ms + 235983346.678219, // 235.983346ms + 298247128.621688, // 298.247128ms + 376939097.538835, // 376.939097ms + 476393801.040133, // 476.393801ms + 602089449.333611, // 602.089449ms + 760949668.545986, // 760.949668ms + 961724871.115294, // 961.724871ms + 1215474250.076283, // 1.21547425s + 1536174946.671824, // 1.536174946s + 1941491945.743876, // 1.941491945s + 2453751106.639811, // 2.453751106s + 3101168926.574770, // 3.101168926s + 3919406774.847209, // 3.919406774s + 4953535208.959157, // 4.953535208s + 6260516572.014802, // 6.260516572s + 7912342618.981298, // 7.912342618s + 9999999999.999969, // 9.999999999s } // NetworkLatencyBuckets are prometheus histogram buckets suitable for a histogram @@ -39,21 +84,66 @@ var IOLatencyBuckets = []float64{ // range during normal operation. var NetworkLatencyBuckets = []float64{ // Generated via TestHistogramBuckets/NetworkLatencyBuckets. - 500000.000000, // 500µs - 860513.842995, // 860.513µs - 1480968.147973, // 1.480968ms - 2548787.184731, // 2.548787ms - 4386533.310619, // 4.386533ms - 7549345.273094, // 7.549345ms - 12992632.226094, // 12.992632ms - 22360679.774998, // 22.360679ms - 38483348.970335, // 38.483348ms - 66230909.027573, // 66.230909ms - 113985228.104760, // 113.985228ms - 196171733.362212, // 196.171733ms - 337616984.325077, // 337.616984ms - 581048177.284016, // 581.048177ms - 999999999.999999, // 999.999999ms + 500000.000000, // 500µs + 568747.715565, // 568.747µs + 646947.927922, // 646.947µs + 735900.312190, // 735.9µs + 837083.242884, // 837.083µs + 952178.364257, // 952.178µs + 1083098.538963, // 1.083098ms + 1232019.639535, // 1.232019ms + 1401416.711034, // 1.401416ms + 1594105.105912, // 1.594105ms + 1813287.274717, // 1.813287ms + 2062605.990318, // 2.062605ms + 2346204.890209, // 2.346204ms + 2668797.343109, // 2.668797ms + 3035744.784401, // 3.035744ms + 3453145.822334, // 3.453145ms + 3927937.595933, // 3.927937ms + 4468011.069141, // 4.468011ms + 5082342.177389, // 5.082342ms + 5781141.006222, // 5.781141ms + 6576021.481300, // 6.576021ms + 7480194.389996, // 7.480194ms + 8508686.942589, // 8.508686ms + 9678592.522117, // 9.678592ms + 11009354.773683, // 11.009354ms + 12523090.754761, // 12.52309ms + 14244958.517175, // 14.244958ms + 16203575.229933, // 16.203575ms + 18431492.792031, // 18.431492ms + 20965738.839853, // 20.965738ms + 23848432.140611, // 23.848432ms + 27127482.599575, // 27.127482ms + 30857387.515093, // 30.857387ms + 35100137.315047, // 35.100137ms + 39926245.827925, // 39.926245ms + 45415922.211464, // 45.415922ms + 51660404.016126, // 51.660404ms + 58763473.538708, // 58.763473ms + 66843182.667648, // 66.843182ms + 76033814.886682, // 76.033814ms + 86488117.045035, // 86.488117ms + 98379837.985822, // 98.379837ms + 111906616.224248, // 111.906616ms + 127293264.668375, // 127.293264ms + 144795506.973983, // 144.795506ms + 164704227.631154, // 164.704227ms + 187350306.418342, // 187.350306ms + 213110117.571795, // 213.110117ms + 242411785.065635, // 242.411785ms + 275742297.964389, // 275.742297ms + 313655604.103963, // 313.655604ms + 356781816.616787, // 356.781816ms + 405837686.312094, // 405.837686ms + 461638513.960647, // 461.638513ms + 525111700.464186, // 525.1117ms + 597312160.111267, // 597.31216ms + 679439853.085354, // 679.439853ms + 772859728.612681, // 772.859728ms + 879124410.201811, // 879.12441ms + 1000000000.000001, // 1s } // BatchProcessLatencyBuckets are prometheus histogram buckets suitable for a @@ -62,20 +152,65 @@ var NetworkLatencyBuckets = []float64{ var BatchProcessLatencyBuckets = []float64{ // Generated via TestHistogramBuckets/BatchProcessLatencyBuckets. 500000000.000000, // 500ms - 789604072.059876, // 789.604072ms - 1246949181.227077, // 1.246949181s - 1969192302.297256, // 1.969192302s - 3109764521.125753, // 3.109764521s - 4910965458.056452, // 4.910965458s - 7755436646.853539, // 7.755436646s - 12247448713.915894, // 12.247448713s - 19341270753.704967, // 19.341270753s - 30543892291.876068, // 30.543892291s - 48235163460.447227, // 48.23516346s - 76173362969.685760, // 1m16.173362969s - 120293595166.717728, // 2m0.293595166s - 189968625172.725128, // 3m9.968625172s - 300000000000.000183, // 5m0s + 557259285.358743, // 557.259285ms + 621075822.237074, // 621.075822ms + 692200537.706851, // 692.200537ms + 771470353.934916, // 771.470353ms + 859818036.218456, // 859.818036ms + 958283168.803309, // 958.283168ms + 1068024387.637287, // 1.068024387s + 1190333014.000928, // 1.190333014s + 1326648249.442152, // 1.326648249s + 1478574110.813123, // 1.47857411s + 1647898304.683320, // 1.647898304s + 1836613263.223422, // 1.836613263s + 2046939589.088547, // 2.046939589s + 2281352185.176006, // 2.281352185s + 2542609376.725576, // 2.542609376s + 2833785368.441068, // 2.833785368s + 3158306418.555065, // 3.158306418s + 3519991155.495853, // 3.519991155s + 3923095511.561431, // 3.923095511s + 4372362802.333632, // 4.372362802s + 4873079541.115184, // 4.873079541s + 5431137645.156319, // 5.431137645s + 6053103765.649553, // 6.053103765s + 6746296557.296375, // 6.746296557s + 7518872796.674253, // 7.518872796s + 8379923362.755980, // 8.379923362s + 9339580208.980864, // 9.339580208s + 10409135585.614676, // 10.409135585s + 11601174915.283792, // 11.601174915s + 12929724885.225649, // 12.929724885s + 14410418498.852003, // 14.410418498s + 16060679028.781363, // 16.060679028s + 17899925035.909710, // 17.899925035s + 19949798866.972237, // 19.949798866s + 22234421319.319225, // 22.234421319s + 24780675469.538071, // 24.780675469s + 27618523005.723442, // 27.618523005s + 30781356785.666904, // 30.781356785s + 34306393769.506477, // 34.306393769s + 38235112950.461639, // 38.23511295s + 42613743436.770157, // 42.613743436s + 47493808428.070732, // 47.493808428s + 52932731487.183495, // 52.932731487s + 58994512241.268242, // 58.994512241s + 65750479463.313522, // 1m5.750479463s + 73280130395.441635, // 1m13.280130395s + 81672066190.318619, // 1m21.67206619s + 91025034477.977859, // 1m31.025034477s + 101449091325.905777, // 1m41.449091325s + 113066896265.136261, // 1m53.066896265s + 126015155620.881943, // 2m6.01515562s + 140446231131.326965, // 2m20.446231131s + 156529932783.144257, // 2m36.529932783s + 174455516959.974152, // 2m54.455516959s + 194433913416.010529, // 3m14.433913416s + 216700207279.419586, // 3m36.700207279s + 241516405291.241699, // 4m1.516405291s + 269174518830.019897, // 4m29.17451883s + 300000000000.000854, // 5m0s } // LongRunning60mLatencyBuckets are prometheus histogram buckets suitable @@ -84,20 +219,65 @@ var BatchProcessLatencyBuckets = []float64{ var LongRunning60mLatencyBuckets = []float64{ // Generated via TestHistogramBuckets/LongRunning60mLatencyBuckets. 500000000.000000, // 500ms - 942961049.923126, // 942.961049ms - 1778351083.344248, // 1.778351083s - 3353831609.364442, // 3.353831609s - 6325065151.263324, // 6.325065151s - 11928580151.734879, // 11.928580151s - 22496372927.944168, // 22.496372927s - 42426406871.192848, // 42.426406871s - 80012898335.451462, // 1m20.012898335s - 150898093243.579315, // 2m30.898093243s - 284582048872.726685, // 4m44.582048872s - 536699575188.601318, // 8m56.699575188s - 1012173589826.278687, // 16m52.173589826s - 1908880541934.094238, // 31m48.880541934s - 3599999999999.998535, // 59m59.999999999s + 581230667.894489, // 581.230667ms + 675658178.602148, // 675.658178ms + 785426508.834601, // 785.426508ms + 913027948.623944, // 913.027948ms + 1061359688.770060, // 1.061359688s + 1233789601.560218, // 1.233789601s + 1434232708.312242, // 1.434232708s + 1667240069.936893, // 1.667240069s + 1938102118.779750, // 1.938102118s + 2252968777.892157, // 2.252968777s + 2618989095.039379, // 2.618989095s + 3044473561.836243, // 3.044473561s + 3539082803.466387, // 3.539082803s + 4114046923.185338, // 4.114046923s + 4782420481.824564, // 4.782420481s + 5559378901.606352, // 5.559378901s + 6462563024.118382, // 6.462563024s + 7512479645.637113, // 7.512479645s + 8732967123.954826, // 8.732967123s + 10151736628.313759, // 10.151736628s + 11801001321.527510, // 11.801001321s + 13718207759.870365, // 13.718207759s + 15946886117.169632, // 15.946886117s + 18537638537.439724, // 18.537638537s + 21549288056.605419, // 21.549288056s + 25050214179.583008, // 25.050214179s + 29119905436.998066, // 29.119905436s + 33850764172.341507, // 33.850764172s + 39350204537.257782, // 39.350204537s + 45743091329.950188, // 45.743091329s + 53174575050.531136, // 53.17457505s + 61813387543.251701, // 1m1.813387543s + 71855673053.170151, // 1m11.855673053s + 83529441681.404266, // 1m23.529441681s + 97099746354.672745, // 1m37.099746354s + 112874700852.223846, // 1m52.874700852s + 131212475529.457443, // 2m11.212475529s + 152529429576.151703, // 2m32.529429576s + 177309564452.224213, // 2m57.309564452s + 206115513141.294464, // 3m26.115513141s + 239601314733.059875, // 3m59.601314733s + 278527264381.388123, // 4m38.527264381s + 323777175806.438293, // 5m23.777175806s + 376378448285.935181, // 6m16.378448285s + 437525393756.650940, // 7m17.525393756s + 508606353667.955078, // 8m28.606353667s + 591235221275.612671, // 9m51.235221275s + 687288085089.540771, // 11m27.288085089s + 798945825465.036499, // 13m18.945825465s + 928743631493.114136, // 15m28.743631493s + 1079628562470.991943, // 17m59.62856247s + 1255026460885.963623, // 20m55.026460885s + 1458919736172.010742, // 24m18.919736172s + 1695937785319.419434, // 28m15.937785319s + 1971462103337.413574, // 32m51.462103337s + 2291748470102.958496, // 38m11.748470102s + 2664068987848.231934, // 44m24.068987848s + 3096877194248.046875, // 51m36.877194248s + 3600000000000.007812, // 1h0m0s } // Count1KBuckets are prometheus histogram buckets suitable for a histogram that diff --git a/pkg/util/metric/histogram_buckets_test.go b/pkg/util/metric/histogram_buckets_test.go index 7fb183d70bf6..6f28454b89ff 100644 --- a/pkg/util/metric/histogram_buckets_test.go +++ b/pkg/util/metric/histogram_buckets_test.go @@ -48,22 +48,22 @@ func TestHistogramBuckets(t *testing.T) { require.InDeltaSlice(t, exp, act, 1 /* delta */, "Please update the bucket boundaries for %s", t.Name()) } t.Run("IOLatencyBuckets", func(t *testing.T) { - exp := prometheus.ExponentialBucketsRange(10e3, 10e9, 15) + exp := prometheus.ExponentialBucketsRange(10e3, 10e9, 60) verifyAndPrint(t, exp, IOLatencyBuckets, LATENCY) }) t.Run("NetworkLatencyBuckets", func(t *testing.T) { - exp := prometheus.ExponentialBucketsRange(500e3, 1e9, 15) + exp := prometheus.ExponentialBucketsRange(500e3, 1e9, 60) verifyAndPrint(t, exp, NetworkLatencyBuckets, LATENCY) }) t.Run("BatchProcessLatencyBuckets", func(t *testing.T) { - exp := prometheus.ExponentialBucketsRange(500e6, 300e9, 15) + exp := prometheus.ExponentialBucketsRange(500e6, 300e9, 60) verifyAndPrint(t, exp, BatchProcessLatencyBuckets, LATENCY) }) t.Run("LongRunning60mLatencyBuckets", func(t *testing.T) { - exp := prometheus.ExponentialBucketsRange(500e6, 3600e9, 15) + exp := prometheus.ExponentialBucketsRange(500e6, 3600e9, 60) verifyAndPrint(t, exp, LongRunning60mLatencyBuckets, LATENCY) }) From 9e2332c39d69d9ce05e61e98a2c073833d38fc4c Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Thu, 2 Feb 2023 12:22:10 -0500 Subject: [PATCH 8/8] backupccl: add 'aws-weekly' tag to restore/tpce/32tb test Epic: none Release note: None --- pkg/cmd/roachtest/tests/restore.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/restore.go b/pkg/cmd/roachtest/tests/restore.go index d850a6a1c66e..1310b53d4bc6 100644 --- a/pkg/cmd/roachtest/tests/restore.go +++ b/pkg/cmd/roachtest/tests/restore.go @@ -696,7 +696,7 @@ func registerRestore(r registry.Registry) { aost: "'2023-01-12 03:00:00'", workload: tpceRestore{customers: 2000000}}), timeout: 24 * time.Hour, - tags: []string{"weekly"}, + tags: []string{"weekly", "aws-weekly"}, }, // TODO(msbutler): add the following tests once roachperf/grafana is hooked up and old tests are // removed: