tugboat

package module
v0.0.0-...-93a8726 Latest Latest
Warning

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

Go to latest
Published: Jan 19, 2022 License: MIT Imports: 35 Imported by: 0

README

tugboat

Tugboat is a hard fork of the Dragonboat project. Dragonboat is an awesome library and it is recommended to use instead of the Tugboat. That being said there are a few notable differences.

Project status

Project is still in very early stages of development going through some major API changes. It is not recommended for a production use.

Differences from Dragonboat

  • Simplified and removed some features
  • Modularised (clean separation of logdb and transport component)
  • Reduced the number of dependencies in the core package
  • All the configuration is coming through the code (not from config files)
  • Built for Go > 1.18 (Some APIs generified)

Documentation

Overview

Package tugboat is a feature complete and highly optimized multi-group Raft implementation for providing consensus in distributed systems.

The NodeHost struct is the facade interface for all features provided by the tugboat package. Each NodeHost instance usually runs on a separate server managing CPU, storage and network resources used for achieving consensus. Each NodeHost manages Raft nodes from different Raft groups known as Raft clusters. Each Raft cluster is identified by its ClusterID, it usually consists of multiple nodes (also known as replicas) each identified by a NodeID value. Nodes from the same Raft cluster suppose to be distributed on different NodeHost instances across the network, this brings fault tolerance for 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 servers) are accessible.

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

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

Tugboat guarantees the linearizability of your I/O when interacting with the IStateMachine or IOnDiskStateMachine instances. In plain English, writes (via making proposals) to your Raft cluster appears to be instantaneous, once a write is completed, all later reads (via linearizable read based on Raft's ReadIndex protocol) 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. For a client, when it retries the proposal that failed to complete by its deadline, it faces the risk of having the same proposal committed and applied twice into the user state machine. Tugboat prevents this by implementing the client session concept described in Diego Ongaro's PhD thesis.

Index

Constants

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

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")
	// 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.
	//
	// Deprecated: ErrBadKey is no longer used.
	ErrBadKey = errors.New("bad key try again later")
	// ErrPendingLeaderTransferExist has been depredicated, use ErrSystemBusy.
	//
	// Deprecated: ErrPendingLeaderTransferExist is deprecated.
	ErrPendingLeaderTransferExist = ErrSystemBusy
	// ErrPendingConfigChangeExist has been deprecicated, use ErrSystemBusy.
	//
	// Deprecated: ErrPendingConfigChangeExist is deprecated.
	ErrPendingConfigChangeExist = ErrSystemBusy
	// ErrPendingSnapshotRequestExist has been deprecated, use ErrSystemBusy.
	//
	// Deprecated: ErrPendingSnapshotRequestExist is deprecated.
	ErrPendingSnapshotRequestExist = ErrSystemBusy
)
View Source
var (
	// ErrNoSnapshot is the error used to indicate that there is no snapshot
	// available.
	ErrNoSnapshot = errors.New("no snapshot available")
)
View Source
var ErrUnknownTarget = errors.New("target address unknown")

ErrUnknownTarget is the error returned when the target address of the node is unknown.

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.

Types

type ClusterInfo

type ClusterInfo struct {
	// Nodes is a map of member node IDs to their Raft addresses.
	Nodes map[uint64]string
	// ClusterID is the cluster ID of the Raft cluster node.
	ClusterID uint64
	// NodeID is the node ID of the Raft cluster node.
	NodeID uint64
	// 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
	// IsNonVoting indicates whether this is a non-voting nonVoting node.
	IsNonVoting 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 INodeRegistry

type INodeRegistry interface {
	Close() error
	Add(clusterID uint64, nodeID uint64, url string)
	Remove(clusterID uint64, nodeID uint64)
	RemoveCluster(clusterID uint64)
	Resolve(clusterID uint64, nodeID uint64) (string, string, error)
}

INodeRegistry is the local registry interface used to keep all known nodes in the system.

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 ITransport

type ITransport interface {
	Name() string
	Send(pb.Message) bool
	SendSnapshot(pb.Message) bool
	GetStreamSink(clusterID uint64, nodeID uint64) pb.IChunkSink
	Close() error
}

ITransport is the interface of the transport layer used for exchanging Raft messages.

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
	// NonVotings is a map of NodeID values to NodeHost Raft addresses for all
	// nonVotings in the Raft cluster.
	NonVotings 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.

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 accessing Dragonboat functionalities.

func NewNodeHost

func NewNodeHost[T raftio.ITransport, L raftio.ILogDB](nhConfig config.NodeHostConfig,
	transportFactory func(requestHandler raftio.MessageHandler, chunkHandler raftio.ChunkHandler) T,
	logdbFactory func(logdb.LogDBCallback, string, string) L) (*NodeHost, error)

NewNodeHost creates a new NodeHost instance. In a typical application, it is expected to have one NodeHost on each server.

func (*NodeHost) Close

func (nh *NodeHost) Close()

Close stops all managed Raft nodes and releases all resources owned by the NodeHost instance.

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.

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.

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.

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 user state machines.

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

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, RequestAddNonVoting, RequestAddWitness methods.

func (*NodeHost) LogDB

func (nh *NodeHost) LogDB() raftio.ILogDB

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. User can wait on the ResultC() channel of the returned RequestState instance to get notified for the outcome 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. 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 as there are too many ongoing client sessions, the Raft paper recommends users to crash the client in such 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 ReadLocalNode method can then be invoked to query the state of the IStateMachine or IOnDiskStateMachine 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. ReadLocalNode is only allowed to be called after receiving a RequestCompleted notification from the ReadIndex method.

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 nonVoting 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 nonVoting 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) RequestAddNonVoting

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

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

