lsmkv

package
v0.0.0-...-72cd24a Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Dec 4, 2023 License: GPL-3.0 Imports: 33 Imported by: 0

Documentation

Overview

LSMKV (= Log-structured Merge-Tree Key-Value Store)

This package contains Weaviate's custom LSM store. While modeled after the usecases that are required for Weaviate to be fast, reliable, and scalable, it is technically completely independent. You could build your own database on top of this key-value store.

Covering the architecture of LSM Stores in general goes beyond the scope of this documentation. Therefore things that are specific to this implementation are highlighted.

Strategies

To understand the different type of buckets in this store, you need to familiarize yourself with the following strategies. A strategy defines a different usecase for a Bucket.

  • "Replace"

    Replace resembles the classical key-value store. Each key has exactly one value. A subsequent PUT on an an existing key, replaces the value (hence the name "replace"). Once replaced a former value can no longer be retrieved, and will eventually be removed in compactions.

  • "Set" (aka "SetCollection")

    A set behaves like an unordered collection of independent values. In other words a single key has multiple values. For example, for key "foo", you could have values "bar1", "bar2", "bazzinga". A bucket of this type is optimized for cheap writes to add new set additions. For example adding another set element has a fixed cost independent of the number of the existing set length. This makes it very well suited for building an inverted index.

    Retrieving a Set has a slight cost to it if a set is spread across multiple segments. This cost will eventually reduce as more and more compactions happen. In the ideal case (fully compacted DB), retrieving a Set requires just a single disk read.

  • "Map" (aka "MapCollection")

    Maps are similar to Sets in the sense that for a single key there are multiple values. However, each value is in itself a key-value pair. This makes this type very similar to a dict or hashmap type. For example for key "foo", you could have value pairs: "bar":17, "baz":19.

    This makes a map a great use case for an inverted index that needs to store additional info beyond just the docid-pointer, such as in the case of a BM25 index where the term frequency needs to be stored.

    The same performance-considerations as for sets apply.

Navigate around these docs

Good entrypoints to learn more about how this package works include Store with New and Store.CreateOrLoadBucket, as well as Bucket with Bucket.Get, Bucket.GetBySecondary, Bucket.Put, etc.

Each strategy also supports cursor types: CursorReplace can be created using Bucket.Cursor, CursorSet can be created with Bucket.SetCursor , and CursorMap can be created with Bucket.MapCursor.

Index

Constants

View Source
const (
	// StrategyReplace allows for idem-potent PUT where the latest takes presence
	StrategyReplace       = "replace"
	StrategySetCollection = "setcollection"
	StrategyMapCollection = "mapcollection"
	StrategyRoaringSet    = "roaringset"
)

Variables

View Source
var ErrInvalidChecksum = errors.New("invalid checksum")

ErrInvalidChecksum indicates that the read file should not be trusted. For any pre-computed data this is a recoverable issue, as the data can simply be re-computed at read-time.

Functions

func CheckExpectedStrategy

func CheckExpectedStrategy(strategy string, expectedStrategies ...string)

func IsExpectedStrategy

func IsExpectedStrategy(strategy string, expectedStrategies ...string) bool

func ParseCollectionNode

func ParseCollectionNode(r io.Reader) (segmentCollectionNode, error)

ParseCollectionNode reads from r and parses the collection values into a segmentCollectionNode

When only given an offset, r is constructed as a *bufio.Reader to avoid first reading the entire segment (could be GBs). Each consecutive read will be buffered to avoid excessive syscalls.

When we already have a finite and manageable []byte (i.e. when we have already seeked to an lsmkv node and have start+end offset), r should be constructed as a *bytes.Reader, since the contents have already been `pread` from the segment contentFile.

func ParseCollectionNodeInto

func ParseCollectionNodeInto(r io.Reader, node *segmentCollectionNode) error

ParseCollectionNodeInto takes the []byte slice and parses it into the specified node. It does not perform any copies and the caller must be aware that memory may be shared between the two. As a result, the caller must make sure that they do not modify "in" while "node" is still in use. A safer alternative is to use ParseCollectionNode.

The primary intention of this function is to provide a way to reuse buffers when the lifetime is controlled tightly, for example in cursors used within compactions. Use at your own risk!

If the buffers of the provided node have enough capacity they will be reused. Only if the capacity is not enough, will an allocation occur. This allocation uses 25% overhead to avoid future allocations for nodes of similar size.

