dragonboat

package module
v3.3.8 Latest Latest
Warning

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

Go to latest
Published: Sep 25, 2023 License: Apache-2.0 Imports: 36 Imported by: 52

README

dragonboat

Dragonboat - A Multi-Group Raft library in Go / 中文版

license Build status Go Report Card codecov Godoc

News

  • 2021-01-20 Dragonboat v3.3 has been released, please check CHANGELOG for all changes.
  • 2020-03-05 Dragonboat v3.2 has been released, please check CHANGELOG for details.

About

Dragonboat is a high performance multi-group Raft consensus library in pure Go.

Consensus algorithms such as Raft provides fault-tolerance by alllowing a system continue to operate as long as the majority member servers are available. For example, a Raft cluster of 5 servers can make progress even if 2 servers fail. It also appears to clients as a single entity with strong data consistency always provided. All Raft replicas can be used to handle read requests for aggregated read throughput.

Dragonboat handles all technical difficulties associated with Raft to allow users to just focus on their application domains. It is also very easy to use, our step-by-step examples can help new users to master it in half an hour.

Features

  • Easy to use pure-Go APIs for building Raft based applications
  • Feature complete and scalable multi-group Raft implementation
  • Disk based and memory based state machine support
  • Fully pipelined and TLS mutual authentication support, ready for high latency open environment
  • Custom Raft log storage and transport support, easy to integrate with latest I/O techs
  • Prometheus based health metrics support
  • Built-in tool to repair Raft clusters that permanently lost the quorum
  • Extensively tested including using Jepsen's Knossos linearizability checker, some results are here

Most features covered in Diego Ongaro's Raft thesis have been supported -

  • leader election, log replication, snapshotting and log compaction
  • membership change
  • ReadIndex protocol for read-only queries
  • leadership transfer
  • non-voting member
  • witness member
  • idempotent update transparent to applications
  • batching and pipelining
  • disk based state machine

Performance

Dragonboat is the fastest open source multi-group Raft implementation on Github.

For 3-nodes system using mid-range hardware (details here) and in-memory state machine, when RocksDB is used as the storage engine, Dragonboat can sustain at 9 million writes per second when the payload is 16bytes each or 11 million mixed I/O per second at 9:1 read:write ratio. High throughput is maintained in geographically distributed environment. When the RTT between nodes is 30ms, 2 million I/O per second can still be achieved using a much larger number of clients. throughput

The number of concurrent active Raft groups affects the overall throughput as requests become harder to be batched. On the other hand, having thousands of idle Raft groups has a much smaller impact on throughput. nodes

Table below shows write latencies in millisecond, Dragonboat has <5ms P99 write latency when handling 8 million writes per second at 16 bytes each. Read latency is lower than writes as the ReadIndex protocol employed for linearizable reads doesn't require fsync-ed disk I/O.

Ops Payload Size 99.9% percentile 99% percentile AVG
1m 16 2.24 1.19 0.79
1m 128 11.11 1.37 0.92
1m 1024 71.61 25.91 3.75
5m 16 4.64 1.95 1.16
5m 128 36.61 6.55 1.96
8m 16 12.01 4.65 2.13

When tested on a single Raft group, Dragonboat can sustain writes at 1.25 million per second when payload is 16 bytes each, average latency is 1.3ms and the P99 latency is 2.6ms. This is achieved when using an average of 3 cores (2.8GHz) on each server.

As visualized below, Stop-the-World pauses caused by Go1.11's GC are sub-millisecond on highly loaded systems. Such very short Stop-the-World pause time is further significantly reduced in Go 1.12. Golang's runtime.ReadMemStats reports that less than 1% of the available CPU time is used by GC on highly loaded system. stw

Requirements

  • x86_64/Linux, x86_64/MacOS or ARM64/Linux, Go 1.15 or 1.14

Getting Started

Master is our unstable branch for development. Please use the latest released versions for any production purposes. For Dragonboat v3.3.x, please follow the instructions in v3.3.x's README.md.

Go 1.14 or above with Go module support is required.

To use Dragonboat, make sure to import the package github.com/lni/dragonboat/v3. Also add "github.com/lni/dragonboat/v3 v3.3.0" to the require section of your project's go.mod file.

By default, Pebble is used for storing Raft Logs in Dragonboat. RocksDB and other storage engines are also supported, more info here.

You can also follow our examples on how to use Dragonboat.

Documents

FAQ, docs, step-by-step examples, DevOps doc, CHANGELOG and online chat are available.

Examples

Dragonboat examples are here.

Status

Dragonboat is production ready.

Contributing

For reporting bugs, please open an issue. For contributing improvements or new features, please send in the pull request.

License

Dragonboat is licensed under the Apache License Version 2.0. See LICENSE for details.

Third party code used in Dragonboat and their licenses is summarized here.

Documentation

Overview

Package dragonboat is a multi-group Raft implementation.

The NodeHost struct is the facade interface for all features provided by the dragonboat package. Each NodeHost instance usually runs on a separate host managing its CPU, storage and network resources. Each NodeHost can manage Raft nodes from many different Raft groups known as Raft clusters. Each Raft cluster is identified by its ClusterID and it usually consists of multiple nodes, each identified its NodeID value. Nodes from the same Raft cluster can be considered as replicas of the same data, they are suppose to be distributed on different NodeHost instances across the network, this brings fault tolerance to machine and network failures as application data stored in the Raft cluster will be available as long as the majority of its managing NodeHost instances (i.e. its underlying hosts) are available.

User applications can leverage the power of the Raft protocol implemented in dragonboat by implementing the IStateMachine or IOnDiskStateMachine component, as defined in github.com/lni/dragonboat/v3/statemachine. Known as user state machines, each IStateMachine and IOnDiskStateMachine instance is in charge of updating, querying and snapshotting application data with minimum exposure to the complexity of the Raft protocol implementation.

User applications can use NodeHost's APIs to update the state of their IStateMachine or IOnDiskStateMachine instances, this is called making proposals. Once accepted by the majority nodes of a Raft cluster, the proposal is considered as committed and it will be applied on all member nodes of the Raft cluster. Applications can also make linearizable reads to query the state of the IStateMachine or IOnDiskStateMachine instances. Dragonboat employs the ReadIndex protocol invented by Diego Ongaro for fast linearizable reads.

Dragonboat guarantees the linearizability of your I/O when interacting with the IStateMachine or IOnDiskStateMachine instances. In plain English, writes (via making proposal) to your Raft cluster appears to be instantaneous, once a write is completed, all later reads (linearizable read using the ReadIndex protocol as implemented and provided in dragonboat) should return the value of that write or a later write. Once a value is returned by a linearizable read, all later reads should return the same value or the result of a later write.

