commitlog

package
v0.14.3 Latest Latest
Warning

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

Go to latest
Published: Apr 22, 2020 License: Apache-2.0 Imports: 33 Imported by: 0

README

Overview

The commitlog package contains all the code for reading and writing commitlog files.

Implementation

Writer

The commitlog writer consists of two main components:

  1. The CommitLog itself which contains all of the public APIs as well as the logic for managing concurrency, rotating files, and inspecting the status of the commitlog (via the ActiveLogs and QueueLength APIs).
  2. The CommitLogWriter which contains all of the logic for opening/closing files and writing bytes to disk.
Commitlog

At a high-level, the CommitLog handles writes by inserting them into a queue (buffered channel) in batches and then dequeing the batches in a single-threaded manner and writing all of the individual writes for a batch into the commitlog file one at a time.

Synchronization

The primary synchronization that happens in the commitlog is via the queue (buffered channel). Many goroutines will insert pending writes into the queue concurrently via the Write() and WriteBatch() APIs, however, only a single goroutine will pull items off the queue and write them to disk.

┌──────────────────────────┐
│Goroutine 1: WriteBatch() ├─────┐                                                         ┌───────────────────────────────────┐          ┌───────────────────────────────────┐
└──────────────────────────┘     │                                                         │                                   │          │                                   │
                                 │                                                         │                                   │          │                                   │
                                 │                                                         │                                   │          │                                   │
┌──────────────────────────┐     │       ┌─────────────────────────────────────┐           │             Commitlog             │          │              Writer               │
│Goroutine 2: WriteBatch() │─────┼──────▶│           Commitlog Queue           ├──────────▶│                                   │─────────▶│                                   │
└──────────────────────────┘     │       └─────────────────────────────────────┘           │ Single-threaded Writer Goroutine  │          │           Write to disk           │
                                 │                                                         │                                   │          │                                   │
                                 │                                                         │                                   │          │                                   │
┌──────────────────────────┐     │                                                         │                                   │          │                                   │
│Goroutine 3: WriteBatch() ├─────┘                                                         └───────────────────────────────────┘          └───────────────────────────────────┘
└──────────────────────────┘

The monopic file that was used to generate the diagram above is located at docs/code_assets/commitlog/queue.monopic.

Since there is only one goroutine pulling items off of the queue, any state that it alone manages can remain unsynchronized since no other goroutines will interact with it.

In addition to the queue, the commitlog has two other forms of synchronization:

  1. The closedState lock which is an RWLock. An RLock is held for the duration of any operation during which the commitlog must remain open.
  2. The flushState lock. The scope of this lock is very narrow as its only used to protect access to the lastFlushAt field.
Rotating Files

Rotating commitlog files is initiated by the RotateLogs() API so that callers can control when this occurs. The CommitLog itself will never rotate files on its own without the RotateLogs() API being called.

The commitlog files are not rotated immediately when the RotateLogs() method is called because that would require a lot of complicated and expensive synchronization with the CommitLogWriter goroutine. Instead, a rotateLogsEventType is pushed into the queue and when the single-threaded writer goroutine pulls this event off of the channel it will rotate the commitlog files (since it has exclusive access to them) and then invoke a callback function which notifies the RotateLogs() method call (which has been blocked this whole time) to complete and return success to the caller.

While the CommitLog only writes to a single file at once, it maintains two open writers at all times so that they can be "hot-swapped" when the commitlog files need to be rotated. This allows the single-threaded writer goroutine to continue uninterrupted by syscalls and I/O during rotation events which in turn prevents the queue from backing up. Otherwise, rotation events could block the writer for so long (while it waited for a new file to be created) that it caused the queue to back up significantly.

When a rotation event occurs, instead of waiting for a new file to be opened, the CommitLog writer goroutine will swap the primary and secondary CommitLogWriter such that the secondary CommitLogWriter (which has an empty file) becomes the primary and vice versa. This allows the CommitLog writer goroutine to continue writing uninterrupted.

In the meantime, a goroutine is started in the background that is responsible for resetting the now secondary (formerly primary) CommitLogWriter by closing it (which will flush any pending / buffered writes to disk) and re-opening it (which will create a new empty commitlog file in anticipation of the next rotation event).

Finally, the next time the CommitLog attempts to rotate its commitlogs it will need to use the associated sync.WaitGroup to ensure that the previously spawned background goroutine has completed resetting the secondary CommitLogWriter before it attempts a new hot-swap.