As a result calling this method only makes sense if you plan on calling it multiple times. Calling it just once on an uninitialized node does not have major advantages over calling ParseCollectionNode.

func ParseReplaceNode

func ParseReplaceNode(r io.Reader, secondaryIndexCount uint16) (segmentReplaceNode, error)

func ParseReplaceNodeIntoMMAP

func ParseReplaceNodeIntoMMAP(r *byteops.ReadWriter, secondaryIndexCount uint16, out *segmentReplaceNode) error

func ParseReplaceNodeIntoPread

func ParseReplaceNodeIntoPread(r io.Reader, secondaryIndexCount uint16, out *segmentReplaceNode) error

func SegmentStrategyFromString

func SegmentStrategyFromString(in string) segmentindex.Strategy

Types

type Bucket

type Bucket struct {
	// contains filtered or unexported fields
}

func MustNewBucket

func MustNewBucket(ctx context.Context, dir, rootDir string, logger logrus.FieldLogger, compactionCallbacks, flushCallbacks cyclemanager.CycleCallbackGroup, opts ...BucketOption) *Bucket

func NewBucket

func NewBucket(ctx context.Context, dir, rootDir string, logger logrus.FieldLogger, compactionCallbacks, flushCallbacks cyclemanager.CycleCallbackGroup, opts ...BucketOption) (*Bucket, error)

func (*Bucket) Compact

func (b *Bucket) Compact()

func (*Bucket) Count

func (b *Bucket) Count() int

func (*Bucket) Cursor

func (b *Bucket) Cursor() *CursorReplace

Cursor holds a RLock for the flushing state. It needs to be closed using the .Close() methods or otherwise the lock will never be released

func (*Bucket) CursorRoaringSet

func (b *Bucket) CursorRoaringSet() CursorRoaringSet

func (*Bucket) CursorRoaringSetKeyOnly

func (b *Bucket) CursorRoaringSetKeyOnly() CursorRoaringSet

func (*Bucket) Delete

func (b *Bucket) Delete(key []byte, opts ...SecondaryKeyOption) error

Delete removes the given row. Note that LSM stores are append only, thus internally this action appends a tombstone. The entry will not be removed until a compaction has run, and even then a compaction does not guarantee the removal of the data right away. This is because an entry could have been created in an older segment than those present in the compaction. This can be seen as an implementation detail, unless the caller expects to free disk space by calling this method. Such freeing is not guaranteed.

Delete is specific to the Replace Strategy. For Maps, you can use Bucket.MapDeleteKey to delete a single key-value pair, for Sets use Bucket.SetDeleteSingle to delete a single set element.

func (*Bucket) DesiredStrategy

func (b *Bucket) DesiredStrategy() string

func (*Bucket) FlushAndSwitch

func (b *Bucket) FlushAndSwitch() error

FlushAndSwitch is typically called periodically and does not require manual calling, but there are some situations where this might be intended, such as in test scenarios or when a force flush is desired.

func (*Bucket) FlushMemtable

func (b *Bucket) FlushMemtable() error

FlushMemtable flushes any active memtable and returns only once the memtable has been fully flushed and a stable state on disk has been reached.

This is a preparatory stage for creating backups.

Method should be run only if flushCycle is not running (was not started, is stopped, or noop impl is provided)

func (*Bucket) Get

func (b *Bucket) Get(key []byte) ([]byte, error)

func (*Bucket) GetBySecondary

func (b *Bucket) GetBySecondary(pos int, key []byte) ([]byte, error)

func (*Bucket) GetBySecondaryIntoMemory

func (b *Bucket) GetBySecondaryIntoMemory(pos int, key []byte, buffer []byte) ([]byte, []byte, error)

func (*Bucket) GetDesiredStrategy

func (b *Bucket) GetDesiredStrategy() string

func (*Bucket) GetDir

func (b *Bucket) GetDir() string

func (*Bucket) GetFlushAfterIdle

func (b *Bucket) GetFlushAfterIdle() time.Duration

func (*Bucket) GetFlushCallbackCtrl

func (b *Bucket) GetFlushCallbackCtrl() cyclemanager.CycleCallbackCtrl

func (*Bucket) GetMemtableThreshold

func (b *Bucket) GetMemtableThreshold() uint64

func (*Bucket) GetRootDir

func (b *Bucket) GetRootDir() string

func (*Bucket) GetSecondaryIndices

func (b *Bucket) GetSecondaryIndices() uint16

func (*Bucket) GetStatus