To strictly provide such guarantee, we need to implement the at-most-once semantic required by linearizability. For a client, when it retries the proposal that failed to complete before its deadline during the previous attempt, it has the risk to have the same proposal committed and applied twice into the user state machine. Dragonboat prevents this by implementing the client session concept described in Diego Ongaro's PhD thesis.

Arbitrary number of Raft clusters can be launched across the network simultaneously to aggregate distributed processing and storage capacities. Users can also make membership change requests to add or remove nodes from any interested Raft cluster.

NodeHost APIs for making the above mentioned requests can be loosely classified into two categories, synchronous and asynchronous APIs. Synchronous APIs will not return until the completion of the requested operation. Their method names all start with Sync*. The asynchronous counterparts of such synchronous APIs, on the other hand, usually return immediately. This allows users to concurrently initiate multiple such asynchronous operations to save the total amount of time required to complete all of them.

Dragonboat is a feature complete Multi-Group Raft implementation - snapshotting, membership change, leadership transfer, non-voting members and disk based state machine are all provided.

Dragonboat is also extensively optimized. The Raft protocol implementation is fully pipelined, meaning proposals can start before the completion of previous proposals. This is critical for system throughput in high latency environment. Dragonboat is also fully batched, internal operations are batched whenever possible to maximize the overall throughput.

Index

Constants

View Source
const (
	// DragonboatMajor is the major version number
	DragonboatMajor = 3
	// DragonboatMinor is the minor version number
	DragonboatMinor = 3
	// DragonboatPatch is the patch version number
	DragonboatPatch = 8
	// DEVVersion is a boolean flag indicating whether this is a dev version
	DEVVersion = false
)

Variables

View Source
var (
	// ErrClosed is returned when a request is made on closed NodeHost instance.
	ErrClosed = errors.New("dragonboat: closed")
	// ErrNodeRemoved indictes that the requested node has been removed.
	ErrNodeRemoved = errors.New("node removed")
	// ErrClusterNotFound indicates that the specified cluster is not found.
	ErrClusterNotFound = errors.New("cluster not found")
	// ErrClusterAlreadyExist indicates that the specified cluster already exist.
	ErrClusterAlreadyExist = errors.New("cluster already exist")
	// ErrClusterNotStopped indicates that the specified cluster is still running
	// and thus prevented the requested operation to be completed.
	ErrClusterNotStopped = errors.New("cluster not stopped")
	// ErrInvalidClusterSettings indicates that cluster settings specified for
	// the StartCluster method are invalid.
	ErrInvalidClusterSettings = errors.New("cluster settings are invalid")
	// ErrClusterNotBootstrapped indicates that the specified cluster has not
	// been boostrapped yet. When starting this node, depending on whether this
	// node is an initial member of the Raft cluster, you must either specify
	// all of its initial members or set the join flag to true.
	// When used correctly, dragonboat only returns this error in the rare
	// situation when you try to restart a node crashed during its previous
	// bootstrap attempt.
	ErrClusterNotBootstrapped = errors.New("cluster not bootstrapped")
	// ErrDeadlineNotSet indicates that the context parameter provided does not
	// carry a deadline.
	ErrDeadlineNotSet = errors.New("deadline not set")
	// ErrInvalidDeadline indicates that the specified deadline is invalid, e.g.
	// time in the past.
	ErrInvalidDeadline = errors.New("invalid deadline")
	// ErrDirNotExist indicates that the specified dir does not exist.
	ErrDirNotExist = errors.New("specified dir does not exist")
)
View Source
var (
	// ErrInvalidOperation indicates that the requested operation is not allowed.
	// e.g. making read or write requests on witness node are not allowed.
	ErrInvalidOperation = errors.New("invalid operation")
	// ErrInvalidAddress indicates that the specified address is invalid.
	ErrInvalidAddress = errors.New("invalid address")
	// ErrInvalidSession indicates that the specified client session is invalid.
	ErrInvalidSession = errors.New("invalid session")
	// ErrTimeoutTooSmall indicates that the specified timeout value is too small.
	ErrTimeoutTooSmall = errors.New("specified timeout value is too small")
	// ErrPayloadTooBig indicates that the payload is too big.
	ErrPayloadTooBig = errors.New("payload is too big")
	// ErrSystemBusy indicates that the system is too busy to handle the request.
	// This might be caused when the Raft node reached its MaxInMemLogSize limit
	// or other system limits. For a requested snapshot, leadership transfer or
	// Raft config change operation, ErrSystemBusy means there is already such a
	// request waiting to be processed.
	ErrSystemBusy = errors.New("system is too busy try again later")
	// ErrClusterClosed indicates that the requested cluster is being shut down.
	ErrClusterClosed = errors.New("raft cluster already closed")
	// ErrClusterNotInitialized indicates that the requested operation can not be
	// completed as the involved raft cluster has not been initialized yet.
	ErrClusterNotInitialized = errors.New("raft cluster not initialized yet")
	// ErrTimeout indicates that the operation timed out.
	ErrTimeout = errors.New("timeout")
	// ErrCanceled indicates that the request has been canceled.
	ErrCanceled = errors.New("request canceled")
	// ErrRejected indicates that the request has been rejected.
	ErrRejected = errors.New("request rejected")
	// ErrAborted indicates that the request has been aborted, usually by user
	// defined behaviours.
	ErrAborted = errors.New("request aborted")
	// ErrClusterNotReady indicates that the request has been dropped as the
	// specified raft cluster is not ready to handle the request. Unknown leader
	// is the most common cause of this error, trying to use a cluster not fully
	// initialized is another major cause of ErrClusterNotReady.
	ErrClusterNotReady = errors.New("request dropped as the cluster is not ready")
	// ErrInvalidTarget indicates that the specified node id invalid.
	ErrInvalidTarget = errors.New("invalid target node ID")
	// ErrInvalidNodeHostID indicates that the NodeHost ID value provided is
	// invalid
	ErrInvalidNodeHostID = errors.New("invalid NodeHost ID value")
)
View Source
var (
	// ErrBadKey indicates that the key is bad, retry the request is recommended.
	//
	// Depreciated: ErrBadKey is no longer used.
	ErrBadKey = errors.New("bad key try again later")
	// ErrPendingLeaderTransferExist has been depredicated, use ErrSystemBusy.
	//
	// Depreciated: ErrPendingLeaderTransferExist is depreciated.
	ErrPendingLeaderTransferExist = ErrSystemBusy
	// ErrPendingConfigChangeExist has been deprecicated, use ErrSystemBusy.
	//
	// Depreciated: ErrPendingConfigChangeExist is depreciated.
	ErrPendingConfigChangeExist = ErrSystemBusy
	// ErrPendingSnapshotRequestExist has been depreciated, use ErrSystemBusy.
	//
	// Depreciated: ErrPendingSnapshotRequestExist is depreciated.
	ErrPendingSnapshotRequestExist = ErrSystemBusy
)
View Source
var (
	// ErrNoSnapshot is the error used to indicate that there is no snapshot
	// available.
	ErrNoSnapshot = errors.New("no snapshot available")
)

