engine

package
v0.0.0-...-1e9670a Latest Latest
Warning

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

Go to latest
Published: Apr 17, 2016 License: Apache-2.0 Imports: 25 Imported by: 0

Documentation

Overview

Package engine provides low-level storage. It interacts with storage backends (e.g. LevelDB, RocksDB, etc.) via the Engine interface. At one level higher, MVCC provides multi-version concurrency control capability on top of an Engine instance.

The Engine interface provides an API for key-value stores. InMem implements an in-memory engine using a sorted map. RocksDB implements an engine for data stored to local disk using RocksDB, a variant of LevelDB.

MVCC provides a multi-version concurrency control system on top of an engine. MVCC is the basis for Cockroach's support for distributed transactions. It is intended for direct use from storage.Range objects.

Notes on MVCC architecture

Each MVCC value contains a metadata key/value pair and one or more version key/value pairs. The MVCC metadata key is the actual key for the value, using the util/encoding.EncodeBytes scheme. The MVCC metadata value is of type MVCCMetadata and contains the most recent version timestamp and an optional roachpb.Transaction message. If set, the most recent version of the MVCC value is a transactional "intent". It also contains some information on the size of the most recent version's key and value for efficient stat counter computations. Notice that it is not necessary to explicitly store the MVCC metadata as its contents can be reconstructed from the most recent versioned value as long as an intent is not present. The implementation takes advantage of this and deletes the MVCC metadata when possible.

Each MVCC version key/value pair has a key which is also binary-encoded, but is suffixed with a decreasing, big-endian encoding of the timestamp (8 bytes for the nanosecond wall time, followed by 4 bytes for the logical time). The MVCC version value is a message of type roachpb.Value. A deletion is is indicated by an empty value. Note that an empty roachpb.Value will encode to a non-empty byte slice. The decreasing encoding on the timestamp sorts the most recent version directly after the metadata key. This increases the likelihood that an Engine.Get() of the MVCC metadata will get the same block containing the most recent version, even if there are many versions. We rely on getting the MVCC metadata key/value and then using it to directly get the MVCC version using the metadata's most recent version timestamp. This avoids using an expensive merge iterator to scan the most recent version. It also allows us to leverage RocksDB's bloom filters.

The binary encoding used on the MVCC keys allows arbitrary keys to be stored in the map (no restrictions on intermediate nil-bytes, for example), while still sorting lexicographically and guaranteeing that all timestamp-suffixed MVCC version keys sort consecutively with the metadata key. We use an escape-based encoding which transforms all nul ("\x00") characters in the key and is terminated with the sequence "\x00\x01", which is guaranteed to not occur elsewhere in the encoded value. See util/encoding/encoding.go for more details.

We considered inlining the most recent MVCC version in the MVCCMetadata. This would reduce the storage overhead of storing the same key twice (which is small due to block compression), and the runtime overhead of two separate DB lookups. On the other hand, all writes that create a new version of an existing key would incur a double write as the previous value is moved out of the MVCCMetadata into its versioned key. Preliminary benchmarks have not shown enough performance improvement to justify this change, although we may revisit this decision if it turns out that multiple versions of the same key are rare in practice.

However, we do allow inlining in order to use the MVCC interface to store non-versioned values. It turns out that not everything which Cockroach needs to store would be efficient or possible using MVCC. Examples include transaction records, response cache entries, stats counters, time series data, and system-local config values. However, supporting a mix of encodings is problematic in terms of resulting complexity. So Cockroach treats an MVCC timestamp of zero to mean an inlined, non-versioned value. These values are replaced if they exist on a Put operation and are cleared from the engine on a delete. Importantly, zero-timestamped MVCC values may be merged, as is necessary for stats counters and time series data.

Package engine is a generated protocol buffer package.

It is generated from these files:
	cockroach/storage/engine/mvcc.proto

It has these top-level messages:
	MVCCMetadata
	MVCCStats

Index

Constants

This section is empty.

Variables

View Source
var (
	// MVCCKeyMax is a maximum mvcc-encoded key value which sorts after
	// all other keys.
	MVCCKeyMax = MakeMVCCMetadataKey(roachpb.KeyMax)
	// NilKey is the nil MVCCKey.
	NilKey = MVCCKey{}
)
View Source
var (
	ErrInvalidLengthMvcc = fmt.Errorf("proto: negative length found during unmarshaling")
	ErrIntOverflowMvcc   = fmt.Errorf("proto: integer overflow")
)

Functions

func ClearRange

func ClearRange(engine Engine, start, end MVCCKey) (int, error)

ClearRange removes a set of entries, from start (inclusive) to end (exclusive). This function returns the number of entries removed. Either all entries within the range will be deleted, or none, and an error will be returned. Note that this function actually removes entries from the storage engine, rather than inserting tombstones, as with deletion through the MVCC.

func IsValidSplitKey

func IsValidSplitKey(key roachpb.Key) bool

IsValidSplitKey returns whether the key is a valid split key. Certain key ranges cannot be split (the meta1 span and the system DB span); split keys chosen within any of these ranges are considered invalid. And a split key equal to Meta2KeyMax (\x03\xff\xff) is considered invalid.

