github.com/m3db/m3@v1.5.0/src/cluster/placementhandler/common.go (about)

     1  // Copyright (c) 2018 Uber Technologies, Inc.
     2  //
     3  // Permission is hereby granted, free of charge, to any person obtaining a copy
     4  // of this software and associated documentation files (the "Software"), to deal
     5  // in the Software without restriction, including without limitation the rights
     6  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     7  // copies of the Software, and to permit persons to whom the Software is
     8  // furnished to do so, subject to the following conditions:
     9  //
    10  // The above copyright notice and this permission notice shall be included in
    11  // all copies or substantial portions of the Software.
    12  //
    13  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    14  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    15  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    16  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    17  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    18  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    19  // THE SOFTWARE.
    20  
    21  // Package placementhandler provides debug/admin placement operations via HTTP
    22  package placementhandler
    23  
    24  import (
    25  	"errors"
    26  	"fmt"
    27  	"net/http"
    28  	"path"
    29  	"strings"
    30  	"time"
    31  
    32  	"github.com/m3db/m3/src/aggregator/aggregator"
    33  	clusterclient "github.com/m3db/m3/src/cluster/client"
    34  	"github.com/m3db/m3/src/cluster/generated/proto/placementpb"
    35  	"github.com/m3db/m3/src/cluster/kv"
    36  	"github.com/m3db/m3/src/cluster/placement"
    37  	"github.com/m3db/m3/src/cluster/placement/algo"
    38  	"github.com/m3db/m3/src/cluster/placementhandler/handleroptions"
    39  	"github.com/m3db/m3/src/cluster/services"
    40  	"github.com/m3db/m3/src/cluster/shard"
    41  	xerrors "github.com/m3db/m3/src/x/errors"
    42  	"github.com/m3db/m3/src/x/instrument"
    43  	xhttp "github.com/m3db/m3/src/x/net/http"
    44  )
    45  
    46  const (
    47  	// ServicesPathName is the services part of the API path.
    48  	ServicesPathName = "services"
    49  	// PlacementPathName is the placement part of the API path.
    50  	PlacementPathName = "placement"
    51  
    52  	m3AggregatorPlacementNamespace = "/placement"
    53  )
    54  
    55  var (
    56  	// M3DBServicePlacementPathName is the M3DB service placement API path.
    57  	M3DBServicePlacementPathName = path.Join(ServicesPathName,
    58  		handleroptions.M3DBServiceName, PlacementPathName)
    59  	// M3AggServicePlacementPathName is the M3Agg service placement API path.
    60  	M3AggServicePlacementPathName = path.Join(ServicesPathName,
    61  		handleroptions.M3AggregatorServiceName, PlacementPathName)
    62  	// M3CoordinatorServicePlacementPathName is the M3Coordinator
    63  	// service placement API path.
    64  	M3CoordinatorServicePlacementPathName = path.Join(ServicesPathName,
    65  		handleroptions.M3CoordinatorServiceName, PlacementPathName)
    66  
    67  	errUnableToParseService    = errors.New("unable to parse service")
    68  	errInstrumentOptionsNotSet = errors.New("instrument options not set")
    69  )
    70  
    71  // HandlerOptions is the options struct for the handler.
    72  type HandlerOptions struct {
    73  	// This is used by other placement Handlers
    74  	// nolint: structcheck
    75  	clusterClient clusterclient.Client
    76  	placement     placement.Configuration
    77  
    78  	m3AggServiceOptions *handleroptions.M3AggServiceOptions
    79  	instrumentOptions   instrument.Options
    80  }
    81  
    82  // Route stores paths from this handler that can be registered by clients.
    83  type Route struct {
    84  	Paths   []string
    85  	Methods []string
    86  	Handler http.Handler
    87  }
    88  
    89  // NewHandlerOptions is the constructor function for HandlerOptions.
    90  func NewHandlerOptions(
    91  	client clusterclient.Client,
    92  	placement placement.Configuration,
    93  	m3AggOpts *handleroptions.M3AggServiceOptions,
    94  	instrumentOpts instrument.Options,
    95  ) (HandlerOptions, error) {
    96  	if instrumentOpts == nil {
    97  		return HandlerOptions{}, errInstrumentOptionsNotSet
    98  	}
    99  	return HandlerOptions{
   100  		clusterClient:       client,
   101  		placement:           placement,
   102  		m3AggServiceOptions: m3AggOpts,
   103  		instrumentOptions:   instrumentOpts,
   104  	}, nil
   105  }
   106  
   107  // Handler represents a generic handler for placement endpoints.
   108  type Handler struct {
   109  	HandlerOptions
   110  
   111  	// nolint: structcheck
   112  	nowFn func() time.Time
   113  }
   114  
   115  // PlacementConfig returns the placement config.
   116  func (h Handler) PlacementConfig() placement.Configuration {
   117  	return h.placement
   118  }
   119  
   120  // PlacementConfigCopy returns a copy of the placement config.
   121  func (h Handler) PlacementConfigCopy() (placement.Configuration, error) {
   122  	return h.placement.DeepCopy()
   123  }
   124  
   125  // Service gets a placement service from m3cluster client
   126  func Service(
   127  	clusterClient clusterclient.Client,
   128  	opts handleroptions.ServiceOptions,
   129  	pConfig placement.Configuration,
   130  	now time.Time,
   131  	validationFn placement.ValidateFn,
   132  ) (placement.Service, error) {
   133  	ps, _, err := ServiceWithAlgo(clusterClient, opts, pConfig, now, validationFn)
   134  	return ps, err
   135  }
   136  
   137  // ServiceWithAlgo gets a placement service from m3cluster client and
   138  // additionally returns an algorithm instance for callers that need fine-grained
   139  // control over placement updates.
   140  func ServiceWithAlgo(
   141  	clusterClient clusterclient.Client,
   142  	opts handleroptions.ServiceOptions,
   143  	pConfig placement.Configuration,
   144  	now time.Time,
   145  	validationFn placement.ValidateFn,
   146  ) (placement.Service, placement.Algorithm, error) {
   147  	overrides := services.NewOverrideOptions()
   148  	switch opts.ServiceName {
   149  	case handleroptions.M3AggregatorServiceName:
   150  		overrides = overrides.
   151  			SetNamespaceOptions(
   152  				overrides.NamespaceOptions().
   153  					SetPlacementNamespace(m3AggregatorPlacementNamespace),
   154  			)
   155  	}
   156  
   157  	cs, err := clusterClient.Services(overrides)
   158  	if err != nil {
   159  		return nil, nil, err
   160  	}
   161  
   162  	if err := opts.Validate(); err != nil {
   163  		return nil, nil, err
   164  	}
   165  
   166  	if !handleroptions.IsAllowedService(opts.ServiceName) {
   167  		return nil, nil, fmt.Errorf(
   168  			"invalid service name: %s, must be one of: %v",
   169  			opts.ServiceName, handleroptions.AllowedServices())
   170  	}
   171  
   172  	sid := opts.ServiceID()
   173  	pOpts := pConfig.NewOptions().
   174  		SetValidZone(opts.ServiceZone).
   175  		SetIsSharded(true).
   176  		SetDryrun(opts.DryRun)
   177  
   178  	switch opts.ServiceName {
   179  	case handleroptions.M3CoordinatorServiceName:
   180  		if pConfig.IsSharded == nil {
   181  			// When no custom value is set, use shardless placement for m3coordinator.
   182  			pOpts = pOpts.SetIsSharded(false)
   183  		} else {
   184  			pOpts = pOpts.SetIsSharded(*pConfig.IsSharded)
   185  		}
   186  		if pConfig.ShardStateMode == nil {
   187  			pOpts = pOpts.SetShardStateMode(placement.StableShardStateOnly)
   188  		}
   189  	case handleroptions.M3AggregatorServiceName:
   190  		var (
   191  			maxAggregationWindowSize = opts.M3Agg.MaxAggregationWindowSize
   192  			warmupDuration           = opts.M3Agg.WarmupDuration
   193  			// For now these are not configurable, but we include them to
   194  			// make the code match r2admin for ease of debugging / migration.
   195  			placementCutoverOpts = m3aggregatorPlacementOpts{}
   196  		)
   197  		pOpts = pOpts.
   198  			// M3Agg expects a mirrored and staged placement.
   199  			SetIsMirrored(true).
   200  			SetIsStaged(true).
   201  			// placementCutover controls when the new placement will begin to be considered
   202  			// the new placement. Since we're trying to do goal-based placement, we set it
   203  			// such that it takes effect immediately.
   204  			SetPlacementCutoverNanosFn(newPlacementCutoverNanosFn(
   205  				now, placementCutoverOpts)).
   206  			// shardCutover controls when the clients (who have received the new placement)
   207  			// will begin dual-writing to the new shards. We could set it to take effect
   208  			// immediately, but right now we use the same logic as r2admin for consistency.
   209  			SetShardCutoverNanosFn(newShardCutoverNanosFn(
   210  				now, maxAggregationWindowSize, warmupDuration, placementCutoverOpts)).
   211  			// Cutoffs control when Leaving shards stop receiving writes.
   212  			SetShardCutoffNanosFn(newShardCutOffNanosFn(
   213  				now, maxAggregationWindowSize, warmupDuration, placementCutoverOpts)).
   214  			SetIsShardCutoverFn(newShardCutOverValidationFn(now)).
   215  			SetIsShardCutoffFn(newShardCutOffValidationFn(now, maxAggregationWindowSize))
   216  	}
   217  
   218  	if validationFn != nil {
   219  		pOpts = pOpts.SetValidateFnBeforeUpdate(validationFn)
   220  	}
   221  	ps, err := cs.PlacementService(sid, pOpts)
   222  	if err != nil {
   223  		return nil, nil, err
   224  	}
   225  
   226  	alg := algo.NewAlgorithm(pOpts)
   227  
   228  	return ps, alg, nil
   229  }
   230  
   231  // ConvertInstancesProto converts a slice of protobuf `Instance`s to `placement.Instance`s
   232  func ConvertInstancesProto(instancesProto []*placementpb.Instance) ([]placement.Instance, error) {
   233  	res := make([]placement.Instance, 0, len(instancesProto))
   234  
   235  	for _, instanceProto := range instancesProto {
   236  		instance, err := placement.NewInstanceFromProto(instanceProto)
   237  		if err != nil {
   238  			return nil, xerrors.NewInvalidParamsError(err)
   239  		}
   240  		res = append(res, instance)
   241  	}
   242  
   243  	return res, nil
   244  }
   245  
   246  // MakeRoutes creates routes for registration in http handlers
   247  func MakeRoutes(
   248  	defaults []handleroptions.ServiceOptionsDefault,
   249  	opts HandlerOptions,
   250  ) []Route {
   251  	// Init
   252  	var (
   253  		initHandler = NewInitHandler(opts)
   254  		initFn      = applyMiddleware(initHandler.ServeHTTP, defaults)
   255  		routes      []Route
   256  	)
   257  	routes = append(routes, Route{
   258  		Paths: []string{
   259  			M3DBInitURL,
   260  			M3AggInitURL,
   261  			M3CoordinatorInitURL,
   262  		},
   263  		Handler: initFn,
   264  		Methods: []string{InitHTTPMethod},
   265  	})
   266  
   267  	// Get
   268  	var (
   269  		getHandler = NewGetHandler(opts)
   270  		getFn      = applyMiddleware(getHandler.ServeHTTP, defaults)
   271  	)
   272  	routes = append(routes, Route{
   273  		Paths: []string{
   274  			M3DBGetURL,
   275  			M3AggGetURL,
   276  			M3CoordinatorGetURL,
   277  		},
   278  		Handler: getFn,
   279  		Methods: []string{GetHTTPMethod},
   280  	})
   281  
   282  	// Delete all
   283  	var (
   284  		deleteAllHandler = NewDeleteAllHandler(opts)
   285  		deleteAllFn      = applyMiddleware(deleteAllHandler.ServeHTTP, defaults)
   286  	)
   287  	routes = append(routes, Route{
   288  		Paths: []string{
   289  			M3DBDeleteAllURL,
   290  			M3AggDeleteAllURL,
   291  			M3CoordinatorDeleteAllURL,
   292  		},
   293  		Handler: deleteAllFn,
   294  		Methods: []string{DeleteAllHTTPMethod},
   295  	})
   296  
   297  	// Add
   298  	var (
   299  		addHandler = NewAddHandler(opts)
   300  		addFn      = applyMiddleware(addHandler.ServeHTTP, defaults)
   301  	)
   302  	routes = append(routes, Route{
   303  		Paths: []string{
   304  			M3DBAddURL,
   305  			M3AggAddURL,
   306  			M3CoordinatorAddURL,
   307  		},
   308  		Handler: addFn,
   309  		Methods: []string{AddHTTPMethod},
   310  	})
   311  
   312  	// Delete
   313  	var (
   314  		deleteHandler = NewDeleteHandler(opts)
   315  		deleteFn      = applyMiddleware(deleteHandler.ServeHTTP, defaults)
   316  	)
   317  	routes = append(routes, Route{
   318  		Paths: []string{
   319  			M3DBDeleteURL,
   320  			M3AggDeleteURL,
   321  			M3CoordinatorDeleteURL,
   322  		},
   323  		Handler: deleteFn,
   324  		Methods: []string{DeleteHTTPMethod},
   325  	})
   326  
   327  	// Remove
   328  	var (
   329  		removeHandler = NewRemoveHandler(opts)
   330  		removeFn      = applyMiddleware(removeHandler.ServeHTTP, defaults)
   331  	)
   332  	routes = append(routes, Route{
   333  		Paths: []string{
   334  			M3DBRemoveURL,
   335  			M3AggRemoveURL,
   336  			M3CoordinatorRemoveURL,
   337  		},
   338  		Handler: removeFn,
   339  		Methods: []string{RemoveHTTPMethod},
   340  	})
   341  
   342  	// Replace
   343  	var (
   344  		replaceHandler = NewReplaceHandler(opts)
   345  		replaceFn      = applyMiddleware(replaceHandler.ServeHTTP, defaults)
   346  	)
   347  	routes = append(routes, Route{
   348  		Paths: []string{
   349  			M3DBReplaceURL,
   350  			M3AggReplaceURL,
   351  			M3CoordinatorReplaceURL,
   352  		},
   353  		Handler: replaceFn,
   354  		Methods: []string{ReplaceHTTPMethod},
   355  	})
   356  
   357  	// Set
   358  	var (
   359  		setHandler = NewSetHandler(opts)
   360  		setFn      = applyMiddleware(setHandler.ServeHTTP, defaults)
   361  	)
   362  	routes = append(routes, Route{
   363  		Paths: []string{
   364  			M3DBSetURL,
   365  			M3AggSetURL,
   366  			M3CoordinatorSetURL,
   367  		},
   368  		Handler: setFn,
   369  		Methods: []string{SetHTTPMethod},
   370  	})
   371  
   372  	return routes
   373  }
   374  
   375  func newPlacementCutoverNanosFn(
   376  	now time.Time, cutoverOpts m3aggregatorPlacementOpts) placement.TimeNanosFn {
   377  	return func() int64 {
   378  		return placementCutoverTime(now, cutoverOpts).UnixNano()
   379  	}
   380  }
   381  
   382  func placementCutoverTime(
   383  	now time.Time, opts m3aggregatorPlacementOpts) time.Time {
   384  	return now.
   385  		Add(opts.maxPositiveSkew).
   386  		Add(opts.maxNegativeSkew).
   387  		Add(opts.propagationDelay)
   388  }
   389  
   390  func newShardCutOffNanosFn(
   391  	now time.Time,
   392  	maxAggregationWindowSize,
   393  	warmup time.Duration,
   394  	cutoverOpts m3aggregatorPlacementOpts) placement.TimeNanosFn {
   395  	return newShardCutoverNanosFn(
   396  		now, maxAggregationWindowSize, warmup, cutoverOpts)
   397  }
   398  
   399  func newShardCutoverNanosFn(
   400  	now time.Time,
   401  	maxAggregationWindowSize,
   402  	warmUpDuration time.Duration,
   403  	cutoverOpts m3aggregatorPlacementOpts) placement.TimeNanosFn {
   404  	return func() int64 {
   405  		var (
   406  			windowSize = maxAggregationWindowSize
   407  			cutover    = placementCutoverTime(now, cutoverOpts).Add(warmUpDuration)
   408  			truncated  = cutover.Truncate(windowSize)
   409  		)
   410  		if truncated.Before(cutover) {
   411  			return truncated.Add(windowSize).UnixNano()
   412  		}
   413  		return truncated.UnixNano()
   414  	}
   415  }
   416  
   417  func newShardCutOverValidationFn(now time.Time) placement.ShardValidateFn {
   418  	return func(s shard.Shard) error {
   419  		switch s.State() {
   420  		case shard.Initializing:
   421  			if s.CutoverNanos() > now.UnixNano() {
   422  				return fmt.Errorf("could not mark shard %d available before cutover time %v", s.ID(), time.Unix(0, s.CutoverNanos()))
   423  			}
   424  			return nil
   425  		default:
   426  			return fmt.Errorf("could not mark shard %d available, invalid state %s", s.ID(), s.State().String())
   427  		}
   428  	}
   429  }
   430  
   431  func newShardCutOffValidationFn(now time.Time, maxAggregationWindowSize time.Duration) placement.ShardValidateFn {
   432  	return func(s shard.Shard) error {
   433  		switch s.State() {
   434  		case shard.Leaving:
   435  			if s.CutoffNanos() > now.UnixNano()-maxAggregationWindowSize.Nanoseconds() {
   436  				return fmt.Errorf("could not return leaving shard %d with cutoff time %v, max aggregation window %v",
   437  					s.ID(), time.Unix(0, s.CutoffNanos()), maxAggregationWindowSize)
   438  			}
   439  			return nil
   440  		default:
   441  			return fmt.Errorf("could not mark shard %d available, invalid state %s", s.ID(), s.State().String())
   442  		}
   443  	}
   444  }
   445  
   446  type m3aggregatorPlacementOpts struct {
   447  	maxPositiveSkew  time.Duration
   448  	maxNegativeSkew  time.Duration
   449  	propagationDelay time.Duration
   450  }
   451  
   452  func validateAllAvailable(p placement.Placement) error {
   453  	var bad []string
   454  	for _, inst := range p.Instances() {
   455  		if !inst.IsAvailable() {
   456  			bad = append(bad, inst.ID())
   457  		}
   458  	}
   459  	if len(bad) > 0 {
   460  		str := strings.Join(bad, ", ")
   461  		err := fmt.Errorf("instances do not have all shards available: [%s]", str)
   462  		return xerrors.NewInvalidParamsError(err)
   463  	}
   464  	return nil
   465  }
   466  
   467  func applyMiddleware(
   468  	next func(svc handleroptions.ServiceNameAndDefaults, w http.ResponseWriter, r *http.Request),
   469  	defaults []handleroptions.ServiceOptionsDefault,
   470  ) http.Handler {
   471  	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
   472  		var (
   473  			svc = handleroptions.ServiceNameAndDefaults{Defaults: defaults}
   474  			err error
   475  		)
   476  		svc.ServiceName, err = parseServiceFromRequest(r)
   477  		if err != nil {
   478  			xhttp.WriteError(w, xhttp.NewError(err, http.StatusBadRequest))
   479  			return
   480  		}
   481  
   482  		next(svc, w, r)
   483  	})
   484  }
   485  
   486  func parseServiceFromRequest(r *http.Request) (string, error) {
   487  	path := r.URL.Path
   488  	components := strings.Split(path, "/")
   489  	for i, c := range components {
   490  		if c == "services" && i+1 < len(components) {
   491  			service := components[i+1]
   492  			if handleroptions.IsAllowedService(service) {
   493  				return service, nil
   494  			}
   495  			return "", fmt.Errorf("unknown service: %s", service)
   496  		}
   497  	}
   498  
   499  	return "", errUnableToParseService
   500  }
   501  
   502  func isStateless(serviceName string) bool {
   503  	switch serviceName {
   504  	case handleroptions.M3CoordinatorServiceName:
   505  		return true
   506  	}
   507  	return false
   508  }
   509  
   510  func deleteAggregatorShardSetIDRelatedKeys(
   511  	svc handleroptions.ServiceNameAndDefaults,
   512  	svcOpts handleroptions.ServiceOptions,
   513  	clusterClient clusterclient.Client,
   514  	shardSetIDs []uint32,
   515  ) error {
   516  	if svc.ServiceName != handleroptions.M3AggregatorServiceName {
   517  		return fmt.Errorf("error deleting aggregator instance keys, bad service: %s",
   518  			svc.ServiceName)
   519  	}
   520  
   521  	kvOpts := kv.NewOverrideOptions().
   522  		SetEnvironment(svcOpts.ServiceEnvironment).
   523  		SetZone(svcOpts.ServiceZone)
   524  
   525  	kvStore, err := clusterClient.Store(kvOpts)
   526  	if err != nil {
   527  		return fmt.Errorf("cannot get KV store to delete aggregator keys: %v", err)
   528  	}
   529  
   530  	var (
   531  		flushTimesMgrOpts = aggregator.NewFlushTimesManagerOptions()
   532  		electionMgrOpts   = aggregator.NewElectionManagerOptions()
   533  		multiErr          = xerrors.NewMultiError()
   534  	)
   535  	for _, shardSetID := range shardSetIDs {
   536  		// Check if flush times key exists, if so delete.
   537  		flushTimesKey := fmt.Sprintf(flushTimesMgrOpts.FlushTimesKeyFmt(),
   538  			shardSetID)
   539  		_, flushTimesKeyErr := kvStore.Get(flushTimesKey)
   540  		if flushTimesKeyErr != nil && flushTimesKeyErr != kv.ErrNotFound {
   541  			multiErr = multiErr.Add(fmt.Errorf(
   542  				"error check flush times key exists for deleted instance: %v",
   543  				flushTimesKeyErr))
   544  		}
   545  		if flushTimesKeyErr == nil {
   546  			// Need to delete the flush times key.
   547  			if _, err := kvStore.Delete(flushTimesKey); err != nil {
   548  				multiErr = multiErr.Add(fmt.Errorf(
   549  					"error delete flush times key for deleted instance: %v", err))
   550  			}
   551  		}
   552  
   553  		// Check if election manager lock key exists, if so delete.
   554  		electionKey := fmt.Sprintf(electionMgrOpts.ElectionKeyFmt(),
   555  			shardSetID)
   556  		_, electionKeyErr := kvStore.Get(electionKey)
   557  		if electionKeyErr != nil && electionKeyErr != kv.ErrNotFound {
   558  			multiErr = multiErr.Add(fmt.Errorf(
   559  				"error checking election key exists for deleted instance: %v", err))
   560  		}
   561  		if electionKeyErr == nil {
   562  			// Need to delete the election key.
   563  			if _, err := kvStore.Delete(flushTimesKey); err != nil {
   564  				multiErr = multiErr.Add(fmt.Errorf(
   565  					"error delete election key for deleted instance: %v", err))
   566  			}
   567  		}
   568  	}
   569  
   570  	return multiErr.FinalError()
   571  }