Skip to content

Commit

Permalink
*: set track smallest / largest keys separately in manifest
Browse files Browse the repository at this point in the history
Currently, only point keys are tracked in the manifest. With the
addition of range keys, the bounds of an SSTable should be computed by
considering the bounds of both the point keys and the range keys, and
taking the smallest or largest across both types of key, respectively.

Add four additional fields, `{Smallest,Largest}{Point,Range}Key`, to
`manifest.FileMetadata` to separately track the point and range key
bounds. The existing `Smallest` and `Largest` fields are used to track
the bounds across both point and range keys.

Update the existing calls sites that set the smallest and largest keys
to set all three types of bounds: point keys, range keys and combined.
  • Loading branch information
nicktrav committed Feb 17, 2022
1 parent 51cf6fb commit e0cc4d4
Show file tree
Hide file tree
Showing 11 changed files with 595 additions and 64 deletions.
4 changes: 4 additions & 0 deletions compaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -2259,6 +2259,10 @@ func (d *DB) runCompaction(
)
}

meta.SmallestPointKey = writerMeta.SmallestPointKey(d.cmp)
meta.LargestPointKey = writerMeta.LargestPointKey(d.cmp)
meta.SmallestRangeKey = writerMeta.SmallestRangeKey
meta.LargestRangeKey = writerMeta.LargestRangeKey
meta.Smallest = writerMeta.Smallest(d.cmp)
meta.Largest = writerMeta.Largest(d.cmp)