func MVCCConditionalPut

func MVCCConditionalPut(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	value roachpb.Value,
	expVal *roachpb.Value,
	txn *roachpb.Transaction,
) error

MVCCConditionalPut sets the value for a specified key only if the expected value matches. If not, the return a ConditionFailedError containing the actual value.

The condition check reads a value from the key using the same operational timestamp as we use to write a value.

func MVCCDelete

func MVCCDelete(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	txn *roachpb.Transaction,
) error

MVCCDelete marks the key deleted so that it will not be returned in future get responses.

func MVCCDeleteRange

func MVCCDeleteRange(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key,
	endKey roachpb.Key,
	max int64,
	timestamp roachpb.Timestamp,
	txn *roachpb.Transaction,
	returnKeys bool,
) ([]roachpb.Key, error)

MVCCDeleteRange deletes the range of key/value pairs specified by start and end keys. Specify max=0 for unbounded deletes.

func MVCCFindSplitKey

func MVCCFindSplitKey(
	ctx context.Context,
	engine Engine,
	rangeID roachpb.RangeID,
	key,
	endKey roachpb.RKey,
	debugFn func(msg string, args ...interface{}),
) (roachpb.Key, error)

MVCCFindSplitKey suggests a split key from the given user-space key range that aims to roughly cut into half the total number of bytes used (in raw key and value byte strings) in both subranges. Specify a snapshot engine to safely invoke this method in a goroutine.

The split key will never be chosen from the key ranges listed in illegalSplitKeySpans.

debugFn, if not nil, is used to print informational log messages about the key finding process.

func MVCCGarbageCollect

func MVCCGarbageCollect(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	keys []roachpb.GCRequest_GCKey,
	timestamp roachpb.Timestamp,
) error

MVCCGarbageCollect creates an iterator on the engine. In parallel it iterates through the keys listed for garbage collection by the keys slice. The engine iterator is seeked in turn to each listed key, clearing all values with timestamps <= to expiration. The timestamp parameter is used to compute the intent age on GC.

func MVCCGet

func MVCCGet(
	ctx context.Context,
	engine Engine,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	consistent bool,
	txn *roachpb.Transaction,
) (*roachpb.Value, []roachpb.Intent, error)

MVCCGet returns the value for the key specified in the request, while satisfying the given timestamp condition. The key may contain arbitrary bytes. If no value for the key exists, or it has been deleted, returns nil for value.

The values of multiple versions for the given key should be organized as follows: ... keyA : MVCCMetadata of keyA keyA_Timestamp_n : value of version_n keyA_Timestamp_n-1 : value of version_n-1 ... keyA_Timestamp_0 : value of version_0 keyB : MVCCMetadata of keyB ...

The consistent parameter indicates that intents should cause WriteIntentErrors. If set to false, a possible intent on the key will be ignored for reading the value (but returned via the roachpb.Intent slice); the previous value (if any) is read instead.

func MVCCGetAsTxn

func MVCCGetAsTxn(
	ctx context.Context,
	engine Engine,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	consistent bool,
	txnMeta roachpb.TxnMeta,
) (*roachpb.Value, []roachpb.Intent, error)

MVCCGetAsTxn constructs a temporary Transaction from the given txn metadata and calls MVCCGet as that transaction. This method is required only for reading intents of a transaction when only its metadata is known and should rarely be used. The read is carried out without the chance of uncertainty restarts.

func MVCCGetProto

func MVCCGetProto(
	ctx context.Context,
	engine Engine,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	consistent bool,
	txn *roachpb.Transaction,
	msg proto.Message,
) (bool, error)

MVCCGetProto fetches the value at the specified key and unmarshals it using a protobuf decoder. Returns true on success or false if the key was not found. In the event of a WriteIntentError when consistent=false, we return the error and the decoded result; for all other errors (or when consistent=true) the decoded value is invalid.

func MVCCGetRangeStats

func MVCCGetRangeStats(
	ctx context.Context,
	engine Engine,
	rangeID roachpb.RangeID,
	ms *MVCCStats,
) error

MVCCGetRangeStats reads stat counters for the specified range and sets the values in the supplied MVCCStats struct.

func MVCCIncrement

func MVCCIncrement(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	txn *roachpb.Transaction,
	inc int64,
) (int64, error)

MVCCIncrement fetches the value for key, and assuming the value is an "integer" type, increments it by inc and stores the new value. The newly incremented value is returned.

An initial value is read from the key using the same operational timestamp as we use to write a value.

func MVCCIterate

func MVCCIterate(ctx context.Context,
	engine Engine,
	startKey,
	endKey roachpb.Key,
	timestamp roachpb.Timestamp,
	consistent bool,
	txn *roachpb.Transaction,
	reverse bool,
	f func(roachpb.KeyValue) (bool, error),
) ([]roachpb.Intent, error)

MVCCIterate iterates over the key range [start,end). At each step of the iteration, f() is invoked with the current key/value pair. If f returns true (done) or an error, the iteration stops and the error is propagated. If the reverse is flag set the iterator will be moved in reverse order.

