Skip to content

Commit

Permalink
Add Zones system table.
Browse files Browse the repository at this point in the history
Work towards: #2090

Additional tweaks:
* fix all tests that depend on the number of system databases/tables
  to use dynamic counts.
* rework the SystemDBSPan loading a bit. change checksum to sha1 and
  include keys.
  • Loading branch information
marc committed Sep 10, 2015
1 parent 4e055b0 commit 9e048b7
Show file tree
Hide file tree
Showing 7 changed files with 107 additions and 35 deletions.
8 changes: 5 additions & 3 deletions kv/dist_sender_server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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 {
Expand Down
30 changes: 14 additions & 16 deletions server/node_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
Expand Down
3 changes: 1 addition & 2 deletions sql/create_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Expand Down
26 changes: 24 additions & 2 deletions sql/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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 (
Expand All @@ -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.
Expand All @@ -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 {
Expand All @@ -115,6 +132,7 @@ func createSystemTable(id ID, cmd string) TableDescriptor {
log.Fatal(err)
}

NumUsedSystemIDs++
return desc
}

Expand All @@ -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{
Expand Down
35 changes: 35 additions & 0 deletions sql/system_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
}
15 changes: 15 additions & 0 deletions sql/testdata/system
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ SHOW TABLES FROM system
descriptor
namespace
users
zones

query ITTB
EXPLAIN (DEBUG) SELECT * FROM system.namespace
Expand All @@ -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
Expand All @@ -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
Expand All @@ -36,6 +39,7 @@ SELECT id FROM system.descriptor
2
3
4
5
1000

# Verify format of system tables.
Expand All @@ -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
Expand All @@ -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
Expand Down
25 changes: 13 additions & 12 deletions storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ package storage

import (
"bytes"
"crypto/sha1"
"crypto/sha256"
"fmt"
"math/rand"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
}
}
Expand Down Expand Up @@ -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
Expand Down

0 comments on commit 9e048b7

Please sign in to comment.