Handling Errors

The current implementation will panic if any I/O errors are ever encountered while writing bytes to disk or opening/closing files. In the future a "commitlog failure policy" similar to Cassandra's "stop" may be introduced.

Testing

The commitlog package is tested via:

  1. Standard unit tests
  2. Property tests
  3. Concurrency tests

File Format

See /docs/m3db/architecture/commitlogs.md.

Documentation

Overview

Package commitlog is a generated GoMock package.

Index

Constants

View Source
const (

	// MaximumQueueSizeQueueChannelSizeRatio is the maximum ratio between the
	// backlog queue size and backlog queue channel size.
	MaximumQueueSizeQueueChannelSizeRatio = 8.0
)

Variables

View Source
var (
	// ErrCommitLogQueueFull is raised when trying to write to the commit log
	// when the queue is full
	ErrCommitLogQueueFull = errors.New("commit log queue is full")
)

Functions

func NewIterator

func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []ErrorWithPath, err error)

NewIterator creates a new commit log iterator

func NextFile added in v0.7.0

func NextFile(opts Options) (string, int, error)

NextFile returns the next commitlog file.

func ReadLogInfo

func ReadLogInfo(filePath string, opts Options) (int64, error)

ReadLogInfo reads the commit log info out of a commitlog file

Types

type CommitLog

type CommitLog interface {
	// Open the commit log
	Open() error

	// Write will write an entry in the commit log for a given series
	Write(
		ctx context.Context,
		series ts.Series,
		datapoint ts.Datapoint,
		unit xtime.Unit,
		annotation ts.Annotation,
	) error

	// WriteBatch is the same as Write, but in batch.
	WriteBatch(
		ctx context.Context,
		writes ts.WriteBatch,
	) error

	// Close the commit log
	Close() error

	// ActiveLogs returns a slice of the active commitlogs.
	ActiveLogs() (persist.CommitLogFiles, error)

	// RotateLogs rotates the commitlog and returns the File that represents
	// the new commitlog file.
	RotateLogs() (persist.CommitLogFile, error)

	// QueueLength returns the number of writes that are currently in the commitlog
	// queue.
	QueueLength() int64
}

CommitLog provides a synchronized commit log

func NewCommitLog

func NewCommitLog(opts Options) (CommitLog, error)

NewCommitLog creates a new commit log

type ErrorWithPath added in v0.4.8

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

ErrorWithPath is an error that includes the path of the file that had the error.

func Files added in v0.4.0

Files returns a slice of all available commit log files on disk along with their associated metadata.

func NewErrorWithPath added in v0.4.8

func NewErrorWithPath(err error, path string) ErrorWithPath

NewErrorWithPath creates a new ErrorWithPath.

func (ErrorWithPath) Error added in v0.4.8

func (e ErrorWithPath) Error() string

Error returns the error.

func (ErrorWithPath) Path added in v0.4.8

func (e ErrorWithPath) Path() string

Path returns the path of the file that the error is associated with.

type FileFilterInfo added in v0.9.0

type FileFilterInfo struct {
	// If isCorrupt is true then File will contain a valid CommitLogFile, otherwise
	// ErrorWithPath will contain an error and the path of the corrupt file.
	File      persist.CommitLogFile
	Err       ErrorWithPath
	IsCorrupt bool
}

FileFilterInfo contains information about a commitog file that can be used to determine whether the iterator should filter it out or not.

type FileFilterPredicate

type FileFilterPredicate func(f FileFilterInfo) bool

FileFilterPredicate is a predicate that allows the caller to determine which commitlogs the iterator should read from.

func ReadAllPredicate

func ReadAllPredicate() FileFilterPredicate

ReadAllPredicate can be passed as the ReadCommitLogPredicate for callers that want a convenient way to read all the commitlogs

type Iterator

type Iterator interface {
	// Next returns whether the iterator has the next value
	Next() bool

	// Current returns the current commit log entry
	Current() (ts.Series, ts.Datapoint, xtime.Unit, ts.Annotation)

	// Err returns an error if an error occurred
	Err() error

	// Close the iterator
	Close()
}

Iterator provides an iterator for commit logs

type IteratorOpts

type IteratorOpts struct {
	CommitLogOptions      Options
	FileFilterPredicate   FileFilterPredicate
	SeriesFilterPredicate SeriesFilterPredicate
}

