Skip to content

Commit

Permalink
Merge #24735
Browse files Browse the repository at this point in the history
24735: sql: sanitize the "no current db" story r=knz a=knz

Fixes #23893.
Fixes #23145.
Updates #24598.
Informs #24056.
Informs #23958.

Prior to this patch, CockroachdB made it excessively simple for client
connections to not have any current database set. Also by design
CockroachDB allows a connection to be opened with a non-existent
database set as current.

This in turn causes various problems related to name resolution and
the `pg_catalog` schema.

To understand these problems we can consider separately two groups of
users, which have separate use cases:

- newcomers to CockroachDB that don't have prior experience with SQL
  or PostgreSQL. These will not know about "databases" in any sense,
  and will use CockroachDB with default options. This includes using
  `cockroach sql` with no current database selected. These users have
  to be taught upfront that they need to "create a database" and "set
  the current database", which they may or may not (more likely not)
  be interested in.

  When these users then transition into writing apps, they will
  copy-paste the conn URL printed by `cockroach start` (which sets no
  current db) into some ORM or framework, and then will regretfully
  observe that their app fail in mysterious ways because the current
  db is not set (also see the next point).

- users of existing applications or SQL drivers that have been
  developed to target PostgreSQL. These apps commonly contain code
  that:

  - connect to a database named "`postgres`" by default, and make this
    default hard(er) to customize. For example Ecto (Elixir ORM)
    doesn't make this configurable.

    (From the PostgreSQL docs: "After initialization, a database
    cluster will contain a database named `postgres`, which is meant as
    a default database for use by utilities, users and third party
    applications. The database server itself does not require the
    postgres database to exist, but many external utility programs
    assume it exists.")

  - (regardless of which db is selected as current), will issue some
    introspection queries using `pg_catalog` before any additional
    initialization is made by the higher-level app code, in particular
    before the app can issue `CREATE DATABASE`. Currently `pg_catalog`
    queries (and several other things) fail if the current database is
    unset or set to a non-existing database.

To address this relatively large class of problems, this patch
modifies CockroachDB as follows:

- all clusters (existing or new) now get two empty databases on
  startup (created by a migration) called `defaultdb` and `postgres`.

- when a client doesn't specify a current db in their conn string
  (i.e. in the pgwire options), the `defaultdb` database is picked up
  instead.

This resolves all the known problems around the situations identified
above.

The two new databases behave like any regular database and are
provided for convenience to newcomers and
compatibility. (Administrators are free to drop them afterwards to
restore the previous situation, if that is actively desired.)

In addition, to make it slightly harder for newcomers to shoot
themselves in the foot, the `database` session variable cannot be set
to the empty string any more if `sql_safe_updates` is otherwise set.

Three alternatives to this approach were considered, discussed with
@jordanlewis and @nvanbenschoten in their quality of pg compatibility
experts, and with @bdarnell in his quality of carer for newcomers, and
ultimately rejected:

A) generate the current db upon connection if it does not exist, make
   connections using the empty string as curdb use `system` instead.

   Rejected because we don't like to auto-create upon connection. Too
   easy to flood the system with bogus database names by typos in the
   connection string.

B) Pre-populate clusters with a special anonymous database (db name =
   empty string).

   Rejected because there are too many areas in CockroachDB with a
   risk to fail silently or with weird error messages as a
   result. Also does not solve the problem of clients that want a db
   called `postgres`.

C) Auto-create a database named after the user upon user creation
   (with an optional flag to skip db creation).

   Rejected because it does not solve the problem of clients that want
   a db called `postgres`. @bdarnell: creating a database per user also
   implicitly encourages bad habits (like sharing certs for the user
   whose name corresponds to the database instead of creating multiple
   users).

D) Implement some magic in the handling of `pg_catalog`, name
   resolution and all other pieces that currently don't like non-existent
   `database` values to create the appearance of something that works.

   Rejected because too complex to implement successfully (there are
   many moving parts that need to be touched to make this work) and
   the resulting magic would be hard to understand by CockroachDB
   contributors and hard to maintain over time.

