Documentation
¶
Overview ¶
Package resources contains integration test resources for spinning up M3 components.
Index ¶
- Constants
- func NewLogger() (*zap.Logger, error)
- func Retry(op func() error) error
- func RetryWithMaxTime(op func() error, maxTime time.Duration) error
- func SetupCluster(cluster M3Resources, opts ClusterOptions) error
- func SetupM3MsgTopics(coord Coordinator, aggInstanceInfo InstanceInfo, opts ClusterOptions) error
- func SetupPlacement(coordAPI Coordinator, coordHost InstanceInfo, aggs Aggregators, ...) error
- func ZapMethod(s string) zapcore.Field
- type Admin
- type Aggregator
- type AggregatorClusterOptions
- type Aggregators
- type ClusterOptions
- type Coordinator
- type CoordinatorClient
- func (c *CoordinatorClient) AddM3msgTopicConsumer(topicOpts M3msgTopicOptions, addRequest admin.TopicAddRequest) (admin.TopicGetResponse, error)
- func (c *CoordinatorClient) AddNamespace(addRequest admin.NamespaceAddRequest) (admin.NamespaceGetResponse, error)
- func (c *CoordinatorClient) ApplyKVUpdate(update string) error
- func (c *CoordinatorClient) CreateDatabase(addRequest admin.DatabaseCreateRequest) (admin.DatabaseCreateResponse, error)
- func (c *CoordinatorClient) DeleteAllPlacements(opts PlacementRequestOptions) error
- func (c *CoordinatorClient) DeleteNamespace(namespaceID string) error
- func (c *CoordinatorClient) GetM3msgTopic(topicOpts M3msgTopicOptions) (admin.TopicGetResponse, error)
- func (c *CoordinatorClient) GetNamespace() (admin.NamespaceGetResponse, error)
- func (c *CoordinatorClient) GetPlacement(opts PlacementRequestOptions) (admin.PlacementGetResponse, error)
- func (c *CoordinatorClient) GraphiteQuery(graphiteReq GraphiteQueryRequest) ([]Datapoint, error)
- func (c *CoordinatorClient) InitM3msgTopic(topicOpts M3msgTopicOptions, initRequest admin.TopicInitRequest) (admin.TopicGetResponse, error)
- func (c *CoordinatorClient) InitPlacement(opts PlacementRequestOptions, initRequest admin.PlacementInitRequest) (admin.PlacementGetResponse, error)
- func (c *CoordinatorClient) InstantQuery(req QueryRequest, headers Headers) (model.Vector, error)
- func (c *CoordinatorClient) InstantQueryWithEngine(req QueryRequest, engine options.QueryEngine, headers Headers) (model.Vector, error)
- func (c *CoordinatorClient) LabelNames(req LabelNamesRequest, headers Headers) (model.LabelNames, error)
- func (c *CoordinatorClient) LabelValues(req LabelValuesRequest, headers Headers) (model.LabelValues, error)
- func (c *CoordinatorClient) RangeQuery(req RangeQueryRequest, headers Headers) (model.Matrix, error)
- func (c *CoordinatorClient) RangeQueryWithEngine(req RangeQueryRequest, engine options.QueryEngine, headers Headers) (model.Matrix, error)
- func (c *CoordinatorClient) RunQuery(verifier ResponseVerifier, query string, headers map[string][]string) error
- func (c *CoordinatorClient) Series(req SeriesRequest, headers Headers) ([]model.Metric, error)
- func (c *CoordinatorClient) UpdateNamespace(req admin.NamespaceUpdateRequest) (admin.NamespaceGetResponse, error)
- func (c *CoordinatorClient) WaitForClusterReady() error
- func (c *CoordinatorClient) WaitForInstances(ids []string) error
- func (c *CoordinatorClient) WaitForNamespace(name string) error
- func (c *CoordinatorClient) WaitForShardsReady() error
- func (c *CoordinatorClient) WriteCarbon(url string, metric string, v float64, t time.Time) error
- func (c *CoordinatorClient) WriteProm(name string, tags map[string]string, samples []prompb.Sample, headers Headers) error
- func (c *CoordinatorClient) WritePromWithRequest(writeRequest prompb.WriteRequest, headers Headers) error
- type CoordinatorClientOptions
- type CoordinatorClusterOptions
- type DBNodeClusterOptions
- type Datapoint
- type ExternalResources
- type GoalStateVerifier
- type GraphiteQueryRequest
- type Headers
- type InstanceInfo
- type LabelNamesRequest
- type LabelValuesRequest
- type M3Resources
- type M3msgTopicOptions
- type MetadataRequest
- type Node
- type Nodes
- type PlacementRequestOptions
- type QueryRequest
- type RangeQueryRequest
- type ResponseVerifier
- type RetryFunc
- type SeriesRequest
- type ServiceType
Constants ¶
const ( // AggName is the name of the aggregated namespace. AggName = "aggregated" // UnaggName is the name of the unaggregated namespace. UnaggName = "default" // ColdWriteNsName is the name for cold write namespace. ColdWriteNsName = "coldWritesRepairAndNoIndex" // AggregatorInputTopic is the m3msg topic name for coordinator->aggregator traffic. AggregatorInputTopic = "aggregator_ingest" // AggregatorOutputTopic is the m3msg topic name for aggregator->coordinator traffic. AggregatorOutputTopic = "aggregated_metrics" )
Variables ¶
This section is empty.
Functions ¶
func NewLogger ¶ added in v1.4.0
NewLogger creates a new development zap logger without stacktraces to cut down on verbosity.
func Retry ¶ added in v1.4.0
Retry is a function for retrying an operation in integration tests. Exponentially backs off between retries with a max wait of 5 seconds Waits up to a minute total for an operation to complete.
func RetryWithMaxTime ¶ added in v1.4.2
RetryWithMaxTime is a function for retrying an operation in integration tests. Exponentially backs off between retries with a max wait of 5 seconds Waits up to the max time provided to complete.
func SetupCluster ¶
func SetupCluster( cluster M3Resources, opts ClusterOptions, ) error
SetupCluster setups m3 cluster on provided docker containers.
func SetupM3MsgTopics ¶ added in v1.4.2
func SetupM3MsgTopics( coord Coordinator, aggInstanceInfo InstanceInfo, opts ClusterOptions, ) error
SetupM3MsgTopics sets up the m3msg topics for the provided coordinator and aggregator.
func SetupPlacement ¶ added in v1.4.2
func SetupPlacement( coordAPI Coordinator, coordHost InstanceInfo, aggs Aggregators, opts AggregatorClusterOptions, ) error
SetupPlacement configures the placement for the provided coordinators and aggregators.
Types ¶
type Admin ¶
type Admin interface {
// GetNamespace gets namespaces.
GetNamespace() (admin.NamespaceGetResponse, error)
// WaitForNamespace blocks until the given namespace is enabled.
// NB: if the name string is empty, this will instead
// check for a successful response.
WaitForNamespace(name string) error
// AddNamespace adds a namespace.
AddNamespace(admin.NamespaceAddRequest) (admin.NamespaceGetResponse, error)
// UpdateNamespace updates the namespace.
UpdateNamespace(admin.NamespaceUpdateRequest) (admin.NamespaceGetResponse, error)
// DeleteNamespace removes the namespace.
DeleteNamespace(namespaceID string) error
// CreateDatabase creates a database.
CreateDatabase(admin.DatabaseCreateRequest) (admin.DatabaseCreateResponse, error)
// GetPlacement gets placements.
GetPlacement(PlacementRequestOptions) (admin.PlacementGetResponse, error)
// InitPlacement initializes placements.
InitPlacement(PlacementRequestOptions, admin.PlacementInitRequest) (admin.PlacementGetResponse, error)
// DeleteAllPlacements deletes all placements for the service specified
// in the PlacementRequestOptions.
DeleteAllPlacements(PlacementRequestOptions) error
// WaitForInstances blocks until the given instance is available.
WaitForInstances(ids []string) error
// WaitForShardsReady waits until all shards gets ready.
WaitForShardsReady() error
// InitM3msgTopic initializes an m3msg topic.
InitM3msgTopic(M3msgTopicOptions, admin.TopicInitRequest) (admin.TopicGetResponse, error)
// GetM3msgTopic gets an m3msg topic.
GetM3msgTopic(M3msgTopicOptions) (admin.TopicGetResponse, error)
// AddM3msgTopicConsumer adds a consumer service to an m3msg topic.
AddM3msgTopicConsumer(M3msgTopicOptions, admin.TopicAddRequest) (admin.TopicGetResponse, error)
// WaitForClusterReady waits until the cluster is ready to receive reads and writes.
WaitForClusterReady() error
// Close closes the wrapper and releases any held resources, including
// deleting docker containers.
Close() error
}
Admin is a wrapper for admin functions.
type Aggregator ¶ added in v1.3.0
type Aggregator interface {
// Start starts the aggregator instance.
Start()
// HostDetails returns this aggregator instance's host details.
HostDetails() (*InstanceInfo, error)
// IsHealthy determines whether an instance is healthy.
IsHealthy() error
// Status returns the instance status.
Status() (aggregator.RuntimeStatus, error)
// Resign asks an aggregator instance to give up its current leader role if applicable.
Resign() error
// Close closes the wrapper and releases any held resources, including
// deleting docker containers.
Close() error
}
Aggregator is an aggregator instance.
type AggregatorClusterOptions ¶ added in v1.4.0
type AggregatorClusterOptions struct {
// RF is the replication factor to use for aggregators.
// It should be 1 for non-replicated mode and 2 for leader-follower mode.
RF int32
// NumShards is the number of shards to use for each RF.
NumShards int32
// NumInstances is the number of aggregator instances in total.
NumInstances int32
// NumIsolationGroups is the number of isolation groups to split
// aggregators into.
NumIsolationGroups int32
}
AggregatorClusterOptions contains the cluster options for spinning up aggregators.
func NewAggregatorClusterOptions ¶ added in v1.4.0
func NewAggregatorClusterOptions() *AggregatorClusterOptions
NewAggregatorClusterOptions creates AggregatorClusterOptions with sane defaults. Aggregator config must still be provided.
func (*AggregatorClusterOptions) Validate ¶ added in v1.4.0
func (a *AggregatorClusterOptions) Validate() error
Validate validates the AggregatorClusterOptions.
type Aggregators ¶ added in v1.4.0
type Aggregators []Aggregator
Aggregators is a slice of aggregators.
func (Aggregators) WaitForHealthy ¶ added in v1.4.0
func (a Aggregators) WaitForHealthy() error
WaitForHealthy waits for each Aggregator in Aggregators to be healthy
type ClusterOptions ¶
type ClusterOptions struct {
// DBNode contains cluster options for spinning up dbnodes.
DBNode *DBNodeClusterOptions
// Aggregator is the optional cluster options for spinning up aggregators.
// If Aggregator is nil, the cluster contains only m3coordinator and dbnodes.
Aggregator *AggregatorClusterOptions
// Coordinator is the options for spinning up the coordinator
Coordinator CoordinatorClusterOptions
}
ClusterOptions contains options for spinning up a new M3 cluster composed of in-process components.
func (*ClusterOptions) Validate ¶ added in v1.4.0
func (opts *ClusterOptions) Validate() error
Validate validates the ClusterOptions.
type Coordinator ¶
type Coordinator interface {
Admin
// Start starts the coordinator instance.
Start()
// HostDetails returns this coordinator instance's host details.
HostDetails() (*InstanceInfo, error)
// ApplyKVUpdate applies a KV update.
ApplyKVUpdate(update string) error
// WriteCarbon writes a carbon metric datapoint at a given time.
WriteCarbon(port int, metric string, v float64, t time.Time) error
// WriteProm writes a prometheus metric. Takes tags/labels as a map for convenience.
WriteProm(name string, tags map[string]string, samples []prompb.Sample, headers Headers) error
// WritePromWithRequest executes a prometheus write request. Allows you to
// provide the request directly which is useful for batch metric requests.
WritePromWithRequest(writeRequest prompb.WriteRequest, headers Headers) error
// RunQuery runs the given query with a given verification function.
RunQuery(verifier ResponseVerifier, query string, headers Headers) error
// InstantQuery runs an instant query with provided headers
InstantQuery(req QueryRequest, headers Headers) (model.Vector, error)
// InstantQueryWithEngine runs an instant query with provided headers and the specified
// query engine.
InstantQueryWithEngine(req QueryRequest, engine options.QueryEngine, headers Headers) (model.Vector, error)
// RangeQuery runs a range query with provided headers
RangeQuery(req RangeQueryRequest, headers Headers) (model.Matrix, error)
// GraphiteQuery retrieves graphite raw data.
GraphiteQuery(GraphiteQueryRequest) ([]Datapoint, error)
// RangeQueryWithEngine runs a range query with provided headers and the specified
// query engine.
RangeQueryWithEngine(req RangeQueryRequest, engine options.QueryEngine, headers Headers) (model.Matrix, error)
// LabelNames return matching label names based on the request.
LabelNames(req LabelNamesRequest, headers Headers) (model.LabelNames, error)
// LabelValues returns matching label values based on the request.
LabelValues(req LabelValuesRequest, headers Headers) (model.LabelValues, error)
// Series returns matching series based on the request.
Series(req SeriesRequest, headers Headers) ([]model.Metric, error)
}
Coordinator is a wrapper for a coordinator. It provides a wrapper on HTTP endpoints that expose cluster management APIs as well as read and write endpoints for series data. TODO: consider having this work on underlying structures.
type CoordinatorClient ¶ added in v1.4.0
type CoordinatorClient struct {
// contains filtered or unexported fields
}
CoordinatorClient is a client use to invoke API calls on a coordinator
func NewCoordinatorClient ¶ added in v1.4.0
func NewCoordinatorClient(opts CoordinatorClientOptions) CoordinatorClient
NewCoordinatorClient creates a new CoordinatorClient.
func (*CoordinatorClient) AddM3msgTopicConsumer ¶ added in v1.4.0
func (c *CoordinatorClient) AddM3msgTopicConsumer( topicOpts M3msgTopicOptions, addRequest admin.TopicAddRequest, ) (admin.TopicGetResponse, error)
AddM3msgTopicConsumer adds a consumer service to an m3msg topic
func (*CoordinatorClient) AddNamespace ¶ added in v1.4.0
func (c *CoordinatorClient) AddNamespace( addRequest admin.NamespaceAddRequest, ) (admin.NamespaceGetResponse, error)
AddNamespace adds a namespace.
func (*CoordinatorClient) ApplyKVUpdate ¶ added in v1.4.0
func (c *CoordinatorClient) ApplyKVUpdate(update string) error
ApplyKVUpdate applies a KV update.
func (*CoordinatorClient) CreateDatabase ¶ added in v1.4.0
func (c *CoordinatorClient) CreateDatabase( addRequest admin.DatabaseCreateRequest, ) (admin.DatabaseCreateResponse, error)
CreateDatabase creates a database.
func (*CoordinatorClient) DeleteAllPlacements ¶ added in v1.4.0
func (c *CoordinatorClient) DeleteAllPlacements(opts PlacementRequestOptions) error
DeleteAllPlacements deletes all placements for the specified service.
func (*CoordinatorClient) DeleteNamespace ¶ added in v1.4.0
func (c *CoordinatorClient) DeleteNamespace(namespaceID string) error
DeleteNamespace removes the namespace.
func (*CoordinatorClient) GetM3msgTopic ¶ added in v1.4.0
func (c *CoordinatorClient) GetM3msgTopic( topicOpts M3msgTopicOptions, ) (admin.TopicGetResponse, error)
GetM3msgTopic fetches an m3msg topic
func (*CoordinatorClient) GetNamespace ¶ added in v1.4.0
func (c *CoordinatorClient) GetNamespace() (admin.NamespaceGetResponse, error)
GetNamespace gets namespaces.
func (*CoordinatorClient) GetPlacement ¶ added in v1.4.0
func (c *CoordinatorClient) GetPlacement(opts PlacementRequestOptions) (admin.PlacementGetResponse, error)
GetPlacement gets placements.
func (*CoordinatorClient) GraphiteQuery ¶ added in v1.4.0
func (c *CoordinatorClient) GraphiteQuery( graphiteReq GraphiteQueryRequest, ) ([]Datapoint, error)
GraphiteQuery retrieves graphite raw data.
func (*CoordinatorClient) InitM3msgTopic ¶ added in v1.4.0
func (c *CoordinatorClient) InitM3msgTopic( topicOpts M3msgTopicOptions, initRequest admin.TopicInitRequest, ) (admin.TopicGetResponse, error)
InitM3msgTopic initializes an m3msg topic
func (*CoordinatorClient) InitPlacement ¶ added in v1.4.0
func (c *CoordinatorClient) InitPlacement( opts PlacementRequestOptions, initRequest admin.PlacementInitRequest, ) (admin.PlacementGetResponse, error)
InitPlacement initializes placements.
func (*CoordinatorClient) InstantQuery ¶ added in v1.4.0
func (c *CoordinatorClient) InstantQuery(req QueryRequest, headers Headers) (model.Vector, error)
InstantQuery runs an instant query with provided headers
func (*CoordinatorClient) InstantQueryWithEngine ¶ added in v1.4.2
func (c *CoordinatorClient) InstantQueryWithEngine( req QueryRequest, engine options.QueryEngine, headers Headers, ) (model.Vector, error)
InstantQueryWithEngine runs an instant query with provided headers and the specified query engine.
func (*CoordinatorClient) LabelNames ¶ added in v1.4.0
func (c *CoordinatorClient) LabelNames( req LabelNamesRequest, headers Headers, ) (model.LabelNames, error)
LabelNames return matching label names based on the request.
func (*CoordinatorClient) LabelValues ¶ added in v1.4.0
func (c *CoordinatorClient) LabelValues( req LabelValuesRequest, headers Headers, ) (model.LabelValues, error)
LabelValues return matching label values based on the request.
func (*CoordinatorClient) RangeQuery ¶ added in v1.4.0
func (c *CoordinatorClient) RangeQuery( req RangeQueryRequest, headers Headers, ) (model.Matrix, error)
RangeQuery runs a range query with provided headers
func (*CoordinatorClient) RangeQueryWithEngine ¶ added in v1.4.2
func (c *CoordinatorClient) RangeQueryWithEngine( req RangeQueryRequest, engine options.QueryEngine, headers Headers, ) (model.Matrix, error)
RangeQueryWithEngine runs a range query with provided headers and the specified query engine.
func (*CoordinatorClient) RunQuery ¶ added in v1.4.0
func (c *CoordinatorClient) RunQuery( verifier ResponseVerifier, query string, headers map[string][]string, ) error
RunQuery runs the given query with a given verification function.
func (*CoordinatorClient) Series ¶ added in v1.4.0
func (c *CoordinatorClient) Series( req SeriesRequest, headers Headers, ) ([]model.Metric, error)
Series returns matching series based on the request.
func (*CoordinatorClient) UpdateNamespace ¶ added in v1.4.0
func (c *CoordinatorClient) UpdateNamespace( req admin.NamespaceUpdateRequest, ) (admin.NamespaceGetResponse, error)
UpdateNamespace updates the namespace.
func (*CoordinatorClient) WaitForClusterReady ¶ added in v1.4.0
func (c *CoordinatorClient) WaitForClusterReady() error
WaitForClusterReady waits until the cluster is ready to receive reads and writes.
func (*CoordinatorClient) WaitForInstances ¶ added in v1.4.0
func (c *CoordinatorClient) WaitForInstances( ids []string, ) error
WaitForInstances blocks until the given instance is available.
func (*CoordinatorClient) WaitForNamespace ¶ added in v1.4.0
func (c *CoordinatorClient) WaitForNamespace(name string) error
WaitForNamespace blocks until the given namespace is enabled. NB: if the name string is empty, this will instead check for a successful response.
func (*CoordinatorClient) WaitForShardsReady ¶ added in v1.4.0
func (c *CoordinatorClient) WaitForShardsReady() error
WaitForShardsReady waits until all shards gets ready.
func (*CoordinatorClient) WriteCarbon ¶ added in v1.4.0
WriteCarbon writes a carbon metric datapoint at a given time.
func (*CoordinatorClient) WriteProm ¶ added in v1.4.0
func (c *CoordinatorClient) WriteProm( name string, tags map[string]string, samples []prompb.Sample, headers Headers, ) error
WriteProm writes a prometheus metric. Takes tags/labels as a map for convenience.
func (*CoordinatorClient) WritePromWithRequest ¶ added in v1.4.2
func (c *CoordinatorClient) WritePromWithRequest(writeRequest prompb.WriteRequest, headers Headers) error
WritePromWithRequest executes a prometheus write request. Allows you to provide the request directly which is useful for batch metric requests.
type CoordinatorClientOptions ¶ added in v1.4.0
type CoordinatorClientOptions struct {
Client *http.Client
HTTPPort int
Logger *zap.Logger
RetryFunc RetryFunc
}
CoordinatorClientOptions are the options for the CoordinatorClient.
type CoordinatorClusterOptions ¶ added in v1.4.0
type CoordinatorClusterOptions struct {
// GeneratePortsi ndicates whether to update the coordinator config to use open ports.
GeneratePorts bool
}
CoordinatorClusterOptions contains the cluster options for spinning up the coordinator.
type DBNodeClusterOptions ¶ added in v1.4.0
type DBNodeClusterOptions struct {
// RF is the replication factor to use for the cluster.
RF int32
// NumShards is the number of shards to use for each RF.
NumShards int32
// NumInstances is the number of dbnode instances per RF.
NumInstances int32
// NumIsolationGroups is the number of isolation groups to split
// nodes into.
NumIsolationGroups int32
}
DBNodeClusterOptions contains the cluster options for spinning up dbnodes.
func NewDBNodeClusterOptions ¶ added in v1.4.0
func NewDBNodeClusterOptions() *DBNodeClusterOptions
NewDBNodeClusterOptions creates DBNodeClusteOptions with sane defaults. DBNode config must still be provided.
func (*DBNodeClusterOptions) Validate ¶ added in v1.4.0
func (d *DBNodeClusterOptions) Validate() error
Validate validates the DBNodeClusterOptions.
type Datapoint ¶ added in v1.4.0
type Datapoint struct {
// Value is the value of the datapoint.
Value *float64
// Timestamp is the timestamp (in seconds) of the datapoint.
Timestamp int64
}
Datapoint is a data point returned by the graphite render query.
type ExternalResources ¶ added in v1.3.0
type ExternalResources interface {
// Setup sets up the external resource so that it's ready
// for use.
Setup() error
// Close stops and cleans up all the resources associated with
// the external resource.
Close() error
}
ExternalResources represents an external (i.e. non-M3) resource that we'd like to be able to spin up for an integration test.
type GoalStateVerifier ¶
GoalStateVerifier verifies that the given results are valid.
type GraphiteQueryRequest ¶ added in v1.4.0
type GraphiteQueryRequest struct {
// Target speicifies a path identifying one or several metrics.
Target string
// From is the beginning of the time period to query.
From time.Time
// Until is the end of the time period to query.
Until time.Time
}
GraphiteQueryRequest represents a graphite render query request.
type InstanceInfo ¶ added in v1.4.0
type InstanceInfo struct {
// ID is the name of the host. It can be hostname or UUID or any other string.
ID string
// Env specifies the zone the host resides in.
Env string
// Zone specifies the zone the host resides in.
Zone string
// Address can be IP address or hostname, this is used to connect to the host.
Address string
// M3msgAddress is the address of the m3msg server if there is one.
M3msgAddress string
// Port is the port number.
Port uint32
// Port is the port of the m3msg server if there is one.
M3msgPort uint32
}
InstanceInfo represents the host information for an instance.
type LabelNamesRequest ¶ added in v1.4.0
type LabelNamesRequest struct {
MetadataRequest
}
LabelNamesRequest contains the parameters for making label names API calls.
type LabelValuesRequest ¶ added in v1.4.0
type LabelValuesRequest struct {
MetadataRequest
// LabelName is the name of the label to retrieve values for.
LabelName string
}
LabelValuesRequest contains the parameters for making label values API calls.
type M3Resources ¶
type M3Resources interface {
// Start starts all the M3 components.
Start()
// Cleanup cleans up after each started component.
Cleanup() error
// Nodes returns all node resources.
Nodes() Nodes
// Coordinator returns the coordinator resource.
Coordinator() Coordinator
// Aggregators returns all aggregator resources.
Aggregators() Aggregators
}
M3Resources represents a set of test M3 components.
type M3msgTopicOptions ¶ added in v1.3.0
type M3msgTopicOptions struct {
// Zone is the zone of the m3msg topic.
Zone string
// Env is the environment of the m3msg topic.
Env string
// TopicName is the topic name of the m3msg topic name.
TopicName string
}
M3msgTopicOptions represents a set of options for an m3msg topic.
type MetadataRequest ¶ added in v1.4.0
type MetadataRequest struct {
// Start is the start timestamp of labels to include.
Start time.Time
// End is the end timestamp of labels to include.
End time.Time
// Match is the series selector that selects series to read label names from.
Match string
}
MetadataRequest contains the parameters for making API requests related to metadata.
func (*MetadataRequest) String ¶ added in v1.4.0
func (m *MetadataRequest) String() string
type Node ¶
type Node interface {
// Start starts the dbnode instance.
Start()
// HostDetails returns this node's host details on the given port.
HostDetails(port int) (*admin.Host, error)
// Health gives this node's health.
Health() (*rpc.NodeHealthResult_, error)
// WaitForBootstrap blocks until the node has bootstrapped.
WaitForBootstrap() error
// WritePoint writes a datapoint to the node directly.
WritePoint(req *rpc.WriteRequest) error
// WriteTaggedPoint writes a datapoint with tags to the node directly.
WriteTaggedPoint(req *rpc.WriteTaggedRequest) error
// WriteTaggedBatchRaw writes a batch of writes to the node directly.
WriteTaggedBatchRaw(req *rpc.WriteTaggedBatchRawRequest) error
// AggregateTiles starts tiles aggregation, waits until it will complete
// and returns the amount of aggregated tiles.
AggregateTiles(req *rpc.AggregateTilesRequest) (int64, error)
// Fetch fetches datapoints.
Fetch(req *rpc.FetchRequest) (*rpc.FetchResult_, error)
// FetchTagged fetches datapoints by tag.
FetchTagged(req *rpc.FetchTaggedRequest) (*rpc.FetchTaggedResult_, error)
// Exec executes the given commands on the node container, returning
// stdout and stderr from the container.
Exec(commands ...string) (string, error)
// GoalStateExec executes the given commands on the node container, retrying
// until applying the verifier returns no error or the default timeout.
GoalStateExec(verifier GoalStateVerifier, commands ...string) error
// Restart restarts this container.
Restart() error
// Close closes the wrapper and releases any held resources, including
// deleting docker containers.
Close() error
}
Node is a wrapper for a db node. It provides a wrapper on HTTP endpoints that expose cluster management APIs as well as read and write endpoints for series data. TODO: consider having this work on underlying structures.
type Nodes ¶
type Nodes []Node
Nodes is a slice of nodes.
func (Nodes) WaitForHealthy ¶
WaitForHealthy waits for each Node in Nodes to be healthy and bootstrapped before returning.
type PlacementRequestOptions ¶ added in v1.3.0
type PlacementRequestOptions struct {
// Service is the type of service for the placement request.
Service ServiceType
// Env is the environment of the placement.
Env string
// Zone is the zone of the placement.
Zone string
}
PlacementRequestOptions represents a set of options for placement-related requests.
type QueryRequest ¶ added in v1.4.0
type QueryRequest struct {
// Query is the Prometheus expression query string.
Query string
// Time is the evaluation timestamp. It is optional.
Time *time.Time
}
QueryRequest represents an instant query request
type RangeQueryRequest ¶ added in v1.4.0
type RangeQueryRequest struct {
// Query is the Prometheus expression query string.
Query string
// Start is the start timestamp of the query range. The default value is time.Now().
Start time.Time
// End is the end timestamp of the query range. The default value is time.Now().
End time.Time
// Step is the query resolution step width. It is default to 15 seconds.
Step time.Duration
}
RangeQueryRequest represents a range query request
type ResponseVerifier ¶
ResponseVerifier is a function that checks if the query response is valid.
type RetryFunc ¶ added in v1.4.0
RetryFunc is a function that retries the provided operation until successful.
type SeriesRequest ¶ added in v1.4.0
type SeriesRequest struct {
MetadataRequest
}
SeriesRequest contains the parameters for making series API calls.
type ServiceType ¶ added in v1.3.0
type ServiceType int
ServiceType represents the type of an m3 service.
const ( // ServiceTypeUnknown is an unknown service type. ServiceTypeUnknown ServiceType = iota // ServiceTypeM3DB represents M3DB service. ServiceTypeM3DB // ServiceTypeM3Aggregator represents M3aggregator service. ServiceTypeM3Aggregator // ServiceTypeM3Coordinator represents M3coordinator service. ServiceTypeM3Coordinator )
Directories
¶
| Path | Synopsis |
|---|---|
|
Package docker contains resources needed to setup docker containers for M3 tests.
|
Package docker contains resources needed to setup docker containers for M3 tests. |
|
Package inprocess contains code for spinning up M3 resources in-process for the sake of integration testing.
|
Package inprocess contains code for spinning up M3 resources in-process for the sake of integration testing. |
|
Package net contains network-related helpers for integration testing.
|
Package net contains network-related helpers for integration testing. |