Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
105937: sql: fix `SHOW SCHEMAS` database name resolution bug r=chengxiong-ruan a=andyyang890

This patch fixes a bug in the `SHOW SCHEMAS FROM db_name` logic where a schema with the name `db_name` in the current database would result in the current database's schemas being erroneously returned instead. The logic will now simply look up the database name instead of using the schema lookup logic.

Fixes #105906 

Release note (bug fix): `SHOW SCHEMAS FROM db_name` will no longer incorrectly show schemas from the current database when the current database has a schema named `db_name`.

106177: Revert "Revert "storage: use size-carrying point tombstones"" r=erikgrinaker a=jbowens

This reverts commit 90c1ee6.

The corruption issue surfaced by this commit was a bug within a feature enabled by the sstable format used within the commit. This feature has been disabled and the bug is tracked cockroachdb/pebble#2705. 

Epic: CRDB-25405
Release note (performance improvement): Improve disk space reclamation heuristics making disk space reclamation more timely.

Co-authored-by: Andy Yang <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
  • Loading branch information
3 people committed Jul 5, 2023
3 parents fe00930 + 786fcec + 3c866e4 commit 042cb00
Show file tree
Hide file tree
Showing 33 changed files with 484 additions and 127 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -303,4 +303,4 @@ trace.opentelemetry.collector string address of an OpenTelemetry trace collecto
trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured tenant-rw
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez tenant-rw
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. tenant-rw
version version 1000023.1-10 set the active cluster version in the format '<major>.<minor>' tenant-rw
version version 1000023.1-14 set the active cluster version in the format '<major>.<minor>' tenant-rw
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -258,6 +258,6 @@
<tr><td><div id="setting-trace-snapshot-rate" class="anchored"><code>trace.snapshot.rate</code></div></td><td>duration</td><td><code>0s</code></td><td>if non-zero, interval at which background trace snapshots are captured</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000023.1-10</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000023.1-14</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
</tbody>
</table>
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/multi_region_show
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ SHOW CREATE DATABASE multi_region_test_placement_restricted_db
database_name create_statement
multi_region_test_placement_restricted_db CREATE DATABASE multi_region_test_placement_restricted_db PRIMARY REGION "ap-southeast-2" REGIONS = "ap-southeast-2", "ca-central-1", "us-east-1" SURVIVE ZONE FAILURE PLACEMENT RESTRICTED

statement error target database or schema does not exist
statement error database "foo" does not exist
SHOW CREATE DATABASE foo