E) Auto-create only one database called `defaultdb`.

   Rejected because it does not solve the problem of clients that want
   a db called `postgres`.

F) Auto-create only one database called `postgres` and use that as
   default when no current db is specified.

   Rejected because not good for branding.

G) Name the `defaultdb` either `def`, `default` or `default_database`.

   The word "`default`" was rejected because it is a SQL keyword
   and would cause queries using it to fail in obscure ways.

   The word "`def`" (similar to what MySQL uses in
   `information_schema`) was rejected because it refers too strongly
   to other uses of this word in programming languages ("define").

   "`default_database`" was rejected because too verbose.

Release note (sql change): new and existing clusters will now contain
two empty databases called `defaultdb` and `postgres` by default. The
database `defaultdb` is automatically used for clients that connect
without a current database set (e.g. without a database component in
the connection URL). The database `postgres` is provided for
compatibility with PostgreSQL client frameworks that require it to
exist when the database server has ben freshly installed. Both new
databases behave like any other regular database.

Release note (general change): existing clusters upgraded to this
version of CockroachDB will automatically see two new empty databases
named `defaultdb` and `postgres`. These were added for compatibility
with PostgreSQL and to ease adoption of CockroachDB. They can be
manually deleted by an administrator if the client applications are
determined not to require them.

Co-authored-by: Raphael 'kena' Poss <knz@cockroachlabs.com>
  • Loading branch information