IteratorOpts is a struct that contains coptions for the Iterator

type MockCommitLog

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

MockCommitLog is a mock of CommitLog interface

func NewMockCommitLog

func NewMockCommitLog(ctrl *gomock.Controller) *MockCommitLog

NewMockCommitLog creates a new mock instance

func (*MockCommitLog) ActiveLogs added in v0.4.8

func (m *MockCommitLog) ActiveLogs() (persist.CommitLogFiles, error)

ActiveLogs mocks base method

func (*MockCommitLog) Close

func (m *MockCommitLog) Close() error

Close mocks base method

func (*MockCommitLog) EXPECT

EXPECT returns an object that allows the caller to indicate expected use

func (*MockCommitLog) Open

func (m *MockCommitLog) Open() error

Open mocks base method

func (*MockCommitLog) QueueLength added in v0.7.3

func (m *MockCommitLog) QueueLength() int64

QueueLength mocks base method

func (*MockCommitLog) RotateLogs added in v0.4.8

func (m *MockCommitLog) RotateLogs() (persist.CommitLogFile, error)

RotateLogs mocks base method

func (*MockCommitLog) Write

func (m *MockCommitLog) Write(ctx context.Context, series ts.Series, datapoint ts.Datapoint, unit time0.Unit, annotation ts.Annotation) error

Write mocks base method

func (*MockCommitLog) WriteBatch added in v0.4.8

func (m *MockCommitLog) WriteBatch(ctx context.Context, writes ts.WriteBatch) error

WriteBatch mocks base method

type MockCommitLogMockRecorder

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

MockCommitLogMockRecorder is the mock recorder for MockCommitLog

func (*MockCommitLogMockRecorder) ActiveLogs added in v0.4.8

func (mr *MockCommitLogMockRecorder) ActiveLogs() *gomock.Call

ActiveLogs indicates an expected call of ActiveLogs

func (*MockCommitLogMockRecorder) Close

func (mr *MockCommitLogMockRecorder) Close() *gomock.Call

Close indicates an expected call of Close

func (*MockCommitLogMockRecorder) Open

Open indicates an expected call of Open

func (*MockCommitLogMockRecorder) QueueLength added in v0.7.3

func (mr *MockCommitLogMockRecorder) QueueLength() *gomock.Call

QueueLength indicates an expected call of QueueLength

func (*MockCommitLogMockRecorder) RotateLogs added in v0.4.8

func (mr *MockCommitLogMockRecorder) RotateLogs() *gomock.Call

RotateLogs indicates an expected call of RotateLogs

func (*MockCommitLogMockRecorder) Write

func (mr *MockCommitLogMockRecorder) Write(ctx, series, datapoint, unit, annotation interface{}) *gomock.Call

Write indicates an expected call of Write

func (*MockCommitLogMockRecorder) WriteBatch added in v0.4.8

func (mr *MockCommitLogMockRecorder) WriteBatch(ctx, writes interface{}) *gomock.Call

WriteBatch indicates an expected call of WriteBatch

type MockIterator

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

MockIterator is a mock of Iterator interface

func NewMockIterator

func NewMockIterator(ctrl *gomock.Controller) *MockIterator

NewMockIterator creates a new mock instance

func (*MockIterator) Close

func (m *MockIterator) Close()

Close mocks base method

func (*MockIterator) Current

func (m *MockIterator) Current() (ts.Series, ts.Datapoint, time0.Unit, ts.Annotation)

Current mocks base method

func (*MockIterator) EXPECT

EXPECT returns an object that allows the caller to indicate expected use

func (*MockIterator) Err

func (m *MockIterator) Err() error

Err mocks base method

func (*MockIterator) Next

func (m *MockIterator) Next() bool

Next mocks base method

type MockIteratorMockRecorder

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

MockIteratorMockRecorder is the mock recorder for MockIterator

func (*MockIteratorMockRecorder) Close

func (mr *MockIteratorMockRecorder) Close() *gomock.Call

Close indicates an expected call of Close

func (*MockIteratorMockRecorder) Current

func (mr *MockIteratorMockRecorder) Current() *gomock.Call

Current indicates an expected call of Current

func (*MockIteratorMockRecorder) Err

Err indicates an expected call of Err

func (*MockIteratorMockRecorder) Next

func (mr *MockIteratorMockRecorder) Next() *gomock.Call

Next indicates an expected call of Next