Functions

func IsTempError

func IsTempError(err error) bool

IsTempError returns a boolean value indicating whether the specified error is a temporary error that worth to be retried later with the exact same input, potentially on a more suitable NodeHost instance.

func WriteHealthMetrics added in v3.2.0

func WriteHealthMetrics(w io.Writer)

WriteHealthMetrics writes all health metrics in Prometheus format to the specified writer. This function is typically called by the metrics http handler.

Types

type ClusterInfo

type ClusterInfo struct {
	// ClusterID is the cluster ID of the Raft cluster node.
	ClusterID uint64
	// NodeID is the node ID of the Raft cluster node.
	NodeID uint64
	// Nodes is a map of member node IDs to their Raft addresses.
	Nodes map[uint64]string
	// ConfigChangeIndex is the current config change index of the Raft node.
	// ConfigChangeIndex is Raft Log index of the last applied membership
	// change entry.
	ConfigChangeIndex uint64
	// StateMachineType is the type of the state machine.
	StateMachineType sm.Type
	// IsLeader indicates whether this is a leader node.
	IsLeader bool
	// IsObserver indicates whether this is a non-voting observer node.
	IsObserver bool
	// IsWitness indicates whether this is a witness node without actual log.
	IsWitness bool
	// Pending is a boolean flag indicating whether details of the cluster node
	// is not available. The Pending flag is set to true usually because the node
	// has not had anything applied yet.
	Pending bool
}

ClusterInfo is a record for representing the state of a Raft cluster based on the knowledge of the local NodeHost instance.

type GossipInfo added in v3.3.0

type GossipInfo struct {
	// Enabled is a boolean flag indicating whether the gossip service is enabled.
	Enabled bool
	// AdvertiseAddress is the advertise address used by the gossip service.
	AdvertiseAddress string
	// NumOfLiveNodeHosts is the number of current live NodeHost instances known
	// to the gossip service. Note that the gossip service always knowns the
	// local NodeHost instance itself. When the NumOfKnownNodeHosts value is 1,
	// it means the gossip service doesn't know any other NodeHost instance that
	// is considered as live.
	NumOfKnownNodeHosts int
}

GossipInfo contains details of the gossip service.

type INodeUser

type INodeUser interface {
	// ClusterID is the cluster ID of the node.
	ClusterID() uint64
	// NodeID is the node ID of the node.
	NodeID() uint64
	// Propose starts an asynchronous proposal on the Raft cluster represented by
	// the INodeUser instance. Its semantics is the same as the Propose() method
	// in NodeHost.
	Propose(s *client.Session,
		cmd []byte, timeout time.Duration) (*RequestState, error)
	// ReadIndex starts the asynchronous ReadIndex protocol used for linearizable
	// reads on the Raft cluster represented by the INodeUser instance. Its
	// semantics is the same as the ReadIndex() method in NodeHost.
	ReadIndex(timeout time.Duration) (*RequestState, error)
}

INodeUser is the interface implemented by a Raft node user type. A Raft node user can be used to directly initiate proposals or read index operations without locating the Raft node in NodeHost's node list first. It is useful when doing bulk load operations on selected clusters.

type Membership

type Membership struct {
	// ConfigChangeID is the Raft entry index of the last applied membership
	// change entry.
	ConfigChangeID uint64
	// Nodes is a map of NodeID values to NodeHost Raft addresses for all regular
	// Raft nodes.
	Nodes map[uint64]string
	// Observers is a map of NodeID values to NodeHost Raft addresses for all
	// observers in the Raft cluster.
	Observers map[uint64]string
	// Witnesses is a map of NodeID values to NodeHost Raft addrsses for all
	// witnesses in the Raft cluster.
	Witnesses map[uint64]string
	// Removed is a set of NodeID values that have been removed from the Raft
	// cluster. They are not allowed to be added back to the cluster.
	Removed map[uint64]struct{}
}

Membership is the struct used to describe Raft cluster membership query results.

type NodeHost

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

NodeHost manages Raft clusters and enables them to share resources such as transport and persistent storage etc. NodeHost is also the central thread safe access point for Dragonboat functionalities.

func NewNodeHost

func NewNodeHost(nhConfig config.NodeHostConfig) (*NodeHost, error)

NewNodeHost creates a new NodeHost instance. The returned NodeHost instance is configured using the specified NodeHostConfig instance. In a typical application, it is expected to have one NodeHost on each server.

func (*NodeHost) CloseSession deprecated

func (nh *NodeHost) CloseSession(ctx context.Context,
	session *client.Session) error

CloseSession closes the specified client session by unregistering it from the system. The specified context parameter must has the timeout value set. This is a synchronous method meaning it will only return after its confirmed completion, failure or timeout.

Closed client session should no longer be used in future proposals.

Deprecated: Use NodeHost.SyncCloseSession instead. NodeHost.CloseSession will be removed in v4.0.

func (*NodeHost) GetClusterMembership deprecated

func (nh *NodeHost) GetClusterMembership(ctx context.Context,
	clusterID uint64) (*Membership, error)

GetClusterMembership returns the membership information from the specified Raft cluster. The specified context parameter must has the timeout value set.

GetClusterMembership guarantees that the returned membership information is linearizable. This is a synchronous method meaning it will only return after its confirmed completion, failure or timeout.

Deprecated: Use NodeHost.SyncGetClusterMembership instead. NodeHost.GetClusterMembership will be removed in v4.0.

func (*NodeHost) GetLeaderID

func (nh *NodeHost) GetLeaderID(clusterID uint64) (uint64, bool, error)

GetLeaderID returns the leader node ID of the specified Raft cluster based on local node's knowledge. The returned boolean value indicates whether the leader information is available.

func (*NodeHost) GetNewSession deprecated

func (nh *NodeHost) GetNewSession(ctx context.Context,
	clusterID uint64) (*client.Session, error)