craig[bot] and knz committed May 22, 2018
2 parents a57927b + cc7a9af commit 01eaefd
Show file tree
Hide file tree
Showing 62 changed files with 957 additions and 1,046 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Expand Up @@ -1970,7 +1970,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) {
gcr := roachpb.GCRequest{
// Bogus span to make it a valid request.
Span: roachpb.Span{
Key: keys.MakeTablePrefix(keys.MaxReservedDescID + 1),
Key: keys.MakeTablePrefix(keys.MinUserDescID),
EndKey: keys.MaxKey,
},
Threshold: tc.Server(0).Clock().Now(),
Expand Down Expand Up @@ -2857,7 +2857,7 @@ func TestBackupRestoreSequence(t *testing.T) {

if _, err := newDB.DB.Exec(
`RESTORE TABLE t FROM $1`, localFoo,
); !testutils.IsError(err, "pq: cannot restore table \"t\" without referenced sequence 52 \\(or \"skip_missing_sequences\" option\\)") {
); !testutils.IsError(err, "pq: cannot restore table \"t\" without referenced sequence 54 \\(or \"skip_missing_sequences\" option\\)") {
t.Fatal(err)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/import_stmt.go
Expand Up @@ -70,7 +70,7 @@ const (
// We need to choose arbitrary database and table IDs. These aren't important,
// but they do match what would happen when creating a new database and
// table on an empty cluster.
defaultCSVParentID sqlbase.ID = keys.MaxReservedDescID + 1
defaultCSVParentID sqlbase.ID = keys.MinNonPredefinedUserDescID
defaultCSVTableID sqlbase.ID = defaultCSVParentID + 1
)

Expand Down
20 changes: 10 additions & 10 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal
Expand Up @@ -36,13 +36,13 @@ CREATE table t2 (a STRING PRIMARY KEY) PARTITION BY LIST (a) (
query IITTI
SELECT * FROM crdb_internal.partitions ORDER BY table_id, index_id, name
----
51 1 NULL p12 1
51 1 p12 p12p3 1
51 1 p12p3 p12p3p8 1
51 1 NULL p6 1
51 1 p6 p6p7 1
51 1 p6 p6p8 1
51 1 p6 p6px 1
51 1 p12 pd 1
51 2 NULL p00 2
52 1 NULL pfoo 1
53 1 NULL p12 1
53 1 p12 p12p3 1
53 1 p12p3 p12p3p8 1
53 1 NULL p6 1
53 1 p6 p6p7 1
53 1 p6 p6p8 1
53 1 p6 p6px 1
53 1 p12 pd 1
53 2 NULL p00 2
54 1 NULL pfoo 1
2 changes: 1 addition & 1 deletion pkg/ccl/partitionccl/partition_test.go
Expand Up @@ -123,7 +123,7 @@ func (t *partitioningTest) parse() error {
return errors.Errorf("expected *tree.CreateTable got %T", stmt)
}
st := cluster.MakeTestingClusterSettings()
const parentID, tableID = keys.MaxReservedDescID + 1, keys.MaxReservedDescID + 2
const parentID, tableID = keys.MinUserDescID, keys.MinUserDescID + 1
t.parsed.tableDesc, err = importccl.MakeSimpleTableDescriptor(
ctx, st, createTable, parentID, tableID, hlc.UnixNano())
if err != nil {
Expand Down
16 changes: 9 additions & 7 deletions pkg/ccl/partitionccl/zone_test.go
Expand Up @@ -48,6 +48,8 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {
zoneOverride := config.DefaultZoneConfig()
zoneOverride.GC.TTLSeconds = 42

dbDescID := uint32(keys.MinNonPredefinedUserDescID)

defaultRow := sqlutils.ZoneRow{
ID: keys.RootNamespaceID,
CLISpecifier: ".default",
Expand All @@ -59,27 +61,27 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {
Config: zoneOverride,
}
dbRow := sqlutils.ZoneRow{
ID: keys.MaxReservedDescID + 1,
ID: dbDescID,
CLISpecifier: "d",
Config: zoneOverride,
}
tableRow := sqlutils.ZoneRow{
ID: keys.MaxReservedDescID + 2,
ID: dbDescID + 1,
CLISpecifier: "d.t",
Config: zoneOverride,
}
primaryRow := sqlutils.ZoneRow{
ID: keys.MaxReservedDescID + 2,
ID: dbDescID + 1,
CLISpecifier: "d.t@primary",
Config: zoneOverride,
}
p0Row := sqlutils.ZoneRow{
ID: keys.MaxReservedDescID + 2,
ID: dbDescID + 1,
CLISpecifier: "d.t.p0",
Config: zoneOverride,
}
p1Row := sqlutils.ZoneRow{
ID: keys.MaxReservedDescID + 2,
ID: dbDescID + 1,
CLISpecifier: "d.t.p1",
Config: zoneOverride,
}
Expand Down Expand Up @@ -220,11 +222,11 @@ func TestInvalidIndexPartitionSetShowZones(t *testing.T) {
}{
{
"ALTER INDEX foo EXPERIMENTAL CONFIGURE ZONE ''",
`no schema has been selected to search index: "foo"`,
`index "foo" does not exist`,
},
{
"EXPERIMENTAL SHOW ZONE CONFIGURATION FOR INDEX foo",
`no schema has been selected to search index: "foo"`,
`index "foo" does not exist`,
},
{
"USE system; ALTER INDEX foo EXPERIMENTAL CONFIGURE ZONE ''",
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/storageccl/bench_test.go
Expand Up @@ -47,7 +47,7 @@ func BenchmarkAddSSTable(b *testing.B) {
defer tc.Stopper().Stop(ctx)
kvDB := tc.Server(0).DB()

id := sqlbase.ID(keys.MaxReservedDescID + 1)
id := sqlbase.ID(keys.MinUserDescID)

var totalLen int64
b.StopTimer()
Expand Down Expand Up @@ -105,7 +105,7 @@ func BenchmarkWriteBatch(b *testing.B) {
defer tc.Stopper().Stop(ctx)
kvDB := tc.Server(0).DB()

id := sqlbase.ID(keys.MaxReservedDescID + 1)
id := sqlbase.ID(keys.MinUserDescID)
var batch engine.RocksDBBatchBuilder

var totalLen int64
Expand Down Expand Up @@ -161,7 +161,7 @@ func BenchmarkImport(b *testing.B) {
defer tc.Stopper().Stop(ctx)
kvDB := tc.Server(0).DB()

id := sqlbase.ID(keys.MaxReservedDescID + 1)
id := sqlbase.ID(keys.MinUserDescID)

var totalLen int64
b.StopTimer()
Expand Down
8 changes: 7 additions & 1 deletion pkg/cli/cli_test.go
Expand Up @@ -749,6 +749,8 @@ func Example_sql() {
// 42 69
// sql --execute=show databases
// Database
// defaultdb
// postgres
// system
// t
// sql -e select 1; select 2
Expand Down Expand Up @@ -2001,7 +2003,7 @@ func checkNodeStatus(t *testing.T, c cliTest, output string, start time.Time) {
testcases = append(testcases,
testCase{"leader_ranges", baseIdx, 3},
testCase{"leaseholder_ranges", baseIdx + 1, 3},
testCase{"ranges", baseIdx + 2, 20},
testCase{"ranges", baseIdx + 2, 22},
testCase{"unavailable_ranges", baseIdx + 3, 1},
testCase{"underreplicated_ranges", baseIdx + 4, 1},
)
Expand Down Expand Up @@ -2239,6 +2241,10 @@ writing ` + os.DevNull + `
debug/nodes/1/ranges/18
debug/nodes/1/ranges/19
debug/nodes/1/ranges/20
debug/nodes/1/ranges/21
debug/nodes/1/ranges/22
debug/schema/defaultdb@details
debug/schema/postgres@details
debug/schema/system@details
debug/schema/system/descriptor
debug/schema/system/eventlog
Expand Down
6 changes: 0 additions & 6 deletions pkg/cli/interactive_tests/test_contextual_help.tcl
Expand Up @@ -6,12 +6,6 @@ start_server $argv

spawn $argv sql

start_test "Check that a client without a current database suggests to use SET."
eexpect "warning: no current database set"
eexpect "SET database"
eexpect root@
end_test

start_test "Check that a syntax error can make suggestions."
send "select * from;\r"
eexpect "syntax error"
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/interactive_tests/test_local_cmds.tcl
Expand Up @@ -107,7 +107,7 @@ expect {
}
root@ {}
}
eexpect "/> "
eexpect "/defaultdb> "
# restore
send "\\set show_times\r"
end_test
Expand Down
7 changes: 6 additions & 1 deletion pkg/cli/interactive_tests/test_txn_prompt.tcl
Expand Up @@ -47,10 +47,15 @@ send "SET DATABASE = testdb;\r"
eexpect "\nSET\r\n"
eexpect root@
eexpect "/testdb>"
send "SET DATABASE = '';\r"
send "SET sql_safe_updates = false;\r"
eexpect "\nSET\r\n"
send "SET database = '';\r"
eexpect "\nSET\r\n"
eexpect root@
eexpect "/>"
send "SET database = 'defaultdb';\r"
eexpect "\nSET\r\n"
eexpect root@
end_test

start_test "Test that prompt becomes OPEN when txn is opened."
Expand Down
12 changes: 6 additions & 6 deletions pkg/cli/sql.go
Expand Up @@ -549,6 +549,12 @@ func (c *cliState) refreshDatabaseName() (string, bool) {
return "", false
}

if dbVal == "" {
// Attempt to be helpful to new users.
fmt.Fprintln(stderr, "warning: no current database set."+
" Use SET database = <dbname> to change, CREATE DATABASE to make a new database.")
}

dbName := formatVal(dbVal.(string),
false /* showPrintableUnicode */, false /* shownewLinesAndTabs */)

Expand All @@ -570,12 +576,6 @@ func preparePrompts(dbURL string) (promptPrefix, fullPrompt, continuePrompt stri
username = parsedURL.User.Username()
}
promptPrefix = fmt.Sprintf("%s@%s", username, parsedURL.Host)

if parsedURL.Path == "" {
// Attempt to be helpful to new users.
fmt.Fprintln(stderr, "warning: no current database set."+
" Use SET database = <dbname> to change, CREATE DATABASE to make a new database.")
}
}

if len(promptPrefix) == 0 {
Expand Down
6 changes: 3 additions & 3 deletions pkg/config/system_test.go
Expand Up @@ -274,7 +274,7 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {
defer leaktest.AfterTest(t)()

const (
start = keys.MaxReservedDescID + 1
start = keys.MinUserDescID
reservedStart = keys.MaxSystemConfigDescID + 1
)

Expand Down Expand Up @@ -401,8 +401,8 @@ func TestGetZoneConfigForKey(t *testing.T) {
{tkey(keys.LeaseTableID), keys.SystemDatabaseID},
{tkey(keys.JobsTableID), keys.SystemDatabaseID},
{tkey(keys.LocationsTableID), keys.SystemDatabaseID},
{tkey(keys.MaxReservedDescID + 1), keys.MaxReservedDescID + 1},
{tkey(keys.MaxReservedDescID + 23), keys.MaxReservedDescID + 23},
{tkey(keys.MinUserDescID), keys.MinUserDescID},
{tkey(keys.MinUserDescID + 22), keys.MinUserDescID + 22},
{roachpb.RKeyMax, keys.RootNamespaceID},
}

Expand Down
11 changes: 10 additions & 1 deletion pkg/keys/constants.go
Expand Up @@ -256,7 +256,7 @@ var (
SystemConfigTableDataMax = roachpb.Key(MakeTablePrefix(MaxSystemConfigDescID + 1))

// UserTableDataMin is the start key of user structured data.
UserTableDataMin = roachpb.Key(MakeTablePrefix(MaxReservedDescID + 1))
UserTableDataMin = roachpb.Key(MakeTablePrefix(MinUserDescID))

// MaxKey is the infinity marker which is larger than any other key.
MaxKey = roachpb.KeyMax
Expand All @@ -277,6 +277,15 @@ const (
// cockroach.
MaxReservedDescID = 49

// MinUserDescID is the first descriptor ID available for user
// structured data.
MinUserDescID = MaxReservedDescID + 1

// MinNonPredefinedUserDescID is the first descriptor ID used by
// user-level objects that are not created automatically on empty
// clusters (default databases).
MinNonPredefinedUserDescID = MinUserDescID + 2

// VirtualDescriptorID is the ID used by all virtual descriptors.
VirtualDescriptorID = math.MaxUint32

Expand Down
4 changes: 2 additions & 2 deletions pkg/server/admin_test.go
Expand Up @@ -248,7 +248,7 @@ func TestAdminAPIDatabases(t *testing.T) {
t.Fatal(err)
}

expectedDBs := []string{"system", testdb}
expectedDBs := []string{"defaultdb", "postgres", "system", testdb}
if a, e := len(resp.Databases), len(expectedDBs); a != e {
t.Fatalf("length of result %d != expected %d", a, e)
}
Expand Down Expand Up @@ -689,7 +689,7 @@ func TestAdminAPIEvents(t *testing.T) {
{sql.EventLogNodeJoin, false, 0, 1},
{sql.EventLogNodeRestart, false, 0, 0},
{sql.EventLogDropDatabase, false, 0, 0},
{sql.EventLogCreateDatabase, false, 0, 1},
{sql.EventLogCreateDatabase, false, 0, 3},
{sql.EventLogDropTable, false, 0, 2},
{sql.EventLogCreateTable, false, 0, 3},
{sql.EventLogSetClusterSetting, false, 0, 5},
Expand Down
26 changes: 21 additions & 5 deletions pkg/server/testserver.go
Expand Up @@ -373,16 +373,32 @@ func ExpectedInitialRangeCount(db *client.DB) (int, error) {
if err != nil {
return 0, err
}
maxDescriptorID := descriptorIDs[len(descriptorIDs)-1]

// System table splits occur at every possible table boundary between the end
// of the system config ID space (keys.MaxSystemConfigDescID) and the system
// table with the maximum ID (maxDescriptorID), even when an ID within the
// span does not have an associated descriptor.
systemTableSplits := int(maxDescriptorID - keys.MaxSystemConfigDescID)
// table with the maximum ID (maxSystemDescriptorID), even when an ID within
// the span does not have an associated descriptor.
maxSystemDescriptorID := descriptorIDs[0]
for _, descID := range descriptorIDs {
if descID > maxSystemDescriptorID && descID <= keys.MaxReservedDescID {
maxSystemDescriptorID = descID
}
}
systemTableSplits := int(maxSystemDescriptorID - keys.MaxSystemConfigDescID)

// User table splits are analogous to system table splits: they occur at every
// possible table boundary between the end of the system ID space
// (keys.MaxReservedDescID) and the user table with the maximum ID
// (maxUserDescriptorID), even when an ID within the span does not have an
// associated descriptor.
maxUserDescriptorID := descriptorIDs[len(descriptorIDs)-1]
userTableSplits := 0
if maxUserDescriptorID >= keys.MaxReservedDescID {
userTableSplits = int(maxUserDescriptorID - keys.MaxReservedDescID)
}

// `n` splits create `n+1` ranges.
return len(config.StaticSplits()) + systemTableSplits + 1, nil
return len(config.StaticSplits()) + systemTableSplits + userTableSplits + 1, nil
}

// WaitForInitialSplits waits for the server to complete its expected initial
Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/conn_executor.go
Expand Up @@ -421,9 +421,13 @@ func (sp sessionParams) sessionData(
if sp.data != nil {
return *sp.data
}
curDb := sp.args.Database
if curDb == "" {
curDb = sessiondata.DefaultDatabaseName
}
sd := sessiondata.SessionData{
ApplicationName: sp.args.ApplicationName,
Database: sp.args.Database,
Database: curDb,
DistSQLMode: sessiondata.DistSQLExecMode(DistSQLClusterExecMode.Get(&settings.SV)),
SearchPath: sqlbase.DefaultSearchPath,
Location: time.UTC,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor_test.go
Expand Up @@ -284,7 +284,7 @@ func TestNonRetriableErrorOnAutoCommit(t *testing.T) {
func TestErrorOnRollback(t *testing.T) {
defer leaktest.AfterTest(t)()

const targetKeyString string = "/Table/51/1/1/0"
const targetKeyString string = "/Table/53/1/1/0"
var injectedErr int64

// We're going to inject an error into our EndTransaction.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/create_test.go
Expand Up @@ -43,7 +43,7 @@ func TestDatabaseDescriptor(t *testing.T) {
defer s.Stopper().Stop(context.TODO())
ctx := context.TODO()

expectedCounter := int64(keys.MaxReservedDescID + 1)
expectedCounter := int64(keys.MinNonPredefinedUserDescID)

// Test values before creating the database.
// descriptor ID counter.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/distsqlrun/cluster_test.go
Expand Up @@ -536,7 +536,7 @@ func TestDistSQLReadsFillGatewayID(t *testing.T) {
if !ok {
return nil
}
if !strings.HasPrefix(scanReq.Span.Key.String(), "/Table/51/1") {
if !strings.HasPrefix(scanReq.Span.Key.String(), "/Table/53/1") {
return nil
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/drop_test.go
Expand Up @@ -501,7 +501,7 @@ func TestDropTable(t *testing.T) {
}

tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "kv")
nameKey := sqlbase.MakeNameMetadataKey(keys.MaxReservedDescID+1, "kv")
nameKey := sqlbase.MakeNameMetadataKey(keys.MinNonPredefinedUserDescID, "kv")
gr, err := kvDB.Get(ctx, nameKey)

if err != nil {
Expand Down Expand Up @@ -590,7 +590,7 @@ func TestDropTableDeleteData(t *testing.T) {

descs = append(descs, sqlbase.GetTableDescriptor(kvDB, "t", tableName))

nameKey := sqlbase.MakeNameMetadataKey(keys.MaxReservedDescID+1, tableName)
nameKey := sqlbase.MakeNameMetadataKey(keys.MinNonPredefinedUserDescID, tableName)
gr, err := kvDB.Get(ctx, nameKey)
if err != nil {
t.Fatal(err)
Expand Down

0 comments on commit 01eaefd

Please sign in to comment.