func (b *Bucket) GetStatus() storagestate.Status

func (*Bucket) GetStrategy

func (b *Bucket) GetStrategy() string

func (*Bucket) GetWalThreshold

func (b *Bucket) GetWalThreshold() uint64

func (*Bucket) IterateMapObjects

func (b *Bucket) IterateMapObjects(ctx context.Context, f func([]byte, []byte, []byte, bool) error) error

func (*Bucket) IterateObjects

func (b *Bucket) IterateObjects(ctx context.Context, f func(object *storobj.Object) error) error

func (*Bucket) ListFiles

func (b *Bucket) ListFiles(ctx context.Context, basePath string) ([]string, error)

ListFiles lists all files that currently exist in the Bucket. The files are only in a stable state if the memtable is empty, and if compactions are paused. If one of those conditions is not given, it errors

func (*Bucket) MapCursor

func (b *Bucket) MapCursor(cfgs ...MapListOption) *CursorMap

func (*Bucket) MapCursorKeyOnly

func (b *Bucket) MapCursorKeyOnly(cfgs ...MapListOption) *CursorMap

func (*Bucket) MapDeleteKey

func (b *Bucket) MapDeleteKey(rowKey, mapKey []byte) error

MapDeleteKey removes one key-value pair from the given map row. Note that LSM stores are append only, thus internally this action appends a tombstone. The entry will not be removed until a compaction has run, and even then a compaction does not guarantee the removal of the data right away. This is because an entry could have been created in an older segment than those present in the compaction. This can be seen as an implementation detail, unless the caller expects to free disk space by calling this method. Such freeing is not guaranteed.

MapDeleteKey is specific to the Map Strategy. For Replace, you can use Bucket.Delete to delete the entire row, for Sets use Bucket.SetDeleteSingle to delete a single set element.

func (*Bucket) MapList

func (b *Bucket) MapList(key []byte, cfgs ...MapListOption) ([]MapPair, error)

MapList returns all map entries for a given row key. The order of map pairs has no specific meaning. For efficient merge operations, pair entries are stored sorted on disk, however that is an implementation detail and not a caller-facing guarantee.

MapList is specific to the Map strategy, for Sets use Bucket.SetList, for Replace use Bucket.Get.

func (*Bucket) MapSet

func (b *Bucket) MapSet(rowKey []byte, kv MapPair) error

MapSet writes one MapPair into the map for the given row key. It is agnostic of whether the row key already exists, as well as agnostic of whether the map key already exists. In both cases it will create the entry if it does not exist or override if it does.

Example to add a new MapPair:

pair := MapPair{Key: []byte("Jane"), Value: []byte("Backend")}
err := bucket.MapSet([]byte("developers"), pair)
if err != nil {
	/* do something */
}

MapSet is specific to the Map Strategy, for Replace use Bucket.Put, and for Set use Bucket.SetAdd instead.

func (*Bucket) MapSetMulti

func (b *Bucket) MapSetMulti(rowKey []byte, kvs []MapPair) error

MapSetMulti is the same as Bucket.MapSet, except that it takes in multiple MapPair objects at the same time.

func (*Bucket) Put

func (b *Bucket) Put(key, value []byte, opts ...SecondaryKeyOption) error

Put creates or replaces a single value for a given key.

err := bucket.Put([]byte("my_key"), []byte("my_value"))
 if err != nil {
	/* do something */
}

If a bucket has a secondary index configured, you can also specify one or more secondary keys, like so:

err := bucket.Put([]byte("my_key"), []byte("my_value"),
	WithSecondaryKey(0, []byte("my_alternative_key")),
)
 if err != nil {
	/* do something */
}

Put is limited to ReplaceStrategy, use Bucket.SetAdd for Set or Bucket.MapSet and Bucket.MapSetMulti.

func (*Bucket) RoaringSetAddBitmap

func (b *Bucket) RoaringSetAddBitmap(key []byte, bm *sroar.Bitmap) error

func (*Bucket) RoaringSetAddList

func (b *Bucket) RoaringSetAddList(key []byte, values []uint64) error

func (*Bucket) RoaringSetAddOne

func (b *Bucket) RoaringSetAddOne(key []byte, value uint64) error

func (*Bucket) RoaringSetGet

func (b *Bucket) RoaringSetGet(key []byte) (*sroar.Bitmap, error)

func (*Bucket) RoaringSetRemoveOne

func (b *Bucket) RoaringSetRemoveOne(key []byte, value uint64) error