GetNewSession starts a synchronous proposal to create, register and return a new client session object for the specified Raft cluster. The specified context parameter must has the timeout value set.

A client session object is used to ensure that a retried proposal, e.g. proposal retried after timeout, will not be applied more than once into the IStateMachine.

Returned client session instance should not be used concurrently. Use multiple client sessions when making concurrent proposals.

Deprecated: Use NodeHost.SyncGetSession instead. NodeHost.GetNewSession will be removed in v4.0.

func (*NodeHost) GetNoOPSession

func (nh *NodeHost) GetNoOPSession(clusterID uint64) *client.Session

GetNoOPSession returns a NO-OP client session ready to be used for making proposals. The NO-OP client session is a dummy client session that will not be checked or enforced. Use this No-OP client session when you want to ignore features provided by client sessions. A NO-OP client session is not registered on the server side and thus not required to be closed at the end of its life cycle.

Returned NO-OP client session instance can be concurrently used in multiple goroutines.

Use this NO-OP client session when your IStateMachine provides idempotence in its own implementation.

NO-OP client session must be used for making proposals on IOnDiskStateMachine based state machine.

func (*NodeHost) GetNodeHostInfo

func (nh *NodeHost) GetNodeHostInfo(opt NodeHostInfoOption) *NodeHostInfo

GetNodeHostInfo returns a NodeHostInfo instance that contains all details of the NodeHost, this includes details of all Raft clusters managed by the the NodeHost instance.

func (*NodeHost) GetNodeUser

func (nh *NodeHost) GetNodeUser(clusterID uint64) (INodeUser, error)

GetNodeUser returns an INodeUser instance ready to be used to directly make proposals or read index operations without locating the node repeatedly in the NodeHost. A possible use case is when loading a large data set say with billions of proposals into the dragonboat based system.

func (*NodeHost) HasNodeInfo

func (nh *NodeHost) HasNodeInfo(clusterID uint64, nodeID uint64) bool

HasNodeInfo returns a boolean value indicating whether the specified node has been bootstrapped on the current NodeHost instance.

func (*NodeHost) ID added in v3.3.0

func (nh *NodeHost) ID() string

ID returns the string representation of the NodeHost ID value. The NodeHost ID is assigned to each NodeHost on its initial creation and it can be used to uniquely identify the NodeHost instance for its entire life cycle. When the system is running in the AddressByNodeHost mode, it is used as the target value when calling the StartCluster, RequestAddNode, RequestAddObserver, RequestAddWitness methods.

func (*NodeHost) NAReadLocalNode

func (nh *NodeHost) NAReadLocalNode(rs *RequestState,
	query []byte) ([]byte, error)

NAReadLocalNode is a variant of ReadLocalNode, it uses byte slice as its input and output data for read only queries to minimize extra heap allocations caused by using interface{}. Users are recommended to use ReadLocalNode unless performance is the top priority.

As an optional method, the underlying state machine must implement the statemachine.IExtended interface. NAReadLocalNode returns statemachine.ErrNotImplemented if the underlying state machine does not implement the statemachine.IExtended interface.

func (*NodeHost) NodeHostConfig

func (nh *NodeHost) NodeHostConfig() config.NodeHostConfig

NodeHostConfig returns the NodeHostConfig instance used for configuring this NodeHost instance.

func (*NodeHost) Propose

func (nh *NodeHost) Propose(session *client.Session, cmd []byte,
	timeout time.Duration) (*RequestState, error)

Propose starts an asynchronous proposal on the Raft cluster specified by the Session object. The input byte slice can be reused for other purposes immediate after the return of this method.

This method returns a RequestState instance or an error immediately. Application can wait on the ResultC() channel of the returned RequestState instance to get notified for the outcome of the proposal and access to the result of the proposal.

After the proposal is completed, i.e. RequestResult is received from the ResultC() channel of the returned RequestState, unless NO-OP client session is used, it is caller's responsibility to update the Session instance accordingly based on the RequestResult.Code value. Basically, when RequestTimeout is returned, you can retry the same proposal without updating your client session instance, when a RequestRejected value is returned, it usually means the session instance has been evicted from the server side, the Raft paper recommends you to crash your client in this highly unlikely event. When the proposal completed successfully with a RequestCompleted value, application must call client.ProposalCompleted() to get the client session ready to be used in future proposals.

func (*NodeHost) ProposeSession

func (nh *NodeHost) ProposeSession(session *client.Session,
	timeout time.Duration) (*RequestState, error)

ProposeSession starts an asynchronous proposal on the specified cluster for client session related operations. Depending on the state of the specified client session object, the supported operations are for registering or unregistering a client session. Application can select on the ResultC() channel of the returned RequestState instance to get notified for the completion (RequestResult.Completed() is true) of the operation.

func (*NodeHost) RaftAddress

func (nh *NodeHost) RaftAddress() string

RaftAddress returns the Raft address of the NodeHost instance, it is the network address by which the NodeHost can be reached by other NodeHost instances for exchanging Raft messages, snapshots and other metadata.

func (*NodeHost) ReadIndex

func (nh *NodeHost) ReadIndex(clusterID uint64,
	timeout time.Duration) (*RequestState, error)

ReadIndex starts the asynchronous ReadIndex protocol used for linearizable read on the specified cluster. This method returns a RequestState instance or an error immediately. Application should wait on the ResultC() channel of the returned RequestState object to get notified on the outcome of the ReadIndex operation. On a successful completion, the ReadLocal method can then be invoked to query the state of the IStateMachine or IOnDiskStateMachine to complete the read operation with linearizability guarantee.

func (*NodeHost) ReadLocalNode

func (nh *NodeHost) ReadLocalNode(rs *RequestState,
	query interface{}) (interface{}, error)

ReadLocalNode queries the Raft node identified by the input RequestState instance. To ensure the IO linearizability, ReadLocalNode should only be called after receiving a RequestCompleted notification from the ReadIndex method. See ReadIndex's example for more details.

func (*NodeHost) RemoveData

func (nh *NodeHost) RemoveData(clusterID uint64, nodeID uint64) error

RemoveData tries to remove all data associated with the specified node. This method should only be used after the node has been deleted from its Raft cluster. Calling RemoveData on a node that is still a Raft cluster member will corrupt the Raft cluster.

RemoveData returns ErrClusterNotStopped when the specified node has not been fully offloaded from the NodeHost instance.

func (*NodeHost) RequestAddNode

func (nh *NodeHost) RequestAddNode(clusterID uint64,
	nodeID uint64, target Target, configChangeIndex uint64,
	timeout time.Duration) (*RequestState, error)

