Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
86488: util: fix loop var capture in TestFastIntMap r=herkolategan a=renatolabs

The `TestFastIntMap` test used `t.Parallel()` with a comment
indicating that it is safe for concurrent execution. However, the `tc`
variable was captured by the closure passed to `t.Run()`; this means
that, in practice, we will not actually run all test cases since the
loop variable will change before it is read by the Go routine (with
very high probability, only the last test case is executed).

This commit fixes the issue by creating a local copy of the loop
variable before the `t.Run()` call.

This bug was found by an updated version of the `loopvarcapture`
linter that has not been merged yet.

Release justification: test-only change.
Release note: None.

86505: sql: remove leftover unused code for window functions r=yuzefovich a=yuzefovich

This commit removes some of the dead code around the planning of window
functions. The code hasn't been used since the optimizer added the
support of the window functions (which was like a couple of years ago).

Release justification: low-risk cleanup.

Release note: None

86619: sql: prevent DROP OWNED BY with synthetic privileges r=ajwerner a=RichardJCai

Release justification: disallowing certain case within new schema change, bug fix

Release note (sql change): DROP OWNED BY cannot be performed if the
user has synthetic privileges (in system.privileges)

Fixes #86499

86764: ui/cluster-ui: fix search filter in active exec overview pages r=xinhaoz a=xinhaoz

Fixes #86556

This commit fixes a bug in the active execution overview pages where
the search filter was not being applied as a filter. This was due to
the search state being erronneously updated to the same value. The
search state is now correctly updated to the new value on change.

Release justification: bug fix
Release note (bug fix): search in active execution overview pages
works as expected, properly filtering out stmts and txns that do
not contain the search string


https://www.loom.com/share/619584d0a50e468da3ee8bbc04f45f75

86765: colexecdisk: extend allowed range for memory usage in a test r=yuzefovich a=yuzefovich

Recently merged a532aac added more
precise accounting when an operator spills to disk, so we need to extend
the allowed range for the memory usage in
`TestExternalSortMemoryAccounting` since we now account for more things.

Fixes: #86716.

Release justification: test-only change.

Release note: None

Co-authored-by: Renato Costa <renato@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: richardjcai <caioftherichard@gmail.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
  • Loading branch information
5 people committed Aug 24, 2022
6 parents bdadde2 + 63c75d8 + 69f6f89 + 2bd3826 + 85ea988 + 92c5d2a commit ba62be7
Show file tree
Hide file tree
Showing 14 changed files with 60 additions and 180 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/colexec/colexecdisk/external_sort_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ func TestExternalSortMemoryAccounting(t *testing.T) {
}
// We cannot guarantee a fixed value, so we use an allowed range.
expMin := memoryLimit
expMax := int64(float64(memoryLimit) * 1.6)
expMax := int64(float64(memoryLimit) * 1.8)
require.GreaterOrEqualf(t, totalMaxMemUsage, expMin, "minimum memory bound not satisfied: "+
"actual %d, expected min %d", totalMaxMemUsage, expMin)
require.GreaterOrEqualf(t, expMax, totalMaxMemUsage, "maximum memory bound not satisfied: "+
Expand Down
14 changes: 0 additions & 14 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -4055,27 +4055,13 @@ func (dsp *DistSQLPlanner) createPlanForWindow(
}

// We definitely added new columns, so we need to update PlanToStreamColMap.
// We need to update the map before adding rendering or projection because
// it is used there.
plan.PlanToStreamColMap = identityMap(plan.PlanToStreamColMap, len(plan.GetResultTypes()))

// windowers do not guarantee maintaining the order at the moment, so we
// reset MergeOrdering. There shouldn't be an ordering here, but we reset it
// defensively (see #35179).
plan.SetMergeOrdering(execinfrapb.Ordering{})

// After the window functions are computed, we need to add rendering or
// projection.
if err := addRenderingOrProjection(n, planCtx, plan); err != nil {
return nil, err
}

if len(plan.GetResultTypes()) != len(plan.PlanToStreamColMap) {
// We added/removed columns while rendering or projecting, so we need to
// update PlanToStreamColMap.
plan.PlanToStreamColMap = identityMap(plan.PlanToStreamColMap, len(plan.GetResultTypes()))
}

return plan, nil
}

Expand Down
103 changes: 0 additions & 103 deletions pkg/sql/distsql_plan_window.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfra/execagg"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -68,105 +67,3 @@ func createWindowFnSpec(

return funcInProgressSpec, outputType, nil
}

// windowers currently cannot maintain the ordering (see #36310).
var windowerMergeOrdering = execinfrapb.Ordering{}

// addRenderingOrProjection checks whether any of the window functions' outputs
// are used in another expression and, if they are, adds rendering to the plan.
// If no rendering is required, it adds a projection to remove all columns that
// were arguments to window functions or were used within OVER clauses.
func addRenderingOrProjection(n *windowNode, planCtx *PlanningCtx, plan *PhysicalPlan) error {
// numWindowFuncsAsIs is the number of window functions output of which is
// used directly (i.e. simply as an output column). Note: the same window
// function might appear multiple times in the query, but its every
// occurrence is replaced by a different windowFuncHolder. For example, on
// query like 'SELECT avg(a) OVER (), avg(a) OVER () + 1 FROM t', only the
// first window function is used "as is."
numWindowFuncsAsIs := 0
for _, render := range n.windowRender {
if _, ok := render.(*windowFuncHolder); ok {
numWindowFuncsAsIs++
}
}
if numWindowFuncsAsIs == len(n.funcs) {
// All window functions' outputs are used directly, so there is no
// rendering to do and simple projection is sufficient.
columns := make([]uint32, len(n.windowRender))
passedThruColIdx := uint32(0)
for i, render := range n.windowRender {
if render == nil {
columns[i] = passedThruColIdx
passedThruColIdx++
} else {
// We have done the type introspection above, so all non-nil renders
// are windowFuncHolders.
holder := render.(*windowFuncHolder)
columns[i] = uint32(holder.outputColIdx)
}
}
plan.AddProjection(columns, windowerMergeOrdering)
return nil
}

// windowNode contains render expressions that might contain:
// 1) IndexedVars that refer to columns by their indices in the full table,
// 2) IndexedVars that replaced regular aggregates that are above
// "windowing level."
// The mapping of both types IndexedVars is stored in n.colAndAggContainer.
renderExprs := make([]tree.TypedExpr, len(n.windowRender))
visitor := replaceWindowFuncsVisitor{
columnsMap: n.colAndAggContainer.idxMap,
}

// All passed through columns are contiguous and at the beginning of the
// output schema.
passedThruColIdx := 0
renderTypes := make([]*types.T, 0, len(n.windowRender))
for i, render := range n.windowRender {
if render != nil {
// render contains at least one reference to windowFuncHolder, so we need
// to walk over the render and replace all windowFuncHolders and (if found)
// IndexedVars using columnsMap and outputColIdx of windowFuncHolders.
renderExprs[i] = visitor.replace(render)
} else {
// render is nil meaning that a column is being passed through.
renderExprs[i] = tree.NewTypedOrdinalReference(passedThruColIdx, plan.GetResultTypes()[passedThruColIdx])
passedThruColIdx++
}
outputType := renderExprs[i].ResolvedType()
renderTypes = append(renderTypes, outputType)
}
return plan.AddRendering(renderExprs, planCtx, plan.PlanToStreamColMap, renderTypes, windowerMergeOrdering)
}

// replaceWindowFuncsVisitor is used to populate render expressions containing
// the results of window functions. It recurses into all expressions except for
// windowFuncHolders (which are replaced by the indices to the corresponding
// output columns) and IndexedVars (which are replaced using columnsMap).
type replaceWindowFuncsVisitor struct {
columnsMap map[int]int
}

var _ tree.Visitor = &replaceWindowFuncsVisitor{}

// VisitPre satisfies the Visitor interface.
func (v *replaceWindowFuncsVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
switch t := expr.(type) {
case *windowFuncHolder:
return false, tree.NewTypedOrdinalReference(t.outputColIdx, t.ResolvedType())
case *tree.IndexedVar:
return false, tree.NewTypedOrdinalReference(v.columnsMap[t.Idx], t.ResolvedType())
}
return true, expr
}

// VisitPost satisfies the Visitor interface.
func (v *replaceWindowFuncsVisitor) VisitPost(expr tree.Expr) tree.Expr {
return expr
}

func (v *replaceWindowFuncsVisitor) replace(typedExpr tree.TypedExpr) tree.TypedExpr {
expr, _ := tree.WalkExpr(v, typedExpr)
return expr.(tree.TypedExpr)
}
8 changes: 8 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/drop_owned_by
Original file line number Diff line number Diff line change
Expand Up @@ -624,3 +624,11 @@ SHOW GRANTS ON DATABASE d3
d3 admin ALL true
d3 public CONNECT false
d3 root ALL true

# Drop owned by should not work if the user has synthetic privileges.

statement ok
GRANT SYSTEM MODIFYCLUSTERSETTING TO testuser

statement error pq: cannot perform drop owned by if role has synthetic privileges; testuser has entries in system.privileges
DROP OWNED BY testuser
7 changes: 2 additions & 5 deletions pkg/sql/opt_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -1086,9 +1086,8 @@ func (ef *execFactory) ConstructProjectSet(
// ConstructWindow is part of the exec.Factory interface.
func (ef *execFactory) ConstructWindow(root exec.Node, wi exec.WindowInfo) (exec.Node, error) {
p := &windowNode{
plan: root.(planNode),
columns: wi.Cols,
windowRender: make([]tree.TypedExpr, len(wi.Cols)),
plan: root.(planNode),
columns: wi.Cols,
}

partitionIdxs := make([]int, len(wi.Partition))
Expand Down Expand Up @@ -1121,8 +1120,6 @@ func (ef *execFactory) ConstructWindow(root exec.Node, wi exec.WindowInfo) (exec
return nil, errors.AssertionFailedf("a RANGE mode frame with an offset bound must have an ORDER BY column")
}
}

p.windowRender[wi.OutputIdxs[i]] = p.funcs[i]
}

return p, nil
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -3225,3 +3225,16 @@ func isCurrentMutationDiscarded(
// Not discarded by any later operation.
return false, descpb.InvalidMutationID
}

// CanPerformDropOwnedBy returns if we can perform DROP OWNED BY in the new
// schema changer. Currently, we do not have an element in the new schema
// changer for system.privileges, thus we cannot properly support drop
// owned by if the user has entries in the system.privileges table.
func (p *planner) CanPerformDropOwnedBy(
ctx context.Context, role username.SQLUsername,
) (bool, error) {
row, err := p.QueryRowEx(ctx, `role-has-synthetic-privileges`, sessiondata.NodeUserSessionDataOverride,
`SELECT count(1) FROM system.privileges WHERE username = $1`, role.Normalized())

return tree.MustBeDInt(row[0]) == 0, err
}
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,12 @@ type SchemaFeatureChecker interface {
// CheckFeature returns if the feature name specified is allowed or disallowed,
// by the database administrator.
CheckFeature(ctx context.Context, featureName tree.SchemaFeatureName) error

// CanPerformDropOwnedBy returns if we can do DROP OWNED BY for the
// given role.
CanPerformDropOwnedBy(
ctx context.Context, role username.SQLUsername,
) (bool, error)
}

// PrivilegeChecker checks an element's privileges.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,8 @@
package scbuildstmt

import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/decodeusername"
Expand All @@ -20,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
)

// DropOwnedBy implements DROP OWNED BY.
Expand All @@ -38,6 +41,17 @@ func DropOwnedBy(b BuildCtx, n *tree.DropOwnedBy) {
if role != b.SessionData().User() && !b.CurrentUserHasAdminOrIsMemberOf(role) {
panic(pgerror.New(pgcode.InsufficientPrivilege, "permission denied to drop objects"))
}
ok, err := b.CanPerformDropOwnedBy(b.EvalCtx().Ctx(), role)
if err != nil {
panic(err)
}
if !ok {
panic(errors.WithHint(pgerror.Newf(
pgcode.FeatureNotSupported,
"cannot perform drop owned by if role has synthetic privileges; %s has entries in system.privileges",
role),
fmt.Sprintf(`perform REVOKE SYSTEM ... for the relevant privileges %s has in system.privileges`, role)))
}
}

var objects []descpb.ID
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -179,6 +180,13 @@ func (s *TestState) CheckFeature(ctx context.Context, featureName tree.SchemaFea
return nil
}

// CanPerformDropOwnedBy implements scbuild.SchemaFeatureCheck.
func (s *TestState) CanPerformDropOwnedBy(
ctx context.Context, role username.SQLUsername,
) (bool, error) {
return true, nil
}

// FeatureChecker implements scbuild.Dependencies
func (s *TestState) FeatureChecker() scbuild.FeatureChecker {
return s
Expand Down
52 changes: 0 additions & 52 deletions pkg/sql/window.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,17 +41,8 @@ type windowNode struct {
// columns is the set of result columns.
columns colinfo.ResultColumns

// A sparse array holding renders specific to this windowNode. This will
// contain nil entries for renders that do not contain window functions,
// and which therefore can be propagated directly from the "wrapped" node.
windowRender []tree.TypedExpr

// The window functions handled by this windowNode.
funcs []*windowFuncHolder

// colAndAggContainer is an IndexedVarContainer that provides indirection
// to migrate IndexedVars and aggregate functions below the windowing level.
colAndAggContainer windowNodeColAndAggContainer
}

func (n *windowNode) startExec(params runParams) error {
Expand Down Expand Up @@ -123,46 +114,3 @@ func (w *windowFuncHolder) Eval(v tree.ExprEvaluator) (tree.Datum, error) {
func (w *windowFuncHolder) ResolvedType() *types.T {
return w.expr.ResolvedType()
}

// windowNodeColAndAggContainer is an IndexedVarContainer providing indirection
// for IndexedVars and aggregation functions found above the windowing level.
// See replaceIndexVarsAndAggFuncs.
type windowNodeColAndAggContainer struct {
// idxMap maps the index of IndexedVars created in replaceIndexVarsAndAggFuncs
// to the index their corresponding results in this container. It permits us to
// add a single render to the source plan per unique expression.
idxMap map[int]int
// sourceInfo contains information on the IndexedVars from the
// source plan where they were originally created.
sourceInfo *colinfo.DataSourceInfo
// aggFuncs maps the index of IndexedVars to their corresponding aggregate function.
aggFuncs map[int]*tree.FuncExpr
// startAggIdx indicates the smallest index to be used by an IndexedVar replacing
// an aggregate function. We don't want to mix these IndexedVars with those
// that replace "original" IndexedVars.
startAggIdx int
}

func (c *windowNodeColAndAggContainer) IndexedVarEval(
idx int, e tree.ExprEvaluator,
) (tree.Datum, error) {
panic("IndexedVarEval should not be called on windowNodeColAndAggContainer")
}

// IndexedVarResolvedType implements the tree.IndexedVarContainer interface.
func (c *windowNodeColAndAggContainer) IndexedVarResolvedType(idx int) *types.T {
if idx >= c.startAggIdx {
return c.aggFuncs[idx].ResolvedType()
}
return c.sourceInfo.SourceColumns[idx].Typ
}

// IndexedVarNodeFormatter implements the tree.IndexedVarContainer interface.
func (c *windowNodeColAndAggContainer) IndexedVarNodeFormatter(idx int) tree.NodeFormatter {
if idx >= c.startAggIdx {
// Avoid duplicating the type annotation by calling .Format directly.
return c.aggFuncs[idx]
}
// Avoid duplicating the type annotation by calling .Format directly.
return c.sourceInfo.NodeFormatter(idx)
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,8 +65,9 @@ export function filterActiveStatements(
}

if (search) {
const searchCaseInsensitive = search.toLowerCase();
filteredStatements = filteredStatements.filter(stmt =>
stmt.query.includes(search),
stmt.query.toLowerCase().includes(searchCaseInsensitive),
);
}

Expand Down Expand Up @@ -211,8 +212,9 @@ export function filterActiveTransactions(
}

if (search) {
filteredTxns = filteredTxns.filter(
txn => !search || txn.query?.includes(search),
const searchCaseInsensitive = search.toLowerCase();
filteredTxns = filteredTxns.filter(txn =>
txn.query?.toLowerCase().includes(searchCaseInsensitive),
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ export const ActiveStatementsView: React.FC<ActiveStatementsViewProps> = ({

const onSubmitSearch = (newSearch: string): void => {
if (newSearch === search) return;
setSearch(search);
setSearch(newSearch);
resetPagination();
};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ export const ActiveTransactionsView: React.FC<ActiveTransactionsViewProps> = ({

const onSubmitSearch = (newSearch: string) => {
if (newSearch === search) return;
setSearch(search);
setSearch(newSearch);
resetPagination();
};

Expand Down
1 change: 1 addition & 0 deletions pkg/util/fast_int_map_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ func TestFastIntMap(t *testing.T) {
{keyRange: 100, valRange: 100},
}
for _, tc := range cases {
tc := tc // necessary since the tests below run in parallel
t.Run(fmt.Sprintf("%d-%d", tc.keyRange, tc.valRange), func(t *testing.T) {
t.Parallel() // SAFE FOR TESTING (this comment is for the linter)
rng, _ := randutil.NewTestRand()
Expand Down

0 comments on commit ba62be7

Please sign in to comment.