Skip to content

Commit

Permalink
Merge pull request #1356 from cockroachdb/vivek/keys
Browse files Browse the repository at this point in the history
Keys for the table data
  • Loading branch information
vivekmenezes committed Jun 16, 2015
2 parents 47b2b0e + 0f06ca5 commit f879530
Show file tree
Hide file tree
Showing 4 changed files with 118 additions and 6 deletions.
3 changes: 3 additions & 0 deletions keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,4 +175,7 @@ var (
StatusStorePrefix = MakeKey(StatusPrefix, proto.Key("store-"))
// StatusNodePrefix stores all status info for nodes.
StatusNodePrefix = MakeKey(StatusPrefix, proto.Key("node-"))
// TableDataPrefix prefixes all Table data to aid in transitioning
// key:value data to Table data, and for ease of debugging.
TableDataPrefix = proto.Key("table-")
)
54 changes: 48 additions & 6 deletions keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,6 @@ import (
"github.com/cockroachdb/cockroach/util/log"
)

const (
maxVarintSize = 10
)

// MakeKey makes a new key which is the concatenation of the
// given inputs, in order.
func MakeKey(keys ...proto.Key) proto.Key {
Expand Down Expand Up @@ -62,7 +58,7 @@ func NodeStatusKey(nodeID int32) proto.Key {

// MakeNameMetadataKey returns the key for the namespace.
func MakeNameMetadataKey(parentID uint32, name string) proto.Key {
k := make([]byte, 0, len(NameMetadataPrefix)+maxVarintSize+len(name))
k := make([]byte, 0, len(NameMetadataPrefix)+encoding.MaxUvarintSize+len(name))
k = append(k, NameMetadataPrefix...)
k = encoding.EncodeUvarint(k, uint64(parentID))
k = append(k, name...)
Expand All @@ -71,12 +67,58 @@ func MakeNameMetadataKey(parentID uint32, name string) proto.Key {

// MakeDescMetadataKey returns the key for the table in namespaceID.
func MakeDescMetadataKey(descID uint32) proto.Key {
k := make([]byte, 0, len(DescMetadataPrefix)+maxVarintSize)
k := make([]byte, 0, len(DescMetadataPrefix)+encoding.MaxUvarintSize)
k = append(k, DescMetadataPrefix...)
k = encoding.EncodeUvarint(k, uint64(descID))
return k
}

// indexKeyBufferWidth returns a likely cap on the width of the index key.
// The buffer width can likely accomodate the encoded constant prefix, tableID,
// indexID, and column values.
//
// This cap is inaccurate because the size of the encoding varies, depending
// on the ints and the bytes being encoded. We really don't care, as long as
// a value is chosen such that the append() builtin used to populate the
// buffer, infrequently reallocates more space.
func indexKeyMaxBufferWidth(columnValues ...[]byte) (width int) {
// Accomodate the constant prefix, tableID, and indexID.
width += len(TableDataPrefix) + 2*encoding.MaxUvarintSize
for _, value := range columnValues {
// Add 2 for encoding
width += len(value) + 2
}
return
}

// populateTableIndexKey populates the key passed in with the
// order encoded values forming the index key.
func populateTableIndexKey(key []byte, tableID, indexID uint32, columnValues ...[]byte) []byte {
key = append(key, TableDataPrefix...)
key = encoding.EncodeUvarint(key, uint64(tableID))
key = encoding.EncodeUvarint(key, uint64(indexID))
for _, value := range columnValues {
key = encoding.EncodeBytes(key, value)
}
return key
}

// MakeTableIndexKey returns a primary or a secondary index key.
func MakeTableIndexKey(tableID, indexID uint32, columnValues ...[]byte) proto.Key {
k := make([]byte, 0, indexKeyMaxBufferWidth(columnValues...))
k = populateTableIndexKey(k, tableID, indexID, columnValues...)
return k
}

// MakeTableDataKey returns a key to a value at a specific row and column
// in the table.
func MakeTableDataKey(tableID, indexID, columnID uint32, columnValues ...[]byte) proto.Key {
k := make([]byte, 0, indexKeyMaxBufferWidth(columnValues...)+encoding.MaxUvarintSize)
k = populateTableIndexKey(k, tableID, indexID, columnValues...)
k = encoding.EncodeUvarint(k, uint64(columnID))
return k
}

// MakeRangeIDKey creates a range-local key based on the range's
// Raft ID, metadata key suffix, and optional detail (e.g. the
// encoded command ID for a response cache entry, etc.).
Expand Down
62 changes: 62 additions & 0 deletions keys/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/cockroachdb/cockroach/proto"
"github.com/cockroachdb/cockroach/util"
"github.com/cockroachdb/cockroach/util/encoding"
"github.com/cockroachdb/cockroach/util/leaktest"
)

Expand Down Expand Up @@ -75,6 +76,67 @@ func TestKeyAddress(t *testing.T) {
}
}

func TestMakeTableIndexKey(t *testing.T) {
defer leaktest.AfterTest(t)
key := MakeTableIndexKey(12, 345, []byte("foo"), []byte("bar"))
expKey := MakeKey(TableDataPrefix, encoding.EncodeUvarint(nil, 12), encoding.EncodeUvarint(nil, 345), encoding.EncodeBytes(nil, []byte("foo")), encoding.EncodeBytes(nil, []byte("bar")))
if !key.Equal(expKey) {
t.Errorf("key %q doesn't match expected %q", key, expKey)
}
// Check that keys are ordered
keys := []proto.Key{
MakeTableIndexKey(0, 0, []byte("foo")),
MakeTableIndexKey(0, 0, []byte("fooo")),
MakeTableIndexKey(0, 1, []byte("bar")),
MakeTableIndexKey(1, 0, []byte("bar")),
MakeTableIndexKey(1, 0, []byte("bar"), []byte("foo")),
MakeTableIndexKey(1, 1, []byte("bar"), []byte("fo")),
MakeTableIndexKey(1, 1, []byte("bar"), []byte("foo")),
MakeTableIndexKey(1, 2, []byte("bar")),
MakeTableIndexKey(2, 2, []byte("ba")),
}
for i := 1; i < len(keys); i++ {
if bytes.Compare(keys[i-1], keys[i]) >= 0 {
t.Errorf("key %d >= key %d", i-1, i)
}
}
}

func TestMakeTableDataKey(t *testing.T) {
defer leaktest.AfterTest(t)
key := MakeTableDataKey(12, 345, 6, []byte("foo"), []byte("bar"))
// Expected key is the TableIndexKey + ColumnID
expKey := MakeKey(MakeTableIndexKey(12, 345, []byte("foo"), []byte("bar")), encoding.EncodeUvarint(nil, 6))
if !key.Equal(expKey) {
t.Errorf("key %q doesn't match expected %q", key, expKey)
}
// Check that keys are ordered
keys := []proto.Key{
// Data-key follows Index key order
MakeTableDataKey(0, 0, 0, []byte("foo")),
MakeTableDataKey(0, 0, 8, []byte("fooo")),
MakeTableDataKey(0, 1, 10, []byte("bar")),
MakeTableDataKey(1, 0, 3, []byte("bar")),
MakeTableDataKey(1, 0, 5, []byte("bar"), []byte("foo")),
MakeTableDataKey(1, 1, 7, []byte("bar"), []byte("fo")),
MakeTableDataKey(1, 1, 4, []byte("bar"), []byte("foo")),
MakeTableDataKey(1, 2, 89, []byte("bar")),
MakeTableDataKey(2, 2, 23, []byte("ba")),
// For the same Index key, Data-key follows column ID order.
MakeTableDataKey(2, 2, 0, []byte("bar"), []byte("foo")),
MakeTableDataKey(2, 2, 7, []byte("bar"), []byte("foo")),
MakeTableDataKey(2, 2, 23, []byte("bar"), []byte("foo")),
MakeTableDataKey(2, 2, 45, []byte("bar"), []byte("foo")),
}

for i := 1; i < len(keys); i++ {
if bytes.Compare(keys[i-1], keys[i]) >= 0 {
t.Errorf("key %d >= key %d", i-1, i)
}
}

}

func TestRangeMetaKey(t *testing.T) {
defer leaktest.AfterTest(t)
testCases := []struct {
Expand Down
5 changes: 5 additions & 0 deletions util/encoding/encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,11 @@ func DecodeVarintDecreasing(b []byte) ([]byte, int64) {
return leftover, ^v
}

const (
// MaxUvarintSize is the maximum size in bytes of an encoded uvarint.
MaxUvarintSize = 9
)

// EncodeUvarint encodes the uint64 value using a variable length
// (length-prefixed) representation. The length is encoded as a single
// byte indicating the number of encoded bytes (-8) to follow. See
Expand Down

0 comments on commit f879530

Please sign in to comment.