RequestAddNode is a Raft cluster membership change method for requesting the specified node to be added to the specified Raft cluster. It starts an asynchronous request to add the node to the Raft cluster membership list. Application can wait on the ResultC() channel of the returned RequestState instance to get notified for the outcome.

If there is already an observer with the same nodeID in the cluster, it will be promoted to a regular node with voting power. The target parameter of the RequestAddNode call is ignored when promoting an observer to a regular node.

After the node is successfully added to the Raft cluster, it is application's responsibility to call StartCluster on the target NodeHost instance to actually start the Raft cluster node.

Requesting a removed node back to the Raft cluster will always be rejected.

By default, the target parameter is the RaftAddress of the NodeHost instance where the new Raft node will be running. Note that fixed IP or static DNS name should be used in RaftAddress in such default mode. When running in the AddressByNodeHostID mode, target should be set to NodeHost's ID value which can be obtained by calling the ID() method.

When the Raft cluster is created with the OrderedConfigChange config flag set as false, the configChangeIndex parameter is ignored. Otherwise, it should be set to the most recent Config Change Index value returned by the SyncGetClusterMembership method. The requested add node operation will be rejected if other membership change has been applied since that earlier call to the SyncGetClusterMembership method.

func (*NodeHost) RequestAddObserver

func (nh *NodeHost) RequestAddObserver(clusterID uint64,
	nodeID uint64, target Target, configChangeIndex uint64,
	timeout time.Duration) (*RequestState, error)

RequestAddObserver is a Raft cluster membership change method for requesting the specified node to be added to the specified Raft cluster as an observer without voting power. It starts an asynchronous request to add the specified node as an observer.

Such observer is able to receive replicated states from the leader node, but it is neither allowed to vote for leader, nor considered as a part of the quorum when replicating state. An observer can be promoted to a regular node with voting power by making a RequestAddNode call using its clusterID and nodeID values. An observer can be removed from the cluster by calling RequestDeleteNode with its clusterID and nodeID values.

Application should later call StartCluster with config.Config.IsObserver set to true on the right NodeHost to actually start the observer instance.

By default, the target parameter is the RaftAddress of the NodeHost instance where the new Raft node will be running. Note that fixed IP or static DNS name should be used in RaftAddress in such default mode. When running in the AddressByNodeHostID mode, target should be set to NodeHost's ID value which can be obtained by calling the ID() method.

When the Raft cluster is created with the OrderedConfigChange config flag set as false, the configChangeIndex parameter is ignored. Otherwise, it should be set to the most recent Config Change Index value returned by the SyncGetClusterMembership method. The requested add observer operation will be rejected if other membership change has been applied since that earlier call to the SyncGetClusterMembership method.

func (*NodeHost) RequestAddWitness added in v3.2.0

func (nh *NodeHost) RequestAddWitness(clusterID uint64,
	nodeID uint64, target Target, configChangeIndex uint64,
	timeout time.Duration) (*RequestState, error)

RequestAddWitness is a Raft cluster membership change method for requesting the specified node to be added as a witness to the given Raft cluster. It starts an asynchronous request to add the specified node as an witness.

A witness can vote in elections but it doesn't have any Raft log or application state machine associated. The witness node can not be used to initiate read, write or membership change operations on its Raft cluster. Section 11.7.2 of Diego Ongaro's thesis contains more info on such witness role.

Application should later call StartCluster with config.Config.IsWitness set to true on the right NodeHost to actually start the witness node.

By default, the target parameter is the RaftAddress of the NodeHost instance where the new Raft node will be running. Note that fixed IP or static DNS name should be used in RaftAddress in such default mode. When running in the AddressByNodeHostID mode, target should be set to NodeHost's ID value which can be obtained by calling the ID() method.

When the Raft cluster is created with the OrderedConfigChange config flag set as false, the configChangeIndex parameter is ignored. Otherwise, it should be set to the most recent Config Change Index value returned by the SyncGetClusterMembership method. The requested add witness operation will be rejected if other membership change has been applied since that earlier call to the SyncGetClusterMembership method.

func (*NodeHost) RequestCompaction added in v3.2.0

func (nh *NodeHost) RequestCompaction(clusterID uint64,
	nodeID uint64) (*SysOpState, error)

RequestCompaction requests a compaction operation to be asynchronously executed in the background to reclaim disk spaces used by Raft Log entries that have already been marked as removed. This includes Raft Log entries that have already been included in created snapshots and Raft Log entries that belong to nodes already permanently removed via NodeHost.RemoveData().

By default, compaction is automatically issued after each snapshot is captured. RequestCompaction can be used to manually trigger such compaction when auto compaction is disabled by the DisableAutoCompactions option in config.Config.

The returned *SysOpState instance can be used to get notified when the requested compaction is completed. ErrRejected is returned when there is nothing to be reclaimed.

func (*NodeHost) RequestDeleteNode

func (nh *NodeHost) RequestDeleteNode(clusterID uint64,
	nodeID uint64,
	configChangeIndex uint64, timeout time.Duration) (*RequestState, error)

RequestDeleteNode is a Raft cluster membership change method for requesting the specified node to be removed from the specified Raft cluster. It starts an asynchronous request to remove the node from the Raft cluster membership list. Application can wait on the ResultC() channel of the returned RequestState instance to get notified for the outcome.

It is not guaranteed that deleted node will automatically close itself and be removed from its managing NodeHost instance. It is application's responsibility to call RemoveCluster on the right NodeHost instance to actually have the cluster node removed from its managing NodeHost instance.

Once a node is successfully deleted from a Raft cluster, it will not be allowed to be added back to the cluster with the same node identity.

When the raft cluster is created with the OrderedConfigChange config flag set as false, the configChangeIndex parameter is ignored. Otherwise, it should be set to the most recent Config Change Index value returned by the SyncGetClusterMembership method. The requested delete node operation will be rejected if other membership change has been applied since that earlier call to the SyncGetClusterMembership method.

func (*NodeHost) RequestLeaderTransfer

func (nh *NodeHost) RequestLeaderTransfer(clusterID uint64,
	targetNodeID uint64) error

RequestLeaderTransfer makes a request to transfer the leadership of the specified Raft cluster to the target node identified by targetNodeID. It returns an error if the request fails to be started. There is no guarantee that such request can be fulfilled, i.e. the leadership transfer can still fail after a successful return of the RequestLeaderTransfer method.

func (*NodeHost) RequestSnapshot

func (nh *NodeHost) RequestSnapshot(clusterID uint64,
	opt SnapshotOption, timeout time.Duration) (*RequestState, error)

