Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
93906: sql/schemachanger: create function declarative r=chengxiong-ruan a=chengxiong-ruan

**sql/schemachanger: rename UpdateBackReferencesInSequences**

  Rename UpdateBackReferencesInSequences to
  UpdateTableBackReferencesInSequences since it's fits more
  specifically for tables referencing sequences.

**sql/schemachanger: resolve schema should check required privilege**

  Previously, we require current user's ownership on a schema
  descriptor when resolving a schema in declarative shcema
  changer. This was true because we only support `DROP SCHEMA`.
  But this won't be true if we want to support `CREATE`s because
  we'd only require `CREATE` privilege on the schema. This pr
  adds a new `RequireOwnership` flag to `ResolveParam` since there
  is no `OWNER` privilege kind.

**sql/schemachanger: Add default privileges elements and grant option field**

  This commit adds a new element for default privileges. Also adds
  a grant option field to UserPrivielges element.

**sql/schemachanger: add id generator and references provider interface**

  This commit adds `ReferenceProviderFactory` and `DescIDGenerator`
  interfaces to help doing CREATEs within declarative schema changer.

**sql/schemachanger: make builder able to work with new descriptors**

  Previously, builder always try to read descriptors because we
  assume all descriptors already exist. This is not true anymore
  for `CREATE` statements. This commit adds a new id set to the
  builder so it knows when to read a descriptor to initilize the
  cache or not.

**sql/schemachanger: create function declarative**

  This commits implements CREATE FUNCTION statements in the
  declarative schema changer.

  It introduces a few new things:
  (1) A new `DESCRIPTOR_ADDING` state for a newly created descritpor.
  In this state the descriptor is visible only in a cache of
  `mutationVisitorState` so that other operations in the same statement
  can see the new descriptor. At this state, descriptor is not written
  to collection and storage yet.
  (2) Introduce operations to update `ObjectParent`, `Owner` and
  `UserPrivileges` elements.
  (3) add helper to able to create user privileges from default privileges
  of database and schemas.

  The rest of the commit is mainly implementing the business logic.

Fixes: #83228

96233: rpc,*: don't panic in `rpc.NewServerEx` r=erikgrinaker a=knz

Found while working on #96226.

Prior to this patch, the function `rpc.NewServerEx()` would panic if it was unable to loads its TLS settings. This is incorrect - we want to report a simple operational error in that case, since the problem is actionable.

The reason why this problem does not seem to directly impact production clusters is that there is another check in `server.NewServer()` that asserts the TLS settings are valid before constructing the rpc server.
This check was incomplete however, because there is a small race condition in between the two where the files could be removed after `server.NewServer()` finds them, and before `rpc.NewServer()` needs them.

We do not generally like `panic()` calls in production code, so this needed an improvement anyway.

Release note: None
Epic: CRDB-14537

96271: logictest: skip flaky test in ranges r=mgartner a=mgartner

Informs #96136

Epic: None

Release note: None

96293: schemachanger: disable KV tracing in statement phase execution r=postamar a=postamar

This commit fixes a bug in which the declarative schema changer logged KV operations to the KV trace which were never actually run.

Fixes #95903.

Release note: None

Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
  • Loading branch information
5 people committed Feb 1, 2023
5 parents da247ff + e0648f0 + ce8961c + 4599475 + e9c7eb2 commit f40df39
Show file tree
Hide file tree
Showing 167 changed files with 7,165 additions and 420 deletions.
1 change: 1 addition & 0 deletions pkg/blobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ go_test(
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
],
)

Expand Down
26 changes: 12 additions & 14 deletions pkg/blobs/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

func createTestResources(t testing.TB) (string, string, *stop.Stopper, func()) {
Expand All @@ -55,27 +56,24 @@ func setUpService(
localExternalDir string,
remoteExternalDir string,
) BlobClientFactory {
s := rpc.NewServer(rpcContext)
s, err := rpc.NewServer(rpcContext)
require.NoError(t, err)

remoteBlobServer, err := NewBlobService(remoteExternalDir)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)

blobspb.RegisterBlobServer(s, remoteBlobServer)
ln, err := netutil.ListenAndServeGRPC(rpcContext.Stopper, s, util.TestAddr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)

s2 := rpc.NewServer(rpcContext)
s2, err := rpc.NewServer(rpcContext)
require.NoError(t, err)
localBlobServer, err := NewBlobService(localExternalDir)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)

blobspb.RegisterBlobServer(s2, localBlobServer)
ln2, err := netutil.ListenAndServeGRPC(rpcContext.Stopper, s2, util.TestAddr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)

