Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
77785: util/admission: minor cleanups r=sumeerbhola a=nvanbenschoten

### pass pebble.Metrics struct by reference

The `pebble.Metrics` struct is 1120 bytes large, so it's a little too large to be passing down a nested call stack by value, even if we only do so once per 15 seconds.

### use ctx.Err() for fast-path cancellation check

`Context.Err` is cheaper than calling `Context.Done` and then polling on the returned channel. `Context.Done` lazily allocates a channel and then writes to an atomic variable. The channel read then incurs a mutex lock.

----

I noticed these while finally doing a deep dive on the admission control implementation. One passing suggestion I'd make is that the "grant chain" concept is subtle and under-documented. I went through most of the package thinking it has something to do with dependent requests (e.g. intent resolution chains on an end txn). It would help for a top-level comment on `grantChainID` (or `continueGrantChain` if we want to keep commentary there) to spell out what grant chains are, their purpose, and how they work with an example.

Release justification: None. Wait for v22.2.

78266: ui: Hide table stats collection setting for non-admins r=ericharmeling a=ericharmeling

This PR hides the table statistics collection setting from non-admins in the DB Console.


https://user-images.githubusercontent.com/27286675/159566500-9310e9cc-eb8d-4a39-ae07-6620d36a27a2.mov

Part 1/2 of #77974.