RequestSnapshot requests a snapshot to be created asynchronously for the specified cluster node. For each node, only one ongoing snapshot operation is allowed.

Users can use an option parameter to specify details of the requested snapshot. For example, when the input SnapshotOption's Exported field is True, a snapshot will be exported to the directory pointed by the ExportPath field of the SnapshotOption instance. Such an exported snapshot is not managed by the system and it is mainly used to repair the cluster when it permanently loses its majority quorum. See the ImportSnapshot method in the tools package for more details.

When the Exported field of the input SnapshotOption instance is set to false, snapshots created as the result of RequestSnapshot are managed by Dragonboat. Users are not suppose to move, copy, modify or delete the generated snapshot. Such requested snapshot will also trigger Raft log and snapshot compactions similar to automatic snapshotting. Users need to subsequently call RequestCompaction(), which can be far more I/O intensive, at suitable time to actually reclaim disk spaces used by Raft log entries and snapshot metadata records.

When a snapshot is requested on a node backed by an IOnDiskStateMachine, only the metadata portion of the state machine will be captured and saved. Requesting snapshots on IOnDiskStateMachine based nodes are typically used to trigger Raft log and snapshot compactions.

RequestSnapshot returns a RequestState instance or an error immediately. Applications can wait on the ResultC() channel of the returned RequestState instance to get notified for the outcome of the create snasphot operation. The RequestResult instance returned by the ResultC() channel tells the outcome of the snapshot operation, when successful, the SnapshotIndex method of the returned RequestResult instance reports the index of the created snapshot.

Requested snapshot operation will be rejected if there is already an existing snapshot in the system at the same Raft log index.

func (*NodeHost) StaleRead

func (nh *NodeHost) StaleRead(clusterID uint64,
	query interface{}) (interface{}, error)

StaleRead queries the specified Raft node directly without any linearizability guarantee.

Users are recommended to use the SyncRead method or a combination of the ReadIndex and ReadLocalNode method to achieve linearizable read.

func (*NodeHost) StartCluster

func (nh *NodeHost) StartCluster(initialMembers map[uint64]Target,
	join bool, create sm.CreateStateMachineFunc, cfg config.Config) error

StartCluster adds the specified Raft cluster node to the NodeHost and starts the node to make it ready for accepting incoming requests. The node to be started is backed by a regular state machine that implements the sm.IStateMachine interface.

The input parameter initialMembers is a map of node ID to node target for all Raft cluster's initial member nodes. By default, the target is the RaftAddress value of the NodeHost where the node will be running. When running in the AddressByNodeHostID mode, target should be set to the NodeHostID value of the NodeHost where the node will be running. See the godoc of NodeHost's ID method for the full definition of NodeHostID. For the same Raft cluster, the same initialMembers map should be specified when starting its initial member nodes on distributed NodeHost instances.

The join flag indicates whether the node is a new node joining an existing cluster. create is a factory function for creating the IStateMachine instance, cfg is the configuration instance that will be passed to the underlying Raft node object, the cluster ID and node ID of the involved node are specified in the ClusterID and NodeID fields of the provided cfg parameter.

Note that this method is not for changing the membership of the specified Raft cluster, it launches a node that is already a member of the Raft cluster.

As a summary, when -

  • starting a brand new Raft cluster, set join to false and specify all initial member node details in the initialMembers map.
  • joining a new node to an existing Raft cluster, set join to true and leave the initialMembers map empty. This requires the joining node to have already been added as a member node of the Raft cluster.
  • restarting an crashed or stopped node, set join to false and leave the initialMembers map to be empty. This applies to both initial member nodes and those joined later.

func (*NodeHost) StartConcurrentCluster

func (nh *NodeHost) StartConcurrentCluster(initialMembers map[uint64]Target,
	join bool, create sm.CreateConcurrentStateMachineFunc, cfg config.Config) error

StartConcurrentCluster is similar to the StartCluster method but it is used to start a Raft node backed by a concurrent state machine.

func (*NodeHost) StartOnDiskCluster

func (nh *NodeHost) StartOnDiskCluster(initialMembers map[uint64]Target,
	join bool, create sm.CreateOnDiskStateMachineFunc, cfg config.Config) error

StartOnDiskCluster is similar to the StartCluster method but it is used to start a Raft node backed by an IOnDiskStateMachine.

func (*NodeHost) Stop

func (nh *NodeHost) Stop()

Stop stops all Raft nodes managed by the NodeHost instance, it also closes all internal components such as the transport and LogDB modules.

func (*NodeHost) StopCluster

func (nh *NodeHost) StopCluster(clusterID uint64) error

StopCluster removes and stops the Raft node associated with the specified Raft cluster from the NodeHost. The node to be removed and stopped is identified by the clusterID value.

Note that this is not the membership change operation to remove the node from the Raft cluster.

func (*NodeHost) StopNode

func (nh *NodeHost) StopNode(clusterID uint64, nodeID uint64) error

StopNode removes the specified Raft cluster node from the NodeHost and stops that running Raft node.

Note that this is not the membership change operation to remove the node from the Raft cluster.

func (*NodeHost) SyncCloseSession

func (nh *NodeHost) SyncCloseSession(ctx context.Context,
	cs *client.Session) error

SyncCloseSession closes the specified client session by unregistering it from the system. The specified context parameter must has the timeout value set. This is a synchronous method meaning it will only return after its confirmed completion, failure or timeout.

Closed client session should no longer be used in future proposals.

func (*NodeHost) SyncGetClusterMembership

func (nh *NodeHost) SyncGetClusterMembership(ctx context.Context,
	clusterID uint64) (*Membership, error)

SyncGetClusterMembership is a rsynchronous method that queries the membership information from the specified Raft cluster. The specified context parameter must has the timeout value set.

SyncGetClusterMembership guarantees that the returned membership information is linearizable.

func (*NodeHost) SyncGetSession

func (nh *NodeHost) SyncGetSession(ctx context.Context,
	clusterID uint64) (*client.Session, error)

SyncGetSession starts a synchronous proposal to create, register and return a new client session object for the specified Raft cluster. The specified context parameter must has the timeout value set.

A client session object is used to ensure that a retried proposal, e.g. proposal retried after timeout, will not be applied more than once into the state machine.

Returned client session instance should not be used concurrently. Use multiple client sessions when you need to concurrently start multiple proposals.

Client session is not supported by IOnDiskStateMachine based state machine. NO-OP client session must be used for making proposals on IOnDiskStateMachine based state machine.

