diff --git a/kv/dist_sender_server_test.go b/kv/dist_sender_server_test.go index 05b684ae622b..179ff3f048bf 100644 --- a/kv/dist_sender_server_test.go +++ b/kv/dist_sender_server_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/kv" "github.com/cockroachdb/cockroach/proto" "github.com/cockroachdb/cockroach/server" + "github.com/cockroachdb/cockroach/sql" "github.com/cockroachdb/cockroach/storage/engine" "github.com/cockroachdb/cockroach/testutils" "github.com/cockroachdb/cockroach/util/hlc" @@ -255,11 +256,12 @@ func TestSingleRangeReverseScan(t *testing.T) { t.Errorf("expected 2 rows; got %d", l) } // Case 3: Test proto.KeyMax - // TODO(marc): this depends on the sql system objects. + // This span covers the system DB keys. + wanted := 1 + len(sql.GetInitialSystemValues()) if rows, err := db.ReverseScan("g", proto.KeyMax, 0); err != nil { t.Fatalf("unexpected error on ReverseScan: %s", err) - } else if l := len(rows); l != 10 { - t.Errorf("expected 10 rows; got %d", l) + } else if l := len(rows); l != wanted { + t.Errorf("expected %d rows; got %d", wanted, l) } // Case 4: Test keys.SystemMax if rows, err := db.ReverseScan(keys.SystemMax, "b", 0); err != nil { diff --git a/server/node_test.go b/server/node_test.go index 2e84daad56eb..e4cd6360820c 100644 --- a/server/node_test.go +++ b/server/node_test.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/proto" "github.com/cockroachdb/cockroach/rpc" "github.com/cockroachdb/cockroach/server/status" + "github.com/cockroachdb/cockroach/sql" "github.com/cockroachdb/cockroach/storage" "github.com/cockroachdb/cockroach/storage/engine" "github.com/cockroachdb/cockroach/util" @@ -114,31 +115,28 @@ func TestBootstrapCluster(t *testing.T) { if err != nil { t.Fatal(err) } - var keys []proto.Key + var foundKeys proto.KeySlice for _, kv := range rows { - keys = append(keys, kv.Key) + foundKeys = append(foundKeys, kv.Key) } - // TODO(marc): this depends on the sql system objects. - var expectedKeys = []proto.Key{ + var expectedKeys = proto.KeySlice{ proto.MakeKey(proto.Key("\x00\x00meta1"), proto.KeyMax), proto.MakeKey(proto.Key("\x00\x00meta2"), proto.KeyMax), - proto.Key("\x00desc-idgen"), proto.Key("\x00node-idgen"), proto.Key("\x00range-tree-root"), proto.Key("\x00store-idgen"), proto.Key("\x00zone"), - proto.Key("\xff\n\x02\n\x01\tsystem\x00\x01\n\x03"), - proto.Key("\xff\n\x02\n\x01\n\x01descriptor\x00\x01\n\x03"), - proto.Key("\xff\n\x02\n\x01\n\x01namespace\x00\x01\n\x03"), - proto.Key("\xff\n\x02\n\x01\n\x01users\x00\x01\n\x03"), - proto.Key("\xff\n\x03\n\x01\n\x01\n\x02"), - proto.Key("\xff\n\x03\n\x01\n\x02\n\x02"), - proto.Key("\xff\n\x03\n\x01\n\x03\n\x02"), - proto.Key("\xff\n\x03\n\x01\n\x04\n\x02"), - } - if !reflect.DeepEqual(keys, expectedKeys) { + } + // Add the initial keys for sql. + for _, kv := range sql.GetInitialSystemValues() { + expectedKeys = append(expectedKeys, kv.Key) + } + // Resort the list. The sql values are not sorted. + sort.Sort(expectedKeys) + + if !reflect.DeepEqual(foundKeys, expectedKeys) { t.Errorf("expected keys mismatch:\n%s\n -- vs. -- \n\n%s", - formatKeys(keys), formatKeys(expectedKeys)) + formatKeys(foundKeys), formatKeys(expectedKeys)) } // TODO(spencer): check values. diff --git a/sql/create_test.go b/sql/create_test.go index 6bb944b4fa09..1008f86e124b 100644 --- a/sql/create_test.go +++ b/sql/create_test.go @@ -72,8 +72,7 @@ func TestDatabaseDescriptor(t *testing.T) { if kvs, err := kvDB.Scan(start, start.PrefixEnd(), 0); err != nil { t.Fatal(err) } else { - // TODO(marc): this is the number of system tables + 1. - if a, e := len(kvs), 4; a != e { + if a, e := len(kvs), sql.NumUsedSystemIDs; a != e { t.Fatalf("expected %d keys to have been written, found %d keys", e, a) } } diff --git a/sql/system.go b/sql/system.go index 7d19f6d7dfe2..af983167c795 100644 --- a/sql/system.go +++ b/sql/system.go @@ -40,9 +40,9 @@ const ( namespaceTableID ID = 2 descriptorTableID ID = 3 usersTableID ID = 4 + zonesTableID ID = 5 // sql CREATE commands and full schema for each system table. - // TODO(marc): wouldn't it be better to use a pre-parsed version? namespaceTableSchema = ` CREATE TABLE system.namespace ( parentID INT, @@ -62,6 +62,13 @@ CREATE TABLE system.users ( username CHAR PRIMARY KEY, hashedPassword BLOB );` + + // Zone settings per DB/Table. + zonesTableSchema = ` +CREATE TABLE system.zones ( + id INT PRIMARY KEY, + config BLOB +);` ) var ( @@ -83,6 +90,9 @@ var ( // UsersTable is the descriptor for the users table. UsersTable = createSystemTable(usersTableID, usersTableSchema) + // ZonesTable is the descriptor for the zones table. + ZonesTable = createSystemTable(zonesTableID, zonesTableSchema) + // SystemAllowedPrivileges describes the privileges allowed for each // system object. No user may have more than those privileges, and // the root user must have exactly those privileges. @@ -92,7 +102,14 @@ var ( namespaceTableID: privilege.ReadData, descriptorTableID: privilege.ReadData, usersTableID: privilege.ReadWriteData, + zonesTableID: privilege.ReadWriteData, } + + // NumUsedSystemIDs is only used in tests that need to know the + // number of system objects created at initialization. + // It gets automatically set to "number of created system tables" + // + 1 (for system database). + NumUsedSystemIDs = 1 ) func createSystemTable(id ID, cmd string) TableDescriptor { @@ -115,6 +132,7 @@ func createSystemTable(id ID, cmd string) TableDescriptor { log.Fatal(err) } + NumUsedSystemIDs++ return desc } @@ -129,13 +147,17 @@ func GetInitialSystemValues() []proto.KeyValue { {SystemDB.ID, &NamespaceTable}, {SystemDB.ID, &DescriptorTable}, {SystemDB.ID, &UsersTable}, + {SystemDB.ID, &ZonesTable}, } + // Initial kv pairs: + // - ID generator + // - 2 per table/database numEntries := 1 + len(systemData)*2 ret := make([]proto.KeyValue, numEntries, numEntries) i := 0 - // We reserve the system IDs. + // Descriptor ID generator. value := proto.Value{} value.SetInteger(int64(MaxReservedDescID + 1)) ret[i] = proto.KeyValue{ diff --git a/sql/system_test.go b/sql/system_test.go new file mode 100644 index 000000000000..09ae625c5274 --- /dev/null +++ b/sql/system_test.go @@ -0,0 +1,35 @@ +// Copyright 2015 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. See the AUTHORS file +// for names of contributors. +// +// Author: Marc Berhault (marc@cockroachlabs.com) + +package sql_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/sql" + "github.com/cockroachdb/cockroach/util/leaktest" +) + +func TestInitialKeys(t *testing.T) { + defer leaktest.AfterTest(t) + + kv := sql.GetInitialSystemValues() + // IDGenerator + 2 for each table/database. + if actual, expected := len(kv), 1+2*sql.NumUsedSystemIDs; actual != expected { + t.Fatalf("Wrong number of initial sql kv pairs: %d, wanted %d", actual, expected) + } +} diff --git a/sql/testdata/system b/sql/testdata/system index ab9689b5736e..7607ab688acb 100644 --- a/sql/testdata/system +++ b/sql/testdata/system @@ -10,6 +10,7 @@ SHOW TABLES FROM system descriptor namespace users +zones query ITTB EXPLAIN (DEBUG) SELECT * FROM system.namespace @@ -19,6 +20,7 @@ EXPLAIN (DEBUG) SELECT * FROM system.namespace 2 /namespace/primary/1/'descriptor'/id 3 true 3 /namespace/primary/1/'namespace'/id 2 true 4 /namespace/primary/1/'users'/id 4 true +5 /namespace/primary/1/'zones'/id 5 true query ITI SELECT * FROM system.namespace @@ -28,6 +30,7 @@ SELECT * FROM system.namespace 1 descriptor 3 1 namespace 2 1 users 4 +1 zones 5 query I SELECT id FROM system.descriptor @@ -36,6 +39,7 @@ SELECT id FROM system.descriptor 2 3 4 +5 1000 # Verify format of system tables. @@ -58,6 +62,12 @@ SHOW COLUMNS FROM system.users; username STRING true hashedPassword BYTES true +query TTT +SHOW COLUMNS FROM system.zones; +---- +id INT true +config BYTES true + # Verify default privileges on system tables. query TTT SHOW GRANTS ON DATABASE system @@ -79,6 +89,11 @@ SHOW GRANTS ON system.users ---- users root DELETE,GRANT,INSERT,SELECT,UPDATE +query TTT +SHOW GRANTS ON system.zones +---- +zones root DELETE,GRANT,INSERT,SELECT,UPDATE + # Non-root users can have privileges on system objects, but limited to GRANT, SELECT. statement error user testuser must not have ALL privileges on system objects GRANT ALL ON DATABASE system TO testuser diff --git a/storage/replica.go b/storage/replica.go index a395c3bfd51d..3f6f0e30628c 100644 --- a/storage/replica.go +++ b/storage/replica.go @@ -22,6 +22,7 @@ package storage import ( "bytes" + "crypto/sha1" "crypto/sha256" "fmt" "math/rand" @@ -202,7 +203,7 @@ type Replica struct { // Last index applied to the state machine. Updated atomically. appliedIndex uint64 configHashes map[int][]byte // Config map sha256 hashes @ last gossip - systemDBHash []byte // sha256 hash of the system config @ last gossip + systemDBHash []byte // sha1 hash of the system config @ last gossip lease unsafe.Pointer // Information for leader lease, updated atomically llMu sync.Mutex // Synchronizes readers' requests for leader lease respCache *ResponseCache // Provides idempotence for retries @@ -1298,20 +1299,22 @@ func (r *Replica) maybeGossipSystemConfigLocked() { ctx := r.context() // TODO(marc): check for bad split in the middle of the SystemDB span. - systemConfig, hash, err := loadSystemConfig(r.rm.Engine()) + kvs, hash, err := loadSystemDBSpan(r.rm.Engine()) if err != nil { - log.Errorc(ctx, "could not load system config: %s", err) + log.Errorc(ctx, "could not load SystemDB span: %s", err) return } if bytes.Equal(r.systemDBHash, hash) { return } + cfg := &config.SystemConfig{Values: kvs} + r.systemDBHash = hash if log.V(1) { log.Infoc(ctx, "gossiping system config from store %d, range %d", r.rm.StoreID(), r.Desc().RangeID) } - if err := r.rm.Gossip().AddInfoProto(gossip.KeySystemDB, systemConfig, 0); err != nil { + if err := r.rm.Gossip().AddInfoProto(gossip.KeySystemDB, cfg, 0); err != nil { log.Errorc(ctx, "failed to gossip system config: %s", err) } } @@ -1534,23 +1537,21 @@ func loadConfigMap(eng engine.Engine, keyPrefix proto.Key, configI gogoproto.Mes return m, sha.Sum(nil), err } -// loadSystemConfig scans the entire SystemDB span and puts the set of key/value -// pairs in the config, generating a sha256 sum. -func loadSystemConfig(eng engine.Engine) (*config.SystemConfig, []byte, error) { +// loadSystemDBSpan scans the entire SystemDB span and returns the full list of +// key/value pairs along with the sha1 checksum of the contents (key and value). +func loadSystemDBSpan(eng engine.Engine) ([]proto.KeyValue, []byte, error) { // TODO(tschottdorf): Currently this does not handle intents well. kvs, _, err := engine.MVCCScan(eng, keys.SystemDBSpan.Start, keys.SystemDBSpan.End, 0, proto.MaxTimestamp, true /* consistent */, nil) if err != nil { return nil, nil, err } - cfg := &config.SystemConfig{ - Values: kvs, - } - sha := sha256.New() + sha := sha1.New() for _, kv := range kvs { + sha.Write(kv.Key) sha.Write(kv.Value.Bytes) } - return cfg, sha.Sum(nil), err + return kvs, sha.Sum(nil), err } // maybeAddToSplitQueue checks whether the current size of the range