type MockOptions

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

MockOptions is a mock of Options interface

func NewMockOptions

func NewMockOptions(ctrl *gomock.Controller) *MockOptions

NewMockOptions creates a new mock instance

func (*MockOptions) BacklogQueueChannelSize added in v0.4.8

func (m *MockOptions) BacklogQueueChannelSize() int

BacklogQueueChannelSize mocks base method

func (*MockOptions) BacklogQueueSize

func (m *MockOptions) BacklogQueueSize() int

BacklogQueueSize mocks base method

func (*MockOptions) BlockSize

func (m *MockOptions) BlockSize() time.Duration

BlockSize mocks base method

func (*MockOptions) BytesPool

func (m *MockOptions) BytesPool() pool.CheckedBytesPool

BytesPool mocks base method

func (*MockOptions) ClockOptions

func (m *MockOptions) ClockOptions() clock.Options

ClockOptions mocks base method

func (*MockOptions) EXPECT

func (m *MockOptions) EXPECT() *MockOptionsMockRecorder

EXPECT returns an object that allows the caller to indicate expected use

func (*MockOptions) FilesystemOptions

func (m *MockOptions) FilesystemOptions() fs.Options

FilesystemOptions mocks base method

func (*MockOptions) FlushInterval

func (m *MockOptions) FlushInterval() time.Duration

FlushInterval mocks base method

func (*MockOptions) FlushSize

func (m *MockOptions) FlushSize() int

FlushSize mocks base method

func (*MockOptions) IdentifierPool

func (m *MockOptions) IdentifierPool() ident.Pool

IdentifierPool mocks base method

func (*MockOptions) InstrumentOptions

func (m *MockOptions) InstrumentOptions() instrument.Options

InstrumentOptions mocks base method

func (*MockOptions) ReadConcurrency

func (m *MockOptions) ReadConcurrency() int

ReadConcurrency mocks base method

func (*MockOptions) SetBacklogQueueChannelSize added in v0.4.8

func (m *MockOptions) SetBacklogQueueChannelSize(value int) Options

SetBacklogQueueChannelSize mocks base method

func (*MockOptions) SetBacklogQueueSize

func (m *MockOptions) SetBacklogQueueSize(value int) Options

SetBacklogQueueSize mocks base method

func (*MockOptions) SetBlockSize

func (m *MockOptions) SetBlockSize(value time.Duration) Options

SetBlockSize mocks base method

func (*MockOptions) SetBytesPool

func (m *MockOptions) SetBytesPool(value pool.CheckedBytesPool) Options

SetBytesPool mocks base method

func (*MockOptions) SetClockOptions

func (m *MockOptions) SetClockOptions(value clock.Options) Options

SetClockOptions mocks base method

func (*MockOptions) SetFilesystemOptions

func (m *MockOptions) SetFilesystemOptions(value fs.Options) Options

SetFilesystemOptions mocks base method

func (*MockOptions) SetFlushInterval

func (m *MockOptions) SetFlushInterval(value time.Duration) Options

SetFlushInterval mocks base method

func (*MockOptions) SetFlushSize

func (m *MockOptions) SetFlushSize(value int) Options

SetFlushSize mocks base method

func (*MockOptions) SetIdentifierPool

func (m *MockOptions) SetIdentifierPool(value ident.Pool) Options

SetIdentifierPool mocks base method

func (*MockOptions) SetInstrumentOptions

func (m *MockOptions) SetInstrumentOptions(value instrument.Options) Options

SetInstrumentOptions mocks base method

func (*MockOptions) SetReadConcurrency

func (m *MockOptions) SetReadConcurrency(concurrency int) Options

SetReadConcurrency mocks base method

func (*MockOptions) SetStrategy

func (m *MockOptions) SetStrategy(value Strategy) Options

SetStrategy mocks base method

func (*MockOptions) Strategy

func (m *MockOptions) Strategy() Strategy

Strategy mocks base method

func (*MockOptions) Validate

func (m *MockOptions) Validate() error

Validate mocks base method

type MockOptionsMockRecorder

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

MockOptionsMockRecorder is the mock recorder for MockOptions

func (*MockOptionsMockRecorder) BacklogQueueChannelSize added in v0.4.8

func (mr *MockOptionsMockRecorder) BacklogQueueChannelSize() *gomock.Call

BacklogQueueChannelSize indicates an expected call of BacklogQueueChannelSize