localDialer := nodedialer.New(rpcContext,
func(nodeID roachpb.NodeID) (net.Addr, error) {
Expand Down
12 changes: 8 additions & 4 deletions pkg/ccl/kvccl/kvtenantccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,8 @@ func TestConnectorGossipSubscription(t *testing.T) {
defer stopper.Stop(ctx)
clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */)
rpcContext := rpc.NewInsecureTestingContext(ctx, clock, stopper)
s := rpc.NewServer(rpcContext)
s, err := rpc.NewServer(rpcContext)
require.NoError(t, err)

// Test setting the cluster ID by setting it to nil then ensuring it's later
// set to the original ID value.
Expand Down Expand Up @@ -359,7 +360,8 @@ func TestConnectorRangeLookup(t *testing.T) {
defer stopper.Stop(ctx)
clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */)
rpcContext := rpc.NewInsecureTestingContext(ctx, clock, stopper)
s := rpc.NewServer(rpcContext)
s, err := rpc.NewServer(rpcContext)
require.NoError(t, err)

rangeLookupRespC := make(chan *roachpb.RangeLookupResponse, 1)
rangeLookupFn := func(_ context.Context, req *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error) {
Expand Down Expand Up @@ -444,7 +446,8 @@ func TestConnectorRetriesUnreachable(t *testing.T) {

clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */)
rpcContext := rpc.NewInsecureTestingContext(ctx, clock, stopper)
s := rpc.NewServer(rpcContext)
s, err := rpc.NewServer(rpcContext)
require.NoError(t, err)

node1 := &roachpb.NodeDescriptor{NodeID: 1, Address: util.MakeUnresolvedAddr("tcp", "1.1.1.1")}
node2 := &roachpb.NodeDescriptor{NodeID: 2, Address: util.MakeUnresolvedAddr("tcp", "2.2.2.2")}
Expand Down Expand Up @@ -539,7 +542,8 @@ func TestConnectorRetriesError(t *testing.T) {
gossipSubFn func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error,
rangeLookupFn func(_ context.Context, req *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error),
) string {
internalServer := rpc.NewServer(rpcContext)
internalServer, err := rpc.NewServer(rpcContext)
require.NoError(t, err)
roachpb.RegisterInternalServer(internalServer, &mockServer{rangeLookupFn: rangeLookupFn, gossipSubFn: gossipSubFn})
ln, err := net.Listen(util.TestAddr.Network(), util.TestAddr.String())
require.NoError(t, err)
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/kvccl/kvtenantccl/setting_overrides_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,8 @@ func TestConnectorSettingOverrides(t *testing.T) {
defer stopper.Stop(ctx)
clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */)
rpcContext := rpc.NewInsecureTestingContext(ctx, clock, stopper)
s := rpc.NewServer(rpcContext)
s, err := rpc.NewServer(rpcContext)
require.NoError(t, err)

tenantID := roachpb.MustMakeTenantID(5)
gossipSubFn := func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error {
Expand Down
10 changes: 10 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

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

9 changes: 6 additions & 3 deletions pkg/ccl/schemachangerccl/schemachanger_ccl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,15 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/sctest"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

func newCluster(t *testing.T, knobs *scexec.TestingKnobs) (*gosql.DB, func()) {
_, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(
func newCluster(
t *testing.T, knobs *scexec.TestingKnobs,
) (serverutils.TestServerInterface, *gosql.DB, func()) {
c, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(
t, 3 /* numServers */, base.TestingKnobs{
SQLDeclarativeSchemaChanger: knobs,
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
Expand All @@ -34,7 +37,7 @@ func newCluster(t *testing.T, knobs *scexec.TestingKnobs) (*gosql.DB, func()) {
},
},
)
return sqlDB, cleanup
return c.Server(0), sqlDB, cleanup
}

func TestDecomposeToElements(t *testing.T) {
Expand Down
12 changes: 12 additions & 0 deletions pkg/ccl/schemachangerccl/testdata/decomp/multiregion
Original file line number Diff line number Diff line change
Expand Up @@ -37,16 +37,19 @@ ElementState:
descriptorId: 104
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 104
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 104
privileges: "2048"
userName: public
withGrantOption: "0"
Status: PUBLIC
- DatabaseRegionConfig:
databaseId: 104
Expand Down Expand Up @@ -326,11 +329,13 @@ ElementState:
descriptorId: 110
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 110
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- ObjectParent:
objectId: 110
Expand Down Expand Up @@ -601,11 +606,13 @@ ElementState:
descriptorId: 109
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 109
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- ObjectParent:
objectId: 109
Expand Down Expand Up @@ -988,11 +995,13 @@ ElementState:
descriptorId: 108
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 108
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- ObjectParent:
objectId: 108
Expand Down Expand Up @@ -1025,16 +1034,19 @@ ElementState:
descriptorId: 106
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 106
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 106
privileges: "512"
userName: public
withGrantOption: "0"
Status: PUBLIC
- ObjectParent:
objectId: 106
Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/schemachangerccl/testdata/decomp/partitioning
Original file line number Diff line number Diff line change
Expand Up @@ -431,11 +431,13 @@ ElementState:
descriptorId: 104
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 104
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- ObjectParent:
objectId: 104
Expand Down Expand Up @@ -750,11 +752,13 @@ ElementState:
descriptorId: 105
privileges: "2"
userName: admin
withGrantOption: "2"
Status: PUBLIC
- UserPrivileges:
descriptorId: 105
privileges: "2"
userName: root
withGrantOption: "2"
Status: PUBLIC
- ObjectParent:
objectId: 105
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,8 +105,8 @@ upsert descriptor #104
+ statementTag: DROP DATABASE
+ targetRanks: <redacted>
+ targets: <redacted>
defaultPrivileges: {}
id: 104
modificationTime: {}
...
regionEnumId: 106
survivalGoal: REGION_FAILURE
Expand Down
1 change: 1 addition & 0 deletions pkg/gossip/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ go_test(
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
"@org_golang_google_grpc//:go_default_library",
],
)
Expand Down
35 changes: 15 additions & 20 deletions pkg/gossip/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
)

Expand Down Expand Up @@ -58,20 +59,17 @@ func startGossipAtAddr(
rpcContext := rpc.NewInsecureTestingContextWithClusterID(ctx, clock, stopper, clusterID)
rpcContext.NodeID.Set(ctx, nodeID)

server := rpc.NewServer(rpcContext)
server, err := rpc.NewServer(rpcContext)
require.NoError(t, err)
g := NewTest(nodeID, stopper, registry, zonepb.DefaultZoneConfigRef())
RegisterGossipServer(server, g)
ln, err := netutil.ListenAndServeGRPC(stopper, server, addr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)
addr = ln.Addr()
if err := g.SetNodeDescriptor(&roachpb.NodeDescriptor{
require.NoError(t, g.SetNodeDescriptor(&roachpb.NodeDescriptor{
NodeID: nodeID,
Address: util.MakeUnresolvedAddr(addr.Network(), addr.String()),
}); err != nil {
t.Fatal(err)
}
}))
g.start(addr)
time.Sleep(time.Millisecond)
return g, rpcContext
Expand Down Expand Up @@ -121,22 +119,20 @@ func startFakeServerGossips(
clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */)
lRPCContext := rpc.NewInsecureTestingContextWithClusterID(ctx, clock, stopper, clusterID)

lserver := rpc.NewServer(lRPCContext)
lserver, err := rpc.NewServer(lRPCContext)
require.NoError(t, err)
local := NewTest(localNodeID, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef())
RegisterGossipServer(lserver, local)
lln, err := netutil.ListenAndServeGRPC(stopper, lserver, util.IsolatedTestAddr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)
local.start(lln.Addr())

rRPCContext := rpc.NewInsecureTestingContextWithClusterID(ctx, clock, stopper, clusterID)
rserver := rpc.NewServer(rRPCContext)
rserver, err := rpc.NewServer(rRPCContext)
require.NoError(t, err)
remote := newFakeGossipServer(rserver, stopper)
rln, err := netutil.ListenAndServeGRPC(stopper, rserver, util.IsolatedTestAddr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)
addr := rln.Addr()
remote.nodeAddr = util.MakeUnresolvedAddr(addr.Network(), addr.String())

Expand Down Expand Up @@ -480,16 +476,15 @@ func TestClientRegisterWithInitNodeID(t *testing.T) {
nodeID := roachpb.NodeID(i + 1)

rpcContext := rpc.NewInsecureTestingContextWithClusterID(ctx, clock, stopper, clusterID)
server := rpc.NewServer(rpcContext)
server, err := rpc.NewServer(rpcContext)
require.NoError(t, err)
// node ID must be non-zero
gnode := NewTest(nodeID, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef())
RegisterGossipServer(server, gnode)
g = append(g, gnode)

ln, err := netutil.ListenAndServeGRPC(stopper, server, util.IsolatedTestAddr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)

// Connect to the first gossip node.
if gossipAddr == "" {
Expand Down
8 changes: 4 additions & 4 deletions pkg/gossip/gossip_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

// TestGossipInfoStore verifies operation of gossip instance infostore.
Expand Down Expand Up @@ -706,7 +707,8 @@ func TestGossipJoinTwoClusters(t *testing.T) {
clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */)
rpcContext := rpc.NewInsecureTestingContextWithClusterID(ctx, clock, stopper, clusterID)

server := rpc.NewServer(rpcContext)
server, err := rpc.NewServer(rpcContext)
require.NoError(t, err)

// node ID must be non-zero
gnode := NewTest(roachpb.NodeID(i+1), stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef())
Expand All @@ -717,9 +719,7 @@ func TestGossipJoinTwoClusters(t *testing.T) {
gnode.clusterID.Set(context.Background(), clusterIDs[i])

ln, err := netutil.ListenAndServeGRPC(stopper, server, util.IsolatedTestAddr)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)
addrs = append(addrs, ln.Addr())

// Only the third node has addresses.
Expand Down
Loading

0 comments on commit f40df39

Please sign in to comment.