func MVCCMerge

func MVCCMerge(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	value roachpb.Value,
) error

MVCCMerge implements a merge operation. Merge adds integer values, concatenates undifferentiated byte slice values, and efficiently combines time series observations if the roachpb.Value tag value indicates the value byte slice is of type TIMESERIES.

func MVCCPut

func MVCCPut(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	value roachpb.Value,
	txn *roachpb.Transaction,
) error

MVCCPut sets the value for a specified key. It will save the value with different versions according to its timestamp and update the key metadata. The timestamp must be passed as a parameter; using the Timestamp field on the value results in an error.

If the timestamp is specified as roachpb.ZeroTimestamp, the value is inlined instead of being written as a timestamp-versioned value. A zero timestamp write to a key precludes a subsequent write using a non-zero timestamp and vice versa. Inlined values require only a single row and never accumulate more than a single value. Successive zero timestamp writes to a key replace the value and deletes clear the value. In addition, zero timestamp values may be merged.

func MVCCPutProto

func MVCCPutProto(
	ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	key roachpb.Key,
	timestamp roachpb.Timestamp,
	txn *roachpb.Transaction,
	msg proto.Message,
) error

MVCCPutProto sets the given key to the protobuf-serialized byte string of msg and the provided timestamp.

func MVCCResolveWriteIntent

func MVCCResolveWriteIntent(ctx context.Context,
	engine Engine,
	ms *MVCCStats,
	intent roachpb.Intent,
) error

MVCCResolveWriteIntent either commits or aborts (rolls back) an extant write intent for a given txn according to commit parameter. ResolveWriteIntent will skip write intents of other txns.

Transaction epochs deserve a bit of explanation. The epoch for a transaction is incremented on transaction retry. Transaction retry is different from abort. Retries occur in SSI transactions when the commit timestamp is not equal to the proposed transaction timestamp. This might be because writes to different keys had to use higher timestamps than expected because of existing, committed value, or because reads pushed the transaction's commit timestamp forward. Retries also occur in the event that the txn tries to push another txn in order to write an intent but fails (i.e. it has lower priority).

Because successive retries of a transaction may end up writing to different keys, the epochs serve to classify which intents get committed in the event the transaction succeeds (all those with epoch matching the commit epoch), and which intents get aborted, even if the transaction succeeds.

TODO(tschottdorf): encountered a bug in which a Txn committed with its original timestamp after laying down intents at higher timestamps. Doesn't look like this code here caught that. Shouldn't resolve intents when they're not at the timestamp the Txn mandates them to be.

func MVCCResolveWriteIntentRange

func MVCCResolveWriteIntentRange(
	ctx context.Context, engine Engine, ms *MVCCStats, intent roachpb.Intent, max int64,
) (int64, error)

MVCCResolveWriteIntentRange commits or aborts (rolls back) the range of write intents specified by start and end keys for a given txn. ResolveWriteIntentRange will skip write intents of other txns. Specify max=0 for unbounded resolves.

func MVCCResolveWriteIntentRangeUsingIter

func MVCCResolveWriteIntentRangeUsingIter(
	ctx context.Context,
	engine Engine,
	iterAndBuf IterAndBuf,
	ms *MVCCStats,
	intent roachpb.Intent,
	max int64,
) (int64, error)

MVCCResolveWriteIntentRangeUsingIter commits or aborts (rolls back) the range of write intents specified by start and end keys for a given txn. ResolveWriteIntentRange will skip write intents of other txns. Specify max=0 for unbounded resolves.

func MVCCResolveWriteIntentUsingIter

func MVCCResolveWriteIntentUsingIter(
	ctx context.Context,
	engine Engine,
	iterAndBuf IterAndBuf,
	ms *MVCCStats,
	intent roachpb.Intent,
) error

MVCCResolveWriteIntentUsingIter is a variant of MVCCResolveWriteIntent that uses iterator and buffer passed as parameters (e.g. when used in a loop).

func MVCCReverseScan

func MVCCReverseScan(
	ctx context.Context,
	engine Engine,
	key,
	endKey roachpb.Key,
	max int64,
	timestamp roachpb.Timestamp,
	consistent bool,
	txn *roachpb.Transaction,
) ([]roachpb.KeyValue, []roachpb.Intent, error)

MVCCReverseScan scans the key range [start,end) key up to some maximum number of results in descending order. Specify max=0 for unbounded scans.

func MVCCScan

func MVCCScan(
	ctx context.Context,
	engine Engine,
	key,
	endKey roachpb.Key,
	max int64,
	timestamp roachpb.Timestamp,
	consistent bool,
	txn *roachpb.Transaction,
) ([]roachpb.KeyValue, []roachpb.Intent, error)

MVCCScan scans the key range [start,end) key up to some maximum number of results in ascending order. Specify max=0 for unbounded scans.

func MVCCSetRangeStats

func MVCCSetRangeStats(ctx context.Context,
	engine Engine,
	rangeID roachpb.RangeID,
	ms *MVCCStats,
) error