func (*Bucket) SetAdd

func (b *Bucket) SetAdd(key []byte, values [][]byte) error

SetAdd adds one or more Set-Entries to a Set for the given key. SetAdd is entirely agnostic of existing entries, it acts as append-only. This also makes it agnostic of whether the key already exists or not.

Example to add two entries to a set:

err := bucket.SetAdd([]byte("my_key"), [][]byte{
	[]byte("one-set-element"), []byte("another-set-element"),
})
if err != nil {
	/* do something */
}

SetAdd is specific to the Set strategy. For Replace, use Bucket.Put, for Map use either Bucket.MapSet or Bucket.MapSetMulti.

func (*Bucket) SetCursor

func (b *Bucket) SetCursor() *CursorSet

SetCursor holds a RLock for the flushing state. It needs to be closed using the .Close() methods or otherwise the lock will never be released

func (*Bucket) SetCursorKeyOnly

func (b *Bucket) SetCursorKeyOnly() *CursorSet

SetCursorKeyOnly returns nil for all values. It has no control over the underlying "inner" cursors which may still retrieve a value which is then discarded. It does however, omit any handling of values, such as decoding, making this considerably more efficient if only keys are required.

The same locking rules as for SetCursor apply.

func (*Bucket) SetDeleteSingle

func (b *Bucket) SetDeleteSingle(key []byte, valueToDelete []byte) error

SetDeleteSingle removes one Set element from the given key. Note that LSM stores are append only, thus internally this action appends a tombstone. The entry will not be removed until a compaction has run, and even then a compaction does not guarantee the removal of the data right away. This is because an entry could have been created in an older segment than those present in the compaction. This can be seen as an implementation detail, unless the caller expects to free disk space by calling this method. Such freeing is not guaranteed.

SetDeleteSingle is specific to the Set Strategy. For Replace, you can use Bucket.Delete to delete the entire row, for Maps use Bucket.MapDeleteKey to delete a single map entry.

func (*Bucket) SetList

func (b *Bucket) SetList(key []byte) ([][]byte, error)

SetList returns all Set entries for a given key.

SetList is specific to the Set Strategy, for Map use Bucket.MapList, and for Replace use Bucket.Get.

func (*Bucket) SetMemtableThreshold

func (b *Bucket) SetMemtableThreshold(size uint64)

func (*Bucket) Shutdown

func (b *Bucket) Shutdown(ctx context.Context) error

func (*Bucket) Strategy

func (b *Bucket) Strategy() string

func (*Bucket) UpdateStatus

func (b *Bucket) UpdateStatus(status storagestate.Status)

UpdateStatus is used by the parent shard to communicate to the bucket when the shard has been set to readonly, or when it is ready for writes.

func (*Bucket) WasDeleted

func (b *Bucket) WasDeleted(key []byte) (bool, error)

WasDeleted determines if an object used to exist in the LSM store

There are 3 different locations that we need to check for the key in this order: active memtable, flushing memtable, and disk segment

func (*Bucket) WriteWAL

func (b *Bucket) WriteWAL() error

the WAL uses a buffer and isn't written until the buffer size is crossed or this function explicitly called. This allows to avoid unnecessary disk writes in larger operations, such as batches. It is sufficient to call write on the WAL just once. This does not make a batch atomic, but it guarantees that the WAL is written before a successful response is returned to the user.

type BucketOption

type BucketOption func(b *Bucket) error

func WithCalcCountNetAdditions

func WithCalcCountNetAdditions(calcCountNetAdditions bool) BucketOption

func WithDynamicMemtableSizing

func WithDynamicMemtableSizing(
	initialMB, maxMB, minActiveSeconds, maxActiveSeconds int,
) BucketOption

func WithForceCompation

func WithForceCompation(opt bool) BucketOption

Background for this option:

We use the LSM store in two places: Our existing key/value and inverted buckets As part of the new brute-force based index (to be built this week).

Brute-force index This is a simple disk-index where we use a cursor to iterate over all objects. This is what we need the force-compaction for. The experimentation so far has shown that the cursor is much more performant on a single segment than it is on multiple segments. This is because with a single segment it’s essentially just one conitiguuous chunk of data on disk that we read through. But with multiple segments (and an unpredicatable order) it ends up being many tiny reads (inefficient). Existing uses of the LSM store For existing uses, e.g. the object store, we don’t want to force-compact. This is because they can grow massive. For example, you could have a 100GB segment, then a new write leads to a new segment that is just a few bytes. If we would force-compact those two we would write 100GB every time the user sends a few bytes to Weaviate. In this case, the existing tiered compaction strategy makes more sense. Configurability of buckets

