From 69b0ea0e171ad47c6f7ba9773c0d77dbd40891f4 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 11 Jul 2018 10:59:25 -0400 Subject: [PATCH] Add rudimentary multi-cluster support for m3coordinator (#785) This adds rudimentary multi-cluster support for m3coordinator so that we can begin to send metrics to different namespaces for different retention periods, etc. Right now this is not optimal as it retrieves data from all matching namespaces that has retention long enough to cover the query, then returns the most granular datapoints that come back and discards any lower granularity datapoints that also came back. This is because we do not have a distributed index in the current M3 open source offering. For read workloads that do not require tens of thousands or hundreds of thousands realtime alert evaluations, this should be quite sufficient given that the write volume absolutely dwarfs the read volume. At some point we'll hopefully have a distributed index that we can use for all use cases and deprecate this behavior. --- glide.lock | 73 ++- glide.yaml | 23 +- .../prometheus/m3dbnode-local-etcd.yml | 5 +- .../prometheus/prometheus-integration-test.sh | 22 +- .../services/m3coordinator/config/config.go | 45 +- src/cmd/services/m3dbnode/main/main.go | 2 +- .../api/v1/handler/database/common.go | 9 +- .../api/v1/handler/database/create.go | 227 ++++--- .../api/v1/handler/database/create_test.go | 464 +++++++++++++-- .../api/v1/handler/placement/add.go | 11 +- .../api/v1/handler/placement/common.go | 30 +- .../api/v1/handler/placement/common_test.go | 52 +- .../api/v1/handler/placement/delete.go | 2 +- .../api/v1/handler/placement/delete_all.go | 2 +- .../api/v1/handler/placement/get.go | 2 +- .../api/v1/handler/placement/init.go | 14 +- .../v1/handler/prometheus/native/read_test.go | 2 + .../v1/handler/prometheus/remote/read_test.go | 10 +- .../handler/prometheus/remote/write_test.go | 6 +- src/coordinator/api/v1/handler/search_test.go | 9 +- src/coordinator/api/v1/httpd/handler.go | 20 +- src/coordinator/api/v1/httpd/handler_test.go | 17 +- .../benchmark/read/main/read_benchmark.go | 34 +- .../benchmark/write/main/write_benchmark.go | 23 +- src/coordinator/executor/engine_test.go | 2 +- .../generated/proto/admin/database.pb.go | 559 +++++++++++++----- .../generated/proto/admin/database.proto | 40 +- src/coordinator/models/tag.go | 2 +- src/coordinator/models/tag_test.go | 3 +- .../services/m3coordinator/server/server.go | 96 +-- src/coordinator/storage/config.go | 50 ++ src/coordinator/storage/config_test.go | 54 ++ .../storage/fanout/storage_test.go | 13 +- src/coordinator/storage/interface.go | 36 ++ src/coordinator/storage/local/cluster.go | 293 +++++++++ src/coordinator/storage/local/cluster_test.go | 171 ++++++ src/coordinator/storage/local/config.go | 194 ++++++ src/coordinator/storage/local/storage.go | 287 ++++++++- src/coordinator/storage/local/storage_test.go | 235 +++++++- src/coordinator/test/local/storage.go | 47 +- src/coordinator/test/seriesiter/mock_iter.go | 17 +- 41 files changed, 2749 insertions(+), 454 deletions(-) create mode 100644 src/coordinator/storage/config.go create mode 100644 src/coordinator/storage/config_test.go create mode 100644 src/coordinator/storage/local/cluster.go create mode 100644 src/coordinator/storage/local/cluster_test.go create mode 100644 src/coordinator/storage/local/config.go diff --git a/glide.lock b/glide.lock index 88256f16b8..254780da6a 100644 --- a/glide.lock +++ b/glide.lock @@ -88,15 +88,13 @@ imports: - wal - wal/walpb - name: github.com/coreos/go-semver - version: 8ab6407b697782a06568d4b7f1db25550ec2e4c6 + version: 568e959cd89871e61434c1143528d9162da89ef2 subpackages: - semver - name: github.com/coreos/go-systemd version: 48702e0da86bd25e76cfef347e2adeb434a0d0a6 subpackages: - - daemon - journal - - util - name: github.com/coreos/pkg version: 97fdf19511ea361ae1c100dd393cc47f8dcfa1e1 subpackages: @@ -113,14 +111,12 @@ imports: - spew - name: github.com/dgrijalva/jwt-go version: d2709f9f1f31ebcda9651b03077758c1f3a0018c -- name: github.com/dgryski/go-bits - version: 2ad8d707cc05b1815ce6ff2543bb5e8d8f9298ef - name: github.com/edsrzf/mmap-go version: 0bce6a6887123b67a60366d2c9fe2dfb74289d2e - name: github.com/fsnotify/fsnotify version: c2828203cd70a50dcccfb2761f8b1f8ceef9a8e9 - name: github.com/ghodss/yaml - version: 0ca9ea5df5451ffdf184b4428c902747c2c11cd7 + version: 73d445a93680fa1a78ae23a5839bad48f32ba1ee - name: github.com/glycerine/go-unsnap-stream version: 9f0cb55181dd3a0a4c168d3dbc72d4aca4853126 - name: github.com/go-kit/kit @@ -201,6 +197,20 @@ imports: version: 07973db6b78acb62ac207d0538055e874b49d90d - name: github.com/m3db/bloom version: 47fe1193cdb900de7193d1f3d26ea9b2cbf6fb31 +- name: github.com/m3db/m3aggregator + version: fd38c07d1a94b8598b6839a9c471e0a6feacfafb + subpackages: + - aggregation + - aggregation/quantile/cm + - aggregator + - aggregator/handler + - aggregator/handler/common + - aggregator/handler/writer + - generated/proto/flush + - hash + - rate + - runtime + - sharding - name: github.com/m3db/m3cluster version: b4935c48a00f47c6f16f71d15d94e1c24785297d subpackages: @@ -240,11 +250,47 @@ imports: - os/fs - x/grpc - name: github.com/m3db/m3metrics - version: 17e4ddf89f2b0c8fbb16b6d5b71a871b793d2a00 + version: f22d8684fa8b42ff30f1d68f6f2be5e465db9a9d subpackages: + - aggregation + - errors + - filters - generated/proto/schema + - matcher + - matcher/cache - metric + - metric/aggregated + - metric/id + - metric/id/m3 + - metric/unaggregated - policy + - protocol/msgpack + - rules + - rules/models + - rules/models/changes +- name: github.com/m3db/m3ninx + version: 7556fa8339674f1d9f559486d1feca18c17d1190 + subpackages: + - doc + - generated/proto/fswriter + - generated/proto/querypb + - idx + - index + - index/segment + - index/segment/fs + - index/segment/fs/encoding + - index/segment/fs/encoding/docs + - index/segment/mem + - index/util + - persist + - postings + - postings/pilosa + - postings/roaring + - search + - search/executor + - search/query + - search/searcher + - x - name: github.com/m3db/m3x version: 216f0e4a8870d0df302aef0550a4b0e0db9291cd vcs: git @@ -362,10 +408,12 @@ imports: - util/strutil - util/testutil - name: github.com/prometheus/tsdb - version: 706602daed1487f7849990678b4ece4599745905 + version: 16b2bf1b45ce3e3536c78ebec5116ea09a69786e subpackages: + - chunkenc - chunks - fileutil + - index - labels - name: github.com/RoaringBitmap/roaring version: 361768d03f0924093d3eed7623f3cf58392620f4 @@ -447,6 +495,7 @@ imports: - zapcore - name: golang.org/x/crypto version: 9419663f5a44be8b34ca85f08abc5fe1be11f8a3 + subpackages: - bcrypt - blowfish @@ -473,6 +522,7 @@ imports: - errgroup - name: golang.org/x/sys version: e48874b42435b4347fc52bdee0424a52abc974d7 + subpackages: - unix - name: golang.org/x/text @@ -502,9 +552,14 @@ imports: version: 5b3c4e850e90a4cf6a20ebd46c8b32a0a3afcb9e subpackages: - balancer + - balancer/base + - balancer/roundrobin + - channelz - codes - connectivity - credentials + - encoding + - encoding/proto - grpclb/grpc_lb_v1/messages - grpclog - health/grpc_health_v1 @@ -514,6 +569,8 @@ imports: - naming - peer - resolver + - resolver/dns + - resolver/passthrough - stats - status - tap diff --git a/glide.yaml b/glide.yaml index 35d4904009..843f146366 100644 --- a/glide.yaml +++ b/glide.yaml @@ -25,6 +25,12 @@ import: - package: github.com/m3db/m3em version: ed532baee45a440f0b08b6893c816634c6978d4d +- package: github.com/m3db/m3aggregator + version: fd38c07d1a94b8598b6839a9c471e0a6feacfafb + +- package: github.com/m3db/m3ninx + version: 7556fa8339674f1d9f559486d1feca18c17d1190 + - package: github.com/m3db/bitset version: 07973db6b78acb62ac207d0538055e874b49d90d @@ -54,6 +60,7 @@ import: - package: github.com/golang/protobuf version: ^1.1.0 + subpackages: - proto - ptypes/timestamp @@ -145,7 +152,7 @@ import: version: ^2.2.6 - package: github.com/m3db/m3metrics - version: 17e4ddf89f2b0c8fbb16b6d5b71a871b793d2a00 + version: f22d8684fa8b42ff30f1d68f6f2be5e465db9a9d subpackages: - policy @@ -161,14 +168,28 @@ import: subpackages: - cmp +# START_PROMETHEUS_DEPS - package: github.com/prometheus/prometheus version: 998dfcbac689ae832ea64ca134fcb096f61a7f62 +# To avoid prometheus/prometheus dependencies from breaking, +# pin the transitive dependencies +- package: github.com/prometheus/common + version: 9e0844febd9e2856f839c9cb974fbd676d1755a8 + +- package: github.com/prometheus/procfs + version: a1dba9ce8baed984a2495b658c82687f8157b98f + +- package: github.com/prometheus/tsdb + version: 16b2bf1b45ce3e3536c78ebec5116ea09a69786e +# END_PROMETHEUS_DEPS + - package: github.com/coreos/pkg version: 4 subpackages: - capnslog +# To avoid conflicting packages not resolving the latest GRPC - package: google.golang.org/grpc version: ~1.7.3 subpackages: diff --git a/scripts/integration-tests/prometheus/m3dbnode-local-etcd.yml b/scripts/integration-tests/prometheus/m3dbnode-local-etcd.yml index 463ef0ab39..4500a7b7a1 100644 --- a/scripts/integration-tests/prometheus/m3dbnode-local-etcd.yml +++ b/scripts/integration-tests/prometheus/m3dbnode-local-etcd.yml @@ -1,6 +1,10 @@ coordinator: listenAddress: 0.0.0.0:7201 + local: + namespace: prometheus_metrics + retention: 48h + metrics: scope: prefix: "coordinator" @@ -10,7 +14,6 @@ coordinator: sanitization: prometheus samplingRate: 1.0 extended: none - dbNamespace: prometheus_metrics db: logging: diff --git a/scripts/integration-tests/prometheus/prometheus-integration-test.sh b/scripts/integration-tests/prometheus/prometheus-integration-test.sh index 67db7cf547..faf53cccb3 100755 --- a/scripts/integration-tests/prometheus/prometheus-integration-test.sh +++ b/scripts/integration-tests/prometheus/prometheus-integration-test.sh @@ -5,11 +5,11 @@ set -xe rm -rf /tmp/m3dbdata/ mkdir -p /tmp/m3dbdata/ -echo "Build M3DB docker image" +echo "Build M3DB docker image" docker-compose -f docker-compose.yml build -echo "Run M3DB docker container" +echo "Run M3DB docker container" docker-compose -f docker-compose.yml up -d dbnode01 @@ -43,13 +43,13 @@ curl -vvvsSf -X POST localhost:7201/api/v1/namespace -d '{ } }' -echo "Sleep while namespace is init'd" +echo "Sleep while namespace is init'd" sleep 10 # TODO Replace sleeps with logic to determine when to proceed [ "$(curl -sSf localhost:7201/api/v1/namespace | jq .registry.namespaces.prometheus_metrics.indexOptions.enabled)" == true ] -echo "Initialization placement" +echo "Initialization placement" curl -vvvsSf -X POST localhost:7201/api/v1/placement/init -d '{ "num_shards": 64, @@ -69,17 +69,17 @@ curl -vvvsSf -X POST localhost:7201/api/v1/placement/init -d '{ [ "$(curl -sSf localhost:7201/api/v1/placement | jq .placement.instances.m3db_local.id)" == '"m3db_local"' ] -echo "Wait for placement to fully initialize" +echo "Wait for placement to fully initialize" sleep 60 # TODO Replace sleeps with logic to determine when to proceed -echo "Start Prometheus container" +echo "Start Prometheus container" docker-compose -f docker-compose.yml up -d prometheus01 sleep 10 -echo "Write data" +echo "Write data" curl -vvvsSf -X POST localhost:9003/writetagged -d '{ "namespace": "prometheus_metrics", @@ -112,12 +112,12 @@ queryResult=$(curl -sSf -X POST localhost:9003/query -d '{ }, "rangeStart": 0, "rangeEnd":'"$(date +"%s")"' -}' | jq '.results | length') +}' | jq '.results | length') -if [ "$queryResult" -lt 1 ]; then - echo "Result not found" +if [ "$queryResult" -lt 1 ]; then + echo "Result not found" exit 1 -else +else echo "Result found" fi diff --git a/src/cmd/services/m3coordinator/config/config.go b/src/cmd/services/m3coordinator/config/config.go index e9f9819033..a8e9625d1e 100644 --- a/src/cmd/services/m3coordinator/config/config.go +++ b/src/cmd/services/m3coordinator/config/config.go @@ -21,34 +21,61 @@ package config import ( - "github.com/m3db/m3db/src/dbnode/client" + "time" + + etcdclient "github.com/m3db/m3cluster/client/etcd" + "github.com/m3db/m3db/src/coordinator/storage/local" "github.com/m3db/m3x/instrument" ) // Configuration is the configuration for the coordinator. type Configuration struct { - // DBClient is the DB client configuration. - DBClient *client.Configuration `yaml:"dbClient"` - // Metrics configuration. Metrics instrument.MetricsConfiguration `yaml:"metrics"` + // Clusters is the DB cluster configurations for read, write and + // query endpoints. + Clusters local.ClustersStaticConfiguration `yaml:"clusters"` + + // LocalConfiguration is the local embedded configuration if running + // coordinator embedded in the DB. + Local *LocalConfiguration `yaml:"local"` + + // ClusterManagement for placemement, namespaces and database management + // endpoints (optional). + ClusterManagement *ClusterManagementConfiguration `yaml:"clusterManagement"` + // ListenAddress is the server listen address. ListenAddress string `yaml:"listenAddress" validate:"nonzero"` // RPC is the RPC configuration. RPC *RPCConfiguration `yaml:"rpc"` - // DBNamespace is the namespace string to use for reads and writes - DBNamespace string `yaml:"dbNamespace"` - - // DecompressWorkerPoolCount is the number of decompression worker pools + // DecompressWorkerPoolCount is the number of decompression worker pools. DecompressWorkerPoolCount int `yaml:"workerPoolCount"` - // DecompressWorkerPoolSize is the size of the worker pool given to each fetch request + // DecompressWorkerPoolSize is the size of the worker pool given to each + // fetch request. DecompressWorkerPoolSize int `yaml:"workerPoolSize"` } +// LocalConfiguration is the local embedded configuration if running +// coordinator embedded in the DB. +type LocalConfiguration struct { + // Namespace is the name of the local namespace to write/read from. + Namespace string `yaml:"namespace" validate:"nonzero"` + + // Retention is the retention of the local namespace to write/read from. + Retention time.Duration `yaml:"retention" validate:"nonzero"` +} + +// ClusterManagementConfiguration is configuration for the placemement, +// namespaces and database management endpoints (optional). +type ClusterManagementConfiguration struct { + // Etcd is the client configuration for etcd. + Etcd etcdclient.Configuration `yaml:"etcd"` +} + // RPCConfiguration is the RPC configuration for the coordinator for // the GRPC server used for remote coordinator to coordinator calls. type RPCConfiguration struct { diff --git a/src/cmd/services/m3dbnode/main/main.go b/src/cmd/services/m3dbnode/main/main.go index df79f4563b..b7f626cc8b 100644 --- a/src/cmd/services/m3dbnode/main/main.go +++ b/src/cmd/services/m3dbnode/main/main.go @@ -68,7 +68,7 @@ func main() { go func() { coordinatorserver.Run(coordinatorserver.RunOptions{ Config: *cfg.Coordinator, - DBConfig: *cfg.DB, + DBConfig: cfg.DB, DBClient: dbClientCh, ClusterClient: clusterClientCh, }) diff --git a/src/coordinator/api/v1/handler/database/common.go b/src/coordinator/api/v1/handler/database/common.go index c9d5b7b922..0f8274f19f 100644 --- a/src/coordinator/api/v1/handler/database/common.go +++ b/src/coordinator/api/v1/handler/database/common.go @@ -37,8 +37,13 @@ type Handler struct { } // RegisterRoutes registers the namespace routes -func RegisterRoutes(r *mux.Router, client clusterclient.Client, cfg config.Configuration, dbCfg dbconfig.DBConfiguration) { +func RegisterRoutes( + r *mux.Router, + client clusterclient.Client, + cfg config.Configuration, + embeddedDbCfg *dbconfig.DBConfiguration, +) { logged := logging.WithResponseTimeLogging - r.HandleFunc(CreateURL, logged(NewCreateHandler(client, cfg, dbCfg)).ServeHTTP).Methods(CreateHTTPMethod) + r.HandleFunc(CreateURL, logged(NewCreateHandler(client, cfg, embeddedDbCfg)).ServeHTTP).Methods(CreateHTTPMethod) } diff --git a/src/coordinator/api/v1/handler/database/create.go b/src/coordinator/api/v1/handler/database/create.go index 68874dc5db..322f1ac40e 100644 --- a/src/coordinator/api/v1/handler/database/create.go +++ b/src/coordinator/api/v1/handler/database/create.go @@ -57,18 +57,52 @@ const ( blockSizeFromExpectedSeriesScalar = idealDatapointsPerBlock * int64(time.Hour) shardMultiplier = 64 - dbTypeLocal dbType = "local" - dbTypeCluster dbType = "cluster" - defaultLocalRetentionPeriod = 24 * time.Hour - minBlockSize = 30 * time.Minute - maxBlockSize = 2 * time.Hour + dbTypeLocal dbType = "local" + dbTypeCluster dbType = "cluster" + + defaultIsolationGroup = "local" + defaultZone = "local" + + defaultLocalRetentionPeriod = 24 * time.Hour + + minRecommendCalculateBlockSize = 30 * time.Minute + maxRecommendCalculateBlockSize = 24 * time.Hour ) +type recommendedBlockSize struct { + forRetentionLessThanOrEqual time.Duration + blockSize time.Duration +} + +var recommendedBlockSizesByRetentionAsc = []recommendedBlockSize{ + { + forRetentionLessThanOrEqual: 12 * time.Hour, + blockSize: 30 * time.Minute, + }, + { + forRetentionLessThanOrEqual: 24 * time.Hour, + blockSize: time.Hour, + }, + { + forRetentionLessThanOrEqual: 7 * 24 * time.Hour, + blockSize: 2 * time.Hour, + }, + { + forRetentionLessThanOrEqual: 30 * 24 * time.Hour, + blockSize: 12 * time.Hour, + }, + { + forRetentionLessThanOrEqual: 365 * 24 * time.Hour, + blockSize: 24 * time.Hour, + }, +} + var ( - errMultipleHostTypes = errors.New("must only specify one type of host") - errMissingRequiredField = errors.New("missing required field") - errInvalidDBType = errors.New("invalid database type") - errMissingPort = errors.New("unable to get port from address") + errMissingRequiredField = errors.New("missing required field") + errInvalidDBType = errors.New("invalid database type") + errMissingEmbeddedDBPort = errors.New("unable to get port from embedded database listen address") + errMissingEmbeddedDBConfig = errors.New("unable to find local embedded database config") + errMissingHostID = errors.New("missing host ID") ) type dbType string @@ -77,16 +111,20 @@ type createHandler struct { placementInitHandler *placement.InitHandler namespaceAddHandler *namespace.AddHandler namespaceDeleteHandler *namespace.DeleteHandler - dbCfg dbconfig.DBConfiguration + embeddedDbCfg *dbconfig.DBConfiguration } // NewCreateHandler returns a new instance of a database create handler. -func NewCreateHandler(client clusterclient.Client, cfg config.Configuration, dbCfg dbconfig.DBConfiguration) http.Handler { +func NewCreateHandler( + client clusterclient.Client, + cfg config.Configuration, + embeddedDbCfg *dbconfig.DBConfiguration, +) http.Handler { return &createHandler{ placementInitHandler: placement.NewInitHandler(client, cfg), namespaceAddHandler: namespace.NewAddHandler(client), namespaceDeleteHandler: namespace.NewDeleteHandler(client), - dbCfg: dbCfg, + embeddedDbCfg: embeddedDbCfg, } } @@ -108,7 +146,7 @@ func (h *createHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - initPlacement, err := h.placementInitHandler.Init(placementRequest) + initPlacement, err := h.placementInitHandler.Init(r, placementRequest) if err != nil { // Attempt to delete the namespace that was just created to maintain idempotency err = h.namespaceDeleteHandler.Delete(namespaceRequest.Name) @@ -159,22 +197,15 @@ func (h *createHandler) parseRequest(r *http.Request) (*admin.NamespaceAddReques return nil, nil, handler.NewParseError(errMissingRequiredField, http.StatusBadRequest) } - if dbType(dbCreateReq.Type) == dbTypeCluster { - // One (and only one) hostname type required - if len(dbCreateReq.Hostnames) == 0 && len(dbCreateReq.HostnameGroups) == 0 { - return nil, nil, handler.NewParseError(errMissingRequiredField, http.StatusBadRequest) - } - - if len(dbCreateReq.Hostnames) > 0 && len(dbCreateReq.HostnameGroups) > 0 { - return nil, nil, handler.NewParseError(errMultipleHostTypes, http.StatusBadRequest) - } + if dbType(dbCreateReq.Type) == dbTypeCluster && len(dbCreateReq.Hosts) == 0 { + return nil, nil, handler.NewParseError(errMissingRequiredField, http.StatusBadRequest) } namespaceAddRequest, err := defaultedNamespaceAddRequest(dbCreateReq) if err != nil { return nil, nil, handler.NewParseError(err, http.StatusBadRequest) } - placementInitRequest, err := defaultedPlacementInitRequest(dbCreateReq, h.dbCfg) + placementInitRequest, err := defaultedPlacementInitRequest(dbCreateReq, h.embeddedDbCfg) if err != nil { return nil, nil, handler.NewParseError(err, http.StatusBadRequest) } @@ -183,61 +214,98 @@ func (h *createHandler) parseRequest(r *http.Request) (*admin.NamespaceAddReques } func defaultedNamespaceAddRequest(r *admin.DatabaseCreateRequest) (*admin.NamespaceAddRequest, error) { - options := dbnamespace.NewOptions() + opts := dbnamespace.NewOptions() switch dbType(r.Type) { case dbTypeLocal, dbTypeCluster: - options.SetRepairEnabled(false) + opts = opts.SetRepairEnabled(false) + retentionOpts := opts.RetentionOptions() if r.RetentionPeriodNanos <= 0 { - options.RetentionOptions().SetRetentionPeriod(defaultLocalRetentionPeriod) + retentionOpts = retentionOpts.SetRetentionPeriod(defaultLocalRetentionPeriod) } else { - options.RetentionOptions().SetRetentionPeriod(time.Duration(r.RetentionPeriodNanos)) + retentionOpts = retentionOpts.SetRetentionPeriod(time.Duration(r.RetentionPeriodNanos)) } - retentionPeriod := options.RetentionOptions().RetentionPeriod() - if r.ExpectedSeriesDatapointsPerHour > 0 { - blockSize := time.Duration(blockSizeFromExpectedSeriesScalar / r.ExpectedSeriesDatapointsPerHour) - if blockSize < minBlockSize { - blockSize = minBlockSize - } else if blockSize > maxBlockSize { - blockSize = maxBlockSize + retentionPeriod := retentionOpts.RetentionPeriod() + + var blockSize time.Duration + switch { + case r.BlockSize != nil && r.BlockSize.Nanos > 0: + blockSize = time.Duration(r.BlockSize.Nanos) + case r.BlockSize != nil && r.BlockSize.ExpectedSeriesDatapointsPerHour > 0: + value := r.BlockSize.ExpectedSeriesDatapointsPerHour + blockSize = time.Duration(blockSizeFromExpectedSeriesScalar / value) + // Snap to the nearest 5mins + blockSizeCeil := blockSize.Truncate(5*time.Minute) + 5*time.Minute + blockSizeFloor := blockSize.Truncate(5 * time.Minute) + if blockSizeFloor%time.Hour == 0 || + blockSizeFloor%30*time.Minute == 0 || + blockSizeFloor%15*time.Minute == 0 || + blockSizeFloor%10*time.Minute == 0 { + // Try snap to hour or 30min or 15min or 10min boundary if possible + blockSize = blockSizeFloor + } else { + blockSize = blockSizeCeil + } + + if blockSize < minRecommendCalculateBlockSize { + blockSize = minRecommendCalculateBlockSize + } else if blockSize > maxRecommendCalculateBlockSize { + blockSize = maxRecommendCalculateBlockSize + } + default: + // Use the maximum block size if we don't find a way to + // recommended one based on request parameters + max := recommendedBlockSizesByRetentionAsc[len(recommendedBlockSizesByRetentionAsc)-1] + blockSize = max.blockSize + for _, elem := range recommendedBlockSizesByRetentionAsc { + if retentionPeriod <= elem.forRetentionLessThanOrEqual { + blockSize = elem.blockSize + break + } } - options.RetentionOptions().SetBlockSize(blockSize) - } else if retentionPeriod <= 12*time.Hour { - options.RetentionOptions().SetBlockSize(30 * time.Minute) - } else if retentionPeriod <= 24*time.Hour { - options.RetentionOptions().SetBlockSize(1 * time.Hour) - } else { - options.RetentionOptions().SetBlockSize(2 * time.Hour) } - options.IndexOptions().SetEnabled(true) - options.IndexOptions().SetBlockSize(options.RetentionOptions().BlockSize()) + retentionOpts = retentionOpts.SetBlockSize(blockSize) + + indexOpts := opts.IndexOptions(). + SetEnabled(true). + SetBlockSize(blockSize) + + opts = opts.SetRetentionOptions(retentionOpts). + SetIndexOptions(indexOpts) default: return nil, errInvalidDBType } return &admin.NamespaceAddRequest{ Name: r.NamespaceName, - Options: dbnamespace.OptionsToProto(options), + Options: dbnamespace.OptionsToProto(opts), }, nil } -func defaultedPlacementInitRequest(r *admin.DatabaseCreateRequest, dbCfg dbconfig.DBConfiguration) (*admin.PlacementInitRequest, error) { +func defaultedPlacementInitRequest( + r *admin.DatabaseCreateRequest, + embeddedDbCfg *dbconfig.DBConfiguration, +) (*admin.PlacementInitRequest, error) { var ( numShards int32 replicationFactor int32 instances []*placementpb.Instance ) - - port, err := portFromAddress(dbCfg.ListenAddress) - if err != nil { - return nil, err - } - switch dbType(r.Type) { case dbTypeLocal: + if embeddedDbCfg == nil { + return nil, errMissingEmbeddedDBConfig + } + + addr := embeddedDbCfg.ListenAddress + port, err := portFromEmbeddedDBConfigListenAddress(addr) + if err != nil { + return nil, err + } + numShards = shardMultiplier replicationFactor = 1 instances = []*placementpb.Instance{ @@ -246,42 +314,51 @@ func defaultedPlacementInitRequest(r *admin.DatabaseCreateRequest, dbCfg dbconfi IsolationGroup: "local", Zone: "embedded", Weight: 1, - Endpoint: fmt.Sprintf("http://localhost:%d", port), + Endpoint: fmt.Sprintf("127.0.0.1:%d", port), Hostname: "localhost", Port: uint32(port), }, } case dbTypeCluster: - // This function assumes only one of Hostnames/HostnameGroups have been set - numHosts := len(r.Hostnames) + len(r.HostnameGroups) + + numHosts := len(r.Hosts) numShards = int32(math.Min(math.MaxInt32, powerOfTwoAtLeast(float64(numHosts*shardMultiplier)))) replicationFactor = r.ReplicationFactor if replicationFactor == 0 { replicationFactor = 3 } + instances = make([]*placementpb.Instance, 0, numHosts) - for _, hostname := range r.Hostnames { - instances = append(instances, &placementpb.Instance{ - Id: hostname, - IsolationGroup: "cluster", - Zone: "embedded", - Weight: 1, - Endpoint: fmt.Sprintf("http://%s:%d", hostname, port), - Hostname: hostname, - Port: uint32(port), - }) - } + for _, host := range r.Hosts { + id := strings.TrimSpace(host.Id) + if id == "" { + return nil, errMissingHostID + } + + isolationGroup := strings.TrimSpace(host.IsolationGroup) + if isolationGroup == "" { + isolationGroup = defaultIsolationGroup + } + + zone := strings.TrimSpace(host.Zone) + if zone == "" { + zone = defaultZone + } + + weight := host.Weight + if weight == 0 { + weight = 1 + } - for _, hostnameGroup := range r.HostnameGroups { instances = append(instances, &placementpb.Instance{ - Id: hostnameGroup.Hostname, - IsolationGroup: hostnameGroup.IsolationGroup, - Zone: "embedded", - Weight: 1, - Endpoint: fmt.Sprintf("http://%s:%d", hostnameGroup.Hostname, port), - Hostname: hostnameGroup.Hostname, - Port: uint32(port), + Id: id, + IsolationGroup: isolationGroup, + Zone: zone, + Weight: weight, + Endpoint: fmt.Sprintf("%s:%d", host.Address, host.Port), + Hostname: host.Address, + Port: host.Port, }) } default: @@ -295,10 +372,10 @@ func defaultedPlacementInitRequest(r *admin.DatabaseCreateRequest, dbCfg dbconfi }, nil } -func portFromAddress(address string) (int, error) { +func portFromEmbeddedDBConfigListenAddress(address string) (int, error) { colonIdx := strings.LastIndex(address, ":") if colonIdx == -1 || colonIdx == len(address)-1 { - return 0, errMissingPort + return 0, errMissingEmbeddedDBPort } return strconv.Atoi(address[colonIdx+1:]) diff --git a/src/coordinator/api/v1/handler/database/create_test.go b/src/coordinator/api/v1/handler/database/create_test.go index 76d9a0c9b1..75ae4fb190 100644 --- a/src/coordinator/api/v1/handler/database/create_test.go +++ b/src/coordinator/api/v1/handler/database/create_test.go @@ -21,11 +21,15 @@ package database import ( + "encoding/json" + "fmt" "io/ioutil" "net/http" "net/http/httptest" "strings" "testing" + "time" + "unicode" "github.com/m3db/m3cluster/client" "github.com/m3db/m3cluster/generated/proto/placementpb" @@ -36,6 +40,7 @@ import ( dbconfig "github.com/m3db/m3db/src/cmd/services/m3dbnode/config" "github.com/m3db/m3db/src/coordinator/api/v1/handler/namespace" "github.com/m3db/m3db/src/coordinator/util/logging" + xtest "github.com/m3db/m3db/src/dbnode/x/test" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" @@ -43,7 +48,7 @@ import ( ) var ( - testDBCfg = dbconfig.DBConfiguration{ + testDBCfg = &dbconfig.DBConfiguration{ ListenAddress: "0.0.0.0:9000", } ) @@ -111,21 +116,77 @@ func TestLocalType(t *testing.T) { body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"namespace\":{\"registry\":{\"namespaces\":{\"testNamespace\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":true,\"repairEnabled\":false,\"retentionOptions\":{\"retentionPeriodNanos\":\"172800000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"120000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"300000000000\"},\"snapshotEnabled\":false,\"indexOptions\":{\"enabled\":false,\"blockSizeNanos\":\"7200000000000\"}}}}},\"placement\":{\"placement\":{\"instances\":{\"localhost\":{\"id\":\"localhost\",\"isolationGroup\":\"local\",\"zone\":\"embedded\",\"weight\":1,\"endpoint\":\"http://localhost:9000\",\"shards\":[],\"shardSetId\":0,\"hostname\":\"localhost\",\"port\":9000}},\"replicaFactor\":0,\"numShards\":0,\"isSharded\":false,\"cutoverTime\":\"0\",\"isMirrored\":false,\"maxShardSetId\":0},\"version\":0}}", string(body)) + + expectedResponse := ` + { + "namespace": { + "registry": { + "namespaces": { + "testNamespace": { + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": true, + "repairEnabled": false, + "retentionOptions": { + "retentionPeriodNanos": "86400000000000", + "blockSizeNanos": "3600000000000", + "bufferFutureNanos": "120000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "300000000000" + }, + "snapshotEnabled": false, + "indexOptions": { + "enabled": true, + "blockSizeNanos": "3600000000000" + } + } + } + } + }, + "placement": { + "placement": { + "instances": { + "localhost": { + "id": "localhost", + "isolationGroup": "local", + "zone": "embedded", + "weight": 1, + "endpoint": "http://localhost:9000", + "shards": [], + "shardSetId": 0, + "hostname": "localhost", + "port": 9000 + } + }, + "replicaFactor": 0, + "numShards": 0, + "isSharded": false, + "cutoverTime": "0", + "isMirrored": false, + "maxShardSetId": 0 + }, + "version": 0 + } + } + ` + assert.Equal(t, stripAllWhitespace(expectedResponse), string(body), + xtest.Diff(mustPrettyJSON(t, expectedResponse), mustPrettyJSON(t, string(body)))) } -func TestClusterTypeHostnames(t *testing.T) { +func TestLocalWithBlockSizeNanos(t *testing.T) { mockClient, mockKV, mockPlacementService := SetupDatabaseTest(t) createHandler := NewCreateHandler(mockClient, config.Configuration{}, testDBCfg) w := httptest.NewRecorder() - jsonInput := ` + jsonInput := fmt.Sprintf(` { "namespaceName": "testNamespace", - "type": "cluster", - "hostnames": ["host1", "host2"] + "type": "local", + "blockSize": {"nanos": %d} } - ` + `, int64(3*time.Hour)) req := httptest.NewRequest("POST", "/database/create", strings.NewReader(jsonInput)) require.NotNil(t, req) @@ -135,29 +196,124 @@ func TestClusterTypeHostnames(t *testing.T) { placementProto := &placementpb.Placement{ Instances: map[string]*placementpb.Instance{ - "host1": &placementpb.Instance{ - Id: "host1", - IsolationGroup: "cluster", + "localhost": &placementpb.Instance{ + Id: "localhost", + IsolationGroup: "local", Zone: "embedded", Weight: 1, - Endpoint: "http://host1:9000", - Hostname: "host1", + Endpoint: "http://localhost:9000", + Hostname: "localhost", Port: 9000, }, - "host2": &placementpb.Instance{ - Id: "host2", - IsolationGroup: "cluster", + }, + } + newPlacement, err := placement.NewPlacementFromProto(placementProto) + require.NoError(t, err) + mockPlacementService.EXPECT().BuildInitialPlacement(gomock.Any(), 64, 1).Return(newPlacement, nil) + + createHandler.ServeHTTP(w, req) + + resp := w.Result() + body, err := ioutil.ReadAll(resp.Body) + assert.NoError(t, err) + assert.Equal(t, http.StatusOK, resp.StatusCode) + + expectedResponse := ` + { + "namespace": { + "registry": { + "namespaces": { + "testNamespace": { + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": true, + "repairEnabled": false, + "retentionOptions": { + "retentionPeriodNanos": "86400000000000", + "blockSizeNanos": "10800000000000", + "bufferFutureNanos": "120000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "300000000000" + }, + "snapshotEnabled": false, + "indexOptions": { + "enabled": true, + "blockSizeNanos": "10800000000000" + } + } + } + } + }, + "placement": { + "placement": { + "instances": { + "localhost": { + "id": "localhost", + "isolationGroup": "local", + "zone": "embedded", + "weight": 1, + "endpoint": "http://localhost:9000", + "shards": [], + "shardSetId": 0, + "hostname": "localhost", + "port": 9000 + } + }, + "replicaFactor": 0, + "numShards": 0, + "isSharded": false, + "cutoverTime": "0", + "isMirrored": false, + "maxShardSetId": 0 + }, + "version": 0 + } + } + ` + assert.Equal(t, stripAllWhitespace(expectedResponse), string(body), + xtest.Diff(mustPrettyJSON(t, expectedResponse), mustPrettyJSON(t, string(body)))) +} + +func TestLocalWithBlockSizeExpectedSeriesDatapointsPerHour(t *testing.T) { + mockClient, mockKV, mockPlacementService := SetupDatabaseTest(t) + createHandler := NewCreateHandler(mockClient, config.Configuration{}, testDBCfg) + w := httptest.NewRecorder() + + min := minRecommendCalculateBlockSize + desiredBlockSize := min + 5*time.Minute + + jsonInput := fmt.Sprintf(` + { + "namespaceName": "testNamespace", + "type": "local", + "blockSize": {"expectedSeriesDatapointsPerHour": %d} + } + `, int64(float64(blockSizeFromExpectedSeriesScalar)/float64(desiredBlockSize))) + + req := httptest.NewRequest("POST", "/database/create", strings.NewReader(jsonInput)) + require.NotNil(t, req) + + mockKV.EXPECT().Get(namespace.M3DBNodeNamespacesKey).Return(nil, kv.ErrNotFound) + mockKV.EXPECT().CheckAndSet(namespace.M3DBNodeNamespacesKey, gomock.Any(), gomock.Not(nil)).Return(1, nil) + + placementProto := &placementpb.Placement{ + Instances: map[string]*placementpb.Instance{ + "localhost": &placementpb.Instance{ + Id: "localhost", + IsolationGroup: "local", Zone: "embedded", Weight: 1, - Endpoint: "http://host2:9000", - Hostname: "host2", + Endpoint: "http://localhost:9000", + Hostname: "localhost", Port: 9000, }, }, } newPlacement, err := placement.NewPlacementFromProto(placementProto) require.NoError(t, err) - mockPlacementService.EXPECT().BuildInitialPlacement(gomock.Any(), 128, 3).Return(newPlacement, nil) + mockPlacementService.EXPECT().BuildInitialPlacement(gomock.Any(), 64, 1).Return(newPlacement, nil) createHandler.ServeHTTP(w, req) @@ -165,10 +321,67 @@ func TestClusterTypeHostnames(t *testing.T) { body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"namespace\":{\"registry\":{\"namespaces\":{\"testNamespace\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":true,\"repairEnabled\":false,\"retentionOptions\":{\"retentionPeriodNanos\":\"172800000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"120000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"300000000000\"},\"snapshotEnabled\":false,\"indexOptions\":{\"enabled\":false,\"blockSizeNanos\":\"7200000000000\"}}}}},\"placement\":{\"placement\":{\"instances\":{\"host1\":{\"id\":\"host1\",\"isolationGroup\":\"cluster\",\"zone\":\"embedded\",\"weight\":1,\"endpoint\":\"http://host1:9000\",\"shards\":[],\"shardSetId\":0,\"hostname\":\"host1\",\"port\":9000},\"host2\":{\"id\":\"host2\",\"isolationGroup\":\"cluster\",\"zone\":\"embedded\",\"weight\":1,\"endpoint\":\"http://host2:9000\",\"shards\":[],\"shardSetId\":0,\"hostname\":\"host2\",\"port\":9000}},\"replicaFactor\":0,\"numShards\":0,\"isSharded\":false,\"cutoverTime\":\"0\",\"isMirrored\":false,\"maxShardSetId\":0},\"version\":0}}", string(body)) + + expectedResponse := fmt.Sprintf(` + { + "namespace": { + "registry": { + "namespaces": { + "testNamespace": { + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": true, + "repairEnabled": false, + "retentionOptions": { + "retentionPeriodNanos": "86400000000000", + "blockSizeNanos": "%d", + "bufferFutureNanos": "120000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "300000000000" + }, + "snapshotEnabled": false, + "indexOptions": { + "enabled": true, + "blockSizeNanos": "%d" + } + } + } + } + }, + "placement": { + "placement": { + "instances": { + "localhost": { + "id": "localhost", + "isolationGroup": "local", + "zone": "embedded", + "weight": 1, + "endpoint": "http://localhost:9000", + "shards": [], + "shardSetId": 0, + "hostname": "localhost", + "port": 9000 + } + }, + "replicaFactor": 0, + "numShards": 0, + "isSharded": false, + "cutoverTime": "0", + "isMirrored": false, + "maxShardSetId": 0 + }, + "version": 0 + } + } + `, desiredBlockSize, desiredBlockSize) + + assert.Equal(t, stripAllWhitespace(expectedResponse), string(body), + xtest.Diff(mustPrettyJSON(t, expectedResponse), mustPrettyJSON(t, string(body)))) } -func TestClusterTypeHostnameGroups(t *testing.T) { +func TestClusterTypeHosts(t *testing.T) { mockClient, mockKV, mockPlacementService := SetupDatabaseTest(t) createHandler := NewCreateHandler(mockClient, config.Configuration{}, testDBCfg) w := httptest.NewRecorder() @@ -177,7 +390,7 @@ func TestClusterTypeHostnameGroups(t *testing.T) { { "namespaceName": "testNamespace", "type": "cluster", - "hostnameGroups": [{"hostname":"host1", "isolationGroup":"group1"}, {"hostname":"host2", "isolationGroup":"group2"}] + "hosts": [{"id": "host1"}, {"id": "host2"}] } ` @@ -191,7 +404,7 @@ func TestClusterTypeHostnameGroups(t *testing.T) { Instances: map[string]*placementpb.Instance{ "host1": &placementpb.Instance{ Id: "host1", - IsolationGroup: "group1", + IsolationGroup: "cluster", Zone: "embedded", Weight: 1, Endpoint: "http://host1:9000", @@ -200,7 +413,7 @@ func TestClusterTypeHostnameGroups(t *testing.T) { }, "host2": &placementpb.Instance{ Id: "host2", - IsolationGroup: "group2", + IsolationGroup: "cluster", Zone: "embedded", Weight: 1, Endpoint: "http://host2:9000", @@ -219,11 +432,78 @@ func TestClusterTypeHostnameGroups(t *testing.T) { body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Equal(t, "{\"namespace\":{\"registry\":{\"namespaces\":{\"testNamespace\":{\"bootstrapEnabled\":true,\"flushEnabled\":true,\"writesToCommitLog\":true,\"cleanupEnabled\":true,\"repairEnabled\":false,\"retentionOptions\":{\"retentionPeriodNanos\":\"172800000000000\",\"blockSizeNanos\":\"7200000000000\",\"bufferFutureNanos\":\"120000000000\",\"bufferPastNanos\":\"600000000000\",\"blockDataExpiry\":true,\"blockDataExpiryAfterNotAccessPeriodNanos\":\"300000000000\"},\"snapshotEnabled\":false,\"indexOptions\":{\"enabled\":false,\"blockSizeNanos\":\"7200000000000\"}}}}},\"placement\":{\"placement\":{\"instances\":{\"host1\":{\"id\":\"host1\",\"isolationGroup\":\"group1\",\"zone\":\"embedded\",\"weight\":1,\"endpoint\":\"http://host1:9000\",\"shards\":[],\"shardSetId\":0,\"hostname\":\"host1\",\"port\":9000},\"host2\":{\"id\":\"host2\",\"isolationGroup\":\"group2\",\"zone\":\"embedded\",\"weight\":1,\"endpoint\":\"http://host2:9000\",\"shards\":[],\"shardSetId\":0,\"hostname\":\"host2\",\"port\":9000}},\"replicaFactor\":0,\"numShards\":0,\"isSharded\":false,\"cutoverTime\":\"0\",\"isMirrored\":false,\"maxShardSetId\":0},\"version\":0}}", string(body)) + + expectedResponse := ` + { + "namespace": { + "registry": { + "namespaces": { + "testNamespace": { + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": true, + "repairEnabled": false, + "retentionOptions": { + "retentionPeriodNanos": "86400000000000", + "blockSizeNanos": "3600000000000", + "bufferFutureNanos": "120000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "300000000000" + }, + "snapshotEnabled": false, + "indexOptions": { + "enabled": true, + "blockSizeNanos": "3600000000000" + } + } + } + } + }, + "placement": { + "placement": { + "instances": { + "host1": { + "id": "host1", + "isolationGroup": "cluster", + "zone": "embedded", + "weight": 1, + "endpoint": "http://host1:9000", + "shards": [], + "shardSetId": 0, + "hostname": "host1", + "port": 9000 + }, + "host2": { + "id": "host2", + "isolationGroup": "cluster", + "zone": "embedded", + "weight": 1, + "endpoint": "http://host2:9000", + "shards": [], + "shardSetId": 0, + "hostname": "host2", + "port": 9000 + } + }, + "replicaFactor": 0, + "numShards": 0, + "isSharded": false, + "cutoverTime": "0", + "isMirrored": false, + "maxShardSetId": 0 + }, + "version": 0 + } + } + ` + assert.Equal(t, stripAllWhitespace(expectedResponse), string(body), + xtest.Diff(mustPrettyJSON(t, expectedResponse), mustPrettyJSON(t, string(body)))) } -func TestClusterTypeMultipleHostnameTypes(t *testing.T) { - mockClient, _, _ := SetupDatabaseTest(t) +func TestClusterTypeHostsWithIsolationGroup(t *testing.T) { + mockClient, mockKV, mockPlacementService := SetupDatabaseTest(t) createHandler := NewCreateHandler(mockClient, config.Configuration{}, testDBCfg) w := httptest.NewRecorder() @@ -231,23 +511,117 @@ func TestClusterTypeMultipleHostnameTypes(t *testing.T) { { "namespaceName": "testNamespace", "type": "cluster", - "hostnameGroups": [{"hostname":"host1", "isolationGroup":"group1"}, {"hostname":"host2", "isolationGroup":"group2"}], - "hostnames": ["other_host1"] + "hosts": [{"id":"host1", "isolationGroup":"group1"}, {"id":"host2", "isolationGroup":"group2"}] } ` req := httptest.NewRequest("POST", "/database/create", strings.NewReader(jsonInput)) require.NotNil(t, req) + mockKV.EXPECT().Get(namespace.M3DBNodeNamespacesKey).Return(nil, kv.ErrNotFound) + mockKV.EXPECT().CheckAndSet(namespace.M3DBNodeNamespacesKey, gomock.Any(), gomock.Not(nil)).Return(1, nil) + + placementProto := &placementpb.Placement{ + Instances: map[string]*placementpb.Instance{ + "host1": &placementpb.Instance{ + Id: "host1", + IsolationGroup: "group1", + Zone: "embedded", + Weight: 1, + Endpoint: "http://host1:9000", + Hostname: "host1", + Port: 9000, + }, + "host2": &placementpb.Instance{ + Id: "host2", + IsolationGroup: "group2", + Zone: "embedded", + Weight: 1, + Endpoint: "http://host2:9000", + Hostname: "host2", + Port: 9000, + }, + }, + } + newPlacement, err := placement.NewPlacementFromProto(placementProto) + require.NoError(t, err) + mockPlacementService.EXPECT().BuildInitialPlacement(gomock.Any(), 128, 3).Return(newPlacement, nil) + createHandler.ServeHTTP(w, req) resp := w.Result() body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) - assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, "{\"error\":\"must only specify one type of host\"}\n", string(body)) -} + assert.Equal(t, http.StatusOK, resp.StatusCode) + expectedResponse := ` + { + "namespace": { + "registry": { + "namespaces": { + "testNamespace": { + "bootstrapEnabled": true, + "flushEnabled": true, + "writesToCommitLog": true, + "cleanupEnabled": true, + "repairEnabled": false, + "retentionOptions": { + "retentionPeriodNanos": "86400000000000", + "blockSizeNanos": "3600000000000", + "bufferFutureNanos": "120000000000", + "bufferPastNanos": "600000000000", + "blockDataExpiry": true, + "blockDataExpiryAfterNotAccessPeriodNanos": "300000000000" + }, + "snapshotEnabled": false, + "indexOptions": { + "enabled": true, + "blockSizeNanos": "3600000000000" + } + } + } + } + }, + "placement": { + "placement": { + "instances": { + "host1": { + "id": "host1", + "isolationGroup": "group1", + "zone": "embedded", + "weight": 1, + "endpoint": "http://host1:9000", + "shards": [], + "shardSetId": 0, + "hostname": "host1", + "port": 9000 + }, + "host2": { + "id": "host2", + "isolationGroup": "group2", + "zone": "embedded", + "weight": 1, + "endpoint": "http://host2:9000", + "shards": [], + "shardSetId": 0, + "hostname": "host2", + "port": 9000 + } + }, + "replicaFactor": 0, + "numShards": 0, + "isSharded": false, + "cutoverTime": "0", + "isMirrored": false, + "maxShardSetId": 0 + }, + "version": 0 + } + } + ` + assert.Equal(t, stripAllWhitespace(expectedResponse), string(body), + xtest.Diff(mustPrettyJSON(t, expectedResponse), mustPrettyJSON(t, string(body)))) +} func TestClusterTypeMissingHostnames(t *testing.T) { mockClient, _, _ := SetupDatabaseTest(t) createHandler := NewCreateHandler(mockClient, config.Configuration{}, testDBCfg) @@ -269,12 +643,12 @@ func TestClusterTypeMissingHostnames(t *testing.T) { body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, "{\"error\":\"missing required field\"}\n", string(body)) + assert.Equal(t, withEndline(`{"error":"missing required field"}`), string(body)) } func TestBadType(t *testing.T) { mockClient, _, _ := SetupDatabaseTest(t) - createHandler := NewCreateHandler(mockClient, config.Configuration{}, dbconfig.DBConfiguration{}) + createHandler := NewCreateHandler(mockClient, config.Configuration{}, nil) w := httptest.NewRecorder() jsonInput := ` @@ -291,5 +665,27 @@ func TestBadType(t *testing.T) { body, err := ioutil.ReadAll(resp.Body) assert.NoError(t, err) assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, "{\"error\":\"invalid database type\"}\n", string(body)) + assert.Equal(t, withEndline(`{"error":"invalid database type"}`), string(body)) +} + +func stripAllWhitespace(str string) string { + return strings.Map(func(r rune) rune { + if unicode.IsSpace(r) { + return -1 + } + return r + }, str) +} + +func mustPrettyJSON(t *testing.T, str string) string { + var unmarshalled map[string]interface{} + err := json.Unmarshal([]byte(str), &unmarshalled) + require.NoError(t, err) + pretty, err := json.MarshalIndent(unmarshalled, "", " ") + require.NoError(t, err) + return string(pretty) +} + +func withEndline(str string) string { + return str + "\n" } diff --git a/src/coordinator/api/v1/handler/placement/add.go b/src/coordinator/api/v1/handler/placement/add.go index d38b182a00..503101f8b3 100644 --- a/src/coordinator/api/v1/handler/placement/add.go +++ b/src/coordinator/api/v1/handler/placement/add.go @@ -60,7 +60,7 @@ func (h *AddHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - placement, err := h.Add(req) + placement, err := h.Add(r, req) if err != nil { logger.Error("unable to add placement", zap.Any("error", err)) handler.Error(w, err, http.StatusInternalServerError) @@ -92,13 +92,16 @@ func (h *AddHandler) parseRequest(r *http.Request) (*admin.PlacementAddRequest, } // Add adds a placement. -func (h *AddHandler) Add(r *admin.PlacementAddRequest) (placement.Placement, error) { - instances, err := ConvertInstancesProto(r.Instances) +func (h *AddHandler) Add( + httpReq *http.Request, + req *admin.PlacementAddRequest, +) (placement.Placement, error) { + instances, err := ConvertInstancesProto(req.Instances) if err != nil { return nil, err } - service, err := Service(h.client, h.cfg) + service, err := Service(h.client, httpReq.Header) if err != nil { return nil, err } diff --git a/src/coordinator/api/v1/handler/placement/common.go b/src/coordinator/api/v1/handler/placement/common.go index a494e7779d..eb3588bcda 100644 --- a/src/coordinator/api/v1/handler/placement/common.go +++ b/src/coordinator/api/v1/handler/placement/common.go @@ -21,6 +21,9 @@ package placement import ( + "net/http" + "strings" + clusterclient "github.com/m3db/m3cluster/client" "github.com/m3db/m3cluster/generated/proto/placementpb" "github.com/m3db/m3cluster/placement" @@ -35,12 +38,16 @@ import ( const ( // DefaultServiceName is the default service ID name DefaultServiceName = "m3db" - // DefaultServiceEnvironment is the default service ID environment DefaultServiceEnvironment = "default_env" - // DefaultServiceZone is the default service ID zone DefaultServiceZone = "embedded" + // HeaderClusterServiceName is the header used to specify the service name. + HeaderClusterServiceName = "Cluster-Service-Name" + // HeaderClusterEnvironmentName is the header used to specify the environment name. + HeaderClusterEnvironmentName = "Cluster-Environment-Name" + // HeaderClusterZoneName is the header used to specify the zone name. + HeaderClusterZoneName = "Cluster-Zone-Name" ) // Handler represents a generic handler for placement endpoints. @@ -52,22 +59,25 @@ type Handler struct { } // Service gets a placement service from m3cluster client -func Service(clusterClient clusterclient.Client, cfg config.Configuration) (placement.Service, error) { +func Service(clusterClient clusterclient.Client, headers http.Header) (placement.Service, error) { cs, err := clusterClient.Services(services.NewOverrideOptions()) if err != nil { return nil, err } serviceName := DefaultServiceName + if v := strings.TrimSpace(headers.Get(HeaderClusterServiceName)); v != "" { + serviceName = v + } + serviceEnvironment := DefaultServiceEnvironment - serviceZone := DefaultServiceZone + if v := strings.TrimSpace(headers.Get(HeaderClusterEnvironmentName)); v != "" { + serviceEnvironment = v + } - if dbClientCfg := cfg.DBClient; dbClientCfg != nil { - if service := dbClientCfg.EnvironmentConfig.Service; service != nil { - serviceName = service.Service - serviceEnvironment = service.Env - serviceZone = service.Zone - } + serviceZone := DefaultServiceZone + if v := strings.TrimSpace(headers.Get(HeaderClusterZoneName)); v != "" { + serviceZone = v } sid := services.NewServiceID(). diff --git a/src/coordinator/api/v1/handler/placement/common_test.go b/src/coordinator/api/v1/handler/placement/common_test.go index 6d424cd989..bf7bdbf8ca 100644 --- a/src/coordinator/api/v1/handler/placement/common_test.go +++ b/src/coordinator/api/v1/handler/placement/common_test.go @@ -22,13 +22,13 @@ package placement import ( "errors" + "net/http" "testing" "github.com/m3db/m3cluster/client" "github.com/m3db/m3cluster/generated/proto/placementpb" "github.com/m3db/m3cluster/placement" "github.com/m3db/m3cluster/services" - "github.com/m3db/m3db/src/cmd/services/m3coordinator/config" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" @@ -38,6 +38,7 @@ import ( func TestPlacementService(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + mockClient := client.NewMockClient(ctrl) require.NotNil(t, mockClient) mockServices := services.NewMockServices(ctrl) @@ -48,24 +49,67 @@ func TestPlacementService(t *testing.T) { mockClient.EXPECT().Services(gomock.Not(nil)).Return(mockServices, nil) mockServices.EXPECT().PlacementService(gomock.Not(nil), gomock.Not(nil)).Return(mockPlacementService, nil) - placementService, err := Service(mockClient, config.Configuration{}) + placementService, err := Service(mockClient, http.Header{}) assert.NoError(t, err) assert.NotNil(t, placementService) // Test Services returns error mockClient.EXPECT().Services(gomock.Not(nil)).Return(nil, errors.New("dummy service error")) - placementService, err = Service(mockClient, config.Configuration{}) + placementService, err = Service(mockClient, http.Header{}) assert.Nil(t, placementService) assert.EqualError(t, err, "dummy service error") // Test PlacementService returns error mockClient.EXPECT().Services(gomock.Not(nil)).Return(mockServices, nil) mockServices.EXPECT().PlacementService(gomock.Not(nil), gomock.Not(nil)).Return(nil, errors.New("dummy placement error")) - placementService, err = Service(mockClient, config.Configuration{}) + placementService, err = Service(mockClient, http.Header{}) assert.Nil(t, placementService) assert.EqualError(t, err, "dummy placement error") } +func TestPlacementServiceWithClusterHeaders(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockClient := client.NewMockClient(ctrl) + require.NotNil(t, mockClient) + mockServices := services.NewMockServices(ctrl) + require.NotNil(t, mockServices) + mockPlacementService := placement.NewMockService(ctrl) + require.NotNil(t, mockPlacementService) + + mockClient.EXPECT().Services(gomock.Not(nil)).Return(mockServices, nil) + + var actual services.ServiceID + mockServices.EXPECT().PlacementService(gomock.Not(nil), gomock.Not(nil)). + DoAndReturn(func( + serviceID services.ServiceID, + opts placement.Options, + ) (placement.Service, error) { + actual = serviceID + return mockPlacementService, nil + }) + + var ( + serviceValue = "foo_svc" + environmentValue = "bar_env" + zoneValue = "baz_zone" + headers = http.Header{} + ) + headers.Add(HeaderClusterServiceName, serviceValue) + headers.Add(HeaderClusterEnvironmentName, environmentValue) + headers.Add(HeaderClusterZoneName, zoneValue) + + placementService, err := Service(mockClient, headers) + assert.NoError(t, err) + assert.NotNil(t, placementService) + + require.NotNil(t, actual) + assert.Equal(t, serviceValue, actual.Name()) + assert.Equal(t, environmentValue, actual.Environment()) + assert.Equal(t, zoneValue, actual.Zone()) +} + func TestConvertInstancesProto(t *testing.T) { instances, err := ConvertInstancesProto([]*placementpb.Instance{}) require.NoError(t, err) diff --git a/src/coordinator/api/v1/handler/placement/delete.go b/src/coordinator/api/v1/handler/placement/delete.go index e739e406b5..dd00e49603 100644 --- a/src/coordinator/api/v1/handler/placement/delete.go +++ b/src/coordinator/api/v1/handler/placement/delete.go @@ -67,7 +67,7 @@ func (h *DeleteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - service, err := Service(h.client, h.cfg) + service, err := Service(h.client, r.Header) if err != nil { handler.Error(w, err, http.StatusInternalServerError) return diff --git a/src/coordinator/api/v1/handler/placement/delete_all.go b/src/coordinator/api/v1/handler/placement/delete_all.go index e5fde04b63..f4f8585150 100644 --- a/src/coordinator/api/v1/handler/placement/delete_all.go +++ b/src/coordinator/api/v1/handler/placement/delete_all.go @@ -52,7 +52,7 @@ func (h *DeleteAllHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { ctx := r.Context() logger := logging.WithContext(ctx) - service, err := Service(h.client, h.cfg) + service, err := Service(h.client, r.Header) if err != nil { handler.Error(w, err, http.StatusInternalServerError) return diff --git a/src/coordinator/api/v1/handler/placement/get.go b/src/coordinator/api/v1/handler/placement/get.go index 4de788d757..90aa8555b5 100644 --- a/src/coordinator/api/v1/handler/placement/get.go +++ b/src/coordinator/api/v1/handler/placement/get.go @@ -53,7 +53,7 @@ func (h *GetHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { ctx := r.Context() logger := logging.WithContext(ctx) - service, err := Service(h.client, h.cfg) + service, err := Service(h.client, r.Header) if err != nil { handler.Error(w, err, http.StatusInternalServerError) return diff --git a/src/coordinator/api/v1/handler/placement/init.go b/src/coordinator/api/v1/handler/placement/init.go index d58c880420..2bb9c6467d 100644 --- a/src/coordinator/api/v1/handler/placement/init.go +++ b/src/coordinator/api/v1/handler/placement/init.go @@ -60,7 +60,7 @@ func (h *InitHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - placement, err := h.Init(req) + placement, err := h.Init(r, req) if err != nil { logger.Error("unable to initialize placement", zap.Any("error", err)) handler.Error(w, err, http.StatusInternalServerError) @@ -92,18 +92,22 @@ func (h *InitHandler) parseRequest(r *http.Request) (*admin.PlacementInitRequest } // Init initializes a placement. -func (h *InitHandler) Init(r *admin.PlacementInitRequest) (placement.Placement, error) { - instances, err := ConvertInstancesProto(r.Instances) +func (h *InitHandler) Init( + httpReq *http.Request, + req *admin.PlacementInitRequest, +) (placement.Placement, error) { + instances, err := ConvertInstancesProto(req.Instances) if err != nil { return nil, err } - service, err := Service(h.client, h.cfg) + service, err := Service(h.client, httpReq.Header) if err != nil { return nil, err } - placement, err := service.BuildInitialPlacement(instances, int(r.NumShards), int(r.ReplicationFactor)) + placement, err := service.BuildInitialPlacement(instances, + int(req.NumShards), int(req.ReplicationFactor)) if err != nil { return nil, err } diff --git a/src/coordinator/api/v1/handler/prometheus/native/read_test.go b/src/coordinator/api/v1/handler/prometheus/native/read_test.go index e6ef778ee0..fe79287bd7 100644 --- a/src/coordinator/api/v1/handler/prometheus/native/read_test.go +++ b/src/coordinator/api/v1/handler/prometheus/native/read_test.go @@ -38,6 +38,7 @@ import ( func TestPromRead(t *testing.T) { logging.InitWithCores(nil) + values, bounds := test.GenerateValuesAndBounds(nil, nil) b := test.NewBlockFromValues(bounds, values) mockStorage := mock.NewMockStorageWithBlocks([]block.Block{b}) @@ -52,6 +53,7 @@ func TestPromRead(t *testing.T) { require.NoError(t, err) require.Len(t, seriesList, 2) s := seriesList[0] + assert.Equal(t, 5, s.Values().Len()) for i := 0; i < s.Values().Len(); i++ { assert.Equal(t, float64(i), s.Values().ValueAt(i)) diff --git a/src/coordinator/api/v1/handler/prometheus/remote/read_test.go b/src/coordinator/api/v1/handler/prometheus/remote/read_test.go index aff1b33ce2..0bd83f5ae3 100644 --- a/src/coordinator/api/v1/handler/prometheus/remote/read_test.go +++ b/src/coordinator/api/v1/handler/prometheus/remote/read_test.go @@ -83,7 +83,7 @@ func setupServer(t *testing.T) *httptest.Server { logging.InitWithCores(nil) ctrl := gomock.NewController(t) // No calls expected on session object - lstore, session := local.NewStorageAndSession(ctrl) + lstore, session := local.NewStorageAndSession(t, ctrl) session.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, false, fmt.Errorf("not initialized")) storage := test.NewSlowStorage(lstore, 10*time.Millisecond) engine := executor.NewEngine(storage) @@ -95,7 +95,7 @@ func setupServer(t *testing.T) *httptest.Server { func TestPromReadParsing(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) promRead := &PromReadHandler{engine: executor.NewEngine(storage), promReadMetrics: promReadTestMetrics} req, _ := http.NewRequest("POST", PromReadURL, generatePromReadBody(t)) @@ -107,7 +107,7 @@ func TestPromReadParsing(t *testing.T) { func TestPromReadParsingBad(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) promRead := &PromReadHandler{engine: executor.NewEngine(storage), promReadMetrics: promReadTestMetrics} req, _ := http.NewRequest("POST", PromReadURL, strings.NewReader("bad body")) _, err := promRead.parseRequest(req) @@ -117,7 +117,7 @@ func TestPromReadParsingBad(t *testing.T) { func TestPromReadStorageWithFetchError(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, session := local.NewStorageAndSession(ctrl) + storage, session := local.NewStorageAndSession(t, ctrl) session.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, true, fmt.Errorf("unable to get data")) promRead := &PromReadHandler{engine: executor.NewEngine(storage), promReadMetrics: promReadTestMetrics} req := generatePromReadRequest() @@ -165,7 +165,7 @@ func TestQueryKillOnTimeout(t *testing.T) { func TestReadErrorMetricsCount(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, session := local.NewStorageAndSession(ctrl) + storage, session := local.NewStorageAndSession(t, ctrl) session.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, true, fmt.Errorf("unable to get data")) reporter := xmetrics.NewTestStatsReporter(xmetrics.NewTestStatsReporterOptions()) diff --git a/src/coordinator/api/v1/handler/prometheus/remote/write_test.go b/src/coordinator/api/v1/handler/prometheus/remote/write_test.go index 93c6396904..2404da206a 100644 --- a/src/coordinator/api/v1/handler/prometheus/remote/write_test.go +++ b/src/coordinator/api/v1/handler/prometheus/remote/write_test.go @@ -83,7 +83,7 @@ func generatePromWriteBody(t *testing.T) io.Reader { func TestPromWriteParsing(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) promWrite := &PromWriteHandler{store: storage} @@ -98,7 +98,7 @@ func TestPromWrite(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, session := local.NewStorageAndSession(ctrl) + storage, session := local.NewStorageAndSession(t, ctrl) session.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() promWrite := &PromWriteHandler{store: storage} @@ -116,7 +116,7 @@ func TestWriteErrorMetricCount(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - storage, session := local.NewStorageAndSession(ctrl) + storage, session := local.NewStorageAndSession(t, ctrl) session.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() reporter := xmetrics.NewTestStatsReporter(xmetrics.NewTestStatsReporterOptions()) diff --git a/src/coordinator/api/v1/handler/search_test.go b/src/coordinator/api/v1/handler/search_test.go index a2b7b8b527..aa3af63572 100644 --- a/src/coordinator/api/v1/handler/search_test.go +++ b/src/coordinator/api/v1/handler/search_test.go @@ -83,6 +83,8 @@ func generateTagIters(ctrl *gomock.Controller) *client.MockTaggedIDsIterator { mockTaggedIDsIter.EXPECT().Next().Return(false) mockTaggedIDsIter.EXPECT().Current().Return(ident.StringID(testNamespace), ident.StringID(testID), seriesiter.GenerateSingleSampleTagIterator(ctrl, seriesiter.GenerateTag())) + mockTaggedIDsIter.EXPECT().Err().Return(nil) + mockTaggedIDsIter.EXPECT().Finalize() return mockTaggedIDsIter } @@ -93,10 +95,11 @@ func searchServer(t *testing.T) *SearchHandler { mockTaggedIDsIter := generateTagIters(ctrl) - storage, session := local.NewStorageAndSession(ctrl) - session.EXPECT().FetchTaggedIDs(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockTaggedIDsIter, false, nil) - search := &SearchHandler{store: storage} + storage, session := local.NewStorageAndSession(t, ctrl) + session.EXPECT().FetchTaggedIDs(gomock.Any(), gomock.Any(), gomock.Any()). + Return(mockTaggedIDsIter, false, nil) + search := &SearchHandler{store: storage} return search } diff --git a/src/coordinator/api/v1/httpd/handler.go b/src/coordinator/api/v1/httpd/handler.go index 7fc997a1b0..cf5b80efe2 100644 --- a/src/coordinator/api/v1/httpd/handler.go +++ b/src/coordinator/api/v1/httpd/handler.go @@ -25,7 +25,7 @@ import ( "net/http/pprof" "os" - m3clusterClient "github.com/m3db/m3cluster/client" + clusterclient "github.com/m3db/m3cluster/client" "github.com/m3db/m3db/src/cmd/services/m3coordinator/config" dbconfig "github.com/m3db/m3db/src/cmd/services/m3dbnode/config" "github.com/m3db/m3db/src/coordinator/api/v1/handler" @@ -58,15 +58,21 @@ type Handler struct { CLFLogger *log.Logger storage storage.Storage engine *executor.Engine - clusterClient m3clusterClient.Client + clusterClient clusterclient.Client config config.Configuration - dbConfig dbconfig.DBConfiguration + embeddedDbCfg *dbconfig.DBConfiguration scope tally.Scope } // NewHandler returns a new instance of handler with routes. -func NewHandler(storage storage.Storage, engine *executor.Engine, clusterClient m3clusterClient.Client, - cfg config.Configuration, dbCfg dbconfig.DBConfiguration, scope tally.Scope) (*Handler, error) { +func NewHandler( + storage storage.Storage, + engine *executor.Engine, + clusterClient clusterclient.Client, + cfg config.Configuration, + embeddedDbCfg *dbconfig.DBConfiguration, + scope tally.Scope, +) (*Handler, error) { r := mux.NewRouter() logger, err := zap.NewProduction() if err != nil { @@ -81,7 +87,7 @@ func NewHandler(storage storage.Storage, engine *executor.Engine, clusterClient engine: engine, clusterClient: clusterClient, config: cfg, - dbConfig: dbCfg, + embeddedDbCfg: embeddedDbCfg, scope: scope, } return h, nil @@ -104,7 +110,7 @@ func (h *Handler) RegisterRoutes() error { if h.clusterClient != nil { placement.RegisterRoutes(h.Router, h.clusterClient, h.config) namespace.RegisterRoutes(h.Router, h.clusterClient) - database.RegisterRoutes(h.Router, h.clusterClient, h.config, h.dbConfig) + database.RegisterRoutes(h.Router, h.clusterClient, h.config, h.embeddedDbCfg) } return nil diff --git a/src/coordinator/api/v1/httpd/handler_test.go b/src/coordinator/api/v1/httpd/handler_test.go index 8a221b21bc..1d3324c0fe 100644 --- a/src/coordinator/api/v1/httpd/handler_test.go +++ b/src/coordinator/api/v1/httpd/handler_test.go @@ -26,7 +26,6 @@ import ( "testing" "github.com/m3db/m3db/src/cmd/services/m3coordinator/config" - dbconfig "github.com/m3db/m3db/src/cmd/services/m3dbnode/config" "github.com/m3db/m3db/src/coordinator/api/v1/handler/prometheus/native" "github.com/m3db/m3db/src/coordinator/api/v1/handler/prometheus/remote" "github.com/m3db/m3db/src/coordinator/executor" @@ -44,9 +43,9 @@ func TestPromRemoteReadGet(t *testing.T) { req, _ := http.NewRequest("GET", remote.PromReadURL, nil) res := httptest.NewRecorder() ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) - h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, dbconfig.DBConfiguration{}, tally.NewTestScope("", nil)) + h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, nil, tally.NewTestScope("", nil)) require.NoError(t, err, "unable to setup handler") err = h.RegisterRoutes() require.NoError(t, err, "unable to register routes") @@ -60,9 +59,9 @@ func TestPromRemoteReadPost(t *testing.T) { req, _ := http.NewRequest("POST", remote.PromReadURL, nil) res := httptest.NewRecorder() ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) - h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, dbconfig.DBConfiguration{}, tally.NewTestScope("", nil)) + h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, nil, tally.NewTestScope("", nil)) require.NoError(t, err, "unable to setup handler") err = h.RegisterRoutes() require.NoError(t, err, "unable to register routes") @@ -76,9 +75,9 @@ func TestPromNativeReadGet(t *testing.T) { req, _ := http.NewRequest("GET", native.PromReadURL, nil) res := httptest.NewRecorder() ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) - h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, dbconfig.DBConfiguration{}, tally.NewTestScope("", nil)) + h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, nil, tally.NewTestScope("", nil)) require.NoError(t, err, "unable to setup handler") h.RegisterRoutes() h.Router.ServeHTTP(res, req) @@ -91,9 +90,9 @@ func TestPromNativeReadPost(t *testing.T) { req, _ := http.NewRequest("POST", native.PromReadURL, nil) res := httptest.NewRecorder() ctrl := gomock.NewController(t) - storage, _ := local.NewStorageAndSession(ctrl) + storage, _ := local.NewStorageAndSession(t, ctrl) - h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, dbconfig.DBConfiguration{}, tally.NewTestScope("", nil)) + h, err := NewHandler(storage, executor.NewEngine(storage), nil, config.Configuration{}, nil, tally.NewTestScope("", nil)) require.NoError(t, err, "unable to setup handler") h.RegisterRoutes() h.Router.ServeHTTP(res, req) diff --git a/src/coordinator/benchmark/read/main/read_benchmark.go b/src/coordinator/benchmark/read/main/read_benchmark.go index 550154f508..187f2b0687 100644 --- a/src/coordinator/benchmark/read/main/read_benchmark.go +++ b/src/coordinator/benchmark/read/main/read_benchmark.go @@ -89,10 +89,10 @@ func main() { // Split on coord vs m3db if coordinator { - log.Println("Benchmarking reads over http endpoint m3coordinator...") + log.Println("benchmarking reads over http endpoint m3coordinator...") benchmarkCoordinator(start, end) } else { - log.Println("Benchmarking reads on m3db...") + log.Println("benchmarking reads on m3db...") benchmarkM3DB(start, end) } } @@ -106,24 +106,24 @@ func benchmarkCoordinator(start, end time.Time) { fetch := func() { r, err := common.PostEncodedSnappy(readEndpoint, promRead) if err != nil { - log.Fatalf("Unable to fetch metrics from m3coordinator, got error %v\n", err) + log.Fatalf("unable to fetch metrics from m3coordinator, got error %v\n", err) } readResponse = make([]byte, r.ContentLength) r.Body.Read(readResponse) r.Body.Close() if r.StatusCode != 200 { - log.Fatalf("HTTP read failed with code %d, error: %s", r.StatusCode, string(readResponse)) + log.Fatalf("http read failed with code %d, error: %s", r.StatusCode, string(readResponse)) } } count := func() int { reqBuf, err := snappy.Decode(nil, readResponse) if err != nil { - log.Fatalf("Unable to decode response, got error %v\n", err) + log.Fatalf("unable to decode response, got error %v\n", err) } var req prompb.ReadResponse if err := proto.Unmarshal(reqBuf, &req); err != nil { - log.Fatalf("Unable to unmarshal prompb response, got error %v\n", err) + log.Fatalf("unable to unmarshal prompb response, got error %v\n", err) } return req.Size() } @@ -134,10 +134,14 @@ func benchmarkCoordinator(start, end time.Time) { func benchmarkM3DB(start, end time.Time) { var cfg config.Configuration if err := xconfig.LoadFile(&cfg, m3dbClientCfg, configLoadOpts); err != nil { - log.Fatalf("Unable to load %s: %v", m3dbClientCfg, err) + log.Fatalf("unable to load %s: %v", m3dbClientCfg, err) } - m3dbClientOpts := cfg.DBClient - m3dbClient, err := m3dbClientOpts.NewClient(client.ConfigurationParameters{}, func(v client.Options) client.Options { + + if len(cfg.Clusters) != 1 { + log.Fatal("invalid config, expected single cluster definition") + } + + m3dbClient, err := cfg.Clusters[0].Client.NewClient(client.ConfigurationParameters{}, func(v client.Options) client.Options { return v.SetWriteBatchSize(batch).SetWriteOpPoolSize(batch * 2) }) if err != nil { @@ -146,7 +150,7 @@ func benchmarkM3DB(start, end time.Time) { session, err := m3dbClient.NewSession() if err != nil { - log.Fatalf("Unable to create m3db client session, got error %v\n", err) + log.Fatalf("unable to create m3db client session, got error %v\n", err) } ids := getUniqueIds() var rawResults encoding.SeriesIterators @@ -156,7 +160,7 @@ func benchmarkM3DB(start, end time.Time) { rawResults, err = session.FetchIDs(namespaceID, ident.NewStringIDsSliceIterator(ids), start, end) namespaceID.Finalize() if err != nil { - log.Fatalf("Unable to fetch metrics from m3db, got error %v\n", err) + log.Fatalf("unable to fetch metrics from m3db, got error %v\n", err) } } @@ -183,16 +187,16 @@ func getUniqueIds() []string { func genericBenchmarker(fetch func(), count countFunc) { start := time.Now() - log.Println("Started benchmark at:", start.Format(time.StampMilli)) + log.Println("started benchmark at:", start.Format(time.StampMilli)) fetch() end := time.Now() - log.Println("Finished benchmark at:", start.Format(time.StampMilli)) + log.Println("finished benchmark at:", start.Format(time.StampMilli)) took := end.Sub(start) // Counting should be done after timer has stopped in case any transforms are required results := count() rate := float64(results) / took.Seconds() - log.Printf("Returned %d timeseries in %fsec (mean values rate %f/sec)\n", results, took.Seconds(), rate) + log.Printf("returned %d timeseries in %fsec (mean values rate %f/sec)\n", results, took.Seconds(), rate) } func generateMatchers() []*prompb.LabelMatcher { @@ -223,7 +227,7 @@ func generatePromReadBody(start, end time.Time) io.Reader { req := generatePromReadRequest(start, end) data, err := proto.Marshal(req) if err != nil { - log.Fatalf("Unable to marshal request, got error %v\n", err) + log.Fatalf("unable to marshal request, got error %v\n", err) } compressed := snappy.Encode(nil, data) b := bytes.NewReader(compressed) diff --git a/src/coordinator/benchmark/write/main/write_benchmark.go b/src/coordinator/benchmark/write/main/write_benchmark.go index 82b2e64efa..4521a6ee86 100644 --- a/src/coordinator/benchmark/write/main/write_benchmark.go +++ b/src/coordinator/benchmark/write/main/write_benchmark.go @@ -75,10 +75,10 @@ func init() { func main() { if coordinator { - log.Println("Benchmarking writes on m3coordinator over http endpoint...") + log.Println("benchmarking writes on m3coordinator over http endpoint...") benchmarkCoordinator() } else { - log.Println("Benchmarking writes on m3db...") + log.Println("benchmarking writes on m3db...") benchmarkM3DB() } } @@ -96,20 +96,23 @@ func benchmarkM3DB() { var cfg config.Configuration if err := xconfig.LoadFile(&cfg, m3dbClientCfg, configLoadOpts); err != nil { - log.Fatalf("Unable to load %s: %v", m3dbClientCfg, err) + log.Fatalf("unable to load %s: %v", m3dbClientCfg, err) } - m3dbClientOpts := cfg.DBClient - m3dbClient, err := m3dbClientOpts.NewClient(client.ConfigurationParameters{}, func(v client.Options) client.Options { + if len(cfg.Clusters) != 1 { + log.Fatal("invalid config, expected single cluster definition") + } + + m3dbClient, err := cfg.Clusters[0].Client.NewClient(client.ConfigurationParameters{}, func(v client.Options) client.Options { return v.SetWriteBatchSize(batch).SetWriteOpPoolSize(batch * 2) }) if err != nil { - log.Fatalf("Unable to create m3db client, got error %v\n", err) + log.Fatalf("unable to create m3db client, got error %v\n", err) } session, err := m3dbClient.NewSession() if err != nil { - log.Fatalf("Unable to create m3db client session, got error %v\n", err) + log.Fatalf("unable to create m3db client session, got error %v\n", err) } workerFunction := func() { @@ -133,7 +136,7 @@ func benchmarkM3DB() { close(ch) wg.Wait() if err := session.Close(); err != nil { - log.Fatalf("Unable to close m3db client session, got error %v\n", err) + log.Fatalf("unable to close m3db client session, got error %v\n", err) } } @@ -210,7 +213,7 @@ func genericBenchmarker(workerFunction func(), appendReadCount func() int, clean itemsRead = appendReadCount() endNanosAtomic int64 ) - log.Println("Started benchmark at:", start.Format(time.StampMilli)) + log.Println("started benchmark at:", start.Format(time.StampMilli)) go func() { for { time.Sleep(time.Second) @@ -225,7 +228,7 @@ func genericBenchmarker(workerFunction func(), appendReadCount func() int, clean cleanup() end := time.Now() - log.Println("Finished benchmark at:", start.Format(time.StampMilli)) + log.Println("finished benchmark at:", start.Format(time.StampMilli)) took := end.Sub(start) atomic.StoreInt64(&endNanosAtomic, end.UnixNano()) rate := float64(itemsRead) / took.Seconds() diff --git a/src/coordinator/executor/engine_test.go b/src/coordinator/executor/engine_test.go index 0c8f17b9a5..5073e7e3b9 100644 --- a/src/coordinator/executor/engine_test.go +++ b/src/coordinator/executor/engine_test.go @@ -36,7 +36,7 @@ import ( func TestExecute(t *testing.T) { logging.InitWithCores(nil) ctrl := gomock.NewController(t) - store, session := local.NewStorageAndSession(ctrl) + store, session := local.NewStorageAndSession(t, ctrl) session.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, false, fmt.Errorf("dummy")) // Results is closed by execute diff --git a/src/coordinator/generated/proto/admin/database.pb.go b/src/coordinator/generated/proto/admin/database.pb.go index a73a838bc4..426060f96c 100644 --- a/src/coordinator/generated/proto/admin/database.pb.go +++ b/src/coordinator/generated/proto/admin/database.pb.go @@ -31,7 +31,8 @@ It has these top-level messages: DatabaseCreateRequest - HostnameGroup + BlockSize + Host DatabaseCreateResponse NamespaceGetResponse NamespaceAddRequest @@ -66,13 +67,12 @@ type DatabaseCreateRequest struct { NumShards int32 `protobuf:"varint,3,opt,name=num_shards,json=numShards,proto3" json:"num_shards,omitempty"` ReplicationFactor int32 `protobuf:"varint,4,opt,name=replication_factor,json=replicationFactor,proto3" json:"replication_factor,omitempty"` // The below two options are used to default retention options - RetentionPeriodNanos int64 `protobuf:"varint,5,opt,name=retention_period_nanos,json=retentionPeriodNanos,proto3" json:"retention_period_nanos,omitempty"` - ExpectedSeriesDatapointsPerHour int64 `protobuf:"varint,6,opt,name=expected_series_datapoints_per_hour,json=expectedSeriesDatapointsPerHour,proto3" json:"expected_series_datapoints_per_hour,omitempty"` - // Only one of the two below hostname types should be taken. This is enforced - // in application code instead of via protobuf oneof to reduce the amount of - // JSON nested structures required for input. - Hostnames []string `protobuf:"bytes,7,rep,name=hostnames" json:"hostnames,omitempty"` - HostnameGroups []*HostnameGroup `protobuf:"bytes,8,rep,name=hostname_groups,json=hostnameGroups" json:"hostname_groups,omitempty"` + RetentionPeriodNanos int64 `protobuf:"varint,5,opt,name=retention_period_nanos,json=retentionPeriodNanos,proto3" json:"retention_period_nanos,omitempty"` + // If no block size fields are set then the block size is + // derived from the length of the retention period + BlockSize *BlockSize `protobuf:"bytes,6,opt,name=block_size,json=blockSize" json:"block_size,omitempty"` + // Required if not using local database type + Hosts []*Host `protobuf:"bytes,7,rep,name=hosts" json:"hosts,omitempty"` } func (m *DatabaseCreateRequest) Reset() { *m = DatabaseCreateRequest{} } @@ -115,51 +115,111 @@ func (m *DatabaseCreateRequest) GetRetentionPeriodNanos() int64 { return 0 } -func (m *DatabaseCreateRequest) GetExpectedSeriesDatapointsPerHour() int64 { +func (m *DatabaseCreateRequest) GetBlockSize() *BlockSize { if m != nil { - return m.ExpectedSeriesDatapointsPerHour + return m.BlockSize } - return 0 + return nil } -func (m *DatabaseCreateRequest) GetHostnames() []string { +func (m *DatabaseCreateRequest) GetHosts() []*Host { if m != nil { - return m.Hostnames + return m.Hosts } return nil } -func (m *DatabaseCreateRequest) GetHostnameGroups() []*HostnameGroup { +type BlockSize struct { + // Explicit block size nanoseconds + Nanos int64 `protobuf:"varint,1,opt,name=nanos,proto3" json:"nanos,omitempty"` + // With the expected series datapoints per hour, use a recommended block size + ExpectedSeriesDatapointsPerHour int64 `protobuf:"varint,2,opt,name=expected_series_datapoints_per_hour,json=expectedSeriesDatapointsPerHour,proto3" json:"expected_series_datapoints_per_hour,omitempty"` +} + +func (m *BlockSize) Reset() { *m = BlockSize{} } +func (m *BlockSize) String() string { return proto.CompactTextString(m) } +func (*BlockSize) ProtoMessage() {} +func (*BlockSize) Descriptor() ([]byte, []int) { return fileDescriptorDatabase, []int{1} } + +func (m *BlockSize) GetNanos() int64 { if m != nil { - return m.HostnameGroups + return m.Nanos } - return nil + return 0 } -type HostnameGroup struct { - Hostname string `protobuf:"bytes,1,opt,name=hostname,proto3" json:"hostname,omitempty"` - IsolationGroup string `protobuf:"bytes,2,opt,name=isolation_group,json=isolationGroup,proto3" json:"isolation_group,omitempty"` +func (m *BlockSize) GetExpectedSeriesDatapointsPerHour() int64 { + if m != nil { + return m.ExpectedSeriesDatapointsPerHour + } + return 0 } -func (m *HostnameGroup) Reset() { *m = HostnameGroup{} } -func (m *HostnameGroup) String() string { return proto.CompactTextString(m) } -func (*HostnameGroup) ProtoMessage() {} -func (*HostnameGroup) Descriptor() ([]byte, []int) { return fileDescriptorDatabase, []int{1} } +type Host struct { + // Name (must be unique) of the host for logging/display purposes, can be hostname if desired, + // or UUID or any other string ID as used in third party inventory system + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // Address can be IP address or hostname, this is used to connect to the host + Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + // Port running the Node RPC listen address (currently TChannel/Thrift Node service) + Port uint32 `protobuf:"varint,3,opt,name=port,proto3" json:"port,omitempty"` + // (Optional) Isolation group is an optional grouping, for instance to isolate replicas by + // zones use zone here, or to isolate by host racks use racks here + IsolationGroup string `protobuf:"bytes,4,opt,name=isolation_group,json=isolationGroup,proto3" json:"isolation_group,omitempty"` + // (Optional) Zone specifies the zone the host resides in, this is optional. + Zone string `protobuf:"bytes,5,opt,name=zone,proto3" json:"zone,omitempty"` + // (Optional) Weight specifies when considering how many shards to take how to weight + // this instance, do not specify to default to a weight of 1. + Weight uint32 `protobuf:"varint,6,opt,name=weight,proto3" json:"weight,omitempty"` +} + +func (m *Host) Reset() { *m = Host{} } +func (m *Host) String() string { return proto.CompactTextString(m) } +func (*Host) ProtoMessage() {} +func (*Host) Descriptor() ([]byte, []int) { return fileDescriptorDatabase, []int{2} } + +func (m *Host) GetId() string { + if m != nil { + return m.Id + } + return "" +} -func (m *HostnameGroup) GetHostname() string { +func (m *Host) GetAddress() string { if m != nil { - return m.Hostname + return m.Address } return "" } -func (m *HostnameGroup) GetIsolationGroup() string { +func (m *Host) GetPort() uint32 { + if m != nil { + return m.Port + } + return 0 +} + +func (m *Host) GetIsolationGroup() string { if m != nil { return m.IsolationGroup } return "" } +func (m *Host) GetZone() string { + if m != nil { + return m.Zone + } + return "" +} + +func (m *Host) GetWeight() uint32 { + if m != nil { + return m.Weight + } + return 0 +} + type DatabaseCreateResponse struct { Namespace *NamespaceGetResponse `protobuf:"bytes,1,opt,name=namespace" json:"namespace,omitempty"` Placement *PlacementGetResponse `protobuf:"bytes,2,opt,name=placement" json:"placement,omitempty"` @@ -168,7 +228,7 @@ type DatabaseCreateResponse struct { func (m *DatabaseCreateResponse) Reset() { *m = DatabaseCreateResponse{} } func (m *DatabaseCreateResponse) String() string { return proto.CompactTextString(m) } func (*DatabaseCreateResponse) ProtoMessage() {} -func (*DatabaseCreateResponse) Descriptor() ([]byte, []int) { return fileDescriptorDatabase, []int{2} } +func (*DatabaseCreateResponse) Descriptor() ([]byte, []int) { return fileDescriptorDatabase, []int{3} } func (m *DatabaseCreateResponse) GetNamespace() *NamespaceGetResponse { if m != nil { @@ -186,7 +246,8 @@ func (m *DatabaseCreateResponse) GetPlacement() *PlacementGetResponse { func init() { proto.RegisterType((*DatabaseCreateRequest)(nil), "admin.DatabaseCreateRequest") - proto.RegisterType((*HostnameGroup)(nil), "admin.HostnameGroup") + proto.RegisterType((*BlockSize)(nil), "admin.BlockSize") + proto.RegisterType((*Host)(nil), "admin.Host") proto.RegisterType((*DatabaseCreateResponse)(nil), "admin.DatabaseCreateResponse") } func (m *DatabaseCreateRequest) Marshal() (dAtA []byte, err error) { @@ -231,29 +292,19 @@ func (m *DatabaseCreateRequest) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintDatabase(dAtA, i, uint64(m.RetentionPeriodNanos)) } - if m.ExpectedSeriesDatapointsPerHour != 0 { - dAtA[i] = 0x30 + if m.BlockSize != nil { + dAtA[i] = 0x32 i++ - i = encodeVarintDatabase(dAtA, i, uint64(m.ExpectedSeriesDatapointsPerHour)) - } - if len(m.Hostnames) > 0 { - for _, s := range m.Hostnames { - dAtA[i] = 0x3a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + i = encodeVarintDatabase(dAtA, i, uint64(m.BlockSize.Size())) + n1, err := m.BlockSize.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n1 } - if len(m.HostnameGroups) > 0 { - for _, msg := range m.HostnameGroups { - dAtA[i] = 0x42 + if len(m.Hosts) > 0 { + for _, msg := range m.Hosts { + dAtA[i] = 0x3a i++ i = encodeVarintDatabase(dAtA, i, uint64(msg.Size())) n, err := msg.MarshalTo(dAtA[i:]) @@ -266,7 +317,7 @@ func (m *DatabaseCreateRequest) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *HostnameGroup) Marshal() (dAtA []byte, err error) { +func (m *BlockSize) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -276,23 +327,73 @@ func (m *HostnameGroup) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *HostnameGroup) MarshalTo(dAtA []byte) (int, error) { +func (m *BlockSize) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int _ = l - if len(m.Hostname) > 0 { + if m.Nanos != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintDatabase(dAtA, i, uint64(m.Nanos)) + } + if m.ExpectedSeriesDatapointsPerHour != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintDatabase(dAtA, i, uint64(m.ExpectedSeriesDatapointsPerHour)) + } + return i, nil +} + +func (m *Host) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Host) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Id) > 0 { dAtA[i] = 0xa i++ - i = encodeVarintDatabase(dAtA, i, uint64(len(m.Hostname))) - i += copy(dAtA[i:], m.Hostname) + i = encodeVarintDatabase(dAtA, i, uint64(len(m.Id))) + i += copy(dAtA[i:], m.Id) } - if len(m.IsolationGroup) > 0 { + if len(m.Address) > 0 { dAtA[i] = 0x12 i++ + i = encodeVarintDatabase(dAtA, i, uint64(len(m.Address))) + i += copy(dAtA[i:], m.Address) + } + if m.Port != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintDatabase(dAtA, i, uint64(m.Port)) + } + if len(m.IsolationGroup) > 0 { + dAtA[i] = 0x22 + i++ i = encodeVarintDatabase(dAtA, i, uint64(len(m.IsolationGroup))) i += copy(dAtA[i:], m.IsolationGroup) } + if len(m.Zone) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintDatabase(dAtA, i, uint64(len(m.Zone))) + i += copy(dAtA[i:], m.Zone) + } + if m.Weight != 0 { + dAtA[i] = 0x30 + i++ + i = encodeVarintDatabase(dAtA, i, uint64(m.Weight)) + } return i, nil } @@ -315,21 +416,21 @@ func (m *DatabaseCreateResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintDatabase(dAtA, i, uint64(m.Namespace.Size())) - n1, err := m.Namespace.MarshalTo(dAtA[i:]) + n2, err := m.Namespace.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n1 + i += n2 } if m.Placement != nil { dAtA[i] = 0x12 i++ i = encodeVarintDatabase(dAtA, i, uint64(m.Placement.Size())) - n2, err := m.Placement.MarshalTo(dAtA[i:]) + n3, err := m.Placement.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n2 + i += n3 } return i, nil } @@ -363,17 +464,12 @@ func (m *DatabaseCreateRequest) Size() (n int) { if m.RetentionPeriodNanos != 0 { n += 1 + sovDatabase(uint64(m.RetentionPeriodNanos)) } - if m.ExpectedSeriesDatapointsPerHour != 0 { - n += 1 + sovDatabase(uint64(m.ExpectedSeriesDatapointsPerHour)) - } - if len(m.Hostnames) > 0 { - for _, s := range m.Hostnames { - l = len(s) - n += 1 + l + sovDatabase(uint64(l)) - } + if m.BlockSize != nil { + l = m.BlockSize.Size() + n += 1 + l + sovDatabase(uint64(l)) } - if len(m.HostnameGroups) > 0 { - for _, e := range m.HostnameGroups { + if len(m.Hosts) > 0 { + for _, e := range m.Hosts { l = e.Size() n += 1 + l + sovDatabase(uint64(l)) } @@ -381,17 +477,43 @@ func (m *DatabaseCreateRequest) Size() (n int) { return n } -func (m *HostnameGroup) Size() (n int) { +func (m *BlockSize) Size() (n int) { var l int _ = l - l = len(m.Hostname) + if m.Nanos != 0 { + n += 1 + sovDatabase(uint64(m.Nanos)) + } + if m.ExpectedSeriesDatapointsPerHour != 0 { + n += 1 + sovDatabase(uint64(m.ExpectedSeriesDatapointsPerHour)) + } + return n +} + +func (m *Host) Size() (n int) { + var l int + _ = l + l = len(m.Id) if l > 0 { n += 1 + l + sovDatabase(uint64(l)) } + l = len(m.Address) + if l > 0 { + n += 1 + l + sovDatabase(uint64(l)) + } + if m.Port != 0 { + n += 1 + sovDatabase(uint64(m.Port)) + } l = len(m.IsolationGroup) if l > 0 { n += 1 + l + sovDatabase(uint64(l)) } + l = len(m.Zone) + if l > 0 { + n += 1 + l + sovDatabase(uint64(l)) + } + if m.Weight != 0 { + n += 1 + sovDatabase(uint64(m.Weight)) + } return n } @@ -567,10 +689,10 @@ func (m *DatabaseCreateRequest) Unmarshal(dAtA []byte) error { } } case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ExpectedSeriesDatapointsPerHour", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockSize", wireType) } - m.ExpectedSeriesDatapointsPerHour = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDatabase @@ -580,16 +702,30 @@ func (m *DatabaseCreateRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ExpectedSeriesDatapointsPerHour |= (int64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthDatabase + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BlockSize == nil { + m.BlockSize = &BlockSize{} + } + if err := m.BlockSize.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 7: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Hostnames", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Hosts", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDatabase @@ -599,26 +735,78 @@ func (m *DatabaseCreateRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthDatabase } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.Hostnames = append(m.Hostnames, string(dAtA[iNdEx:postIndex])) + m.Hosts = append(m.Hosts, &Host{}) + if err := m.Hosts[len(m.Hosts)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field HostnameGroups", wireType) + default: + iNdEx = preIndex + skippy, err := skipDatabase(dAtA[iNdEx:]) + if err != nil { + return err } - var msglen int + if skippy < 0 { + return ErrInvalidLengthDatabase + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BlockSize) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatabase + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlockSize: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlockSize: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Nanos", wireType) + } + m.Nanos = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDatabase @@ -628,23 +816,30 @@ func (m *DatabaseCreateRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + m.Nanos |= (int64(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthDatabase - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExpectedSeriesDatapointsPerHour", wireType) } - m.HostnameGroups = append(m.HostnameGroups, &HostnameGroup{}) - if err := m.HostnameGroups[len(m.HostnameGroups)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.ExpectedSeriesDatapointsPerHour = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatabase + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExpectedSeriesDatapointsPerHour |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDatabase(dAtA[iNdEx:]) @@ -666,7 +861,7 @@ func (m *DatabaseCreateRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *HostnameGroup) Unmarshal(dAtA []byte) error { +func (m *Host) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -689,15 +884,15 @@ func (m *HostnameGroup) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: HostnameGroup: wiretype end group for non-group") + return fmt.Errorf("proto: Host: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: HostnameGroup: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Host: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Hostname", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -722,9 +917,57 @@ func (m *HostnameGroup) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Hostname = string(dAtA[iNdEx:postIndex]) + m.Id = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Address", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatabase + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDatabase + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Address = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Port", wireType) + } + m.Port = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatabase + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Port |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field IsolationGroup", wireType) } @@ -753,6 +996,54 @@ func (m *HostnameGroup) Unmarshal(dAtA []byte) error { } m.IsolationGroup = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Zone", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatabase + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDatabase + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Zone = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Weight", wireType) + } + m.Weight = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatabase + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Weight |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipDatabase(dAtA[iNdEx:]) @@ -1000,35 +1291,39 @@ func init() { } var fileDescriptorDatabase = []byte{ - // 465 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x92, 0x41, 0x6f, 0xd3, 0x30, - 0x14, 0xc7, 0x09, 0x59, 0xc7, 0xe2, 0x69, 0x1d, 0x58, 0x63, 0x8a, 0x06, 0x94, 0xa8, 0x08, 0x91, - 0x0b, 0x8d, 0xb4, 0x71, 0xe1, 0xc0, 0x05, 0x26, 0x36, 0x10, 0xaa, 0xaa, 0x8c, 0x7b, 0xe4, 0x26, - 0x8f, 0x26, 0x52, 0xe3, 0x67, 0x6c, 0x47, 0x82, 0x23, 0x5f, 0x00, 0xf1, 0xb1, 0x38, 0xf2, 0x11, - 0x50, 0xf9, 0x22, 0x28, 0xaf, 0x89, 0xbb, 0xed, 0xda, 0x4b, 0x64, 0xff, 0xff, 0x3f, 0xff, 0x9f, - 0xf3, 0x9e, 0xd9, 0x87, 0x45, 0x65, 0xcb, 0x66, 0x3e, 0xc9, 0xb1, 0x4e, 0xea, 0xb3, 0x62, 0xbe, - 0xfe, 0x18, 0x9d, 0x27, 0x39, 0xa2, 0x2e, 0x2a, 0x29, 0x2c, 0xea, 0x64, 0x01, 0x12, 0xb4, 0xb0, - 0x50, 0x24, 0x4a, 0xa3, 0xc5, 0x44, 0x14, 0x75, 0x25, 0x93, 0x42, 0x58, 0x31, 0x17, 0x06, 0x26, - 0x24, 0xf2, 0x01, 0xa9, 0x27, 0x1f, 0xb7, 0x48, 0x94, 0xa2, 0x06, 0xa3, 0x44, 0xde, 0x45, 0x6e, - 0x95, 0xa5, 0x96, 0x22, 0x87, 0x1a, 0xa4, 0x5d, 0x67, 0x8d, 0x7f, 0xf8, 0xec, 0xe1, 0x79, 0x77, - 0xe3, 0x77, 0x1a, 0x84, 0x85, 0x14, 0xbe, 0x36, 0x60, 0x2c, 0x7f, 0xce, 0x86, 0xae, 0x70, 0xd6, - 0xae, 0x42, 0x2f, 0xf2, 0xe2, 0x20, 0x3d, 0x70, 0xea, 0x54, 0xd4, 0xc0, 0x39, 0xdb, 0xb1, 0xdf, - 0x15, 0x84, 0x77, 0xc9, 0xa4, 0x35, 0x7f, 0xc2, 0x98, 0x6c, 0xea, 0xcc, 0x94, 0x42, 0x17, 0x26, - 0xf4, 0x23, 0x2f, 0x1e, 0xa4, 0x81, 0x6c, 0xea, 0x2b, 0x12, 0xf8, 0x4b, 0xc6, 0x35, 0xa8, 0x65, - 0x95, 0x0b, 0x5b, 0xa1, 0xcc, 0xbe, 0x88, 0xdc, 0xa2, 0x0e, 0x77, 0x08, 0x7b, 0x70, 0xcd, 0x79, - 0x4f, 0x06, 0x7f, 0xc5, 0x8e, 0x35, 0x58, 0x90, 0x04, 0x2b, 0xd0, 0x15, 0x16, 0x99, 0x14, 0x12, - 0x4d, 0x38, 0x88, 0xbc, 0xd8, 0x4f, 0x8f, 0x9c, 0x3b, 0x23, 0x73, 0xda, 0x7a, 0xfc, 0x13, 0x7b, - 0x06, 0xdf, 0x14, 0xe4, 0x16, 0x8a, 0xcc, 0x80, 0xae, 0xc0, 0x64, 0xed, 0x64, 0x14, 0x56, 0xd2, - 0x9a, 0x36, 0x26, 0x2b, 0xb1, 0xd1, 0xe1, 0x2e, 0x45, 0x3c, 0xed, 0xd1, 0x2b, 0x22, 0xcf, 0x1d, - 0x38, 0x03, 0x7d, 0x89, 0x8d, 0xe6, 0x8f, 0x59, 0x50, 0xa2, 0xb1, 0xf4, 0xeb, 0xe1, 0xbd, 0xc8, - 0x8f, 0x83, 0x74, 0x23, 0xf0, 0x37, 0xec, 0xb0, 0xdf, 0x64, 0x0b, 0x8d, 0x8d, 0x32, 0xe1, 0x5e, - 0xe4, 0xc7, 0xfb, 0xa7, 0x47, 0x13, 0xea, 0xfa, 0xe4, 0xb2, 0x73, 0x2f, 0x5a, 0x33, 0x1d, 0x96, - 0xd7, 0xb7, 0x66, 0xfc, 0x99, 0x1d, 0xdc, 0x00, 0xf8, 0x09, 0xdb, 0xeb, 0x91, 0xae, 0xe9, 0x6e, - 0xcf, 0x5f, 0xb0, 0xc3, 0xca, 0xe0, 0x72, 0xdd, 0x3a, 0x2a, 0xd6, 0xb5, 0x7e, 0xe8, 0x64, 0x0a, - 0x19, 0xff, 0xf4, 0xd8, 0xf1, 0xed, 0xc9, 0x1a, 0x85, 0xd2, 0x00, 0x7f, 0xcd, 0x02, 0x37, 0x44, - 0x2a, 0xb0, 0x7f, 0xfa, 0xa8, 0xbb, 0xe9, 0xb4, 0xd7, 0x2f, 0xc0, 0xf6, 0x7c, 0xba, 0xa1, 0xdb, - 0xa3, 0xee, 0x09, 0x51, 0xe1, 0xcd, 0xd1, 0x59, 0xaf, 0xdf, 0x38, 0xea, 0xe8, 0xb7, 0xf7, 0x7f, - 0xaf, 0x46, 0xde, 0x9f, 0xd5, 0xc8, 0xfb, 0xbb, 0x1a, 0x79, 0xbf, 0xfe, 0x8d, 0xee, 0xcc, 0x77, - 0xe9, 0x0d, 0x9e, 0xfd, 0x0f, 0x00, 0x00, 0xff, 0xff, 0xdd, 0xdd, 0x93, 0x44, 0x6f, 0x03, 0x00, - 0x00, + // 532 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x52, 0xcd, 0x6e, 0x13, 0x31, + 0x10, 0x66, 0x93, 0xa6, 0xd5, 0x3a, 0x6a, 0x28, 0x56, 0x89, 0x56, 0x20, 0x42, 0x08, 0x42, 0xe4, + 0x42, 0x56, 0x4a, 0xb9, 0x70, 0x2d, 0x15, 0x2d, 0x08, 0x45, 0x91, 0xf3, 0x00, 0x2b, 0xef, 0x7a, + 0x48, 0x2c, 0xb2, 0xb6, 0xb1, 0xbd, 0x02, 0xf2, 0x10, 0x88, 0x2b, 0xaf, 0xc0, 0x93, 0x70, 0xe4, + 0x11, 0x50, 0x78, 0x11, 0xb4, 0xb3, 0x3f, 0x05, 0xae, 0xbd, 0x44, 0x33, 0xdf, 0xf7, 0xcd, 0xe4, + 0xdb, 0xcf, 0x43, 0x5e, 0xaf, 0xa5, 0xdf, 0x14, 0xe9, 0x2c, 0xd3, 0x79, 0x9c, 0x9f, 0x89, 0xb4, + 0xfa, 0x71, 0x36, 0x8b, 0x33, 0xad, 0xad, 0x90, 0x8a, 0x7b, 0x6d, 0xe3, 0x35, 0x28, 0xb0, 0xdc, + 0x83, 0x88, 0x8d, 0xd5, 0x5e, 0xc7, 0x5c, 0xe4, 0x52, 0xc5, 0x82, 0x7b, 0x9e, 0x72, 0x07, 0x33, + 0x04, 0x69, 0x0f, 0xd1, 0x7b, 0x6f, 0x6e, 0xb0, 0x51, 0xf1, 0x1c, 0x9c, 0xe1, 0x59, 0xbd, 0xf2, + 0x46, 0xbb, 0xcc, 0x96, 0x67, 0x90, 0x83, 0xf2, 0xd5, 0xae, 0xc9, 0xf7, 0x0e, 0xb9, 0x7b, 0x51, + 0x3b, 0x7e, 0x69, 0x81, 0x7b, 0x60, 0xf0, 0xa1, 0x00, 0xe7, 0xe9, 0x13, 0x32, 0x68, 0xff, 0x38, + 0x29, 0xab, 0x28, 0x18, 0x07, 0xd3, 0x90, 0x1d, 0xb7, 0xe8, 0x82, 0xe7, 0x40, 0x29, 0x39, 0xf0, + 0x9f, 0x0d, 0x44, 0x1d, 0x24, 0xb1, 0xa6, 0x0f, 0x08, 0x51, 0x45, 0x9e, 0xb8, 0x0d, 0xb7, 0xc2, + 0x45, 0xdd, 0x71, 0x30, 0xed, 0xb1, 0x50, 0x15, 0xf9, 0x0a, 0x01, 0xfa, 0x8c, 0x50, 0x0b, 0x66, + 0x2b, 0x33, 0xee, 0xa5, 0x56, 0xc9, 0x3b, 0x9e, 0x79, 0x6d, 0xa3, 0x03, 0x94, 0xdd, 0xf9, 0x8b, + 0x79, 0x85, 0x04, 0x7d, 0x4e, 0x86, 0x16, 0x3c, 0x28, 0x14, 0x1b, 0xb0, 0x52, 0x8b, 0x44, 0x71, + 0xa5, 0x5d, 0xd4, 0x1b, 0x07, 0xd3, 0x2e, 0x3b, 0x6d, 0xd9, 0x25, 0x92, 0x8b, 0x92, 0xa3, 0x31, + 0x21, 0xe9, 0x56, 0x67, 0xef, 0x13, 0x27, 0x77, 0x10, 0x1d, 0x8e, 0x83, 0x69, 0x7f, 0x7e, 0x32, + 0xc3, 0x10, 0x66, 0xe7, 0x25, 0xb1, 0x92, 0x3b, 0x60, 0x61, 0xda, 0x94, 0xf4, 0x11, 0xe9, 0x6d, + 0xb4, 0xf3, 0x2e, 0x3a, 0x1a, 0x77, 0xa7, 0xfd, 0x79, 0xbf, 0xd6, 0x5e, 0x69, 0xe7, 0x59, 0xc5, + 0x4c, 0x34, 0x09, 0xdb, 0x51, 0x7a, 0x4a, 0x7a, 0x95, 0x8b, 0x00, 0x5d, 0x54, 0x0d, 0x7d, 0x4b, + 0x1e, 0xc3, 0x27, 0x03, 0x99, 0x07, 0x91, 0x38, 0xb0, 0x12, 0x5c, 0x52, 0x1e, 0x84, 0xd1, 0x52, + 0x79, 0x57, 0xba, 0x4f, 0x36, 0xba, 0xb0, 0x98, 0x56, 0x97, 0x3d, 0x6c, 0xa4, 0x2b, 0x54, 0x5e, + 0xb4, 0xc2, 0x25, 0xd8, 0x2b, 0x5d, 0xd8, 0xc9, 0xb7, 0x80, 0x1c, 0x94, 0x06, 0xe8, 0x80, 0x74, + 0xa4, 0xa8, 0x1f, 0xa0, 0x23, 0x05, 0x8d, 0xc8, 0x11, 0x17, 0xc2, 0x82, 0x73, 0x75, 0xf0, 0x4d, + 0x5b, 0xbe, 0x87, 0xd1, 0xd6, 0x63, 0xea, 0xc7, 0x0c, 0x6b, 0xfa, 0x94, 0xdc, 0x96, 0x4e, 0x6f, + 0xab, 0xb8, 0xd7, 0x56, 0x17, 0x06, 0xd3, 0x0e, 0xd9, 0xa0, 0x85, 0x2f, 0x4b, 0xb4, 0x1c, 0xde, + 0x69, 0x05, 0x18, 0x6c, 0xc8, 0xb0, 0xa6, 0x43, 0x72, 0xf8, 0x11, 0xe4, 0x7a, 0xe3, 0x31, 0xc4, + 0x63, 0x56, 0x77, 0x93, 0x2f, 0x01, 0x19, 0xfe, 0x7f, 0x39, 0xce, 0x68, 0xe5, 0x80, 0xbe, 0x20, + 0x61, 0x7b, 0x24, 0x68, 0xba, 0x3f, 0xbf, 0x5f, 0xc7, 0xb9, 0x68, 0xf0, 0x4b, 0xf0, 0x8d, 0x9e, + 0x5d, 0xab, 0xcb, 0xd1, 0xf6, 0x44, 0xf1, 0xd3, 0xae, 0x47, 0x97, 0x0d, 0xfe, 0xcf, 0x68, 0xab, + 0x3e, 0x3f, 0xf9, 0xb1, 0x1f, 0x05, 0x3f, 0xf7, 0xa3, 0xe0, 0xd7, 0x7e, 0x14, 0x7c, 0xfd, 0x3d, + 0xba, 0x95, 0x1e, 0xe2, 0x8d, 0x9f, 0xfd, 0x09, 0x00, 0x00, 0xff, 0xff, 0xa8, 0x52, 0x89, 0xf4, + 0xcf, 0x03, 0x00, 0x00, } diff --git a/src/coordinator/generated/proto/admin/database.proto b/src/coordinator/generated/proto/admin/database.proto index 3e6666c88b..4a5d4b89dd 100644 --- a/src/coordinator/generated/proto/admin/database.proto +++ b/src/coordinator/generated/proto/admin/database.proto @@ -13,19 +13,41 @@ message DatabaseCreateRequest { // Optional fields that may be inferred depending on database type int32 num_shards = 3; int32 replication_factor = 4; + // The below two options are used to default retention options int64 retention_period_nanos = 5; - int64 expected_series_datapoints_per_hour = 6; - // Only one of the two below hostname types should be taken. This is enforced - // in application code instead of via protobuf oneof to reduce the amount of - // JSON nested structures required for input. - repeated string hostnames = 7; - repeated HostnameGroup hostname_groups = 8; + + // If no block size fields are set then the block size is + // derived from the length of the retention period + BlockSize block_size = 6; + + // Required if not using local database type + repeated Host hosts = 7; +} + +message BlockSize { + // Explicit block size nanoseconds + int64 nanos = 1; + // With the expected series datapoints per hour, use a recommended block size + int64 expected_series_datapoints_per_hour = 2; } -message HostnameGroup { - string hostname = 1; - string isolation_group = 2; +message Host { + // Name (must be unique) of the host for logging/display purposes, can be hostname if desired, + // or UUID or any other string ID as used in third party inventory system + string id = 1; + // Address can be IP address or hostname, this is used to connect to the host + string address = 2; + // Port running the Node RPC listen address (currently TChannel/Thrift Node service) + uint32 port = 3; + // (Optional) Isolation group is an optional grouping, for instance to isolate replicas by + // zones use zone here, or to isolate by host racks use racks here + string isolation_group = 4; + // (Optional) Zone specifies the zone the host resides in, this is optional. + string zone = 5; + // (Optional) Weight specifies when considering how many shards to take how to weight + // this instance, do not specify to default to a weight of 1. + uint32 weight = 6; } message DatabaseCreateResponse { diff --git a/src/coordinator/models/tag.go b/src/coordinator/models/tag.go index cbaf1b53f3..51c9f27538 100644 --- a/src/coordinator/models/tag.go +++ b/src/coordinator/models/tag.go @@ -55,7 +55,7 @@ type MatchType int // Possible MatchTypes. const ( - MatchEqual MatchType = iota + MatchEqual MatchType = iota MatchNotEqual MatchRegexp MatchNotRegexp diff --git a/src/coordinator/models/tag_test.go b/src/coordinator/models/tag_test.go index c6c15935a2..3ac6395a10 100644 --- a/src/coordinator/models/tag_test.go +++ b/src/coordinator/models/tag_test.go @@ -107,5 +107,6 @@ func TestMatchType(t *testing.T) { func TestTagID(t *testing.T) { tags := make(Tags) tags["t1"] = "v1" - assert.Equal(t, tags.ID(), "t1=v1,") + tags["t2"] = "v2" + assert.Equal(t, tags.ID(), "t1=v1,t2=v2,") } diff --git a/src/coordinator/services/m3coordinator/server/server.go b/src/coordinator/services/m3coordinator/server/server.go index 12682fc7c7..f362674d74 100644 --- a/src/coordinator/services/m3coordinator/server/server.go +++ b/src/coordinator/services/m3coordinator/server/server.go @@ -47,6 +47,7 @@ import ( "github.com/m3db/m3db/src/coordinator/util/logging" "github.com/m3db/m3db/src/dbnode/client" xconfig "github.com/m3db/m3x/config" + "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" "github.com/m3db/m3x/pool" xsync "github.com/m3db/m3x/sync" @@ -56,11 +57,17 @@ import ( ) const ( - defaultNamespace = "metrics" defaultWorkerPoolCount = 4096 defaultWorkerPoolSize = 20 ) +var ( + defaultLocalConfiguration = &config.LocalConfiguration{ + Namespace: "default", + Retention: 2 * 24 * time.Hour, + } +) + // RunOptions provides options for running the server // with backwards compatibility if only solely adding fields. type RunOptions struct { @@ -71,15 +78,13 @@ type RunOptions struct { // instead of parsing ConfigFile if ConfigFile is not specified. Config config.Configuration - // DBConfig is the M3DB config, which is used to - DBConfig dbconfig.DBConfiguration + // DBConfig is the local M3DB config when running embedded. + DBConfig *dbconfig.DBConfiguration - // DBClient is the M3DB client to use instead of instantiating a new one - // from client config. + // DBClient is the local M3DB client when running embedded. DBClient <-chan client.Client - // ClusterClient is the M3DB cluster client to use instead of instantiating - // one from the client config. + // ClusterClient is the local M3DB cluster client when running embedded. ClusterClient <-chan clusterclient.Client } @@ -112,11 +117,13 @@ func Run(runOpts RunOptions) { clusterClientCh = runOpts.ClusterClient } - if clusterClientCh == nil && cfg.DBClient != nil && cfg.DBClient.EnvironmentConfig.Service != nil { - clusterSvcClientOpts := cfg.DBClient.EnvironmentConfig.Service.NewOptions() + if clusterClientCh == nil && cfg.ClusterManagement != nil { + // We resolved an etcd configuration for cluster management endpoints + etcdCfg := cfg.ClusterManagement.Etcd + clusterSvcClientOpts := etcdCfg.NewOptions() clusterClient, err := etcd.NewConfigServiceClient(clusterSvcClientOpts) if err != nil { - logger.Fatal("unable to create etcd client", zap.Any("error", err)) + logger.Fatal("unable to create cluster management etcd client", zap.Any("error", err)) } clusterClientSendableCh := make(chan clusterclient.Client, 1) @@ -124,32 +131,34 @@ func Run(runOpts RunOptions) { clusterClientCh = clusterClientSendableCh } - var dbClientCh <-chan client.Client - if runOpts.DBClient != nil { - dbClientCh = runOpts.DBClient - } - - if dbClientCh == nil { - // If not provided create cluster client and DB client - clientCfg := cfg.DBClient - if clientCfg == nil { - logger.Fatal("missing coordinator m3db client configuration") + var clusters local.Clusters + if len(cfg.Clusters) > 0 { + clusters, err = cfg.Clusters.NewClusters() + if err != nil { + logger.Fatal("unable to connect to clusters", zap.Any("error", err)) } - - dbClient, err := clientCfg.NewClient(client.ConfigurationParameters{}) + } else { + localCfg := cfg.Local + if localCfg == nil { + localCfg = defaultLocalConfiguration + } + dbClientCh := runOpts.DBClient + if dbClientCh == nil { + logger.Fatal("no clusters configured and not running local embedded cluster") + } + session := m3db.NewAsyncSession(func() (client.Client, error) { + return <-dbClientCh, nil + }, nil) + clusters, err = local.NewClusters(local.UnaggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID(localCfg.Namespace), + Session: session, + Retention: localCfg.Retention, + }) if err != nil { - logger.Fatal("unable to create m3db client", zap.Any("error", err)) + logger.Fatal("unable to connect to clusters", zap.Any("error", err)) } - - dbClientSendableCh := make(chan client.Client, 1) - dbClientSendableCh <- dbClient - dbClientCh = dbClientSendableCh } - session := m3db.NewAsyncSession(func() (client.Client, error) { - return <-dbClientCh, nil - }, nil) - workerPoolCount := cfg.DecompressWorkerPoolCount if workerPoolCount == 0 { workerPoolCount = defaultWorkerPoolCount @@ -175,12 +184,17 @@ func Run(runOpts RunOptions) { return workerPool }) - fanoutStorage, storageCleanup := setupStorages(logger, session, cfg, objectPool) + fanoutStorage, storageCleanup := setupStorages(logger, clusters, cfg, objectPool) defer storageCleanup() - clusterClient := m3dbcluster.NewAsyncClient(func() (clusterclient.Client, error) { - return <-clusterClientCh, nil - }, nil) + var clusterClient clusterclient.Client + if clusterClientCh != nil { + // Only use a cluster client if we are going to receive one, that + // way passing nil to httpd NewHandler disables the endpoints entirely + clusterClient = m3dbcluster.NewAsyncClient(func() (clusterclient.Client, error) { + return <-clusterClientCh, nil + }, nil) + } handler, err := httpd.NewHandler(fanoutStorage, executor.NewEngine(fanoutStorage), clusterClient, cfg, runOpts.DBConfig, scope) @@ -201,19 +215,15 @@ func Run(runOpts RunOptions) { signal.Notify(sigChan, os.Interrupt, syscall.SIGTERM) <-sigChan - if err := session.Close(); err != nil { - logger.Fatal("unable to close m3db client session", zap.Any("error", err)) + if err := clusters.Close(); err != nil { + logger.Fatal("unable to close M3DB cluster sessions", zap.Any("error", err)) } } -func setupStorages(logger *zap.Logger, session client.Session, cfg config.Configuration, workerPool pool.ObjectPool) (storage.Storage, func()) { +func setupStorages(logger *zap.Logger, clusters local.Clusters, cfg config.Configuration, workerPool pool.ObjectPool) (storage.Storage, func()) { cleanup := func() {} - namespace := defaultNamespace - if cfg.DBNamespace != "" { - namespace = cfg.DBNamespace - } - localStorage := local.NewStorage(session, namespace, workerPool) + localStorage := local.NewStorage(clusters, workerPool) stores := []storage.Storage{localStorage} remoteEnabled := false if cfg.RPC != nil && cfg.RPC.Enabled { diff --git a/src/coordinator/storage/config.go b/src/coordinator/storage/config.go new file mode 100644 index 0000000000..0fcce7dd3b --- /dev/null +++ b/src/coordinator/storage/config.go @@ -0,0 +1,50 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package storage + +import "fmt" + +// ValidateMetricsType validates a stored metrics type. +func ValidateMetricsType(v MetricsType) error { + for _, valid := range validMetricsTypes { + if valid == v { + return nil + } + } + return fmt.Errorf("invalid stored metrics type '%v': should be one of %v", + v, validMetricsTypes) +} + +// UnmarshalYAML unmarshals a stored merics type. +func (v *MetricsType) UnmarshalYAML(unmarshal func(interface{}) error) error { + var str string + if err := unmarshal(&str); err != nil { + return err + } + for _, valid := range validMetricsTypes { + if str == valid.String() { + *v = valid + return nil + } + } + return fmt.Errorf("invalid MetricsType '%s' valid types are: %v", + str, validMetricsTypes) +} diff --git a/src/coordinator/storage/config_test.go b/src/coordinator/storage/config_test.go new file mode 100644 index 0000000000..04b71c46a0 --- /dev/null +++ b/src/coordinator/storage/config_test.go @@ -0,0 +1,54 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package storage + +import ( + "fmt" + "math" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + yaml "gopkg.in/yaml.v2" +) + +func TestValidateMetricsType(t *testing.T) { + assert.NoError(t, ValidateMetricsType(UnaggregatedMetricsType)) + assert.Error(t, ValidateMetricsType(MetricsType(math.MaxUint64))) +} + +func TestMetricsTypeUnmarshalYAML(t *testing.T) { + type config struct { + Type MetricsType `yaml:"type"` + } + + for _, value := range validMetricsTypes { + str := fmt.Sprintf("type: %s\n", value.String()) + + var cfg config + require.NoError(t, yaml.Unmarshal([]byte(str), &cfg)) + + assert.Equal(t, value, cfg.Type) + } + + var cfg config + require.Error(t, yaml.Unmarshal([]byte("type: not_a_known_type\n"), &cfg)) +} diff --git a/src/coordinator/storage/fanout/storage_test.go b/src/coordinator/storage/fanout/storage_test.go index 2fb103f72c..20432d611a 100644 --- a/src/coordinator/storage/fanout/storage_test.go +++ b/src/coordinator/storage/fanout/storage_test.go @@ -73,8 +73,8 @@ func setupFanoutRead(t *testing.T, output bool, response ...*fetchResponse) stor } ctrl := gomock.NewController(t) - store1, session1 := local.NewStorageAndSession(ctrl) - store2, session2 := local.NewStorageAndSession(ctrl) + store1, session1 := local.NewStorageAndSession(t, ctrl) + store2, session2 := local.NewStorageAndSession(t, ctrl) session1.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(response[0].result, true, response[0].err) session2.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(response[len(response)-1].result, true, response[len(response)-1].err) @@ -91,8 +91,8 @@ func setupFanoutRead(t *testing.T, output bool, response ...*fetchResponse) stor func setupFanoutWrite(t *testing.T, output bool, errs ...error) storage.Storage { setup() ctrl := gomock.NewController(t) - store1, session1 := local.NewStorageAndSession(ctrl) - store2, session2 := local.NewStorageAndSession(ctrl) + store1, session1 := local.NewStorageAndSession(t, ctrl) + store2, session2 := local.NewStorageAndSession(t, ctrl) session1.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(errs[0]) session2.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(errs[len(errs)-1]) stores := []storage.Storage{ @@ -118,7 +118,10 @@ func TestFanoutReadError(t *testing.T) { func TestFanoutReadSuccess(t *testing.T) { store := setupFanoutRead(t, true, &fetchResponse{result: fakeIterator(t)}, &fetchResponse{result: fakeIterator(t)}) - res, err := store.Fetch(context.TODO(), &storage.FetchQuery{}, &storage.FetchOptions{}) + res, err := store.Fetch(context.TODO(), &storage.FetchQuery{ + Start: time.Now().Add(-time.Hour), + End: time.Now(), + }, &storage.FetchOptions{}) require.NoError(t, err, "no error on read") assert.NotNil(t, res) assert.NoError(t, store.Close()) diff --git a/src/coordinator/storage/interface.go b/src/coordinator/storage/interface.go index 698ea50754..938d824f52 100644 --- a/src/coordinator/storage/interface.go +++ b/src/coordinator/storage/interface.go @@ -100,6 +100,7 @@ type WriteQuery struct { Datapoints ts.Datapoints Unit xtime.Unit Annotation []byte + Attributes Attributes } func (q *WriteQuery) String() string { @@ -129,3 +130,38 @@ type QueryResult struct { FetchResult *FetchResult Err error } + +// MetricsType is a type of stored metrics. +type MetricsType uint + +const ( + // UnaggregatedMetricsType is an unaggregated metrics type. + UnaggregatedMetricsType MetricsType = iota + // AggregatedMetricsType is an aggregated metrics type. + AggregatedMetricsType +) + +var ( + validMetricsTypes = []MetricsType{ + UnaggregatedMetricsType, + AggregatedMetricsType, + } +) + +func (t MetricsType) String() string { + switch t { + case UnaggregatedMetricsType: + return "unaggregated" + case AggregatedMetricsType: + return "aggregated" + default: + return "unknown" + } +} + +// Attributes is a set of stored metrics attributes. +type Attributes struct { + MetricsType MetricsType + Retention time.Duration + Resolution time.Duration +} diff --git a/src/coordinator/storage/local/cluster.go b/src/coordinator/storage/local/cluster.go new file mode 100644 index 0000000000..d89cea2a40 --- /dev/null +++ b/src/coordinator/storage/local/cluster.go @@ -0,0 +1,293 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package local + +import ( + "errors" + "fmt" + "io" + "sync" + "time" + + "github.com/m3db/m3db/src/coordinator/storage" + "github.com/m3db/m3db/src/dbnode/client" + xerrors "github.com/m3db/m3x/errors" + "github.com/m3db/m3x/ident" +) + +var ( + errNamespaceIDNotSet = errors.New("namespace ID not set") + errSessionNotSet = errors.New("session not set") + errRetentionNotSet = errors.New("retention not set") + errResolutionNotSet = errors.New("resolution not set") +) + +// Clusters is a flattened collection of local storage clusters and namespaces. +type Clusters interface { + io.Closer + + // ClusterNamespaces returns all known cluster namespaces. + ClusterNamespaces() []ClusterNamespace + + // UnaggregatedClusterNamespace returns the valid unaggregated + // cluster namespace. + UnaggregatedClusterNamespace() ClusterNamespace + + // AggregatedClusterNamespace returns an aggregated cluster namespace + // at a specific retention and resolution. + AggregatedClusterNamespace(attrs RetentionResolution) (ClusterNamespace, bool) +} + +// RetentionResolution is a tuple of retention and resolution that describes +// an aggregated metrics policy. +type RetentionResolution struct { + Retention time.Duration + Resolution time.Duration +} + +// ClusterNamespace is a local storage cluster namespace. +type ClusterNamespace interface { + NamespaceID() ident.ID + Attributes() storage.Attributes + Session() client.Session +} + +// UnaggregatedClusterNamespaceDefinition is the definition for the +// cluster namespace that holds unaggregated metrics data. +type UnaggregatedClusterNamespaceDefinition struct { + NamespaceID ident.ID + Session client.Session + Retention time.Duration +} + +// Validate will validate the cluster namespace definition. +func (def UnaggregatedClusterNamespaceDefinition) Validate() error { + if def.NamespaceID == nil || len(def.NamespaceID.String()) == 0 { + return errNamespaceIDNotSet + } + if def.Session == nil { + return errSessionNotSet + } + if def.Retention <= 0 { + return errRetentionNotSet + } + return nil +} + +// AggregatedClusterNamespaceDefinition is a definition for a +// cluster namespace that holds aggregated metrics data at a +// specific retention and resolution. +type AggregatedClusterNamespaceDefinition struct { + NamespaceID ident.ID + Session client.Session + Retention time.Duration + Resolution time.Duration +} + +// Validate validates the cluster namespace definition. +func (def AggregatedClusterNamespaceDefinition) Validate() error { + if def.NamespaceID == nil || len(def.NamespaceID.String()) == 0 { + return errNamespaceIDNotSet + } + if def.Session == nil { + return errSessionNotSet + } + if def.Retention <= 0 { + return errRetentionNotSet + } + if def.Resolution <= 0 { + return errResolutionNotSet + } + return nil +} + +type clusters struct { + namespaces []ClusterNamespace + unaggregatedNamespace ClusterNamespace + aggregatedNamespaces map[RetentionResolution]ClusterNamespace +} + +// NewClusters instantiates a new Clusters instance. +func NewClusters( + unaggregatedClusterNamespace UnaggregatedClusterNamespaceDefinition, + aggregatedClusterNamespaces ...AggregatedClusterNamespaceDefinition, +) (Clusters, error) { + expectedAggregated := len(aggregatedClusterNamespaces) + expectedAll := 1 + expectedAggregated + namespaces := make([]ClusterNamespace, 0, expectedAll) + aggregatedNamespaces := make(map[RetentionResolution]ClusterNamespace, + expectedAggregated) + + def := unaggregatedClusterNamespace + unaggregatedNamespace, err := newUnaggregatedClusterNamespace(def) + if err != nil { + return nil, err + } + + namespaces = append(namespaces, unaggregatedNamespace) + for _, def := range aggregatedClusterNamespaces { + namespace, err := newAggregatedClusterNamespace(def) + if err != nil { + return nil, err + } + + namespaces = append(namespaces, namespace) + key := RetentionResolution{ + Retention: namespace.Attributes().Retention, + Resolution: namespace.Attributes().Resolution, + } + + _, exists := aggregatedNamespaces[key] + if exists { + return nil, fmt.Errorf("duplicate aggregated namespace exists for: "+ + "retention=%s, resolution=%s", + key.Retention.String(), key.Resolution.String()) + } + + aggregatedNamespaces[key] = namespace + } + + return &clusters{ + namespaces: namespaces, + unaggregatedNamespace: unaggregatedNamespace, + aggregatedNamespaces: aggregatedNamespaces, + }, nil +} + +func (c *clusters) ClusterNamespaces() []ClusterNamespace { + return c.namespaces +} + +func (c *clusters) UnaggregatedClusterNamespace() ClusterNamespace { + return c.unaggregatedNamespace +} + +func (c *clusters) AggregatedClusterNamespace( + attrs RetentionResolution, +) (ClusterNamespace, bool) { + namespace, ok := c.aggregatedNamespaces[attrs] + return namespace, ok +} + +func (c *clusters) Close() error { + var ( + wg sync.WaitGroup + syncMultiErrs syncMultiErrs + uniqueSessions []client.Session + ) + // Collect unique sessions, some namespaces may share same + // client session (same cluster) + uniqueSessions = append(uniqueSessions, c.unaggregatedNamespace.Session()) + for _, namespace := range c.aggregatedNamespaces { + unique := true + for _, session := range uniqueSessions { + if namespace.Session() == session { + unique = false + break + } + } + if unique { + uniqueSessions = append(uniqueSessions, namespace.Session()) + } + } + + for _, session := range uniqueSessions { + session := session // Capture for lambda + wg.Add(1) + go func() { + defer wg.Done() + err := session.Close() + syncMultiErrs.add(err) + }() + } + + wg.Wait() + + return syncMultiErrs.finalError() +} + +type clusterNamespace struct { + namespaceID ident.ID + attributes storage.Attributes + session client.Session +} + +func newUnaggregatedClusterNamespace( + def UnaggregatedClusterNamespaceDefinition, +) (ClusterNamespace, error) { + if err := def.Validate(); err != nil { + return nil, err + } + return &clusterNamespace{ + namespaceID: def.NamespaceID, + attributes: storage.Attributes{ + MetricsType: storage.UnaggregatedMetricsType, + Retention: def.Retention, + }, + session: def.Session, + }, nil +} + +func newAggregatedClusterNamespace( + def AggregatedClusterNamespaceDefinition, +) (ClusterNamespace, error) { + if err := def.Validate(); err != nil { + return nil, err + } + return &clusterNamespace{ + namespaceID: def.NamespaceID, + attributes: storage.Attributes{ + MetricsType: storage.AggregatedMetricsType, + Retention: def.Retention, + Resolution: def.Resolution, + }, + session: def.Session, + }, nil +} + +func (n *clusterNamespace) NamespaceID() ident.ID { + return n.namespaceID +} + +func (n *clusterNamespace) Attributes() storage.Attributes { + return n.attributes +} + +func (n *clusterNamespace) Session() client.Session { + return n.session +} + +type syncMultiErrs struct { + sync.Mutex + multiErr xerrors.MultiError +} + +func (errs *syncMultiErrs) add(err error) { + errs.Lock() + errs.multiErr = errs.multiErr.Add(err) + errs.Unlock() +} + +func (errs *syncMultiErrs) finalError() error { + errs.Lock() + defer errs.Unlock() + return errs.multiErr.FinalError() +} diff --git a/src/coordinator/storage/local/cluster_test.go b/src/coordinator/storage/local/cluster_test.go new file mode 100644 index 0000000000..3f7e3a1f59 --- /dev/null +++ b/src/coordinator/storage/local/cluster_test.go @@ -0,0 +1,171 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package local + +import ( + "fmt" + "strings" + "testing" + "time" + + "github.com/m3db/m3db/src/coordinator/storage" + "github.com/m3db/m3db/src/dbnode/client" + "github.com/m3db/m3x/ident" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestNewClustersWithDuplicateAggregatedClusterNamespace(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + _, err := NewClusters(UnaggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID("metrics_unagg"), + Session: client.NewMockSession(ctrl), + Retention: 2 * 24 * time.Hour, + }, AggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID("metrics_agg0"), + Session: client.NewMockSession(ctrl), + Retention: 7 * 24 * time.Hour, + Resolution: time.Minute, + }, AggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID("metrics_agg1"), + Session: client.NewMockSession(ctrl), + Retention: 7 * 24 * time.Hour, + Resolution: time.Minute, + }) + require.Error(t, err) + + str := err.Error() + assert.True(t, strings.Contains(str, "duplicate aggregated namespace"), + fmt.Sprintf("unexpected error: %s", err.Error())) +} + +func TestNewClustersFromConfig(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + newClient1, mockSession1 := newTestClientFromConfig(ctrl) + newClient2, mockSession2 := newTestClientFromConfig(ctrl) + cfg := ClustersStaticConfiguration{ + ClusterStaticConfiguration{ + newClientFromConfig: newClient1, + Namespaces: []ClusterStaticNamespaceConfiguration{ + ClusterStaticNamespaceConfiguration{ + Namespace: "unaggregated", + StorageMetricsType: storage.UnaggregatedMetricsType, + Retention: 7 * 24 * time.Hour, + }, + }, + }, + ClusterStaticConfiguration{ + newClientFromConfig: newClient2, + Namespaces: []ClusterStaticNamespaceConfiguration{ + ClusterStaticNamespaceConfiguration{ + Namespace: "aggregated0", + StorageMetricsType: storage.AggregatedMetricsType, + Retention: 30 * 24 * time.Hour, + Resolution: time.Minute, + }, + ClusterStaticNamespaceConfiguration{ + Namespace: "aggregated1", + StorageMetricsType: storage.AggregatedMetricsType, + Retention: 365 * 24 * time.Hour, + Resolution: 10 * time.Minute, + }, + }, + }, + } + + clusters, err := cfg.NewClusters() + require.NoError(t, err) + + // Resolve expected clusters and check attributes + unaggregatedNs := clusters.UnaggregatedClusterNamespace() + assert.Equal(t, "unaggregated", unaggregatedNs.NamespaceID().String()) + assert.Equal(t, storage.Attributes{ + MetricsType: storage.UnaggregatedMetricsType, + Retention: 7 * 24 * time.Hour, + }, unaggregatedNs.Attributes()) + assert.True(t, mockSession1 == unaggregatedNs.Session()) + + aggregated1Month1Minute, ok := clusters.AggregatedClusterNamespace(RetentionResolution{ + Retention: 30 * 24 * time.Hour, + Resolution: time.Minute, + }) + require.True(t, ok) + assert.Equal(t, "aggregated0", aggregated1Month1Minute.NamespaceID().String()) + assert.Equal(t, storage.Attributes{ + MetricsType: storage.AggregatedMetricsType, + Retention: 30 * 24 * time.Hour, + Resolution: time.Minute, + }, aggregated1Month1Minute.Attributes()) + assert.True(t, mockSession2 == aggregated1Month1Minute.Session()) + + aggregated1Year10Minute, ok := clusters.AggregatedClusterNamespace(RetentionResolution{ + Retention: 365 * 24 * time.Hour, + Resolution: 10 * time.Minute, + }) + require.True(t, ok) + assert.Equal(t, "aggregated1", aggregated1Year10Minute.NamespaceID().String()) + assert.Equal(t, storage.Attributes{ + MetricsType: storage.AggregatedMetricsType, + Retention: 365 * 24 * time.Hour, + Resolution: 10 * time.Minute, + }, aggregated1Year10Minute.Attributes()) + assert.True(t, mockSession2 == aggregated1Year10Minute.Session()) + + // Ensure cannot resolve unexpected clusters + _, ok = clusters.AggregatedClusterNamespace(RetentionResolution{ + Retention: time.Hour, + Resolution: time.Minute, + }) + require.False(t, ok) + + // Close sessions at most once each + mockSession1.EXPECT().Close().Return(nil).Times(1) + mockSession2.EXPECT().Close().Return(nil).Times(1) + + err = clusters.Close() + require.NoError(t, err) +} + +func newTestClientFromConfig(ctrl *gomock.Controller) ( + newClientFromConfig, + *client.MockSession, +) { + mockSession := client.NewMockSession(ctrl) + + mockClient := client.NewMockClient(ctrl) + mockClient.EXPECT().DefaultSession().Return(mockSession, nil).AnyTimes() + + newClientFn := func( + _ client.Configuration, + _ client.ConfigurationParameters, + _ ...client.CustomOption, + ) (client.Client, error) { + return mockClient, nil + } + + return newClientFn, mockSession +} diff --git a/src/coordinator/storage/local/config.go b/src/coordinator/storage/local/config.go new file mode 100644 index 0000000000..83b1b5b900 --- /dev/null +++ b/src/coordinator/storage/local/config.go @@ -0,0 +1,194 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package local + +import ( + "fmt" + "sync" + "time" + + "github.com/m3db/m3db/src/coordinator/storage" + "github.com/m3db/m3db/src/dbnode/client" + "github.com/m3db/m3x/ident" +) + +var ( + defaultNewClientConfigurationParams = client.ConfigurationParameters{} +) + +// ClustersStaticConfiguration is a set of static cluster configurations. +type ClustersStaticConfiguration []ClusterStaticConfiguration + +type newClientFromConfig func( + cfg client.Configuration, + params client.ConfigurationParameters, + custom ...client.CustomOption, +) (client.Client, error) + +// ClusterStaticConfiguration is a static cluster configuration. +type ClusterStaticConfiguration struct { + newClientFromConfig newClientFromConfig + Client client.Configuration `yaml:"client"` + Namespaces []ClusterStaticNamespaceConfiguration `yaml:"namespaces"` +} + +func (c ClusterStaticConfiguration) newClient( + params client.ConfigurationParameters, + custom ...client.CustomOption, +) (client.Client, error) { + if c.newClientFromConfig != nil { + return c.newClientFromConfig(c.Client, params, custom...) + } + return c.Client.NewClient(params, custom...) +} + +// ClusterStaticNamespaceConfiguration describes the namespaces in a +// static cluster. +type ClusterStaticNamespaceConfiguration struct { + Namespace string `yaml:"namespace"` + StorageMetricsType storage.MetricsType `yaml:"storageMetricsType"` + Retention time.Duration `yaml:"retention" validate:"nonzero"` + Resolution time.Duration `yaml:"resolution" validate:"min=0"` +} + +type unaggregatedClusterNamespaceConfiguration struct { + client client.Client + namespace ClusterStaticNamespaceConfiguration + result clusterConnectResult +} + +type aggregatedClusterNamespacesConfiguration struct { + client client.Client + namespaces []ClusterStaticNamespaceConfiguration + result clusterConnectResult +} + +type clusterConnectResult struct { + session client.Session + err error +} + +// NewClusters instantiates a new Clusters instance. +func (c ClustersStaticConfiguration) NewClusters() (Clusters, error) { + var ( + numUnaggregatedClusterNamespaces int + numAggregatedClusterNamespaces int + unaggregatedClusterNamespaceCfg = &unaggregatedClusterNamespaceConfiguration{} + aggregatedClusterNamespacesCfgs []*aggregatedClusterNamespacesConfiguration + unaggregatedClusterNamespace UnaggregatedClusterNamespaceDefinition + aggregatedClusterNamespaces []AggregatedClusterNamespaceDefinition + ) + for _, clusterCfg := range c { + client, err := clusterCfg.newClient(defaultNewClientConfigurationParams) + if err != nil { + return nil, err + } + + aggregatedClusterNamespacesCfg := &aggregatedClusterNamespacesConfiguration{ + client: client, + } + + for _, n := range clusterCfg.Namespaces { + switch n.StorageMetricsType { + case storage.UnaggregatedMetricsType: + numUnaggregatedClusterNamespaces++ + if numUnaggregatedClusterNamespaces > 1 { + return nil, fmt.Errorf("only one unaggregated cluster namespace "+ + "can be specified: specified %d", numUnaggregatedClusterNamespaces) + } + + unaggregatedClusterNamespaceCfg.client = client + unaggregatedClusterNamespaceCfg.namespace = n + + case storage.AggregatedMetricsType: + numAggregatedClusterNamespaces++ + + aggregatedClusterNamespacesCfg.namespaces = + append(aggregatedClusterNamespacesCfg.namespaces, n) + + default: + return nil, fmt.Errorf("unknown storage metrics type: %v", + n.StorageMetricsType) + } + } + + if len(aggregatedClusterNamespacesCfg.namespaces) > 0 { + aggregatedClusterNamespacesCfgs = + append(aggregatedClusterNamespacesCfgs, aggregatedClusterNamespacesCfg) + } + } + + if numUnaggregatedClusterNamespaces != 1 { + return nil, fmt.Errorf("one unaggregated cluster namespace "+ + "must be specified: specified %d", numUnaggregatedClusterNamespaces) + } + + // Connect to all clusters in parallel + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + cfg := unaggregatedClusterNamespaceCfg + cfg.result.session, cfg.result.err = cfg.client.DefaultSession() + }() + for _, cfg := range aggregatedClusterNamespacesCfgs { + cfg := cfg // Capture var + wg.Add(1) + go func() { + defer wg.Done() + cfg.result.session, cfg.result.err = cfg.client.DefaultSession() + }() + } + + // Wait + wg.Wait() + + if unaggregatedClusterNamespaceCfg.result.err != nil { + return nil, fmt.Errorf("could not connect to unaggregated cluster: %v", + unaggregatedClusterNamespaceCfg.result.err) + } + + unaggregatedClusterNamespace = UnaggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID(unaggregatedClusterNamespaceCfg.namespace.Namespace), + Session: unaggregatedClusterNamespaceCfg.result.session, + Retention: unaggregatedClusterNamespaceCfg.namespace.Retention, + } + + for i, cfg := range aggregatedClusterNamespacesCfgs { + if cfg.result.err != nil { + return nil, fmt.Errorf("could not connect to aggregated cluster #%d: %v", + i, cfg.result.err) + } + + for _, n := range cfg.namespaces { + def := AggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID(n.Namespace), + Session: cfg.result.session, + Retention: n.Retention, + Resolution: n.Resolution, + } + aggregatedClusterNamespaces = append(aggregatedClusterNamespaces, def) + } + } + + return NewClusters(unaggregatedClusterNamespace, + aggregatedClusterNamespaces...) +} diff --git a/src/coordinator/storage/local/storage.go b/src/coordinator/storage/local/storage.go index 8683829cd2..e4f161d535 100644 --- a/src/coordinator/storage/local/storage.go +++ b/src/coordinator/storage/local/storage.go @@ -22,6 +22,9 @@ package local import ( "context" + goerrors "errors" + "fmt" + "sync" "time" "github.com/m3db/m3db/src/coordinator/block" @@ -29,25 +32,25 @@ import ( "github.com/m3db/m3db/src/coordinator/models" "github.com/m3db/m3db/src/coordinator/storage" "github.com/m3db/m3db/src/coordinator/util/execution" - "github.com/m3db/m3db/src/dbnode/client" + "github.com/m3db/m3db/src/dbnode/storage/index" + xerrors "github.com/m3db/m3x/errors" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/pool" xtime "github.com/m3db/m3x/time" ) +var ( + errNoLocalClustersFulfillsQuery = goerrors.New("no clusters can fulfill query") +) + type localStorage struct { - session client.Session - namespace ident.ID + clusters Clusters workerPool pool.ObjectPool } // NewStorage creates a new local Storage instance. -func NewStorage(session client.Session, namespace string, workerPool pool.ObjectPool) storage.Storage { - return &localStorage{ - session: session, - namespace: ident.StringID(namespace), - workerPool: workerPool, - } +func NewStorage(clusters Clusters, workerPool pool.ObjectPool) storage.Storage { + return &localStorage{clusters: clusters, workerPool: workerPool} } func (s *localStorage) Fetch(ctx context.Context, query *storage.FetchQuery, options *storage.FetchOptions) (*storage.FetchResult, error) { @@ -65,14 +68,64 @@ func (s *localStorage) Fetch(ctx context.Context, query *storage.FetchQuery, opt return nil, err } - opts := storage.FetchOptionsToM3Options(options, query) + // NB(r): Since we don't use a single index we fan out to each + // cluster that can completely fulfill this range and then prefer the + // highest resolution (most fine grained) results. + // This needs to be optimized, however this is a start. + var ( + opts = storage.FetchOptionsToM3Options(options, query) + namespaces = s.clusters.ClusterNamespaces() + now = time.Now() + fetches = 0 + result multiFetchResult + wg sync.WaitGroup + ) + for _, namespace := range namespaces { + namespace := namespace // Capture var + + clusterStart := now.Add(-1 * namespace.Attributes().Retention) + + // Only include if cluster can completely fulfill the range + if clusterStart.After(query.Start) { + continue + } + + fetches++ + + wg.Add(1) + go func() { + r, err := s.fetch(namespace, m3query, opts) + result.add(namespace.Attributes(), r, err) + wg.Done() + }() + } + + if fetches == 0 { + return nil, errNoLocalClustersFulfillsQuery + } + + wg.Wait() + if err := result.err.FinalError(); err != nil { + return nil, err + } + return result.result, nil +} + +func (s *localStorage) fetch( + namespace ClusterNamespace, + query index.Query, + opts index.QueryOptions, +) (*storage.FetchResult, error) { + namespaceID := namespace.NamespaceID() + session := namespace.Session() + // TODO (nikunj): Handle second return param - iters, _, err := s.session.FetchTagged(s.namespace, m3query, opts) + iters, _, err := session.FetchTagged(namespaceID, query, opts) if err != nil { return nil, err } - return storage.SeriesIteratorsToFetchResult(iters, s.namespace, s.workerPool) + return storage.SeriesIteratorsToFetchResult(iters, namespaceID, s.workerPool) } func (s *localStorage) FetchTags(ctx context.Context, query *storage.FetchQuery, options *storage.FetchOptions) (*storage.SearchResults, error) { @@ -90,9 +143,54 @@ func (s *localStorage) FetchTags(ctx context.Context, query *storage.FetchQuery, return nil, err } - opts := storage.FetchOptionsToM3Options(options, query) + var ( + opts = storage.FetchOptionsToM3Options(options, query) + namespaces = s.clusters.ClusterNamespaces() + now = time.Now() + fetches = 0 + result multiFetchTagsResult + wg sync.WaitGroup + ) + for _, namespace := range namespaces { + namespace := namespace // Capture var + + clusterStart := now.Add(-1 * namespace.Attributes().Retention) + + // Only include if cluster can completely fulfill the range + if clusterStart.After(query.Start) { + continue + } + + fetches++ + + wg.Add(1) + go func() { + result.add(s.fetchTags(namespace, m3query, opts)) + wg.Done() + }() + } + + if fetches == 0 { + return nil, errNoLocalClustersFulfillsQuery + } + + wg.Wait() + if err := result.err.FinalError(); err != nil { + return nil, err + } + return result.result, nil +} + +func (s *localStorage) fetchTags( + namespace ClusterNamespace, + query index.Query, + opts index.QueryOptions, +) (*storage.SearchResults, error) { + namespaceID := namespace.NamespaceID() + session := namespace.Session() + // TODO (juchan): Handle second return param - iter, _, err := s.session.FetchTaggedIDs(s.namespace, m3query, opts) + iter, _, err := session.FetchTaggedIDs(namespaceID, query, opts) if err != nil { return nil, err } @@ -106,6 +204,10 @@ func (s *localStorage) FetchTags(ctx context.Context, query *storage.FetchQuery, metrics = append(metrics, m) } + if err := iter.Err(); err != nil { + return nil, err + } + iter.Finalize() return &storage.SearchResults{ Metrics: metrics, @@ -131,6 +233,7 @@ func (s *localStorage) Write(ctx context.Context, query *storage.WriteQuery) err unit: query.Unit, id: id, tagIterator: storage.TagsToIdentTagIterator(query.Tags), + attributes: query.Attributes, } requests := make([]execution.Request, len(query.Datapoints)) @@ -161,11 +264,46 @@ func (s *localStorage) FetchBlocks( return res, nil } +func (s *localStorage) Close() error { + return nil +} + func (w *writeRequest) Process(ctx context.Context) error { common := w.writeRequestCommon store := common.store id := ident.StringID(common.id) - return store.session.WriteTagged(store.namespace, id, common.tagIterator, w.timestamp, w.value, common.unit, common.annotation) + + var ( + namespace ClusterNamespace + err error + ) + switch common.attributes.MetricsType { + case storage.UnaggregatedMetricsType: + namespace = store.clusters.UnaggregatedClusterNamespace() + case storage.AggregatedMetricsType: + attrs := RetentionResolution{ + Retention: common.attributes.Retention, + Resolution: common.attributes.Resolution, + } + var exists bool + namespace, exists = store.clusters.AggregatedClusterNamespace(attrs) + if !exists { + err = fmt.Errorf("no configured cluster namespace for: retention=%s, resolution=%s", + attrs.Retention.String(), attrs.Resolution.String()) + } + default: + metricsType := common.attributes.MetricsType + err = fmt.Errorf("invalid write request metrics type: %s (%d)", + metricsType.String(), uint(metricsType)) + } + if err != nil { + return err + } + + namespaceID := namespace.NamespaceID() + session := namespace.Session() + return session.WriteTagged(namespaceID, id, common.tagIterator, + w.timestamp, w.value, common.unit, common.annotation) } type writeRequestCommon struct { @@ -174,6 +312,7 @@ type writeRequestCommon struct { unit xtime.Unit id string tagIterator ident.TagIterator + attributes storage.Attributes } type writeRequest struct { @@ -190,7 +329,119 @@ func newWriteRequest(writeRequestCommon *writeRequestCommon, timestamp time.Time } } -func (s *localStorage) Close() error { - s.namespace.Finalize() - return nil +type multiFetchResult struct { + sync.Mutex + result *storage.FetchResult + err xerrors.MultiError + dedupeFirstAttrs storage.Attributes + dedupeMap map[string]multiFetchResultSeries +} + +type multiFetchResultSeries struct { + idx int + attrs storage.Attributes +} + +func (r *multiFetchResult) add( + attrs storage.Attributes, + result *storage.FetchResult, + err error, +) { + r.Lock() + defer r.Unlock() + + if err != nil { + r.err = r.err.Add(err) + return + } + + if r.result == nil { + r.result = result + r.dedupeFirstAttrs = attrs + return + } + + r.result.HasNext = r.result.HasNext && result.HasNext + r.result.LocalOnly = r.result.LocalOnly && result.LocalOnly + + // Need to dedupe + if r.dedupeMap == nil { + r.dedupeMap = make(map[string]multiFetchResultSeries, len(r.result.SeriesList)) + for idx, s := range r.result.SeriesList { + r.dedupeMap[s.Name()] = multiFetchResultSeries{ + idx: idx, + attrs: r.dedupeFirstAttrs, + } + } + } + + for _, s := range result.SeriesList { + id := s.Name() + existing, exists := r.dedupeMap[id] + if exists && existing.attrs.Resolution <= attrs.Resolution { + // Already exists and resolution of result we are adding is not as precise + continue + } + + // Does not exist already or more precise, add result + var idx int + if !exists { + idx = len(r.result.SeriesList) + r.result.SeriesList = append(r.result.SeriesList, s) + } else { + idx = existing.idx + r.result.SeriesList[idx] = s + } + + r.dedupeMap[id] = multiFetchResultSeries{ + idx: idx, + attrs: attrs, + } + } +} + +type multiFetchTagsResult struct { + sync.Mutex + result *storage.SearchResults + err xerrors.MultiError + dedupeMap map[string]struct{} +} + +func (r *multiFetchTagsResult) add( + result *storage.SearchResults, + err error, +) { + r.Lock() + defer r.Unlock() + + if err != nil { + r.err = r.err.Add(err) + return + } + + if r.result == nil { + r.result = result + return + } + + // Need to dedupe + if r.dedupeMap == nil { + r.dedupeMap = make(map[string]struct{}, len(r.result.Metrics)) + for _, s := range r.result.Metrics { + r.dedupeMap[s.ID] = struct{}{} + } + } + + for _, s := range result.Metrics { + id := s.ID + _, exists := r.dedupeMap[id] + if exists { + // Already exists + continue + } + + // Does not exist already, add result + r.result.Metrics = append(r.result.Metrics, s) + r.dedupeMap[id] = struct{}{} + } } diff --git a/src/coordinator/storage/local/storage_test.go b/src/coordinator/storage/local/storage_test.go index 9bda725b2a..14e9fb5dc1 100644 --- a/src/coordinator/storage/local/storage_test.go +++ b/src/coordinator/storage/local/storage_test.go @@ -22,16 +22,19 @@ package local import ( "context" + "fmt" + "math" + "strings" "testing" "time" - "github.com/m3db/m3db/src/coordinator/errors" "github.com/m3db/m3db/src/coordinator/models" "github.com/m3db/m3db/src/coordinator/storage" "github.com/m3db/m3db/src/coordinator/test/seriesiter" "github.com/m3db/m3db/src/coordinator/ts" "github.com/m3db/m3db/src/coordinator/util/logging" "github.com/m3db/m3db/src/dbnode/client" + "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" "github.com/golang/mock/gomock" @@ -39,13 +42,47 @@ import ( "github.com/stretchr/testify/require" ) -func setup(ctrl *gomock.Controller) (storage.Storage, *client.MockSession) { +var testRetention = 30 * 24 * time.Hour + +type testSessions struct { + unaggregated1MonthRetention *client.MockSession + aggregated1MonthRetention1MinuteResolution *client.MockSession +} + +func (s testSessions) forEach(fn func(session *client.MockSession)) { + for _, session := range []*client.MockSession{ + s.unaggregated1MonthRetention, + s.aggregated1MonthRetention1MinuteResolution, + } { + fn(session) + } +} + +func setup( + t *testing.T, + ctrl *gomock.Controller, +) (storage.Storage, testSessions) { logging.InitWithCores(nil) logger := logging.WithContext(context.TODO()) defer logger.Sync() - session := client.NewMockSession(ctrl) - storage := NewStorage(session, "metrics", nil) - return storage, session + unaggregated1MonthRetention := client.NewMockSession(ctrl) + aggregated1MonthRetention1MinuteResolution := client.NewMockSession(ctrl) + clusters, err := NewClusters(UnaggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID("metrics_unaggregated"), + Session: unaggregated1MonthRetention, + Retention: testRetention, + }, AggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID("metrics_aggregated"), + Session: aggregated1MonthRetention1MinuteResolution, + Retention: testRetention, + Resolution: time.Minute, + }) + require.NoError(t, err) + storage := NewStorage(clusters, nil) + return storage, testSessions{ + unaggregated1MonthRetention: unaggregated1MonthRetention, + aggregated1MonthRetention1MinuteResolution: aggregated1MonthRetention1MinuteResolution, + } } func newFetchReq() *storage.FetchQuery { @@ -85,32 +122,95 @@ func newWriteQuery() *storage.WriteQuery { } } -func setupLocalWrite(t *testing.T) storage.Storage { - ctrl := gomock.NewController(t) - store, session := setup(ctrl) - session.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() +func setupLocalWrite(t *testing.T, ctrl *gomock.Controller) storage.Storage { + store, sessions := setup(t, ctrl) + session := sessions.unaggregated1MonthRetention + session.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() return store } func TestLocalWriteEmpty(t *testing.T) { - store := setupLocalWrite(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store := setupLocalWrite(t, ctrl) err := store.Write(context.TODO(), nil) assert.Error(t, err) } func TestLocalWriteSuccess(t *testing.T) { - store := setupLocalWrite(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store := setupLocalWrite(t, ctrl) writeQuery := newWriteQuery() err := store.Write(context.TODO(), writeQuery) assert.NoError(t, err) assert.NoError(t, store.Close()) } +func TestLocalWriteAggregatedNoClusterNamespaceError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store, _ := setup(t, ctrl) + writeQuery := newWriteQuery() + // Use unsupported retention/resolution + writeQuery.Attributes = storage.Attributes{ + MetricsType: storage.AggregatedMetricsType, + Retention: 1234, + Resolution: 5678, + } + err := store.Write(context.TODO(), writeQuery) + assert.Error(t, err) + assert.True(t, strings.Contains(err.Error(), "no configured cluster namespace"), + fmt.Sprintf("unexpected error string: %v", err.Error())) +} + +func TestLocalWriteAggregatedInvalidMetricsTypeError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store, _ := setup(t, ctrl) + writeQuery := newWriteQuery() + // Use unsupported retention/resolution + writeQuery.Attributes = storage.Attributes{ + MetricsType: storage.MetricsType(math.MaxUint64), + Retention: 30 * 24 * time.Hour, + } + err := store.Write(context.TODO(), writeQuery) + assert.Error(t, err) + assert.True(t, strings.Contains(err.Error(), "invalid write request"), + fmt.Sprintf("unexpected error string: %v", err.Error())) +} + +func TestLocalWriteAggregatedSuccess(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store, sessions := setup(t, ctrl) + + writeQuery := newWriteQuery() + writeQuery.Attributes = storage.Attributes{ + MetricsType: storage.AggregatedMetricsType, + Retention: 30 * 24 * time.Hour, + Resolution: time.Minute, + } + + session := sessions.aggregated1MonthRetention1MinuteResolution + session.EXPECT().WriteTagged(gomock.Any(), gomock.Any(), gomock.Any(), + gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Times(len(writeQuery.Datapoints)) + + err := store.Write(context.TODO(), writeQuery) + assert.NoError(t, err) + assert.NoError(t, store.Close()) +} + func TestLocalRead(t *testing.T) { ctrl := gomock.NewController(t) - store, session := setup(ctrl) + defer ctrl.Finish() + store, sessions := setup(t, ctrl) testTags := seriesiter.GenerateTag() - session.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()).Return(seriesiter.NewMockSeriesIters(ctrl, testTags, 1, 2), true, nil) + sessions.forEach(func(session *client.MockSession) { + session.EXPECT().FetchTagged(gomock.Any(), gomock.Any(), gomock.Any()). + Return(seriesiter.NewMockSeriesIters(ctrl, testTags, 1, 2), true, nil) + }) searchReq := newFetchReq() results, err := store.Fetch(context.TODO(), searchReq, &storage.FetchOptions{Limit: 100}) assert.NoError(t, err) @@ -123,16 +223,113 @@ func TestLocalRead(t *testing.T) { assert.Equal(t, tags, results.SeriesList[0].Tags) } -func setupLocalSearch(t *testing.T) storage.Storage { +func TestLocalReadNoClustersForTimeRangeError(t *testing.T) { ctrl := gomock.NewController(t) - store, session := setup(ctrl) - session.EXPECT().FetchTaggedIDs(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, false, errors.ErrNotImplemented) - return store + defer ctrl.Finish() + store, _ := setup(t, ctrl) + searchReq := newFetchReq() + searchReq.Start = time.Now().Add(-2 * testRetention) + searchReq.End = time.Now() + _, err := store.Fetch(context.TODO(), searchReq, &storage.FetchOptions{Limit: 100}) + require.Error(t, err) + assert.Equal(t, errNoLocalClustersFulfillsQuery, err) } -func TestLocalSearchExpectedFail(t *testing.T) { - store := setupLocalSearch(t) +func TestLocalSearchError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store, sessions := setup(t, ctrl) + sessions.forEach(func(session *client.MockSession) { + session.EXPECT().FetchTaggedIDs(gomock.Any(), gomock.Any(), gomock.Any()). + Return(nil, false, fmt.Errorf("an error")) + }) + searchReq := newFetchReq() _, err := store.FetchTags(context.TODO(), searchReq, &storage.FetchOptions{Limit: 100}) assert.Error(t, err) } + +func TestLocalSearchSuccess(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + store, sessions := setup(t, ctrl) + + type testFetchTaggedID struct { + id string + namespace string + tagName string + tagValue string + } + + fetches := []testFetchTaggedID{ + { + id: "foo", + namespace: "metrics_unaggregated", + tagName: "qux", + tagValue: "qaz", + }, + { + id: "bar", + namespace: "metrics_aggregated", + tagName: "qel", + tagValue: "quz", + }, + } + + sessions.forEach(func(session *client.MockSession) { + var f testFetchTaggedID + switch { + case session == sessions.unaggregated1MonthRetention: + f = fetches[0] + case session == sessions.aggregated1MonthRetention1MinuteResolution: + f = fetches[1] + default: + require.FailNow(t, "unexpected session") + } + iter := client.NewMockTaggedIDsIterator(ctrl) + gomock.InOrder( + iter.EXPECT().Next().Return(true), + iter.EXPECT().Current().Return( + ident.StringID(f.namespace), + ident.StringID(f.id), + ident.NewTagsIterator(ident.NewTags( + ident.Tag{ + Name: ident.StringID(f.tagName), + Value: ident.StringID(f.tagValue), + })), + ), + iter.EXPECT().Next().Return(false), + iter.EXPECT().Err().Return(nil), + iter.EXPECT().Finalize(), + ) + + session.EXPECT().FetchTaggedIDs(gomock.Any(), gomock.Any(), gomock.Any()). + Return(iter, true, nil) + }) + searchReq := newFetchReq() + result, err := store.FetchTags(context.TODO(), searchReq, &storage.FetchOptions{Limit: 100}) + require.NoError(t, err) + + require.Equal(t, len(fetches), len(result.Metrics)) + + expected := make(map[string]testFetchTaggedID) + for _, f := range fetches { + expected[f.id] = f + } + + actual := make(map[string]*models.Metric) + for _, m := range result.Metrics { + actual[m.ID] = m + } + + for id, actual := range actual { + expected, ok := expected[id] + require.True(t, ok) + + assert.Equal(t, expected.id, actual.ID) + assert.Equal(t, expected.namespace, actual.Namespace) + assert.Equal(t, models.Tags{ + expected.tagName: expected.tagValue, + }, actual.Tags) + } +} diff --git a/src/coordinator/test/local/storage.go b/src/coordinator/test/local/storage.go index 4aaf685665..772495132f 100644 --- a/src/coordinator/test/local/storage.go +++ b/src/coordinator/test/local/storage.go @@ -1,16 +1,59 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + package local import ( + "testing" + "time" + "github.com/m3db/m3db/src/coordinator/storage" "github.com/m3db/m3db/src/coordinator/storage/local" "github.com/m3db/m3db/src/dbnode/client" + "github.com/m3db/m3x/ident" "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" +) + +const ( + // TestNamespaceID is the namespace of the test unaggregated namespace + // used by local storage. + TestNamespaceID = "metrics" + // TestRetention is the retention of the test unaggregated namespace + // used by local storage. + TestRetention = 30 * 24 * time.Hour ) // NewStorageAndSession generates a new local storage and mock session -func NewStorageAndSession(ctrl *gomock.Controller) (storage.Storage, *client.MockSession) { +func NewStorageAndSession( + t *testing.T, + ctrl *gomock.Controller, +) (storage.Storage, *client.MockSession) { session := client.NewMockSession(ctrl) - storage := local.NewStorage(session, "metrics", nil) + clusters, err := local.NewClusters(local.UnaggregatedClusterNamespaceDefinition{ + NamespaceID: ident.StringID(TestNamespaceID), + Session: session, + Retention: TestRetention, + }) + require.NoError(t, err) + storage := local.NewStorage(clusters, nil) return storage, session } diff --git a/src/coordinator/test/seriesiter/mock_iter.go b/src/coordinator/test/seriesiter/mock_iter.go index 0292520c21..365801ee91 100644 --- a/src/coordinator/test/seriesiter/mock_iter.go +++ b/src/coordinator/test/seriesiter/mock_iter.go @@ -61,10 +61,16 @@ func NewMockSeriesIterSlice(ctrl *gomock.Controller, tagGenerator func() ident.T mockIter.EXPECT().Next().Return(false).Times(1) now := time.Now() for i := 0; i < numValues; i++ { - mockIter.EXPECT().Current().Return(m3ts.Datapoint{Timestamp: now.Add(time.Duration(i * 10) * time.Second), Value: float64(i)}, xtime.Millisecond, nil).Times(1) + mockIter.EXPECT().Current().Return(m3ts.Datapoint{Timestamp: now.Add(time.Duration(i*10) * time.Second), Value: float64(i)}, xtime.Millisecond, nil).Times(1) } + + tags := tagGenerator() mockIter.EXPECT().ID().Return(ident.StringID("foo")).Times(1) - mockIter.EXPECT().Tags().Return(tagGenerator()).Times(1) + mockIter.EXPECT().Tags().Return(tags).Times(1) + mockIter.EXPECT().Close().Do(func() { + // Make sure to close the tags generated when closing the iter + tags.Close() + }) iteratorList = append(iteratorList, mockIter) } @@ -81,7 +87,12 @@ func NewMockSeriesIters(ctrl *gomock.Controller, tags ident.Tag, len int, numVal mockIters := encoding.NewMockSeriesIterators(ctrl) mockIters.EXPECT().Iters().Return(iteratorList) mockIters.EXPECT().Len().Return(len) - mockIters.EXPECT().Close() + mockIters.EXPECT().Close().Do(func() { + // Make sure to close the iterators + for _, iter := range iteratorList { + iter.Close() + } + }) return mockIters }