MVCCSetRangeStats sets stat counters for specified range.

func MergeInternalTimeSeriesData

func MergeInternalTimeSeriesData(
	sources ...roachpb.InternalTimeSeriesData,
) (roachpb.InternalTimeSeriesData, error)

MergeInternalTimeSeriesData exports the engine's C++ merge logic for InternalTimeSeriesData to higher level packages. This is intended primarily for consumption by high level testing of time series functionality.

func PutProto

func PutProto(engine Engine, key MVCCKey, msg proto.Message) (keyBytes, valBytes int64, err error)

PutProto sets the given key to the protobuf-serialized byte string of msg and the provided timestamp. Returns the length in bytes of key and the value.

Types

type Engine

type Engine interface {
	// Open initializes the engine.
	Open() error
	// Close closes the engine, freeing up any outstanding resources.
	Close()
	// Attrs returns the engine/store attributes.
	Attrs() roachpb.Attributes
	// Put sets the given key to the value provided.
	Put(key MVCCKey, value []byte) error
	// Get returns the value for the given key, nil otherwise.
	Get(key MVCCKey) ([]byte, error)
	// GetProto fetches the value at the specified key and unmarshals it
	// using a protobuf decoder. Returns true on success or false if the
	// key was not found. On success, returns the length in bytes of the
	// key and the value.
	GetProto(key MVCCKey, msg proto.Message) (ok bool, keyBytes, valBytes int64, err error)
	// Iterate scans from start to end keys, visiting at most max
	// key/value pairs. On each key value pair, the function f is
	// invoked. If f returns an error or if the scan itself encounters
	// an error, the iteration will stop and return the error.
	// If the first result of f is true, the iteration stops.
	Iterate(start, end MVCCKey, f func(MVCCKeyValue) (bool, error)) error
	// Clear removes the item from the db with the given key.
	// Note that clear actually removes entries from the storage
	// engine, rather than inserting tombstones.
	Clear(key MVCCKey) error
	// Merge is a high-performance write operation used for values which are
	// accumulated over several writes. Multiple values can be merged
	// sequentially into a single key; a subsequent read will return a "merged"
	// value which is computed from the original merged values.
	//
	// Merge currently provides specialized behavior for three data types:
	// integers, byte slices, and time series observations. Merged integers are
	// summed, acting as a high-performance accumulator.  Byte slices are simply
	// concatenated in the order they are merged. Time series observations
	// (stored as byte slices with a special tag on the roachpb.Value) are
	// combined with specialized logic beyond that of simple byte slices.
	//
	// The logic for merges is written in db.cc in order to be compatible with RocksDB.
	Merge(key MVCCKey, value []byte) error
	// Capacity returns capacity details for the engine's available storage.
	Capacity() (roachpb.StoreCapacity, error)
	// ApproximateSize returns the approximate number of bytes the engine is
	// using to store data for the given range of keys.
	ApproximateSize(start, end MVCCKey) (uint64, error)
	// Flush causes the engine to write all in-memory data to disk
	// immediately.
	Flush() error
	// NewIterator returns a new instance of an Iterator over this engine. When
	// prefix is non-nil, Seek will use the user-key prefix of the supplied MVCC
	// key to restrict which sstables are searched, but iteration (using Next)
	// over keys without the same user-key prefix will not work correctly (keys
	// may be skipped). The caller must invoke Iterator.Close() when finished
	// with the iterator to free resources.
	NewIterator(prefix roachpb.Key) Iterator
	// NewSnapshot returns a new instance of a read-only snapshot
	// engine. Snapshots are instantaneous and, as long as they're
	// released relatively quickly, inexpensive. Snapshots are released
	// by invoking Close(). Note that snapshots must not be used after the
	// original engine has been stopped.
	NewSnapshot() Engine
	// NewBatch returns a new instance of a batched engine which wraps
	// this engine. Batched engines accumulate all mutations and apply
	// them atomically on a call to Commit().
	NewBatch() Engine
	// Commit atomically applies any batched updates to the underlying
	// engine. This is a noop unless the engine was created via NewBatch().
	Commit() error
	// Defer adds a callback to be run after the batch commits
	// successfully.  If Commit() fails (or if this engine was not
	// created via NewBatch()), deferred callbacks are not called. As
	// with the defer statement, the last callback to be deferred is the
	// first to be executed.
	Defer(fn func())
	// Closed returns true if the engine has been close or not usable.
	// Objects backed by this engine (e.g. Iterators) can check this to ensure
	// that they are not using an closed engine.
	Closed() bool
	// GetStats retrieves stats from the engine.
	GetStats() (*Stats, error)
}

Engine is the interface that wraps the core operations of a key/value store.

func NewRocksDB

func NewRocksDB(attrs roachpb.Attributes, dir string, cacheSize, memtableBudget, maxSize int64,
	stopper *stop.Stopper) Engine

NewRocksDB allocates and returns a new RocksDB object.

type GarbageCollector

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

GarbageCollector GCs MVCC key/values using a zone-specific GC policy allows either the union or intersection of maximum # of versions and maximum age.

