Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
84509: cdc: add builtin function to emulate json encoding r=[miretskiy,rafiss,chengxiong-ruan] a=HonoreDB

There's been a [longstanding debugging request](#75730) to be able to see how a row would get encoded by a changefeed with certain options. The main obstacle was removing the need for the builtins package to depend on any package implementing a builtin, which got easier thanks to the introduction of the builtinsregistry. This PR builds on that work to make everything that references builtin functions load-order independent. The first commit removes the load order effect on function OIDs, and the second changes everything that iterates over all builtins to instead register a hook to be called on each builtin as it gets registered. Finally, we add a crdb_internal function that's registered and implemented outside of the builtins package.

Release note (sql change): Added the crdb_internal.to_json_as_changefeed_with_flags function to help
debug json changefeeds. 

Release justification: Improves stability by ensuring that version changes don't alter OIDs and helping with debugging.

86757: build: disable release justification r=kpatron-cockroachlabs a=kpatron-cockroachlabs

Once the release-22.2 branch is cut we can disable release justification
githook check and let blathers justification check take over.

Notes:
* to be merged after the release 22.2 branch cut
* `blathers/release-justification-check` will pass after `STABILITY_PERIOD`
is turned off [as part of the branch cut playbook]

Release note: None
Release justification: internal-only / non-production
/ release-process code change

87196: rules: suppress a dep-rule in special case r=Xiang-Gu a=Xiang-Gu

Commit 1: very minor comment, function location, code changes;
Commit 2: suppress a dep-rule that causes forward incompatibility
for dropping a table with rowLevelTTL in the mixed version state.

Previously, in a mixed version state (v22.1 and v22.2), if a old node
    drops a rowLevelTTL table and the new shcema changer job was adopted
    by a new node, then there is a dep rule that cannot satisfied and hence
    causes forward incompatibility. This PR fixes this by suppressing this
    dep rule if this particular case happens (namely, it suppress the rule
    if `from` is Table element, `to` is a RowLevelTTL element, and there is
    no `PUBLIC` status of the table element anywhere in the graph).

Fixes: #86672

Release note: None

Release justification: fixed a release blocker bug.


87203: sql: make UDF more robust with db/schema renaming r=chengxiong-ruan a=chengxiong-ruan

There are 5 commits:
1. disallow cross-db references in udf. Also a side effect to serialize user defined type names in udf body as type OIDs.
2. check udf dependencies before renaming a db.
3. check dependencies before renaming a schema.
4. deserialize sequence OID for `SHOW CREATE FUNCTION`.
5. deserialize type OID for `SHOW CREATE FUNCTION`.

Release justification: necessary and low risk bug fixes to make UDF more robust with schema/db renaming.

87287: sql,backupccl: allow owners to also control schedules r=benbardin a=adityamaru

This change relaxes the admin only check that was enforced
when resuming, pasuing, dropping and altering a backup schedule
to also allow non-admin owners of the schedules to perform
these control operations.

Release note (sql change): Owners of a backup schedule can now
control their schedule using the supported pause, resume, drop
and alter queries.

Release justification: high impact change to introduce finer grained
permission control to disaster recovery operations

87298: cli: don't show password in \c metacommand r=knz a=rafiss

fixes #87294

Release note (cli change): The \c metacommand no longer shows the
password in plaintext.

Release justification: low risk change

Co-authored-by: Aaron Zinger <[email protected]>
Co-authored-by: Kyle Patron <[email protected]>
Co-authored-by: Xiang Gu <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: adityamaru <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
7 people committed Sep 7, 2022
7 parents 819577e + 2ea997c + daf9a9d + 660efc7 + c400eec + 0e5371e + 887e4d9 commit 99a9587
Show file tree
Hide file tree
Showing 73 changed files with 3,440 additions and 399 deletions.
2 changes: 1 addition & 1 deletion githooks/commit-msg
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ IFS='
notes=($($grep -iE '^release note' "$1"))

