Categorygithub.com/couchbase/cbgt
modulepackage
1.4.7
Repository: https://github.com/couchbase/cbgt.git
Documentation: pkg.go.dev

# README

cbgt

The cbgt project provides a golang library that helps manage distributed partitions (or data shards) across an elastic cluster of servers.

GoDoc Coverage Status

Documentation

  • REST API Reference - these REST API Reference docs come from cbft, which uses the cbgt library.

NOTE: This library initializes math's random seed (rand.Seed(time.Now().UTC().UnixNano())) for unique id generation.

# Packages

No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author

# Functions

AtomicCopyMetrics copies uint64 metrics from s to r (from source to result), and also applies an optional fn function to each metric.
This basic partition func first tries a direct lookup by partition string, else it tries the "" partition.
BlanceMap reconstructs a blance map from an existing plan.
BlancePartitionModel returns a blance library PartitionModel and model constraints based on an input index definition.
BlancePlanPIndexes invokes the blance library's generic PlanNextMap() algorithm to create a new pindex layout plan.
Functionally determine the delta of which feeds need creation and which should be shut down.
CalcMovingPartitionsCount attempts to compute the number of moving partitions during a rebalance, given few node count statistics of the cluster.
CalcNodesLayout computes information about the nodes based on the index definitions, node definitions, and the current plan.
Calculate node deltas (nodes added & nodes removed).
Functionally determine the delta of which pindexes need creation and which should be shut down on our local node (mgrUUID).
Split logical indexes into PIndexes and assign PIndexes to nodes.
CasePlanFrozen returns true if the plan for the indexDef is frozen, in which case it also populates endPlanPIndexes with a clone of the indexDef's plans from begPlanPIndexes, except for the updated Index Control Status (CanRead and CanWrite) from the latest index.
CBAuthHttpGet is a couchbase-specific http.Get(), for use in a cbauth'ed environment.
CBAuthHttpGetWithClient is a couchbase-specific http.Get() with *http.Client parameterisation, for use in a cbauth'ed environment.
CBAuthURL rewrites a URL with credentials, for use in a cbauth'ed environment.
CBPartitions parses a sourceParams for a couchbase data-source/feed.
CBPartitionSeqs returns a map keyed by partition/vbucket ID with values of each vbucket's UUID / high_seqno.
CBSourceUUIDLookUp fetches the sourceUUID for the provided sourceName.
CBStats returns a map of aggregated ("aggStats") and per-node stats ("nodesStats").
CBVBucketLookUp looks up the source vBucketID for a given document ID and index.
CfgGetClusterOptions returns the cluster level options.
Returns index definitions from a Cfg provider.
No description provided by the author
Retrieves node definitions from a Cfg provider.
Retrieves PlanPIndexes from a Cfg provider.
CfgGetVersion returns the Cfg version.
CfgNodeDefsKey returns the Cfg access key for a NodeDef kind.
CfgRemoveNodeDef removes a NodeDef with the given uuid from the Cfg.
CfgRemoveNodeDefForce removes a NodeDef with the given uuid from the Cfg ignoring the cas checks.
CfgSetClusterOptions sets the cluster level options.
Updates index definitions on a Cfg provider.
No description provided by the author
Updates node definitions on a Cfg provider.
Updates PlanPIndexes on a Cfg provider.
Returns true if a given version is modern enough to modify the Cfg.
No description provided by the author
No description provided by the author
ConsistencyWaitDone() waits for either the cancelCh or doneCh to finish, and provides the partition's seq if it was the cancelCh.
ConsistencyWaitGroup waits for all the partitions from a group of pindexes to reach a required consistency level.
ConsistencyWaitPartitions waits for the given partitions to reach the required consistency level.
ConsistencyWaitPIndex waits for all the partitions in a pindex to reach the required consistency level.
CopyPlanPIndexes returns a copy of the given planPIndexes, albeit with a new UUID and given version.
Copy plans iff their planner version is the current.
CouchbaseBucket is a helper function to connect to a couchbase bucket.
CouchbaseParseSourceName parses a sourceName, if it's a couchbase REST/HTTP URL, into a server URL, poolName and bucketName.
CouchbasePartitions parses a sourceParams for a couchbase data-source/feed.
CouchbasePartitionSeqs returns a map keyed by partition/vbucket ID with values of each vbucket's UUID / high_seqno.
CouchbaseSourceUUIDLookUp fetches the sourceUUID for the provided sourceName.
CouchbaseSourceVBucketLookUp looks up the source vBucketID for a given document ID and index.
CouchbaseStats returns a map of aggregated ("aggStats") and per-node stats ("nodesStats").
DataSourcePartitions is a helper function that returns the data source partitions for a named data source or feed type.
DataSourcePrepParams parses and validates the sourceParams, possibly transforming it.
No description provided by the author
DefaultMaxPartitionsPerPIndex retrieves "defaultMaxPartitionsPerPIndex" from the manager options, if available.
ErrorToString is a helper func that returns e.Error(), but also returns "" for nil error.
Calls f() in a loop, sleeping in an exponential backoff if needed.
FeedNameForPIndex functionally computes the name of a feed given a pindex.
No description provided by the author
FilesFeedPartitions returns the partitions, controlled by FilesFeedParams.NumPartitions, for a FilesFeed instance.
FilesFindMatches finds all leaf file paths in a subdirectory tree that match any in an optional array of regExps (regular expression strings).
FilesPathToPartition hashes a file path to a partition.
GetDirectorySize computes the size of given directory recursively.
No description provided by the author
GetIndexDef retrieves the IndexDef and PIndexImplType for an index.
No description provided by the author
GetNodePlanParam returns a relevant NodePlanParam for a given node from a nodePlanParams, defaulting to a less-specific NodePlanParam if needed.
Return nodes' UUIDs, weights and hierarchy.
Returns the current(i.e.
No description provided by the author
No description provided by the author
IndentJSON is a helper func that returns indented JSON for its interface{} x parameter.
IndexPartitionSettings returns the settings to be used for MaxPartitionsPerPIndex and IndexPartitions.
IsFeatureSupportedByCluster checks whether the given feature is supported across the cluster/given NodeDefs.
No description provided by the author
No description provided by the author
NewCfgCB returns a Cfg implementation that reads/writes its entries from/to a couchbase bucket, using DCP streams to subscribe to changes.
NewCfgCBEx is a more advanced version of NewCfgCB(), with more initialization options via the options map.
NewCfgMem returns an empty CfgMem instance.
NewCfgMetaKv returns a CfgMetaKv that reads and stores its single configuration file in the metakv.
NewCfgSimple returns a CfgSimple that reads and stores its single configuration file in the provided file path.
NewDCPFeed creates a new, ready-to-be-started DCP feed.
NewDCPFeedParams returns a DCPFeedParams initialized with default values.
NewDestStats creates a new, ready-to-use DestStats.
NewFilesFeed creates a ready-to-be-started FilesFeed.
Returns an intiialized IndexDefs.
No description provided by the author
NewManager returns a new, ready-to-be-started Manager instance.
NewManagerEx returns a new, ready-to-be-started Manager instance, with additional options.
NewMsgRing returns a MsgRing of a given ringSize.
NewNILFeed creates a ready-to-be-started NILFeed instance.
Returns an initialized NodeDefs.
Creates a pindex, including its backend implementation structures, and its files.
NewPIndexImpl creates an index partition of the given, registered index type.
NewPIndexImplEx creates an index partition of the given, registered index type.
No description provided by the author
No description provided by the author
Returns an initialized PlanPIndexes.
No description provided by the author
No description provided by the author
NewTAPFeed creates a new, ready-to-be-started TAPFeed.
No description provided by the author
No description provided by the author
A NoopPlannerHook is a no-op planner hook that just returns its input.
NormaliseNodeWeights updates the node weights reflective of the existing partition count on those nodes based on the given plans and update factor.
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
OpenPIndex reopens a previously created pindex.
OpenPIndexImpl loads an index partition of the given, registered index type from a given path.
OpenPIndexImplUsing loads an index partition of the given, registered index type from a given path with the given indexParams.
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
ParsePartitionsToVBucketIds is specific to couchbase data-sources/feeds, converting a set of partition strings from a dests map to vbucketId numbers.
Retrieves a pindex name from a pindex path.
PIndexImplTypeForIndex retrieves from the Cfg provider the index type for a given index.
Returns true if both the PIndex meets the PlanPIndex, ignoring UUID.
Computes the PIndex name from the storage path.
Computes the storage path for a pindex.
Plan runs the planner once.
PlannerCheckVersion errors if a version string is too low.
PlannerGetIndexDefs retrieves index definitions from a Cfg.
PlannerGetNodeDefs retrieves node definitions from a Cfg.
PlannerGetPlan retrieves plan related info from the Cfg.
PlannerGetPlanPIndexes retrieves the planned pindexes from a Cfg.
NOTE: PlanPIndex.Name must be unique across the cluster and ideally functionally based off of the indexDef so that the SamePlanPIndex() comparison works even if concurrent planners are racing to calculate plans.
PlanPIndexNodeCanRead returns true if PlanPIndexNode.CanRead is true; it's useful as a filter arg for Manager.CoveringPIndexes().
PlanPIndexNodeCanWrite returns true if PlanPIndexNode.CanWrite is true; it's useful as a filter arg for Manager.CoveringPIndexes().
PlanPIndexNodeOk always returns true; it's useful as a filter arg for Manager.CoveringPIndexes().
PrimaryFeedPartitions generates partition strings based on a PrimarySourceParams.NumPartitions parameter.
No description provided by the author
No description provided by the author
ReadableStackTrace tries to capture the caller stack frame for the calling function in a panic scenario.
No description provided by the author
No description provided by the author
RegisterFeedType is invoked at init/startup time to register a FeedType.
No description provided by the author
RegisterPIndexImplType registers a index type into the system.
RegisterSecurityNotifications registers for the cbauth's security callbacks.
No description provided by the author
No description provided by the author
Returns true if both PlanPIndex are the same, ignoring PlanPIndex.UUID.
Returns true if both PlanPIndexes are the same, where we ignore any differences in UUID, ImplVersion or PlannerVersion.
No description provided by the author
No description provided by the author
Split an IndexDef into 1 or more PlanPIndex'es, assigning data source partitions from the IndexDef to a PlanPIndex based on modulus of MaxPartitionsPerPIndex.
StartDCPFeed starts a DCP related feed and is registered at init/startup time with the system via RegisterFeedType().
StartFilesFeed starts a FilesFeed and is the the callback function registered at init/startup time.
No description provided by the author
No description provided by the author
StartDCPFeed starts a TAP related feed and is registered at init/startup time with the system via RegisterFeedType().
StringsIntersectStrings returns a brand new array that has the intersection of a and b.
StringsRemoveDuplicates removes any duplicate strings from the give slice.
StringsRemoveStrings returns a copy of stringArr, but with some strings removed, keeping the same order as stringArr.
StringsToMap connverts an array of (perhaps duplicated) strings into a map with key of those strings and values of true, and is useful for simple set-like operations.
StructChanges uses reflection to compare the fields of two structs, which must the same type, and returns info on the changes of field values.
Returns true if PlanPIndex children in a are a subset of those in b, using SamePlanPIndex() for sameness comparion.
Time invokes a func f and updates the totalDuration, totalCount and maxDuration metrics.
TimeoutCancelChan creates a channel that closes after a given timeout in milliseconds.
Timer updates a metrics.Timer.
UnregisterNodes removes the given nodes (by their UUID) from the nodes wanted & known cfg entries.
UnregisterNodesWithRetries removes the given nodes (by their UUID) from the nodes wanted & known cfg entries, and performs retries a max number of times if there were CAS conflict errors.
VBucketIdToPartitionDest is specific to couchbase data-sources/feeds, choosing the right Dest based on a vbucketId.
VerifyEffectiveClusterVersion checks the cluster version values, and if the cluster contains any node which is lower than the given myVersion, then return false.
Compares two dotted versioning strings, like "1.0.1" and "1.2.3".
WriteTimerJSON writes a metrics.Timer instance as JSON to a io.Writer.

# Constants

No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
the last permissible ID for an FTS system event; any additional type of event shouldn't exceed the below number.
DEST_EXTRAS_TYPE_DCP represents the extras that comes from DCP protocol.
DEST_EXTRAS_TYPE_GOCBCORE_DCP represents gocb DCP mutation/deletion metadata not included in DataUpdate/DataDelete (GocbcoreDCPExtras).
DEST_EXTRAS_TYPE_GOCBCORE_SCOPE_COLLECTION represents gocb DCP mutation/deletion scope id and collection id written to []byte of len=8 (4bytes each).
DEST_EXTRAS_TYPE_MCREQUEST represents the MCRequest from DCP protocol.
DEST_EXTRAS_TYPE_NIL means there are no extras as part of a Dest.DataUpdate/DataDelete invocation.
DEST_EXTRAS_TYPE_TAP represents the extras that comes from TAP protocol.
No description provided by the author
No description provided by the author
Default values for feed parameters.
FeedAllotmentOnePerPIndex specifies that there should be only a single feed per pindex.
FeedAllotmentOption is the manager option key used the specify how feeds should be alloted or assigned.
No description provided by the author
5 minutes.
No description provided by the author
No description provided by the author
INDEX_DEFS_KEY is the key used for Cfg access.
INDEX_NAME_REGEXP is used to validate index definition names.
3073.
3075.
3074.
No description provided by the author
No description provided by the author
No description provided by the author
Check pointing of the rebalance status To be updated by rebalance orchestrator To be consumed by all nodes to cache the status of last rebalance operation This is helpful in case of nodes dieing during rebalance (orchestrator and non-orchestrator).
No description provided by the author
MANAGER_MAX_EVENTS limits the number of events tracked by a Manager for diagnosis/debugging.
No description provided by the author
NODE_DEFS_KEY is used for Cfg access.
NODE_DEFS_KNOWN is used for Cfg access.
NODE_DEFS_WANTED is used for Cfg access.
No description provided by the author
PINDEXES_RESTART suggests a reboot of the pindexes.
No description provided by the author
PLAN_PINDEXES_KEY is used for Cfg access.
QUERY_CTL_DEFAULT_TIMEOUT_MS is the default query timeout.
Rebalance completed successfully (without any errors).
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
event_code block assigned to FTS events ranges from 3072 - 4095.
3076.
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
The cbgt.VERSION tracks persistence versioning (schema/format of persisted data and configuration).
No description provided by the author
No description provided by the author
No description provided by the author

# Variables

No description provided by the author
CfgAppVersion is a global Cfg variable supposed to be overridden by applications to indicate the current application version.
Prefix of paths stored in metakv, and should be immutable after process init()'ialization.
No description provided by the author
No description provided by the author
Function overrride to close a gocbcore.DCPAgent.
Frequency of type time.Duration to check the state of the cluster that the couchbase.Bucket instance is a part of.
DataSourceUUID is a helper function that fetches the sourceUUID for the sourceName.
DCPFeedBufferAckThreshold is representative of the percentage of the connection_buffer_size when the consumer will ack back to the producer.
DCPFeedBufferSizeBytes is representative of connection_buffer_size for DCP to enable flow control, defaults at 20MB.
DCPFeedPrefix should be immutable after process init()'ialization.
DCPNoopTimeIntervalSecs is representative of set_noop_interval for DCP to enable no-op messages, defaults at 2min.
DefaultCfgDebounceOffsetInMs represents the default value for the debounce interval for the config events.
Index Control Default Values Applications interested in updating these defaults must do so at init() time.
Index Control Default Values Applications interested in updating these defaults must do so at init() time.
DefaultNodeOffsetMultiplier represents the default value for of an offset multiplier for nodes.
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
Key is sourceType.
Function overrride to set up a gocbcore.DCPAgent servers: single URL or multiple URLs delimited by ';'.
This implementation of GetPoolsDefaultForBucket works with CBAUTH only; For all other authtypes, the application will have to override this function.
GocbcoreAgentSetupTimeout is the time alloted for completing setup of a gocbcore.Agent or a gocbcore.DCPAgent, two factors .
20MB (to match KV).
GocbcoreConnectTimeout and GocbcoreKVConnectTimeout are timeouts used by gocbcore to connect to the cluster manager and KV.
No description provided by the author
GocbcoreStatsTimeout is the time alloted to obtain a response from the server for a stats request.
No description provided by the author
No description provided by the author
No description provided by the author
Go's default is 30 secs.
Go's default is 30 secs.
Go's default is 1 secs.
Go's default is 90 secs.
Go's default is 100 (0 means no limit).
Go's default is 2.
Go's default is 10 secs.
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
Should only be overwritten during process init()'ialization.
No description provided by the author
MsgRingMaxSmallBufSize is the max pool size for reused buf's.
MsgRingMaxSmallBufSize is the cutoff point, in bytes, in which a msg ring categorizes a buf as small versus large for reuse.
NodeFeatureAdvMetaEncoding represents the feature flag for the advanced metadata encoding that comprises of two format changes.
NodeFeatureLeanPlan represents the feature flag for PlanPIndexes under extras.
PINDEX_STORE_MAX_ERRORS is the max number of errors that a PIndexStoreStats will track.
PIndexImplTypes is a global registry of pindex type backends or implementations.
PlannerHooks allows advanced applications to register callbacks into the planning computation, in order to adjust the planning outcome.
PlanPIndexFilters represent registered PlanPIndexFilter func's, and should only be modified during process init()'ialization.
15 min default.
No description provided by the author
A way for applications to hook into the janitor's rollback phases.
RootCAsProvider supports override capability when authType isn't "cbauth" and the application wants to use unique TLS config for feeds.
the StreamingEndpointListener must be set up by the importing package before setting up the manager instance to enable caching of bucket scope info.
TempPathPrefix indicates the prefix string applied to name a temp directory.
No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
Should only be overwritten during process init()'ialization.
No description provided by the author

# Structs

No description provided by the author
No description provided by the author
No description provided by the author
No description provided by the author
Implements both Dest and PIndexImpl interfaces.
No description provided by the author
No description provided by the author
No description provided by the author
CBAuthParams are common couchbase data-source/feed specific connection parameters that may be part of a sourceParams JSON.
CBAuthParamsSasl implements the cbdatasource.ServerCredProvider interface.
The error used on mismatches of CAS (compare and set/swap) values.
CfgCB is an implementation of Cfg that uses a couchbase bucket, and uses DCP to get change notifications.
See the Cfg.Subscribe() method.
CfgMem is a local-only, memory-only implementation of Cfg interface that's useful for development and testing.
CfgMemEntry is a CAS-Val pairing tracked by CfgMem.
No description provided by the author
No description provided by the author
CfgSimple is a local-only, persisted (in a single file) implementation of the Cfg interface that's useful for non-clustered, single-node instances for developers.
ClusterOptions stores the configurable cluster-level manager options.
No description provided by the author
ConfigAnalyzeRequest wraps up the various configuration parameters that the PIndexImplType implementations deals with.
ConsistencyParams represent the consistency requirements of a client's request.
A ConsistencyWaitReq represents a runtime consistency wait request for a partition.
CoveringPIndexes represents a non-overlapping, disjoint set of PIndexes that cover all the partitions of an index.
CoveringPIndexesSpec represent the arguments for computing the covering pindexes for an index.
No description provided by the author
A DCPFeed implements both Feed and cbdatasource.Receiver interfaces, and forwards any incoming cbdatasource.Receiver callbacks to the relevant, hooked-up Dest instances.
DCPFeedParams are DCP data-source/feed specific connection parameters that may be part of a sourceParams JSON and is a superset of CBAuthParams.
A DestForwarder implements the Dest interface by forwarding method calls to the Dest returned by a DestProvider.
DestStats holds the common stats or metrics for a Dest.
DiagHandler allows modules to provide their own additions in response to "diag" or diagnostic information requests.
Documentation is used for auto-generated documentation.
An ErrorConsistencyWait represents an error or timeout while waiting for a partition to reach some consistency requirements.
ErrorLocalPIndexHealth represents the unavailable pindexes and the corresponding error details which is discovered during the consistency checks.
A FeedType represents an immutable registration of a single feed type or data source type.
FileDoc represents the JSON for each file/document that will be emitted by a FilesFeed as a data source.
FilesFeed is a Feed interface implementation that that emits file contents from a local subdirectory tree.
FilesFeedParams represents the JSON expected as the sourceParams for a FilesFeed.
GocbcoreDCPExtras packages additional DCP mutation metadata for use by DataUpdateEx, DataDeleteEx.
A GocbcoreDCPFeed implements both Feed and gocb.StreamObserver interfaces, and forwards any incoming gocb.StreamObserver callbacks to the relevant, hooked-up Dest instances.
No description provided by the author
An IndexDef is a logical index definition.
An IndexDefEnveloped overrides IndexDef with Params and SourceParams fields that are enveloped JSON (JSON encoded as strings), for backwards compatibility.
An IndexDefNested overrides IndexDef with Params and SourceParams fields that are JSON nested objects instead of strings, for easier-to-use API.
An IndexDefs is zero or more index definitions.
IndexPrepParams can be used to override any of the unset index parameters.
No description provided by the author
LeanIndexPlanPIndexes deduplicate the IndexDef parts from PlanPIndexes.
LeanPlanPIndex represents the lean version of PlanPIndex.
LeanPlanPIndexes represents the deduplicated split version of planPIndexes grouped by indexName.
A Manager represents a runtime node in a cluster.
ManagerStats represents the stats/metrics tracked by a Manager instance.
A MsgRing wraps an io.Writer, and remembers a ring of previous writes to the io.Writer.
A NILFeed implements the Feed interface and never feeds any data to its Dest instances.
A NodeDef is a node definition.
A NodeDefs is comprised of zero or more node definitions.
A NodePlanParam defines whether a particular node can service a particular index definition.
No description provided by the author
No description provided by the author
A PIndex represents a partition of an index, or an "index partition".
Note that these callbacks are invoked within the manager's sync mutex context, it is the responsibility of the user to ensure that they do NOT reacquire the manager mutex or any api that does within the callbacks.
PIndexImplType defines the functions that every pindex implementation type must register on startup.
PIndexStoreStats provides some common stats/metrics and error tracking that some pindex type backends can reuse.
A PlannerHookInfo is the in/out information provided to PlannerHook callbacks.
A PlanParams holds input parameters to the planner, that control how the planner should split an index definition into one or more index partitions, and how the planner should assign those index partitions to nodes.
A PlanPIndex represents the plan for a particular index partition, including on what nodes that the index partition is assigned to.
A PlanPIndexEnveloped overrides PlanPIndex with IndexParams and SourceParams fields that are enveloped JSON (JSON encoded as strings), for backwards compatibility.
A PlanPIndexes is comprised of zero or more planPIndexes.
PlanPIndexesShared represents a PlanPIndexes that has been deduplicated into shared parts.
PlanPIndexIndexDef represents the shared, repeated index definition part of a PlanPIndex.
A PlanPIndexNested overrides PlanPIndex with IndexParams and SourceParams fields that are JSON nested objects instead of strings, for easier-to-use API.
A PlanPIndexNode represents the kind of service a node has been assigned to provide for an index partition.
PlanPIndexNodeRef represents an assignment of a pindex to a node.
PlanPIndexSourceDef represents the shared, repeated source definition part of a PlanPIndex.
A PrimaryFeed implements both the Feed and Dest interfaces, for chainability; and is also useful for testing.
PrimarySourceParams represents the JSON for the sourceParams for a primary feed.
QueryCtl defines the JSON parameters that control query execution and which are independent of any specific pindex type.
QueryCtlParams defines the JSON that includes the "ctl" part of a query request.
RemotePlanPIndex associations are returned by CoveringPIndexes().
SecurityContext let us register multiple tls config update callbacks and acts as a wrapper for handling config changes.
No description provided by the author
StopAfterSourceParams defines optional fields for the sourceParams that can stop the data source feed (i.e., index ingest) if the seqs per partition have been reached.
A TAPFeed implements the Feed interface and handles the TAP protocol to receive data from a couchbase data source.
TAPFeedParams represents the JSON of the sourceParams for a TAP feed.
TaskRequest represent a generic task request like "compact" or "encrypt" for partitions.
No description provided by the author
A UUIDSeq associates a UUID (such as from a partition's UUID) with a seq number, with an optional source's sequence number.
No description provided by the author
A wrapper over the HTTP client.

# Interfaces

Cfg is the interface that configuration providers must implement.
ConsistencyWaiter interface represents a service that can wait for consistency.
Dest interface defines the data sink or destination for data that cames from a data-source.
DestCollection interface needs to be implemented by the dest/pindex implementations which consumes data from the collections.
DestEx interface defines the data sink or destination for data that comes from a data-source for any generic implementations.
A DestProvider returns the Dest to use for different kinds of operations and is used in conjunction with a DestForwarder.
A Feed interface represents an abstract data source.
No description provided by the author
The FeedEx interface will be used to represent extended functionality for a DCP Feed.
No description provided by the author
ManagerEventHandlers represents the callback interface where an application can receive important event callbacks from a Manager.
PIndexImpl represents a runtime pindex implementation instance, whose runtime type depends on the pindex's type.
TaskRequestHandler represents the interface that need to implemented by the partitions for using the task scatter gatherer.
VersionReader is an interface to be implemented by the configuration providers who supports the verification of homogeneousness of the cluster before performing certain Key/Values updates related to the cluster status.

# Type aliases

ConfigRefreshNotifier defines the SecuritySetting's refresh callback signature.
Key is partition or partition/partitionUUID.
A CwrQueue is a consistency wait request queue, implementing the heap.Interface for ConsistencyWaitReq's, and is heap ordered by sequence number.
DestExtrasType represents the encoding for the Dest.DataUpdate/DataDelete() extras parameter.
A DestPartitionFunc allows a level of indirection/abstraction for the Feed-to-Dest relationship.
Performs a lookup of a source partition given a document id.
Returns the current partitions and their seq's.
Each Feed or data-source type knows of the data partitions for a data source.
Returns the sourceUUID for a data source.
A FeedStartFunc is part of a FeedType registration as is invoked by a Manager when a new feed instance needs to be started.
Returns the current stats from a data source, if available, where the result is dependent on the data source / feed type.
GetAllMetaKvChildrenFunc defines the callback func signature that's invoked during supported feature check in a cluster.
Values for LAST_REBALANCE_STATUS_KEY.
PartitionErrMap tracks errors with the name of the partition where it occurred.
PlannerEventHandlerCallback is an optional event callback for an external planner that wish to receive direct notifications of custom cfg events (bypassing the metakv/cfg event subscription model) directly from the local manager instance.
A PlannerFilter callback func should return true if the plans for an indexDef should be updated during CalcPlan(), and should return false if the plans for the indexDef should be remain untouched.
A PlannerHook is an optional callback func supplied by the application via PlannerHooks and is invoked during planning.
PlanPIndexFilter is used to filter out nodes being considered by CoveringPIndexes().
PlanPIndexNodeRefs represents assignments of pindexes to nodes.
ResultCode represents the return code indicative of the various operations recommended by the pindex implementations upon detecting a config change.
No description provided by the author
TaskPartitionStatusMap tracks the current state of a task across the partitions.
No description provided by the author