func MakeGarbageCollector

func MakeGarbageCollector(now roachpb.Timestamp, policy config.GCPolicy) GarbageCollector

MakeGarbageCollector allocates and returns a new GC, with expiration computed based on current time and policy.TTLSeconds.

func (GarbageCollector) Filter

func (gc GarbageCollector) Filter(keys []MVCCKey, values [][]byte) roachpb.Timestamp

Filter makes decisions about garbage collection based on the garbage collection policy for batches of values for the same key. Returns the timestamp including, and after which, all values should be garbage collected. If no values should be GC'd, returns roachpb.ZeroTimestamp.

type InMem

type InMem struct {
	*RocksDB
}

InMem wraps RocksDB and configures it for in-memory only storage.

func NewInMem

func NewInMem(attrs roachpb.Attributes, cacheSize int64, stopper *stop.Stopper) InMem

NewInMem allocates and returns a new, opened InMem engine.

type IterAndBuf

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

IterAndBuf used to pass iterators and buffers between MVCC* calls, allowing reuse without the callers needing to know the particulars.

func GetIterAndBuf

func GetIterAndBuf(engine Engine) IterAndBuf

GetIterAndBuf returns a IterAndBuf for passing into various MVCC* methods.

func (IterAndBuf) Cleanup

func (b IterAndBuf) Cleanup()

Cleanup must be called to release the resources when done.

type Iterator

type Iterator interface {
	// Close frees up resources held by the iterator.
	Close()
	// Seek advances the iterator to the first key in the engine which
	// is >= the provided key.
	Seek(key MVCCKey)
	// SeekReverse advances the iterator to the first key in the engine which
	// is <= the provided key.
	SeekReverse(key MVCCKey)
	// Valid returns true if the iterator is currently valid. An
	// iterator which hasn't been seeked or has gone past the end of the
	// key range is invalid.
	Valid() bool
	// Next advances the iterator to the next key/value in the
	// iteration. After this call, Valid() will be true if the
	// iterator was not positioned at the last key.
	Next()
	// Prev moves the iterator backward to the previous key/value
	// in the iteration. After this call, Valid() will be true if the
	// iterator was not positioned at the first key.
	Prev()
	// Key returns the current key as a byte slice.
	Key() MVCCKey
	// Value returns the current value as a byte slice.
	Value() []byte
	// ValueProto unmarshals the value the iterator is currently
	// pointing to using a protobuf decoder.
	ValueProto(msg proto.Message) error

	// Error returns the error, if any, which the iterator encountered.
	Error() error
	// ComputeStats scans the underlying engine from start to end keys and
	// computes stats counters based on the values. This method is used after a
	// range is split to recompute stats for each subrange. The start key is
	// always adjusted to avoid counting local keys in the event stats are being
	// recomputed for the first range (i.e. the one with start key == KeyMin).
	// The nowNanos arg specifies the wall time in nanoseconds since the
	// epoch and is used to compute the total age of all intents.
	ComputeStats(start, end MVCCKey, nowNanos int64) (MVCCStats, error)
	// contains filtered or unexported methods
}

Iterator is an interface for iterating over key/value pairs in an engine. Iterator implementations are thread safe unless otherwise noted.

type MVCCKey

type MVCCKey struct {
	Key       roachpb.Key
	Timestamp roachpb.Timestamp
}

MVCCKey is a versioned key, distinguished from roachpb.Key with the addition of a timestamp.

func MakeMVCCMetadataKey

func MakeMVCCMetadataKey(key roachpb.Key) MVCCKey

MakeMVCCMetadataKey creates an MVCCKey from a roachpb.Key.

func (MVCCKey) EncodedSize

func (k MVCCKey) EncodedSize() int

EncodedSize returns the size of the MVCCKey when encoded.

func (MVCCKey) Equal

func (k MVCCKey) Equal(l MVCCKey) bool

Equal returns whether two keys are identical.

func (MVCCKey) IsValue

func (k MVCCKey) IsValue() bool

IsValue returns true iff the timestamp is non-zero.

func (MVCCKey) Less

func (k MVCCKey) Less(l MVCCKey) bool

Less compares two keys.

func (MVCCKey) Next

func (k MVCCKey) Next() MVCCKey

Next returns the next key.

func (MVCCKey) String

func (k MVCCKey) String() string

String returns a string-formatted version of the key.

type MVCCKeyValue

type MVCCKeyValue struct {
	Key   MVCCKey
	Value []byte
}

MVCCKeyValue contains the raw bytes of the value for a key.

func Scan

func Scan(engine Engine, start, end MVCCKey, max int64) ([]MVCCKeyValue, error)

Scan returns up to max key/value objects starting from start (inclusive) and ending at end (non-inclusive). Specify max=0 for unbounded scans.

type MVCCMetadata