Such nonVoting 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 nonVoting can be promoted to a regular node with voting power by making a RequestAddNode call using its clusterID and nodeID values. An nonVoting can be removed from the cluster by calling RequestDeleteNode with its clusterID and nodeID values.

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

See the godoc of the RequestAddNode method for the details of the target and configChangeIndex parameters.

func (*NodeHost) RequestAddObserver deprecated

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.

Deprecated: use RequestAddNonVoting instead.

func (*NodeHost) RequestAddWitness

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.

See the godoc of the RequestAddNode method for the details of the target and configChangeIndex parameters.

func (*NodeHost) RequestCompaction

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 StopCluster 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.

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.

Each 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.

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.

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 deprecated

func (nh *NodeHost) Stop()

Stop closes and releases all resources owned by the NodeHost instance including Raft nodes managed by the NodeHost.

Deprecated: Use Close instead.

func (*NodeHost) StopCluster

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

StopCluster stops the Raft node associated with the specified Raft cluster.

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

func (*NodeHost) StopNode

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

StopNode stops the specified Raft node.

Note that this is not the membership change operation required 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 in a synchronous manner. The specified context parameter must has the timeout value set.

Closed client session should not 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.

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 is not thread safe.

Client session is not supported by IOnDiskStateMachine based user state machines. NO-OP client session must be used on IOnDiskStateMachine based state machines.

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 interface{} 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. 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 context object 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 context object must have its deadline set.

func (*NodeHost) SyncRequestAddNonVoting

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

SyncRequestAddNonVoting is the synchronous variant of the RequestAddNonVoting method. See RequestAddNonVoting for more details.

The input context object must have its deadline set.

func (*NodeHost) SyncRequestAddObserver deprecated

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

SyncRequestAddObserver is the synchronous variant of the RequestAddObserver.

Deprecated: use SyncRequestAddNonVoting instead.

func (*NodeHost) SyncRequestAddWitness

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 context object 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 context object 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 context object must has deadline set.

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

func (*NodeHost) Transport

func (nh *NodeHost) Transport() ITransport

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
	// 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 Registry

type Registry struct {
	Addr sync.Map // map of raftio.NodeInfo => string
	// contains filtered or unexported fields
}

Registry is used to manage all known node addresses in the multi raft system. The transport layer uses this address registry to locate nodes.

func NewNodeRegistry

func NewNodeRegistry(streamConnections uint64, v config.TargetValidator) *Registry

NewNodeRegistry returns a new Registry object.

func (*Registry) Add

func (n *Registry) Add(clusterID uint64, nodeID uint64, target string)

Add adds the specified node and its target info to the registry.

func (*Registry) Close

func (n *Registry) Close() error

Close closes the node registry.

func (*Registry) Remove

func (n *Registry) Remove(clusterID uint64, nodeID uint64)

Remove removes a remote from the node registry.

func (*Registry) RemoveCluster

func (n *Registry) RemoveCluster(clusterID uint64)

RemoveCluster removes all nodes info associated with the specified cluster.

func (*Registry) Resolve

func (n *Registry) Resolve(clusterID uint64, nodeID uint64) (string, string, error)

Resolve looks up the Addr of the specified node.

type RequestResult

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

RequestResult is the result struct returned for the request.

func (*RequestResult) Aborted

func (rr *RequestResult) Aborted() bool

Aborted returns a boolean value indicating the request is aborted.

func (*RequestResult) Committed

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

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

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

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 {
	// 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.
	ExportPath string
	// CompactionOverhead is the compaction overhead value to use for the
	// requested snapshot operation when OverrideCompactionOverhead is set to
	// true. This field is ignored when exporting a snapshot.
	CompactionOverhead uint64
	// Exported is a boolean flag indicating whether to export the requested
	// snapshot. For an exported snapshot, users are responsible for managing the
	// snapshot files. An exported snapshot is usually used to repair the cluster
	// when it permanently loses its majority quorum. See the ImportSnapshot method
	// in the tools package for more details.
	Exported bool
	// OverrideCompactionOverhead defines whether the requested snapshot operation
	// should override the compaction overhead setting specified in node's config.
	// This field is ignored when exporting a snapshot.
	OverrideCompactionOverhead bool
}

SnapshotOption is the options supported 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. This default option causes a regular snapshot to be generated.

type SysOpState

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

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

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

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

type Target

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
Code generated by protoc-gen-gogo.
Code generated by protoc-gen-gogo.
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
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.
tests
Package tests contains various helper functions and modules used in tests.
Package tests contains various helper functions and modules used in tests.
vfs
Package logger manages loggers used in Tugboat.
Package logger manages loggers used in Tugboat.
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 rsm implements State Machines used in Dragonboat.
Package rsm implements State Machines used in Dragonboat.
Package statemachine contains the definitions of the IStateMachine and IOnDiskStateMachine interfaces for supporting the replicated state machine approach.
Package statemachine contains the definitions of the IStateMachine and IOnDiskStateMachine interfaces for supporting the replicated state machine approach.
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.

Jump to

Keyboard shortcuts

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