# Set this to 1 to require a release justification note.
require_justification=1
require_justification=0
justification=($($grep -iE '^release justification: \S+' "$1"))

IFS=$saveIFS
Expand Down
2 changes: 1 addition & 1 deletion githooks/prepare-commit-msg
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@
#
# Prepare the commit message by adding a release note.

require_justification=1
require_justification=0
set -euo pipefail

if [[ "${2-}" = "message" ]]; then
Expand Down
20 changes: 10 additions & 10 deletions pkg/ccl/backupccl/alter_backup_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@ func loadSchedules(
if err != nil {
return s, err
}

args := &backuppb.ScheduledBackupExecutionArgs{}
if err := pbtypes.UnmarshalAny(schedule.ExecutionArgs().Args, args); err != nil {
return s, errors.Wrap(err, "un-marshaling args")
Expand Down Expand Up @@ -109,8 +108,7 @@ func doAlterBackupSchedules(
eval *alterBackupScheduleEval,
resultsCh chan<- tree.Datums,
) error {
s, err := loadSchedules(
ctx, p, eval)
s, err := loadSchedules(ctx, p, eval)
if err != nil {
return err
}
Expand All @@ -126,14 +124,16 @@ func doAlterBackupSchedules(
s.incJob.ScheduleID())
}

// Note that even ADMIN is subject to these restrictions. We expect to
// add a finer-grained permissions model soon.
if s.fullJob.Owner() != p.User() {
return pgerror.Newf(pgcode.InsufficientPrivilege, "only the OWNER of a schedule may alter it")
// Check that the user is admin or the owner of the schedules being altered.
isAdmin, err := p.UserHasAdminRole(ctx, p.User())
if err != nil {
return err
}

if s.incJob != nil && s.incJob.Owner() != p.User() {
return pgerror.Newf(pgcode.InsufficientPrivilege, "only the OWNER of a schedule may alter it")
isOwnerOfFullJob := s.fullJob == nil || s.fullJob.Owner() == p.User()
isOwnerOfIncJob := s.incJob == nil || s.incJob.Owner() == p.User()
if !isAdmin && !(isOwnerOfFullJob && isOwnerOfIncJob) {
return pgerror.New(pgcode.InsufficientPrivilege, "must be admin or owner of the "+
"schedules being altered.")
}

if s, err = processFullBackupRecurrence(
Expand Down
122 changes: 113 additions & 9 deletions pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
new-server name=s1
----

subtest create-scheduled-privileges

exec-sql
CREATE DATABASE foo;
----
Expand All @@ -15,15 +17,15 @@ CREATE EXTERNAL CONNECTION foo AS 'userfile:///foo';

# Admin can create all schedules.
exec-sql
CREATE SCHEDULE foocluster FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly';
CREATE SCHEDULE foocluster_admin FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly';
----

exec-sql
CREATE SCHEDULE foodb FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly';
CREATE SCHEDULE foodb_admin FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly';
----

exec-sql
CREATE SCHEDULE footable FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly';
CREATE SCHEDULE footable_admin FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly';
----

# Non-root admin can create all schedules.
Expand All @@ -33,15 +35,15 @@ GRANT ADMIN TO testuser;
----

exec-sql user=testuser
CREATE SCHEDULE foocluster FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly';
CREATE SCHEDULE foocluster_admintestuser FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly';
----

exec-sql user=testuser
CREATE SCHEDULE foodb FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly';
CREATE SCHEDULE foodb_admintestuser FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly';
----

exec-sql user=testuser
CREATE SCHEDULE footable FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly';
CREATE SCHEDULE footable_admintestuser FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly';
----

# Non-root non-admin cannot create any schedules.
Expand Down Expand Up @@ -78,13 +80,115 @@ GRANT USAGE ON EXTERNAL CONNECTION foo TO testuser;
----

exec-sql user=testuser
CREATE SCHEDULE foocluster FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly';
CREATE SCHEDULE foocluster_testuser FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly';
----

exec-sql user=testuser
CREATE SCHEDULE foodb FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly';
CREATE SCHEDULE foodb_testuser FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly';
----

exec-sql user=testuser
CREATE SCHEDULE footable FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly';
CREATE SCHEDULE footable_testuser FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly';
----

subtest end

subtest control-schedules-privileges

# Save the schedule IDs for the schedules created by admin root.
let $fullID $incID
with schedules as (show schedules) select id from schedules where label='foocluster_admin' order by command->>'backup_type' asc;
----

query-sql
with schedules as (SHOW SCHEDULES FOR BACKUP) SELECT label, command FROM schedules WHERE id IN
($fullID, $incID) ORDER BY next_run;
----
foocluster_admin BACKUP INTO LATEST IN 'external://foo/cluster' WITH detached
foocluster_admin BACKUP INTO 'external://foo/cluster' WITH detached

# nonadmin testuser is not allowed to drop a schedule they do not own.
exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser
DROP SCHEDULE $fullID
----
regex matches error

exec-sql
GRANT ADMIN TO testuser
----

# testuser has been granted admin so the drop should succeed.
exec-sql user=testuser
DROP SCHEDULE $fullID
----

query-sql
with schedules as (SHOW SCHEDULES FOR BACKUP) SELECT label,command,owner FROM schedules WHERE id IN
($fullID, $incID) ORDER BY next_run;
----
foocluster_admin BACKUP INTO LATEST IN 'external://foo/cluster' WITH detached root

exec-sql
REVOKE ADMIN FROM testuser
----

exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser
DROP SCHEDULE $incID
----
regex matches error

# Save the schedule IDs for the schedules that were created by non-admin testuser.
let $testuserFullID $testuserIncID
with schedules as (show schedules) select id from schedules where label='foocluster_testuser' order by command->>'backup_type' asc;
----

query-sql
with schedules as (SHOW SCHEDULES FOR BACKUP) SELECT label, command, owner FROM schedules WHERE id
IN ($testuserFullID, $testuserIncID) ORDER BY next_run;
----
foocluster_testuser BACKUP INTO LATEST IN 'external://foo/cluster' WITH detached testuser
foocluster_testuser BACKUP INTO 'external://foo/cluster' WITH detached testuser

# testuser owns these schedules so should be able to pause, resume, drop, alter
# them without admin.
exec-sql user=testuser
PAUSE SCHEDULE $testuserFullID;
PAUSE SCHEDULE $testuserIncID;
----

exec-sql user=testuser
RESUME SCHEDULE $testuserFullID;
RESUME SCHEDULE $testuserIncID;
----

exec-sql user=testuser
ALTER BACKUP SCHEDULE $testuserFullID SET WITH revision_history = false;
----

exec-sql user=testuser
DROP SCHEDULE $testuserFullID;
DROP SCHEDULE $testuserIncID;
----

# But testuser can't drop, alter, resume or pause the root owned schedules.
exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to PAUSE it) user=testuser
PAUSE SCHEDULE $incID
----
regex matches error

exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to RESUME it) user=testuser
RESUME SCHEDULE $incID
----
regex matches error

exec-sql user=testuser expect-error-regex=(incremental backup schedule [0-9]+ has no corresponding full backup schedule)
ALTER BACKUP SCHEDULE $incID SET WITH revision_history = false;
----
regex matches error

exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser
DROP SCHEDULE $incID;
----
regex matches error

subtest end
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ go_library(
"//pkg/sql/sem/catid",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/sem/volatility",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sqlutil",
"//pkg/sql/types",
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/changefeedccl/cdcevent/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent",
visibility = ["//visibility:public"],
deps = [
"//pkg/build",
"//pkg/ccl/changefeedccl/changefeedbase",
"//pkg/keys",
"//pkg/kv",
Expand All @@ -34,6 +35,7 @@ go_library(
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/iterutil",
"//pkg/util/log",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
Expand All @@ -57,6 +59,7 @@ go_test(
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/descpb",
Expand Down
29 changes: 29 additions & 0 deletions pkg/ccl/changefeedccl/cdcevent/event.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand All @@ -21,10 +22,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/cache"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)
Expand Down Expand Up @@ -530,3 +533,29 @@ func TestingGetFamilyIDFromKey(
_, familyID, err := decoder.(*eventDecoder).rfCache.tableDescForKey(context.Background(), key, ts)
return familyID, err
}

// MakeRowFromTuple converts a SQL datum produced by, for example, SELECT ROW(foo.*),
// into the same kind of cdcevent.Row you'd get as a result of an insert, but without
// the primary key.
func MakeRowFromTuple(evalCtx *eval.Context, t *tree.DTuple) Row {
r := Projection{EventDescriptor: &EventDescriptor{}}
names := t.ResolvedType().TupleLabels()
for i, d := range t.D {
var name string
if names == nil {
name = fmt.Sprintf("col%d", i+1)
} else {
name = names[i]
}
r.AddValueColumn(name, d.ResolvedType())
if err := r.SetValueDatumAt(evalCtx, i, d); err != nil {
if build.IsRelease() {
log.Warningf(context.Background(), "failed to set row value from tuple due to error %v", err)
_ = r.SetValueDatumAt(evalCtx, i, tree.DNull)
} else {
panic(err)
}
}
}
return Row(r)
}
61 changes: 61 additions & 0 deletions pkg/ccl/changefeedccl/cdcevent/event_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -397,3 +400,61 @@ func slurpDatums(t *testing.T, it Iterator) (res []string) {
}))
return res
}

func TestMakeRowFromTuple(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

i := tree.NewDInt(1234)
f := tree.NewDFloat(12.34)
s := tree.NewDString("testing")
typ := types.MakeTuple([]*types.T{types.Int, types.Float, types.String})
unlabeledTuple := tree.NewDTuple(typ, i, f, s)
st := cluster.MakeTestingClusterSettings()
evalCtx := eval.MakeTestingEvalContext(st)

rowFromUnlabeledTuple := MakeRowFromTuple(&evalCtx, unlabeledTuple)
expectedCols := []struct {
name string
typ *types.T
valAsString string
}{
{name: "col1", typ: types.Int, valAsString: "1234"},
{name: "col2", typ: types.Float, valAsString: "12.34"},
{name: "col3", typ: types.String, valAsString: "testing"},
}

remainingCols := expectedCols

require.NoError(t, rowFromUnlabeledTuple.ForEachColumn().Datum(func(d tree.Datum, col ResultColumn) error {
current := remainingCols[0]
remainingCols = remainingCols[1:]
require.Equal(t, current.name, col.Name)
require.Equal(t, current.typ, col.Typ)
require.Equal(t, current.valAsString, tree.AsStringWithFlags(d, tree.FmtExport))
return nil
}))

require.Empty(t, remainingCols)

typ.InternalType.TupleLabels = []string{"a", "b", "c"}
labeledTuple := tree.NewDTuple(typ, i, f, s)

expectedCols[0].name = "a"
expectedCols[1].name = "b"
expectedCols[2].name = "c"

remainingCols = expectedCols

rowFromLabeledTuple := MakeRowFromTuple(&evalCtx, labeledTuple)

require.NoError(t, rowFromLabeledTuple.ForEachColumn().Datum(func(d tree.Datum, col ResultColumn) error {
current := remainingCols[0]
remainingCols = remainingCols[1:]
require.Equal(t, current.name, col.Name)
require.Equal(t, current.typ, col.Typ)
require.Equal(t, current.valAsString, tree.AsStringWithFlags(d, tree.FmtExport))
return nil
}))

}
Loading

0 comments on commit 99a9587

Please sign in to comment.