type MVCCMetadata struct {
	Txn *cockroach_roachpb1.TxnMeta `protobuf:"bytes,1,opt,name=txn" json:"txn,omitempty"`
	// The timestamp of the most recent versioned value if this is a
	// value that may have multiple versions. For values which may have
	// only one version, the data is stored inline (via raw_bytes), and
	// timestamp is set to zero.
	Timestamp cockroach_roachpb1.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"`
	// Is the most recent value a deletion tombstone?
	Deleted bool `protobuf:"varint,3,opt,name=deleted" json:"deleted"`
	// The size in bytes of the most recent encoded key.
	KeyBytes int64 `protobuf:"varint,4,opt,name=key_bytes,json=keyBytes" json:"key_bytes"`
	// The size in bytes of the most recent versioned value.
	ValBytes int64 `protobuf:"varint,5,opt,name=val_bytes,json=valBytes" json:"val_bytes"`
	// Inline value, used for non-versioned values with zero
	// timestamp. This provides an efficient short circuit of the normal
	// MVCC metadata sentinel and subsequent version rows. If timestamp
	// == (0, 0), then there is only a single MVCC metadata row with
	// value inlined, and with empty timestamp, key_bytes, and
	// val_bytes.
	RawBytes []byte `protobuf:"bytes,6,opt,name=raw_bytes,json=rawBytes" json:"raw_bytes,omitempty"`
	// This provides a measure of protection against replays caused by
	// Raft duplicating merge commands.
	MergeTimestamp *cockroach_roachpb1.Timestamp `protobuf:"bytes,7,opt,name=merge_timestamp,json=mergeTimestamp" json:"merge_timestamp,omitempty"`
}

MVCCMetadata holds MVCC metadata for a key. Used by storage/engine/mvcc.go.

func NewPopulatedMVCCMetadata

func NewPopulatedMVCCMetadata(r randyMvcc, easy bool) *MVCCMetadata

func (*MVCCMetadata) Descriptor

func (*MVCCMetadata) Descriptor() ([]byte, []int)

func (MVCCMetadata) IsInline

func (meta MVCCMetadata) IsInline() bool

IsInline returns true if the value is inlined in the metadata.

func (MVCCMetadata) IsIntentOf

func (meta MVCCMetadata) IsIntentOf(txn *roachpb.Transaction) bool

IsIntentOf returns true if the meta record is an intent of the supplied transaction.

func (*MVCCMetadata) Marshal

func (m *MVCCMetadata) Marshal() (data []byte, err error)

func (*MVCCMetadata) MarshalTo

func (m *MVCCMetadata) MarshalTo(data []byte) (int, error)

func (*MVCCMetadata) ProtoMessage

func (*MVCCMetadata) ProtoMessage()

func (*MVCCMetadata) Reset

func (m *MVCCMetadata) Reset()

func (*MVCCMetadata) Size

func (m *MVCCMetadata) Size() (n int)

func (*MVCCMetadata) String

func (m *MVCCMetadata) String() string

func (*MVCCMetadata) Unmarshal

func (m *MVCCMetadata) Unmarshal(data []byte) error

func (MVCCMetadata) Value

func (meta MVCCMetadata) Value() roachpb.Value

Value returns the inline value.

type MVCCStats

type MVCCStats struct {
	// last_update_nanos is a timestamp at which the ages were last
	// updated. See the comment on MVCCStats.
	LastUpdateNanos int64 `protobuf:"fixed64,1,opt,name=last_update_nanos,json=lastUpdateNanos" json:"last_update_nanos"`
	// intent_age is the cumulative age of the tracked intents.
	// See the comment on MVCCStats.
	IntentAge int64 `protobuf:"fixed64,2,opt,name=intent_age,json=intentAge" json:"intent_age"`
	// gc_bytes_age is the cumulative age of the non-live data (i.e.
	// data included in key_bytes and val_bytes, but not live_bytes).
	// See the comment on MVCCStats.
	GCBytesAge int64 `protobuf:"fixed64,3,opt,name=gc_bytes_age,json=gcBytesAge" json:"gc_bytes_age"`
	// live_bytes is the number of bytes stored in keys and values which can in
	// principle be read by means of a Scan or Get, including intents but not
	// deletion tombstones (or their intents). Note that the size of the meta kv
	// pair (which could be explicit or implicit) is included in this.
	// Only the meta kv pair counts for the actual length of the encoded key
	// (regular pairs only count the timestamp suffix).
	LiveBytes int64 `protobuf:"fixed64,4,opt,name=live_bytes,json=liveBytes" json:"live_bytes"`
	// live_count is the number of meta keys tracked under live_bytes.
	LiveCount int64 `protobuf:"fixed64,5,opt,name=live_count,json=liveCount" json:"live_count"`
	// key_bytes is the number of bytes stored in all non-system
	// keys, including live, meta, old, and deleted keys.
	// Only meta keys really account for the "full" key; value
	// keys only for the timestamp suffix.
	KeyBytes int64 `protobuf:"fixed64,6,opt,name=key_bytes,json=keyBytes" json:"key_bytes"`
	// key_count is the number of meta keys tracked under key_bytes.
	KeyCount int64 `protobuf:"fixed64,7,opt,name=key_count,json=keyCount" json:"key_count"`
	// value_bytes is the number of bytes in all non-system version
	// values, including meta values.
	ValBytes int64 `protobuf:"fixed64,8,opt,name=val_bytes,json=valBytes" json:"val_bytes"`
	// val_count is the number of meta values tracked under val_bytes.
	ValCount int64 `protobuf:"fixed64,9,opt,name=val_count,json=valCount" json:"val_count"`
	// intent_bytes is the number of bytes in intent key-value
	// pairs (without their meta keys).
	IntentBytes int64 `protobuf:"fixed64,10,opt,name=intent_bytes,json=intentBytes" json:"intent_bytes"`
	// intent_count is the number of keys tracked under intent_bytes.
	// It is equal to the number of meta keys in the system with
	// a non-empty Transaction proto.
	IntentCount int64 `protobuf:"fixed64,11,opt,name=intent_count,json=intentCount" json:"intent_count"`
	// sys_bytes is the number of bytes stored in system-local kv-pairs.
	// This tracks the same quantity as (key_bytes + val_bytes), but
	// for system-local metadata keys (which aren't counted in either
	// key_bytes or val_bytes). Each of the keys falling into this group
	// is documented in keys/constants.go under the localPrefix constant
	// and is prefixed by either LocalRangeIDPrefix or LocalRangePrefix.
	SysBytes int64 `protobuf:"fixed64,12,opt,name=sys_bytes,json=sysBytes" json:"sys_bytes"`
	// sys_count is the number of meta keys tracked under sys_bytes.
	SysCount int64 `protobuf:"fixed64,13,opt,name=sys_count,json=sysCount" json:"sys_count"`
}