func (*NodeHost) SyncPropose

func (nh *NodeHost) SyncPropose(ctx context.Context,
	session *client.Session, cmd []byte) (sm.Result, error)

SyncPropose makes a synchronous proposal on the Raft cluster specified by the input client session object. The specified context parameter must has the timeout value set.

SyncPropose returns the result returned by IStateMachine or IOnDiskStateMachine's Update method, or the error encountered. The input byte slice can be reused for other purposes immediate after the return of this method.

After calling SyncPropose, unless NO-OP client session is used, it is caller's responsibility to update the client session instance accordingly based on SyncPropose's outcome. Basically, when a ErrTimeout error is returned, application can retry the same proposal without updating the client session instance. When ErrInvalidSession error is returned, it usually means the session instance has been evicted from the server side, the Raft paper recommends to crash the client in this highly unlikely event. When the proposal completed successfully, caller must call client.ProposalCompleted() to get it ready to be used in future proposals.

func (*NodeHost) SyncRead

func (nh *NodeHost) SyncRead(ctx context.Context, clusterID uint64,
	query interface{}) (interface{}, error)

SyncRead performs a synchronous linearizable read on the specified Raft cluster. The specified context parameter must has the timeout value set. The query byte slice specifies what to query, it will be passed to the Lookup method of the IStateMachine or IOnDiskStateMachine after the system determines that it is safe to perform the local read on IStateMachine or IOnDiskStateMachine. It returns the query result from the Lookup method or the error encountered.

func (*NodeHost) SyncRemoveData

func (nh *NodeHost) SyncRemoveData(ctx context.Context,
	clusterID uint64, nodeID uint64) error

SyncRemoveData is the synchronous variant of the RemoveData. It waits for the specified node to be fully offloaded or until the ctx instance is cancelled or timeout.

Similar to RemoveData, calling SyncRemoveData on a node that is still a Raft cluster member will corrupt the Raft cluster.

func (*NodeHost) SyncRequestAddNode

func (nh *NodeHost) SyncRequestAddNode(ctx context.Context,
	clusterID uint64, nodeID uint64,
	target string, configChangeIndex uint64) error

SyncRequestAddNode is the synchronous variant of the RequestAddNode method. See RequestAddNode for more details.

The input ctx must have its deadline set.

func (*NodeHost) SyncRequestAddObserver

func (nh *NodeHost) SyncRequestAddObserver(ctx context.Context,
	clusterID uint64, nodeID uint64,
	target string, configChangeIndex uint64) error

SyncRequestAddObserver is the synchronous variant of the RequestAddObserver method. See RequestAddObserver for more details.

The input ctx must have its deadline set.

func (*NodeHost) SyncRequestAddWitness added in v3.2.0

func (nh *NodeHost) SyncRequestAddWitness(ctx context.Context,
	clusterID uint64, nodeID uint64,
	target string, configChangeIndex uint64) error

SyncRequestAddWitness is the synchronous variant of the RequestAddWitness method. See RequestAddWitness for more details.

The input ctx must have its deadline set.

func (*NodeHost) SyncRequestDeleteNode

func (nh *NodeHost) SyncRequestDeleteNode(ctx context.Context,
	clusterID uint64, nodeID uint64, configChangeIndex uint64) error

SyncRequestDeleteNode is the synchronous variant of the RequestDeleteNode method. See RequestDeleteNode for more details.

The input ctx must have its deadline set.

func (*NodeHost) SyncRequestSnapshot

func (nh *NodeHost) SyncRequestSnapshot(ctx context.Context,
	clusterID uint64, opt SnapshotOption) (uint64, error)

SyncRequestSnapshot is the synchronous variant of the RequestSnapshot method. See RequestSnapshot for more details.

The input ctx must has deadline set.

SyncRequestSnapshot returns the index of the created snapshot or the error encountered.

type NodeHostInfo

type NodeHostInfo struct {
	// NodeHostID is the unique identifier of the NodeHost instance.
	NodeHostID string
	// RaftAddress is the public address of the NodeHost used for exchanging Raft
	// messages, snapshots and other metadata with other NodeHost instances.
	RaftAddress string
	// Gossip contains gossip service related information.
	Gossip GossipInfo
	// ClusterInfo is a list of all Raft clusters managed by the NodeHost
	ClusterInfoList []ClusterInfo
	// LogInfo is a list of raftio.NodeInfo values representing all Raft logs
	// stored on the NodeHost.
	LogInfo []raftio.NodeInfo
}

NodeHostInfo provides info about the NodeHost, including its managed Raft cluster nodes and available Raft logs saved in its local persistent storage.

type NodeHostInfoOption

type NodeHostInfoOption struct {
	// SkipLogInfo is the boolean flag indicating whether Raft Log info should be
	// skipped when querying the NodeHostInfo.
	SkipLogInfo bool
}

NodeHostInfoOption is the option type used when querying NodeHostInfo.

var DefaultNodeHostInfoOption NodeHostInfoOption

DefaultNodeHostInfoOption is the default NodeHostInfoOption value. It requests the GetNodeHostInfo method to return all supported info.

type RequestResult

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

RequestResult is the result struct returned for the request.

func (*RequestResult) Aborted added in v3.2.0

func (rr *RequestResult) Aborted() bool

Aborted returns a boolean value indicating the request is aborted.

func (*RequestResult) Committed added in v3.3.0

func (rr *RequestResult) Committed() bool

Committed returns a boolean value indicating whether the request has been committed by Raft.

func (*RequestResult) Completed

func (rr *RequestResult) Completed() bool

Completed returns a boolean value indicating whether the request completed successfully. For proposals, it means the proposal has been committed by the Raft cluster and applied on the local node. For ReadIndex operation, it means the cluster is now ready for a local read.

func (*RequestResult) Dropped added in v3.1.0

func (rr *RequestResult) Dropped() bool

Dropped returns a boolean flag indicating whether the request has been dropped as the leader is unavailable or not ready yet. Such dropped requests can usually be retried once the leader is ready.

func (*RequestResult) GetResult

func (rr *RequestResult) GetResult() sm.Result

GetResult returns the result value of the request. When making a proposal, the returned result is the value returned by the Update method of the IStateMachine instance.

func (*RequestResult) Rejected

func (rr *RequestResult) Rejected() bool

Rejected returns a boolean value indicating the request is rejected. For a proposal, it means that the used client session instance is not registered or it has been evicted on the server side. When requesting a client session to be registered, Rejected means the another client session with the same client ID has already been registered. When requesting a client session to be unregistered, Rejected means the specified client session is not found on the server side. For a membership change request, it means the request is out of order and thus not applied.