func WithIdleThreshold

func WithIdleThreshold(threshold time.Duration) BucketOption

func WithKeepTombstones

func WithKeepTombstones(keepTombstones bool) BucketOption

func WithLegacyMapSorting

func WithLegacyMapSorting() BucketOption

func WithMemtableThreshold

func WithMemtableThreshold(threshold uint64) BucketOption

func WithMonitorCount

func WithMonitorCount() BucketOption

func WithPread

func WithPread(with bool) BucketOption

func WithSecondaryIndices

func WithSecondaryIndices(count uint16) BucketOption

func WithStrategy

func WithStrategy(strategy string) BucketOption

func WithUseBloomFilter

func WithUseBloomFilter(useBloomFilter bool) BucketOption

func WithWalThreshold

func WithWalThreshold(threshold uint64) BucketOption

type CommitType

type CommitType uint16
const (
	CommitTypeReplace CommitType = iota // replace strategy

	// collection strategy - this can handle all cases as updates and deletes are
	// only appends in a collection strategy
	CommitTypeCollection
	CommitTypeRoaringSet
)

func (CommitType) Is

func (ct CommitType) Is(checkedCommitType CommitType) bool

func (CommitType) String

func (ct CommitType) String() string

type CursorMap

type CursorMap struct {
	// contains filtered or unexported fields
}

func (*CursorMap) Close

func (c *CursorMap) Close()

func (*CursorMap) First

func (c *CursorMap) First() ([]byte, []MapPair)

func (*CursorMap) Next

func (c *CursorMap) Next() ([]byte, []MapPair)

func (*CursorMap) Seek

func (c *CursorMap) Seek(key []byte) ([]byte, []MapPair)

type CursorReplace

type CursorReplace struct {
	// contains filtered or unexported fields
}

func (*CursorReplace) Close

func (c *CursorReplace) Close()

func (*CursorReplace) First

func (c *CursorReplace) First() ([]byte, []byte)

func (*CursorReplace) Next

func (c *CursorReplace) Next() ([]byte, []byte)

func (*CursorReplace) Seek

func (c *CursorReplace) Seek(key []byte) ([]byte, []byte)

type CursorRoaringSet

type CursorRoaringSet interface {
	First() ([]byte, *sroar.Bitmap)
	Next() ([]byte, *sroar.Bitmap)
	Seek([]byte) ([]byte, *sroar.Bitmap)
	Close()
}

type CursorSet

type CursorSet struct {
	// contains filtered or unexported fields
}

func (*CursorSet) Close

func (c *CursorSet) Close()

func (*CursorSet) First

func (c *CursorSet) First() ([]byte, [][]byte)

func (*CursorSet) Next

func (c *CursorSet) Next() ([]byte, [][]byte)

func (*CursorSet) Seek

func (c *CursorSet) Seek(key []byte) ([]byte, [][]byte)

type MapListOption

type MapListOption func(c *MapListOptionConfig)

func MapListAcceptDuplicates

func MapListAcceptDuplicates() MapListOption

func MapListLegacySortingRequired

func MapListLegacySortingRequired() MapListOption

type MapListOptionConfig

type MapListOptionConfig struct {
	// contains filtered or unexported fields
}

type MapPair

type MapPair struct {
	Key       []byte
	Value     []byte
	Tombstone bool
}

func (MapPair) Bytes

func (kv MapPair) Bytes() ([]byte, error)

func (MapPair) EncodeBytes

func (kv MapPair) EncodeBytes(buf []byte) error

func (*MapPair) FromBytes

func (kv *MapPair) FromBytes(in []byte, keyOnly bool) error

func (*MapPair) FromBytesReusable

func (kv *MapPair) FromBytesReusable(in []byte, keyOnly bool) error

func (MapPair) Size

func (kv MapPair) Size() int

Size() returns the exact size in bytes that will be used when Bytes() is called

type Memtable

type Memtable struct {
	sync.RWMutex
	// contains filtered or unexported fields
}

func (*Memtable) ActiveDuration

func (m *Memtable) ActiveDuration() time.Duration

func (*Memtable) IdleDuration

func (m *Memtable) IdleDuration() time.Duration

func (*Memtable) Size

func (m *Memtable) Size() uint64