MVCCStats tracks byte and instance counts for various groups of keys, values, or key-value pairs; see the field comments for details.

It also tracks two cumulative ages, namely that of intents and non-live (i.e. GC-able) bytes. This computation is intrinsically linked to last_update_nanos and is easy to get wrong. Updates happen only once every full second, as measured by last_update_nanos/1e9. That is, forward updates don't change last_update_nanos until an update at a timestamp which, truncated to the second, is ahead of last_update_nanos/1e9. Then, that difference in seconds times the base quantity (excluding the currently running update) is added to the age. It gets more complicated when data is accounted for with a timestamp behind last_update_nanos. In this case, if more than a second has passed (computed via truncation above), the ages have to be adjusted to account for this late addition. This isn't hard: add the new data's base quantity times the (truncated) number of seconds behind. Important to keep in mind with those computations is that (x/1e9 - y/1e9) does not equal (x-y)/1e9 in most cases.

Note that this struct must be kept at a fixed size by using fixed-size encodings for all fields and by making all fields non-nullable. This is so that it can predict its own impact on the size of the system-local kv-pairs.

func NewPopulatedMVCCStats

func NewPopulatedMVCCStats(r randyMvcc, easy bool) *MVCCStats

func (*MVCCStats) AccountForSelf

func (ms *MVCCStats) AccountForSelf(rangeID roachpb.RangeID) error

AccountForSelf adjusts ms to account for the predicted impact it will have on the values that it records when the structure is initially stored. Specifically, MVCCStats is stored on the RangeStats key, which means that its creation will have an impact on system-local data size and key count.

func (*MVCCStats) Add

func (ms *MVCCStats) Add(oms MVCCStats)

Add adds values from oms to ms. The ages will be moved forward to the larger of the LastUpdateNano timestamps involved.

func (*MVCCStats) AgeTo

func (ms *MVCCStats) AgeTo(nowNanos int64)

AgeTo encapsulates the complexity of computing the increment in age quantities contained in MVCCStats. Two MVCCStats structs only add and subtract meaningfully if their LastUpdateNanos matches, so aging them to the max of their LastUpdateNanos is a prerequisite. If nowNanos is behind ms.LastUpdateNanos, this method is a noop.

func (*MVCCStats) Descriptor

func (*MVCCStats) Descriptor() ([]byte, []int)

func (MVCCStats) GCBytes

func (ms MVCCStats) GCBytes() int64

GCBytes is a convenience function which returns the number of gc bytes, that is the key and value bytes excluding the live bytes.

func (*MVCCStats) Marshal

func (m *MVCCStats) Marshal() (data []byte, err error)

func (*MVCCStats) MarshalTo

func (m *MVCCStats) MarshalTo(data []byte) (int, error)

func (*MVCCStats) ProtoMessage

func (*MVCCStats) ProtoMessage()

func (*MVCCStats) Reset

func (m *MVCCStats) Reset()

func (*MVCCStats) Size

func (m *MVCCStats) Size() (n int)

func (*MVCCStats) String

func (m *MVCCStats) String() string

func (*MVCCStats) Subtract

func (ms *MVCCStats) Subtract(oms MVCCStats)

Subtract removes oms from ms. The ages will be moved forward to the larger of the LastUpdateNano timestamps involved.

func (*MVCCStats) Unmarshal

func (m *MVCCStats) Unmarshal(data []byte) error

type RocksDB

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

RocksDB is a wrapper around a RocksDB database instance.

func (*RocksDB) ApproximateSize