Expand Down
18 changes: 11 additions & 7 deletions flush_external.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,17 @@ func flushExternalTable(untypedDB interface{}, path string, originalMeta *fileMe
d.mu.Unlock()

m := &fileMetadata{
FileNum: fileNum,
Size: originalMeta.Size,
CreationTime: time.Now().Unix(),
Smallest: originalMeta.Smallest,
Largest: originalMeta.Largest,
SmallestSeqNum: originalMeta.SmallestSeqNum,
LargestSeqNum: originalMeta.LargestSeqNum,
FileNum: fileNum,
Size: originalMeta.Size,
CreationTime: time.Now().Unix(),
SmallestPointKey: originalMeta.SmallestPointKey,
LargestPointKey: originalMeta.LargestPointKey,
SmallestRangeKey: originalMeta.SmallestRangeKey,
LargestRangeKey: originalMeta.LargestRangeKey,
Smallest: originalMeta.Smallest,
Largest: originalMeta.Largest,
SmallestSeqNum: originalMeta.SmallestSeqNum,
LargestSeqNum: originalMeta.LargestSeqNum,
}

// Hard link the sstable into the DB directory.
Expand Down
148 changes: 120 additions & 28 deletions ingest.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/private"
"github.com/cockroachdb/pebble/internal/rangekey"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
)
Expand Down Expand Up @@ -84,8 +85,6 @@ func ingestLoad1(
meta.FileNum = fileNum
meta.Size = uint64(stat.Size())
meta.CreationTime = time.Now().Unix()
meta.Smallest = InternalKey{}
meta.Largest = InternalKey{}

// Avoid loading into into the table cache for collecting stats if we
// don't need to. If there are no range deletions, we have all the
Expand All @@ -98,9 +97,7 @@ func ingestLoad1(
// calculating stats before we can remove the original link.
maybeSetStatsFromProperties(meta, &r.Properties)

smallestSet, largestSet := false, false
empty := true

hasPoints := false
{
iter, err := r.NewIter(nil /* lower */, nil /* upper */)
if err != nil {
Expand All @@ -111,9 +108,7 @@ func ingestLoad1(
if err := ingestValidateKey(opts, key); err != nil {
return nil, err
}
empty = false
meta.Smallest = key.Clone()
smallestSet = true
meta.SmallestPointKey = key.Clone()
}
if err := iter.Error(); err != nil {
return nil, err
Expand All @@ -122,9 +117,8 @@ func ingestLoad1(
if err := ingestValidateKey(opts, key); err != nil {
return nil, err
}
empty = false
meta.Largest = key.Clone()
largestSet = true
meta.LargestPointKey = key.Clone()
hasPoints = true // Implies smallest point key was also set.
}
if err := iter.Error(); err != nil {
return nil, err
Expand All @@ -141,10 +135,9 @@ func ingestLoad1(
if err := ingestValidateKey(opts, key); err != nil {
return nil, err
}
empty = false
if !smallestSet ||
base.InternalCompare(opts.Comparer.Compare, meta.Smallest, *key) > 0 {
meta.Smallest = key.Clone()
if !hasPoints ||
base.InternalCompare(opts.Comparer.Compare, meta.SmallestPointKey, *key) > 0 {
meta.SmallestPointKey = key.Clone()
}
}
if err := iter.Error(); err != nil {
Expand All @@ -154,18 +147,87 @@ func ingestLoad1(
if err := ingestValidateKey(opts, key); err != nil {
return nil, err
}
empty = false
end := base.MakeRangeDeleteSentinelKey(val)
if !largestSet ||
base.InternalCompare(opts.Comparer.Compare, meta.Largest, end) < 0 {
meta.Largest = end.Clone()
if !hasPoints ||
base.InternalCompare(opts.Comparer.Compare, meta.LargestPointKey, end) < 0 {
meta.LargestPointKey = end.Clone()
hasPoints = true // Implies smallest point key was also set.
}
}
}

// Update the range-key bounds for the table.
var hasRanges bool
{
iter, err := r.NewRawRangeKeyIter()
if err != nil {
return nil, err
}
if iter != nil {
defer iter.Close()
if key, _ := iter.First(); key != nil {
if err := ingestValidateKey(opts, key); err != nil {
return nil, err
}
meta.SmallestRangeKey = key.Clone()
}
if err := iter.Error(); err != nil {
return nil, err
}
if key, value := iter.Last(); key != nil {
if err := ingestValidateKey(opts, key); err != nil {
return nil, err
}
// As range keys are fragmented, the end key of the last range key in
// the table provides the upper bound for the table.
end, _, ok := rangekey.DecodeEndKey(key.Kind(), value)
if !ok {
return nil, errors.Newf("pebble: could not decode range end key")
}
meta.LargestRangeKey = base.MakeRangeKeySentinelKey(end).Clone()
hasRanges = true // Implies smallest range key was also set.
}
if err := iter.Error(); err != nil {
return nil, err
}
}
}

if empty {
if !hasPoints && !hasRanges {
return nil, nil
}

// Compute the overall smallest / largest fields from the point and key
// ranges.
switch {
case !hasRanges:
// Table has only point keys. Use the point key bounds.
meta.Smallest = meta.SmallestPointKey.Clone()
meta.Largest = meta.LargestPointKey.Clone()
case !hasPoints:
// Table has only range key. Use the range key bounds.
meta.Smallest = meta.SmallestRangeKey.Clone()
meta.Largest = meta.LargestRangeKey.Clone()
default:
// Table has both points and ranges. Compute the bounds by considering both
// the point and range key bounds.
if base.InternalCompare(opts.Comparer.Compare, meta.SmallestPointKey, meta.SmallestRangeKey) < 0 {
meta.Smallest = meta.SmallestPointKey.Clone()
} else {
meta.Smallest = meta.SmallestRangeKey.Clone()
}
if base.InternalCompare(opts.Comparer.Compare, meta.LargestPointKey, meta.LargestRangeKey) > 0 {
meta.Largest = meta.LargestPointKey.Clone()
} else {
meta.Largest = meta.LargestRangeKey.Clone()
}
}

// Sanity check that the various bounds on the file were set consistently.
if err := meta.Validate(opts.Comparer.Compare, opts.Comparer.FormatKey); err != nil {
return nil, err
}

return meta, nil
}

Expand Down Expand Up @@ -313,19 +375,47 @@ func ingestMemtableOverlaps(cmp Compare, mem flushable, meta []*fileMetadata) bo
return false
}

func ingestUpdateSeqNum(opts *Options, dirname string, seqNum uint64, meta []*fileMetadata) error {
func ingestUpdateSeqNum(
cmp Compare,
format base.FormatKey,
seqNum uint64,
meta []*fileMetadata,
) error {
setSeqFn := func(k base.InternalKey) base.InternalKey {
return base.MakeInternalKey(k.UserKey, seqNum, k.Kind())
}
for _, m := range meta {
m.Smallest = base.MakeInternalKey(m.Smallest.UserKey, seqNum, m.Smallest.Kind())
// Don't update the seqnum for the largest key if that key is a range
// deletion sentinel key as doing so unintentionally extends the bounds of
// the table.
if m.Largest.Trailer != InternalKeyRangeDeleteSentinel {
m.Largest = base.MakeInternalKey(m.Largest.UserKey, seqNum, m.Largest.Kind())
// TODO(travers): We currently lack a means of determining whether one of
// the bounds is "unset" (e.g. a table without point keys). We use a lack of
// user key as a cheap way of checking, for now, until we have a better way.
if m.SmallestPointKey.UserKey != nil {
m.SmallestPointKey = setSeqFn(m.SmallestPointKey)
}
if m.SmallestRangeKey.UserKey != nil {
m.SmallestRangeKey = setSeqFn(m.SmallestRangeKey)
}
m.Smallest = setSeqFn(m.Smallest)
// Only update the seqnum for the largest key if that key is not an
// "exclusive sentinel" (i.e. a range deletion sentinel or a range key
// boundary), as doing so effectively drops the exclusive sentinel (by
// lowering the seqnum from the max value), and extends the bounds of the
// table.
// NB: as the largest range key is always an exclusive sentinel, it is never
// updated.
if m.LargestPointKey.UserKey != nil && !m.LargestPointKey.IsExclusiveSentinel() {
m.LargestPointKey = setSeqFn(m.LargestPointKey)
}
if !m.Largest.IsExclusiveSentinel() {
m.Largest = setSeqFn(m.Largest)
}
// Setting smallestSeqNum == largestSeqNum triggers the setting of
// Properties.GlobalSeqNum when an sstable is loaded.
m.SmallestSeqNum = seqNum
m.LargestSeqNum = seqNum
// Ensure the new bounds are consistent.
if err := m.Validate(cmp, format); err != nil {
return err
}
seqNum++
}
return nil
Expand Down Expand Up @@ -611,7 +701,9 @@ func (d *DB) Ingest(paths []string) error {
// metadata. Writing the metadata to the manifest when the
// version edit is applied is the mechanism that persists the
// sequence number. The sstables themselves are left unmodified.
if err = ingestUpdateSeqNum(d.opts, d.dirname, seqNum, meta); err != nil {
if err = ingestUpdateSeqNum(
d.cmp, d.opts.Comparer.FormatKey, seqNum, meta,
); err != nil {
return
}

Expand Down
Loading

0 comments on commit e0cc4d4

Please sign in to comment.