func (*MockOptionsMockRecorder) BacklogQueueSize

func (mr *MockOptionsMockRecorder) BacklogQueueSize() *gomock.Call

BacklogQueueSize indicates an expected call of BacklogQueueSize

func (*MockOptionsMockRecorder) BlockSize

func (mr *MockOptionsMockRecorder) BlockSize() *gomock.Call

BlockSize indicates an expected call of BlockSize

func (*MockOptionsMockRecorder) BytesPool

func (mr *MockOptionsMockRecorder) BytesPool() *gomock.Call

BytesPool indicates an expected call of BytesPool

func (*MockOptionsMockRecorder) ClockOptions

func (mr *MockOptionsMockRecorder) ClockOptions() *gomock.Call

ClockOptions indicates an expected call of ClockOptions

func (*MockOptionsMockRecorder) FilesystemOptions

func (mr *MockOptionsMockRecorder) FilesystemOptions() *gomock.Call

FilesystemOptions indicates an expected call of FilesystemOptions

func (*MockOptionsMockRecorder) FlushInterval

func (mr *MockOptionsMockRecorder) FlushInterval() *gomock.Call

FlushInterval indicates an expected call of FlushInterval

func (*MockOptionsMockRecorder) FlushSize

func (mr *MockOptionsMockRecorder) FlushSize() *gomock.Call

FlushSize indicates an expected call of FlushSize

func (*MockOptionsMockRecorder) IdentifierPool

func (mr *MockOptionsMockRecorder) IdentifierPool() *gomock.Call

IdentifierPool indicates an expected call of IdentifierPool

func (*MockOptionsMockRecorder) InstrumentOptions

func (mr *MockOptionsMockRecorder) InstrumentOptions() *gomock.Call

InstrumentOptions indicates an expected call of InstrumentOptions

func (*MockOptionsMockRecorder) ReadConcurrency

func (mr *MockOptionsMockRecorder) ReadConcurrency() *gomock.Call

ReadConcurrency indicates an expected call of ReadConcurrency

func (*MockOptionsMockRecorder) SetBacklogQueueChannelSize added in v0.4.8

func (mr *MockOptionsMockRecorder) SetBacklogQueueChannelSize(value interface{}) *gomock.Call

SetBacklogQueueChannelSize indicates an expected call of SetBacklogQueueChannelSize

func (*MockOptionsMockRecorder) SetBacklogQueueSize

func (mr *MockOptionsMockRecorder) SetBacklogQueueSize(value interface{}) *gomock.Call

SetBacklogQueueSize indicates an expected call of SetBacklogQueueSize

func (*MockOptionsMockRecorder) SetBlockSize

func (mr *MockOptionsMockRecorder) SetBlockSize(value interface{}) *gomock.Call

SetBlockSize indicates an expected call of SetBlockSize

func (*MockOptionsMockRecorder) SetBytesPool

func (mr *MockOptionsMockRecorder) SetBytesPool(value interface{}) *gomock.Call

SetBytesPool indicates an expected call of SetBytesPool

func (*MockOptionsMockRecorder) SetClockOptions

func (mr *MockOptionsMockRecorder) SetClockOptions(value interface{}) *gomock.Call

SetClockOptions indicates an expected call of SetClockOptions

func (*MockOptionsMockRecorder) SetFilesystemOptions

func (mr *MockOptionsMockRecorder) SetFilesystemOptions(value interface{}) *gomock.Call

SetFilesystemOptions indicates an expected call of SetFilesystemOptions

func (*MockOptionsMockRecorder) SetFlushInterval

func (mr *MockOptionsMockRecorder) SetFlushInterval(value interface{}) *gomock.Call

SetFlushInterval indicates an expected call of SetFlushInterval

func (*MockOptionsMockRecorder) SetFlushSize

func (mr *MockOptionsMockRecorder) SetFlushSize(value interface{}) *gomock.Call

SetFlushSize indicates an expected call of SetFlushSize

func (*MockOptionsMockRecorder) SetIdentifierPool

func (mr *MockOptionsMockRecorder) SetIdentifierPool(value interface{}) *gomock.Call

SetIdentifierPool indicates an expected call of SetIdentifierPool

func (*MockOptionsMockRecorder) SetInstrumentOptions

func (mr *MockOptionsMockRecorder) SetInstrumentOptions(value interface{}) *gomock.Call

SetInstrumentOptions indicates an expected call of SetInstrumentOptions