func (r *RocksDB) ApproximateSize(start, end MVCCKey) (uint64, error)

ApproximateSize returns the approximate number of bytes on disk that RocksDB is using to store data for the given range of keys.

func (*RocksDB) Attrs

func (r *RocksDB) Attrs() roachpb.Attributes

Attrs returns the list of attributes describing this engine. This may include a specification of disk type (e.g. hdd, ssd, fio, etc.) and potentially other labels to identify important attributes of the engine.

func (*RocksDB) Capacity

func (r *RocksDB) Capacity() (roachpb.StoreCapacity, error)

Capacity queries the underlying file system for disk capacity information.

func (*RocksDB) Clear

func (r *RocksDB) Clear(key MVCCKey) error

Clear removes the item from the db with the given key.

func (*RocksDB) Close

func (r *RocksDB) Close()

Close closes the database by deallocating the underlying handle.

func (*RocksDB) Closed

func (r *RocksDB) Closed() bool

Closed returns true if the engine is closed.

func (*RocksDB) Commit

func (r *RocksDB) Commit() error

Commit is a noop for RocksDB engine.

func (*RocksDB) Compact

func (r *RocksDB) Compact()

Compact forces compaction on the database. This is currently used only to force partial merges to occur in unit tests.

func (*RocksDB) Defer

func (r *RocksDB) Defer(func())

Defer is not implemented for RocksDB engine.

func (*RocksDB) Destroy

func (r *RocksDB) Destroy() error

Destroy destroys the underlying filesystem data associated with the database.

func (*RocksDB) Flush

func (r *RocksDB) Flush() error

Flush causes RocksDB to write all in-memory data to disk immediately.

func (*RocksDB) Get

func (r *RocksDB) Get(key MVCCKey) ([]byte, error)

Get returns the value for the given key.

func (*RocksDB) GetProto

func (r *RocksDB) GetProto(key MVCCKey, msg proto.Message) (
	ok bool, keyBytes, valBytes int64, err error)

GetProto fetches the value at the specified key and unmarshals it.

func (*RocksDB) GetStats

func (r *RocksDB) GetStats() (*Stats, error)

GetStats retrieves stats from this Engine's RocksDB instance and returns it in a new instance of Stats.

func (*RocksDB) Iterate

func (r *RocksDB) Iterate(start, end MVCCKey, f func(MVCCKeyValue) (bool, error)) error

Iterate iterates from start to end keys, invoking f on each key/value pair. See engine.Iterate for details.

func (*RocksDB) Merge

func (r *RocksDB) Merge(key MVCCKey, value []byte) error

Merge implements the RocksDB merge operator using the function goMergeInit to initialize missing values and goMerge to merge the old and the given value into a new value, which is then stored under key. Currently 64-bit counter logic is implemented. See the documentation of goMerge and goMergeInit for details.

The key and value byte slices may be reused safely. merge takes a copy of them before returning.

func (*RocksDB) NewBatch

func (r *RocksDB) NewBatch() Engine

NewBatch returns a new batch wrapping this rocksdb engine.

func (*RocksDB) NewIterator

func (r *RocksDB) NewIterator(prefix roachpb.Key) Iterator

NewIterator returns an iterator over this rocksdb engine.

func (*RocksDB) NewSnapshot

func (r *RocksDB) NewSnapshot() Engine

NewSnapshot creates a snapshot handle from engine and returns a read-only rocksDBSnapshot engine.

func (*RocksDB) Open

func (r *RocksDB) Open() error

Open creates options and opens the database. If the database doesn't yet exist at the specified directory, one is initialized from scratch. The RocksDB Open and Close methods are reference counted such that subsequent Open calls to an already opened RocksDB instance only bump the reference count. The RocksDB is only closed when a sufficient number of Close calls are performed to bring the reference count down to 0.

func (*RocksDB) Put

func (r *RocksDB) Put(key MVCCKey, value []byte) error

Put sets the given key to the value provided.

The key and value byte slices may be reused safely. put takes a copy of them before returning.

func (*RocksDB) String

func (r *RocksDB) String() string

String formatter.

type Stats

type Stats struct {
	BlockCacheHits           int64
	BlockCacheMisses         int64
	BlockCacheUsage          int64
	BlockCachePinnedUsage    int64
	BloomFilterPrefixChecked int64
	BloomFilterPrefixUseful  int64
	MemtableHits             int64
	MemtableMisses           int64
	MemtableTotalSize        int64
	Flushes                  int64
	Compactions              int64
	TableReadersMemEstimate  int64
}

Stats is a set of RocksDB stats. These are all described in RocksDB

Currently, we collect stats from the following sources:

  1. RocksDB's internal "tickers" (i.e. counters). They're defined in rocksdb/statistics.h
  2. DBEventListener, which implements RocksDB's EventListener interface.
  3. rocksdb::DB::GetProperty().

This is a good resource describing RocksDB's memory-related stats: https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB

type Version

type Version struct {
	Version storageVersion
}

Version stores all the version information for all stores and is used as the format for the version file.

Directories

Path Synopsis

Jump to

Keyboard shortcuts

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