(For background, see #77974 (comment))

Release justification: low-risk change

Release note: None

78268: roachprod: force wait=true for `stop --signal=9` r=srosenberg a=tbg

Touches #77334.

Release note: None


78470: execinfrapb: break the dependency on sem/builtins r=yuzefovich a=yuzefovich

This commit moves several utility functions from `execinfrapb` into
`execinfra` package in order to break the dependency of the former on
`sem/builtins` (which eventually depends on the `c-deps`).

Fixes: #78453.

Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Eric Harmeling <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
5 people committed Mar 28, 2022
5 parents f8e03e2 + 4a696f2 + 82a8527 + 5861187 + 1cecb4d commit 7d941bb
Show file tree
Hide file tree
Showing 24 changed files with 276 additions and 244 deletions.
4 changes: 4 additions & 0 deletions pkg/roachprod/install/cluster_synced.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,10 @@ func (c *SyncedCluster) newSession(node Node) (session, error) {
// When running roachprod stop without other flags, the signal is 9 (SIGKILL)
// and wait is true.
func (c *SyncedCluster) Stop(ctx context.Context, l *logger.Logger, sig int, wait bool) error {
if sig == 9 {
// `kill -9` without wait is never what a caller wants. See #77334.
wait = true
}
display := fmt.Sprintf("%s: stopping", c.Name)
if wait {
display += " and waiting"
Expand Down
4 changes: 3 additions & 1 deletion pkg/roachprod/roachprod.go
Original file line number Diff line number Diff line change
Expand Up @@ -675,7 +675,9 @@ func Monitor(
type StopOpts struct {
ProcessTag string
Sig int
Wait bool
// If Wait is set, roachprod waits until the PID disappears (i.e. the
// process has terminated).
Wait bool // forced to true when Sig == 9
}

// DefaultStopOpts returns StopOpts populated with the default values used by Stop.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colexec/colexecagg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_library(
"//pkg/sql/colexecerror",
"//pkg/sql/colexecop",
"//pkg/sql/colmem",
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/colexec/colexecagg/aggregate_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -446,16 +447,16 @@ func ProcessAggregations(
aggregations []execinfrapb.AggregatorSpec_Aggregation,
inputTypes []*types.T,
) (
constructors []execinfrapb.AggregateConstructor,
constructors []execinfra.AggregateConstructor,
constArguments []tree.Datums,
outputTypes []*types.T,
err error,
) {
constructors = make([]execinfrapb.AggregateConstructor, len(aggregations))
constructors = make([]execinfra.AggregateConstructor, len(aggregations))
constArguments = make([]tree.Datums, len(aggregations))
outputTypes = make([]*types.T, len(aggregations))
for i, aggFn := range aggregations {
constructors[i], constArguments[i], outputTypes[i], err = execinfrapb.GetAggregateConstructor(
constructors[i], constArguments[i], outputTypes[i], err = execinfra.GetAggregateConstructor(
evalCtx, semaCtx, &aggFn, inputTypes,
)
if err != nil {
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/colexec/colexecagg/aggregators_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package colexecagg
import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand All @@ -30,7 +31,7 @@ type NewAggregatorArgs struct {
InputTypes []*types.T
Spec *execinfrapb.AggregatorSpec
EvalCtx *tree.EvalContext
Constructors []execinfrapb.AggregateConstructor
Constructors []execinfra.AggregateConstructor
ConstArguments []tree.Datums
OutputTypes []*types.T
}
6 changes: 3 additions & 3 deletions pkg/sql/colexec/colexecagg/default_agg_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)
Expand Down Expand Up @@ -117,7 +117,7 @@ func (a *default_AGGKINDAgg) Reset() {

func newDefault_AGGKINDAggAlloc(
allocator *colmem.Allocator,
constructor execinfrapb.AggregateConstructor,
constructor execinfra.AggregateConstructor,
evalCtx *tree.EvalContext,
inputArgsConverter *colconv.VecToDatumConverter,
numArguments int,
Expand Down Expand Up @@ -147,7 +147,7 @@ type default_AGGKINDAggAlloc struct {
aggAllocBase
aggFuncs []default_AGGKINDAgg

constructor execinfrapb.AggregateConstructor
constructor execinfra.AggregateConstructor
evalCtx *tree.EvalContext
// inputArgsConverter is a converter from coldata.Vecs to tree.Datums that
// is shared among all aggregate functions and is managed by the aggregator
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/colexec/colexecagg/hash_default_agg.eg.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

6 changes: 3 additions & 3 deletions pkg/sql/colexec/colexecagg/ordered_default_agg.eg.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

5 changes: 3 additions & 2 deletions pkg/sql/distsql/columnar_operators_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
Expand Down Expand Up @@ -273,7 +274,7 @@ func TestAggregatorAgainstProcessor(t *testing.T) {
for _, typ := range aggFnInputTypes {
hasJSONColumn = hasJSONColumn || typ.Family() == types.JsonFamily
}
if _, outputType, err := execinfrapb.GetAggregateInfo(aggFn, aggFnInputTypes...); err == nil {
if _, outputType, err := execinfra.GetAggregateInfo(aggFn, aggFnInputTypes...); err == nil {
outputTypes[i] = outputType
break
}
Expand Down Expand Up @@ -1200,7 +1201,7 @@ func TestWindowFunctionsAgainstProcessor(t *testing.T) {
}
windowerSpec.WindowFns[0].Frame = generateWindowFrame(t, rng, &ordering, inputTypes)

_, outputType, err := execinfrapb.GetWindowFunctionInfo(fun, argTypes...)
_, outputType, err := execinfra.GetWindowFunctionInfo(fun, argTypes...)
require.NoError(t, err)
pspec := &execinfrapb.ProcessorSpec{
Input: []execinfrapb.InputSyncSpec{{ColumnTypes: inputTypes}},
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1913,7 +1913,7 @@ func (dsp *DistSQLPlanner) planAggregators(
for j, c := range e.ColIdx {
argTypes[j] = inputTypes[c]
}
_, outputType, err := execinfrapb.GetAggregateInfo(localFunc, argTypes...)
_, outputType, err := execinfra.GetAggregateInfo(localFunc, argTypes...)
if err != nil {
return err
}
Expand Down Expand Up @@ -1965,7 +1965,7 @@ func (dsp *DistSQLPlanner) planAggregators(
// the current aggregation e.
argTypes[i] = intermediateTypes[argIdxs[i]]
}
_, outputType, err := execinfrapb.GetAggregateInfo(finalInfo.Fn, argTypes...)
_, outputType, err := execinfra.GetAggregateInfo(finalInfo.Fn, argTypes...)
if err != nil {
return err
}
Expand Down Expand Up @@ -2120,7 +2120,7 @@ func (dsp *DistSQLPlanner) planAggregators(
}
copy(argTypes[len(agg.ColIdx):], info.argumentsColumnTypes[i])
var err error
_, returnTyp, err := execinfrapb.GetAggregateInfo(agg.Func, argTypes...)
_, returnTyp, err := execinfra.GetAggregateInfo(agg.Func, argTypes...)
if err != nil {
return err
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/distsql_plan_window.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package sql

import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -109,7 +110,7 @@ func (s *windowPlanState) createWindowFnSpec(
for i, argIdx := range funcInProgress.argsIdxs {
argTypes[i] = s.plan.GetResultTypes()[argIdx]
}
_, outputType, err := execinfrapb.GetWindowFunctionInfo(funcSpec, argTypes...)
_, outputType, err := execinfra.GetWindowFunctionInfo(funcSpec, argTypes...)
if err != nil {
return execinfrapb.WindowerSpec_WindowFn{}, outputType, err
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/execinfra/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ load("//build:STRINGER.bzl", "stringer")
go_library(
name = "execinfra",
srcs = [
"aggregatorbase.go",
"base.go",
"flow_context.go",
"metadata_test_receiver.go",
Expand Down Expand Up @@ -48,6 +49,7 @@ go_library(
"//pkg/sql/row",
"//pkg/sql/rowenc",
"//pkg/sql/rowenc/valueside",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlliveness",
Expand Down
Loading

0 comments on commit 7d941bb

Please sign in to comment.