func (*RequestResult) SnapshotIndex

func (rr *RequestResult) SnapshotIndex() uint64

SnapshotIndex returns the index of the generated snapshot when the RequestResult is from a snapshot related request. Invoking this method on RequestResult instances not related to snapshots will cause panic.

func (*RequestResult) Terminated

func (rr *RequestResult) Terminated() bool

Terminated returns a boolean value indicating the request terminated due to the requested Raft cluster is being shut down.

func (*RequestResult) Timeout

func (rr *RequestResult) Timeout() bool

Timeout returns a boolean value indicating whether the request timed out.

type RequestResultCode

type RequestResultCode int

RequestResultCode is the result code returned to the client to indicate the outcome of the request.

func (RequestResultCode) String

func (c RequestResultCode) String() string

type RequestState

type RequestState struct {

	// CompletedC is a channel for delivering request result to users.
	//
	// Deprecated: CompletedC has been deprecated. Use ResultC() or AppliedC()
	// instead.
	CompletedC chan RequestResult
	// contains filtered or unexported fields
}

RequestState is the object used to provide request result to users.

func (*RequestState) AppliedC added in v3.3.0

func (r *RequestState) AppliedC() chan RequestResult

AppliedC returns a channel of RequestResult for delivering request result. The returned channel reports the final outcomes of proposals and config changes, the return value can be of one of the Completed(), Dropped(), Timeout(), Rejected(), Terminated() or Aborted() values.

Use ResultC() when the client wants to be notified when proposals or config changes are committed.

func (*RequestState) Release

func (r *RequestState) Release()

Release puts the RequestState instance back to an internal pool so it can be reused. Release is normally called after all RequestResult values have been received from the ResultC() channel.

func (*RequestState) ResultC added in v3.3.0

func (r *RequestState) ResultC() chan RequestResult

ResultC returns a channel of RequestResult for delivering request results to users. When NotifyCommit is not enabled, the behaviour of the returned channel is the same as the one returned by the AppliedC() method. When NotifyCommit is enabled, up to two RequestResult values can be received from the returned channel. For example, for a successfully proposal that is eventually committed and applied, the returned chan RequestResult will return a RequestResult value to indicate the proposal is committed first, it will be followed by another RequestResult value indicating the proposal has been applied into the state machine.

Use AppliedC() when your client don't need extra notification when proposals and config changes are committed.

type SnapshotOption

type SnapshotOption struct {
	// CompactionOverhead is the compaction overhead value to use for the request
	// snapshot operation when OverrideCompactionOverhead is true. This field is
	// ignored when exporting a snapshot, that is when Exported is true.
	CompactionOverhead uint64
	// ExportPath is the path where the exported snapshot should be stored, it
	// must point to an existing directory for which the current user has write
	// permission to it.
	ExportPath string
	// Exported is a boolean flag indicating whether the snapshot requested to
	// be generated should be exported. For an exported snapshot, it is users'
	// responsibility to manage the snapshot files. By default, a requested
	// snapshot is not considered as exported, such a regular snapshot is managed
	// the system.
	Exported bool
	// OverrideCompactionOverhead defines whether the requested snapshot operation
	// should override the compaction overhead setting specified in node's config.
	// This field is ignored by the system when exporting a snapshot.
	OverrideCompactionOverhead bool
}

SnapshotOption is the options users can specify when requesting a snapshot to be generated.

var DefaultSnapshotOption SnapshotOption

DefaultSnapshotOption is the default SnapshotOption value to use when requesting a snapshot to be generated by using NodeHost's RequestSnapshot method. DefaultSnapshotOption causes a regular snapshot to be generated and the generated snapshot is managed by the system.

type SysOpState added in v3.2.0

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

SysOpState is the object used to provide system maintenance operation result to users.

func (*SysOpState) CompletedC deprecated added in v3.2.0

func (o *SysOpState) CompletedC() <-chan struct{}

CompletedC returns a struct{} chan that is closed when the requested operation is completed.

Deprecated: CompletedC() has been deprecated. Use ResultC() instead.

func (*SysOpState) ResultC added in v3.3.0

func (o *SysOpState) ResultC() <-chan struct{}

ResultC returns a struct{} chan that is closed when the requested operation is completed.

type Target added in v3.3.0

type Target = string

Target is the type used to specify where a node is running. Target is remote NodeHost's RaftAddress value when NodeHostConfig.AddressByNodeHostID is not set. Target will use NodeHost's ID value when NodeHostConfig.AddressByNodeHostID is set.

Directories

Path Synopsis
Package client implements the client Session struct for identifying proposal clients and their progress.
Package client implements the client Session struct for identifying proposal clients and their progress.
Package config contains functions and types used for managing dragonboat's configurations.
Package config contains functions and types used for managing dragonboat's configurations.
internal
id
logdb
Package logdb implements the persistent log storage used by Dragonboat.
Package logdb implements the persistent log storage used by Dragonboat.
logdb/kv/rocksdb/gorocksdb
Package gorocksdb provides the ability to create and access RocksDB databases.
Package gorocksdb provides the ability to create and access RocksDB databases.
raft
Package raft is a distributed consensus package that implements the Raft protocol.
Package raft is a distributed consensus package that implements the Raft protocol.
rsm
Package rsm implements State Machines used in Dragonboat.
Package rsm implements State Machines used in Dragonboat.
settings
Package settings is used for managing internal parameters that can be set at compile time by expert level users.
Package settings is used for managing internal parameters that can be set at compile time by expert level users.
tests
Package tests contains various helper functions and modules used in tests.
Package tests contains various helper functions and modules used in tests.
transport
Package transport implements the transport component used for exchanging Raft messages between NodeHosts.
Package transport implements the transport component used for exchanging Raft messages between NodeHosts.
vfs
Package logger manages loggers used in dragonboat.
Package logger manages loggers used in dragonboat.
plugin
rocksdb
Package rocksdb provides factory functions for creating RocksDB based Log DB.
Package rocksdb provides factory functions for creating RocksDB based Log DB.
tee
Package raftio contains structs, interfaces and function definitions required to build custom persistent Raft log storage and transport modules.
Package raftio contains structs, interfaces and function definitions required to build custom persistent Raft log storage and transport modules.
Package statemachine contains the definitions of the IStateMachine and IOnDiskStateMachine interfaces to be implemented by application state machine types.
Package statemachine contains the definitions of the IStateMachine and IOnDiskStateMachine interfaces to be implemented by application state machine types.

Jump to

Keyboard shortcuts

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