Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
93991: tree: correct mutation/DDL property for some opaque operators r=ZhouXing19 a=rafiss

fixes #91713

In ed733ad, a framework was added to mark certain opaque operators as DDL or mutations.

This was enhanced in 06581b3, but that change wasn't exhaustive since it marked some statements as read-only, even if they could perform DDL.

With the addition of `StatementType()` in
8962176, we can make this a little more correct.

This allows the check at
https://github.com/cockroachdb/cockroach/blob/48ef0d89e6179c0d348a5236ad308d81fa392f7c/pkg/sql/opt/exec/execbuilder/relational.go#L163 to work correctly, and reject operations that shouldn't be allowed when using a read-only transaction.

To explain each change:
- BACKUP can modify job state and write to userfiles, so shouldn't be allowed in read-only mode.
- SET commands are always allowed in read-only mode in order to match Postgres behavior, and since those changes are all in-memory and session setting modifications don't respect transactions anyway.
- The crdb_internal tenant functions modify system tables.
- GRANT, REVOKE, and many other privilege-related statements are "DCL" (data control language), and all modify system tables or descriptors.

Release note (bug fix): CREATE ROLE, DELETE ROLE, GRANT, and REVOKE statements no longer work when the transaction is in read-only mode.

94025: storage: return error from {MVCC,Engine}Iterator.Value r=mgartner,nicktrav a=sumeerbhola

Informs cockroachdb/pebble#1170

Epic: CRDB-20378

Release note: None

94031: kvstreamer: fix the usage of the range iterator r=yuzefovich a=yuzefovich

Previously, after `Seek`ing the range iterator to the next key in the batch of requests in the streamer we forgot to check the validity of the iterator. In particular, this could lead to a crash of the process if `Seek` encountered an error for whatever reason. In practice, I've only observed this when running TPCH with high concurrency when GOMEMLIMIT is set.

The bug was introduced in an innocently-looking refactor in 041b104.

Epic: None

Release note (bug fix): CockroachDB could previously crash in rare circumstances when evaluating lookup and index joins. The bug is present since 22.2.0 release. Temporary workaround without upgrading to the release with this fix is changing the value of undocumented cluster setting `sql.distsql.use_streamer.enabled` to `false`.

94073: roachtest: replace ordinal column reference in transfer-leases r=mgartner a=mgartner

This commit fixes a broken roachtest caused by #93754.