# Test that showing localities works for databases and schemas with weird
Expand Down
19 changes: 19 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -542,6 +542,17 @@ const (
// that (optionally) embed below-raft admission data.
V23_2_UseACRaftEntryEntryEncodings

// V23_2_PebbleFormatDeleteSizedAndObsolete upgrades Pebble's format major
// version to FormatDeleteSizedAndObsolete, allowing use of a new sstable
// format version Pebblev4. This version has two improvements:
// a) It allows the use of DELSIZED point tombstones.
// b) It encodes the obsolence of keys in a key-kind bit.
V23_2_PebbleFormatDeleteSizedAndObsolete

// V23_2_UseSizedPebblePointTombstones enables the use of Pebble's new
// DeleteSized operations.
V23_2_UseSizedPebblePointTombstones

// *************************************************
// Step (1) Add new versions here.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -943,6 +954,14 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_2_UseACRaftEntryEntryEncodings,
Version: roachpb.Version{Major: 23, Minor: 1, Internal: 10},
},
{
Key: V23_2_PebbleFormatDeleteSizedAndObsolete,
Version: roachpb.Version{Major: 23, Minor: 1, Internal: 12},
},
{
Key: V23_2_UseSizedPebblePointTombstones,
Version: roachpb.Version{Major: 23, Minor: 1, Internal: 14},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
10 changes: 2 additions & 8 deletions pkg/cmd/roachtest/tests/tombstones.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,12 +29,6 @@ import (
// registerPointTombstone registers the point tombstone test.
func registerPointTombstone(r registry.Registry) {
r.Add(registry.TestSpec{
Skip: "pebble#2340",
SkipDetails: "This roachtest is implemented ahead of implementing and using " +
"pebble#2340 within Cockroach. Currently, this roachtest fails through " +
"a timeout because the disk space corresponding to the large KVs is " +
"never reclaimed. Once pebble#2340 is integrated into Cockroach, we " +
"expect this to begin passing, and we can un-skip it.",
Name: "point-tombstone/heterogeneous-value-sizes",
Owner: registry.OwnerStorage,
Cluster: r.MakeClusterSpec(4),
Expand Down Expand Up @@ -136,7 +130,7 @@ func registerPointTombstone(r registry.Registry) {
require.LessOrEqual(t, statsAfterDeletes.livePercentage, 0.10)

// Wait for garbage collection to delete the non-live data.
targetSize := uint64(2 << 30) /* 2 GB */
targetSize := uint64(3 << 30) /* 3 GiB */
t.Status("waiting for garbage collection and compaction to reduce on-disk size to ", humanize.IBytes(targetSize))
m = c.NewMonitor(ctx, c.Range(1, 3))
m.Go(func(ctx context.Context) error {
Expand Down Expand Up @@ -172,7 +166,7 @@ type tableSizeInfo struct {
}

func (info tableSizeInfo) String() string {
return fmt.Sprintf("databaseID: %d, tableID: %d, rangeCount: %d, approxDiskBytes: %s, liveBytes: %s, totalBytes: %s, livePercentage: %.1f",
return fmt.Sprintf("databaseID: %d, tableID: %d, rangeCount: %d, approxDiskBytes: %s, liveBytes: %s, totalBytes: %s, livePercentage: %.2f",
info.databaseID,
info.tableID,
info.rangeCount,
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batch_spanset_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func TestSpanSetBatchBoundaries(t *testing.T) {
}

t.Run("writes before range", func(t *testing.T) {
if err := batch.ClearUnversioned(outsideKey.Key); !isWriteSpanErr(err) {
if err := batch.ClearUnversioned(outsideKey.Key, storage.ClearOptions{}); !isWriteSpanErr(err) {
t.Errorf("ClearUnversioned: unexpected error %v", err)
}
if err := batch.ClearRawRange(outsideKey.Key, outsideKey2.Key, true, true); !isWriteSpanErr(err) {
Expand All @@ -93,7 +93,7 @@ func TestSpanSetBatchBoundaries(t *testing.T) {
})

t.Run("writes after range", func(t *testing.T) {
if err := batch.ClearUnversioned(outsideKey3.Key); !isWriteSpanErr(err) {
if err := batch.ClearUnversioned(outsideKey3.Key, storage.ClearOptions{}); !isWriteSpanErr(err) {
t.Errorf("ClearUnversioned: unexpected error %v", err)
}
if err := batch.ClearRawRange(insideKey2.Key, outsideKey4.Key, true, true); !isWriteSpanErr(err) {
Expand Down Expand Up @@ -303,7 +303,7 @@ func TestSpanSetBatchTimestamps(t *testing.T) {
}

for _, batch := range []storage.Batch{batchBefore, batchNonMVCC} {
if err := batch.ClearUnversioned(wkey.Key); !isWriteSpanErr(err) {
if err := batch.ClearUnversioned(wkey.Key, storage.ClearOptions{}); !isWriteSpanErr(err) {
t.Errorf("ClearUnversioned: unexpected error %v", err)
}
{
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/kvstorage/init.go
Original file line number Diff line number Diff line change
Expand Up @@ -504,7 +504,7 @@ func LoadAndReconcileReplicas(ctx context.Context, eng storage.Engine) ([]Replic
// TODO(tbg): if clearRangeData were in this package we could destroy more
// effectively even if for some reason we had in the past written state
// other than the HardState here (not supposed to happen, but still).
if err := eng.ClearUnversioned(logstore.NewStateLoader(repl.RangeID).RaftHardStateKey()); err != nil {
if err := eng.ClearUnversioned(logstore.NewStateLoader(repl.RangeID).RaftHardStateKey(), storage.ClearOptions{}); err != nil {
return nil, errors.Wrapf(err, "removing HardState for r%d", repl.RangeID)
}
log.Eventf(ctx, "removed legacy uninitialized replica for r%s", repl.RangeID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/loqrecovery/record.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func RegisterOfflineRecoveryEvents(
continue
}
if removeEvent {
if err := readWriter.ClearUnversioned(iter.UnsafeKey().Key); err != nil {
if err := readWriter.ClearUnversioned(iter.UnsafeKey().Key, storage.ClearOptions{}); err != nil {
processingErrors = errors.CombineErrors(processingErrors, errors.Wrapf(
err, "failed to delete replica recovery record at key %s", iter.UnsafeKey()))
continue
Expand Down
5 changes: 4 additions & 1 deletion pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -2607,7 +2607,10 @@ func handleTruncatedStateBelowRaftPreApply(
// avoid allocating when constructing Raft log keys (16 bytes).
prefix := prefixBuf.RaftLogPrefix()
for idx := currentTruncatedState.Index + 1; idx <= suggestedTruncatedState.Index; idx++ {
if err := readWriter.ClearUnversioned(keys.RaftLogKeyFromPrefix(prefix, idx)); err != nil {
if err := readWriter.ClearUnversioned(
keys.RaftLogKeyFromPrefix(prefix, idx),
storage.ClearOptions{},
); err != nil {
return false, errors.Wrapf(err, "unable to clear truncated Raft entries for %+v at index %d",
suggestedTruncatedState, idx)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1823,7 +1823,7 @@ func TestOptimizePuts(t *testing.T) {
require.NoError(t, tc.engine.ClearMVCCRangeKey(storage.MVCCRangeKey{
StartKey: c.exKey, EndKey: c.exEndKey, Timestamp: hlc.MinTimestamp}))
} else if c.exKey != nil {
require.NoError(t, tc.engine.ClearUnversioned(c.exKey))
require.NoError(t, tc.engine.ClearUnversioned(c.exKey, storage.ClearOptions{}))
}
}
}
Expand Down
21 changes: 13 additions & 8 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -415,6 +415,11 @@ func (i *EngineIterator) Value() ([]byte, error) {
return i.i.Value()
}

// ValueLen is part of the storage.EngineIterator interface.
func (i *EngineIterator) ValueLen() int {
return i.i.ValueLen()
}

// UnsafeRawEngineKey is part of the storage.EngineIterator interface.
func (i *EngineIterator) UnsafeRawEngineKey() []byte {
return i.i.UnsafeRawEngineKey()
Expand Down Expand Up @@ -522,34 +527,34 @@ func (s spanSetWriter) checkAllowed(key roachpb.Key) error {
return nil
}

func (s spanSetWriter) ClearMVCC(key storage.MVCCKey) error {
func (s spanSetWriter) ClearMVCC(key storage.MVCCKey, opts storage.ClearOptions) error {
if err := s.checkAllowed(key.Key); err != nil {
return err
}
return s.w.ClearMVCC(key)
return s.w.ClearMVCC(key, opts)
}

func (s spanSetWriter) ClearUnversioned(key roachpb.Key) error {
func (s spanSetWriter) ClearUnversioned(key roachpb.Key, opts storage.ClearOptions) error {
if err := s.checkAllowed(key); err != nil {
return err
}
return s.w.ClearUnversioned(key)
return s.w.ClearUnversioned(key, opts)
}

func (s spanSetWriter) ClearIntent(
key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID,
key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID, opts storage.ClearOptions,
) error {
if err := s.checkAllowed(key); err != nil {
return err
}
return s.w.ClearIntent(key, txnDidNotUpdateMeta, txnUUID)
return s.w.ClearIntent(key, txnDidNotUpdateMeta, txnUUID, opts)
}

func (s spanSetWriter) ClearEngineKey(key storage.EngineKey) error {
func (s spanSetWriter) ClearEngineKey(key storage.EngineKey, opts storage.ClearOptions) error {
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
return err
}
return s.w.ClearEngineKey(key)
return s.w.ClearEngineKey(key, opts)
}

func (s spanSetWriter) SingleClearEngineKey(key storage.EngineKey) error {
Expand Down
14 changes: 1 addition & 13 deletions pkg/sql/delegate/show_schemas.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,18 +62,6 @@ func (d *delegator) delegateShowCreateAllSchemas() (tree.Statement, error) {
// Returns an error if there is no current database, or if the specified
// database doesn't exist.
func (d *delegator) getSpecifiedOrCurrentDatabase(specifiedDB tree.Name) (tree.Name, error) {
var name cat.SchemaName
if specifiedDB != "" {
// Note: the schema name may be interpreted as database name,
// see name_resolution.go.
name.SchemaName = specifiedDB
name.ExplicitSchema = true
}

flags := cat.Flags{AvoidDescriptorCaches: true}
_, resName, err := d.catalog.ResolveSchema(d.ctx, flags, &name)
if err != nil {
return "", err
}
return resName.CatalogName, nil
return d.catalog.LookupDatabaseName(d.ctx, flags, string(specifiedDB))
}
75 changes: 75 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/database
Original file line number Diff line number Diff line change
Expand Up @@ -309,6 +309,11 @@ CREATE TABLE db69713.s.pg_constraintdef_test (
statement ok
DROP DATABASE db69713;

statement ok
RESET DATABASE;

subtest end

# Ensure user must exist to create with owner.
statement error role/user "fake_user" does not exist
CREATE DATABASE aa with owner fake_user
Expand Down Expand Up @@ -361,3 +366,73 @@ database_name owner primary_region secondary_region regions surviva
ifnotexistsownerdb testuser NULL NULL {} NULL

subtest end

subtest regression_105906

statement ok
CREATE SCHEMA regression_105906

statement ok
CREATE DATABASE regression_105906

query TT colnames,rowsort
SHOW SCHEMAS
----
schema_name owner
crdb_internal NULL
information_schema NULL
pg_catalog NULL
pg_extension NULL
public admin
regression_105906 root

# Note: regression_105906 should not appear in the list of schemas below
query TT colnames,rowsort
SHOW SCHEMAS FROM regression_105906
----
schema_name owner
crdb_internal NULL
information_schema NULL
pg_catalog NULL
pg_extension NULL
public admin

statement ok
DROP DATABASE regression_105906

statement ok
DROP SCHEMA regression_105906

statement ok
CREATE SCHEMA "rEgReSsIoN 105906"

statement ok
CREATE DATABASE "rEgReSsIoN 105906"

query T rowsort
SELECT schema_name FROM [SHOW SCHEMAS]
----
public
rEgReSsIoN 105906
crdb_internal
information_schema
pg_catalog
pg_extension

# Note: "rEgReSsIoN 105906" should not appear in the list of schemas below
query T rowsort
SELECT schema_name FROM [SHOW SCHEMAS FROM "rEgReSsIoN 105906"]
----
public
crdb_internal
information_schema
pg_catalog
pg_extension

statement ok
DROP SCHEMA "rEgReSsIoN 105906"

statement ok
DROP DATABASE "rEgReSsIoN 105906"

subtest end
8 changes: 8 additions & 0 deletions pkg/sql/opt/cat/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,14 @@ type Flags struct {
// returned by the Resolve methods (schemas and data sources) *must* be
// immutable after construction, and therefore also thread-safe.
type Catalog interface {
// LookupDatabaseName locates a database with the given name and returns
// the name if found. If no name is provided, it will return the name of
// the current database. An error is returned if no database with the given
// name exists or in the case of an empty name, there is no current database.
// TODO(yang): This function can be extended if needed in the future
// to return a new cat.Database type similar to ResolveSchema.
LookupDatabaseName(ctx context.Context, flags Flags, name string) (tree.Name, error)

// ResolveSchema locates a schema with the given name and returns it along
// with the resolved SchemaName (which has all components filled in).
// If the SchemaName is empty, returns the current database/schema (if one is
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/testutils/testcat/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ go_library(
"//pkg/sql/sem/tree/treecmp",
"//pkg/sql/sem/volatility",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sqlerrors",
"//pkg/sql/stats",
"//pkg/sql/types",
"//pkg/sql/vtable",
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/opt/testutils/testcat/test_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/treeprinter"
Expand Down Expand Up @@ -77,6 +78,15 @@ func New() *Catalog {
}
}

func (tc *Catalog) LookupDatabaseName(
_ context.Context, _ cat.Flags, name string,
) (tree.Name, error) {
if name != testDB {
return "", sqlerrors.NewUndefinedDatabaseError(name)
}
return tree.Name(name), nil
}

// ResolveSchema is part of the cat.Catalog interface.
func (tc *Catalog) ResolveSchema(
_ context.Context, _ cat.Flags, name *cat.SchemaName,
Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,25 @@ func (os *optSchema) getDescriptorForPermissionsCheck() catalog.Descriptor {
return os.database
}

// LookupDatabaseName implements the cat.Catalog interface.
func (oc *optCatalog) LookupDatabaseName(
ctx context.Context, flags cat.Flags, name string,
) (tree.Name, error) {
if flags.AvoidDescriptorCaches {
defer func(prev bool) {
oc.planner.skipDescriptorCache = prev
}(oc.planner.skipDescriptorCache)
oc.planner.skipDescriptorCache = true
}
if name == "" {
name = oc.planner.CurrentDatabase()
}
if err := oc.planner.LookupDatabase(ctx, name); err != nil {
return "", err
}
return tree.Name(name), nil
}

// ResolveSchema is part of the cat.Catalog interface.
func (oc *optCatalog) ResolveSchema(
ctx context.Context, flags cat.Flags, name *cat.SchemaName,
Expand Down
Loading

0 comments on commit 042cb00

Please sign in to comment.