Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
85007: privilege,externalconn: add EXTERNALCONNECTION system privilege r=benbardin a=adityamaru

This change introduces a `EXTERNALCONNECTION` system privilege.
This privilege is required by a user to be able to create an external
connection. Root and admin have these system privileges by default, and
are capable of granting this privilege to other users/roles with or without
the grant option.

Fixes: #85006

Release note (sql change): introduce a `EXTERNALCONNECTION`
system privilege that is required to create an External Connection object
to represent an underlying resource.

85037: ui: remove unused end-to-end UI tests r=nathanstilwell a=sjbarag

The end-to-end UI tests written in Cypress weren't updated since October
2021, and haven't been run in at least as long. They in fact no longer
seemed to execute, and there were six major-version releases of
Cypress since the one used in this project. There had been enough skew
in both the CockroachDB product and in the Cypress platform that
upgrading to a newer version in-place is more work than simply tearing
out the old tests and starting from scratch. Remove the existing
db-console Cypress tests in preparation for a from-scratch
reimplementation.

Release note: None

85040: roachtest: preserve vmodule settings for upgraded servers r=lidorcarmel a=lidorcarmel

Without this patch upgraded servers in a roachtest are using the default
settings. For example, the rebalance roachtest is upgrading servers in
order to run with mixed version binaries, and those servers are running
without the right settings for logging (don't use vmodule).

This small fix makes sure that we pass the right options to the upgraded
binary, so that for the rebalance roachtest we will maintain the 'vmodule'
settings.

Release note: None

85071: kvserver: forward-port deflake of TestAddReplicaViaLearner r=erikgrinaker a=itsbilal

In #84223, we noticed that TestAddReplicaViaLearner was
flaky on CI in release-22.1, for something that the test
was not testing. This change forward-ports the fix that
worked there.

Release note: None.

Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Sean Barag <[email protected]>
Co-authored-by: Lidor Carmel <[email protected]>
Co-authored-by: Bilal Akhtar <[email protected]>
  • Loading branch information
5 people committed Jul 26, 2022
5 parents d7b901d + 10fe096 + 3d171b2 + 2c40767 + 4b23525 commit c149ddb
Show file tree
Hide file tree
Showing 44 changed files with 202 additions and 1,344 deletions.
5 changes: 0 additions & 5 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -1931,8 +1931,3 @@ build/variables.mk: Makefile build/archive/contents/Makefile pkg/ui/Makefile bui
include build/variables.mk
$(foreach v,$(filter-out $(strip $(VALID_VARS)),$(.VARIABLES)),\
$(if $(findstring command line,$(origin $v)),$(error Variable '$v' is not recognized by this Makefile)))

# Cypress e2e tests
.PHONY: db-console-e2e-test
db-console-e2e-test: pkg/ui/yarn.opt.installed
cd pkg/ui/workspaces/db-console && yarn cypress:run
6 changes: 6 additions & 0 deletions pkg/ccl/cloudccl/externalconn/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,12 @@ func TestDataDriven(t *testing.T) {
externalConnTestCluster.InitializeTenant(ctx, tenantID)

case "exec-sql":
if d.HasArg("user") {
var user string
d.ScanArgs(t, "user", &user)
resetToRootUser := externalConnTestCluster.SetSQLDBForUser(tenantID, user)
defer resetToRootUser()
}
if err := tenant.ExecWithErr(d.Input); err != nil {
return fmt.Sprint(err.Error())
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,3 +45,37 @@ inspect-system-table
----

subtest end

subtest create-external-connection-global-privilege

exec-sql
CREATE USER testuser;
----

exec-sql user=testuser
CREATE EXTERNAL CONNECTION privileged AS 'nodelocal://1/foo'
----
pq: only users with the EXTERNALCONNECTION system privilege are allowed to CREATE EXTERNAL CONNECTION

exec-sql
GRANT SYSTEM EXTERNALCONNECTION TO testuser;
----

exec-sql user=testuser
CREATE EXTERNAL CONNECTION privileged AS 'nodelocal://1/foo'
----

inspect-system-table
----
privileged STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/foo"}}}

exec-sql
REVOKE SYSTEM EXTERNALCONNECTION FROM testuser;
----

exec-sql user=testuser
CREATE EXTERNAL CONNECTION privileged AS 'nodelocal://1/foo'
----
pq: only users with the EXTERNALCONNECTION system privilege are allowed to CREATE EXTERNAL CONNECTION

subtest end
Original file line number Diff line number Diff line change
Expand Up @@ -48,3 +48,37 @@ inspect-system-table
----

subtest end

subtest create-external-connection-global-privilege

exec-sql
CREATE USER testuser;
----

exec-sql user=testuser
CREATE EXTERNAL CONNECTION privileged AS 'nodelocal://1/foo'
----
pq: only users with the EXTERNALCONNECTION system privilege are allowed to CREATE EXTERNAL CONNECTION

exec-sql
GRANT SYSTEM EXTERNALCONNECTION TO testuser;
----

exec-sql user=testuser
CREATE EXTERNAL CONNECTION privileged AS 'nodelocal://1/foo'
----

inspect-system-table
----
privileged STORAGE {"nodelocal": {"cfg": {"nodeId": 1, "path": "/foo"}}}

exec-sql
REVOKE SYSTEM EXTERNALCONNECTION FROM testuser;
----

exec-sql user=testuser
CREATE EXTERNAL CONNECTION privileged AS 'nodelocal://1/foo'
----
pq: only users with the EXTERNALCONNECTION system privilege are allowed to CREATE EXTERNAL CONNECTION

subtest end
5 changes: 4 additions & 1 deletion pkg/cloud/externalconn/record.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,9 +183,12 @@ func (e *ExternalConnection) Create(
return err
}

// CREATE EXTERNAL CONNECTION is only allowed for users with the
// `EXTERNALCONNECTION` system privilege. We run the query as `node`
// since the user might not have `INSERT` on the system table.
createQuery := "INSERT INTO system.external_connections (%s) VALUES (%s) RETURNING connection_name"
row, retCols, err := ex.QueryRowExWithCols(ctx, "ExternalConnection.Create", txn,
sessiondata.InternalExecutorOverride{User: user},
sessiondata.InternalExecutorOverride{User: username.NodeUserName()},
fmt.Sprintf(createQuery, strings.Join(cols, ","), generatePlaceholders(len(qargs))),
qargs...,
)
Expand Down
57 changes: 50 additions & 7 deletions pkg/cloud/externalconn/utils/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,15 +42,55 @@ func NewHandle(t *testing.T, tc *testcluster.TestCluster) *Handle {
}
}

// SetSQLDBForUser sets the tenants' SQL runner to a PGURL connection using the
// passed in `user`. The method returns a function that resets the tenants' SQL
// runner to a PGURL connection for the root user.
func (h *Handle) SetSQLDBForUser(tenantID roachpb.TenantID, user string) func() {
tenantState, ok := h.ts[tenantID]
if !ok {
h.t.Fatalf("tenant ID %d has not been initialized", tenantID)
}

resetToRootUser := func() {
tenantState.curDB = tenantState.userToDB[username.RootUserName().Normalized()]
}

if runner, ok := tenantState.userToDB[user]; ok {
tenantState.curDB = runner
return resetToRootUser
}

pgURL, cleanup := sqlutils.PGUrl(h.t, h.tc.Server(0).ServingSQLAddr(),
"TestBackupRestoreDataDriven", url.User(user))
userSQLDB, err := gosql.Open("postgres", pgURL.String())
require.NoError(h.t, err)
tenantState.curDB = sqlutils.MakeSQLRunner(userSQLDB)
tenantState.userToDB[user] = tenantState.curDB
tenantState.cleanupFns = append(tenantState.cleanupFns, func() {
require.NoError(h.t, userSQLDB.Close())
cleanup()
})

return resetToRootUser
}

// InitializeTenant initializes a tenant with the given ID, returning the
// relevant tenant state.
func (h *Handle) InitializeTenant(ctx context.Context, tenID roachpb.TenantID) {
testServer := h.tc.Server(0)
tenantState := &Tenant{t: h.t}
tenantState := &Tenant{t: h.t, userToDB: make(map[string]*sqlutils.SQLRunner)}
if tenID == roachpb.SystemTenantID {
tenantState.TestTenantInterface = testServer
tenantState.db = sqlutils.MakeSQLRunner(h.tc.ServerConn(0))
tenantState.cleanup = func() {} // noop
pgURL, cleanupPGUrl := sqlutils.PGUrl(h.t, tenantState.SQLAddr(), "System",
url.User(username.RootUser))
userSQLDB, err := gosql.Open("postgres", pgURL.String())
require.NoError(h.t, err)
tenantState.curDB = sqlutils.MakeSQLRunner(userSQLDB)
tenantState.userToDB[username.RootUserName().Normalized()] = tenantState.curDB
tenantState.cleanupFns = append(tenantState.cleanupFns, func() {
require.NoError(h.t, userSQLDB.Close())
cleanupPGUrl()
})
} else {
tenantArgs := base.TestTenantArgs{
TenantID: tenID,
Expand All @@ -63,11 +103,12 @@ func (h *Handle) InitializeTenant(ctx context.Context, tenID roachpb.TenantID) {
tenantSQLDB, err := gosql.Open("postgres", pgURL.String())
require.NoError(h.t, err)

tenantState.db = sqlutils.MakeSQLRunner(tenantSQLDB)
tenantState.cleanup = func() {
tenantState.curDB = sqlutils.MakeSQLRunner(tenantSQLDB)
tenantState.userToDB[username.RootUserName().Normalized()] = tenantState.curDB
tenantState.cleanupFns = append(tenantState.cleanupFns, func() {
require.NoError(h.t, tenantSQLDB.Close())
cleanupPGUrl()
}
})
}

h.ts[tenID] = tenantState
Expand All @@ -91,7 +132,9 @@ func (h *Handle) Tenants() []*Tenant {
// Cleanup frees up internal resources.
func (h *Handle) Cleanup() {
for _, tenantState := range h.ts {
tenantState.cleanup()
for _, cleanup := range tenantState.cleanupFns {
cleanup()
}
}
h.ts = nil
}
15 changes: 8 additions & 7 deletions pkg/cloud/externalconn/utils/tenant_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,30 +24,31 @@ import (
type Tenant struct {
serverutils.TestTenantInterface

t *testing.T
db *sqlutils.SQLRunner
cleanup func()
t *testing.T
userToDB map[string]*sqlutils.SQLRunner
curDB *sqlutils.SQLRunner
cleanupFns []func()
}

// Exec is a wrapper around gosql.Exec that kills the test on error. It records
// the execution timestamp for subsequent use.
func (s *Tenant) Exec(query string, args ...interface{}) {
s.db.Exec(s.t, query, args...)
s.curDB.Exec(s.t, query, args...)
}

// ExecWithErr is like Exec but returns the error, if any. It records the
// execution timestamp for subsequent use.
func (s *Tenant) ExecWithErr(query string, args ...interface{}) error {
_, err := s.db.DB.ExecContext(context.Background(), query, args...)
_, err := s.curDB.DB.ExecContext(context.Background(), query, args...)
return err
}

// Query is a wrapper around gosql.Query that kills the test on error.
func (s *Tenant) Query(query string, args ...interface{}) *gosql.Rows {
return s.db.Query(s.t, query, args...)
return s.curDB.Query(s.t, query, args...)
}

// QueryWithErr is like Query but returns the error.
func (s *Tenant) QueryWithErr(query string, args ...interface{}) (*gosql.Rows, error) {
return s.db.DB.QueryContext(context.Background(), query, args...)
return s.curDB.DB.QueryContext(context.Background(), query, args...)
}
7 changes: 4 additions & 3 deletions pkg/cmd/roachtest/tests/follower_reads.go
Original file line number Diff line number Diff line change
Expand Up @@ -877,15 +877,16 @@ func runFollowerReadsMixedVersionSingleRegionTest(
// Start the cluster at the old version.
settings := install.MakeClusterSettings()
settings.Binary = uploadVersion(ctx, t, c, c.All(), predecessorVersion)
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.All())
startOpts := option.DefaultStartOpts()
c.Start(ctx, t.L(), startOpts, settings, c.All())
topology := topologySpec{multiRegion: false}
data := initFollowerReadsDB(ctx, t, c, topology)

// Upgrade one node to the new version and run the test.
randNode := 1 + rand.Intn(c.Spec().NodeCount)
t.L().Printf("upgrading n%d to current version", randNode)
nodeToUpgrade := c.Node(randNode)
upgradeNodes(ctx, nodeToUpgrade, curVersion, t, c)
upgradeNodes(ctx, nodeToUpgrade, startOpts, curVersion, t, c)
runFollowerReadsTest(ctx, t, c, topologySpec{multiRegion: false}, exactStaleness, data)

// Upgrade the remaining nodes to the new version and run the test.
Expand All @@ -897,6 +898,6 @@ func runFollowerReadsMixedVersionSingleRegionTest(
remainingNodes = remainingNodes.Merge(c.Node(i + 1))
}
t.L().Printf("upgrading nodes %s to current version", remainingNodes)
upgradeNodes(ctx, remainingNodes, curVersion, t, c)
upgradeNodes(ctx, remainingNodes, startOpts, curVersion, t, c)
runFollowerReadsTest(ctx, t, c, topologySpec{multiRegion: false}, exactStaleness, data)
}
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/rebalance_load.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ func registerRebalanceLoad(r registry.Registry) {
// `cockroach` flag will be used.
const newVersion = ""
c.Start(ctx, t.L(), startOpts, settings, roachNodes)
upgradeNodes(ctx, nodesToUpgrade, newVersion, t, c)
upgradeNodes(ctx, nodesToUpgrade, startOpts, newVersion, t, c)
} else {
c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes)
c.Start(ctx, t.L(), startOpts, settings, roachNodes)
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,7 +373,7 @@ func uploadAndStart(nodes option.NodeListOption, v string) versionStep {
// Use a waitForUpgradeStep() for that.
func binaryUpgradeStep(nodes option.NodeListOption, newVersion string) versionStep {
return func(ctx context.Context, t test.Test, u *versionUpgradeTest) {
upgradeNodes(ctx, nodes, newVersion, t, u.c)
upgradeNodes(ctx, nodes, option.DefaultStartOpts(), newVersion, t, u.c)
// TODO(nvanbenschoten): add upgrade qualification step. What should we
// test? We could run logictests. We could add custom logic here. Maybe
// this should all be pushed to nightly migration tests instead.
Expand All @@ -383,6 +383,7 @@ func binaryUpgradeStep(nodes option.NodeListOption, newVersion string) versionSt
func upgradeNodes(
ctx context.Context,
nodes option.NodeListOption,
startOpts option.StartOpts,
newVersion string,
t test.Test,
c cluster.Cluster,
Expand Down Expand Up @@ -411,7 +412,6 @@ func upgradeNodes(

binary := uploadVersion(ctx, t, c, c.Node(node), newVersion)
settings := install.MakeClusterSettings(install.BinaryOption(binary))
startOpts := option.DefaultStartOpts()
c.Start(ctx, t.L(), startOpts, settings, c.Node(node))
}
}
Expand Down
8 changes: 7 additions & 1 deletion pkg/kv/kvserver/replica_learner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,10 +146,16 @@ func TestAddReplicaViaLearner(t *testing.T) {
// The happy case! \o/

blockUntilSnapshotCh := make(chan struct{})
var receivedSnap int64
blockSnapshotsCh := make(chan struct{})
knobs, ltk := makeReplicationTestKnobs()
ltk.storeKnobs.ReceiveSnapshot = func(h *kvserverpb.SnapshotRequest_Header) error {
close(blockUntilSnapshotCh)
if atomic.CompareAndSwapInt64(&receivedSnap, 0, 1) {
close(blockUntilSnapshotCh)
} else {
// Do nothing. We aren't interested in subsequent snapshots.
return nil
}
select {
case <-blockSnapshotsCh:
case <-time.After(10 * time.Second):
Expand Down
13 changes: 5 additions & 8 deletions pkg/sql/create_external_connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -60,16 +62,11 @@ func (p *planner) makeExternalConnectionEval(
func (p *planner) createExternalConnection(
params runParams, n *tree.CreateExternalConnection,
) error {
// TODO(adityamaru): Check that the user has `CREATEEXTERNALCONNECTION` global
// privilege once we add support for it. Remove admin only check.
hasAdmin, err := params.p.HasAdminRole(params.ctx)
if err != nil {
return err
}
if !hasAdmin {
if err := params.p.CheckPrivilege(params.ctx, syntheticprivilege.GlobalPrivilegeObject,
privilege.EXTERNALCONNECTION); err != nil {
return pgerror.New(
pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to CREATE EXTERNAL CONNECTION")
"only users with the EXTERNALCONNECTION system privilege are allowed to CREATE EXTERNAL CONNECTION")
}

// TODO(adityamaru): Add some metrics to track CREATE EXTERNAL CONNECTION
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
"//pkg/base",
"//pkg/build",
"//pkg/build/bazel",
"//pkg/cloud/externalconn/providers",
"//pkg/clusterversion",
"//pkg/kv/kvserver",
"//pkg/roachpb",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/build/bazel"
_ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/providers" // imported to register ExternalConnection providers
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down
Loading

0 comments on commit c149ddb

Please sign in to comment.