type SecondaryKeyOption

type SecondaryKeyOption func(s secondaryIndexKeys) error

func WithSecondaryKey

func WithSecondaryKey(pos int, key []byte) SecondaryKeyOption

type SegmentGroup

type SegmentGroup struct {
	// contains filtered or unexported fields
}

func (*SegmentGroup) Compact

func (sg *SegmentGroup) Compact()

func (*SegmentGroup) Len

func (sg *SegmentGroup) Len() int

type Store

type Store struct {
	// contains filtered or unexported fields
}

Store groups multiple buckets together, it "owns" one folder on the file system

func New

func New(dir, rootDir string, logger logrus.FieldLogger,
	shardCompactionCallbacks, shardFlushCallbacks cyclemanager.CycleCallbackGroup,
) (*Store, error)

New initializes a new Store based on the root dir. If state is present on disk, it is loaded, if the folder is empty a new store is initialized in there.

func (*Store) Bucket

func (s *Store) Bucket(name string) *Bucket

func (*Store) CreateBucket

func (s *Store) CreateBucket(ctx context.Context, bucketName string,
	opts ...BucketOption,
) error

Creates bucket, first removing any files if already exist Bucket can not be registered in bucketsByName before removal

func (*Store) CreateOrLoadBucket

func (s *Store) CreateOrLoadBucket(ctx context.Context, bucketName string,
	opts ...BucketOption,
) error

CreateOrLoadBucket registers a bucket with the given name. If state on disk exists for this bucket it is loaded, otherwise created. Pass [BucketOptions] to configure the strategy of a bucket. The strategy defaults to "replace". For example, to load or create a map-type bucket, do:

ctx := context.Background()
err := store.CreateOrLoadBucket(ctx, "my_bucket_name", WithStrategy(StrategyReplace))
if err != nil { /* handle error */ }

// you can now access the bucket using store.Bucket()
b := store.Bucket("my_bucket_name")

func (*Store) FlushMemtables

func (s *Store) FlushMemtables(ctx context.Context) error

FlushMemtable flushes any active memtable and returns only once the memtable has been fully flushed and a stable state on disk has been reached.

This is a preparatory stage for creating backups.

A timeout should be specified for the input context as some flushes are long-running, in which case it may be better to fail the backup attempt and retry later, than to block indefinitely.

func (*Store) GetBucketsByName

func (s *Store) GetBucketsByName() map[string]*Bucket

func (*Store) ListFiles

func (s *Store) ListFiles(ctx context.Context, basePath string) ([]string, error)

func (*Store) PauseCompaction

func (s *Store) PauseCompaction(ctx context.Context) error

PauseCompaction waits for all ongoing compactions to finish, then makes sure that no new compaction can be started.

This is a preparatory stage for creating backups.

A timeout should be specified for the input context as some compactions are long-running, in which case it may be better to fail the backup attempt and retry later, than to block indefinitely.

func (*Store) RenameBucket

func (s *Store) RenameBucket(ctx context.Context, bucketName, newBucketName string) error

func (*Store) ReplaceBuckets

func (s *Store) ReplaceBuckets(ctx context.Context, bucketName, replacementBucketName string) error

Replaces 1st bucket with 2nd one. Both buckets have to registered in bucketsByName. 2nd bucket swaps the 1st one in bucketsByName using 1st one's name, 2nd one's name is deleted. Dir path of 2nd bucket is changed to dir of 1st bucket as well as all other related paths of bucket resources (segment group, memtables, commit log). Dir path of 1st bucket is temporarily suffixed with "___del", later on bucket is shutdown and its files deleted. 2nd bucket becomes 1st bucket

func (*Store) ResumeCompaction

func (s *Store) ResumeCompaction(ctx context.Context) error

ResumeCompaction starts the compaction cycle again. It errors if compactions were not paused

func (*Store) Shutdown

func (s *Store) Shutdown(ctx context.Context) error

func (*Store) UpdateBucketsStatus

func (s *Store) UpdateBucketsStatus(targetStatus storagestate.Status)

func (*Store) WriteWALs

func (s *Store) WriteWALs() error

Directories

Path Synopsis
ent contains common types used throughout various lsmkv (sub-)packages
ent contains common types used throughout various lsmkv (sub-)packages
The "roaringset" package contains all the LSM business logic that is unique to the "RoaringSet" strategy
The "roaringset" package contains all the LSM business logic that is unique to the "RoaringSet" strategy

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL