Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
75854: sql: update SHOW GRANTS ON DATABASE to include grant options r=RichardJCai a=ecwall

refs #73394

Release note (sql): SHOW GRANTS ON DATABASE includes is_grantable column

75892: distsql: cleanup log tag annotation r=andreimatei a=andreimatei

A context annotation was not doing what it wanted; it was inadvertently
a no-op.

This patch also makes FlowCtx.AmbientCtx not be embedded. Embedded
AmbientCtxs are annoying because one can't easily find where they're
used.

Release note: None

75901: distsql: improve flow cleanup r=andreimatei a=andreimatei

FlowBase.Cleanup(ctx) was closing the tracing span from ctx, if any.
This was assuming that the respective ctx is the one associated with the
flow, and that the flow always has a span. These are brittle
assumptions; in particular, I intend to avoid opening spans for local
flows when there's no particular reason to (i.e. when not collecting
stats), to save on their cost.

This patch has the Flow explicitly keep track of which Span it owns, if
any.

Release note: None

75910: migration/migrations: skip TestEnsureSpanConfigReconciliation r=irfansharif a=otan

Refs: #75849

Reason: flaky test

Generated by bin/skip-test.

Release justification: non-production code changes

Release note: None

Co-authored-by: Evan Wall <[email protected]>
Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
4 people committed Feb 3, 2022
5 parents a950be5 + 960c177 + dbdab60 + 2e6a4f3 + b81accf commit 7499c27
Show file tree
Hide file tree
Showing 17 changed files with 222 additions and 197 deletions.
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/restore_old_versions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -439,9 +439,9 @@ func restoreV201ZoneconfigPrivilegeTest(exportDir string) func(t *testing.T) {
require.NoError(t, err)
sqlDB.Exec(t, `RESTORE FROM $1`, LocalFoo)
testDBGrants := [][]string{
{"test", "admin", "ALL"},
{"test", "root", "ALL"},
{"test", "testuser", "ZONECONFIG"},
{"test", "admin", "ALL", "true"},
{"test", "root", "ALL", "true"},
{"test", "testuser", "ZONECONFIG", "false"},
}
sqlDB.CheckQueryResults(t, `show grants on database test`, testDBGrants)

Expand Down
22 changes: 11 additions & 11 deletions pkg/ccl/backupccl/testdata/backup-restore/restore-grants
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ CREATE TABLE testdb.testtable_greeting_owner (a testdb.greeting_owner);
query-sql
SHOW GRANTS ON DATABASE testdb FOR user1;
----
testdb user1 ALL
testdb user1 ALL true

query-sql
SHOW GRANTS ON SCHEMA public FOR user1;
Expand All @@ -76,7 +76,7 @@ SHOW GRANTS ON TABLE testdb.testtable_simple FOR user1;
query-sql
SHOW GRANTS ON DATABASE testdb FOR testuser;
----
testdb testuser ALL
testdb testuser ALL true

query-sql
SHOW GRANTS ON SCHEMA public FOR testuser;
Expand Down Expand Up @@ -155,10 +155,10 @@ RESTORE testdb.sc.othertable, testdb.testtable_greeting_usage FROM 'nodelocal://
query-sql
SHOW GRANTS ON DATABASE testuser_db;
----
testuser_db admin ALL
testuser_db public CONNECT
testuser_db root ALL
testuser_db testuser CREATE
testuser_db admin ALL true
testuser_db public CONNECT false
testuser_db root ALL true
testuser_db testuser CREATE false

query-sql
SHOW GRANTS ON SCHEMA public;
Expand Down Expand Up @@ -296,7 +296,7 @@ SHOW GRANTS ON testdb.sc.othertable FOR testuser;
query-sql
SHOW GRANTS ON DATABASE testdb FOR admin;
----
testdb admin ALL
testdb admin ALL true

query-sql
SHOW GRANTS ON SCHEMA testdb.public FOR admin;
Expand Down Expand Up @@ -387,7 +387,7 @@ SHOW GRANTS ON testdb.sc.othertable FOR testuser;
query-sql
SHOW GRANTS ON DATABASE testdb FOR admin;
----
testdb admin ALL
testdb admin ALL true

query-sql
SHOW GRANTS ON SCHEMA testdb.public FOR admin;
Expand Down Expand Up @@ -433,7 +433,7 @@ RESTORE FROM 'nodelocal://0/test/';
query-sql
SHOW GRANTS ON DATABASE testdb FOR user1;
----
testdb user1 ALL
testdb user1 ALL true

query-sql
SHOW GRANTS ON SCHEMA testdb.public FOR user1;
Expand All @@ -458,7 +458,7 @@ SHOW GRANTS ON TABLE testdb.testtable_simple FOR user1;
query-sql
SHOW GRANTS ON DATABASE testdb FOR testuser;
----
testdb testuser ALL
testdb testuser ALL true

query-sql
SHOW GRANTS ON SCHEMA testdb.public FOR testuser;
Expand Down Expand Up @@ -496,7 +496,7 @@ ALTER TYPE testdb.greeting_owner ADD VALUE 'new' BEFORE 'howdy';
query-sql
SHOW GRANTS ON DATABASE testdb FOR admin;
----
testdb admin ALL
testdb admin ALL true

query-sql
SHOW GRANTS ON SCHEMA testdb.public FOR admin;
Expand Down
2 changes: 2 additions & 0 deletions pkg/migration/migrations/migrate_span_configs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand All @@ -34,6 +35,7 @@ import (
// span config reconciliation attempt, blocking until it occurs.
func TestEnsureSpanConfigReconciliation(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.WithIssue(t, 75849, "flaky test")
defer log.Scope(t).Close(t)

ctx := context.Background()
Expand Down
12 changes: 11 additions & 1 deletion pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/jobs"
Expand Down Expand Up @@ -4569,7 +4570,8 @@ var crdbInternalClusterDatabasePrivilegesTable = virtualSchemaTable{
CREATE TABLE crdb_internal.cluster_database_privileges (
database_name STRING NOT NULL,
grantee STRING NOT NULL,
privilege_type STRING NOT NULL
privilege_type STRING NOT NULL,
is_grantable STRING
)`,
populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
return forEachDatabaseDesc(ctx, p, dbContext, true, /* requiresPrivileges */
Expand All @@ -4578,13 +4580,21 @@ CREATE TABLE crdb_internal.cluster_database_privileges (
dbNameStr := tree.NewDString(db.GetName())
// TODO(knz): This should filter for the current user, see
// https://github.com/cockroachdb/cockroach/issues/35572
populateGrantOption := p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.ValidateGrantOption)
for _, u := range privs {
userNameStr := tree.NewDString(u.User.Normalized())
for _, priv := range u.Privileges {
var isGrantable tree.Datum
if populateGrantOption {
isGrantable = yesOrNoDatum(priv.GrantOption)
} else {
isGrantable = tree.DNull
}
if err := addRow(
dbNameStr, // database_name
userNameStr, // grantee
tree.NewDString(priv.Kind.String()), // privilege_type
isGrantable, // is_grantable
); err != nil {
return err
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/delegate/show_grants.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ func (d *delegator) delegateShowGrants(n *tree.ShowGrants) (tree.Statement, erro
const dbPrivQuery = `
SELECT database_name,
grantee,
privilege_type
privilege_type,
is_grantable::boolean
FROM "".crdb_internal.cluster_database_privileges`
const schemaPrivQuery = `
SELECT table_catalog AS database_name,
Expand Down
11 changes: 6 additions & 5 deletions pkg/sql/distsql/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -376,7 +376,7 @@ func (ds *ServerImpl) setupFlow(
// to restore the original value which can have data races under stress.
isVectorized := req.EvalContext.SessionData.VectorizeMode != sessiondatapb.VectorizeOff
f := newFlow(
flowCtx, ds.flowRegistry, rowSyncFlowConsumer, batchSyncFlowConsumer,
flowCtx, sp, ds.flowRegistry, rowSyncFlowConsumer, batchSyncFlowConsumer,
localState.LocalProcs, isVectorized, onFlowCleanup, req.StatementSQL,
)
opt := flowinfra.FuseNormally
Expand All @@ -397,8 +397,8 @@ func (ds *ServerImpl) setupFlow(
return ctx, nil, nil, err
}
if !f.IsLocal() {
flowCtx.AddLogTag("f", f.GetFlowCtx().ID.Short())
flowCtx.AnnotateCtx(ctx)
flowCtx.AmbientContext.AddLogTag("f", f.GetFlowCtx().ID.Short())
ctx = flowCtx.AmbientContext.AnnotateCtx(ctx)
telemetry.Inc(sqltelemetry.DistSQLExecCounter)
}
if f.IsVectorized() {
Expand Down Expand Up @@ -493,6 +493,7 @@ func (ds *ServerImpl) newFlowContext(

func newFlow(
flowCtx execinfra.FlowCtx,
sp *tracing.Span,
flowReg *flowinfra.FlowRegistry,
rowSyncFlowConsumer execinfra.RowReceiver,
batchSyncFlowConsumer execinfra.BatchReceiver,
Expand All @@ -501,7 +502,7 @@ func newFlow(
onFlowCleanup func(),
statementSQL string,
) flowinfra.Flow {
base := flowinfra.NewFlowBase(flowCtx, flowReg, rowSyncFlowConsumer, batchSyncFlowConsumer, localProcessors, onFlowCleanup, statementSQL)
base := flowinfra.NewFlowBase(flowCtx, sp, flowReg, rowSyncFlowConsumer, batchSyncFlowConsumer, localProcessors, onFlowCleanup, statementSQL)
if isVectorized {
return colflow.NewVectorizedFlow(base)
}
Expand Down Expand Up @@ -669,7 +670,7 @@ func (ds *ServerImpl) flowStreamInt(
}
defer cleanup()
log.VEventf(ctx, 1, "connected inbound stream %s/%d", flowID.Short(), streamID)
return streamStrategy.Run(f.AnnotateCtx(ctx), stream, msg, f)
return streamStrategy.Run(f.AmbientContext.AnnotateCtx(ctx), stream, msg, f)
}

// FlowStream is part of the execinfrapb.DistSQLServer interface.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/distsql/vectorized_panic_propagation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) {
}
base := flowinfra.NewFlowBase(
flowCtx,
nil, /* sp */
nil, /* flowReg */
nil, /* rowSyncFlowConsumer */
nil, /* batchSyncFlowConsumer */
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/execinfra/flow_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
// FlowCtx encompasses the configuration parameters needed for various flow
// components.
type FlowCtx struct {
log.AmbientContext
AmbientContext log.AmbientContext

Cfg *ServerConfig

Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/flowinfra/flow.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,10 @@ type FlowBase struct {
ctxCancel context.CancelFunc
ctxDone <-chan struct{}

// sp is the span that this Flow runs in. Can be nil if no span was created
// for the flow. Flow.Cleanup() finishes it.
sp *tracing.Span

// spec is the request that produced this flow. Only used for debugging.
spec *execinfrapb.FlowSpec

Expand Down Expand Up @@ -246,8 +250,12 @@ func (f *FlowBase) Started() bool {
var _ Flow = &FlowBase{}

// NewFlowBase creates a new FlowBase.
//
// sp, if not nil, is the Span corresponding to the flow. The flow takes
// ownership; Cleanup() will finish it.
func NewFlowBase(
flowCtx execinfra.FlowCtx,
sp *tracing.Span,
flowReg *FlowRegistry,
rowSyncFlowConsumer execinfra.RowReceiver,
batchSyncFlowConsumer execinfra.BatchReceiver,
Expand All @@ -270,6 +278,7 @@ func NewFlowBase(
}
return &FlowBase{
FlowCtx: flowCtx,
sp: sp,
flowRegistry: flowReg,
rowSyncFlowConsumer: rowSyncFlowConsumer,
batchSyncFlowConsumer: batchSyncFlowConsumer,
Expand Down Expand Up @@ -497,15 +506,14 @@ func (f *FlowBase) Cleanup(ctx context.Context) {
f.Descriptors.ReleaseAll(ctx)
}

sp := tracing.SpanFromContext(ctx)
if sp != nil {
defer sp.Finish()
if f.sp != nil {
defer f.sp.Finish()
if f.Gateway && f.CollectStats {
// If this is the gateway node and we're collecting execution stats,
// output the maximum memory usage to the flow span. Note that
// non-gateway nodes use the last outbox to send this information
// over.
sp.RecordStructured(&execinfrapb.ComponentStats{
f.sp.RecordStructured(&execinfrapb.ComponentStats{
Component: execinfrapb.FlowComponentID(f.NodeID.SQLInstanceID(), f.FlowCtx.ID),
FlowStats: execinfrapb.FlowStats{
MaxMemUsage: optional.MakeUint(uint64(f.FlowCtx.EvalCtx.Mon.MaximumBytes())),
Expand Down
Loading

0 comments on commit 7499c27

Please sign in to comment.