func (*MockOptionsMockRecorder) SetReadConcurrency

func (mr *MockOptionsMockRecorder) SetReadConcurrency(concurrency interface{}) *gomock.Call

SetReadConcurrency indicates an expected call of SetReadConcurrency

func (*MockOptionsMockRecorder) SetStrategy

func (mr *MockOptionsMockRecorder) SetStrategy(value interface{}) *gomock.Call

SetStrategy indicates an expected call of SetStrategy

func (*MockOptionsMockRecorder) Strategy

func (mr *MockOptionsMockRecorder) Strategy() *gomock.Call

Strategy indicates an expected call of Strategy

func (*MockOptionsMockRecorder) Validate

func (mr *MockOptionsMockRecorder) Validate() *gomock.Call

Validate indicates an expected call of Validate

type Options

type Options interface {
	// Validate validates the Options.
	Validate() error

	// SetClockOptions sets the clock options.
	SetClockOptions(value clock.Options) Options

	// ClockOptions returns the clock options.
	ClockOptions() clock.Options

	// SetInstrumentOptions sets the instrumentation options.
	SetInstrumentOptions(value instrument.Options) Options

	// InstrumentOptions returns the instrumentation options,
	InstrumentOptions() instrument.Options

	// SetBlockSize sets the block size.
	SetBlockSize(value time.Duration) Options

	// BlockSize returns the block size.
	BlockSize() time.Duration

	// SetFilesystemOptions sets the filesystem options.
	SetFilesystemOptions(value fs.Options) Options

	// FilesystemOptions returns the filesystem options.
	FilesystemOptions() fs.Options

	// SetFlushSize sets the flush size.
	SetFlushSize(value int) Options

	// FlushSize returns the flush size.
	FlushSize() int

	// SetStrategy sets the strategy.
	SetStrategy(value Strategy) Options

	// Strategy returns the strategy.
	Strategy() Strategy

	// SetFlushInterval sets the flush interval.
	SetFlushInterval(value time.Duration) Options

	// FlushInterval returns the flush interval.
	FlushInterval() time.Duration

	// SetBacklogQueueSize sets the backlog queue size.
	SetBacklogQueueSize(value int) Options

	// BacklogQueueSize returns the backlog queue size.
	BacklogQueueSize() int

	// SetBacklogQueueChannelSize sets the size of the Golang channel
	// that backs the queue.
	SetBacklogQueueChannelSize(value int) Options

	// BacklogQueueChannelSize returns the size of the Golang channel
	// that backs the queue.
	BacklogQueueChannelSize() int

	// SetBytesPool sets the checked bytes pool.
	SetBytesPool(value pool.CheckedBytesPool) Options

	// BytesPool returns the checked bytes pool.
	BytesPool() pool.CheckedBytesPool

	// SetReadConcurrency sets the concurrency of the reader.
	SetReadConcurrency(concurrency int) Options

	// ReadConcurrency returns the concurrency of the reader.
	ReadConcurrency() int

	// SetIdentifierPool sets the IdentifierPool to use for pooling identifiers.
	SetIdentifierPool(value ident.Pool) Options

	// IdentifierPool returns the IdentifierPool to use for pooling identifiers.
	IdentifierPool() ident.Pool
}

Options represents the options for the commit log.

func NewOptions

func NewOptions() Options

NewOptions creates new commit log options

type SeriesFilterPredicate

type SeriesFilterPredicate func(id ident.ID, namespace ident.ID) bool

SeriesFilterPredicate is a predicate that determines whether datapoints for a given series should be returned from the Commit log reader. The predicate is pushed down to the reader level to prevent having to run the same function for every datapoint for a given series.

func ReadAllSeriesPredicate

func ReadAllSeriesPredicate() SeriesFilterPredicate

ReadAllSeriesPredicate can be passed as the seriesPredicate for callers that want a convenient way to read all series in the commitlogs

type Strategy

type Strategy int

Strategy describes the commit log writing strategy

const (
	// StrategyWriteWait describes the strategy that waits
	// for the buffered commit log chunk that contains a write to flush
	// before acknowledging a write
	StrategyWriteWait Strategy = iota

	// StrategyWriteBehind describes the strategy that does not wait
	// for the buffered commit log chunk that contains a write to flush
	// before acknowledging a write
	StrategyWriteBehind
)

Jump to

Keyboard shortcuts

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