From 8d1df5538e93b96032fb20e090a07c3b2a3cc4a3 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 7 Aug 2022 13:09:52 -0400 Subject: [PATCH 1/5] sql: UDF with empty result should evaluate to NULL If the last statement in a UDF returns no rows, the UDF will evaluate to NULL. Prior to this commit the evaluation of the UDF would panic. Release note: None --- pkg/sql/logictest/testdata/logic_test/udf | 175 ++++++++++++---------- pkg/sql/routine.go | 4 + 2 files changed, 98 insertions(+), 81 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 9f12d72de568..5c0a9ab97722 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -310,73 +310,6 @@ ALTER FUNCTION f() SET SCHEMA test_sc statement error pq: unimplemented: alter function depends on extension not supported.* ALTER FUNCTION f() DEPENDS ON EXTENSION postgis -subtest execution - -statement ok -INSERT INTO ab VALUES (1, 1), (2, 2), (3, 3), (4, 1), (5, 1) - -statement ok -CREATE FUNCTION one() RETURNS INT LANGUAGE SQL AS 'SELECT 2-1'; - -query I -SELECT one() ----- -1 - -query I colnames -SELECT * FROM one() ----- -one -1 - -query III colnames -SELECT *, one() FROM ab WHERE a = one() ----- -a b one -1 1 1 - -query III colnames -SELECT *, one() FROM ab WHERE b = one() ----- -a b one -1 1 1 -4 1 1 -5 1 1 - -query II colnames -SELECT * FROM ab WHERE b = one() + 1 ----- -a b -2 2 - -statement ok -CREATE FUNCTION max_in_values() RETURNS INT LANGUAGE SQL AS $$ - SELECT i FROM (VALUES (1, 0), (2, 0), (3, 0)) AS v(i, j) ORDER BY i DESC -$$ - -query I -SELECT max_in_values() ----- -3 - -statement ok -CREATE FUNCTION fetch_one_then_two() RETURNS INT LANGUAGE SQL AS $$ - SELECT b FROM ab WHERE a = 1; - SELECT b FROM ab WHERE a = 2; -$$ - -query II -SELECT i, fetch_one_then_two() -FROM (VALUES (1), (2), (3)) AS v(i) -WHERE i = fetch_one_then_two() ----- -2 2 - -query I colnames -SELECT * FROM fetch_one_then_two() ----- -fetch_one_then_two -2 subtest udf_pg_proc @@ -399,9 +332,9 @@ query TTTTTBBBTITTTTT SELECT oid, proname, pronamespace, proowner, prolang, proleakproof, proisstrict, proretset, provolatile, pronargs, prorettype, proargtypes, proargmodes, proargnames, prosrc FROM pg_catalog.pg_proc WHERE proname IN ('proc_f', 'proc_f_2'); ---- -100124 proc_f 4101115737 1546506610 14 false false false v 1 20 20 {i} NULL SELECT 1; -100125 proc_f 4101115737 1546506610 14 true true true i 2 25 25 20 {i,i} {"",b} SELECT 'hello'; -100128 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; +100121 proc_f 4101115737 1546506610 14 false false false v 1 20 20 {i} NULL SELECT 1; +100122 proc_f 4101115737 1546506610 14 true true true i 2 25 25 20 {i,i} {"",b} SELECT 'hello'; +100125 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; subtest create_function_statements @@ -419,7 +352,7 @@ CREATE FUNCTION public.proc_f(IN INT8) LANGUAGE SQL AS $$ SELECT 1; -$$ 104 test 105 public 124 proc_f +$$ 104 test 105 public 121 proc_f CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) RETURNS SETOF STRING IMMUTABLE @@ -428,7 +361,7 @@ CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) LANGUAGE SQL AS $$ SELECT 'hello'; -$$ 104 test 105 public 125 proc_f +$$ 104 test 105 public 122 proc_f CREATE FUNCTION sc.proc_f_2(IN STRING) RETURNS STRING VOLATILE @@ -437,7 +370,7 @@ CREATE FUNCTION sc.proc_f_2(IN STRING) LANGUAGE SQL AS $$ SELECT 'hello'; -$$ 104 test 127 sc 128 proc_f_2 +$$ 104 test 124 sc 125 proc_f_2 statement ok CREATE DATABASE test_cross_db; @@ -459,7 +392,7 @@ CREATE FUNCTION public.proc_f(IN INT8) LANGUAGE SQL AS $$ SELECT 1; -$$ 104 test 105 public 124 proc_f +$$ 104 test 105 public 121 proc_f CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) RETURNS SETOF STRING IMMUTABLE @@ -468,7 +401,7 @@ CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) LANGUAGE SQL AS $$ SELECT 'hello'; -$$ 104 test 105 public 125 proc_f +$$ 104 test 105 public 122 proc_f CREATE FUNCTION sc.proc_f_2(IN STRING) RETURNS STRING VOLATILE @@ -477,7 +410,7 @@ CREATE FUNCTION sc.proc_f_2(IN STRING) LANGUAGE SQL AS $$ SELECT 'hello'; -$$ 104 test 127 sc 128 proc_f_2 +$$ 104 test 124 sc 125 proc_f_2 CREATE FUNCTION public.f_cross_db() RETURNS INT8 VOLATILE @@ -486,7 +419,7 @@ CREATE FUNCTION public.f_cross_db() LANGUAGE SQL AS $$ SELECT 1; -$$ 129 test_cross_db 130 public 131 f_cross_db +$$ 126 test_cross_db 127 public 128 f_cross_db subtest show_create_function @@ -557,12 +490,12 @@ proc_implicit query I SELECT 'proc_implicit'::REGPROC::INT; ---- -100126 +100123 query T SELECT '100126'::REGPROC; ---- -proc_implicit +100126 query T SELECT 'sc.proc_f_2'::REGPROC; @@ -572,7 +505,7 @@ proc_f_2 query I SELECT 'sc.proc_f_2'::REGPROC::INT; ---- -100128 +100125 statement error pq: unknown function: no_such_func() SELECT 'no_such_func'::REGPROC; @@ -583,7 +516,7 @@ SELECT 'proc_f'::REGPROC; query T SELECT 100126::regproc; ---- -proc_implicit +100126 query I SELECT 100117::regproc::INT; @@ -845,3 +778,83 @@ CREATE FUNCTION public.test_vf_f() AS $$ SELECT lower('hello'); $$ + + +subtest execution + +statement ok +INSERT INTO ab VALUES (1, 1), (2, 2), (3, 3), (4, 1), (5, 1) + +statement ok +CREATE FUNCTION one() RETURNS INT LANGUAGE SQL AS 'SELECT 2-1'; + +query I +SELECT one() +---- +1 + +query I colnames +SELECT * FROM one() +---- +one +1 + +query III colnames +SELECT *, one() FROM ab WHERE a = one() +---- +a b one +1 1 1 + +query III colnames +SELECT *, one() FROM ab WHERE b = one() +---- +a b one +1 1 1 +4 1 1 +5 1 1 + +query II colnames +SELECT * FROM ab WHERE b = one() + 1 +---- +a b +2 2 + +statement ok +CREATE FUNCTION max_in_values() RETURNS INT LANGUAGE SQL AS $$ + SELECT i FROM (VALUES (1, 0), (2, 0), (3, 0)) AS v(i, j) ORDER BY i DESC +$$ + +query I +SELECT max_in_values() +---- +3 + +statement ok +CREATE FUNCTION fetch_one_then_two() RETURNS INT LANGUAGE SQL AS $$ + SELECT b FROM ab WHERE a = 1; + SELECT b FROM ab WHERE a = 2; +$$ + +query II +SELECT i, fetch_one_then_two() +FROM (VALUES (1), (2), (3)) AS v(i) +WHERE i = fetch_one_then_two() +---- +2 2 + +query I colnames +SELECT * FROM fetch_one_then_two() +---- +fetch_one_then_two +2 + +statement ok +CREATE TABLE empty (e INT); +CREATE FUNCTION empty_result() RETURNS INT LANGUAGE SQL AS $$ + SELECT e FROM empty +$$ + +query I +SELECT empty_result() +---- +NULL diff --git a/pkg/sql/routine.go b/pkg/sql/routine.go index 918fe8c7b032..3d8ca5a93cd0 100644 --- a/pkg/sql/routine.go +++ b/pkg/sql/routine.go @@ -75,6 +75,10 @@ func (p *planner) EvalRoutineExpr( if err != nil { return nil, err } + if res == nil { + // Return NULL if there are no results. + return tree.DNull, nil + } return res[0], nil } From 5ce9d6e6aa9ad3c64e1d09129543772c9f5e4e89 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 7 Aug 2022 13:24:52 -0400 Subject: [PATCH 2/5] sql: support UDFs with named arguments UDFs with named arguments can now be evaluated. During query planning, statements in the function body are built with a scope that includes the named arguments for the function as columns. This allows references to arguments to be resolved as variables. During evaluation, the input expressions are first evaluated into datums. When a plan is built for each statement in the UDF, the argument columns in the expression are replaced with the input datums before the expression is optimized. Note that anonymous arguments and integer references to arguments (e.g., `$1`) are not yet supported. Also, the formatting of `UDFExpr`s has been improved to show argument columns and input expressions. Release note: None --- pkg/sql/faketreeeval/evalctx.go | 2 +- pkg/sql/logictest/testdata/logic_test/udf | 88 +++++ pkg/sql/opt/exec/execbuilder/scalar.go | 43 ++- pkg/sql/opt/memo/expr_format.go | 63 ++-- pkg/sql/opt/memo/testdata/logprops/udf | 115 +++---- pkg/sql/opt/norm/testdata/rules/udf | 15 +- pkg/sql/opt/ops/scalar.opt | 17 + pkg/sql/opt/optbuilder/scalar.go | 69 +++- pkg/sql/opt/optbuilder/testdata/udf | 384 +++++++++++++++++++--- pkg/sql/routine.go | 12 +- pkg/sql/sem/eval/deps.go | 7 +- pkg/sql/sem/eval/expr.go | 16 +- pkg/sql/sem/tree/routine.go | 28 +- 13 files changed, 699 insertions(+), 160 deletions(-) diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index c22c4a6ef51f..59c60619af40 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -391,7 +391,7 @@ func (ep *DummyEvalPlanner) EvalSubquery(expr *tree.Subquery) (tree.Datum, error // EvalRoutineExpr is part of the eval.Planner interface. func (ep *DummyEvalPlanner) EvalRoutineExpr( - ctx context.Context, expr *tree.RoutineExpr, + ctx context.Context, expr *tree.RoutineExpr, input tree.Datums, ) (tree.Datum, error) { return nil, errors.WithStack(errEvalPlanner) } diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 5c0a9ab97722..850ae5f74f39 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -858,3 +858,91 @@ query I SELECT empty_result() ---- NULL + +statement ok +CREATE FUNCTION int_identity(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT i'; + +query I +SELECT int_identity(1) +---- +1 + +query I +SELECT int_identity(10 + int_identity(1)) +---- +11 + +query II +SELECT a+b, int_identity(a+b) FROM ab WHERE a = int_identity(a) AND b = int_identity(b) +---- +2 2 +4 4 +6 6 +5 5 +6 6 + +# Define some custom arithmetic functions that we can write interesting tests +# with that use builtin operators as oracles. +statement ok +CREATE FUNCTION add(x INT, y INT) RETURNS INT LANGUAGE SQL AS 'SELECT x+y'; + +statement ok +CREATE FUNCTION sub(x INT, y INT) RETURNS INT LANGUAGE SQL AS 'SELECT x-y'; + +statement ok +CREATE FUNCTION mult(x INT, y INT) RETURNS INT LANGUAGE SQL AS 'SELECT x*y'; + +query II +SELECT a + a + a + b + b + b, add(a, add(a, add(a, add(b, add(b, b))))) FROM ab +---- +6 6 +12 12 +18 18 +15 15 +18 18 + +query II +SELECT (a * (a + b)) - b, sub(mult(a, add(a, b)), b) FROM ab +---- +1 1 +6 6 +15 15 +19 19 +29 29 + +query II +SELECT a * (3 + b - a) + a * b * a, add(mult(a, add(3, sub(b, a))), mult(a, mult(b, a))) FROM ab +---- +4 4 +14 14 +36 36 +16 16 +20 20 + +statement ok +CREATE FUNCTION fetch_b(arg_a INT) RETURNS INT LANGUAGE SQL AS $$ + SELECT b FROM ab WHERE a = arg_a +$$ + +query II +SELECT b, fetch_b(a) FROM ab +---- +1 1 +2 2 +3 3 +1 1 +1 1 + +query II +SELECT b + (a * 7) - (a * b), add(fetch_b(a), sub(mult(a, 7), mult(a, fetch_b(a)))) FROM ab +---- +7 7 +12 12 +15 15 +25 25 +31 31 + +query I +SELECT fetch_b(99999999) +---- +NULL diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index 3b2da7058f27..17825b96fedf 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -654,6 +654,33 @@ func (b *Builder) addSubquery( // buildUDF builds a UDF expression into a typed expression that can be // evaluated. func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.TypedExpr, error) { + udf := scalar.(*memo.UDFExpr) + + // Build the input expressions. + var err error + var inputExprs tree.TypedExprs + if len(udf.Input) > 0 { + inputExprs = make(tree.TypedExprs, len(udf.Input)) + for i := range udf.Input { + inputExprs[i], err = b.buildScalar(ctx, udf.Input[i]) + if err != nil { + return nil, err + } + } + } + + // argOrd returns the ordinal of the arguments that the given column ID + // represents. If the column does not represent an argument, then ok=false + // is returned. + argOrd := func(col opt.ColumnID) (ord int, ok bool) { + for i, argCol := range udf.ArgCols { + if col == argCol { + return i, true + } + } + return 0, false + } + // Create a tree.RoutinePlanFn that can plan the statements in the UDF body. // We do this planning in a separate memo. We use an exec.Factory passed to // the closure rather than b.factory to support executing plans that are @@ -663,17 +690,23 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ // exec.Factory to avoid import cycles. // // Note: we put o outside of the function so we allocate it only once. - udf := scalar.(*memo.UDFExpr) var o xform.Optimizer - planFn := func(ctx context.Context, ref tree.RoutineExecFactory, stmtIdx int) (tree.RoutinePlan, error) { + planFn := func( + ctx context.Context, ref tree.RoutineExecFactory, stmtIdx int, input tree.Datums, + ) (tree.RoutinePlan, error) { o.Init(ctx, b.evalCtx, b.catalog) f := o.Factory() stmt := udf.Body[stmtIdx] - // Copy the expression into a new memo. - // TODO(mgartner): Replace argument references with constant values. + // Copy the expression into a new memo. Replace argument references with + // input datums. var replaceFn norm.ReplaceFunc replaceFn = func(e opt.Expr) opt.Expr { + if v, ok := e.(*memo.VariableExpr); ok { + if ord, ok := argOrd(v.Col); ok { + return f.ConstructConstVal(input[ord], v.Typ) + } + } return f.CopyAndReplaceDefault(e, replaceFn) } f.CopyAndReplace(stmt, stmt.PhysProps, replaceFn) @@ -704,5 +737,5 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ } return plan, nil } - return tree.NewTypedRoutineExpr(udf.Name, planFn, len(udf.Body), udf.Typ), nil + return tree.NewTypedRoutineExpr(udf.Name, inputExprs, planFn, len(udf.Body), udf.Typ), nil } diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index 7aa5a9f93038..4af877d98398 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -352,7 +352,7 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { *UpsertDistinctOnExpr, *EnsureUpsertDistinctOnExpr: private := e.Private().(*GroupingPrivate) if !f.HasFlags(ExprFmtHideColumns) && !private.GroupingCols.Empty() { - f.formatColList(e, tp, "grouping columns:", private.GroupingCols.ToList()) + f.formatRelColList(e, tp, "grouping columns:", private.GroupingCols.ToList()) } if !f.HasFlags(ExprFmtHidePhysProps) && !private.Ordering.Any() { tp.Childf("internal-ordering: %s", private.Ordering) @@ -388,8 +388,8 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { *UnionAllExpr, *IntersectAllExpr, *ExceptAllExpr, *LocalityOptimizedSearchExpr: private := e.Private().(*SetPrivate) if !f.HasFlags(ExprFmtHideColumns) { - f.formatColList(e, tp, "left columns:", private.LeftCols) - f.formatColList(e, tp, "right columns:", private.RightCols) + f.formatRelColList(e, tp, "left columns:", private.LeftCols) + f.formatRelColList(e, tp, "right columns:", private.RightCols) } if !f.HasFlags(ExprFmtHidePhysProps) && !private.Ordering.Any() { tp.Childf("internal-ordering: %s", private.Ordering) @@ -550,7 +550,7 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { tp.Childf("lookup columns are key") } if t.IsFirstJoinInPairedJoiner { - f.formatColList(e, tp, "first join in paired joiner; continuation column:", opt.ColList{t.ContinuationCol}) + f.formatRelColList(e, tp, "first join in paired joiner; continuation column:", opt.ColList{t.ContinuationCol}) } if t.IsSecondJoinInPairedJoiner { tp.Childf("second join in paired joiner") @@ -570,7 +570,7 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { tp.Childf("prefix key columns: %v = %v", t.PrefixKeyCols, idxCols) } if t.IsFirstJoinInPairedJoiner { - f.formatColList(e, tp, "first join in paired joiner; continuation column:", opt.ColList{t.ContinuationCol}) + f.formatRelColList(e, tp, "first join in paired joiner; continuation column:", opt.ColList{t.ContinuationCol}) } n := tp.Child("inverted-expr") f.formatExpr(t.InvertedExpr, n) @@ -638,7 +638,7 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { tp.Child("columns: ") } if t.CanaryCol != 0 { - f.formatColList(e, tp, "canary column:", opt.ColList{t.CanaryCol}) + f.formatRelColList(e, tp, "canary column:", opt.ColList{t.CanaryCol}) f.formatOptionalColList(e, tp, "fetch columns:", t.FetchCols) f.formatMutationCols(e, tp, "insert-mapping:", t.InsertCols, t.Table) f.formatMutationCols(e, tp, "update-mapping:", t.UpdateCols, t.Table) @@ -731,8 +731,8 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { tp.Childf("deduplicate") } tp.Childf("working table binding: &%d", t.WithID) - f.formatColList(e, tp, "initial columns:", t.InitialCols) - f.formatColList(e, tp, "recursive columns:", t.RecursiveCols) + f.formatRelColList(e, tp, "initial columns:", t.InitialCols) + f.formatRelColList(e, tp, "recursive columns:", t.RecursiveCols) } default: @@ -897,9 +897,18 @@ func (f *ExprFmtCtx) formatScalar(scalar opt.ScalarExpr, tp treeprinter.Node) { func (f *ExprFmtCtx) formatScalarWithLabel( label string, scalar opt.ScalarExpr, tp treeprinter.Node, ) { - formatUDFBody := func(udf *UDFExpr, tp treeprinter.Node) { + formatUDFInputAndBody := func(udf *UDFExpr, tp treeprinter.Node) { + var n treeprinter.Node + if len(udf.ArgCols) > 0 { + f.formatColList(tp, "args:", udf.ArgCols, opt.ColSet{} /* notNullCols */) + n = tp.Child("input") + for i := range udf.Input { + f.formatExpr(udf.Input[i], n) + } + } + n = tp.Child("body") for i := range udf.Body { - f.formatExpr(udf.Body[i], tp) + f.formatExpr(udf.Body[i], n) } } @@ -962,7 +971,7 @@ func (f *ExprFmtCtx) formatScalarWithLabel( fmt.Fprintf(f.Buffer, "udf: %s", udf.Name) f.FormatScalarProps(scalar) tp = tp.Child(f.Buffer.String()) - formatUDFBody(udf, tp) + formatUDFInputAndBody(udf, tp) return } @@ -1014,19 +1023,19 @@ func (f *ExprFmtCtx) formatScalarWithLabel( } var intercepted bool - if f.HasFlags(ExprFmtHideScalars) && ScalarFmtInterceptor != nil { - if str := ScalarFmtInterceptor(f, scalar); str != "" { - f.Buffer.WriteString(str) - intercepted = true - } - } - if udf, ok := scalar.(*UDFExpr); ok { + if udf, ok := scalar.(*UDFExpr); ok && !f.HasFlags(ExprFmtHideScalars) { // A UDF function body will be printed after the scalar props, so // pre-emptively set intercepted=true to avoid the default // formatScalarPrivate formatting below. fmt.Fprintf(f.Buffer, "udf: %s", udf.Name) intercepted = true } + if !intercepted && f.HasFlags(ExprFmtHideScalars) && ScalarFmtInterceptor != nil { + if str := ScalarFmtInterceptor(f, scalar); str != "" { + f.Buffer.WriteString(str) + intercepted = true + } + } if !intercepted { fmt.Fprintf(f.Buffer, "%v", scalar.Op()) f.formatScalarPrivate(scalar) @@ -1038,9 +1047,8 @@ func (f *ExprFmtCtx) formatScalarWithLabel( } tp = tp.Child(f.Buffer.String()) - if udf, ok := scalar.(*UDFExpr); ok { - // Always print UDF function body. - formatUDFBody(udf, tp) + if udf, ok := scalar.(*UDFExpr); ok && !f.HasFlags(ExprFmtHideScalars) { + formatUDFInputAndBody(udf, tp) } if !intercepted { @@ -1282,7 +1290,7 @@ func (f *ExprFmtCtx) formatColumns( return } if presentation.Any() { - f.formatColList(nd, tp, "columns:", cols) + f.formatRelColList(nd, tp, "columns:", cols) return } @@ -1311,13 +1319,20 @@ func (f *ExprFmtCtx) formatColumns( tp.Child(f.Buffer.String()) } +// formatRelColList constructs a new treeprinter child containing the specified +// list of columns formatted using the formatCol method. +func (f *ExprFmtCtx) formatRelColList( + nd RelExpr, tp treeprinter.Node, heading string, colList opt.ColList, +) { + f.formatColList(tp, heading, colList, nd.Relational().NotNullCols) +} + // formatColList constructs a new treeprinter child containing the specified // list of columns formatted using the formatCol method. func (f *ExprFmtCtx) formatColList( - nd RelExpr, tp treeprinter.Node, heading string, colList opt.ColList, + tp treeprinter.Node, heading string, colList opt.ColList, notNullCols opt.ColSet, ) { if len(colList) > 0 { - notNullCols := nd.Relational().NotNullCols f.Buffer.Reset() f.Buffer.WriteString(heading) for _, col := range colList { diff --git a/pkg/sql/opt/memo/testdata/logprops/udf b/pkg/sql/opt/memo/testdata/logprops/udf index f267e2c803f8..c860a73c2a48 100644 --- a/pkg/sql/opt/memo/testdata/logprops/udf +++ b/pkg/sql/opt/memo/testdata/logprops/udf @@ -38,17 +38,18 @@ project └── plus [as="?column?":6, type=int, outer=(1), volatile] ├── variable: a:1 [type=int] └── udf: fn_volatile [type=int] - └── project - ├── columns: "?column?":5(int!null) - ├── cardinality: [1 - 1] - ├── key: () - ├── fd: ()-->(5) - ├── values - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ └── tuple [type=tuple] - └── projections - └── const: 1 [as="?column?":5, type=int] + └── body + └── project + ├── columns: "?column?":5(int!null) + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(5) + ├── values + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ └── tuple [type=tuple] + └── projections + └── const: 1 [as="?column?":5, type=int] build SELECT a FROM ab WHERE b = fn_immutable() @@ -76,17 +77,18 @@ project └── eq [type=bool, outer=(2), immutable, constraints=(/2: (/NULL - ]), fd=()-->(2)] ├── variable: b:2 [type=int] └── udf: fn_immutable [type=int] - └── project - ├── columns: "?column?":5(int!null) - ├── cardinality: [1 - 1] - ├── key: () - ├── fd: ()-->(5) - ├── values - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ └── tuple [type=tuple] - └── projections - └── const: 1 [as="?column?":5, type=int] + └── body + └── project + ├── columns: "?column?":5(int!null) + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(5) + ├── values + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ └── tuple [type=tuple] + └── projections + └── const: 1 [as="?column?":5, type=int] build SELECT a FROM ab WHERE b = fn_immutable() + fn_stable() @@ -115,29 +117,31 @@ project ├── variable: b:2 [type=int] └── plus [type=int] ├── udf: fn_immutable [type=int] - │ └── project - │ ├── columns: "?column?":5(int!null) - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(5) - │ ├── values - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - │ │ └── tuple [type=tuple] - │ └── projections - │ └── const: 1 [as="?column?":5, type=int] + │ └── body + │ └── project + │ ├── columns: "?column?":5(int!null) + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(5) + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── tuple [type=tuple] + │ └── projections + │ └── const: 1 [as="?column?":5, type=int] └── udf: fn_stable [type=int] - └── project - ├── columns: "?column?":6(int!null) - ├── cardinality: [1 - 1] - ├── key: () - ├── fd: ()-->(6) - ├── values - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ └── tuple [type=tuple] - └── projections - └── const: 1 [as="?column?":6, type=int] + └── body + └── project + ├── columns: "?column?":6(int!null) + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(6) + ├── values + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ └── tuple [type=tuple] + └── projections + └── const: 1 [as="?column?":6, type=int] build SELECT a FROM ab WHERE b = fn_leakproof() @@ -163,14 +167,15 @@ project └── eq [type=bool, outer=(2), constraints=(/2: (/NULL - ]), fd=()-->(2)] ├── variable: b:2 [type=int] └── udf: fn_leakproof [type=int] - └── project - ├── columns: "?column?":5(int!null) - ├── cardinality: [1 - 1] - ├── key: () - ├── fd: ()-->(5) - ├── values - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ └── tuple [type=tuple] - └── projections - └── const: 1 [as="?column?":5, type=int] + └── body + └── project + ├── columns: "?column?":5(int!null) + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(5) + ├── values + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ └── tuple [type=tuple] + └── projections + └── const: 1 [as="?column?":5, type=int] diff --git a/pkg/sql/opt/norm/testdata/rules/udf b/pkg/sql/opt/norm/testdata/rules/udf index 45d37429eb3e..5f9cb9c8716c 100644 --- a/pkg/sql/opt/norm/testdata/rules/udf +++ b/pkg/sql/opt/norm/testdata/rules/udf @@ -14,10 +14,11 @@ values ├── fd: ()-->(2) └── tuple └── udf: one - └── values - ├── columns: "?column?":1!null - ├── cardinality: [1 - 1] - ├── key: () - ├── fd: ()-->(1) - └── tuple - └── const: 1 + └── body + └── values + ├── columns: "?column?":1!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(1) + └── tuple + └── const: 1 diff --git a/pkg/sql/opt/ops/scalar.opt b/pkg/sql/opt/ops/scalar.opt index e1b5589fbae5..ebd23dd83312 100644 --- a/pkg/sql/opt/ops/scalar.opt +++ b/pkg/sql/opt/ops/scalar.opt @@ -1219,14 +1219,31 @@ define NthValue { # function body, and a pointer to its type. [Scalar] define UDF { + # Input contains the scalar expressions given as input to the UDF. + Input ScalarListExpr _ UDFPrivate } [Private] define UDFPrivate { + # Name is the name of the function. Name string + + # Body contains a relational expression for each statement in the function + # body. Body RelListExpr + + # ArgCols is a list of columns that are references to arguments of the + # function. The i-th column in the list corresponds to the i-th argument of + # the function. During execution of the UDF, these columns are replaced with + # the constant value inputs to the function. + ArgCols ColList + + # Typ is the return type of the function. Typ Type + + # Volatility is the user-provided volatility of the function given during + # CREATE FUNCTION. Volatility Volatility } diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 71d603aa0c72..ac9625da7c72 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -536,7 +536,7 @@ func (b *Builder) buildFunction( } if f.ResolvedOverload().Body != "" { - return b.buildUDF(f, def, inScope, outScope, outCol) + return b.buildUDF(f, def, inScope, outScope, outCol, colRefs) } if f.ResolvedOverload().Class == tree.AggregateClass { @@ -595,39 +595,88 @@ func (b *Builder) buildFunction( // buildUDF builds a set of memo groups that represents a user-defined function // invocation. -// TODO(mgartner): Support UDFs with arguments. func (b *Builder) buildUDF( f *tree.FuncExpr, def *tree.ResolvedFunctionDefinition, inScope, outScope *scope, outCol *scopeColumn, + colRefs *opt.ColSet, ) (out opt.ScalarExpr) { o := f.ResolvedOverload() + + // Build the input expressions. + var input memo.ScalarListExpr + if len(f.Exprs) > 0 { + input = make(memo.ScalarListExpr, len(f.Exprs)) + for i, pexpr := range f.Exprs { + input[i] = b.buildScalar( + pexpr.(tree.TypedExpr), + inScope, + nil, /* outScope */ + nil, /* outCol */ + colRefs, + ) + } + } + + // Create a new scope for building the statements in the function body. We + // start with an empty scope because a statement in the function body cannot + // refer to anything from the outer expression. If there are function + // arguments, we add them as columns to the scope so that references to them + // can be resolved. + // + // TODO(mgartner): Support anonymous arguments and placeholder-like syntax + // for referencing arguments, e.g., $1. + // + // TODO(mgartner): We may need to set bodyScope.atRoot=true to prevent + // CTEs that mutate and are not at the top-level. + // + bodyScope := b.allocScope() + var argCols opt.ColList + if o.Types.Length() > 0 { + args, ok := o.Types.(tree.ArgTypes) + if !ok { + // TODO(mgartner): Create an issue for this and link it here. + panic(unimplemented.New("user-defined functions", + "variadiac user-defined functions are not yet supported")) + } + argCols = make(opt.ColList, len(args)) + for i := range args { + arg := &args[i] + id := b.factory.Metadata().AddColumn(arg.Name, arg.Typ) + argCols[i] = id + bodyScope.appendColumn(&scopeColumn{ + name: scopeColName(tree.Name(arg.Name)), + typ: arg.Typ, + id: id, + }) + } + } + + // Parse the function body. stmts, err := parser.Parse(o.Body) if err != nil { panic(err) } + // Build an expression for each statement in the function body. rels := make(memo.RelListExpr, len(stmts)) for i := range stmts { - // A statement inside a UDF body cannot refer to anything from the outer - // expression calling the function, so we use an empty scope. - // TODO(mgartner): We may need to set bodyScope.atRoot=true to prevent - // CTEs that mutate and are not at the top-level. - bodyScope := b.allocScope() - bodyScope = b.buildStmt(stmts[i].AST, nil /* desiredTypes */, bodyScope) + stmtScope := b.buildStmt(stmts[i].AST, nil /* desiredTypes */, bodyScope) rels[i] = memo.RelRequiredPropsExpr{ - RelExpr: bodyScope.expr, - PhysProps: bodyScope.makePhysicalProps(), + RelExpr: stmtScope.expr, + PhysProps: stmtScope.makePhysicalProps(), } } out = b.factory.ConstructUDF( + input, &memo.UDFPrivate{ Name: def.Name, Body: rels, Typ: f.ResolvedType(), Volatility: o.Volatility, + ArgCols: argCols, }, ) return b.finishBuildScalar(f, out, inScope, outScope, outCol) diff --git a/pkg/sql/opt/optbuilder/testdata/udf b/pkg/sql/opt/optbuilder/testdata/udf index a4f2d341dfc4..538956039278 100644 --- a/pkg/sql/opt/optbuilder/testdata/udf +++ b/pkg/sql/opt/optbuilder/testdata/udf @@ -6,6 +6,10 @@ CREATE TABLE abc ( ) ---- +# -------------------------------------------------- +# UDFs without arguments. +# -------------------------------------------------- + build SELECT foo() ---- @@ -31,12 +35,13 @@ project │ └── tuple └── projections └── udf: one [as=one:2] - └── project - ├── columns: "?column?":1!null - ├── values - │ └── tuple - └── projections - └── const: 1 [as="?column?":1] + └── body + └── project + ├── columns: "?column?":1!null + ├── values + │ └── tuple + └── projections + └── const: 1 [as="?column?":1] build format=show-scalars SELECT *, one() FROM abc @@ -47,12 +52,13 @@ project │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 └── projections └── udf: one [as=one:7] - └── project - ├── columns: "?column?":6!null - ├── values - │ └── tuple - └── projections - └── const: 1 [as="?column?":6] + └── body + └── project + ├── columns: "?column?":6!null + ├── values + │ └── tuple + └── projections + └── const: 1 [as="?column?":6] build format=show-scalars SELECT * FROM abc WHERE one() = c @@ -66,12 +72,13 @@ project └── filters └── eq ├── udf: one - │ └── project - │ ├── columns: "?column?":6!null - │ ├── values - │ │ └── tuple - │ └── projections - │ └── const: 1 [as="?column?":6] + │ └── body + │ └── project + │ ├── columns: "?column?":6!null + │ ├── values + │ │ └── tuple + │ └── projections + │ └── const: 1 [as="?column?":6] └── variable: c:3 build format=show-scalars @@ -87,40 +94,329 @@ project │ └── eq │ ├── variable: c:3 │ └── udf: two - │ ├── project - │ │ ├── columns: "?column?":6!null - │ │ ├── values - │ │ │ └── tuple - │ │ └── projections - │ │ └── const: 1 [as="?column?":6] - │ └── project - │ ├── columns: "?column?":7!null - │ ├── values - │ │ └── tuple - │ └── projections - │ └── const: 2 [as="?column?":7] + │ └── body + │ ├── project + │ │ ├── columns: "?column?":6!null + │ │ ├── values + │ │ │ └── tuple + │ │ └── projections + │ │ └── const: 1 [as="?column?":6] + │ └── project + │ ├── columns: "?column?":7!null + │ ├── values + │ │ └── tuple + │ └── projections + │ └── const: 2 [as="?column?":7] └── projections ├── plus [as="?column?":9] │ ├── variable: a:1 │ └── udf: one - │ └── project - │ ├── columns: "?column?":8!null - │ ├── values - │ │ └── tuple - │ └── projections - │ └── const: 1 [as="?column?":8] + │ └── body + │ └── project + │ ├── columns: "?column?":8!null + │ ├── values + │ │ └── tuple + │ └── projections + │ └── const: 1 [as="?column?":8] └── plus [as="?column?":12] ├── variable: b:2 └── udf: two - ├── project - │ ├── columns: "?column?":10!null - │ ├── values - │ │ └── tuple - │ └── projections - │ └── const: 1 [as="?column?":10] + └── body + ├── project + │ ├── columns: "?column?":10!null + │ ├── values + │ │ └── tuple + │ └── projections + │ └── const: 1 [as="?column?":10] + └── project + ├── columns: "?column?":11!null + ├── values + │ └── tuple + └── projections + └── const: 2 [as="?column?":11] + + +# -------------------------------------------------- +# UDFs with named arguments. +# -------------------------------------------------- + +exec-ddl +CREATE FUNCTION add(x INT, y INT) RETURNS INT LANGUAGE SQL AS $$ + SELECT x+y; +$$; +---- + +build format=show-scalars +SELECT add(1, 2) +---- +project + ├── columns: add:4 + ├── values + │ └── tuple + └── projections + └── udf: add [as=add:4] + ├── args: x:1 y:2 + ├── input + │ ├── const: 1 + │ └── const: 2 + └── body + └── project + ├── columns: "?column?":3 + ├── values + │ └── tuple + └── projections + └── plus [as="?column?":3] + ├── variable: x:1 + └── variable: y:2 + +build format=show-scalars +SELECT add(add(1, 2), 3) +---- +project + ├── columns: add:7 + ├── values + │ └── tuple + └── projections + └── udf: add [as=add:7] + ├── args: x:4 y:5 + ├── input + │ ├── udf: add + │ │ ├── args: x:1 y:2 + │ │ ├── input + │ │ │ ├── const: 1 + │ │ │ └── const: 2 + │ │ └── body + │ │ └── project + │ │ ├── columns: "?column?":3 + │ │ ├── values + │ │ │ └── tuple + │ │ └── projections + │ │ └── plus [as="?column?":3] + │ │ ├── variable: x:1 + │ │ └── variable: y:2 + │ └── const: 3 + └── body + └── project + ├── columns: "?column?":6 + ├── values + │ └── tuple + └── projections + └── plus [as="?column?":6] + ├── variable: x:4 + └── variable: y:5 + +build format=show-scalars +SELECT add(a, b) FROM abc +---- +project + ├── columns: add:9 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + └── projections + └── udf: add [as=add:9] + ├── args: x:6 y:7 + ├── input + │ ├── variable: a:1 + │ └── variable: b:2 + └── body └── project - ├── columns: "?column?":11!null + ├── columns: "?column?":8 ├── values │ └── tuple └── projections - └── const: 2 [as="?column?":11] + └── plus [as="?column?":8] + ├── variable: x:6 + └── variable: y:7 + +build format=show-scalars +SELECT * FROM abc WHERE a = add(b, c) +---- +project + ├── columns: a:1!null b:2 c:3 + └── select + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + └── filters + └── eq + ├── variable: a:1 + └── udf: add + ├── args: x:6 y:7 + ├── input + │ ├── variable: b:2 + │ └── variable: c:3 + └── body + └── project + ├── columns: "?column?":8 + ├── values + │ └── tuple + └── projections + └── plus [as="?column?":8] + ├── variable: x:6 + └── variable: y:7 + +build format=show-scalars +SELECT * FROM abc WHERE a = add(add(b, c), 3) +---- +project + ├── columns: a:1!null b:2 c:3 + └── select + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + └── filters + └── eq + ├── variable: a:1 + └── udf: add + ├── args: x:9 y:10 + ├── input + │ ├── udf: add + │ │ ├── args: x:6 y:7 + │ │ ├── input + │ │ │ ├── variable: b:2 + │ │ │ └── variable: c:3 + │ │ └── body + │ │ └── project + │ │ ├── columns: "?column?":8 + │ │ ├── values + │ │ │ └── tuple + │ │ └── projections + │ │ └── plus [as="?column?":8] + │ │ ├── variable: x:6 + │ │ └── variable: y:7 + │ └── const: 3 + └── body + └── project + ├── columns: "?column?":11 + ├── values + │ └── tuple + └── projections + └── plus [as="?column?":11] + ├── variable: x:9 + └── variable: y:10 + +exec-ddl +CREATE FUNCTION fetch_b(a_arg INT) RETURNS INT LANGUAGE SQL AS $$ + SELECT b FROM abc WHERE a = a_arg +$$; +---- + +build format=show-scalars +SELECT fetch_b(1) +---- +project + ├── columns: fetch_b:7 + ├── values + │ └── tuple + └── projections + └── udf: fetch_b [as=fetch_b:7] + ├── args: a_arg:1 + ├── input + │ └── const: 1 + └── body + └── project + ├── columns: b:3 + └── select + ├── columns: a:2!null b:3 c:4 crdb_internal_mvcc_timestamp:5 tableoid:6 + ├── scan abc + │ └── columns: a:2!null b:3 c:4 crdb_internal_mvcc_timestamp:5 tableoid:6 + └── filters + └── eq + ├── variable: a:2 + └── variable: a_arg:1 + +build format=show-scalars +SELECT fetch_b(add(1, 2)) +---- +project + ├── columns: fetch_b:10 + ├── values + │ └── tuple + └── projections + └── udf: fetch_b [as=fetch_b:10] + ├── args: a_arg:4 + ├── input + │ └── udf: add + │ ├── args: x:1 y:2 + │ ├── input + │ │ ├── const: 1 + │ │ └── const: 2 + │ └── body + │ └── project + │ ├── columns: "?column?":3 + │ ├── values + │ │ └── tuple + │ └── projections + │ └── plus [as="?column?":3] + │ ├── variable: x:1 + │ └── variable: y:2 + └── body + └── project + ├── columns: b:6 + └── select + ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 tableoid:9 + ├── scan abc + │ └── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 tableoid:9 + └── filters + └── eq + ├── variable: a:5 + └── variable: a_arg:4 + +build format=show-scalars +SELECT * FROM abc WHERE b = fetch_b(a) +---- +project + ├── columns: a:1!null b:2!null c:3 + └── select + ├── columns: a:1!null b:2!null c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 tableoid:5 + └── filters + └── eq + ├── variable: b:2 + └── udf: fetch_b + ├── args: a_arg:6 + ├── input + │ └── variable: a:1 + └── body + └── project + ├── columns: b:8 + └── select + ├── columns: a:7!null b:8 c:9 crdb_internal_mvcc_timestamp:10 tableoid:11 + ├── scan abc + │ └── columns: a:7!null b:8 c:9 crdb_internal_mvcc_timestamp:10 tableoid:11 + └── filters + └── eq + ├── variable: a:7 + └── variable: a_arg:6 + +exec-ddl +CREATE FUNCTION shadowed_a(a INT) RETURNS INT LANGUAGE SQL AS $$ + SELECT c FROM abc WHERE abc.b = a +$$; +---- + +# The column "a" from the table takes precedence over the argument "a". +build format=show-scalars +SELECT shadowed_a(1) +---- +project + ├── columns: shadowed_a:7 + ├── values + │ └── tuple + └── projections + └── udf: shadowed_a [as=shadowed_a:7] + ├── args: a:1 + ├── input + │ └── const: 1 + └── body + └── project + ├── columns: c:4 + └── select + ├── columns: abc.a:2!null b:3!null c:4 crdb_internal_mvcc_timestamp:5 tableoid:6 + ├── scan abc + │ └── columns: abc.a:2!null b:3 c:4 crdb_internal_mvcc_timestamp:5 tableoid:6 + └── filters + └── eq + ├── variable: b:3 + └── variable: abc.a:2 diff --git a/pkg/sql/routine.go b/pkg/sql/routine.go index 3d8ca5a93cd0..1ea1216cb836 100644 --- a/pkg/sql/routine.go +++ b/pkg/sql/routine.go @@ -21,24 +21,24 @@ import ( // routine's PlanFn to generate a plan for each statement in the routine, then // runs the plans. The resulting value of the last statement in the routine is // returned. -// TODO(mgartner): Support executing multi-statement routines. func (p *planner) EvalRoutineExpr( - ctx context.Context, expr *tree.RoutineExpr, + ctx context.Context, expr *tree.RoutineExpr, input tree.Datums, ) (result tree.Datum, err error) { - typs := []*types.T{expr.ResolvedType()} + retTypes := []*types.T{expr.ResolvedType()} // The result of the routine is the result of the last statement. The result // of any preceding statements is ignored. We set up a rowResultWriter that // can store the results of the final statement here. var rch rowContainerHelper - rch.Init(typs, p.ExtendedEvalContext(), "routine" /* opName */) + rch.Init(retTypes, p.ExtendedEvalContext(), "routine" /* opName */) defer rch.Close(ctx) rrw := NewRowResultWriter(&rch) // Execute each statement in the routine sequentially. + ef := newExecFactory(p) for i := 0; i < expr.NumStmts; i++ { // Generate a plan for executing the ith statement. - plan, err := expr.PlanFn(ctx, newExecFactory(p), i) + plan, err := expr.PlanFn(ctx, ef, i, input) if err != nil { return nil, err } @@ -69,7 +69,7 @@ func (p *planner) EvalRoutineExpr( // Adding the limit would be valid because any other rows after the // first can simply be ignored. The limit could also be beneficial // because it could allow additional query plan optimizations. - rightRowsIterator := newRowContainerIterator(ctx, rch, typs) + rightRowsIterator := newRowContainerIterator(ctx, rch, retTypes) defer rightRowsIterator.Close() res, err := rightRowsIterator.Next() if err != nil { diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index c64cc71a7a43..fa8f787c49b4 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -203,8 +203,11 @@ type Planner interface { // EvalSubquery returns the Datum for the given subquery node. EvalSubquery(expr *tree.Subquery) (tree.Datum, error) - // EvalRoutineExpr evaluates a routine and returns the resulting datum. - EvalRoutineExpr(ctx context.Context, expr *tree.RoutineExpr) (tree.Datum, error) + // EvalRoutineExpr evaluates a routine with the given input datums and + // returns the resulting datum. + EvalRoutineExpr( + ctx context.Context, expr *tree.RoutineExpr, input tree.Datums, + ) (tree.Datum, error) // UnsafeUpsertDescriptor is used to repair descriptors in dire // circumstances. See the comment on the planner implementation. diff --git a/pkg/sql/sem/eval/expr.go b/pkg/sql/sem/eval/expr.go index 589bba8dff1a..26a8e13ae94c 100644 --- a/pkg/sql/sem/eval/expr.go +++ b/pkg/sql/sem/eval/expr.go @@ -596,7 +596,21 @@ func (e *evaluator) EvalSubquery(subquery *tree.Subquery) (tree.Datum, error) { } func (e *evaluator) EvalRoutineExpr(routine *tree.RoutineExpr) (tree.Datum, error) { - return e.Planner.EvalRoutineExpr(e.Context, routine) + var err error + var input tree.Datums + if len(routine.Input) > 0 { + // Evaluate each input expression. + // TODO(mgartner): Use a scratch tree.Datums to avoid allocation on + // every invocation. + input = make(tree.Datums, len(routine.Input)) + for i := range routine.Input { + input[i], err = routine.Input[i].Eval(e) + if err != nil { + return nil, err + } + } + } + return e.Planner.EvalRoutineExpr(e.Context, routine, input) } func (e *evaluator) EvalTuple(t *tree.Tuple) (tree.Datum, error) { diff --git a/pkg/sql/sem/tree/routine.go b/pkg/sql/sem/tree/routine.go index 3dc5fb1ca168..48e3445c1144 100644 --- a/pkg/sql/sem/tree/routine.go +++ b/pkg/sql/sem/tree/routine.go @@ -18,7 +18,9 @@ import ( // RoutinePlanFn creates a plan for the execution of one statement within a // routine. -type RoutinePlanFn func(_ context.Context, _ RoutineExecFactory, stmtIdx int) (RoutinePlan, error) +type RoutinePlanFn func( + _ context.Context, _ RoutineExecFactory, stmtIdx int, input Datums, +) (RoutinePlan, error) // RoutinePlan represents a plan for a statement in a routine. It currently maps // to exec.Plan. We use the empty interface here rather then exec.Plan to avoid @@ -36,18 +38,27 @@ type RoutineExecFactory interface{} // function. It is only created by execbuilder - it is never constructed during // parsing. type RoutineExpr struct { - PlanFn RoutinePlanFn + // Input contains the input expressions to the routine. + Input TypedExprs + + // PlanFn returns an exec plan for a given statement in the routine. + PlanFn RoutinePlanFn + + // NumStmts is the number of statements in the routine. NumStmts int - Typ *types.T + + // Typ is the type of the routine's result. + Typ *types.T name string } // NewTypedRoutineExpr returns a new RoutineExpr that is well-typed. func NewTypedRoutineExpr( - name string, planFn RoutinePlanFn, numStmts int, typ *types.T, + name string, input TypedExprs, planFn RoutinePlanFn, numStmts int, typ *types.T, ) *RoutineExpr { return &RoutineExpr{ + Input: input, PlanFn: planFn, NumStmts: numStmts, Typ: typ, @@ -69,7 +80,14 @@ func (node *RoutineExpr) ResolvedType() *types.T { // Format is part of the Expr interface. func (node *RoutineExpr) Format(ctx *FmtCtx) { - ctx.Printf("%s()", node.name) + ctx.Printf("%s(", node.name) + for i := range node.Input { + node.Input[i].Format(ctx) + if i > 0 { + ctx.WriteString(", ") + } + } + ctx.WriteByte(')') } // Walk is part of the Expr interface. From 7829091ed9141353b0595a05898cedc75d614968 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 7 Aug 2022 14:39:36 -0400 Subject: [PATCH 3/5] sql: do not evaluate strict UDFs if any input values are NULL A UDF can have one of two behaviors when it is invoked with NULL inputs: 1. If the UDF is `CALLED ON NULL INPUT` (the default) then the function is evaluated regardless of whether or not any of the input values are NULL. 2. If the UDF `RETURNS NULL ON NULL INPUT` or is `STRICT` then the function is not evaluated if any of the input values are NULL. Instead, the function directly results in NULL. This commit implements these two behaviors. In the future, we can add a normalization rule that folds a strict UDF if any of its inputs are constant NULL values. Release note: None --- pkg/sql/opt/exec/execbuilder/scalar.go | 9 +++++- pkg/sql/opt/exec/execbuilder/testdata/udf | 36 ++++++++++++++++++++++ pkg/sql/opt/ops/scalar.opt | 5 +++ pkg/sql/opt/optbuilder/scalar.go | 11 ++++--- pkg/sql/opt/testutils/testcat/testdata/udf | 10 ++++++ pkg/sql/routine.go | 10 ++++++ pkg/sql/sem/tree/routine.go | 23 ++++++++++---- 7 files changed, 92 insertions(+), 12 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index 17825b96fedf..de49affc1595 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -737,5 +737,12 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ } return plan, nil } - return tree.NewTypedRoutineExpr(udf.Name, inputExprs, planFn, len(udf.Body), udf.Typ), nil + return tree.NewTypedRoutineExpr( + udf.Name, + inputExprs, + planFn, + len(udf.Body), + udf.Typ, + udf.CalledOnNullInput, + ), nil } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/udf b/pkg/sql/opt/exec/execbuilder/testdata/udf index 9728f87808fc..4d855af35967 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/udf +++ b/pkg/sql/opt/exec/execbuilder/testdata/udf @@ -59,3 +59,39 @@ Scan /Table/106/1/1/0 Scan /Table/106/1/2/0 Scan /Table/106/1/1/0 Scan /Table/106/1/2/0 + +statement ok +CREATE FUNCTION fetch_a_of_1(i INT) RETURNS INT CALLED ON NULL INPUT LANGUAGE SQL AS $$ + SELECT a FROM t WHERE k = 1 +$$ + +statement ok +CREATE FUNCTION fetch_a_of_1_strict(i INT) RETURNS INT RETURNS NULL ON NULL INPUT LANGUAGE SQL AS $$ + SELECT a FROM t WHERE k = 1 +$$ + +statement ok +CREATE FUNCTION fetch_a_of_2_strict(i INT, j INT) RETURNS INT STRICT LANGUAGE SQL AS $$ + SELECT a FROM t WHERE k = 2 +$$ + +# When the function is CALLED ON NULL INPUT then it should be evaluated +# regardless of whether or not any of its inputs are NULL. The trace proves that +# the function is evaluated. It shows the scan performed by the statement in the +# function body. +query T kvtrace +SELECT fetch_a_of_1(NULL::INT) +---- +Scan /Table/106/1/1/0 + +# When the function RETURNS NULL ON NULL INPUT or STRICT then it should not be +# evaluated if any of its inputs are NULL. The empty traces prove that the +# function is not evaluated. No scan is performed for the statement in the +# function body. +query T kvtrace +SELECT fetch_a_of_1_strict(NULL::INT) +---- + +query T kvtrace +SELECT fetch_a_of_2_strict(1, NULL::INT) +---- diff --git a/pkg/sql/opt/ops/scalar.opt b/pkg/sql/opt/ops/scalar.opt index ebd23dd83312..e045a654aa5e 100644 --- a/pkg/sql/opt/ops/scalar.opt +++ b/pkg/sql/opt/ops/scalar.opt @@ -1245,6 +1245,11 @@ define UDFPrivate { # Volatility is the user-provided volatility of the function given during # CREATE FUNCTION. Volatility Volatility + + # CalledOnNullInput is true if the function should be called when any of its + # inputs are NULL. If false, the function will not be evaluated in the + # presence of NULL inputs, and will instead evaluate directly to NULL. + CalledOnNullInput bool } # KVOptions is a set of KVOptionItems that specify arbitrary keys and values diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index ac9625da7c72..38d85e5ae20d 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -672,11 +672,12 @@ func (b *Builder) buildUDF( out = b.factory.ConstructUDF( input, &memo.UDFPrivate{ - Name: def.Name, - Body: rels, - Typ: f.ResolvedType(), - Volatility: o.Volatility, - ArgCols: argCols, + Name: def.Name, + ArgCols: argCols, + Body: rels, + Typ: f.ResolvedType(), + Volatility: o.Volatility, + CalledOnNullInput: o.CalledOnNullInput, }, ) return b.finishBuildScalar(f, out, inScope, outScope, outCol) diff --git a/pkg/sql/opt/testutils/testcat/testdata/udf b/pkg/sql/opt/testutils/testcat/testdata/udf index 3ffc7a2a8608..8d48dcb2b3ec 100644 --- a/pkg/sql/opt/testutils/testcat/testdata/udf +++ b/pkg/sql/opt/testutils/testcat/testdata/udf @@ -55,3 +55,13 @@ SHOW CREATE FUNCTION e ---- FUNCTION e(i: int) -> int [immutable, called-on-null-input=false] └── SELECT i + +exec-ddl +CREATE FUNCTION f() RETURNS INT RETURNS NULL ON NULL INPUT LANGUAGE SQL AS 'SELECT 1' +---- + +exec-ddl +SHOW CREATE FUNCTION f +---- +FUNCTION f() -> int [volatile, called-on-null-input=false] + └── SELECT 1 diff --git a/pkg/sql/routine.go b/pkg/sql/routine.go index 1ea1216cb836..6e55bca6e10a 100644 --- a/pkg/sql/routine.go +++ b/pkg/sql/routine.go @@ -24,6 +24,16 @@ import ( func (p *planner) EvalRoutineExpr( ctx context.Context, expr *tree.RoutineExpr, input tree.Datums, ) (result tree.Datum, err error) { + // If the routine should not be called on null input, then directly return + // NULL if any of the datums in the input are NULL. + if !expr.CalledOnNullInput { + for i := range input { + if input[i] == tree.DNull { + return tree.DNull, nil + } + } + } + retTypes := []*types.T{expr.ResolvedType()} // The result of the routine is the result of the last statement. The result diff --git a/pkg/sql/sem/tree/routine.go b/pkg/sql/sem/tree/routine.go index 48e3445c1144..00dfdfcf6c5b 100644 --- a/pkg/sql/sem/tree/routine.go +++ b/pkg/sql/sem/tree/routine.go @@ -50,19 +50,30 @@ type RoutineExpr struct { // Typ is the type of the routine's result. Typ *types.T + // CalledOnNullInput is true if the function should be called when any of + // its inputs are NULL. If false, the function will not be evaluated in the + // presence of null inputs, and will instead evaluate directly to NULL. + CalledOnNullInput bool + name string } // NewTypedRoutineExpr returns a new RoutineExpr that is well-typed. func NewTypedRoutineExpr( - name string, input TypedExprs, planFn RoutinePlanFn, numStmts int, typ *types.T, + name string, + input TypedExprs, + planFn RoutinePlanFn, + numStmts int, + typ *types.T, + calledOnNullInput bool, ) *RoutineExpr { return &RoutineExpr{ - Input: input, - PlanFn: planFn, - NumStmts: numStmts, - Typ: typ, - name: name, + Input: input, + PlanFn: planFn, + NumStmts: numStmts, + Typ: typ, + CalledOnNullInput: calledOnNullInput, + name: name, } } From c4bf42a770ecebb36c2af88b33562aed89078d97 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Mon, 8 Aug 2022 17:18:25 -0400 Subject: [PATCH 4/5] sql: make mutations visible to volatile UDFs The volatility of a UDF affects the visibility of mutations made by the statement calling the function. A volatile function will see these mutations. Also, statements within a volatile function's body will see changes made by previous statements the function body (note that this is left untested in this commit because we do not currently support mutations within UDF bodies). In contrast, a stable, immutable, or leakproof function will see a snapshot of the data as of the start of the statement calling the function. Release note: None --- pkg/sql/logictest/testdata/logic_test/udf | 39 +++++++++++++++++++++++ pkg/sql/opt/exec/execbuilder/scalar.go | 1 + pkg/sql/opt/ops/scalar.opt | 7 ++++ pkg/sql/routine.go | 22 +++++++++++++ pkg/sql/sem/tree/routine.go | 11 +++++++ 5 files changed, 80 insertions(+) diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 850ae5f74f39..cdb6fafb7a90 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -946,3 +946,42 @@ query I SELECT fetch_b(99999999) ---- NULL + +subtest volatility + +statement ok +CREATE TABLE kv (k INT PRIMARY KEY, v INT); +INSERT INTO kv VALUES (1, 1), (2, 2), (3, 3); +CREATE FUNCTION get_l(i INT) RETURNS INT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$ + SELECT v FROM kv WHERE k = i; +$$; +CREATE FUNCTION get_i(i INT) RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ + SELECT v FROM kv WHERE k = i; +$$; +CREATE FUNCTION get_s(i INT) RETURNS INT STABLE LANGUAGE SQL AS $$ + SELECT v FROM kv WHERE k = i; +$$; +CREATE FUNCTION get_v(i INT) RETURNS INT VOLATILE LANGUAGE SQL AS $$ + SELECT v FROM kv WHERE k = i; +$$; +CREATE FUNCTION int_identity_v(i INT) RETURNS INT VOLATILE LANGUAGE SQL AS $$ + SELECT i; +$$; + +# Only the volatile functions should see the changes made by the UPDATE in the +# CTE. +query IIIIIIII colnames +WITH u AS ( + UPDATE kv SET v = v + 10 RETURNING k +) +SELECT + get_l(k) l1, get_l(int_identity_v(k)) l2, + get_i(k) i1, get_i(int_identity_v(k)) i2, + get_s(k) s1, get_s(int_identity_v(k)) s2, + get_v(k) v1, get_v(int_identity_v(k)) v2 +FROM u; +---- +l1 l2 i1 i2 s1 s2 v1 v2 +1 1 1 1 1 1 11 11 +2 2 2 2 2 2 12 12 +3 3 3 3 3 3 13 13 diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index de49affc1595..a0b767b05484 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -743,6 +743,7 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ planFn, len(udf.Body), udf.Typ, + udf.Volatility, udf.CalledOnNullInput, ), nil } diff --git a/pkg/sql/opt/ops/scalar.opt b/pkg/sql/opt/ops/scalar.opt index e045a654aa5e..5e1ea80f9e11 100644 --- a/pkg/sql/opt/ops/scalar.opt +++ b/pkg/sql/opt/ops/scalar.opt @@ -1244,6 +1244,13 @@ define UDFPrivate { # Volatility is the user-provided volatility of the function given during # CREATE FUNCTION. + # + # Volatility affects the visibility of mutations made by the statement + # calling the function. A volatile function will see these mutations. Also, + # statements within a volatile function's body will see changes made by + # previous statements in the function body. In contrast, a stable, + # immutable, or leakproof function will see a snapshot of the data as of the + # start of the statement calling the function. Volatility Volatility # CalledOnNullInput is true if the function should be called when any of its diff --git a/pkg/sql/routine.go b/pkg/sql/routine.go index 6e55bca6e10a..7227c49de401 100644 --- a/pkg/sql/routine.go +++ b/pkg/sql/routine.go @@ -13,7 +13,9 @@ package sql import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -44,6 +46,18 @@ func (p *planner) EvalRoutineExpr( defer rch.Close(ctx) rrw := NewRowResultWriter(&rch) + // Configure stepping for volatile routines so that mutations made by the + // invoking statement are visible to the routine. + txn := p.Txn() + if expr.Volatility == volatility.Volatile { + prevSteppingMode := txn.ConfigureStepping(ctx, kv.SteppingEnabled) + prevSeqNum := txn.GetLeafTxnInputState(ctx).ReadSeqNum + defer func() { + _ = p.Txn().ConfigureStepping(ctx, prevSteppingMode) + err = txn.SetReadSeqNum(prevSeqNum) + }() + } + // Execute each statement in the routine sequentially. ef := newExecFactory(p) for i := 0; i < expr.NumStmts; i++ { @@ -62,6 +76,14 @@ func (p *planner) EvalRoutineExpr( w = &droppingResultWriter{} } + // Place a sequence point before each statement in the routine for + // volatile functions. + if expr.Volatility == volatility.Volatile { + if err := txn.Step(ctx); err != nil { + return nil, err + } + } + // TODO(mgartner): Add a new tracing.ChildSpan to the context for better // tracing of UDFs, like we do with apply-joins. err = runPlanInsidePlan(ctx, p.RunParams(ctx), plan.(*planComponents), w) diff --git a/pkg/sql/sem/tree/routine.go b/pkg/sql/sem/tree/routine.go index 00dfdfcf6c5b..828fc9c5c508 100644 --- a/pkg/sql/sem/tree/routine.go +++ b/pkg/sql/sem/tree/routine.go @@ -13,6 +13,7 @@ package tree import ( "context" + "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -50,6 +51,14 @@ type RoutineExpr struct { // Typ is the type of the routine's result. Typ *types.T + // Volatility affects the visibility of mutations made by the statement + // invoking the routine. A volatile routine will see these mutations. Also, + // statements within a volatile function's body will see changes made by + // previous statements in the routine. In contrast, a stable, immutable, + // or leakproof function will see a snapshot of the data as of the start of + // the statement calling the function. + Volatility volatility.V + // CalledOnNullInput is true if the function should be called when any of // its inputs are NULL. If false, the function will not be evaluated in the // presence of null inputs, and will instead evaluate directly to NULL. @@ -65,6 +74,7 @@ func NewTypedRoutineExpr( planFn RoutinePlanFn, numStmts int, typ *types.T, + v volatility.V, calledOnNullInput bool, ) *RoutineExpr { return &RoutineExpr{ @@ -72,6 +82,7 @@ func NewTypedRoutineExpr( PlanFn: planFn, NumStmts: numStmts, Typ: typ, + Volatility: v, CalledOnNullInput: calledOnNullInput, name: name, } From 88ee320bddf8d27e21e3e6f8f483a205ec5f2d02 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Fri, 5 Aug 2022 18:31:20 -0600 Subject: [PATCH 5/5] admission: add support for disk bandwidth as a bottleneck resource We assume that: - There is a provisioned known limit on the sum of read and write bandwidth. This limit is allowed to change. - Admission control can only shape the rate of admission of writes. Writes also cause reads, since compactions do reads and writes. There are multiple challenges: - We are unable to precisely track the causes of disk read bandwidth, since we do not have observability into what reads missed the OS page cache. That is, we don't know how much of the reads were due to incoming reads (that we don't shape) and how much due to compaction read bandwidth. - We don't shape incoming reads. - There can be a large time lag between the shaping of incoming writes, and when it affects actual writes in the system, since compaction backlog can build up in various levels of the LSM store. - Signals of overload are coarse, since we cannot view all the internal queues that can build up due to resource overload. For instance, different examples of bandwidth saturation exhibit different latency effects, presumably because the queue buildup is different. So it is non-trivial to approach full utilization without risking high latency. Due to these challenges, and previous design attempts that were quite complicated (and incomplete), we adopt a goal of simplicity of design, and strong abstraction boundaries. - The disk load is abstracted using an enum. The diskLoadWatcher can be evolved independently. - The approach uses easy to understand small multiplicative increase and large multiplicative decrease, (unlike what we do for flush and compaction tokens, where we try to more precisely calculate the sustainable rates). Since we are using a simple approach that is somewhat coarse in its behavior, we start by limiting its application to two kinds of writes: - Incoming writes that are deemed "elastic": This can be done by introducing a work-class (in addition to admissionpb.WorkPriority), or by implying a work-class from the priority (e.g. priorities < NormalPri are deemed elastic). This prototype does the latter. - Optional compactions: We assume that the LSM store is configured with a ceiling on number of regular concurrent compactions, and if it needs more it can request resources for additional (optional) compactions. These latter compactions can be limited by this approach. See cockroachdb/pebble/issues/1329 for motivation. This control on compactions is not currently implemented and is future work (though the prototype in https://github.com/cockroachdb/cockroach/pull/82813 had code for it). The reader should start with disk_bandwidth.go, consisting of - diskLoadWatcher: which computes load levels. - diskBandwidthLimiter: It used the load level computed by diskLoadWatcher to limit write tokens for elastic writes and in the future will also limit compactions. There is significant refactoring and changes in granter.go and work_queue.go. This is driven by the fact that: - Previously the tokens were for L0 and now we need to support tokens for bytes into L0 and tokens for bytes into the LSM (the former being a subset of the latter). - Elastic work is in a different WorkQueue than regular work, but they are competing for the same tokens. A different WorkQueue is needed to prevent a situation where elastic work for one tenant is queued ahead of regualar work from another tenant, and stops the latter from making progress due to lack of elastic tokens. The latter is handled by allowing kvSlotGranter to multiplex across multiple requesters, via multiple child granters. A number of interfaces are adjusted to make this viable. In general, the GrantCoordinator is now slightly dumber and some of that logic is moved into the granters. For the former (handling two kinds of tokens), I considered adding multiple resource dimensions to the granter-requester interaction but found it too complicated. Instead we rely on the observation that we request tokens based on the total incoming bytes of the request (not just L0), and when the request is completed, tell the granter how many bytes went into L0. The latter allows us to return tokens to L0. So at the time the request is completed, we can account separately for the L0 tokens and these new tokens for all incoming bytes (which we are calling disk bandwidth tokens, since they are constrained based on disk bandwidth). This is a cleaned up version of the prototype in https://github.com/cockroachdb/cockroach/pull/82813 which contains the experimental results. The plumbing from the KV layer to populate the disk reads, writes and provisioned bandwidth is absent in this PR, and will be added in a subsequent PR. Disk bandwidth bottlenecks are considered only if both the following are true: - DiskStats.ProvisionedBandwidth is non-zero. - The cluster setting admission.disk_bandwidth_tokens.elastic.enabled is true (defaults to true). Informs #82898 Release note: None (the cluster setting mentioned earlier is useless since the integration with CockroachDB will be in a future PR). --- docs/generated/settings/settings.html | 1 + pkg/util/admission/BUILD.bazel | 2 + pkg/util/admission/disk_bandwidth.go | 367 ++++++++ pkg/util/admission/disk_bandwidth_test.go | 103 +++ pkg/util/admission/doc.go | 2 + pkg/util/admission/granter.go | 811 ++++++++++++++---- pkg/util/admission/granter_test.go | 248 ++++-- pkg/util/admission/store_token_estimation.go | 239 +++--- .../admission/store_token_estimation_test.go | 21 +- .../admission/testdata/disk_bandwidth_limiter | 41 + pkg/util/admission/testdata/disk_load_watcher | 78 ++ .../testdata/format_adjust_tokens_stats.txt | 4 +- pkg/util/admission/testdata/granter | 174 +++- pkg/util/admission/testdata/io_load_listener | 547 ++++++------ .../testdata/store_per_work_token_estimator | 79 +- pkg/util/admission/testdata/store_work_queue | 142 ++- pkg/util/admission/work_queue.go | 61 +- pkg/util/admission/work_queue_test.go | 57 +- 18 files changed, 2249 insertions(+), 728 deletions(-) create mode 100644 pkg/util/admission/disk_bandwidth.go create mode 100644 pkg/util/admission/disk_bandwidth_test.go create mode 100644 pkg/util/admission/testdata/disk_bandwidth_limiter create mode 100644 pkg/util/admission/testdata/disk_load_watcher diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 1cde863bee26..f2db8bc2a28f 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -1,6 +1,7 @@ + diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index 904c3fa97b65..11e528f3b374 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "admission", srcs = [ + "disk_bandwidth.go", "doc.go", "granter.go", "store_token_estimation.go", @@ -32,6 +33,7 @@ go_library( go_test( name = "admission_test", srcs = [ + "disk_bandwidth_test.go", "granter_test.go", "store_token_estimation_test.go", "work_queue_test.go", diff --git a/pkg/util/admission/disk_bandwidth.go b/pkg/util/admission/disk_bandwidth.go new file mode 100644 index 000000000000..63846c56bfeb --- /dev/null +++ b/pkg/util/admission/disk_bandwidth.go @@ -0,0 +1,367 @@ +// Copyright 2022 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 admission + +import ( + "context" + "math" + + "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/redact" +) + +// The functionality in this file is geared towards preventing chronic overload +// of disk bandwidth which typically results in severely high latency for all work. +// +// For now, we assume that: +// - There is a provisioned limit on the sum of read and write bandwidth. This +// limit is allowed to change. This is true for block devices of major cloud +// providers. +// - Admission control can only shape the rate of admission of writes. Writes +// also cause reads, since compactions do reads and writes. +// +// There are multiple challenges: +// - We are unable to precisely track the causes of disk read bandwidth, since +// we do not have observability into what reads missed the OS page cache. +// That is we don't know how much of the reads were due to incoming reads +// (that we don't shape) and how much due to compaction read bandwidth. +// +// - We don't shape incoming reads. +// +// - There can be a large lag (1+min) between the shaping of incoming writes, +// and when it affects actual writes in the system, since compaction backlog +// can build up in various levels of the LSM store. +// +// - Signals of overload are coarse, since we cannot view all the internal +// queues that can build up due to resource overload. For instance, +// different examples of bandwidth saturation exhibit very different +// latency effects, presumably because the queue buildup is different. So it +// is non-trivial to approach full utilization without risking high latency. +// +// Due to these challenges, we adopt a goal of simplicity of design, and +// strong abstraction boundaries. +// +// - The disk load is abstracted using an enum, diskLoadLevel. The +// diskLoadWatcher, that maps load signals to this enum, can be evolved +// independently. +// +// - The approach uses easy to understand small multiplicative increase and +// large multiplicative decrease, (unlike what we do for flush and +// compaction tokens, where we try to more precisely calculate the +// sustainable rates). +// +// Since we are using a simple approach that is somewhat coarse in its behavior, +// we start by limiting its application to two kinds of writes (the second one +// is future work, and not yet implemented): +// +// - Incoming writes that are deemed "elastic": This can be done by +// introducing a work-class (in addition to admissionpb.WorkPriority), or by +// implying a work-class from the priority (e.g. priorities < NormalPri are +// deemed elastic). +// +// - Optional compactions: We assume that the LSM store is configured with a +// ceiling on number of regular concurrent compactions, and if it needs more +// it can request resources for additional (optional) compactions. These +// latter compactions can be limited by this approach. See +// https://github.com/cockroachdb/pebble/issues/1329 for motivation. +// TODO(sumeer): this compaction control is not yet done, though how to do +// it is included in the prototype in +// https://github.com/cockroachdb/cockroach/pull/82813 +// +// Extending this to all incoming writes is future work. + +// The load level of a disk. +type diskLoadLevel int8 + +const ( + // diskLoadLow implies no need to shape anything. + diskLoadLow diskLoadLevel = iota + // diskLoadModerate implies shaping and small multiplicative increase. + diskLoadModerate + // diskLoadHigh implies shaping and hold steady. + diskLoadHigh + // diskLoadOverload implies shaping and large multiplicative decrease. + diskLoadOverload +) + +func diskLoadLevelString(level diskLoadLevel) redact.SafeString { + switch level { + case diskLoadLow: + return "low" + case diskLoadModerate: + return "moderate" + case diskLoadHigh: + return "high" + case diskLoadOverload: + return "overload" + } + return "" +} + +// diskLoadWatcher computes the diskLoadLevel based on provided stats. +type diskLoadWatcher struct { + lastInterval intervalDiskLoadInfo + lastUtil float64 + loadLevel diskLoadLevel +} + +// intervalDiskLoadInfo provides disk stats over an adjustmentInterval. +type intervalDiskLoadInfo struct { + // readBandwidth is the measure disk read bandwidth in bytes/s. + readBandwidth int64 + // writeBandwidth is the measured disk write bandwidth in bytes/s. + writeBandwidth int64 + // provisionedBandwidth is the aggregate (read+write) provisioned bandwidth + // in bytes/s. + provisionedBandwidth int64 +} + +// setIntervalInfo is called at the same time as ioLoadListener.pebbleMetricsTick. +func (d *diskLoadWatcher) setIntervalInfo(load intervalDiskLoadInfo) { + lastInterval := load + util := float64(load.readBandwidth+load.writeBandwidth) / float64(load.provisionedBandwidth) + // The constants and other heuristics in the following logic can seem + // extremely arbitrary: they were subject to some tuning and evolution based + // on the experiments in https://github.com/cockroachdb/cockroach/pull/82813 + // that used (a) an artificial provisioned bandwidth limit lower than the + // actual, to see how well the system stayed within that limit, (b) an + // actual provisioned bandwidth limit. The difficulty in general is that + // small changes can have outsize influence if a higher number of + // compactions start happening. + var loadLevel diskLoadLevel + const lowUtilThreshold = 0.3 + const moderateUtilThreshold = 0.7 + const highUtilThreshold = 0.95 + const highlyOverUtilizedThreshold = 2.0 + const smallDelta = 0.05 + if util < lowUtilThreshold { + // Were at moderate or lower and have not increased significantly and the + // lastUtil was also low, then we can afford to stop limiting tokens. We + // are trying to carefully narrow this case since not limiting tokens can + // blow up bandwidth. + // + // An alternative would be to never have unlimited tokens, since that + // ensures there is always some reasonable bound in place. It may mean + // that the initial tokens are insufficient and the tokens catch up to + // what is needed with some lag, and during that time there is unnecessary + // queueing. This downside could be avoided by ramping up faster. This + // alternative is worth investigating. + + if d.loadLevel <= diskLoadModerate && util < d.lastUtil+smallDelta && + d.lastUtil < lowUtilThreshold { + loadLevel = diskLoadLow + } else { + // util is increasing, or we just dropped from something higher than + // moderate. Give it more time at moderate, where we will gradually + // increase tokens. + loadLevel = diskLoadModerate + } + } else if util < moderateUtilThreshold { + // Wide band from [0.3,0.7) where we gradually increase tokens. Also, 0.7 + // is deliberately a lowish fraction since the effect on compactions can + // lag and kick in later. We are ok with accepting a lower utilization for + // elastic work to make progress. + loadLevel = diskLoadModerate + } else if util < highUtilThreshold || + (util < highlyOverUtilizedThreshold && util < d.lastUtil-smallDelta) { + // Wide band from [0.7,0.95) where we will hold the number of tokens + // steady. We don't want to overreact and decrease too early since + // compaction bandwidth usage can be lumpy. For this same reason, if we + // are trending downward, we want to hold. Note that util < 2 will always + // be true in typical configurations where one cannot actually exceed + // provisioned bandwidth -- but we also run experiments where we + // artificially constrain the provisioned bandwidth, where this is useful. + // And it is possible that some production settings may set a slightly + // lower value of provisioned bandwidth, if they want to further reduce + // the probability of hitting the real provisioned bandwidth due to + // elastic work. + loadLevel = diskLoadHigh + } else { + // Overloaded. We will reduce tokens. + loadLevel = diskLoadOverload + } + *d = diskLoadWatcher{ + lastInterval: lastInterval, + lastUtil: util, + loadLevel: loadLevel, + } + // TODO(sumeer): Use the history of fsync latency and the value in the + // current interval, and if high, increase the load level computed earlier. + // We shouldn't rely fully on syncLatencyMicros since (a) sync latency could + // arise due to an external unrelated outage, (b) some customers may set + // fsync to be a noop. As an alternative to sync latency, we could also + // consider looking at fluctuations of peak-rate that the WAL writer can + // sustain. +} + +func (d *diskLoadWatcher) getLoadLevel() diskLoadLevel { + return d.loadLevel +} + +func (d diskLoadWatcher) SafeFormat(p redact.SafePrinter, _ rune) { + p.Printf("disk bandwidth: read: %s/s, write: %s/s, provisioned: %s/s, util: %.2f", + humanizeutil.IBytes(d.lastInterval.readBandwidth), + humanizeutil.IBytes(d.lastInterval.writeBandwidth), + humanizeutil.IBytes(d.lastInterval.provisionedBandwidth), d.lastUtil) +} + +// intervalLSMInfo provides stats about the LSM over an adjustmentInterval. +type intervalLSMInfo struct { + // Flushed bytes + Ingested bytes seen by the LSM. Ingested bytes incur the + // cost of writing a sstable, even though that is done outside Pebble, so + // ingestion is similar in cost to flushing. Ingested bytes don't cause WAL + // writes, but we ignore that difference for simplicity, and just work with + // the sum of flushed and ingested bytes. + incomingBytes int64 + // regularTokensUsed and elasticTokensUsed are the byte tokens used for + // regular and elastic work respectively. Each of these includes both + // writes that will get flushed and ingested bytes. The + // regularTokensUsed+elasticTokensUsed do not need to sum up to + // incomingBytes, since these stats are produced by different sources. + regularTokensUsed int64 + elasticTokensUsed int64 +} + +type diskBandwidthLimiterState struct { + smoothedIncomingBytes float64 + smoothedElasticFraction float64 + elasticTokens int64 + + prevElasticTokensUsed int64 +} + +// diskBandwidthLimiter produces tokens for elastic work. +type diskBandwidthLimiter struct { + diskLoadWatcher diskLoadWatcher + state diskBandwidthLimiterState +} + +func makeDiskBandwidthLimiter() diskBandwidthLimiter { + return diskBandwidthLimiter{ + state: diskBandwidthLimiterState{ + elasticTokens: math.MaxInt64, + }, + } +} + +// computeElasticTokens is called every adjustmentInterval. +func (d *diskBandwidthLimiter) computeElasticTokens( + ctx context.Context, id intervalDiskLoadInfo, il intervalLSMInfo, +) (elasticTokens int64) { + d.diskLoadWatcher.setIntervalInfo(id) + const alpha = 0.5 + prev := d.state + smoothedIncomingBytes := alpha*float64(il.incomingBytes) + (1-alpha)*prev.smoothedIncomingBytes + smoothedElasticFraction := prev.smoothedElasticFraction + var intElasticFraction float64 + if il.regularTokensUsed+il.elasticTokensUsed > 0 { + intElasticFraction = + float64(il.elasticTokensUsed) / float64(il.regularTokensUsed+il.elasticTokensUsed) + smoothedElasticFraction = alpha*intElasticFraction + (1-alpha)*prev.smoothedElasticFraction + } + intElasticBytes := int64(float64(il.incomingBytes) * intElasticFraction) + ll := d.diskLoadWatcher.getLoadLevel() + + // The constants and other heuristics in the following logic can seem + // arbitrary: they were subject to some tuning and evolution based on the + // experiments in https://github.com/cockroachdb/cockroach/pull/82813 that + // used (a) an artificial provisioned bandwidth limit lower than the actual, + // to see how well the system stayed within that limit, (b) an actual + // provisioned bandwidth limit. The difficulty in general is that small + // changes can have outsize influence if a higher number of compactions + // start happening, or the compaction backlog is cleared. + // + // TODO(sumeer): experiment with a more sophisticated controller for the + // elastic token adjustment, e.g. a PID (Proportional-Integral-Derivative) + // controller. + doLog := true + switch ll { + case diskLoadLow: + elasticTokens = math.MaxInt64 + if elasticTokens == prev.elasticTokens { + doLog = false + } + // else we stay in the common case of low bandwidth usage. + case diskLoadModerate: + tokensFullyUtilized := + // elasticTokens == MaxInt64 is also considered fully utilized since we + // can never fully utilize unlimited tokens. + prev.elasticTokens == math.MaxInt64 || + (prev.elasticTokens > 0 && float64(il.elasticTokensUsed)/float64(prev.elasticTokens) >= 0.8) + + if tokensFullyUtilized { + // Smoothed elastic bytes plus 10% of smoothedIncomingBytes is given to + // elastic work. That is, we are increasing the total incoming bytes by + // 10% (not just the elastic bytes by 10%). Note that each token + // represents 1 incoming byte. + elasticBytes := (smoothedElasticFraction + 0.1) * smoothedIncomingBytes + // Sometimes we see the tokens not increasing even though we are staying + // for multiple intervals at moderate. This is because the smoothed + // fraction and incoming bytes can be decreasing. We do want to increase + // tokens since we know there is spare capacity, so we try many ways + // (that don't look at smoothed numbers only). Also, we sometimes come + // here due to an overload=>moderate transition because compaction + // bandwidth usage can be lumpy (high when there is a backlog and then + // dropping severely) -- in that case we want to start increasing + // immediately, since we have likely decreased too much. + intBasedElasticTokens := (smoothedElasticFraction + 0.1) * float64(il.incomingBytes) + elasticBytes = math.Max(elasticBytes, intBasedElasticTokens) + elasticBytes = math.Max(elasticBytes, 1.1*float64(il.elasticTokensUsed)) + elasticTokens = int64(elasticBytes) + if elasticTokens == 0 { + // Don't get stuck in a situation where smoothedIncomingBytes are 0. + elasticTokens = math.MaxInt64 + } + } else { + // No change. + elasticTokens = prev.elasticTokens + } + case diskLoadHigh: + // No change. + elasticTokens = prev.elasticTokens + case diskLoadOverload: + // Sometimes we come here after a low => overload transition. The + // intElasticBytes will be very high because tokens were unlimited. We + // don't want to use that as the starting point of the decrease if the + // smoothed value is lower. Hence, the min logic below, to try to dampen + // the increase quickly. + elasticTokens = int64(0.5 * math.Min(float64(intElasticBytes), + smoothedElasticFraction*smoothedIncomingBytes)) + } + // We can end up with 0 elastic tokens here -- e.g. if intElasticBytes was 0 + // but we were still overloaded because of compactions. The trouble with 0 + // elastic tokens is that if we don't admit anything, we cannot correct an + // occasional poor estimate of the per-request bytes. So we decide to give + // out at least 1 token. A single elastic request should not be too big for + // this to matter. + elasticTokens = max(1, elasticTokens) + d.state = diskBandwidthLimiterState{ + smoothedIncomingBytes: smoothedIncomingBytes, + smoothedElasticFraction: smoothedElasticFraction, + elasticTokens: elasticTokens, + prevElasticTokensUsed: il.elasticTokensUsed, + } + if doLog { + log.Infof(ctx, "%v", d) + } + return elasticTokens +} + +func (d *diskBandwidthLimiter) SafeFormat(p redact.SafePrinter, _ rune) { + ib := humanizeutil.IBytes + level := d.diskLoadWatcher.getLoadLevel() + p.Printf("diskBandwidthLimiter %s (%v): elastic-frac: %.2f, incoming: %s, "+ + "elastic-tokens (used %s): %s", + diskLoadLevelString(level), d.diskLoadWatcher, d.state.smoothedElasticFraction, + ib(int64(d.state.smoothedIncomingBytes)), ib(d.state.prevElasticTokensUsed), + ib(d.state.elasticTokens)) +} diff --git a/pkg/util/admission/disk_bandwidth_test.go b/pkg/util/admission/disk_bandwidth_test.go new file mode 100644 index 000000000000..c06f60e90baa --- /dev/null +++ b/pkg/util/admission/disk_bandwidth_test.go @@ -0,0 +1,103 @@ +// Copyright 2022 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 admission + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/redact" +) + +func TestDiskLoadWatcher(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var dlw diskLoadWatcher + watcherToString := func() string { + level := dlw.getLoadLevel() + return fmt.Sprintf("%s\nload-level: %s", redact.Sprint(dlw), + diskLoadLevelString(level)) + } + + datadriven.RunTest(t, testutils.TestDataPath(t, "disk_load_watcher"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + dlw = diskLoadWatcher{} + return watcherToString() + + case "interval-info": + var readBandwidth, writeBandwidth, provisionedBandwidth int + d.ScanArgs(t, "read-bw", &readBandwidth) + d.ScanArgs(t, "write-bw", &writeBandwidth) + d.ScanArgs(t, "provisioned-bw", &provisionedBandwidth) + dlw.setIntervalInfo(intervalDiskLoadInfo{ + readBandwidth: int64(readBandwidth), + writeBandwidth: int64(writeBandwidth), + provisionedBandwidth: int64(provisionedBandwidth), + }) + return watcherToString() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) +} + +func TestDiskBandwidthLimiter(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var dbl diskBandwidthLimiter + dblToString := func() string { + return string(redact.Sprint(&dbl)) + } + + datadriven.RunTest(t, testutils.TestDataPath(t, "disk_bandwidth_limiter"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + dbl = makeDiskBandwidthLimiter() + return dblToString() + + case "compute": + var readBandwidth, writeBandwidth, provisionedBandwidth int + d.ScanArgs(t, "read-bw", &readBandwidth) + d.ScanArgs(t, "write-bw", &writeBandwidth) + d.ScanArgs(t, "provisioned-bw", &provisionedBandwidth) + diskLoad := intervalDiskLoadInfo{ + readBandwidth: int64(readBandwidth), + writeBandwidth: int64(writeBandwidth), + provisionedBandwidth: int64(provisionedBandwidth), + } + var incomingBytes, regularTokensUsed, elasticTokensUsed int + d.ScanArgs(t, "incoming-bytes", &incomingBytes) + d.ScanArgs(t, "regular-tokens-used", ®ularTokensUsed) + d.ScanArgs(t, "elastic-tokens-used", &elasticTokensUsed) + lsmInfo := intervalLSMInfo{ + incomingBytes: int64(incomingBytes), + regularTokensUsed: int64(regularTokensUsed), + elasticTokensUsed: int64(elasticTokensUsed), + } + dbl.computeElasticTokens(context.Background(), diskLoad, lsmInfo) + return dblToString() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) +} diff --git a/pkg/util/admission/doc.go b/pkg/util/admission/doc.go index 57c525408fc9..62cdfdd55fdf 100644 --- a/pkg/util/admission/doc.go +++ b/pkg/util/admission/doc.go @@ -48,6 +48,8 @@ // either in a comment here or a separate RFC. // +// TODO(sumeer): update with all the recent changes. + // Internal organization: // // The package is mostly structured as a set of interfaces that are meant to diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index da7188805b7e..89c1369af0b7 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -92,6 +92,19 @@ var MinFlushUtilizationFraction = settings.RegisterFloatSetting( "consider lowering it slightly (after consultation with domain experts)", 0.5, settings.PositiveFloat) +// DiskBandwidthTokensForElasticEnabled controls whether the disk bandwidth +// resource is considered as a possible bottleneck resource. When it becomes a +// bottleneck, tokens for elastic work are limited based on available disk +// bandwidth. The default is true since actually considering disk bandwidth as +// a bottleneck resource requires additional configuration (outside the +// admission package) to calculate the provisioned bandwidth. +var DiskBandwidthTokensForElasticEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "admission.disk_bandwidth_tokens.elastic.enabled", + "when true, and provisioned bandwidth for the disk corresponding to a store is configured, "+ + "tokens for elastic work will be limited if disk bandwidth becomes a bottleneck", + true).WithPublic() + // grantChainID is the ID for a grant chain. See continueGrantChain for // details. type grantChainID uint64 @@ -121,6 +134,10 @@ type requester interface { close() } +type requesterClose interface { + close() +} + // grantKind represents the two kind of ways we grant admission: using a slot // or a token. The slot terminology is akin to a scheduler, where a scheduling // slot must be free for a thread to run. But unlike a scheduler, we don't @@ -164,7 +181,6 @@ type granter interface { tryGet(count int64) bool // returnGrant is called for: // - returning slots after use. - // - returning tokens when the work did not end up using all the tokens. // - returning either slots or tokens when the grant raced with the work // being canceled, and the grantee did not end up doing any work. // @@ -174,6 +190,9 @@ type granter interface { // when the goroutine doing the work noticed that it had been granted, there // is a possibility that that raced with cancellation. // + // Do not use this for doing store IO-related token adjustments when work is + // done -- that should be done via granterWithStoreWriteDone.storeWriteDone. + // // REQUIRES: count > 0. count == 1 for slots. returnGrant(count int64) // tookWithoutPermission informs the granter that a slot or tokens were @@ -188,6 +207,9 @@ type granter interface { // - Tokens: this is useful when the initial estimated tokens for a unit of // work turned out to be an underestimate. // + // Do not use this for doing store IO-related token adjustments when work is + // done -- that should be done via granterWithStoreWriteDone.storeWriteDone. + // // REQUIRES: count > 0. count == 1 for slots. tookWithoutPermission(count int64) // continueGrantChain is called by the requester at some point after grant @@ -346,31 +368,44 @@ const ( // grant because a shared resource (CPU or memory) is overloaded. For grant // chains, this is a signal to terminate. grantFailDueToSharedResource - // grantFailLocal is returned when the granter is unable to grant due to a - // local constraint -- insufficient tokens or slots. + // grantFailLocal is returned when the granter is unable to grant due to (a) + // a local constraint -- insufficient tokens or slots, or (b) no work is + // waiting. grantFailLocal ) -// granterWithLockedCalls is an extension of the granter and requester -// interfaces that is used as an internal implementation detail of the -// GrantCoordinator. Note that an implementer of granterWithLockedCalls is -// mainly passing things through to the GrantCoordinator where the main logic -// lives. The *Locked() methods are where the differences in slots and tokens -// are handled. +// granterWithLockedCalls is an encapsulation of typically one +// granter-requester pair, and for kvStoreTokenGranter of two +// granter-requester pairs (one for each workClass). It is used as an internal +// implementation detail of the GrantCoordinator. An implementer of +// granterWithLockedCalls responds to calls from its granter(s) by calling +// into the GrantCoordinator, which then calls the various *Locked() methods. +// The demuxHandle is meant to be opaque to the GrantCoordinator, and is used +// when this interface encapsulates multiple granter-requester pairs -- it is +// currently used only by kvStoreTokenGranter, where it is a workClass. The +// *Locked() methods are where the differences in slots and various kinds of +// tokens are handled. type granterWithLockedCalls interface { - granter - // tryGetLocked is the real implementation of tryGet in the granter interface. - // Additionally, it is also used when continuing a grant chain. - tryGetLocked(count int64) grantResult - // returnGrantLocked is the real implementation of returnGrant. - returnGrantLocked(count int64) + // tryGetLocked is the real implementation of tryGet from the granter + // interface. demuxHandle is an opaque handle that was passed into the + // GrantCoordinator. + tryGetLocked(count int64, demuxHandle int8) grantResult + // returnGrantLocked is the real implementation of returnGrant from the + // granter interface. demuxHandle is an opaque handle that was passed into + // the GrantCoordinator. + returnGrantLocked(count int64, demuxHandle int8) // tookWithoutPermissionLocked is the real implementation of - // tookWithoutPermission. - tookWithoutPermissionLocked(count int64) + // tookWithoutPermission from the granter interface. demuxHandle is an + // opaque handle that was passed into the GrantCoordinator. + tookWithoutPermissionLocked(count int64, demuxHandle int8) - // getPairedRequester returns the requester implementation that this granter - // interacts with. - getPairedRequester() requester + // The following methods are for direct use by GrantCoordinator. + + // requesterHasWaitingRequests returns whether some requester associated + // with the granter has waiting requests. + requesterHasWaitingRequests() bool + // tryGrantLocked is used to attempt to grant to waiting requests. + tryGrantLocked(grantChainID grantChainID) grantResult } // For the cpu-bound slot case we have background activities (like Pebble @@ -438,11 +473,7 @@ type slotGranter struct { } var _ granterWithLockedCalls = &slotGranter{} - -// getPairedRequester implements granterWithLockedCalls. -func (sg *slotGranter) getPairedRequester() requester { - return sg.requester -} +var _ granter = &slotGranter{} // grantKind implements granter. func (sg *slotGranter) grantKind() grantKind { @@ -451,11 +482,11 @@ func (sg *slotGranter) grantKind() grantKind { // tryGet implements granter. func (sg *slotGranter) tryGet(count int64) bool { - return sg.coord.tryGet(sg.workKind, count) + return sg.coord.tryGet(sg.workKind, count, 0 /*arbitrary*/) } // tryGetLocked implements granterWithLockedCalls. -func (sg *slotGranter) tryGetLocked(count int64) grantResult { +func (sg *slotGranter) tryGetLocked(count int64, _ int8) grantResult { if count != 1 { panic(errors.AssertionFailedf("unexpected count: %d", count)) } @@ -475,7 +506,7 @@ func (sg *slotGranter) tryGetLocked(count int64) grantResult { // returnGrant implements granter. func (sg *slotGranter) returnGrant(count int64) { - sg.coord.returnGrant(sg.workKind, count) + sg.coord.returnGrant(sg.workKind, count, 0 /*arbitrary*/) } func (sg *slotGranter) tryGetSoftSlots(count int) int { @@ -505,7 +536,7 @@ func (sg *slotGranter) returnSoftSlots(count int) { } // returnGrantLocked implements granterWithLockedCalls. -func (sg *slotGranter) returnGrantLocked(count int64) { +func (sg *slotGranter) returnGrantLocked(count int64, _ int8) { if count != 1 { panic(errors.AssertionFailedf("unexpected count: %d", count)) } @@ -518,11 +549,11 @@ func (sg *slotGranter) returnGrantLocked(count int64) { // tookWithoutPermission implements granter. func (sg *slotGranter) tookWithoutPermission(count int64) { - sg.coord.tookWithoutPermission(sg.workKind, count) + sg.coord.tookWithoutPermission(sg.workKind, count, 0 /*arbitrary*/) } // tookWithoutPermissionLocked implements granterWithLockedCalls. -func (sg *slotGranter) tookWithoutPermissionLocked(count int64) { +func (sg *slotGranter) tookWithoutPermissionLocked(count int64, _ int8) { if count != 1 { panic(errors.AssertionFailedf("unexpected count: %d", count)) } @@ -535,6 +566,27 @@ func (sg *slotGranter) continueGrantChain(grantChainID grantChainID) { sg.coord.continueGrantChain(sg.workKind, grantChainID) } +// requesterHasWaitingRequests implements granterWithLockedCalls. +func (sg *slotGranter) requesterHasWaitingRequests() bool { + return sg.requester.hasWaitingRequests() +} + +// tryGrantLocked implements granterWithLockedCalls. +func (sg *slotGranter) tryGrantLocked(grantChainID grantChainID) grantResult { + res := sg.tryGetLocked(1, 0 /*arbitrary*/) + if res == grantSuccess { + slots := sg.requester.granted(grantChainID) + if slots == 0 { + // Did not accept grant. + sg.returnGrantLocked(1, 0 /*arbitrary*/) + return grantFailLocal + } else if slots != 1 { + panic(errors.AssertionFailedf("unexpected count %d", slots)) + } + } + return res +} + // tokenGranter implements granterWithLockedCalls. type tokenGranter struct { coord *GrantCoordinator @@ -552,11 +604,7 @@ type tokenGranter struct { } var _ granterWithLockedCalls = &tokenGranter{} - -// getPairedRequester implements granterWithLockedCalls. -func (tg *tokenGranter) getPairedRequester() requester { - return tg.requester -} +var _ granter = &tokenGranter{} func (tg *tokenGranter) refillBurstTokens(skipTokenEnforcement bool) { tg.availableBurstTokens = tg.maxBurstTokens @@ -570,11 +618,11 @@ func (tg *tokenGranter) grantKind() grantKind { // tryGet implements granter. func (tg *tokenGranter) tryGet(count int64) bool { - return tg.coord.tryGet(tg.workKind, count) + return tg.coord.tryGet(tg.workKind, count, 0 /*arbitrary*/) } // tryGetLocked implements granterWithLockedCalls. -func (tg *tokenGranter) tryGetLocked(count int64) grantResult { +func (tg *tokenGranter) tryGetLocked(count int64, _ int8) grantResult { if tg.cpuOverload != nil && tg.cpuOverload.isOverloaded() { return grantFailDueToSharedResource } @@ -587,11 +635,11 @@ func (tg *tokenGranter) tryGetLocked(count int64) grantResult { // returnGrant implements granter. func (tg *tokenGranter) returnGrant(count int64) { - tg.coord.returnGrant(tg.workKind, count) + tg.coord.returnGrant(tg.workKind, count, 0 /*arbitrary*/) } // returnGrantLocked implements granterWithLockedCalls. -func (tg *tokenGranter) returnGrantLocked(count int64) { +func (tg *tokenGranter) returnGrantLocked(count int64, _ int8) { tg.availableBurstTokens += count if tg.availableBurstTokens > tg.maxBurstTokens { tg.availableBurstTokens = tg.maxBurstTokens @@ -600,11 +648,11 @@ func (tg *tokenGranter) returnGrantLocked(count int64) { // tookWithoutPermission implements granter. func (tg *tokenGranter) tookWithoutPermission(count int64) { - tg.coord.tookWithoutPermission(tg.workKind, count) + tg.coord.tookWithoutPermission(tg.workKind, count, 0 /*arbitrary*/) } // tookWithoutPermissionLocked implements granterWithLockedCalls. -func (tg *tokenGranter) tookWithoutPermissionLocked(count int64) { +func (tg *tokenGranter) tookWithoutPermissionLocked(count int64, _ int8) { tg.availableBurstTokens -= count } @@ -613,11 +661,60 @@ func (tg *tokenGranter) continueGrantChain(grantChainID grantChainID) { tg.coord.continueGrantChain(tg.workKind, grantChainID) } +// requesterHasWaitingRequests implements granterWithLockedCalls. +func (tg *tokenGranter) requesterHasWaitingRequests() bool { + return tg.requester.hasWaitingRequests() +} + +// tryGrantLocked implements granterWithLockedCalls. +func (tg *tokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult { + res := tg.tryGetLocked(1, 0 /*arbitrary*/) + if res == grantSuccess { + tokens := tg.requester.granted(grantChainID) + if tokens == 0 { + // Did not accept grant. + tg.returnGrantLocked(1, 0 /*arbitrary*/) + return grantFailLocal + } else if tokens > 1 { + tg.tookWithoutPermissionLocked(tokens-1, 0 /*arbitrary*/) + } + } + return res +} + +type workClass int8 + +const ( + // regularWorkClass is for work corresponding to workloads that are + // throughput and latency sensitive. + regularWorkClass workClass = iota + // elasticWorkClass is for work corresponding to workloads that can handle + // reduced throughput, possibly by taking longer to finish a workload. It is + // not latency sensitive. + elasticWorkClass + numWorkClasses +) + // kvStoreTokenGranter implements granterWithLockedCalls. It is used for -// grants to KVWork to a store, that is limited by IO tokens. +// grants to KVWork to a store, that is limited by IO tokens. It encapsulates +// two granter-requester pairs, for the two workClasses. The granter in these +// pairs is implemented by kvStoreTokenChildGranter, and the requester by +// WorkQueue. We have separate WorkQueues for these work classes so that we +// don't have a situation where tenant1's elastic work is queued ahead of +// tenant2's regular work (due to inter-tenant fairness) and blocks the latter +// from getting tokens, because elastic tokens are exhausted (and tokens for +// regular work are not exhausted). +// +// The kvStoreTokenChildGranters delegate the actual interaction to +// their "parent", kvStoreTokenGranter. For elasticWorkClass, multiple kinds +// of tokens need to be acquired, (a) the usual IO tokens (based on +// compactions out of L0 and flushes into L0) and (b) elastic disk bandwidth +// tokens, which are based on disk bandwidth as a constrained resource, and +// apply to all the elastic incoming bytes into the LSM. type kvStoreTokenGranter struct { - coord *GrantCoordinator - requester requester + coord *GrantCoordinator + regularRequester requester + elasticRequester requester // There is no rate limiting in granting these tokens. That is, they are all // burst tokens. availableIOTokens int64 @@ -628,55 +725,119 @@ type kvStoreTokenGranter struct { ioTokensExhaustedDurationMetric *metric.Counter exhaustedStart time.Time - writeLM, ingestedLM tokensLinearModel + // Disk bandwidth tokens. + elasticDiskBWTokensAvailable int64 + diskBWTokensUsed [numWorkClasses]int64 + + // Estimation models. + l0WriteLM, l0IngestLM, ingestLM tokensLinearModel } var _ granterWithLockedCalls = &kvStoreTokenGranter{} var _ granterWithIOTokens = &kvStoreTokenGranter{} -var _ granterWithStoreWriteDone = &kvStoreTokenGranter{} -// getPairedRequester implements granterWithLockedCalls. -func (sg *kvStoreTokenGranter) getPairedRequester() requester { - return sg.requester +// kvStoreTokenChildGranter handles a particular workClass. Its methods +// pass-through to the parent after adding the workClass as a parameter. +type kvStoreTokenChildGranter struct { + workClass workClass + parent *kvStoreTokenGranter } +var _ granterWithStoreWriteDone = &kvStoreTokenChildGranter{} +var _ granter = &kvStoreTokenChildGranter{} + // grantKind implements granter. -func (sg *kvStoreTokenGranter) grantKind() grantKind { +func (cg *kvStoreTokenChildGranter) grantKind() grantKind { return token } // tryGet implements granter. -func (sg *kvStoreTokenGranter) tryGet(count int64) bool { - return sg.coord.tryGet(KVWork, count) +func (cg *kvStoreTokenChildGranter) tryGet(count int64) bool { + return cg.parent.tryGet(cg.workClass, count) +} + +// returnGrant implements granter. +func (cg *kvStoreTokenChildGranter) returnGrant(count int64) { + cg.parent.returnGrant(cg.workClass, count) +} + +// tookWithoutPermission implements granter. +func (cg *kvStoreTokenChildGranter) tookWithoutPermission(count int64) { + cg.parent.tookWithoutPermission(cg.workClass, count) +} + +// continueGrantChain implements granter. +func (cg *kvStoreTokenChildGranter) continueGrantChain(grantChainID grantChainID) { + // Ignore since grant chains are not used for store tokens. +} + +// storeWriteDone implements granterWithStoreWriteDone. +func (cg *kvStoreTokenChildGranter) storeWriteDone( + originalTokens int64, doneInfo StoreWorkDoneInfo, +) (additionalTokens int64) { + return cg.parent.storeWriteDone(cg.workClass, originalTokens, doneInfo) +} + +func (sg *kvStoreTokenGranter) tryGet(workClass workClass, count int64) bool { + return sg.coord.tryGet(KVWork, count, int8(workClass)) } // tryGetLocked implements granterWithLockedCalls. -func (sg *kvStoreTokenGranter) tryGetLocked(count int64) grantResult { - if sg.availableIOTokens > 0 { - sg.subtractTokens(count, false) - return grantSuccess +func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grantResult { + wc := workClass(demuxHandle) + // NB: ideally if regularRequester.hasWaitingRequests() returns true and + // wc==elasticWorkClass we should reject this request, since it means that + // more important regular work is waiting. However, we rely on the + // assumption that elasticWorkClass, once throttled, will have a non-empty + // queue, and since the only case where tryGetLocked is called for + // elasticWorkClass is when the queue is empty, this case should be rare + // (and not cause a performance isolation failure). + switch wc { + case regularWorkClass: + if sg.availableIOTokens > 0 { + sg.subtractTokens(count, false) + sg.diskBWTokensUsed[wc] += count + return grantSuccess + } + case elasticWorkClass: + if sg.elasticDiskBWTokensAvailable > 0 && sg.availableIOTokens > 0 { + sg.elasticDiskBWTokensAvailable -= count + sg.subtractTokens(count, false) + sg.diskBWTokensUsed[wc] += count + return grantSuccess + } } return grantFailLocal } -// returnGrant implements granter. -func (sg *kvStoreTokenGranter) returnGrant(count int64) { - sg.coord.returnGrant(KVWork, count) +func (sg *kvStoreTokenGranter) returnGrant(workClass workClass, count int64) { + sg.coord.returnGrant(KVWork, count, int8(workClass)) } // returnGrantLocked implements granterWithLockedCalls. -func (sg *kvStoreTokenGranter) returnGrantLocked(count int64) { +func (sg *kvStoreTokenGranter) returnGrantLocked(count int64, demuxHandle int8) { + wc := workClass(demuxHandle) + // Return count tokens to the "IO tokens". sg.subtractTokens(-count, false) + if wc == elasticWorkClass { + // Return count tokens to the elastic disk bandwidth tokens. + sg.elasticDiskBWTokensAvailable += count + } + sg.diskBWTokensUsed[wc] -= count } -// tookWithoutPermission implements granter. -func (sg *kvStoreTokenGranter) tookWithoutPermission(count int64) { - sg.coord.tookWithoutPermission(KVWork, count) +func (sg *kvStoreTokenGranter) tookWithoutPermission(workClass workClass, count int64) { + sg.coord.tookWithoutPermission(KVWork, count, int8(workClass)) } // tookWithoutPermissionLocked implements granterWithLockedCalls. -func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64) { +func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64, demuxHandle int8) { + wc := workClass(demuxHandle) sg.subtractTokens(count, false) + if wc == elasticWorkClass { + sg.elasticDiskBWTokensAvailable -= count + } + sg.diskBWTokensUsed[wc] += count } // subtractTokens is a helper function that subtracts count tokens (count can @@ -701,9 +862,45 @@ func (sg *kvStoreTokenGranter) subtractTokens(count int64, forceTickMetric bool) } } -// continueGrantChain implements granter. -func (sg *kvStoreTokenGranter) continueGrantChain(grantChainID grantChainID) { - sg.coord.continueGrantChain(KVWork, grantChainID) +// requesterHasWaitingRequests implements granterWithLockedCalls. +func (sg *kvStoreTokenGranter) requesterHasWaitingRequests() bool { + return sg.regularRequester.hasWaitingRequests() || sg.elasticRequester.hasWaitingRequests() +} + +// tryGrantLocked implements granterWithLockedCalls. +func (sg *kvStoreTokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult { + // First try granting to regular requester. + for wc := range sg.diskBWTokensUsed { + req := sg.regularRequester + if workClass(wc) == elasticWorkClass { + req = sg.elasticRequester + } + if req.hasWaitingRequests() { + res := sg.tryGetLocked(1, int8(wc)) + if res == grantSuccess { + tookTokenCount := req.granted(grantChainID) + if tookTokenCount == 0 { + // Did not accept grant. + sg.returnGrantLocked(1, int8(wc)) + // Continue with the loop since this requester does not have waiting + // requests. If the loop terminates we will correctly return + // grantFailLocal. + } else { + // May have taken more. + if tookTokenCount > 1 { + sg.tookWithoutPermissionLocked(tookTokenCount-1, int8(wc)) + } + return grantSuccess + } + } else { + // Was not able to get token. Do not continue with looping to grant to + // less important work (though it would be harmless since won't be + // able to get a token for that either). + return res + } + } + } + return grantFailLocal } // setAvailableIOTokensLocked implements granterWithIOTokens. @@ -722,17 +919,36 @@ func (sg *kvStoreTokenGranter) setAvailableIOTokensLocked(tokens int64) (tokensU return tokensUsed } +// setAvailableElasticDiskBandwidthTokensLocked implements +// granterWithIOTokens. +func (sg *kvStoreTokenGranter) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { + sg.elasticDiskBWTokensAvailable += tokens + if sg.elasticDiskBWTokensAvailable > tokens { + sg.elasticDiskBWTokensAvailable = tokens + } +} + +// getDiskTokensUsedAndResetLocked implements granterWithIOTokens. +func (sg *kvStoreTokenGranter) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { + result := sg.diskBWTokensUsed + for i := range sg.diskBWTokensUsed { + sg.diskBWTokensUsed[i] = 0 + } + return result +} + // setAdmittedModelsLocked implements granterWithIOTokens. func (sg *kvStoreTokenGranter) setAdmittedDoneModelsLocked( - writeLM tokensLinearModel, ingestedLM tokensLinearModel, + l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, ) { - sg.writeLM = writeLM - sg.ingestedLM = ingestedLM + sg.l0WriteLM = l0WriteLM + sg.l0IngestLM = l0IngestLM + sg.ingestLM = ingestLM } // storeWriteDone implements granterWithStoreWriteDone. func (sg *kvStoreTokenGranter) storeWriteDone( - originalTokens int64, doneInfo StoreWorkDoneInfo, + wc workClass, originalTokens int64, doneInfo StoreWorkDoneInfo, ) (additionalTokens int64) { // Normally, we follow the structure of a foo() method calling into a foo() // method on the GrantCoordinator, which then calls fooLocked() on the @@ -751,22 +967,33 @@ func (sg *kvStoreTokenGranter) storeWriteDone( // Reminder: coord.mu protects the state in the kvStoreTokenGranter. sg.coord.mu.Lock() exhaustedFunc := func() bool { - return sg.availableIOTokens <= 0 + return sg.availableIOTokens <= 0 || + (wc == elasticWorkClass && sg.elasticDiskBWTokensAvailable <= 0) } wasExhausted := exhaustedFunc() - actualTokens := - int64(float64(doneInfo.WriteBytes)*sg.writeLM.multiplier) + sg.writeLM.constant + - int64(float64(doneInfo.IngestedBytes)*sg.ingestedLM.multiplier) + sg.ingestedLM.constant - additionalTokensNeeded := actualTokens - originalTokens - sg.subtractTokens(additionalTokensNeeded, false) - if additionalTokensNeeded < 0 { + actualL0WriteTokens := sg.l0WriteLM.applyLinearModel(doneInfo.WriteBytes) + actualL0IngestTokens := sg.l0IngestLM.applyLinearModel(doneInfo.IngestedBytes) + actualL0Tokens := actualL0WriteTokens + actualL0IngestTokens + additionalL0TokensNeeded := actualL0Tokens - originalTokens + sg.subtractTokens(additionalL0TokensNeeded, false) + actualIngestTokens := sg.ingestLM.applyLinearModel(doneInfo.IngestedBytes) + additionalDiskBWTokensNeeded := (actualL0WriteTokens + actualIngestTokens) - originalTokens + if wc == elasticWorkClass { + sg.elasticDiskBWTokensAvailable -= additionalDiskBWTokensNeeded + } + sg.diskBWTokensUsed[wc] += additionalDiskBWTokensNeeded + if additionalL0TokensNeeded < 0 || additionalDiskBWTokensNeeded < 0 { isExhausted := exhaustedFunc() if wasExhausted && !isExhausted { sg.coord.tryGrant() } } sg.coord.mu.Unlock() - return additionalTokensNeeded + // For multi-tenant fairness accounting, we choose to ignore disk bandwidth + // tokens. Ideally, we'd have multiple resource dimensions for the fairness + // decisions, but we don't necessarily need something more sophisticated + // like "Dominant Resource Fairness". + return additionalL0TokensNeeded } // GrantCoordinator is the top-level object that coordinates grants across @@ -788,8 +1015,9 @@ type GrantCoordinator struct { // NB: Some granters can be nil. granters [numWorkKinds]granterWithLockedCalls // The WorkQueues behaving as requesters in each granterWithLockedCalls. - // This is kept separately only to service GetWorkQueue calls. - queues [numWorkKinds]requester + // This is kept separately only to service GetWorkQueue calls and to call + // close(). + queues [numWorkKinds]requesterClose // The cpu fields can be nil, and the IO field can be nil, since a // GrantCoordinator typically handles one of these two resources. cpuOverloadIndicator cpuOverloadIndicator @@ -893,8 +1121,8 @@ type makeRequesterFunc func( opts workQueueOptions) requester type makeStoreRequesterFunc func( - _ log.AmbientContext, granter granterWithStoreWriteDone, settings *cluster.Settings, - opts workQueueOptions) storeRequester + _ log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone, + settings *cluster.Settings, opts workQueueOptions) storeRequester // NewGrantCoordinators constructs GrantCoordinators and WorkQueues for a // regular cluster node. Caller is responsible for hooking up @@ -950,8 +1178,9 @@ func NewGrantCoordinators( } kvSlotAdjuster.granter = kvg - coord.queues[KVWork] = makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork)) - kvg.requester = coord.queues[KVWork] + req := makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork)) + coord.queues[KVWork] = req + kvg.requester = req coord.granters[KVWork] = kvg tg := &tokenGranter{ @@ -961,9 +1190,10 @@ func NewGrantCoordinators( maxBurstTokens: opts.SQLKVResponseBurstTokens, cpuOverload: kvSlotAdjuster, } - coord.queues[SQLKVResponseWork] = makeRequester( + req = makeRequester( ambientCtx, SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork)) - tg.requester = coord.queues[SQLKVResponseWork] + coord.queues[SQLKVResponseWork] = req + tg.requester = req coord.granters[SQLKVResponseWork] = tg tg = &tokenGranter{ @@ -973,9 +1203,10 @@ func NewGrantCoordinators( maxBurstTokens: opts.SQLSQLResponseBurstTokens, cpuOverload: kvSlotAdjuster, } - coord.queues[SQLSQLResponseWork] = makeRequester(ambientCtx, + req = makeRequester(ambientCtx, SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork)) - tg.requester = coord.queues[SQLSQLResponseWork] + coord.queues[SQLSQLResponseWork] = req + tg.requester = req coord.granters[SQLSQLResponseWork] = tg sg := &slotGranter{ @@ -985,9 +1216,10 @@ func NewGrantCoordinators( cpuOverload: kvSlotAdjuster, usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } - coord.queues[SQLStatementLeafStartWork] = makeRequester(ambientCtx, + req = makeRequester(ambientCtx, SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) - sg.requester = coord.queues[SQLStatementLeafStartWork] + coord.queues[SQLStatementLeafStartWork] = req + sg.requester = req coord.granters[SQLStatementLeafStartWork] = sg sg = &slotGranter{ @@ -997,13 +1229,18 @@ func NewGrantCoordinators( cpuOverload: kvSlotAdjuster, usedSlotsMetric: metrics.SQLRootStartUsedSlots, } - coord.queues[SQLStatementRootStartWork] = makeRequester(ambientCtx, + req = makeRequester(ambientCtx, SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) - sg.requester = coord.queues[SQLStatementRootStartWork] + coord.queues[SQLStatementRootStartWork] = req + sg.requester = req coord.granters[SQLStatementRootStartWork] = sg metricStructs = appendMetricStructsForQueues(metricStructs, coord) + // TODO(sumeerbhola): these metrics are shared across all stores and all + // priorities across stores (even the coarser workClasses, which are a + // mapping from priority, share the same metrics). Fix this by adding + // labeled Prometheus metrics. storeWorkQueueMetrics := makeWorkQueueMetrics(string(workKindString(KVWork)) + "-stores") metricStructs = append(metricStructs, storeWorkQueueMetrics) makeStoreRequester := makeStoreWorkQueue @@ -1052,9 +1289,10 @@ func NewGrantCoordinatorSQL( maxBurstTokens: opts.SQLKVResponseBurstTokens, cpuOverload: sqlNodeCPU, } - coord.queues[SQLKVResponseWork] = makeRequester(ambientCtx, + req := makeRequester(ambientCtx, SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork)) - tg.requester = coord.queues[SQLKVResponseWork] + coord.queues[SQLKVResponseWork] = req + tg.requester = req coord.granters[SQLKVResponseWork] = tg tg = &tokenGranter{ @@ -1064,9 +1302,10 @@ func NewGrantCoordinatorSQL( maxBurstTokens: opts.SQLSQLResponseBurstTokens, cpuOverload: sqlNodeCPU, } - coord.queues[SQLSQLResponseWork] = makeRequester(ambientCtx, + req = makeRequester(ambientCtx, SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork)) - tg.requester = coord.queues[SQLSQLResponseWork] + coord.queues[SQLSQLResponseWork] = req + tg.requester = req coord.granters[SQLSQLResponseWork] = tg sg := &slotGranter{ @@ -1076,9 +1315,10 @@ func NewGrantCoordinatorSQL( cpuOverload: sqlNodeCPU, usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } - coord.queues[SQLStatementLeafStartWork] = makeRequester(ambientCtx, + req = makeRequester(ambientCtx, SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) - sg.requester = coord.queues[SQLStatementLeafStartWork] + coord.queues[SQLStatementLeafStartWork] = req + sg.requester = req coord.granters[SQLStatementLeafStartWork] = sg sg = &slotGranter{ @@ -1088,9 +1328,10 @@ func NewGrantCoordinatorSQL( cpuOverload: sqlNodeCPU, usedSlotsMetric: metrics.SQLRootStartUsedSlots, } - coord.queues[SQLStatementRootStartWork] = makeRequester(ambientCtx, + req = makeRequester(ambientCtx, SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) - sg.requester = coord.queues[SQLStatementRootStartWork] + coord.queues[SQLStatementRootStartWork] = req + sg.requester = req coord.granters[SQLStatementRootStartWork] = sg return coord, appendMetricStructsForQueues(metricStructs, coord) @@ -1192,14 +1433,14 @@ func (coord *GrantCoordinator) CPULoad(runnable int, procs int, samplePeriod tim } // tryGet is called by granter.tryGet with the WorkKind. -func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64) bool { +func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64, demuxHandle int8) bool { coord.mu.Lock() defer coord.mu.Unlock() // It is possible that a grant chain is active, and has not yet made its way // to this workKind. So it may be more reasonable to queue. But we have some // concerns about incurring the delay of multiple goroutine context switches // so we ignore this case. - res := coord.granters[workKind].tryGetLocked(count) + res := coord.granters[workKind].tryGetLocked(count, demuxHandle) switch res { case grantSuccess: // Grant chain may be active, but it did not get in the way of this grant, @@ -1222,13 +1463,13 @@ func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64) bool { } // returnGrant is called by granter.returnGrant with the WorkKind. -func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64) { +func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64, demuxHandle int8) { coord.mu.Lock() defer coord.mu.Unlock() - coord.granters[workKind].returnGrantLocked(count) + coord.granters[workKind].returnGrantLocked(count, demuxHandle) if coord.grantChainActive { if coord.grantChainIndex > workKind && - coord.granters[workKind].getPairedRequester().hasWaitingRequests() { + coord.granters[workKind].requesterHasWaitingRequests() { // There are waiting requests that will not be served by the grant chain. // Better to terminate it and start afresh. if !coord.tryTerminateGrantChain() { @@ -1244,10 +1485,12 @@ func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64) { // tookWithoutPermission is called by granter.tookWithoutPermission with the // WorkKind. -func (coord *GrantCoordinator) tookWithoutPermission(workKind WorkKind, count int64) { +func (coord *GrantCoordinator) tookWithoutPermission( + workKind WorkKind, count int64, demuxHandle int8, +) { coord.mu.Lock() defer coord.mu.Unlock() - coord.granters[workKind].tookWithoutPermissionLocked(count) + coord.granters[workKind].tookWithoutPermissionLocked(count, demuxHandle) } // continueGrantChain is called by granter.continueGrantChain with the @@ -1341,33 +1584,21 @@ OuterLoop: // remaining will be nil. continue } - req := granter.getPairedRequester() - for req.hasWaitingRequests() && !localDone { - // Get 1 token or slot. - res := granter.tryGetLocked(1) + for granter.requesterHasWaitingRequests() && !localDone { + chainID := noGrantChain + if grantBurstCount+1 == grantBurstLimit && coord.useGrantChains { + chainID = coord.grantChainID + } + res := granter.tryGrantLocked(chainID) switch res { case grantSuccess: - chainID := noGrantChain - if grantBurstCount+1 == grantBurstLimit && coord.useGrantChains { - chainID = coord.grantChainID - } - tookCount := req.granted(chainID) - if tookCount == 0 { - // Did not accept grant. - granter.returnGrantLocked(1) - } else { - // May have taken more. - if tookCount > 1 { - granter.tookWithoutPermissionLocked(tookCount - 1) - } - grantBurstCount++ - if grantBurstCount == grantBurstLimit && coord.useGrantChains { - coord.grantChainActive = true - if startingChain { - coord.grantChainStartTime = timeutil.Now() - } - return + grantBurstCount++ + if grantBurstCount == grantBurstLimit && coord.useGrantChains { + coord.grantChainActive = true + if startingChain { + coord.grantChainStartTime = timeutil.Now() } + return } case grantFailDueToSharedResource: break OuterLoop @@ -1426,7 +1657,8 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { s.Printf(" used-soft: %d", g.usedSoftSlots) } case *kvStoreTokenGranter: - s.Printf(" io-avail: %d", g.availableIOTokens) + s.Printf(" io-avail: %d, elastic-disk-bw-tokens-avail: %d", g.availableIOTokens, + g.elasticDiskBWTokensAvailable) } case SQLStatementLeafStartWork, SQLStatementRootStartWork: if coord.granters[i] != nil { @@ -1566,8 +1798,14 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo } kvg := &kvStoreTokenGranter{ - coord: coord, + coord: coord, + // Setting tokens to unlimited is defensive. We expect that + // pebbleMetricsTick and allocateIOTokensTick will get called during + // initialization, which will also set these to unlimited. + availableIOTokens: unlimitedTokens / ticksInAdjustmentInterval, + startingIOTokens: unlimitedTokens / ticksInAdjustmentInterval, ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration, + elasticDiskBWTokensAvailable: unlimitedTokens / ticksInAdjustmentInterval, } opts := makeWorkQueueOptions(KVWork) // This is IO work, so override the usesTokens value. @@ -1575,15 +1813,29 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo // Share the WorkQueue metrics across all stores. // TODO(sumeer): add per-store WorkQueue state for debug.zip and db console. opts.metrics = &sgc.workQueueMetrics - storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, kvg, sgc.settings, opts) + granters := [numWorkClasses]granterWithStoreWriteDone{ + &kvStoreTokenChildGranter{ + workClass: regularWorkClass, + parent: kvg, + }, + &kvStoreTokenChildGranter{ + workClass: elasticWorkClass, + parent: kvg, + }, + } + + storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, granters, sgc.settings, opts) coord.queues[KVWork] = storeReq - kvg.requester = storeReq + requesters := storeReq.getRequesters() + kvg.regularRequester = requesters[regularWorkClass] + kvg.elasticRequester = requesters[elasticWorkClass] coord.granters[KVWork] = kvg coord.ioLoadListener = &ioLoadListener{ storeID: storeID, settings: sgc.settings, kvRequester: storeReq, perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), + diskBandwidthLimiter: makeDiskBandwidthLimiter(), } coord.ioLoadListener.mu.Mutex = &coord.mu coord.ioLoadListener.mu.kvGranter = kvg @@ -1860,6 +2112,26 @@ type StoreMetrics struct { *pebble.Metrics WriteStallCount int64 *pebble.InternalIntervalMetrics + // Optional. + DiskStats DiskStats +} + +// DiskStats provide low-level stats about the disk resources used for a +// store. We assume that the disk is not shared across multiple stores. +// However, transient and moderate usage that is not due to the store is +// tolerable, since the diskBandwidthLimiter is only using this to compute +// elastic tokens and is designed to deal with significant attribution +// uncertainty. +// +// DiskStats are not always populated. A ProvisionedBandwidth of 0 represents +// that the stats should be ignored. +type DiskStats struct { + // BytesRead is the cumulative bytes read. + BytesRead uint64 + // BytesWritten is the cumulative bytes written. + BytesWritten uint64 + // ProvisionedBandwidth is the total provisioned bandwidth in bytes/s. + ProvisionedBandwidth int64 } // granterWithIOTokens is used to abstract kvStoreTokenGranter for testing. @@ -1876,6 +2148,13 @@ type granterWithIOTokens interface { // that tokensUsed can be negative, though that will be rare, since it is // possible for tokens to be returned. setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) + // setAvailableElasticDiskBandwidthTokensLocked bounds the available tokens + // that can be granted to elastic work. These tokens are based on disk + // bandwidth being a bottleneck resource. + setAvailableElasticDiskBandwidthTokensLocked(tokens int64) + // getDiskTokensUsedAndResetLocked returns the disk bandwidth tokens used + // since the last such call. + getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 // setAdmittedDoneModelsLocked supplies the models to use when // storeWriteDone is called, to adjust token consumption. Note that these // models are not used for token adjustment at admission time -- that is @@ -1883,7 +2162,8 @@ type granterWithIOTokens interface { // asymmetry is due to the need to use all the functionality of WorkQueue at // admission time. See the long explanatory comment at the beginning of // store_token_estimation.go, regarding token estimation. - setAdmittedDoneModelsLocked(writeLM tokensLinearModel, ingestedLM tokensLinearModel) + setAdmittedDoneModelsLocked(l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, + ingestLM tokensLinearModel) } // granterWithStoreWriteDone is used to abstract kvStoreTokenGranter for @@ -1941,7 +2221,8 @@ type storeRequestEstimates struct { // storeRequester is used to abstract *StoreWorkQueue for testing. type storeRequester interface { - requester + requesterClose + getRequesters() [numWorkClasses]requester getStoreAdmissionStats() storeAdmissionStats setStoreRequestEstimates(estimates storeRequestEstimates) } @@ -1953,6 +2234,12 @@ type ioLoadListenerState struct { curL0Bytes int64 // Cumulative. cumWriteStallCount int64 + diskBW struct { + // Cumulative + bytesRead uint64 + bytesWritten uint64 + incomingLSMBytes uint64 + } // Exponentially smoothed per interval values. @@ -1969,13 +2256,19 @@ type ioLoadListenerState struct { flushUtilTargetFraction float64 // totalNumByteTokens represents the tokens to give out until the next call to - // adjustTokens. They are parceled out in small intervals. tokensAllocated + // adjustTokens. They are parceled out in small intervals. byteTokensAllocated // represents what has been given out. - totalNumByteTokens int64 - tokensAllocated int64 + totalNumByteTokens int64 + byteTokensAllocated int64 // Used tokens can be negative if some tokens taken in one interval were // returned in another, but that will be extremely rare. - tokensUsed int64 + byteTokensUsed int64 + + // elasticDiskBWTokens represents the tokens to give out until the next call + // to adjustTokens. They are parceled out in small intervals. + // elasticDiskTokensAllocated represents what has been given out. + elasticDiskBWTokens int64 + elasticDiskBWTokensAllocated int64 } // ioLoadListener adjusts tokens in kvStoreTokenGranter for IO, specifically due to @@ -1983,6 +2276,52 @@ type ioLoadListenerState struct { // completion is not an indicator that the "resource usage" has ceased -- it // just means that the write has been applied to the WAL. Most of the work is // in flushing to sstables and the following compactions, which happens later. +// +// +// Token units are in bytes and used to protect a number of virtual or +// physical resource bottlenecks: +// - Compactions out of L0: compactions out of L0 can fall behind and cause +// too many sub-levels or files in L0. +// - Flushes into L0: flushes of memtables to L0 can fall behind and cause +// write stalls due to too many memtables. +// - Disk bandwidth: there is typically an aggregate read+write provisioned +// bandwidth, and if it is fully utilized, IO operations can start queueing +// and encounter high latency. +// +// For simplicity, after ioLoadListener computes the tokens due to compaction +// or flush bottlenecks, it computes the minimum and passes that value to +// granterWithIOTokens.setAvailableIOTokensLocked. That is, instead of working +// with multiple token dimensions, these two token dimensions get collapsed +// into one for enforcement purposes. This also helps simplify the integration +// with WorkQueue which is dealing with a single dimension. The consumption of +// these tokens is based on how many bytes an admitted work adds to L0. +// +// The disk bandwidth constraint is used to compute a token count for elastic +// work (see disk_bandwidth.go for the reasons why this is limited to elastic +// work). Additionally, these tokens are meant be consumed for all incoming +// bytes into the LSM, and not just those written to L0 e.g. ingested bytes +// into L3 should also consume such tokens. Note that we call these disk +// bandwidth tokens, but that is a misnomer -- these are tokens for incoming +// bytes into the LSM, motivated by disk bandwidth as a bottleneck resource, +// and not consumed for every write to the disk (e.g. by compactions). Since +// these tokens are consumed by all incoming bytes into the LSM, and not just +// those into L0, it suggests explicitly modeling this as a separate +// dimension. However, since modeling as a separate dimension everywhere would +// lead to code complexity, we adopt the following compromise: +// +// - Like the other token dimensions, ioLoadListener computes a different +// elastic token count (using diskBandwidthLimiter), and a different model +// for consumption (via +// storePerWorkTokenEstimator.atDoneDiskBWTokensLinearModel). +// +// - granterWithIOTokens, implemented by kvStoreTokenGranter, which enforces +// the token count, also treats this as a separate dimension. +// +// - WorkQueue works with a single dimension, so the tokens consumed at +// admission time are based on L0-bytes estimate. However, when +// StoreWorkQueue informs kvStoreTokenGranter of work completion (by calling +// storeWriteDone), the tokens are adjusted differently for the +// flush/compaction L0 tokens and for the "disk bandwidth" tokens. type ioLoadListener struct { storeID int32 settings *cluster.Settings @@ -1998,6 +2337,7 @@ type ioLoadListener struct { statsInitialized bool adjustTokensResult perWorkTokenEstimator storePerWorkTokenEstimator + diskBandwidthLimiter diskBandwidthLimiter } const unlimitedTokens = math.MaxInt64 @@ -2056,21 +2396,34 @@ const adjustmentInterval = 15 const ticksInAdjustmentInterval = 60 const ioTokenTickDuration = 250 * time.Millisecond +func cumLSMWriteAndIngestedBytes( + m *pebble.Metrics, +) (writeAndIngestedBytes uint64, ingestedBytes uint64) { + for i := range m.Levels { + writeAndIngestedBytes += m.Levels[i].BytesIngested + m.Levels[i].BytesFlushed + ingestedBytes += m.Levels[i].BytesIngested + } + return writeAndIngestedBytes, ingestedBytes +} + // pebbleMetricsTicks is called every adjustmentInterval seconds, and decides // the token allocations until the next call. func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMetrics) { + ctx = logtags.AddTag(ctx, "s", io.storeID) m := metrics.Metrics if !io.statsInitialized { io.statsInitialized = true sas := io.kvRequester.getStoreAdmissionStats() - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], sas) + cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) + io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) io.adjustTokensResult = adjustTokensResult{ ioLoadListenerState: ioLoadListenerState{ cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, curL0Bytes: m.Levels[0].Size, cumWriteStallCount: metrics.WriteStallCount, // No initial limit, i.e, the first interval is unlimited. - totalNumByteTokens: unlimitedTokens, + totalNumByteTokens: unlimitedTokens, + elasticDiskBWTokens: unlimitedTokens, }, aux: adjustTokensAuxComputations{}, ioThreshold: &admissionpb.IOThreshold{ @@ -2080,6 +2433,9 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe L0NumFilesThreshold: math.MaxInt64, }, } + io.diskBW.bytesRead = metrics.DiskStats.BytesRead + io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten + io.diskBW.incomingLSMBytes = cumLSMIncomingBytes io.copyAuxEtcFromPerWorkEstimator() return } @@ -2087,32 +2443,57 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe } // allocateTokensTick gives out 1/ticksInAdjustmentInterval of the -// totalNumByteTokens every 250ms. +// various tokens every 250ms. func (io *ioLoadListener) allocateTokensTick() { - var toAllocate int64 - // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up - // calculation. - if io.totalNumByteTokens >= unlimitedTokens-(ticksInAdjustmentInterval-1) { - toAllocate = io.totalNumByteTokens / ticksInAdjustmentInterval - } else { - // Round up so that we don't accumulate tokens to give in a burst on the - // last tick. - toAllocate = (io.totalNumByteTokens + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval - if toAllocate < 0 { - panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate)) - } - if toAllocate+io.tokensAllocated > io.totalNumByteTokens { - toAllocate = io.totalNumByteTokens - io.tokensAllocated + allocateFunc := func(total int64, allocated int64) (toAllocate int64) { + // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up + // calculation. + if total >= unlimitedTokens-(ticksInAdjustmentInterval-1) { + toAllocate = total / ticksInAdjustmentInterval + } else { + // Round up so that we don't accumulate tokens to give in a burst on the + // last tick. + toAllocate = (total + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval + if toAllocate < 0 { + panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate)) + } + if toAllocate+allocated > total { + toAllocate = total - allocated + } } + return toAllocate + } + // INVARIANT: toAllocate* >= 0. + toAllocateByteTokens := allocateFunc(io.totalNumByteTokens, io.byteTokensAllocated) + if toAllocateByteTokens < 0 { + panic(errors.AssertionFailedf("toAllocateByteTokens is negative %d", toAllocateByteTokens)) + } + toAllocateElasticDiskBWTokens := + allocateFunc(io.elasticDiskBWTokens, io.elasticDiskBWTokensAllocated) + if toAllocateElasticDiskBWTokens < 0 { + panic(errors.AssertionFailedf("toAllocateElasticDiskBWTokens is negative %d", + toAllocateElasticDiskBWTokens)) } // INVARIANT: toAllocate >= 0. io.mu.Lock() defer io.mu.Unlock() - io.tokensAllocated += toAllocate - if io.tokensAllocated < 0 { - panic(errors.AssertionFailedf("tokens allocated is negative %d", io.tokensAllocated)) + io.byteTokensAllocated += toAllocateByteTokens + if io.byteTokensAllocated < 0 { + panic(errors.AssertionFailedf("tokens allocated is negative %d", io.byteTokensAllocated)) + } + io.byteTokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocateByteTokens) + io.elasticDiskBWTokensAllocated += toAllocateElasticDiskBWTokens + io.mu.kvGranter.setAvailableElasticDiskBandwidthTokensLocked(toAllocateElasticDiskBWTokens) +} + +func computeIntervalDiskLoadInfo( + prevCumBytesRead uint64, prevCumBytesWritten uint64, diskStats DiskStats, +) intervalDiskLoadInfo { + return intervalDiskLoadInfo{ + readBandwidth: int64((diskStats.BytesRead - prevCumBytesRead) / adjustmentInterval), + writeBandwidth: int64((diskStats.BytesWritten - prevCumBytesWritten) / adjustmentInterval), + provisionedBandwidth: diskStats.ProvisionedBandwidth, } - io.tokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocate) } // adjustTokens computes a new value of totalNumByteTokens (and resets @@ -2133,17 +2514,43 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics MinFlushUtilizationFraction.Get(&io.settings.SV), ) io.adjustTokensResult = res - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], sas) + cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) + { + // Disk Bandwidth tokens. + io.aux.diskBW.intervalDiskLoadInfo = computeIntervalDiskLoadInfo( + io.diskBW.bytesRead, io.diskBW.bytesWritten, metrics.DiskStats) + io.mu.Lock() + diskTokensUsed := io.mu.kvGranter.getDiskTokensUsedAndResetLocked() + io.mu.Unlock() + io.aux.diskBW.intervalLSMInfo = intervalLSMInfo{ + incomingBytes: int64(cumLSMIncomingBytes) - int64(io.diskBW.incomingLSMBytes), + regularTokensUsed: diskTokensUsed[regularWorkClass], + elasticTokensUsed: diskTokensUsed[elasticWorkClass], + } + if metrics.DiskStats.ProvisionedBandwidth > 0 { + io.elasticDiskBWTokens = io.diskBandwidthLimiter.computeElasticTokens(ctx, + io.aux.diskBW.intervalDiskLoadInfo, io.aux.diskBW.intervalLSMInfo) + io.elasticDiskBWTokensAllocated = 0 + } + if metrics.DiskStats.ProvisionedBandwidth == 0 || + !DiskBandwidthTokensForElasticEnabled.Get(&io.settings.SV) { + io.elasticDiskBWTokens = unlimitedTokens + } + io.diskBW.bytesRead = metrics.DiskStats.BytesRead + io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten + io.diskBW.incomingLSMBytes = cumLSMIncomingBytes + } + io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) io.copyAuxEtcFromPerWorkEstimator() requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() io.kvRequester.setStoreRequestEstimates(requestEstimates) - writeLM, ingestedLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() + l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() io.mu.Lock() - io.mu.kvGranter.setAdmittedDoneModelsLocked(writeLM, ingestedLM) + io.mu.kvGranter.setAdmittedDoneModelsLocked(l0WriteLM, l0IngestLM, ingestLM) io.mu.Unlock() - if _, overloaded := res.ioThreshold.Score(); overloaded || res.aux.doLogFlush { - log.Infof(logtags.AddTag(ctx, "s", io.storeID), "IO overload: %s", - io.adjustTokensResult) + if _, overloaded := io.ioThreshold.Score(); overloaded || io.aux.doLogFlush || + io.elasticDiskBWTokens != unlimitedTokens { + log.Infof(ctx, "IO overload: %s", io.adjustTokensResult) } } @@ -2156,9 +2563,10 @@ func (io *ioLoadListener) copyAuxEtcFromPerWorkEstimator() { io.adjustTokensResult.aux.perWorkTokensAux = io.perWorkTokenEstimator.aux requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() io.adjustTokensResult.requestEstimates = requestEstimates - writeLM, ingestedLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() - io.adjustTokensResult.writeLM = writeLM - io.adjustTokensResult.ingestedLM = ingestedLM + l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone() + io.adjustTokensResult.l0WriteLM = l0WriteLM + io.adjustTokensResult.l0IngestLM = l0IngestLM + io.adjustTokensResult.ingestLM = ingestLM } type tokenKind int8 @@ -2183,8 +2591,15 @@ type adjustTokensAuxComputations struct { perWorkTokensAux perWorkTokensAux doLogFlush bool + + diskBW struct { + intervalDiskLoadInfo intervalDiskLoadInfo + intervalLSMInfo intervalLSMInfo + } } +// adjustTokensInner is used for computing tokens based on compaction and +// flush bottlenecks. func (*ioLoadListener) adjustTokensInner( ctx context.Context, prev ioLoadListenerState, @@ -2360,7 +2775,7 @@ func (*ioLoadListener) adjustTokensInner( // Have we used, over the last (15s) cycle, more than 90% of the tokens we // would give out for the next cycle? If yes, highTokenUsage is true. highTokenUsage := - float64(prev.tokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction + float64(prev.byteTokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction if intWriteStalls > 0 { // Try decrease since there were write-stalls. numDecreaseSteps := 1 @@ -2456,8 +2871,8 @@ func (*ioLoadListener) adjustTokensInner( smoothedNumFlushTokens: smoothedNumFlushTokens, flushUtilTargetFraction: flushUtilTargetFraction, totalNumByteTokens: totalNumByteTokens, - tokensAllocated: 0, - tokensUsed: 0, + byteTokensAllocated: 0, + byteTokensUsed: 0, }, aux: adjustTokensAuxComputations{ intL0AddedBytes: intL0AddedBytes, @@ -2465,7 +2880,7 @@ func (*ioLoadListener) adjustTokensInner( intFlushTokens: intFlushTokens, intFlushUtilization: intFlushUtilization, intWriteStalls: intWriteStalls, - prevTokensUsed: prev.tokensUsed, + prevTokensUsed: prev.byteTokensUsed, tokenKind: tokenKind, doLogFlush: doLogFlush, }, @@ -2477,8 +2892,9 @@ func (*ioLoadListener) adjustTokensInner( type adjustTokensResult struct { ioLoadListenerState requestEstimates storeRequestEstimates - writeLM tokensLinearModel - ingestedLM tokensLinearModel + l0WriteLM tokensLinearModel + l0IngestLM tokensLinearModel + ingestLM tokensLinearModel aux adjustTokensAuxComputations ioThreshold *admissionpb.IOThreshold // never nil } @@ -2508,18 +2924,18 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { ib(res.aux.perWorkTokensAux.intL0WriteBypassedAccountedBytes)) // Ingestion bytes that we expected because requests told admission control. p.Printf("%s acc-ingest (%s bypassed) + ", - ib(res.aux.perWorkTokensAux.intL0IngestedAccountedBytes), - ib(res.aux.perWorkTokensAux.intL0IngestedBypassedAccountedBytes)) + ib(res.aux.perWorkTokensAux.intIngestedAccountedBytes), + ib(res.aux.perWorkTokensAux.intIngestedBypassedAccountedBytes)) // The models we are fitting to compute tokens based on the reported size of // the write and ingest. p.Printf("write-model %.2fx+%s (smoothed %.2fx+%s) + ", - res.aux.perWorkTokensAux.intWriteLinearModel.multiplier, - ib(res.aux.perWorkTokensAux.intWriteLinearModel.constant), - res.writeLM.multiplier, ib(res.writeLM.constant)) + res.aux.perWorkTokensAux.intL0WriteLinearModel.multiplier, + ib(res.aux.perWorkTokensAux.intL0WriteLinearModel.constant), + res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant)) p.Printf("ingested-model %.2fx+%s (smoothed %.2fx+%s) + ", - res.aux.perWorkTokensAux.intIngestedLinearModel.multiplier, - ib(res.aux.perWorkTokensAux.intIngestedLinearModel.constant), - res.ingestedLM.multiplier, ib(res.ingestedLM.constant)) + res.aux.perWorkTokensAux.intL0IngestedLinearModel.multiplier, + ib(res.aux.perWorkTokensAux.intL0IngestedLinearModel.constant), + res.l0IngestLM.multiplier, ib(res.l0IngestLM.constant)) // The tokens used per request at admission time, when no size information // is known. p.Printf("at-admission-tokens %s, ", ib(res.requestEstimates.writeTokens)) @@ -2542,6 +2958,18 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { } else { p.SafeString("all") } + if res.elasticDiskBWTokens != unlimitedTokens { + p.Printf("; elastic tokens %s (used %s, regular used %s): "+ + "write model %.2fx+%s ingest model %.2fx+%s, ", + ib(res.elasticDiskBWTokens), ib(res.aux.diskBW.intervalLSMInfo.elasticTokensUsed), + ib(res.aux.diskBW.intervalLSMInfo.regularTokensUsed), + res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant), + res.ingestLM.multiplier, ib(res.ingestLM.constant)) + p.Printf("disk bw read %s write %s provisioned %s", + ib(res.aux.diskBW.intervalDiskLoadInfo.readBandwidth), + ib(res.aux.diskBW.intervalDiskLoadInfo.writeBandwidth), + ib(res.aux.diskBW.intervalDiskLoadInfo.provisionedBandwidth)) + } } func (res adjustTokensResult) String() string { @@ -2594,6 +3022,15 @@ var ( } ) +// TODO(irfansharif): we are lacking metrics for IO tokens and load, including +// metrics from helper classes used by ioLoadListener, like the code in +// disk_bandwidth.go and store_token_estimation.go. Additionally, what we have +// below is per node, while we want such metrics per store. We should add +// these metrics via StoreGrantCoordinators.SetPebbleMetricsProvider, which is +// used to construct the per-store GrantCoordinator. These metrics should be +// embedded in kvserver.StoreMetrics. We should also separate the metrics +// related to cpu slots from the IO metrics. + // GranterMetrics are metrics associated with a GrantCoordinator. type GranterMetrics struct { KVTotalSlots *metric.Gauge diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 146beef9ea61..4c5a607412ed 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -37,10 +37,11 @@ import ( ) type testRequester struct { - workKind WorkKind - granter granter - usesTokens bool - buf *strings.Builder + workKind WorkKind + additionalID string + granter granter + usesTokens bool + buf *strings.Builder waitingRequests bool returnValueFromGranted int64 @@ -48,14 +49,14 @@ type testRequester struct { } var _ requester = &testRequester{} -var _ storeRequester = &testRequester{} func (tr *testRequester) hasWaitingRequests() bool { return tr.waitingRequests } func (tr *testRequester) granted(grantChainID grantChainID) int64 { - fmt.Fprintf(tr.buf, "%s: granted in chain %d, and returning %d\n", workKindString(tr.workKind), + fmt.Fprintf(tr.buf, "%s%s: granted in chain %d, and returning %d\n", + workKindString(tr.workKind), tr.additionalID, grantChainID, tr.returnValueFromGranted) tr.grantChainID = grantChainID return tr.returnValueFromGranted @@ -65,30 +66,50 @@ func (tr *testRequester) close() {} func (tr *testRequester) tryGet(count int64) { rv := tr.granter.tryGet(count) - fmt.Fprintf(tr.buf, "%s: tryGet(%d) returned %t\n", workKindString(tr.workKind), count, rv) + fmt.Fprintf(tr.buf, "%s%s: tryGet(%d) returned %t\n", workKindString(tr.workKind), + tr.additionalID, count, rv) } func (tr *testRequester) returnGrant(count int64) { - fmt.Fprintf(tr.buf, "%s: returnGrant(%d)\n", workKindString(tr.workKind), count) + fmt.Fprintf(tr.buf, "%s%s: returnGrant(%d)\n", workKindString(tr.workKind), tr.additionalID, + count) tr.granter.returnGrant(count) } func (tr *testRequester) tookWithoutPermission(count int64) { - fmt.Fprintf(tr.buf, "%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind), count) + fmt.Fprintf(tr.buf, "%s%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind), + tr.additionalID, count) tr.granter.tookWithoutPermission(count) } func (tr *testRequester) continueGrantChain() { - fmt.Fprintf(tr.buf, "%s: continueGrantChain\n", workKindString(tr.workKind)) + fmt.Fprintf(tr.buf, "%s%s: continueGrantChain\n", workKindString(tr.workKind), + tr.additionalID) tr.granter.continueGrantChain(tr.grantChainID) } -func (tr *testRequester) getStoreAdmissionStats() storeAdmissionStats { +type storeTestRequester struct { + requesters [numWorkClasses]*testRequester +} + +var _ storeRequester = &storeTestRequester{} + +func (str *storeTestRequester) getRequesters() [numWorkClasses]requester { + var rv [numWorkClasses]requester + for i := range str.requesters { + rv[i] = str.requesters[i] + } + return rv +} + +func (str *storeTestRequester) close() {} + +func (str *storeTestRequester) getStoreAdmissionStats() storeAdmissionStats { // Only used by ioLoadListener, so don't bother. return storeAdmissionStats{} } -func (tr *testRequester) setStoreRequestEstimates(estimates storeRequestEstimates) { +func (str *storeTestRequester) setStoreRequestEstimates(estimates storeRequestEstimates) { // Only used by ioLoadListener, so don't bother. } @@ -119,7 +140,9 @@ func TestGranterBasic(t *testing.T) { defer log.Scope(t).Close(t) var ambientCtx log.AmbientContext - var requesters [numWorkKinds]*testRequester + // requesters[numWorkKinds] is used for kv elastic work, when working with a + // store grant coordinator. + var requesters [numWorkKinds + 1]*testRequester var coord *GrantCoordinator var ssg *SoftSlotGranter clearRequesterAndCoord := func() { @@ -188,16 +211,29 @@ func TestGranterBasic(t *testing.T) { storeCoordinators := &StoreGrantCoordinators{ settings: settings, makeStoreRequesterFunc: func( - ambientCtx log.AmbientContext, granter granterWithStoreWriteDone, + ambientCtx log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone, settings *cluster.Settings, opts workQueueOptions) storeRequester { - req := &testRequester{ - workKind: KVWork, - granter: granter, - usesTokens: true, - buf: &buf, - returnValueFromGranted: 0, + makeTestRequester := func(wc workClass) *testRequester { + req := &testRequester{ + workKind: KVWork, + granter: granters[wc], + usesTokens: true, + buf: &buf, + returnValueFromGranted: 0, + } + switch wc { + case regularWorkClass: + req.additionalID = "-regular" + case elasticWorkClass: + req.additionalID = "-elastic" + } + return req } - requesters[KVWork] = req + req := &storeTestRequester{} + req.requesters[regularWorkClass] = makeTestRequester(regularWorkClass) + req.requesters[elasticWorkClass] = makeTestRequester(elasticWorkClass) + requesters[KVWork] = req.requesters[regularWorkClass] + requesters[numWorkKinds] = req.requesters[elasticWorkClass] return req }, kvIOTokensExhaustedDuration: metrics.KVIOTokensExhaustedDuration, @@ -210,6 +246,12 @@ func TestGranterBasic(t *testing.T) { unsafeGranter, ok := storeCoordinators.gcMap.Load(int64(1)) require.True(t, ok) coord = (*GrantCoordinator)(unsafeGranter) + kvStoreGranter := coord.granters[KVWork].(*kvStoreTokenGranter) + // Use the same model for all 3 kinds of models. + tlm := tokensLinearModel{multiplier: 0.5, constant: 50} + coord.mu.Lock() + kvStoreGranter.setAdmittedDoneModelsLocked(tlm, tlm, tlm) + coord.mu.Unlock() return flushAndReset() case "set-has-waiting-requests": @@ -285,6 +327,27 @@ func TestGranterBasic(t *testing.T) { coord.testingTryGrant() return flushAndReset() + case "set-elastic-disk-bw-tokens": + var tokens int + d.ScanArgs(t, "tokens", &tokens) + // We are not using a real ioLoadListener, and simply setting the + // tokens (the ioLoadListener has its own test). + coord.mu.Lock() + coord.granters[KVWork].(*kvStoreTokenGranter).setAvailableElasticDiskBandwidthTokensLocked( + int64(tokens)) + coord.mu.Unlock() + coord.testingTryGrant() + return flushAndReset() + + case "store-write-done": + var origTokens, writeBytes int + d.ScanArgs(t, "orig-tokens", &origTokens) + d.ScanArgs(t, "write-bytes", &writeBytes) + requesters[scanWorkKind(t, d)].granter.(granterWithStoreWriteDone).storeWriteDone( + int64(origTokens), StoreWorkDoneInfo{WriteBytes: int64(writeBytes)}) + coord.testingTryGrant() + return flushAndReset() + case "try-get-soft-slots": var slots int d.ScanArgs(t, "slots", &slots) @@ -304,20 +367,22 @@ func TestGranterBasic(t *testing.T) { }) } -func scanWorkKind(t *testing.T, d *datadriven.TestData) WorkKind { +func scanWorkKind(t *testing.T, d *datadriven.TestData) int8 { var kindStr string d.ScanArgs(t, "work", &kindStr) switch kindStr { case "kv": - return KVWork + return int8(KVWork) case "sql-kv-response": - return SQLKVResponseWork + return int8(SQLKVResponseWork) case "sql-sql-response": - return SQLSQLResponseWork + return int8(SQLSQLResponseWork) case "sql-leaf-start": - return SQLStatementLeafStartWork + return int8(SQLStatementLeafStartWork) case "sql-root-start": - return SQLStatementRootStartWork + return int8(SQLStatementRootStartWork) + case "kv-elastic": + return int8(numWorkKinds) } panic("unknown WorkKind") } @@ -375,10 +440,16 @@ func TestStoreCoordinators(t *testing.T) { Settings: settings, makeRequesterFunc: makeRequesterFunc, makeStoreRequesterFunc: func( - ctx log.AmbientContext, granter granterWithStoreWriteDone, settings *cluster.Settings, - opts workQueueOptions) storeRequester { - req := makeRequesterFunc(ctx, KVWork, granter, settings, opts) - return req.(*testRequester) + ctx log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone, + settings *cluster.Settings, opts workQueueOptions) storeRequester { + reqReg := makeRequesterFunc(ctx, KVWork, granters[regularWorkClass], settings, opts) + reqElastic := makeRequesterFunc(ctx, KVWork, granters[elasticWorkClass], settings, opts) + str := &storeTestRequester{} + str.requesters[regularWorkClass] = reqReg.(*testRequester) + str.requesters[regularWorkClass].additionalID = "-regular" + str.requesters[elasticWorkClass] = reqElastic.(*testRequester) + str.requesters[elasticWorkClass].additionalID = "-elastic" + return str }, } coords, _ := NewGrantCoordinators(ambientCtx, opts) @@ -392,8 +463,8 @@ func TestStoreCoordinators(t *testing.T) { // Setting the metrics provider will cause the initialization of two // GrantCoordinators for the two stores. storeCoords.SetPebbleMetricsProvider(context.Background(), &mp, &mp) - // Now we have 1+2 = 3 KVWork requesters. - require.Equal(t, 3, len(requesters)) + // Now we have 1+2*2 = 5 KVWork requesters. + require.Equal(t, 5, len(requesters)) // Confirm that the store IDs are as expected. var actualStores []int32 @@ -415,7 +486,9 @@ func TestStoreCoordinators(t *testing.T) { requesters[i].tryGet(1) } require.Equal(t, - "kv: tryGet(1) returned false\nkv: tryGet(1) returned true\nkv: tryGet(1) returned true\n", + "kv: tryGet(1) returned false\n"+ + "kv-regular: tryGet(1) returned true\nkv-elastic: tryGet(1) returned true\n"+ + "kv-regular: tryGet(1) returned true\nkv-elastic: tryGet(1) returned true\n", buf.String()) coords.Close() } @@ -427,16 +500,12 @@ type testRequesterForIOLL struct { var _ storeRequester = &testRequesterForIOLL{} -func (r *testRequesterForIOLL) hasWaitingRequests() bool { - panic("unimplemented") -} +func (r *testRequesterForIOLL) close() {} -func (r *testRequesterForIOLL) granted(grantChainID grantChainID) int64 { +func (r *testRequesterForIOLL) getRequesters() [numWorkClasses]requester { panic("unimplemented") } -func (r *testRequesterForIOLL) close() {} - func (r *testRequesterForIOLL) getStoreAdmissionStats() storeAdmissionStats { return r.stats } @@ -446,8 +515,9 @@ func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEs } type testGranterWithIOTokens struct { - buf strings.Builder - allTokensUsed bool + buf strings.Builder + allTokensUsed bool + diskBandwidthTokensUsed [numWorkClasses]int64 } var _ granterWithIOTokens = &testGranterWithIOTokens{} @@ -460,13 +530,24 @@ func (g *testGranterWithIOTokens) setAvailableIOTokensLocked(tokens int64) (toke return 0 } +func (g *testGranterWithIOTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { + fmt.Fprintf(&g.buf, " setAvailableElasticDiskTokens: %s", + tokensForTokenTickDurationToString(tokens)) +} + +func (g *testGranterWithIOTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { + return g.diskBandwidthTokensUsed +} + func (g *testGranterWithIOTokens) setAdmittedDoneModelsLocked( - writeLM tokensLinearModel, ingestedLM tokensLinearModel, + l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, ) { - fmt.Fprintf(&g.buf, "setAdmittedDoneModelsLocked: write-lm: ") - printLinearModel(&g.buf, writeLM) - fmt.Fprintf(&g.buf, " ingested-lm: ") - printLinearModel(&g.buf, ingestedLM) + fmt.Fprintf(&g.buf, "setAdmittedDoneModelsLocked: l0-write-lm: ") + printLinearModel(&g.buf, l0WriteLM) + fmt.Fprintf(&g.buf, " l0-ingest-lm: ") + printLinearModel(&g.buf, l0IngestLM) + fmt.Fprintf(&g.buf, " ingest-lm: ") + printLinearModel(&g.buf, ingestLM) fmt.Fprintf(&g.buf, "\n") } @@ -497,6 +578,7 @@ func TestIOLoadListener(t *testing.T) { settings: st, kvRequester: req, perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), + diskBandwidthLimiter: makeDiskBandwidthLimiter(), } // The mutex is needed by ioLoadListener but is not useful in this // test -- the channels provide synchronization and prevent this @@ -527,6 +609,14 @@ func TestIOLoadListener(t *testing.T) { MinFlushUtilizationFraction.Override(ctx, &st.SV, float64(percent)/100) return "" + // TODO(sumeer): the output printed by set-state is hard to follow. It + // prints the internal fields which are hard to interpret, and it prints + // a properly formatted ioLoadListenerState. The latter is supposed to + // be easier to understand, but reviewers have noted that it is still + // challenging to understand whether the output is correct. Come up with + // more easily consumable output. Additionally, the input uses + // cumulative values, so one has to look at the preceding testdata -- we + // could instead accept the interval delta as input. case "set-state": // Setup state used as input for token adjustment. var metrics pebble.Metrics @@ -568,6 +658,25 @@ func TestIOLoadListener(t *testing.T) { d.ScanArgs(t, "all-tokens-used", &allTokensUsed) } kvGranter.allTokensUsed = allTokensUsed + var provisionedBandwidth, bytesRead, bytesWritten int + if d.HasArg("provisioned-bandwidth") { + d.ScanArgs(t, "provisioned-bandwidth", &provisionedBandwidth) + } + if d.HasArg("bytes-read") { + d.ScanArgs(t, "bytes-read", &bytesRead) + } + if d.HasArg("bytes-written") { + d.ScanArgs(t, "bytes-written", &bytesWritten) + } + if d.HasArg("disk-bw-tokens-used") { + var regularTokensUsed, elasticTokensUsed int + d.ScanArgs(t, "disk-bw-tokens-used", ®ularTokensUsed, &elasticTokensUsed) + kvGranter.diskBandwidthTokensUsed[regularWorkClass] = int64(regularTokensUsed) + kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = int64(elasticTokensUsed) + } else { + kvGranter.diskBandwidthTokensUsed[regularWorkClass] = 0 + kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = 0 + } var printOnlyFirstTick bool if d.HasArg("print-only-first-tick") { d.ScanArgs(t, "print-only-first-tick", &printOnlyFirstTick) @@ -576,6 +685,11 @@ func TestIOLoadListener(t *testing.T) { Metrics: &metrics, WriteStallCount: int64(writeStallCount), InternalIntervalMetrics: im, + DiskStats: DiskStats{ + BytesRead: uint64(bytesRead), + BytesWritten: uint64(bytesWritten), + ProvisionedBandwidth: int64(provisionedBandwidth), + }, }) var buf strings.Builder // Do the ticks until just before next adjustment. @@ -617,7 +731,7 @@ func TestIOLoadListenerOverflow(t *testing.T) { for i := int64(0); i < adjustmentInterval; i++ { // Override the totalNumByteTokens manually to trigger the overflow bug. ioll.totalNumByteTokens = math.MaxInt64 - i - ioll.tokensAllocated = 0 + ioll.byteTokensAllocated = 0 for j := 0; j < ticksInAdjustmentInterval; j++ { ioll.allocateTokensTick() } @@ -639,18 +753,30 @@ type testGranterNonNegativeTokens struct { t *testing.T } +var _ granterWithIOTokens = &testGranterNonNegativeTokens{} + func (g *testGranterNonNegativeTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { require.LessOrEqual(g.t, int64(0), tokens) return 0 } +func (g *testGranterNonNegativeTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) { + require.LessOrEqual(g.t, int64(0), tokens) +} + +func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 { + return [numWorkClasses]int64{} +} + func (g *testGranterNonNegativeTokens) setAdmittedDoneModelsLocked( - writeLM tokensLinearModel, ingestedLM tokensLinearModel, + l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, ) { - require.LessOrEqual(g.t, 0.5, writeLM.multiplier) - require.LessOrEqual(g.t, int64(0), writeLM.constant) - require.Less(g.t, 0.0, ingestedLM.multiplier) - require.LessOrEqual(g.t, int64(0), ingestedLM.constant) + require.LessOrEqual(g.t, 0.5, l0WriteLM.multiplier) + require.LessOrEqual(g.t, int64(0), l0WriteLM.constant) + require.Less(g.t, 0.0, l0IngestLM.multiplier) + require.LessOrEqual(g.t, int64(0), l0IngestLM.constant) + require.LessOrEqual(g.t, 0.5, ingestLM.multiplier) + require.LessOrEqual(g.t, int64(0), ingestLM.constant) } // TODO(sumeer): we now do more work outside adjustTokensInner, so the parts @@ -706,6 +832,7 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) { // stats and negative values) don't cause panics or tokens to be negative. func TestBadIOLoadListenerStats(t *testing.T) { var m pebble.Metrics + var d DiskStats req := &testRequesterForIOLL{} ctx := context.Background() @@ -715,10 +842,17 @@ func TestBadIOLoadListenerStats(t *testing.T) { m.Levels[0].NumFiles = int64(rand.Uint64()) m.Levels[0].Size = int64(rand.Uint64()) m.Levels[0].BytesFlushed = rand.Uint64() - m.Levels[0].BytesIngested = rand.Uint64() + for i := range m.Levels { + m.Levels[i].BytesIngested = rand.Uint64() + } + d.BytesRead = rand.Uint64() + d.BytesWritten = rand.Uint64() + d.ProvisionedBandwidth = 1 << 20 req.stats.admittedCount = rand.Uint64() req.stats.writeAccountedBytes = rand.Uint64() req.stats.ingestedAccountedBytes = rand.Uint64() + req.stats.statsToIgnore.Bytes = rand.Uint64() + req.stats.statsToIgnore.ApproxIngestedIntoL0Bytes = rand.Uint64() } kvGranter := &testGranterNonNegativeTokens{t: t} st := cluster.MakeTestingClusterSettings() @@ -726,6 +860,7 @@ func TestBadIOLoadListenerStats(t *testing.T) { settings: st, kvRequester: req, perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), + diskBandwidthLimiter: makeDiskBandwidthLimiter(), } ioll.mu.Mutex = &syncutil.Mutex{} ioll.mu.kvGranter = kvGranter @@ -734,6 +869,7 @@ func TestBadIOLoadListenerStats(t *testing.T) { ioll.pebbleMetricsTick(ctx, StoreMetrics{ Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}, + DiskStats: d, }) for j := 0; j < ticksInAdjustmentInterval; j++ { ioll.allocateTokensTick() @@ -742,7 +878,9 @@ func TestBadIOLoadListenerStats(t *testing.T) { require.LessOrEqual(t, float64(0), ioll.smoothedNumFlushTokens) require.LessOrEqual(t, float64(0), ioll.flushUtilTargetFraction) require.LessOrEqual(t, int64(0), ioll.totalNumByteTokens) - require.LessOrEqual(t, int64(0), ioll.tokensAllocated) + require.LessOrEqual(t, int64(0), ioll.byteTokensAllocated) + require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokens) + require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokensAllocated) } } } diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go index 41d92dce3f64..caad156e7f50 100644 --- a/pkg/util/admission/store_token_estimation.go +++ b/pkg/util/admission/store_token_estimation.go @@ -23,83 +23,87 @@ import "github.com/cockroachdb/pebble" // should rightfully consume, based on models that are "trained" on actual // resource consumption observed, and reported work sizes, in the past. // -// The token estimation is complicated by the fact that many writes do not -// integrate with admission control. Specifically, even if they did not ask to -// be admitted, it would be beneficial for them to provide the information at -// (b), so we could subtract tokens for the work they did. The list of -// significant non-integrated writes is: +// We use models in which the "actual tokens" are computed as a linear +// function of the bytes claimed in StoreWorkDoneInfo (which we call +// accounted_bytes), i.e. actual_tokens = a*accounted_bytes + b, and the result +// of this computation (which can only be done after completion of the work) +// is used to acquire (without blocking) additional tokens. The model thus +// tries to make sure that one token reflects one byte of work. For example, +// if the model is initially (a=1, b=0) and each incoming request acquires +// 1000 tokens but ends up writing 2000 bytes, the model should update to +// roughly (a=2, b=0), and future requests will, upon completion, acquire an +// additional 1000 tokens to even the score. The actual fitting is performed +// on aggregates over recent requests, and the more work is done "outside" of +// admission control, the less useful the multiplier becomes; the model will +// degrade into one with a larger constant term and is expected to perform +// poorly. // -// 1. Range snapshot application (does ingestion). +// We now justify the use of a linear model. A model with only a constant term +// (the b term above) is not able to handle multiple simultaneous workloads +// executing on a node, since they can have very different work sizes. We +// desire workload agnostic token estimation so a mix of various workloads can +// share the same token estimation model. A model with +// actual_tokens=accounted_bytes is also not suitable for 2 reasons: +// - For writes (that are accomplished via the memtable) we only have the size +// of the raft log entry in accounted_bytes and not the size of the later +// state machine application. // -// 2. State machine application for regular writes: StoreWorkDoneInfo, -// WriteBytes only accounts for the write to the raft log. +// - For ingests, we also need to fit a model where accounted_bytes is the +// size of the ingest, and actual_tokens is the size that landed in L0. // -// 3. Follower writes, to the raft log and to the state machine, both regular -// and ingests. +// We note that a multiplier term (the a term above) can accomplish both goals. +// The first bullet can be handled by a multiplier that is approximately 2. +// Ingests have some fraction that get ingested into L0, i.e., a multiplier +// <= 1. // -// Over time, we should change the code to perform step (b) for (1) and for -// the raft log for (3). This will require noticing under raft that a write is -// at the leaseholder, so that we don't account for the raft log write a -// second time (since it has already been accounted for at proposal -// evaluation). The current code here is designed to fit that future world, -// while also attempting to deal with the limitations of this lack of -// integration. +// One complication with ingests is range snapshot application. They happen +// infrequently and can add a very large number of bytes, which are often +// ingested below L0. We don't want to skew our ingest models based on these +// range snapshots, so we explicitly ignore them in modeling. // -// Specifically, in such a future world, even though we have the raft log -// writes for regular writes, and ingest bytes for ingestion requests, we have -// a mix of workloads that are concurrently doing writes. These could be -// multiple SQL-originating workloads with different size writes, and bulk -// workloads that are doing index backfills. We desire workload agnostic token -// estimation so a mix of various workloads can share the same token -// estimation model. +// So now that we've justified the a term, one question arises is whether we +// need a b term. Historically we have had sources of error that are due to +// lack of integration with admission control, and we do not want to skew the +// a term significantly. So the fitting approach has a b term, but attempts to +// minimize the b term while keeping the a term within some configured bounds. +// The [min,max] bounds on the a term prevent wild fluctuations and are set +// based on what we know about the system. // -// We also have the immediate practical requirement that if there are bytes -// added to L0 (based on Pebble stats) that are unaccounted for by -// StoreWorkDoneInfo, we should compensate for that by adjusting our -// estimates. This may lead to per-work over-estimation, but that is better -// than an unhealthy LSM. A consequence of this is that we are willing to add -// tokens based on unaccounted write bytes to work that only did ingestion and -// willing to add tokens based on unaccounted ingested bytes to work that only -// did regular writes. +// The estimation of a and b is done by tokensLinearModelFitter. It is used +// to fit 3 models. +// - [l0WriteLM] Mapping the write accounted bytes to bytes added to L0: We +// expect the multiplier a to be close to 2, due to the subsequent +// application to the state machine. So it would be reasonable to constrain +// a to [1, 2]. However, in experiments we've seen inconsistencies between +// Pebble stats and admission control stats, due to choppiness in work +// getting done, which is better modeled by allowing multiplier a to be less +// constrained. So we use [0.5, 3]. // -// We observe: -// - The lack of integration of state machine application can be mostly -// handled by a multiplier on the bytes written to the raft log. Say a -// multiplier of ~2. -// -// - We expect that most range snapshot applications will ingest into levels -// below L0, so if we limit our attention to Pebble stats relating to -// ingestion into L0, we may not see the effect of these unaccounted bytes, -// which will result in more accurate estimation. -// -// - Ingests have some fraction that get ingested into L0, i.e., a multiplier -// <= 1. -// -// Based on these observations we adopt a linear model for estimating the -// actual bytes (y), given the accounted bytes (x): y = a.x + b. The -// estimation of a and b is done by tokensLinearModelFitter. We constrain the -// interval of a to be [min,max] to prevent wild fluctuations and to account -// for what we know about the system: -// - For writes, we expect the multiplier a to be close to 2, due to the -// subsequent application to the state machine. So it would be reasonable to -// constrain a to [1, 2]. However, in experiments we've seen inconsistencies -// between Pebble stats and admission control stats, due to choppiness in -// work getting done, which is better modeled by allowing multiplier a to be -// less constrained. So we use [0.5, 3]. -// -// - For ingests, we expect the multiplier a to be <= 1, since some fraction -// of the ingest goes into L0. So it would be reasonable to constrain a to -// [0, 1]. For the same reason as the previous bullet, we use [0.001, 1.5]. -// This lower-bound of 0.001 is debatable, since it will cause some token -// consumption even if all ingested bytes are going to levels below L0. +// - [l0IngestLM] Mapping the ingest accounted bytes (which is the total bytes +// in the ingest, and not just to L0), to the bytes added to L0: We expect +// the multiplier a to be <= 1, since some fraction of the ingest goes into +// L0. So it would be reasonable to constrain a to [0, 1]. For the same +// reason as the previous bullet, we use [0.001, 1.5]. This lower-bound of +// 0.001 is debatable, since it will cause some token consumption even if +// all ingested bytes are going to levels below L0. // TODO(sumeer): consider lowering the lower bound, after experimentation. // -// In both the writes and ingests case, y is the bytes being added to L0. +// - [ingestLM] Mapping the ingest accounted bytes to the total ingested bytes +// added to the LSM. We can expect a multiplier of 1. For now, we use bounds +// of [0.5, 1.5]. // // NB: these linear models will be workload agnostic if most of the bytes are // modeled via the a.x term, and not via the b term, since workloads are // likely (at least for regular writes) to vary significantly in x. +// See the comment above for the justification of these constants. +const l0WriteMultiplierMin = 0.5 +const l0WriteMultiplierMax = 3.0 +const l0IngestMultiplierMin = 0.001 +const l0IngestMultiplierMax = 1.5 +const ingestMultiplierMin = 0.5 +const ingestMultiplierMax = 1.5 + // tokensLinearModel represents a model y = multiplier.x + constant. type tokensLinearModel struct { multiplier float64 @@ -107,6 +111,10 @@ type tokensLinearModel struct { constant int64 } +func (m tokensLinearModel) applyLinearModel(b int64) int64 { + return int64(float64(b)*m.multiplier) + m.constant +} + // tokensLinearModelFitter fits y = multiplier.x + constant, based on the // current interval and then exponentially smooths the multiplier and // constant. @@ -129,14 +137,14 @@ type tokensLinearModelFitter struct { smoothedLinearModel tokensLinearModel smoothedPerWorkAccountedBytes int64 - // Should be set to true for the ingested bytes model: if all bytes are + // Should be set to true for the L0 ingested bytes model: if all bytes are // ingested below L0, the actual bytes will be zero and the accounted bytes // non-zero. We need to update the model in this case. - updateWithZeroActualNonZeroAccountedForIngestedModel bool + updateWithZeroActualNonZeroAccountedForL0IngestedModel bool } func makeTokensLinearModelFitter( - multMin float64, multMax float64, updateWithZeroActualNonZeroAccountedForIngestedModel bool, + multMin float64, multMax float64, updateWithZeroActualNonZeroAccountedForL0IngestedModel bool, ) tokensLinearModelFitter { return tokensLinearModelFitter{ multiplierMin: multMin, @@ -145,8 +153,8 @@ func makeTokensLinearModelFitter( multiplier: (multMin + multMax) / 2, constant: 1, }, - smoothedPerWorkAccountedBytes: 1, - updateWithZeroActualNonZeroAccountedForIngestedModel: updateWithZeroActualNonZeroAccountedForIngestedModel, + smoothedPerWorkAccountedBytes: 1, + updateWithZeroActualNonZeroAccountedForL0IngestedModel: updateWithZeroActualNonZeroAccountedForL0IngestedModel, } } @@ -178,7 +186,7 @@ func (f *tokensLinearModelFitter) updateModelUsingIntervalStats( accountedBytes int64, actualBytes int64, workCount int64, ) { if workCount <= 1 || (actualBytes <= 0 && - (!f.updateWithZeroActualNonZeroAccountedForIngestedModel || accountedBytes <= 0)) { + (!f.updateWithZeroActualNonZeroAccountedForL0IngestedModel || accountedBytes <= 0)) { // Don't want to update the model if workCount is very low or actual bytes // is zero (except for the exceptions in the if-condition above). // @@ -249,13 +257,17 @@ func (f *tokensLinearModelFitter) updateModelUsingIntervalStats( } type storePerWorkTokenEstimator struct { - atAdmissionWorkTokens int64 - atDoneWriteTokensLinearModel tokensLinearModelFitter + atAdmissionWorkTokens int64 + atDoneL0WriteTokensLinearModel tokensLinearModelFitter + atDoneL0IngestTokensLinearModel tokensLinearModelFitter + // Unlike the models above that model bytes into L0, this model computes all + // ingested bytes into the LSM. atDoneIngestTokensLinearModel tokensLinearModelFitter cumStoreAdmissionStats storeAdmissionStats cumL0WriteBytes uint64 cumL0IngestedBytes uint64 + cumLSMIngestedBytes uint64 // Tracked for logging and copied out of here. aux perWorkTokensAux @@ -264,41 +276,49 @@ type storePerWorkTokenEstimator struct { // perWorkTokensAux encapsulates auxiliary (informative) numerical state that // helps in understanding the behavior of storePerWorkTokenEstimator. type perWorkTokensAux struct { - intWorkCount int64 - intL0WriteBytes int64 - intL0IngestedBytes int64 - intL0WriteAccountedBytes int64 - intL0IngestedAccountedBytes int64 - intWriteLinearModel tokensLinearModel - intIngestedLinearModel tokensLinearModel + intWorkCount int64 + intL0WriteBytes int64 + intL0IngestedBytes int64 + intLSMIngestedBytes int64 + intL0WriteAccountedBytes int64 + intIngestedAccountedBytes int64 + intL0WriteLinearModel tokensLinearModel + intL0IngestedLinearModel tokensLinearModel + intIngestedLinearModel tokensLinearModel // The bypassed count and bytes are also included in the overall interval // stats. - intBypassedWorkCount int64 - intL0WriteBypassedAccountedBytes int64 - intL0IngestedBypassedAccountedBytes int64 + intBypassedWorkCount int64 + intL0WriteBypassedAccountedBytes int64 + intIngestedBypassedAccountedBytes int64 - // The ignored bytes are included in intL0IngestedBytes, and may even be - // higher than that value because these are from a different source. + // The ignored bytes are included in intL0IngestedBytes, and in + // intLSMWriteAndIngestedBytes, and may even be higher than that value + // because these are from a different source. intL0IgnoredIngestedBytes int64 } func makeStorePerWorkTokenEstimator() storePerWorkTokenEstimator { return storePerWorkTokenEstimator{ - atAdmissionWorkTokens: 1, - atDoneWriteTokensLinearModel: makeTokensLinearModelFitter(0.5, 3, false), - atDoneIngestTokensLinearModel: makeTokensLinearModelFitter(0.001, 1.5, true), + atAdmissionWorkTokens: 1, + atDoneL0WriteTokensLinearModel: makeTokensLinearModelFitter( + l0WriteMultiplierMin, l0WriteMultiplierMax, false), + atDoneL0IngestTokensLinearModel: makeTokensLinearModelFitter( + l0IngestMultiplierMin, l0IngestMultiplierMax, true), + atDoneIngestTokensLinearModel: makeTokensLinearModelFitter( + ingestMultiplierMin, ingestMultiplierMax, false), } } // NB: first call to updateEstimates only initializes the cumulative values. func (e *storePerWorkTokenEstimator) updateEstimates( - l0Metrics pebble.LevelMetrics, admissionStats storeAdmissionStats, + l0Metrics pebble.LevelMetrics, cumLSMIngestedBytes uint64, admissionStats storeAdmissionStats, ) { if e.cumL0WriteBytes == 0 { e.cumStoreAdmissionStats = admissionStats e.cumL0WriteBytes = l0Metrics.BytesFlushed e.cumL0IngestedBytes = l0Metrics.BytesIngested + e.cumLSMIngestedBytes = cumLSMIngestedBytes return } intL0WriteBytes := int64(l0Metrics.BytesFlushed) - int64(e.cumL0WriteBytes) @@ -313,14 +333,24 @@ func (e *storePerWorkTokenEstimator) updateEstimates( int64(e.cumStoreAdmissionStats.admittedCount) intL0WriteAccountedBytes := int64(admissionStats.writeAccountedBytes) - int64(e.cumStoreAdmissionStats.writeAccountedBytes) - // Note that these are not really L0 ingested bytes, since we don't know how + // Note that these are not L0 ingested bytes, since we don't know how // many did go to L0. - intL0IngestedAccountedBytes := int64(admissionStats.ingestedAccountedBytes) - + intIngestedAccountedBytes := int64(admissionStats.ingestedAccountedBytes) - int64(e.cumStoreAdmissionStats.ingestedAccountedBytes) - e.atDoneWriteTokensLinearModel.updateModelUsingIntervalStats( + e.atDoneL0WriteTokensLinearModel.updateModelUsingIntervalStats( intL0WriteAccountedBytes, intL0WriteBytes, intWorkCount) + e.atDoneL0IngestTokensLinearModel.updateModelUsingIntervalStats( + intIngestedAccountedBytes, adjustedIntL0IngestedBytes, intWorkCount) + // Ingest across all levels model. + intLSMIngestedBytes := int64(cumLSMIngestedBytes) - int64(e.cumLSMIngestedBytes) + intIgnoredIngestedBytes := + int64(admissionStats.statsToIgnore.Bytes) - int64(e.cumStoreAdmissionStats.statsToIgnore.Bytes) + adjustedIntLSMIngestedBytes := intLSMIngestedBytes - intIgnoredIngestedBytes + if adjustedIntLSMIngestedBytes < 0 { + adjustedIntLSMIngestedBytes = 0 + } e.atDoneIngestTokensLinearModel.updateModelUsingIntervalStats( - intL0IngestedAccountedBytes, adjustedIntL0IngestedBytes, intWorkCount) + intIngestedAccountedBytes, adjustedIntLSMIngestedBytes, intWorkCount) intL0TotalBytes := intL0WriteBytes + adjustedIntL0IngestedBytes if intWorkCount > 1 && intL0TotalBytes > 0 { @@ -332,18 +362,20 @@ func (e *storePerWorkTokenEstimator) updateEstimates( e.atAdmissionWorkTokens = max(1, e.atAdmissionWorkTokens) } e.aux = perWorkTokensAux{ - intWorkCount: intWorkCount, - intL0WriteBytes: intL0WriteBytes, - intL0IngestedBytes: intL0IngestedBytes, - intL0WriteAccountedBytes: intL0WriteAccountedBytes, - intL0IngestedAccountedBytes: intL0IngestedAccountedBytes, - intWriteLinearModel: e.atDoneWriteTokensLinearModel.intLinearModel, - intIngestedLinearModel: e.atDoneIngestTokensLinearModel.intLinearModel, + intWorkCount: intWorkCount, + intL0WriteBytes: intL0WriteBytes, + intL0IngestedBytes: intL0IngestedBytes, + intLSMIngestedBytes: intLSMIngestedBytes, + intL0WriteAccountedBytes: intL0WriteAccountedBytes, + intIngestedAccountedBytes: intIngestedAccountedBytes, + intL0WriteLinearModel: e.atDoneL0WriteTokensLinearModel.intLinearModel, + intL0IngestedLinearModel: e.atDoneL0IngestTokensLinearModel.intLinearModel, + intIngestedLinearModel: e.atDoneIngestTokensLinearModel.intLinearModel, intBypassedWorkCount: int64(admissionStats.aux.bypassedCount) - int64(e.cumStoreAdmissionStats.aux.bypassedCount), intL0WriteBypassedAccountedBytes: int64(admissionStats.aux.writeBypassedAccountedBytes) - int64(e.cumStoreAdmissionStats.aux.writeBypassedAccountedBytes), - intL0IngestedBypassedAccountedBytes: int64(admissionStats.aux.ingestedBypassedAccountedBytes) - + intIngestedBypassedAccountedBytes: int64(admissionStats.aux.ingestedBypassedAccountedBytes) - int64(e.cumStoreAdmissionStats.aux.ingestedBypassedAccountedBytes), intL0IgnoredIngestedBytes: intL0IgnoredIngestedBytes, } @@ -351,6 +383,7 @@ func (e *storePerWorkTokenEstimator) updateEstimates( e.cumStoreAdmissionStats = admissionStats e.cumL0WriteBytes = l0Metrics.BytesFlushed e.cumL0IngestedBytes = l0Metrics.BytesIngested + e.cumLSMIngestedBytes = cumLSMIngestedBytes } func (e *storePerWorkTokenEstimator) getStoreRequestEstimatesAtAdmission() storeRequestEstimates { @@ -358,9 +391,11 @@ func (e *storePerWorkTokenEstimator) getStoreRequestEstimatesAtAdmission() store } func (e *storePerWorkTokenEstimator) getModelsAtAdmittedDone() ( - writeLM tokensLinearModel, - ingestedLM tokensLinearModel, + l0WriteLM tokensLinearModel, + l0IngestLM tokensLinearModel, + ingestLM tokensLinearModel, ) { - return e.atDoneWriteTokensLinearModel.smoothedLinearModel, + return e.atDoneL0WriteTokensLinearModel.smoothedLinearModel, + e.atDoneL0IngestTokensLinearModel.smoothedLinearModel, e.atDoneIngestTokensLinearModel.smoothedLinearModel } diff --git a/pkg/util/admission/store_token_estimation_test.go b/pkg/util/admission/store_token_estimation_test.go index 0115688a36f8..958da9b7b992 100644 --- a/pkg/util/admission/store_token_estimation_test.go +++ b/pkg/util/admission/store_token_estimation_test.go @@ -89,6 +89,7 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { var estimator storePerWorkTokenEstimator var l0Metrics pebble.LevelMetrics var admissionStats storeAdmissionStats + var cumLSMIngestedBytes uint64 datadriven.RunTest(t, testutils.TestDataPath(t, "store_per_work_token_estimator"), func(t *testing.T, d *datadriven.TestData) string { @@ -106,6 +107,12 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { d.ScanArgs(t, "ingested", &intIngested) l0Metrics.BytesFlushed += intFlushed l0Metrics.BytesIngested += intIngested + cumLSMIngestedBytes += intIngested + if d.HasArg("other-levels-ingested") { + var otherLevelsIngested uint64 + d.ScanArgs(t, "other-levels-ingested", &otherLevelsIngested) + cumLSMIngestedBytes += otherLevelsIngested + } var admitted, writeAccounted, ingestedAccounted uint64 d.ScanArgs(t, "admitted", &admitted) d.ScanArgs(t, "write-accounted", &writeAccounted) @@ -126,17 +133,21 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { var ignoreIngestedIntoL0 int d.ScanArgs(t, "ignore-ingested-into-L0", &ignoreIngestedIntoL0) admissionStats.statsToIgnore.ApproxIngestedIntoL0Bytes += uint64(ignoreIngestedIntoL0) + admissionStats.statsToIgnore.Bytes += uint64(ignoreIngestedIntoL0) } - estimator.updateEstimates(l0Metrics, admissionStats) - wlm, ilm := estimator.getModelsAtAdmittedDone() - require.Equal(t, wlm, estimator.atDoneWriteTokensLinearModel.smoothedLinearModel) + estimator.updateEstimates(l0Metrics, cumLSMIngestedBytes, admissionStats) + wL0lm, iL0lm, ilm := estimator.getModelsAtAdmittedDone() + require.Equal(t, wL0lm, estimator.atDoneL0WriteTokensLinearModel.smoothedLinearModel) + require.Equal(t, iL0lm, estimator.atDoneL0IngestTokensLinearModel.smoothedLinearModel) require.Equal(t, ilm, estimator.atDoneIngestTokensLinearModel.smoothedLinearModel) var b strings.Builder fmt.Fprintf(&b, "interval state: %+v\n", estimator.aux) fmt.Fprintf(&b, "at-admission-tokens: %d\n", estimator.getStoreRequestEstimatesAtAdmission().writeTokens) - fmt.Fprintf(&b, "write-tokens: ") - printLinearModelFitter(&b, estimator.atDoneWriteTokensLinearModel) + fmt.Fprintf(&b, "L0-write-tokens: ") + printLinearModelFitter(&b, estimator.atDoneL0WriteTokensLinearModel) + fmt.Fprintf(&b, "L0-ingest-tokens: ") + printLinearModelFitter(&b, estimator.atDoneL0IngestTokensLinearModel) fmt.Fprintf(&b, "ingest-tokens: ") printLinearModelFitter(&b, estimator.atDoneIngestTokensLinearModel) return b.String() diff --git a/pkg/util/admission/testdata/disk_bandwidth_limiter b/pkg/util/admission/testdata/disk_bandwidth_limiter new file mode 100644 index 000000000000..011f295f7afc --- /dev/null +++ b/pkg/util/admission/testdata/disk_bandwidth_limiter @@ -0,0 +1,41 @@ +init +---- +diskBandwidthLimiter low (disk bandwidth: read: 0 B/s, write: 0 B/s, provisioned: 0 B/s, util: 0.00): elastic-frac: 0.00, incoming: 0 B, elastic-tokens (used 0 B): 8.0 EiB + +# Elastic fraction for the interval is 0.25, and exponential smoothing causes +# it to rise to 0.12. elastic-tokens is 22 since the max logic picks the +# interval incoming bytes and a multiplier of elastic-fr+0.1, which means +# 100*(0.12+0.1)=22 +compute read-bw=100 write-bw=300 provisioned-bw=1000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=10 +---- +diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 1000 B/s, util: 0.40): elastic-frac: 0.12, incoming: 50 B, elastic-tokens (used 10 B): 22 B + +# Repeat. No change in elastic tokens since the 22B of elastic tokens were +# underutilized. +compute read-bw=100 write-bw=300 provisioned-bw=1000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=10 +---- +diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 1000 B/s, util: 0.40): elastic-frac: 0.19, incoming: 75 B, elastic-tokens (used 10 B): 22 B + +compute read-bw=100 write-bw=300 provisioned-bw=1000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=21 +---- +diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 1000 B/s, util: 0.40): elastic-frac: 0.30, incoming: 87 B, elastic-tokens (used 21 B): 39 B + +# No change in elastic tokens because again underutilized. +compute read-bw=100 write-bw=300 provisioned-bw=2000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=21 +---- +diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 2.0 KiB/s, util: 0.20): elastic-frac: 0.36, incoming: 93 B, elastic-tokens (used 21 B): 39 B + +# Disk load is low, so unlimited tokens. +compute read-bw=100 write-bw=300 provisioned-bw=2000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=21 +---- +diskBandwidthLimiter low (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 2.0 KiB/s, util: 0.20): elastic-frac: 0.38, incoming: 96 B, elastic-tokens (used 21 B): 8.0 EiB + +# Overloaded. Note that the min logic makes the elastic tokens much lower than 1000*0.5=500. +compute read-bw=100 write-bw=2000 provisioned-bw=2000 incoming-bytes=1000 regular-tokens-used=30 elastic-tokens-used=1000 +---- +diskBandwidthLimiter overload (disk bandwidth: read: 100 B/s, write: 2.0 KiB/s, provisioned: 2.0 KiB/s, util: 1.05): elastic-frac: 0.68, incoming: 548 B, elastic-tokens (used 1000 B): 185 B + +# Disk load is high, so no change to tokens. +compute read-bw=100 write-bw=1600 provisioned-bw=2000 incoming-bytes=1000 regular-tokens-used=30 elastic-tokens-used=1000 +---- +diskBandwidthLimiter high (disk bandwidth: read: 100 B/s, write: 1.6 KiB/s, provisioned: 2.0 KiB/s, util: 0.85): elastic-frac: 0.82, incoming: 774 B, elastic-tokens (used 1000 B): 185 B diff --git a/pkg/util/admission/testdata/disk_load_watcher b/pkg/util/admission/testdata/disk_load_watcher new file mode 100644 index 000000000000..d83bc88d4c42 --- /dev/null +++ b/pkg/util/admission/testdata/disk_load_watcher @@ -0,0 +1,78 @@ +init +---- +disk bandwidth: read: 0 B/s, write: 0 B/s, provisioned: 0 B/s, util: 0.00 +load-level: low + +# 40% utilization means moderate load. +interval-info read-bw=10 write-bw=10 provisioned-bw=50 +---- +disk bandwidth: read: 10 B/s, write: 10 B/s, provisioned: 50 B/s, util: 0.40 +load-level: moderate + +# 15% utilization ought to be low load, but we wait until one more round of +# low load before making the change. +interval-info read-bw=10 write-bw=20 provisioned-bw=200 +---- +disk bandwidth: read: 10 B/s, write: 20 B/s, provisioned: 200 B/s, util: 0.15 +load-level: moderate + +# Same 15% utilization, and now it is low load. +interval-info read-bw=10 write-bw=20 provisioned-bw=200 +---- +disk bandwidth: read: 10 B/s, write: 20 B/s, provisioned: 200 B/s, util: 0.15 +load-level: low + +# 95% utilization, so overloaded. +interval-info read-bw=90 write-bw=100 provisioned-bw=200 +---- +disk bandwidth: read: 90 B/s, write: 100 B/s, provisioned: 200 B/s, util: 0.95 +load-level: overload + +# Utilization can exceed 100%. +interval-info read-bw=190 write-bw=100 provisioned-bw=200 +---- +disk bandwidth: read: 190 B/s, write: 100 B/s, provisioned: 200 B/s, util: 1.45 +load-level: overload + +# Utilization exceeds 100% but decreased sufficiently to count as high load. +interval-info read-bw=150 write-bw=100 provisioned-bw=200 +---- +disk bandwidth: read: 150 B/s, write: 100 B/s, provisioned: 200 B/s, util: 1.25 +load-level: high + +# Same utilization, and since did not decrease, it is considered overloaded +# again. +interval-info read-bw=150 write-bw=100 provisioned-bw=200 +---- +disk bandwidth: read: 150 B/s, write: 100 B/s, provisioned: 200 B/s, util: 1.25 +load-level: overload + +# Small decrease and still above 95% so overloaded. +interval-info read-bw=150 write-bw=92 provisioned-bw=200 +---- +disk bandwidth: read: 150 B/s, write: 92 B/s, provisioned: 200 B/s, util: 1.21 +load-level: overload + +# Larger decrease, so even though above 95% it is high load. +interval-info read-bw=150 write-bw=81 provisioned-bw=200 +---- +disk bandwidth: read: 150 B/s, write: 81 B/s, provisioned: 200 B/s, util: 1.16 +load-level: high + +# Utilization < 70% so moderate load. +interval-info read-bw=120 write-bw=19 provisioned-bw=200 +---- +disk bandwidth: read: 120 B/s, write: 19 B/s, provisioned: 200 B/s, util: 0.69 +load-level: moderate + +# 85% utilization, so high load. +interval-info read-bw=70 write-bw=100 provisioned-bw=200 +---- +disk bandwidth: read: 70 B/s, write: 100 B/s, provisioned: 200 B/s, util: 0.85 +load-level: high + +# 68% utilization so considered moderate load. +interval-info read-bw=35 write-bw=100 provisioned-bw=200 +---- +disk bandwidth: read: 35 B/s, write: 100 B/s, provisioned: 200 B/s, util: 0.68 +load-level: moderate diff --git a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt index d3e334a9d2e0..d2d851f3bb39 100644 --- a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt +++ b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt @@ -1,6 +1,6 @@ echo ---- zero: -compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all +compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all; elastic tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B real-numbers: -compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) due to L0 growth (used 0 B) +compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) due to L0 growth (used 0 B); elastic tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter index c37f76c0619f..e63a02650886 100644 --- a/pkg/util/admission/testdata/granter +++ b/pkg/util/admission/testdata/granter @@ -378,89 +378,213 @@ sql-sql-response: avail: -1 sql-leaf-start: used: 0, total: 2 sql-root-start: us init-store-grant-coordinator ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 153722867280912930 +(chain: id: 0 active: false index: 5) io-avail: 153722867280912930, elastic-disk-bw-tokens-avail: 153722867280912930 + +# Set tokens to a large value that permits all request sizes in this file. +set-io-tokens tokens=100000 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 100000, elastic-disk-bw-tokens-avail: 153722867280912930 + +# Set elastic tokens to a large value that permits all request sizes in this +# file. +set-elastic-disk-bw-tokens tokens=100000 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 100000, elastic-disk-bw-tokens-avail: 100000 # Initial tokens are effectively unlimited. try-get work=kv v=10000 ---- -kv: tryGet(10000) returned true +kv-regular: tryGet(10000) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 153722867280902930 +(chain: id: 0 active: false index: 5) io-avail: 90000, elastic-disk-bw-tokens-avail: 100000 # Set the io tokens to a smaller value. set-io-tokens tokens=500 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 500 +(chain: id: 0 active: false index: 5) io-avail: 500, elastic-disk-bw-tokens-avail: 100000 -# Subtract 100 tokens. -took-without-permission work=kv v=100 +# Subtract 100 tokens for elastic work. Note that elastic-disk-bw-tokens-avail also decreases by 100. +took-without-permission work=kv-elastic v=100 ---- -kv: tookWithoutPermission(100) +kv-elastic: tookWithoutPermission(100) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 400 +(chain: id: 0 active: false index: 5) io-avail: 400, elastic-disk-bw-tokens-avail: 99900 # Add 200 tokens. return-grant work=kv v=200 ---- -kv: returnGrant(200) +kv-regular: returnGrant(200) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 600 +(chain: id: 0 active: false index: 5) io-avail: 600, elastic-disk-bw-tokens-avail: 99900 # Setup waiting requests that want 400 tokens each. set-has-waiting-requests work=kv v=true ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 600 +(chain: id: 0 active: false index: 5) io-avail: 600, elastic-disk-bw-tokens-avail: 99900 set-return-value-from-granted work=kv v=400 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 600 +(chain: id: 0 active: false index: 5) io-avail: 600, elastic-disk-bw-tokens-avail: 99900 # Returning tokens triggers granting and 2 requests will be granted until the # tokens become <= 0. return-grant work=kv v=100 ---- -kv: returnGrant(100) -kv: granted in chain 0, and returning 400 -kv: granted in chain 0, and returning 400 +kv-regular: returnGrant(100) +kv-regular: granted in chain 0, and returning 400 +kv-regular: granted in chain 0, and returning 400 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: -100, elastic-disk-bw-tokens-avail: 99900 set-return-value-from-granted work=kv v=100 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: -100, elastic-disk-bw-tokens-avail: 99900 # No tokens to give. try-get work=kv ---- -kv: tryGet(1) returned false +kv-regular: tryGet(1) returned false GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: -100, elastic-disk-bw-tokens-avail: 99900 # Increment by 50 tokens. set-io-tokens tokens=50 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -50 +(chain: id: 0 active: false index: 5) io-avail: -50, elastic-disk-bw-tokens-avail: 99900 # Return another 50 tokens. Since the number of tokens is 0, there is no # grant. return-grant work=kv v=50 ---- -kv: returnGrant(50) +kv-regular: returnGrant(50) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 0 +(chain: id: 0 active: false index: 5) io-avail: 0, elastic-disk-bw-tokens-avail: 99900 # As soon as the tokens > 0, it will grant. return-grant work=kv v=1 ---- -kv: returnGrant(1) -kv: granted in chain 0, and returning 100 +kv-regular: returnGrant(1) +kv-regular: granted in chain 0, and returning 100 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 99900 + +# Have waiting requests for kv-elastic too. +set-has-waiting-requests work=kv-elastic v=true +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 99900 + +# The kv-elastic waiting requests need 200 tokens each. +set-return-value-from-granted work=kv-elastic v=200 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 99900 + +# Since there are regular requests waiting, those are granted first. +return-grant work=kv-elastic v=400 +---- +kv-elastic: returnGrant(400) +kv-regular: granted in chain 0, and returning 100 +kv-regular: granted in chain 0, and returning 100 +kv-regular: granted in chain 0, and returning 100 +kv-regular: granted in chain 0, and returning 100 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 100300 + +# No more regular requests waiting. +set-has-waiting-requests work=kv v=false +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 100300 + +# kv-elastic is granted. +set-io-tokens tokens=100 +---- +kv-elastic: granted in chain 0, and returning 200 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -199, elastic-disk-bw-tokens-avail: 100100 + +set-elastic-disk-bw-tokens tokens=50 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -199, elastic-disk-bw-tokens-avail: 50 + +# Both kinds of tokens are decremented and become negative. +set-io-tokens tokens=200 +---- +kv-elastic: granted in chain 0, and returning 200 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: -199, elastic-disk-bw-tokens-avail: -150 + +# IO tokens become positive. But no grant to elastic work since +# elastic-disk-bw tokens are negative. +set-io-tokens tokens=300 +---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99 +(chain: id: 0 active: false index: 5) io-avail: 101, elastic-disk-bw-tokens-avail: -150 + +# Regular kv work can get tokens. +try-get work=kv v=10 +---- +kv-regular: tryGet(10) returned true +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 91, elastic-disk-bw-tokens-avail: -150 + +# Elastic kv work cannot get tokens. +try-get work=kv-elastic v=10 +---- +kv-elastic: tryGet(10) returned false +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 91, elastic-disk-bw-tokens-avail: -150 + +# Still negative. +set-elastic-disk-bw-tokens tokens=50 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 91, elastic-disk-bw-tokens-avail: -100 + +# Even more IO tokens. +set-io-tokens tokens=400 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 400, elastic-disk-bw-tokens-avail: -100 + +# Finally both tokens are positive and we grant until the elastic-disk-bw +# tokens become negative. +set-elastic-disk-bw-tokens tokens=120 +---- +kv-elastic: granted in chain 0, and returning 200 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 200, elastic-disk-bw-tokens-avail: -180 + +# Note that TestGranterBasic hard-codes the models to be 0.5x+50, so +# 0.5*40+50=70. So 70-10=60 additional tokens are needed based on the write +# model. We also apply the ingest model so 0.5*0+50=50, which means a total of +# 60+50 additional tokens are needed, hence the decrease by 110 of both +# io-avail and elastic-disk-bw-tokens-avail. +store-write-done work=kv-elastic orig-tokens=10 write-bytes=40 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 90, elastic-disk-bw-tokens-avail: -290 + +store-write-done work=kv orig-tokens=400 write-bytes=40 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 370, elastic-disk-bw-tokens-avail: -290 + +# Both tokens become positive, since 280 tokens are returned, so one work is granted. +store-write-done work=kv-elastic orig-tokens=400 write-bytes=40 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 650, elastic-disk-bw-tokens-avail: -10 + ##################################################################### # Test soft slots diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index da00f07bc167..85a0d1c44cd3 100644 --- a/pkg/util/admission/testdata/io_load_listener +++ b/pkg/util/admission/testdata/io_load_listener @@ -12,67 +12,67 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} -tick: 0, setAvailableIOTokens: unlimited -tick: 1, setAvailableIOTokens: unlimited -tick: 2, setAvailableIOTokens: unlimited -tick: 3, setAvailableIOTokens: unlimited -tick: 4, setAvailableIOTokens: unlimited -tick: 5, setAvailableIOTokens: unlimited -tick: 6, setAvailableIOTokens: unlimited -tick: 7, setAvailableIOTokens: unlimited -tick: 8, setAvailableIOTokens: unlimited -tick: 9, setAvailableIOTokens: unlimited -tick: 10, setAvailableIOTokens: unlimited -tick: 11, setAvailableIOTokens: unlimited -tick: 12, setAvailableIOTokens: unlimited -tick: 13, setAvailableIOTokens: unlimited -tick: 14, setAvailableIOTokens: unlimited -tick: 15, setAvailableIOTokens: unlimited -tick: 16, setAvailableIOTokens: unlimited -tick: 17, setAvailableIOTokens: unlimited -tick: 18, setAvailableIOTokens: unlimited -tick: 19, setAvailableIOTokens: unlimited -tick: 20, setAvailableIOTokens: unlimited -tick: 21, setAvailableIOTokens: unlimited -tick: 22, setAvailableIOTokens: unlimited -tick: 23, setAvailableIOTokens: unlimited -tick: 24, setAvailableIOTokens: unlimited -tick: 25, setAvailableIOTokens: unlimited -tick: 26, setAvailableIOTokens: unlimited -tick: 27, setAvailableIOTokens: unlimited -tick: 28, setAvailableIOTokens: unlimited -tick: 29, setAvailableIOTokens: unlimited -tick: 30, setAvailableIOTokens: unlimited -tick: 31, setAvailableIOTokens: unlimited -tick: 32, setAvailableIOTokens: unlimited -tick: 33, setAvailableIOTokens: unlimited -tick: 34, setAvailableIOTokens: unlimited -tick: 35, setAvailableIOTokens: unlimited -tick: 36, setAvailableIOTokens: unlimited -tick: 37, setAvailableIOTokens: unlimited -tick: 38, setAvailableIOTokens: unlimited -tick: 39, setAvailableIOTokens: unlimited -tick: 40, setAvailableIOTokens: unlimited -tick: 41, setAvailableIOTokens: unlimited -tick: 42, setAvailableIOTokens: unlimited -tick: 43, setAvailableIOTokens: unlimited -tick: 44, setAvailableIOTokens: unlimited -tick: 45, setAvailableIOTokens: unlimited -tick: 46, setAvailableIOTokens: unlimited -tick: 47, setAvailableIOTokens: unlimited -tick: 48, setAvailableIOTokens: unlimited -tick: 49, setAvailableIOTokens: unlimited -tick: 50, setAvailableIOTokens: unlimited -tick: 51, setAvailableIOTokens: unlimited -tick: 52, setAvailableIOTokens: unlimited -tick: 53, setAvailableIOTokens: unlimited -tick: 54, setAvailableIOTokens: unlimited -tick: 55, setAvailableIOTokens: unlimited -tick: 56, setAvailableIOTokens: unlimited -tick: 57, setAvailableIOTokens: unlimited -tick: 58, setAvailableIOTokens: unlimited -tick: 59, setAvailableIOTokens: unlimited +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 1, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 2, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 3, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 4, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 5, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 6, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 7, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 8, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 9, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 10, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 11, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 12, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 13, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 14, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 15, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 16, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 17, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 18, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 19, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 20, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 21, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 22, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 23, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 24, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 25, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 26, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 27, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 28, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 29, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 30, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 31, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 32, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 33, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 34, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 35, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 36, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 37, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 38, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 39, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 40, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 41, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 42, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 43, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 44, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 45, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 46, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 47, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 48, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 49, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 50, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 51, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 52, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 53, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 54, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 55, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 56, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 57, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 58, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited +tick: 59, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited prep-admission-stats admitted=10000 write-bytes=40000 ---- @@ -85,69 +85,69 @@ prep-admission-stats admitted=10000 write-bytes=40000 set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 0 B [≈0 B]; admitting 12 KiB (rate 833 B/s) due to L0 growth (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:12500 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:5} writeLM:{multiplier:2 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intL0WriteAccountedBytes:40000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:2.25 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:101000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 5 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.00x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 209 -tick: 1, setAvailableIOTokens: 209 -tick: 2, setAvailableIOTokens: 209 -tick: 3, setAvailableIOTokens: 209 -tick: 4, setAvailableIOTokens: 209 -tick: 5, setAvailableIOTokens: 209 -tick: 6, setAvailableIOTokens: 209 -tick: 7, setAvailableIOTokens: 209 -tick: 8, setAvailableIOTokens: 209 -tick: 9, setAvailableIOTokens: 209 -tick: 10, setAvailableIOTokens: 209 -tick: 11, setAvailableIOTokens: 209 -tick: 12, setAvailableIOTokens: 209 -tick: 13, setAvailableIOTokens: 209 -tick: 14, setAvailableIOTokens: 209 -tick: 15, setAvailableIOTokens: 209 -tick: 16, setAvailableIOTokens: 209 -tick: 17, setAvailableIOTokens: 209 -tick: 18, setAvailableIOTokens: 209 -tick: 19, setAvailableIOTokens: 209 -tick: 20, setAvailableIOTokens: 209 -tick: 21, setAvailableIOTokens: 209 -tick: 22, setAvailableIOTokens: 209 -tick: 23, setAvailableIOTokens: 209 -tick: 24, setAvailableIOTokens: 209 -tick: 25, setAvailableIOTokens: 209 -tick: 26, setAvailableIOTokens: 209 -tick: 27, setAvailableIOTokens: 209 -tick: 28, setAvailableIOTokens: 209 -tick: 29, setAvailableIOTokens: 209 -tick: 30, setAvailableIOTokens: 209 -tick: 31, setAvailableIOTokens: 209 -tick: 32, setAvailableIOTokens: 209 -tick: 33, setAvailableIOTokens: 209 -tick: 34, setAvailableIOTokens: 209 -tick: 35, setAvailableIOTokens: 209 -tick: 36, setAvailableIOTokens: 209 -tick: 37, setAvailableIOTokens: 209 -tick: 38, setAvailableIOTokens: 209 -tick: 39, setAvailableIOTokens: 209 -tick: 40, setAvailableIOTokens: 209 -tick: 41, setAvailableIOTokens: 209 -tick: 42, setAvailableIOTokens: 209 -tick: 43, setAvailableIOTokens: 209 -tick: 44, setAvailableIOTokens: 209 -tick: 45, setAvailableIOTokens: 209 -tick: 46, setAvailableIOTokens: 209 -tick: 47, setAvailableIOTokens: 209 -tick: 48, setAvailableIOTokens: 209 -tick: 49, setAvailableIOTokens: 209 -tick: 50, setAvailableIOTokens: 209 -tick: 51, setAvailableIOTokens: 209 -tick: 52, setAvailableIOTokens: 209 -tick: 53, setAvailableIOTokens: 209 -tick: 54, setAvailableIOTokens: 209 -tick: 55, setAvailableIOTokens: 209 -tick: 56, setAvailableIOTokens: 209 -tick: 57, setAvailableIOTokens: 209 -tick: 58, setAvailableIOTokens: 209 -tick: 59, setAvailableIOTokens: 169 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 1, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 2, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 3, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 4, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 5, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 6, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 7, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 8, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 9, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 10, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 11, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 12, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 13, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 14, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 15, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 16, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 17, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 18, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 19, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 20, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 21, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 22, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 23, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 24, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 25, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 26, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 27, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 28, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 29, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 30, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 31, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 32, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 33, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 34, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 35, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 36, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 37, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 38, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 39, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 40, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 41, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 42, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 43, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 44, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 45, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 46, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 47, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 48, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 49, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 50, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 51, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 52, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 53, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 54, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 55, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 56, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 57, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 58, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited +tick: 59, setAvailableIOTokens: 169 setAvailableElasticDiskTokens: unlimited prep-admission-stats admitted=20000 write-bytes=80000 ---- @@ -157,78 +157,78 @@ prep-admission-stats admitted=20000 write-bytes=80000 set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 7 B, compacted 98 KiB [≈73 KiB], flushed 0 B [≈0 B]; admitting 24 KiB (rate 1.6 KiB/s) due to L0 growth (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:7} writeLM:{multiplier:2.125 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intL0WriteAccountedBytes:40000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:2.25 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:201000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 7 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.12x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 417 -tick: 1, setAvailableIOTokens: 417 -tick: 2, setAvailableIOTokens: 417 -tick: 3, setAvailableIOTokens: 417 -tick: 4, setAvailableIOTokens: 417 -tick: 5, setAvailableIOTokens: 417 -tick: 6, setAvailableIOTokens: 417 -tick: 7, setAvailableIOTokens: 417 -tick: 8, setAvailableIOTokens: 417 -tick: 9, setAvailableIOTokens: 417 -tick: 10, setAvailableIOTokens: 417 -tick: 11, setAvailableIOTokens: 417 -tick: 12, setAvailableIOTokens: 417 -tick: 13, setAvailableIOTokens: 417 -tick: 14, setAvailableIOTokens: 417 -tick: 15, setAvailableIOTokens: 417 -tick: 16, setAvailableIOTokens: 417 -tick: 17, setAvailableIOTokens: 417 -tick: 18, setAvailableIOTokens: 417 -tick: 19, setAvailableIOTokens: 417 -tick: 20, setAvailableIOTokens: 417 -tick: 21, setAvailableIOTokens: 417 -tick: 22, setAvailableIOTokens: 417 -tick: 23, setAvailableIOTokens: 417 -tick: 24, setAvailableIOTokens: 417 -tick: 25, setAvailableIOTokens: 417 -tick: 26, setAvailableIOTokens: 417 -tick: 27, setAvailableIOTokens: 417 -tick: 28, setAvailableIOTokens: 417 -tick: 29, setAvailableIOTokens: 417 -tick: 30, setAvailableIOTokens: 417 -tick: 31, setAvailableIOTokens: 417 -tick: 32, setAvailableIOTokens: 417 -tick: 33, setAvailableIOTokens: 417 -tick: 34, setAvailableIOTokens: 417 -tick: 35, setAvailableIOTokens: 417 -tick: 36, setAvailableIOTokens: 417 -tick: 37, setAvailableIOTokens: 417 -tick: 38, setAvailableIOTokens: 417 -tick: 39, setAvailableIOTokens: 417 -tick: 40, setAvailableIOTokens: 417 -tick: 41, setAvailableIOTokens: 417 -tick: 42, setAvailableIOTokens: 417 -tick: 43, setAvailableIOTokens: 417 -tick: 44, setAvailableIOTokens: 417 -tick: 45, setAvailableIOTokens: 417 -tick: 46, setAvailableIOTokens: 417 -tick: 47, setAvailableIOTokens: 417 -tick: 48, setAvailableIOTokens: 417 -tick: 49, setAvailableIOTokens: 417 -tick: 50, setAvailableIOTokens: 417 -tick: 51, setAvailableIOTokens: 417 -tick: 52, setAvailableIOTokens: 417 -tick: 53, setAvailableIOTokens: 417 -tick: 54, setAvailableIOTokens: 417 -tick: 55, setAvailableIOTokens: 417 -tick: 56, setAvailableIOTokens: 417 -tick: 57, setAvailableIOTokens: 417 -tick: 58, setAvailableIOTokens: 417 -tick: 59, setAvailableIOTokens: 397 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 1, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 2, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 3, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 4, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 5, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 6, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 7, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 8, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 9, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 10, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 11, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 12, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 13, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 14, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 15, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 16, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 17, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 18, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 19, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 20, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 21, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 22, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 23, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 24, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 25, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 26, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 27, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 28, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 29, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 30, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 31, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 32, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 33, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 34, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 35, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 36, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 37, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 38, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 39, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 40, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 41, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 42, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 43, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 44, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 45, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 46, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 47, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 48, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 49, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 50, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 51, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 52, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 53, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 54, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 55, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 56, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 57, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 58, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited +tick: 59, setAvailableIOTokens: 397 setAvailableElasticDiskTokens: unlimited # No delta. This used to trigger an overflow bug. set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 7 B, compacted 0 B [≈37 KiB], flushed 0 B [≈0 B]; admitting 21 KiB (rate 1.4 KiB/s) due to L0 growth (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:21875 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:7} writeLM:{multiplier:2.125 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:201000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 7 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.12x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 365 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 365 setAvailableElasticDiskTokens: unlimited prep-admission-stats admitted=30000 write-bytes=120000 ---- @@ -239,10 +239,10 @@ prep-admission-stats admitted=30000 write-bytes=120000 set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=20 print-only-first-tick=true ---- compaction score 1.000 (21 ssts, 20 sub-levels), L0 growth 293 KiB (write 293 KiB ingest 0 B ignored 0 B): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 3.00x+18 B (smoothed 2.56x+9 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 18 B, compacted 293 KiB [≈165 KiB], flushed 0 B [≈0 B]; admitting all -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:160937.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:18} writeLM:{multiplier:2.5625 constant:9} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intL0WriteAccountedBytes:40000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:3 constant:18} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:160937.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:9} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:3 constant:18} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:501000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 18 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.56x+9 ingested-lm: 0.75x+1 -setAvailableIOTokens: unlimited +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+9 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited # Test cases with more information in storeAdmissionStats. init @@ -255,8 +255,8 @@ prep-admission-stats admitted=0 set-state l0-bytes=1000 l0-added-write=1000 l0-added-ingested=0 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} -tick: 0, setAvailableIOTokens: unlimited +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited # L0 will see an addition of 200,000 bytes. 150,000 bytes were mentioned by # the admitted requests. @@ -264,13 +264,18 @@ prep-admission-stats admitted=10 write-bytes=130000 ingested-bytes=20000 ---- {admittedCount:10 writeAccountedBytes:130000 ingestedAccountedBytes:20000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} +# The ingested model can be fit with a multiplier of ~1.5 for the interval, +# but since the l0-ingest-lm model had a previous multiplier of 0.75 and the +# ingest-model had a previous multiplier of 1.0 and we do exponential +# smoothing with alpha=0.5, we end up the multipliers +# of 1.12 and 1.25 respectively. set-state l0-bytes=1000 l0-added-write=171000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 166 KiB ingest 29 KiB ignored 0 B): requests 10 (0 bypassed) with 127 KiB acc-write (0 B bypassed) + 20 KiB acc-ingest (0 B bypassed) + write-model 1.31x+1 B (smoothed 1.53x+1 B) + ingested-model 1.50x+1 B (smoothed 1.12x+1 B) + at-admission-tokens 9.8 KiB, compacted 195 KiB [≈98 KiB], flushed 0 B [≈0 B]; admitting 24 KiB (rate 1.6 KiB/s) due to L0 growth (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:10000} writeLM:{multiplier:1.5288076923076923 constant:1} ingestedLM:{multiplier:1.125 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intL0WriteAccountedBytes:130000 intL0IngestedAccountedBytes:20000 intWriteLinearModel:{multiplier:1.3076153846153846 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.5288076923076923 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMIngestedBytes:30000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intL0WriteLinearModel:{multiplier:1.3076153846153846 constant:1} intL0IngestedLinearModel:{multiplier:1.4995 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:201000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 10000 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.53x+1 ingested-lm: 1.12x+1 -setAvailableIOTokens: 417 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.53x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 +setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited # L0 will see an addition of 20,000 bytes, all of which are accounted for. # Since the ingested bytes in this interval are 0, the constant for the @@ -282,15 +287,13 @@ prep-admission-stats admitted=20 write-bytes=150000 ingested-bytes=20000 set-state l0-bytes=1000 l0-added-write=191000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB ingest 0 B ignored 0 B): requests 10 (0 bypassed) with 20 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 1.00x+1 B (smoothed 1.26x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + at-admission-tokens 5.9 KiB, compacted 20 KiB [≈59 KiB], flushed 0 B [≈0 B]; admitting 27 KiB (rate 1.8 KiB/s) due to L0 growth (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:27500 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:6000} writeLM:{multiplier:1.2641538461538462 constant:1} ingestedLM:{multiplier:1.125 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intL0WriteAccountedBytes:20000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0.9995 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:221000} smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:6000} l0WriteLM:{multiplier:1.2641538461538462 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.9995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:221000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 6000 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.26x+1 ingested-lm: 1.12x+1 -setAvailableIOTokens: 459 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.26x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 +setAvailableIOTokens: 459 setAvailableElasticDiskTokens: unlimited # L0 will see an addition of 20,000 bytes, but we think we have added 100,000 # bytes to L0. We don't let unaccounted bytes become negative. -# Since the ingested bytes in this interval are 0, the constant for the -# ingested model is decayed by a factor of 2. prep-admission-stats admitted=30 write-bytes=250000 ingested-bytes=20000 ingested-into-l0=20000 ---- {admittedCount:30 writeAccountedBytes:250000 ingestedAccountedBytes:20000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} @@ -298,10 +301,10 @@ prep-admission-stats admitted=30 write-bytes=250000 ingested-bytes=20000 ingeste set-state l0-bytes=1000 l0-added-write=211000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB ingest 0 B ignored 0 B): requests 10 (0 bypassed) with 98 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.50x+1 B (smoothed 0.88x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + at-admission-tokens 3.9 KiB, compacted 20 KiB [≈39 KiB], flushed 0 B [≈0 B]; admitting 23 KiB (rate 1.5 KiB/s) due to L0 growth (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:23750 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:4000} writeLM:{multiplier:0.8820769230769231 constant:1} ingestedLM:{multiplier:1.125 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intL0WriteAccountedBytes:100000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0.5 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:241000} smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:4000} l0WriteLM:{multiplier:0.8820769230769231 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:241000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 4000 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 0.88x+1 ingested-lm: 1.12x+1 -setAvailableIOTokens: 396 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 0.88x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 +setAvailableIOTokens: 396 setAvailableElasticDiskTokens: unlimited # Test case with flush tokens. init @@ -314,27 +317,27 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=1000 l0-files=1 l0-sublevels=1 print-only-first-tick=true ---- compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} -tick: 0, setAvailableIOTokens: unlimited +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited # Flush loop utilization is too low for the interval flush tokens to # contribute to the smoothed value, or for tokens to become limited. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=100 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈4.9 KiB], flushed 7.3 KiB [≈0 B]; admitting all -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:5000 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:5000 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: unlimited +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited # Flush loop utilization is high enough, so we compute flush tokens for limiting admission. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=10 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈2.4 KiB], flushed 7.3 KiB [≈7.3 KiB]; admitting 11 KiB (rate 750 B/s) due to memtable flush (multiplier 1.500) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:2500 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:2500 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 188 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 188 setAvailableElasticDiskTokens: unlimited # Write stalls are happening, so decrease the flush utilization target # fraction from 1.5 to 1.475. But the peak flush rate has also increased since @@ -342,10 +345,10 @@ setAvailableIOTokens: 188 set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=1 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈1.2 KiB], flushed 73 KiB [≈40 KiB]; admitting 59 KiB (rate 4.0 KiB/s) due to memtable flush (multiplier 1.475) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 smoothedIntL0CompactedBytes:1250 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:1250 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1015 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1015 setAvailableElasticDiskTokens: unlimited # Two write stalls happened, so decrease the flush utilization target fraction # by a bigger step, from 1.475 to 1.425. Since the smoothed peak flush rate is @@ -353,29 +356,29 @@ setAvailableIOTokens: 1015 set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=3 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈625 B], flushed 73 KiB [≈57 KiB]; admitting 81 KiB (rate 5.4 KiB/s) due to memtable flush (multiplier 1.425) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 smoothedIntL0CompactedBytes:625 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:625 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1381 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1381 setAvailableElasticDiskTokens: unlimited # Five more write stalls, so the the flush utilization target fraction is # decreased to 1.35. The smoothed peak flush rate continues to increase. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=8 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈312 B], flushed 73 KiB [≈65 KiB]; admitting 88 KiB (rate 5.8 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 smoothedIntL0CompactedBytes:312 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:312 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1498 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1498 setAvailableElasticDiskTokens: unlimited # Another write stall, and the flush utilization target fraction drops to 1.325. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=9 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈156 B], flushed 73 KiB [≈69 KiB]; admitting 92 KiB (rate 6.1 KiB/s) due to memtable flush (multiplier 1.325) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 smoothedIntL0CompactedBytes:156 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:156 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1564 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1564 setAvailableElasticDiskTokens: unlimited # Set a lower bound of 1.3 on the flush utilization target fraction. set-min-flush-util percent=130 @@ -386,20 +389,20 @@ set-min-flush-util percent=130 set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=10 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈78 B], flushed 73 KiB [≈71 KiB]; admitting 92 KiB (rate 6.2 KiB/s) due to memtable flush (multiplier 1.300) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 smoothedIntL0CompactedBytes:78 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:78 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1580 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1580 setAvailableElasticDiskTokens: unlimited # Despite another write stall, the flush utilization target fraction does not # decrease since it is already at the lower bound. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=11 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈39 B], flushed 73 KiB [≈72 KiB]; admitting 94 KiB (rate 6.3 KiB/s) due to memtable flush (multiplier 1.300) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 smoothedIntL0CompactedBytes:39 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:39 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1603 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1603 setAvailableElasticDiskTokens: unlimited # Bump up the lower bound to 1.35, which is greater than the current flush # utilization target fraction. @@ -411,65 +414,107 @@ set-min-flush-util percent=135 set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=12 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈19 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.5 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 smoothedIntL0CompactedBytes:19 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:19 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1676 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1676 setAvailableElasticDiskTokens: unlimited # The flush utilization is too low, so there is no limit on flush tokens. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=100 write-stall-count=13 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈9 B], flushed 73 KiB [≈73 KiB]; admitting all -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:9 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:9 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: unlimited +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited # Flush utilization is high enough, so flush tokens are again limited. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈4 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.6 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:4 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:4 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1682 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1682 setAvailableElasticDiskTokens: unlimited # No write stalls, and token utilization is high, which will have an effect # in the next pebbleMetricsTick. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈2 B], flushed 73 KiB [≈73 KiB]; admitting 99 KiB (rate 6.6 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:2 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:2 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1685 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1685 setAvailableElasticDiskTokens: unlimited # No write stalls, and token utilization was high, so flush utilization # target fraction is increased to 1.375. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈1 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) due to memtable flush (multiplier 1.375) (used 197 KiB) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:1 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:1 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1718 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1718 setAvailableElasticDiskTokens: unlimited # No write stalls, and token utilization was high, so flush utilization # target fraction is increased to 1.4. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 73 KiB [≈73 KiB]; admitting 102 KiB (rate 6.8 KiB/s) due to memtable flush (multiplier 1.400) (used 201 KiB) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1750 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1750 setAvailableElasticDiskTokens: unlimited # There is a write stall, so even though token utilization is high, we # decrease flush utilization target fraction to 1.375. set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=14 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) due to memtable flush (multiplier 1.375) (used 205 KiB) -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +store-request-estimates: writeTokens: 1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: 1719 setAvailableElasticDiskTokens: unlimited + +# Test disk bandwidth tokens. +init +---- + +set-state l0-bytes=100 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-bandwidth=10 l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:0} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited + +set-state l0-bytes=100 l0-added-write=100000 bytes-read=1000000 bytes-written=2000000 provisioned-bandwidth=10 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 0 B [≈0 B]; admitting all; elastic tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 65 KiB write 130 KiB provisioned 10 B +{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:66666 writeBandwidth:133333 provisionedBandwidth:10} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} +store-request-estimates: writeTokens: 1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: 105 + +set-state l0-bytes=100 l0-added-write=200000 bytes-read=2000000 bytes-written=4000000 provisioned-bandwidth=4000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 0 B [≈0 B]; admitting all; elastic tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 130 KiB write 260 KiB provisioned 3.8 MiB +{ioLoadListenerState:{cumL0AddedBytes:200000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:2000000 bytesWritten:4000000 incomingLSMBytes:200000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:75000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:133333 writeBandwidth:266666 provisionedBandwidth:4000000} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} +store-request-estimates: writeTokens: 1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: 105 + +set-state l0-bytes=100 l0-added-write=300000 bytes-read=3000000 bytes-written=6000000 provisioned-bandwidth=4000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈85 KiB], flushed 0 B [≈0 B]; admitting all; elastic tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 195 KiB write 391 KiB provisioned 3.8 MiB +{ioLoadListenerState:{cumL0AddedBytes:300000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:3000000 bytesWritten:6000000 incomingLSMBytes:300000} smoothedIntL0CompactedBytes:87500 smoothedCompactionByteTokens:87500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:200000 writeBandwidth:400000 provisionedBandwidth:4000000} intervalLSMInfo:{incomingBytes:300000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} +store-request-estimates: writeTokens: 1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: 105 + +set-state l0-bytes=100 l0-added-write=400000 bytes-read=4000000 bytes-written=8000000 provisioned-bandwidth=5000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈92 KiB], flushed 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumL0AddedBytes:400000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:4000000 bytesWritten:8000000 incomingLSMBytes:400000} smoothedIntL0CompactedBytes:93750 smoothedCompactionByteTokens:93750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:266666 writeBandwidth:533333 provisionedBandwidth:5000000} intervalLSMInfo:{incomingBytes:400000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 -setAvailableIOTokens: 1719 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited diff --git a/pkg/util/admission/testdata/store_per_work_token_estimator b/pkg/util/admission/testdata/store_per_work_token_estimator index 62b8a5a32236..decd67d0ee3f 100644 --- a/pkg/util/admission/testdata/store_per_work_token_estimator +++ b/pkg/util/admission/testdata/store_per_work_token_estimator @@ -4,71 +4,86 @@ init # First call initializes. update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 ---- -interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} at-admission-tokens: 1 -write-tokens: int: 0.00x+0 smoothed: 1.75x+1 per-work-accounted: 1 -ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +L0-write-tokens: int: 0.00x+0 smoothed: 1.75x+1 per-work-accounted: 1 +L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 # Writes account for ~1/2 of what is written, reflecting what can happen with # application to the state machine. No ingests. update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intL0WriteAccountedBytes:500 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} at-admission-tokens: 50 -write-tokens: int: 1.98x+1 smoothed: 1.86x+1 per-work-accounted: 25 -ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +L0-write-tokens: int: 1.98x+1 smoothed: 1.86x+1 per-work-accounted: 25 +L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 # Same as previous, except some of these are bypassed. Will not affect the model. update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 bypassed-count=4 bypassed-write=300 bypassed-ingested=0 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intL0WriteAccountedBytes:500 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:4 intL0WriteBypassedAccountedBytes:300 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:4 intL0WriteBypassedAccountedBytes:300 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} at-admission-tokens: 75 -write-tokens: int: 1.98x+1 smoothed: 1.92x+1 per-work-accounted: 37 -ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +L0-write-tokens: int: 1.98x+1 smoothed: 1.92x+1 per-work-accounted: 37 +L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 # Ingestion also happens. Bumps up the at-admission-tokens since at that time # we can't differentiate between writes and ingests. The constants in the # linear models stays 1, since we can fit effectively using the multipliers. # This means a mix of regular writes and sstable ingests (say index -# backfills), will not effect the cost attributed to regular writes. -update flushed=1000 ingested=1000 admitted=10 write-accounted=500 ingested-accounted=4000 bypassed-count=2 bypassed-write=0 bypassed-ingested=2000 +# backfills), will not effect the cost attributed to regular writes. The +# L0-ingest model will be fit based on accounted bytes of 4000, and actual +# bytes (ingested into L0) of 1000, hence the 0.25 multiplier. The ingest +# model uses all the ingested bytes including other-levels-ingested, so the +# observed bytes are 1000+9000=10000, and the accounted bytes are 4000, so the +# max multiplier of 1.5 is used and the rest handled in the additive term. +update flushed=1000 ingested=1000 other-levels-ingested=9000 admitted=10 write-accounted=500 ingested-accounted=4000 bypassed-count=2 bypassed-write=0 bypassed-ingested=2000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000 intL0WriteAccountedBytes:500 intL0IngestedAccountedBytes:4000 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0.2475 constant:1} intBypassedWorkCount:2 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:2000 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000 intLSMIngestedBytes:10000 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:4000 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0.2475 constant:1} intIngestedLinearModel:{multiplier:1.5 constant:400} intBypassedWorkCount:2 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:2000 intL0IgnoredIngestedBytes:0} at-admission-tokens: 137 -write-tokens: int: 1.98x+1 smoothed: 1.95x+1 per-work-accounted: 43 -ingest-tokens: int: 0.25x+1 smoothed: 0.50x+1 per-work-accounted: 200 +L0-write-tokens: int: 1.98x+1 smoothed: 1.95x+1 per-work-accounted: 43 +L0-ingest-tokens: int: 0.25x+1 smoothed: 0.50x+1 per-work-accounted: 200 +ingest-tokens: int: 1.50x+400 smoothed: 1.25x+200 per-work-accounted: 200 -# No ingestion observed by LSM, though ingested-accounted is non-zero -- this -# updates the model since all these ingested bytes could have gone to levels -# lower than L0. -update flushed=1000 ingested=0 admitted=10 write-accounted=450 ingested-accounted=500 +# No ingestion into L0 observed by LSM, though ingested-accounted is non-zero +# -- this updates the L0-ingest model since all these ingested bytes have gone +# to levels lower than L0. For the ingest model, the observed bytes are 500, +# and the accounted bytes are 500, hence the multiplier close to 1.0. +update flushed=1000 ingested=0 other-levels-ingested=500 admitted=10 write-accounted=450 ingested-accounted=500 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intL0WriteAccountedBytes:450 intL0IngestedAccountedBytes:500 intWriteLinearModel:{multiplier:2.2 constant:1} intIngestedLinearModel:{multiplier:0.001 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:500 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:500 intL0WriteLinearModel:{multiplier:2.2 constant:1} intL0IngestedLinearModel:{multiplier:0.001 constant:1} intIngestedLinearModel:{multiplier:0.98 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} at-admission-tokens: 118 -write-tokens: int: 2.20x+1 smoothed: 2.08x+1 per-work-accounted: 44 -ingest-tokens: int: 0.00x+1 smoothed: 0.25x+1 per-work-accounted: 125 +L0-write-tokens: int: 2.20x+1 smoothed: 2.08x+1 per-work-accounted: 44 +L0-ingest-tokens: int: 0.00x+1 smoothed: 0.25x+1 per-work-accounted: 125 +ingest-tokens: int: 0.98x+1 smoothed: 1.11x+100 per-work-accounted: 125 # Large amount of ingestion. Bumps up at-admission-tokens. -update flushed=1000 ingested=1000000 admitted=10 write-accounted=450 ingested-accounted=2000000 +update flushed=1000 ingested=1000000 other-levels-ingested=2000000 admitted=10 write-accounted=450 ingested-accounted=2000000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000000 intL0WriteAccountedBytes:450 intL0IngestedAccountedBytes:2000000 intWriteLinearModel:{multiplier:2.2 constant:1} intIngestedLinearModel:{multiplier:0.499995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000000 intLSMIngestedBytes:3000000 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:2000000 intL0WriteLinearModel:{multiplier:2.2 constant:1} intL0IngestedLinearModel:{multiplier:0.499995 constant:1} intIngestedLinearModel:{multiplier:1.499995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} at-admission-tokens: 50109 -write-tokens: int: 2.20x+1 smoothed: 2.14x+1 per-work-accounted: 44 -ingest-tokens: int: 0.50x+1 smoothed: 0.37x+1 per-work-accounted: 100062 +L0-write-tokens: int: 2.20x+1 smoothed: 2.14x+1 per-work-accounted: 44 +L0-ingest-tokens: int: 0.50x+1 smoothed: 0.37x+1 per-work-accounted: 100062 +ingest-tokens: int: 1.50x+1 smoothed: 1.31x+50 per-work-accounted: 100062 # Large number of bytes ingested into L0, but only 2000 are not to be ignored. -# So we can fit a reasonable model. +# So we can fit a reasonable model. The L0-ingest and ingest models are the +# same since no bytes were ingested into levels other than L0. update flushed=0 ingested=1000000 admitted=10 write-accounted=0 ingested-accounted=2000 ignore-ingested-into-L0=998000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:2000 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000} +interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0.995 constant:1} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000} at-admission-tokens: 25154 -write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44 -ingest-tokens: int: 0.99x+1 smoothed: 0.68x+1 per-work-accounted: 50131 +L0-write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44 +L0-ingest-tokens: int: 0.99x+1 smoothed: 0.68x+1 per-work-accounted: 50131 +ingest-tokens: int: 0.99x+1 smoothed: 1.15x+25 per-work-accounted: 50131 # Repeat of the previous. update flushed=0 ingested=1000000 admitted=10 write-accounted=0 ingested-accounted=2000 ignore-ingested-into-L0=998000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:2000 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000} +interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0.995 constant:1} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000} at-admission-tokens: 12677 -write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44 -ingest-tokens: int: 0.99x+1 smoothed: 0.84x+1 per-work-accounted: 25165 +L0-write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44 +L0-ingest-tokens: int: 0.99x+1 smoothed: 0.84x+1 per-work-accounted: 25165 +ingest-tokens: int: 0.99x+1 smoothed: 1.07x+13 per-work-accounted: 25165 diff --git a/pkg/util/admission/testdata/store_work_queue b/pkg/util/admission/testdata/store_work_queue index a3584af56438..ae615d87acd6 100644 --- a/pkg/util/admission/testdata/store_work_queue +++ b/pkg/util/admission/testdata/store_work_queue @@ -3,7 +3,8 @@ init print ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:0 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:1} @@ -12,35 +13,37 @@ set-try-get-return-value v=true admit id=1 tenant=53 priority=0 create-time-millis=1 bypass=false ---- -tryGet: returning true -id 1: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:1 admissionEnabled:true} +tryGet regular: returning true +id 1: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:1 workClass:0 admissionEnabled:true} work-done id=1 ---- -storeWriteDone: originalTokens 1, doneBytes(write 0,ingested 0) returning 0 +storeWriteDone regular: originalTokens 1, doneBytes(write 0,ingested 0) returning 0 set-store-request-estimates write-tokens=100 ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 1, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:1 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:100} admit id=2 tenant=55 priority=0 create-time-millis=1 bypass=false ---- -tryGet: returning true -id 2: admit succeeded with handle {tenantID:{InternalValue:55} writeTokens:100 admissionEnabled:true} +tryGet regular: returning true +id 2: admit succeeded with handle {tenantID:{InternalValue:55} writeTokens:100 workClass:0 admissionEnabled:true} admit id=3 tenant=53 priority=0 create-time-millis=1 bypass=false ---- -tryGet: returning true -id 3: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:100 admissionEnabled:true} +tryGet regular: returning true +id 3: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:100 workClass:0 admissionEnabled:true} print ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 101, w: 1, fifo: -128 tenant-id: 55 used: 100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:1 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:100} @@ -49,85 +52,176 @@ set-try-get-return-value v=false admit id=4 tenant=57 priority=0 create-time-millis=1 bypass=false ---- -tryGet: returning false +tryGet regular: returning false work-done id=2 additional-tokens=500 ---- -storeWriteDone: originalTokens 100, doneBytes(write 0,ingested 0) returning 500 +storeWriteDone regular: originalTokens 100, doneBytes(write 0,ingested 0) returning 500 print ---- -closed epoch: 0 tenantHeap len: 1 top tenant: 57 +regular workqueue: closed epoch: 0 tenantHeap len: 1 top tenant: 57 tenant-id: 53 used: 101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 0, w: 1, fifo: -128 waiting work heap: [0: pri: 0, ct: 1, epoch: 0, qt: 0] +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:2 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:100} granted ---- -continueGrantChain 0 -id 4: admit succeeded with handle {tenantID:{InternalValue:57} writeTokens:100 admissionEnabled:true} -granted: returned 100 +continueGrantChain regular 0 +id 4: admit succeeded with handle {tenantID:{InternalValue:57} writeTokens:100 workClass:0 admissionEnabled:true} +granted regular: returned 100 print ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:2 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:100} work-done id=3 ingested-bytes=1000000 additional-tokens=50000 ---- -storeWriteDone: originalTokens 100, doneBytes(write 0,ingested 1000000) returning 50000 +storeWriteDone regular: originalTokens 100, doneBytes(write 0,ingested 1000000) returning 50000 print ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 50101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:3 writeAccountedBytes:0 ingestedAccountedBytes:1000000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:100} set-store-request-estimates write-tokens=10000 ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 50101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:3 writeAccountedBytes:0 ingestedAccountedBytes:1000000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:10000} work-done id=4 write-bytes=2000 ingested-bytes=1000 additional-tokens=2000 ---- -storeWriteDone: originalTokens 100, doneBytes(write 2000,ingested 1000) returning 2000 +storeWriteDone regular: originalTokens 100, doneBytes(write 2000,ingested 1000) returning 2000 print ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 50101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:4 writeAccountedBytes:2000 ingestedAccountedBytes:1001000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} estimates:{writeTokens:10000} bypassed-work-done work-count=10 write-bytes=1000 ingested-bytes=1000000 ---- -closed epoch: 0 tenantHeap len: 0 +storeWriteDone regular: originalTokens 0, doneBytes(write 1000,ingested 1000000) returning 2000 + +print +---- +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 50101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}} estimates:{writeTokens:10000} stats-to-ignore ingested-bytes=12000 ingested-into-L0-bytes=9000 ---- -closed epoch: 0 tenantHeap len: 0 +regular workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 50101, w: 1, fifo: -128 + tenant-id: 55 used: 600, w: 1, fifo: -128 + tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 +stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}} +estimates:{writeTokens:10000} + +# Elastic work. +admit id=5 tenant=53 priority=-30 create-time-millis=1 bypass=false +---- +tryGet elastic: returning false + +# Noop since the only waiting work is elastic and this is granting to regular. +# work. +granted +---- +granted regular: returned 0 + +granted elastic=true +---- +continueGrantChain elastic 0 +id 5: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:10000 workClass:1 admissionEnabled:true} +granted elastic: returned 10000 + +print +---- +regular workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 50101, w: 1, fifo: -128 + tenant-id: 55 used: 600, w: 1, fifo: -128 + tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 10000, w: 1, fifo: -128 +stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}} +estimates:{writeTokens:10000} + +set-try-get-return-value v=true elastic=true +---- + +admit id=6 tenant=54 priority=-40 create-time-millis=3 bypass=false +---- +tryGet elastic: returning true +id 6: admit succeeded with handle {tenantID:{InternalValue:54} writeTokens:10000 workClass:1 admissionEnabled:true} + +print +---- +regular workqueue: closed epoch: 0 tenantHeap len: 0 tenant-id: 53 used: 50101, w: 1, fifo: -128 tenant-id: 55 used: 600, w: 1, fifo: -128 tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 10000, w: 1, fifo: -128 + tenant-id: 54 used: 10000, w: 1, fifo: -128 stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}} estimates:{writeTokens:10000} + +work-done id=5 write-bytes=1000 additional-tokens=200 +---- +storeWriteDone elastic: originalTokens 10000, doneBytes(write 1000,ingested 0) returning 200 + +print +---- +regular workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 50101, w: 1, fifo: -128 + tenant-id: 55 used: 600, w: 1, fifo: -128 + tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 10200, w: 1, fifo: -128 + tenant-id: 54 used: 10000, w: 1, fifo: -128 +stats:{admittedCount:15 writeAccountedBytes:4000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}} +estimates:{writeTokens:10000} + +work-done id=6 ingested-bytes=500 additional-tokens=500 +---- +storeWriteDone elastic: originalTokens 10000, doneBytes(write 0,ingested 500) returning 500 + +print +---- +regular workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 50101, w: 1, fifo: -128 + tenant-id: 55 used: 600, w: 1, fifo: -128 + tenant-id: 57 used: 2100, w: 1, fifo: -128 +elastic workqueue: closed epoch: 0 tenantHeap len: 0 + tenant-id: 53 used: 10200, w: 1, fifo: -128 + tenant-id: 54 used: 10500, w: 1, fifo: -128 +stats:{admittedCount:16 writeAccountedBytes:4000 ingestedAccountedBytes:2001500 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}} +estimates:{writeTokens:10000} diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index dbf7cd258df5..c51e42167bc4 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -1559,11 +1559,11 @@ type StoreWriteWorkInfo struct { // StoreWorkQueue is responsible for admission to a store. type StoreWorkQueue struct { - q WorkQueue + q [numWorkClasses]WorkQueue // Only calls storeWriteDone. The rest of the interface is used by // WorkQueue. - granter granterWithStoreWriteDone - mu struct { + granters [numWorkClasses]granterWithStoreWriteDone + mu struct { syncutil.RWMutex estimates storeRequestEstimates stats storeAdmissionStats @@ -1577,6 +1577,7 @@ type StoreWorkHandle struct { tenantID roachpb.TenantID // The writeTokens acquired by this request. Must be > 0. writeTokens int64 + workClass workClass admissionEnabled bool } @@ -1593,15 +1594,21 @@ func (h StoreWorkHandle) AdmissionEnabled() bool { func (q *StoreWorkQueue) Admit( ctx context.Context, info StoreWriteWorkInfo, ) (handle StoreWorkHandle, err error) { + // For now, we compute a workClass based on priority. + wc := regularWorkClass + if info.Priority < admissionpb.NormalPri { + wc = elasticWorkClass + } h := StoreWorkHandle{ - tenantID: info.TenantID, + tenantID: info.TenantID, + workClass: wc, } q.mu.RLock() estimates := q.mu.estimates q.mu.RUnlock() h.writeTokens = estimates.writeTokens info.WorkInfo.requestedCount = h.writeTokens - enabled, err := q.q.Admit(ctx, info.WorkInfo) + enabled, err := q.q[wc].Admit(ctx, info.WorkInfo) if err != nil { return StoreWorkHandle{}, err } @@ -1616,13 +1623,6 @@ type StoreWorkDoneInfo struct { // the write-batch is empty, which happens when all the bytes are being // added via sstable ingestion. NB: it is possible for both WriteBytes and // IngestedBytes to be 0 if nothing was actually written. - // - // TODO(sumeer): WriteBytes will under count the actual effect on the Pebble - // store shared by the raft log and the state machine, since this only - // reflects the changes to the raft log. We compensate for this with an - // additive adjustment which is the same across all writes regardless of - // bytes. We should consider using an adjustment that is proportional to the - // WriteBytes. WriteBytes int64 // The size of the sstables, for ingests. Zero if there were no ingests. IngestedBytes int64 @@ -1635,8 +1635,8 @@ func (q *StoreWorkQueue) AdmittedWorkDone(h StoreWorkHandle, doneInfo StoreWorkD return nil } q.updateStoreAdmissionStats(1, doneInfo, false) - additionalTokens := q.granter.storeWriteDone(h.writeTokens, doneInfo) - q.q.adjustTenantTokens(h.tenantID, additionalTokens) + additionalTokens := q.granters[h.workClass].storeWriteDone(h.writeTokens, doneInfo) + q.q[h.workClass].adjustTenantTokens(h.tenantID, additionalTokens) return nil } @@ -1645,7 +1645,9 @@ func (q *StoreWorkQueue) AdmittedWorkDone(h StoreWorkHandle, doneInfo StoreWorkD // estimation model. func (q *StoreWorkQueue) BypassedWorkDone(workCount int64, doneInfo StoreWorkDoneInfo) { q.updateStoreAdmissionStats(uint64(workCount), doneInfo, true) - _ = q.granter.storeWriteDone(0, doneInfo) + // Since we have no control over such work, we choose to count it as + // regularWorkClass. + _ = q.granters[regularWorkClass].storeWriteDone(0, doneInfo) } // StatsToIgnore is called for range snapshot ingestion -- see the comment in @@ -1674,19 +1676,24 @@ func (q *StoreWorkQueue) updateStoreAdmissionStats( // SetTenantWeights passes through to WorkQueue.SetTenantWeights. func (q *StoreWorkQueue) SetTenantWeights(tenantWeights map[uint64]uint32) { - q.q.SetTenantWeights(tenantWeights) -} - -func (q *StoreWorkQueue) hasWaitingRequests() bool { - return q.q.hasWaitingRequests() + for i := range q.q { + q.q[i].SetTenantWeights(tenantWeights) + } } -func (q *StoreWorkQueue) granted(grantChainID grantChainID) int64 { - return q.q.granted(grantChainID) +// getRequesters implements storeRequester. +func (q *StoreWorkQueue) getRequesters() [numWorkClasses]requester { + var result [numWorkClasses]requester + for i := range q.q { + result[i] = &q.q[i] + } + return result } func (q *StoreWorkQueue) close() { - q.q.close() + for i := range q.q { + q.q[i].close() + } } func (q *StoreWorkQueue) getStoreAdmissionStats() storeAdmissionStats { @@ -1703,14 +1710,16 @@ func (q *StoreWorkQueue) setStoreRequestEstimates(estimates storeRequestEstimate func makeStoreWorkQueue( ambientCtx log.AmbientContext, - granter granterWithStoreWriteDone, + granters [numWorkClasses]granterWithStoreWriteDone, settings *cluster.Settings, opts workQueueOptions, ) storeRequester { q := &StoreWorkQueue{ - granter: granter, + granters: granters, + } + for i := range q.q { + initWorkQueue(&q.q[i], ambientCtx, KVWork, granters[i], settings, opts) } - initWorkQueue(&q.q, ambientCtx, KVWork, granter, settings, opts) // Arbitrary initial value. This will be replaced before any meaningful // token constraints are enforced. q.mu.estimates = storeRequestEstimates{ diff --git a/pkg/util/admission/work_queue_test.go b/pkg/util/admission/work_queue_test.go index 7b43fc7518a1..7e74ab6a4649 100644 --- a/pkg/util/admission/work_queue_test.go +++ b/pkg/util/admission/work_queue_test.go @@ -56,29 +56,30 @@ func (b *builderWithMu) stringAndReset() string { } type testGranter struct { + name string buf *builderWithMu r requester returnValueFromTryGet bool additionalTokens int64 } -var _ granter = &testGranter{} +var _ granterWithStoreWriteDone = &testGranter{} func (tg *testGranter) grantKind() grantKind { return slot } func (tg *testGranter) tryGet(count int64) bool { - tg.buf.printf("tryGet: returning %t", tg.returnValueFromTryGet) + tg.buf.printf("tryGet%s: returning %t", tg.name, tg.returnValueFromTryGet) return tg.returnValueFromTryGet } func (tg *testGranter) returnGrant(count int64) { - tg.buf.printf("returnGrant %d", count) + tg.buf.printf("returnGrant%s %d", tg.name, count) } func (tg *testGranter) tookWithoutPermission(count int64) { - tg.buf.printf("tookWithoutPermission %d", count) + tg.buf.printf("tookWithoutPermission%s %d", tg.name, count) } func (tg *testGranter) continueGrantChain(grantChainID grantChainID) { - tg.buf.printf("continueGrantChain %d", grantChainID) + tg.buf.printf("continueGrantChain%s %d", tg.name, grantChainID) } func (tg *testGranter) grant(grantChainID grantChainID) { rv := tg.r.granted(grantChainID) @@ -89,13 +90,13 @@ func (tg *testGranter) grant(grantChainID grantChainID) { // concurrency_manager_test.go. time.Sleep(50 * time.Millisecond) } - tg.buf.printf("granted: returned %d", rv) + tg.buf.printf("granted%s: returned %d", tg.name, rv) } func (tg *testGranter) storeWriteDone( originalTokens int64, doneInfo StoreWorkDoneInfo, ) (additionalTokens int64) { - tg.buf.printf("storeWriteDone: originalTokens %d, doneBytes(write %d,ingested %d) returning %d", - originalTokens, doneInfo.WriteBytes, doneInfo.IngestedBytes, tg.additionalTokens) + tg.buf.printf("storeWriteDone%s: originalTokens %d, doneBytes(write %d,ingested %d) returning %d", + tg.name, originalTokens, doneInfo.WriteBytes, doneInfo.IngestedBytes, tg.additionalTokens) return tg.additionalTokens } @@ -448,12 +449,24 @@ func TestPriorityStates(t *testing.T) { }) } +func tryScanWorkClass(t *testing.T, d *datadriven.TestData) workClass { + wc := regularWorkClass + if d.HasArg("elastic") { + var b bool + d.ScanArgs(t, "elastic", &b) + if b { + wc = elasticWorkClass + } + } + return wc +} + /* TestStoreWorkQueueBasic is a datadriven test with the following commands: init admit id= tenant= priority= create-time-millis= bypass= -set-try-get-return-value v= -granted +set-try-get-return-value v= [elastic=] +granted [elastic=] cancel-work id= work-done id= [write-bytes=] [ingested-bytes=] [additional-tokens=] print @@ -469,14 +482,15 @@ func TestStoreWorkQueueBasic(t *testing.T) { } } defer closeFn() - var tg *testGranter + var tg [numWorkClasses]*testGranter var wrkMap workMap var buf builderWithMu var st *cluster.Settings printQueue := func() string { q.mu.Lock() defer q.mu.Unlock() - return fmt.Sprintf("%s\nstats:%+v\nestimates:%+v", q.q.String(), q.mu.stats, + return fmt.Sprintf("regular workqueue: %s\nelastic workqueue: %s\nstats:%+v\nestimates:%+v", + q.q[regularWorkClass].String(), q.q[elasticWorkClass].String(), q.mu.stats, q.mu.estimates) } @@ -485,15 +499,18 @@ func TestStoreWorkQueueBasic(t *testing.T) { switch d.Cmd { case "init": closeFn() - tg = &testGranter{buf: &buf} + tg[regularWorkClass] = &testGranter{name: " regular", buf: &buf} + tg[elasticWorkClass] = &testGranter{name: " elastic", buf: &buf} opts := makeWorkQueueOptions(KVWork) opts.usesTokens = true opts.timeSource = timeutil.NewManualTime(timeutil.FromUnixMicros(0)) opts.disableEpochClosingGoroutine = true st = cluster.MakeTestingClusterSettings() q = makeStoreWorkQueue(log.MakeTestingAmbientContext(tracing.NewTracer()), - tg, st, opts).(*StoreWorkQueue) - tg.r = q + [numWorkClasses]granterWithStoreWriteDone{tg[regularWorkClass], tg[elasticWorkClass]}, + st, opts).(*StoreWorkQueue) + tg[regularWorkClass].r = q.getRequesters()[regularWorkClass] + tg[elasticWorkClass].r = q.getRequesters()[elasticWorkClass] wrkMap.resetMap() return "" @@ -537,7 +554,8 @@ func TestStoreWorkQueueBasic(t *testing.T) { case "set-try-get-return-value": var v bool d.ScanArgs(t, "v", &v) - tg.returnValueFromTryGet = v + wc := tryScanWorkClass(t, d) + tg[wc].returnValueFromTryGet = v return "" case "set-store-request-estimates": @@ -549,7 +567,8 @@ func TestStoreWorkQueueBasic(t *testing.T) { return printQueue() case "granted": - tg.grant(0) + wc := tryScanWorkClass(t, d) + tg[wc].grant(0) return buf.stringAndReset() case "cancel-work": @@ -589,7 +608,7 @@ func TestStoreWorkQueueBasic(t *testing.T) { if !work.admitted { return fmt.Sprintf("id not admitted: %d\n", id) } - tg.additionalTokens = int64(additionalTokens) + tg[work.handle.workClass].additionalTokens = int64(additionalTokens) require.NoError(t, q.AdmittedWorkDone(work.handle, StoreWorkDoneInfo{ WriteBytes: int64(writeBytes), @@ -607,7 +626,7 @@ func TestStoreWorkQueueBasic(t *testing.T) { WriteBytes: int64(writeBytes), IngestedBytes: int64(ingestedBytes), }) - return printQueue() + return buf.stringAndReset() case "stats-to-ignore": var ingestedBytes, ingestedIntoL0Bytes int
SettingTypeDefaultDescription
admission.disk_bandwidth_tokens.elastic.enabledbooleantruewhen true, and provisioned bandwidth for the disk corresponding to a store is configured, tokens for elastic work will be limited if disk bandwidth becomes a bottleneck
admission.epoch_lifo.enabledbooleanfalsewhen true, epoch-LIFO behavior is enabled when there is significant delay in admission
admission.epoch_lifo.epoch_closing_delta_durationduration5msthe delta duration before closing an epoch, for epoch-LIFO admission control ordering
admission.epoch_lifo.epoch_durationduration100msthe duration of an epoch, for epoch-LIFO admission control ordering