Fixes #94058

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: sumeerbhola <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
5 people committed Dec 21, 2022
5 parents 92740ec + 50a7999 + a269b14 + 3125bc7 + 3613d7b commit 8d3a94e
Show file tree
Hide file tree
Showing 29 changed files with 202 additions and 60 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_data_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ func slurpSSTablesLatestKey(
} else if !ok || !it.UnsafeKey().Less(end) {
break
}
val, err := storage.DecodeMVCCValue(it.Value())
val, err := storage.DecodeMVCCValueAndErr(it.Value())
if err != nil {
t.Fatal(err)
}
Expand Down
6 changes: 5 additions & 1 deletion pkg/ccl/streamingccl/replicationutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,9 +70,13 @@ func ScanSST(
} else if !valid {
break
}
v, err := pointIter.Value()
if err != nil {
return err
}
if err = mvccKeyValOp(storage.MVCCKeyValue{
Key: pointIter.Key(),
Value: pointIter.Value(),
Value: v,
}); err != nil {
return err
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -351,10 +351,12 @@ func assertExactlyEqualKVs(
// Since the iterator goes from latest to older versions, we compare
// starting from the end of the slice that is sorted by timestamp.
latestVersionInChain := valueTimestampTuples[len(valueTimestampTuples)-1]
v, err := it.Value()
require.NoError(t, err)
require.Equal(t, roachpb.KeyValue{
Key: it.Key().Key,
Value: roachpb.Value{
RawBytes: it.Value(),
RawBytes: v,
Timestamp: it.Key().Timestamp,
},
}, latestVersionInChain)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -404,10 +404,12 @@ func assertEqualKVs(
// Since the iterator goes from latest to older versions, we compare
// starting from the end of the slice that is sorted by timestamp.
latestVersionInChain := valueTimestampTuples[len(valueTimestampTuples)-1]
v, err := it.Value()
require.NoError(t, err)
require.Equal(t, roachpb.KeyValue{
Key: it.Key().Key,
Value: roachpb.Value{
RawBytes: it.Value(),
RawBytes: v,
Timestamp: it.Key().Timestamp,
},
}, latestVersionInChain)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/quit.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ func (q *quitTest) createRanges(ctx context.Context) {
db := q.c.Conn(ctx, q.t.L(), 1)
defer db.Close()
if _, err := db.ExecContext(ctx, fmt.Sprintf(`
CREATE TABLE t(x, y, PRIMARY KEY(x)) AS SELECT @1, 1 FROM generate_series(1,%[1]d)`,
CREATE TABLE t(x, y, PRIMARY KEY(x)) AS SELECT i, 1 FROM generate_series(1,%[1]d) g(i)`,
numRanges)); err != nil {
q.Fatal(err)
}
Expand Down
12 changes: 8 additions & 4 deletions pkg/kv/kvclient/kvstreamer/streamer.go
Original file line number Diff line number Diff line change
Expand Up @@ -532,7 +532,10 @@ func (s *Streamer) Enqueue(ctx context.Context, reqs []roachpb.RequestUnion) (re
}
var reqsKeysScratch []roachpb.Key
var newNumRangesPerScanRequestMemoryUsage int64
for {
for ; ; ri.Seek(ctx, rs.Key, scanDir) {
if !ri.Valid() {
return ri.Error()
}
// Find all requests that touch the current range.
var singleRangeReqs []roachpb.RequestUnion
var positions []int
Expand Down Expand Up @@ -626,11 +629,12 @@ func (s *Streamer) Enqueue(ctx context.Context, reqs []roachpb.RequestUnion) (re
requestsToServe = append(requestsToServe, r)
s.enqueuedSingleRangeRequests += len(singleRangeReqs)

if !ri.NeedAnother(rs) {
// This was the last range.
if allRequestsAreWithinSingleRange || !ri.NeedAnother(rs) {
// This was the last range. Breaking here rather than Seek'ing the
// iterator to RKeyMax (and, thus, invalidating it) allows us to
// avoid adding a confusing message into the trace.
break
}
ri.Seek(ctx, rs.Key, scanDir)
}

if streamerLocked {
Expand Down
12 changes: 10 additions & 2 deletions pkg/kv/kvnemesis/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,11 @@ func (e *Engine) Get(key roachpb.Key, ts hlc.Timestamp) roachpb.Value {
return roachpb.Value{}
}
var valCopy []byte
e.b, valCopy = e.b.Copy(iter.Value(), 0 /* extraCap */)
v, err := iter.ValueAndErr()
if err != nil {
panic(err)
}
e.b, valCopy = e.b.Copy(v, 0 /* extraCap */)
mvccVal, err := storage.DecodeMVCCValue(valCopy)
if err != nil {
panic(err)
Expand Down Expand Up @@ -128,7 +132,11 @@ func (e *Engine) Iterate(
hasPoint, _ := iter.HasPointAndRange()
var keyCopy, valCopy []byte
e.b, keyCopy = e.b.Copy(iter.Key(), 0 /* extraCap */)
e.b, valCopy = e.b.Copy(iter.Value(), 0 /* extraCap */)
v, err := iter.ValueAndErr()
if err != nil {
fn(nil, nil, hlc.Timestamp{}, nil, err)
}
e.b, valCopy = e.b.Copy(v, 0 /* extraCap */)
if hasPoint {
key, err := storage.DecodeMVCCKey(keyCopy)
if err != nil {
Expand Down
12 changes: 10 additions & 2 deletions pkg/kv/kvserver/gc/gc_old_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,11 @@ func runGCOld(
expBaseKey = iterKey.Key
if !iterKey.IsValue() {
keys = []storage.MVCCKey{iter.Key()}
vals = [][]byte{iter.Value()}
v, err := iter.Value()
if err != nil {
return Info{}, err
}
vals = [][]byte{v}
continue
}
// An implicit metadata.
Expand All @@ -203,8 +207,12 @@ func runGCOld(
// determine that there is no intent.
vals = [][]byte{nil}
}
v, err := iter.Value()
if err != nil {
return Info{}, err
}
keys = append(keys, iter.Key())
vals = append(vals, iter.Value())
vals = append(vals, v)
}
// Handle last collected set of keys/vals.
processKeysAndValues()
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/gc/gc_random_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -495,7 +495,8 @@ func getExpectationsGenerator(
p, r := it.HasPointAndRange()
if p {
k := it.Key()
v := it.Value()
v, err := it.Value()
require.NoError(t, err)
if len(baseKey) == 0 {
baseKey = k.Key
// We are only interested in range tombstones covering current point,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/rditer/replica_data_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -334,7 +334,7 @@ func (ri *ReplicaMVCCDataIterator) Key() storage.MVCCKey {
}

// Value returns the current value. Only called in tests.
func (ri *ReplicaMVCCDataIterator) Value() []byte {
func (ri *ReplicaMVCCDataIterator) Value() ([]byte, error) {
return ri.it.Value()
}

Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -2301,9 +2301,13 @@ func (r *Replica) printRaftTail(
if err != nil {
return sb.String(), err
}
v, err := it.Value()
if err != nil {
return sb.String(), err
}
kv := storage.MVCCKeyValue{
Key: mvccKey,
Value: it.Value(),
Value: v,
}
sb.WriteString(truncateEntryString(SprintMVCCKeyValue(kv, true /* printKey */), 2000))
sb.WriteRune('\n')
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,7 +144,7 @@ func (i *MVCCIterator) Key() storage.MVCCKey {
}

// Value is part of the storage.MVCCIterator interface.
func (i *MVCCIterator) Value() []byte {
func (i *MVCCIterator) Value() ([]byte, error) {
return i.i.Value()
}

Expand Down Expand Up @@ -391,7 +391,7 @@ func (i *EngineIterator) EngineKey() (storage.EngineKey, error) {
}

// Value is part of the storage.EngineIterator interface.
func (i *EngineIterator) Value() []byte {
func (i *EngineIterator) Value() ([]byte, error) {
return i.i.Value()
}

Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/catalog/lease/kv_writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,10 +159,12 @@ func getRawHistoryKVs(
k := it.Key()
suffix, _, err := codec.DecodeTablePrefix(k.Key)
require.NoError(t, err)
v, err := it.Value()
require.NoError(t, err)
row := roachpb.KeyValue{
Key: suffix,
Value: roachpb.Value{
RawBytes: it.Value(),
RawBytes: v,
},
}
row.Value.ClearChecksum()
Expand Down
53 changes: 53 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/txn
Original file line number Diff line number Diff line change
Expand Up @@ -1028,6 +1028,59 @@ SELECT nextval('a')
statement error cannot execute setval\(\) in a read-only transaction
SELECT setval('a', 2)

statement error cannot execute CREATE ROLE in a read-only transaction
CREATE ROLE my_user

statement error cannot execute ALTER ROLE in a read-only transaction
ALTER ROLE testuser SET default_int_size = 4

statement error cannot execute DROP ROLE in a read-only transaction
DROP ROLE testuser

statement error cannot execute SET CLUSTER SETTING in a read-only transaction
SET CLUSTER SETTING sql.auth.change_own_password.enabled = true

statement error cannot execute GRANT in a read-only transaction
GRANT admin TO testuser

statement error cannot execute REVOKE in a read-only transaction
REVOKE admin FROM testuser

statement error cannot execute GRANT in a read-only transaction
GRANT CONNECT ON DATABASE test TO testuser

statement error cannot execute create_tenant\(\) in a read-only transaction
SELECT crdb_internal.create_tenant(3)

statement error cannot execute rename_tenant\(\) in a read-only transaction
SELECT crdb_internal.rename_tenant(3, 'new')

statement error cannot execute destroy_tenant\(\) in a read-only transaction
SELECT crdb_internal.destroy_tenant(3)

# SET session variable should work in a read-only txn.
statement ok
SET intervalstyle = 'postgres'

statement ok
SET SESSION CHARACTERISTICS AS TRANSACTION PRIORITY NORMAL

statement ok
SET SESSION AUTHORIZATION DEFAULT

statement ok
BEGIN

# DECLARE and FETCH CURSOR should work in a read-only txn.
statement ok
DECLARE foo CURSOR FOR SELECT 1

statement ok
FETCH 1 foo

statement ok
COMMIT

query T
SHOW TRANSACTION STATUS
----
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/row/fetcher_mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func slurpUserDataKVs(t testing.TB, e storage.Engine) []roachpb.KeyValue {
if !it.UnsafeKey().IsValue() {
return errors.Errorf("found intent key %v", it.UnsafeKey())
}
mvccValue, err := storage.DecodeMVCCValue(it.Value())
mvccValue, err := storage.DecodeMVCCValueAndErr(it.Value())
if err != nil {
t.Fatal(err)
}
Expand Down
27 changes: 17 additions & 10 deletions pkg/sql/sem/tree/stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ type canModifySchema interface {
// CanModifySchema returns true if the statement can modify
// the database schema.
func CanModifySchema(stmt Statement) bool {
if stmt.StatementReturnType() == DDL {
if stmt.StatementReturnType() == DDL || stmt.StatementType() == TypeDDL {
return true
}
scm, ok := stmt.(canModifySchema)
Expand All @@ -125,13 +125,20 @@ func CanModifySchema(stmt Statement) bool {

// CanWriteData returns true if the statement can modify data.
func CanWriteData(stmt Statement) bool {
if stmt.StatementType() == TypeDCL {
// Commands like GRANT and REVOKE modify system tables.
return true
}
switch stmt.(type) {
// Normal write operations.
case *Insert, *Delete, *Update, *Truncate:
return true
// Import operations.
case *CopyFrom, *Import, *Restore:
return true
// Backup creates a job and allows you to write into userfiles.
case *Backup:
return true
// CockroachDB extensions.
case *Split, *Unsplit, *Relocate, *RelocateRange, *Scatter:
return true
Expand Down Expand Up @@ -477,7 +484,7 @@ func (*AlterSequence) StatementTag() string { return "ALTER SEQUENCE" }
func (*AlterRole) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*AlterRole) StatementType() StatementType { return TypeDDL }
func (*AlterRole) StatementType() StatementType { return TypeDCL }

// StatementTag returns a short string identifying the type of statement.
func (*AlterRole) StatementTag() string { return "ALTER ROLE" }
Expand All @@ -488,7 +495,7 @@ func (*AlterRole) hiddenFromShowQueries() {}
func (*AlterRoleSet) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*AlterRoleSet) StatementType() StatementType { return TypeDDL }
func (*AlterRoleSet) StatementType() StatementType { return TypeDCL }

// StatementTag returns a short string identifying the type of statement.
func (*AlterRoleSet) StatementTag() string { return "ALTER ROLE" }
Expand Down Expand Up @@ -844,7 +851,7 @@ func (*CreateType) modifiesSchema() bool { return true }
func (*CreateRole) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*CreateRole) StatementType() StatementType { return TypeDDL }
func (*CreateRole) StatementType() StatementType { return TypeDCL }

// StatementTag returns a short string identifying the type of statement.
func (*CreateRole) StatementTag() string { return "CREATE ROLE" }
Expand Down Expand Up @@ -912,7 +919,7 @@ func (d *Discard) StatementTag() string {
func (n *DeclareCursor) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*DeclareCursor) StatementType() StatementType { return TypeDCL }
func (*DeclareCursor) StatementType() StatementType { return TypeDML }

// StatementTag returns a short string identifying the type of statement.
func (*DeclareCursor) StatementTag() string { return "DECLARE CURSOR" }
Expand Down Expand Up @@ -975,7 +982,7 @@ func (*DropSequence) StatementTag() string { return "DROP SEQUENCE" }
func (*DropRole) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*DropRole) StatementType() StatementType { return TypeDDL }
func (*DropRole) StatementType() StatementType { return TypeDCL }

// StatementTag returns a short string identifying the type of statement.
func (*DropRole) StatementTag() string { return "DROP ROLE" }
Expand Down Expand Up @@ -1342,7 +1349,7 @@ func (*SelectClause) StatementTag() string { return "SELECT" }
func (*SetVar) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*SetVar) StatementType() StatementType { return TypeDCL }
func (*SetVar) StatementType() StatementType { return TypeDML }

// StatementTag returns a short string identifying the type of statement.
func (n *SetVar) StatementTag() string {
Expand All @@ -1365,7 +1372,7 @@ func (*SetClusterSetting) StatementTag() string { return "SET CLUSTER SETTING" }
func (*SetTransaction) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*SetTransaction) StatementType() StatementType { return TypeDCL }
func (*SetTransaction) StatementType() StatementType { return TypeTCL }

// StatementTag returns a short string identifying the type of statement.
func (*SetTransaction) StatementTag() string { return "SET TRANSACTION" }
Expand Down Expand Up @@ -1395,7 +1402,7 @@ func (*SetZoneConfig) StatementTag() string { return "CONFIGURE ZONE" }
func (*SetSessionAuthorizationDefault) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*SetSessionAuthorizationDefault) StatementType() StatementType { return TypeDCL }
func (*SetSessionAuthorizationDefault) StatementType() StatementType { return TypeDML }

// StatementTag returns a short string identifying the type of statement.
func (*SetSessionAuthorizationDefault) StatementTag() string { return "SET" }
Expand All @@ -1404,7 +1411,7 @@ func (*SetSessionAuthorizationDefault) StatementTag() string { return "SET" }
func (*SetSessionCharacteristics) StatementReturnType() StatementReturnType { return Ack }

// StatementType implements the Statement interface.
func (*SetSessionCharacteristics) StatementType() StatementType { return TypeDCL }
func (*SetSessionCharacteristics) StatementType() StatementType { return TypeDML }

// StatementTag returns a short string identifying the type of statement.
func (*SetSessionCharacteristics) StatementTag() string { return "SET" }
Expand Down
Loading

0 comments on commit 8d3a94e

Please sign in to comment.