github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/cmd/tools/dtest/harness/harness.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 harness
    22  
    23  import (
    24  	"fmt"
    25  	"io"
    26  	"io/ioutil"
    27  	"net/http"
    28  	_ "net/http/pprof" // _ is used for pprof
    29  	"os"
    30  	"path"
    31  	"strconv"
    32  	"sync"
    33  	"sync/atomic"
    34  	"time"
    35  
    36  	etcdclient "github.com/m3db/m3/src/cluster/client/etcd"
    37  	"github.com/m3db/m3/src/cluster/placement"
    38  	"github.com/m3db/m3/src/cluster/services"
    39  	"github.com/m3db/m3/src/cluster/shard"
    40  	"github.com/m3db/m3/src/cmd/tools/dtest/config"
    41  	"github.com/m3db/m3/src/cmd/tools/dtest/util"
    42  	"github.com/m3db/m3/src/cmd/tools/dtest/util/seed"
    43  	"github.com/m3db/m3/src/dbnode/integration/generate"
    44  	"github.com/m3db/m3/src/dbnode/kvconfig"
    45  	"github.com/m3db/m3/src/dbnode/namespace"
    46  	"github.com/m3db/m3/src/dbnode/retention"
    47  	"github.com/m3db/m3/src/dbnode/x/m3em/convert"
    48  	m3emnode "github.com/m3db/m3/src/dbnode/x/m3em/node"
    49  	"github.com/m3db/m3/src/m3em/build"
    50  	"github.com/m3db/m3/src/m3em/cluster"
    51  	hb "github.com/m3db/m3/src/m3em/generated/proto/heartbeat"
    52  	"github.com/m3db/m3/src/m3em/node"
    53  	xgrpc "github.com/m3db/m3/src/m3em/x/grpc"
    54  	xclock "github.com/m3db/m3/src/x/clock"
    55  	xerrors "github.com/m3db/m3/src/x/errors"
    56  	"github.com/m3db/m3/src/x/ident"
    57  	"github.com/m3db/m3/src/x/instrument"
    58  	xtcp "github.com/m3db/m3/src/x/tcp"
    59  
    60  	"github.com/gogo/protobuf/proto"
    61  	"go.uber.org/zap"
    62  )
    63  
    64  const (
    65  	buildFilename  = "m3dbnode"
    66  	configFilename = "m3dbnode.yaml"
    67  )
    68  
    69  type closeFn func() error
    70  
    71  // DTestHarness makes it easier to maintain the lifecycle of
    72  // resources used for DTests.
    73  type DTestHarness struct {
    74  	io.Closer
    75  
    76  	sync.Mutex
    77  	cluster cluster.Cluster
    78  
    79  	closing          int32
    80  	closers          []closeFn
    81  	cliOpts          *config.Args
    82  	conf             *config.Configuration
    83  	harnessDir       string
    84  	iopts            instrument.Options
    85  	logger           *zap.Logger
    86  	placementService placement.Service
    87  	nodeOpts         node.Options
    88  	clusterOpts      cluster.Options
    89  	nodes            []m3emnode.Node
    90  }
    91  
    92  // New constructs a new DTestHarness
    93  func New(cliOpts *config.Args, rawLogger *zap.Logger) *DTestHarness {
    94  	dt := &DTestHarness{
    95  		cliOpts: cliOpts,
    96  		logger:  rawLogger,
    97  		iopts:   instrument.NewOptions().SetLogger(rawLogger),
    98  	}
    99  
   100  	logger := rawLogger.Sugar()
   101  	// create temporary directory for use on local host
   102  	dir, err := ioutil.TempDir("", "dtest")
   103  	if err != nil {
   104  		logger.Fatalf("unable to create temp dir: %v", err.Error())
   105  	}
   106  	dt.harnessDir = dir
   107  	dt.addCloser(func() error {
   108  		return os.RemoveAll(dir)
   109  	})
   110  
   111  	// parse configuration
   112  	conf, err := config.New(cliOpts.DTestConfigPath)
   113  	if err != nil {
   114  		logger.Fatalf("unable to read configuration file: %v", err.Error())
   115  	}
   116  	dt.conf = conf
   117  	dt.startPProfServer()
   118  
   119  	zone, err := conf.Zone()
   120  	if err != nil {
   121  		logger.Fatalf("unable to read configuration zone: %v", err)
   122  	}
   123  
   124  	// make kv config
   125  	var (
   126  		svcID = dt.serviceID()
   127  		eopts = conf.KV.NewOptions()
   128  		popts = defaultPlacementOptions(zone, dt.iopts)
   129  	)
   130  	kvClient, err := etcdclient.NewConfigServiceClient(eopts)
   131  	if err != nil {
   132  		logger.Fatalf("unable to create kv client: %v", err)
   133  	}
   134  
   135  	// set the namespace in kv
   136  	kvStore, err := kvClient.KV()
   137  	if err != nil {
   138  		logger.Fatalf("unable to create kv store: %v", err)
   139  	}
   140  
   141  	protoValue, err := defaultNamespaceProtoValue()
   142  	if err != nil {
   143  		logger.Fatalf("unable to create proto value: %v", err)
   144  	}
   145  
   146  	_, err = kvStore.Set(kvconfig.NamespacesKey, protoValue)
   147  	if err != nil {
   148  		logger.Fatalf("unable to set initial namespace value: %v", err)
   149  	}
   150  
   151  	// register cleanup in the end
   152  	dt.addCloser(func() error {
   153  		_, err := kvStore.Delete(kvconfig.NamespacesKey)
   154  		return err
   155  	})
   156  
   157  	// make placement service
   158  	topoServices, err := kvClient.Services(nil)
   159  	if err != nil {
   160  		logger.Fatalf("unable to create topology services: %v", err)
   161  	}
   162  	pSvc, err := topoServices.PlacementService(svcID, popts)
   163  	if err != nil {
   164  		logger.Fatalf("unable to create placement service %v", err)
   165  	}
   166  	dt.placementService = pSvc
   167  
   168  	// set default node options
   169  	no := conf.M3EM.Node.Options(dt.iopts)
   170  	dt.nodeOpts = no.SetHeartbeatOptions(
   171  		no.HeartbeatOptions().
   172  			SetEnabled(true).
   173  			SetHeartbeatRouter(dt.newHeartbeatRouter()))
   174  
   175  	// parse node configurations
   176  	nodes, err := dt.conf.Nodes(dt.nodeOpts, cliOpts.NumNodes)
   177  	if err != nil {
   178  		logger.Fatalf("unable to create m3em nodes: %v", err)
   179  	}
   180  	dt.nodes = nodes
   181  
   182  	// default cluster options
   183  	co := conf.M3EM.Cluster.Options(dt.iopts)
   184  	dt.clusterOpts = co.
   185  		SetPlacementService(pSvc).
   186  		SetServiceBuild(newBuild(rawLogger, cliOpts.NodeBuildPath)).
   187  		SetServiceConfig(newConfig(rawLogger, cliOpts.NodeConfigPath)).
   188  		SetSessionToken(cliOpts.SessionToken).
   189  		SetSessionOverride(cliOpts.SessionOverride).
   190  		SetNodeListener(util.NewPullLogsAndPanicListener(rawLogger, dt.harnessDir))
   191  
   192  	if cliOpts.InitialReset {
   193  		dt.initialReset()
   194  	}
   195  
   196  	return dt
   197  }
   198  
   199  func (dt *DTestHarness) initialReset() {
   200  	svcNodes, err := convert.AsServiceNodes(dt.nodes)
   201  	if err != nil {
   202  		dt.logger.Fatal("unable to cast nodes", zap.Error(err))
   203  	}
   204  
   205  	var (
   206  		concurrency = dt.clusterOpts.NodeConcurrency()
   207  		timeout     = dt.clusterOpts.NodeOperationTimeout()
   208  		teardownFn  = func(n node.ServiceNode) error { return n.Teardown() }
   209  		exec        = node.NewConcurrentExecutor(svcNodes, concurrency, timeout, teardownFn)
   210  	)
   211  	if err := exec.Run(); err != nil {
   212  		dt.logger.Fatal("unable to reset nodes", zap.Error(err))
   213  	}
   214  }
   215  
   216  func (dt *DTestHarness) startPProfServer() {
   217  	serverAddress := fmt.Sprintf("0.0.0.0:%d", dt.conf.DTest.DebugPort)
   218  	go func() {
   219  		if err := http.ListenAndServe(serverAddress, nil); err != nil {
   220  			dt.logger.Fatal("unable to serve debug server", zap.Error(err))
   221  		}
   222  	}()
   223  	dt.logger.Info("serving pprof endpoints", zap.String("address", serverAddress))
   224  }
   225  
   226  // Close releases any resources held by the harness
   227  func (dt *DTestHarness) Close() error {
   228  	var multiErr xerrors.MultiError
   229  	atomic.StoreInt32(&dt.closing, 1)
   230  	for i := len(dt.closers) - 1; i >= 0; i-- {
   231  		closer := dt.closers[i]
   232  		multiErr = multiErr.Add(closer())
   233  	}
   234  	return multiErr.FinalError()
   235  }
   236  
   237  // Nodes returns the m3emnode.Node(s)
   238  func (dt *DTestHarness) Nodes() []m3emnode.Node {
   239  	return dt.nodes
   240  }
   241  
   242  // Configuration returns the parsed configuration struct
   243  func (dt *DTestHarness) Configuration() *config.Configuration {
   244  	return dt.conf
   245  }
   246  
   247  // Cluster constructs a cluster based on the options set in the harness
   248  func (dt *DTestHarness) Cluster() cluster.Cluster {
   249  	dt.Lock()
   250  	defer dt.Unlock()
   251  	if cluster := dt.cluster; cluster != nil {
   252  		return cluster
   253  	}
   254  
   255  	svcNodes, err := convert.AsServiceNodes(dt.nodes)
   256  	if err != nil {
   257  		dt.logger.Fatal("unable to cast nodes", zap.Error(err))
   258  	}
   259  
   260  	testCluster, err := cluster.New(svcNodes, dt.clusterOpts)
   261  	if err != nil {
   262  		dt.logger.Fatal("unable to create cluster", zap.Error(err))
   263  	}
   264  	dt.addCloser(testCluster.Teardown)
   265  	dt.cluster = testCluster
   266  	return testCluster
   267  }
   268  
   269  // ClusterOptions returns the cluster options
   270  func (dt *DTestHarness) ClusterOptions() cluster.Options {
   271  	return dt.clusterOpts
   272  }
   273  
   274  // SetClusterOptions sets the cluster options
   275  func (dt *DTestHarness) SetClusterOptions(co cluster.Options) {
   276  	dt.clusterOpts = co
   277  }
   278  
   279  // BootstrapTimeout returns the bootstrap timeout configued
   280  func (dt *DTestHarness) BootstrapTimeout() time.Duration {
   281  	return dt.Configuration().DTest.BootstrapTimeout
   282  }
   283  
   284  // Seed seeds the cluster nodes within the placement with data
   285  func (dt *DTestHarness) Seed(nodes []node.ServiceNode) error {
   286  	c := dt.Cluster()
   287  	if c.Status() == cluster.ClusterStatusUninitialized {
   288  		return fmt.Errorf("cluster must be Setup() prior to seeding it with data")
   289  	}
   290  
   291  	seedConfigs := dt.conf.DTest.Seeds
   292  	if len(seedConfigs) == 0 {
   293  		dt.logger.Info("no seed configurations provided, skipping.")
   294  		return nil
   295  	}
   296  
   297  	for _, conf := range seedConfigs {
   298  		if err := dt.seedWithConfig(nodes, conf); err != nil {
   299  			return err
   300  		}
   301  	}
   302  
   303  	return nil
   304  }
   305  
   306  func delayedNowFn(delay time.Duration) xclock.NowFn {
   307  	if delay == 0 {
   308  		return time.Now
   309  	}
   310  
   311  	return func() time.Time {
   312  		return time.Now().Add(-delay)
   313  	}
   314  }
   315  
   316  func (dt *DTestHarness) seedWithConfig(nodes []node.ServiceNode, seedConf config.SeedConfig) error {
   317  	dt.logger.Info("seeding data with configuration", zap.Any("config", seedConf))
   318  
   319  	seedDir := path.Join(dt.harnessDir, "seed", seedConf.Namespace)
   320  	if err := os.MkdirAll(seedDir, os.FileMode(0755)|os.ModeDir); err != nil {
   321  		return fmt.Errorf("unable to create seed dir: %v", err)
   322  	}
   323  
   324  	iopts := instrument.NewOptions().SetLogger(dt.logger)
   325  	generateOpts := generate.NewOptions().
   326  		SetRetentionPeriod(seedConf.Retention).
   327  		SetBlockSize(seedConf.BlockSize).
   328  		SetFilePathPrefix(seedDir).
   329  		SetClockOptions(xclock.NewOptions().SetNowFn(delayedNowFn(seedConf.Delay)))
   330  
   331  	seedDataOpts := seed.NewOptions().
   332  		SetInstrumentOptions(iopts).
   333  		SetGenerateOptions(generateOpts)
   334  
   335  	generator := seed.NewGenerator(seedDataOpts)
   336  	outputNamespace := ident.StringID(seedConf.Namespace)
   337  
   338  	if err := generator.Generate(namespace.Context{ID: outputNamespace}, seedConf.LocalShardNum); err != nil {
   339  		return fmt.Errorf("unable to generate data: %v", err)
   340  	}
   341  
   342  	generatedDataPath := path.Join(generateOpts.FilePathPrefix(), "data")
   343  	fakeShardDir := newShardDir(generatedDataPath, outputNamespace, seedConf.LocalShardNum)
   344  	localFiles, err := ioutil.ReadDir(fakeShardDir)
   345  	if err != nil {
   346  		return fmt.Errorf("unable to list local shard directory, err: %v", err)
   347  	}
   348  
   349  	// transfer the generated data to the remote hosts
   350  	var (
   351  		dataDir      = dt.conf.DTest.DataDir
   352  		co           = dt.ClusterOptions()
   353  		placement    = dt.Cluster().Placement()
   354  		concurrency  = co.NodeConcurrency()
   355  		timeout      = co.NodeOperationTimeout()
   356  		transferFunc = func(n node.ServiceNode) error {
   357  			for _, file := range localFiles {
   358  				base := path.Base(file.Name())
   359  				srcPath := path.Join(fakeShardDir, base)
   360  				paths := generatePaths(placement, n, outputNamespace, base, dataDir)
   361  				if err := n.TransferLocalFile(srcPath, paths, true); err != nil {
   362  					return err
   363  				}
   364  			}
   365  			return nil
   366  		}
   367  	)
   368  
   369  	dt.logger.Info("transferring data to nodes")
   370  	transferDataExecutor := node.NewConcurrentExecutor(nodes, concurrency, timeout, transferFunc)
   371  	if err := transferDataExecutor.Run(); err != nil {
   372  		return fmt.Errorf("unable to transfer generated data, err: %v", err)
   373  	}
   374  
   375  	return nil
   376  }
   377  
   378  func newShardDir(prefix string, ns ident.ID, shard uint32) string {
   379  	return path.Join(prefix, ns.String(), strconv.FormatUint(uint64(shard), 10))
   380  }
   381  
   382  func generatePaths(
   383  	placement placement.Placement,
   384  	n node.ServiceNode,
   385  	ns ident.ID,
   386  	file string,
   387  	dataDir string,
   388  ) []string {
   389  	paths := []string{}
   390  	pi, ok := placement.Instance(n.ID())
   391  	if !ok {
   392  		return paths
   393  	}
   394  	shards := pi.Shards().AllIDs()
   395  	for _, s := range shards {
   396  		paths = append(paths, path.Join(newShardDir(dataDir, ns, s), file))
   397  	}
   398  	return paths
   399  }
   400  
   401  // WaitUntilAllBootstrapped waits until all the provided nodes are bootstrapped, or
   402  // the configured bootstrap timeout period; whichever is sooner. It returns an error
   403  // indicating if all the nodes finished bootstrapping.
   404  func (dt *DTestHarness) WaitUntilAllBootstrapped(nodes []node.ServiceNode) error {
   405  	m3emnodes, err := convert.AsNodes(nodes)
   406  	if err != nil {
   407  		return fmt.Errorf("unable to cast nodes: %v", err)
   408  	}
   409  
   410  	watcher := util.NewNodesWatcher(m3emnodes, dt.logger, dt.conf.DTest.BootstrapReportInterval)
   411  	if allBootstrapped := watcher.WaitUntilAll(m3emnode.Node.Bootstrapped, dt.BootstrapTimeout()); !allBootstrapped {
   412  		return fmt.Errorf("unable to bootstrap all nodes, err = %v", watcher.PendingAsError())
   413  	}
   414  	return nil
   415  }
   416  
   417  // WaitUntilAllShardsAvailable waits until the placement service has all shards marked
   418  // available, or the configured bootstrap timeout period; whichever is sooner. It returns
   419  // an error indicating if all the nodes finished bootstrapping.
   420  func (dt *DTestHarness) WaitUntilAllShardsAvailable() error {
   421  	allAvailable := xclock.WaitUntil(dt.AllShardsAvailable, dt.BootstrapTimeout())
   422  	if !allAvailable {
   423  		return fmt.Errorf("all shards not available")
   424  	}
   425  	return nil
   426  }
   427  
   428  // AllShardsAvailable returns if the placement service has all shards marked available
   429  func (dt *DTestHarness) AllShardsAvailable() bool {
   430  	p, err := dt.placementService.Placement()
   431  	if err != nil {
   432  		return false
   433  	}
   434  
   435  	// for all instances
   436  	// for each shard
   437  	// if any shard is not available, return false
   438  	for _, inst := range p.Instances() {
   439  		for _, s := range inst.Shards().All() {
   440  			if s.State() != shard.Available {
   441  				return false
   442  			}
   443  		}
   444  	}
   445  
   446  	// all shards are available
   447  	return true
   448  }
   449  
   450  // AnyInstanceShardHasState returns a flag if the placement service has any instance
   451  // with the specified shard state
   452  func (dt *DTestHarness) AnyInstanceShardHasState(id string, state shard.State) bool {
   453  	p, err := dt.placementService.Placement()
   454  	if err != nil {
   455  		return false
   456  	}
   457  
   458  	inst, ok := p.Instance(id)
   459  	if !ok {
   460  		return false
   461  	}
   462  
   463  	for _, s := range inst.Shards().All() {
   464  		if s.State() == state {
   465  			return true
   466  		}
   467  	}
   468  
   469  	return false
   470  }
   471  
   472  func (dt *DTestHarness) newHeartbeatRouter() node.HeartbeatRouter {
   473  	hbPort := dt.conf.M3EM.HeartbeatPort
   474  	listenAddress := fmt.Sprintf("0.0.0.0:%d", hbPort)
   475  	listener, err := xtcp.NewTCPListener(listenAddress, 3*time.Minute)
   476  	if err != nil {
   477  		dt.logger.Fatal("could not create TCP Listener", zap.Error(err))
   478  	}
   479  	// listener is closed when hbServer.Serve returns
   480  
   481  	hostname, err := os.Hostname()
   482  	if err != nil {
   483  		dt.logger.Fatal("could not retrieve hostname", zap.Error(err))
   484  	}
   485  
   486  	externalAddress := fmt.Sprintf("%s:%d", hostname, hbPort)
   487  	hbRouter := node.NewHeartbeatRouter(externalAddress)
   488  	hbServer := xgrpc.NewServer(nil)
   489  	hb.RegisterHeartbeaterServer(hbServer, hbRouter)
   490  	go func() {
   491  		err := hbServer.Serve(listener)
   492  		if err != nil {
   493  			if closing := atomic.LoadInt32(&dt.closing); closing == 0 {
   494  				dt.logger.Fatal("could not create heartbeat server", zap.Error(err))
   495  			}
   496  			// we're closing the server, which will trigger this path. we don't want to error on it
   497  			dt.logger.Info("stopping heartbeatserver, server closed or inaccessible", zap.Error(err))
   498  		}
   499  	}()
   500  	dt.logger.Info("serving HeartbeatRouter",
   501  		zap.String("address", listenAddress), zap.String("external", externalAddress))
   502  
   503  	dt.addCloser(func() error {
   504  		hbServer.GracefulStop()
   505  		return nil
   506  	})
   507  	return hbRouter
   508  }
   509  
   510  func (dt *DTestHarness) addCloser(fn closeFn) {
   511  	dt.closers = append(dt.closers, fn)
   512  }
   513  
   514  func (dt *DTestHarness) serviceID() services.ServiceID {
   515  	return services.NewServiceID().
   516  		SetName(dt.conf.DTest.ServiceID).
   517  		SetEnvironment(dt.conf.KV.Env).
   518  		SetZone(dt.conf.KV.Zone)
   519  }
   520  
   521  func defaultPlacementOptions(zone string, iopts instrument.Options) placement.Options {
   522  	return placement.NewOptions().
   523  		SetIsSharded(true).
   524  		SetAllowPartialReplace(true).
   525  		SetInstrumentOptions(iopts).
   526  		SetValidZone(zone)
   527  }
   528  
   529  func newBuild(logger *zap.Logger, filename string) build.ServiceBuild {
   530  	bld := build.NewServiceBuild(buildFilename, filename)
   531  	logger.Info("marking service build", zap.Any("build", bld))
   532  	return bld
   533  }
   534  
   535  func newConfig(logger *zap.Logger, filename string) build.ServiceConfiguration {
   536  	bytes, err := ioutil.ReadFile(filename)
   537  	if err != nil {
   538  		logger.Fatal("unable to read", zap.String("filename", filename), zap.Error(err))
   539  	}
   540  	conf := build.NewServiceConfig(configFilename, bytes)
   541  	logger.Info("read service config", zap.String("filename", filename))
   542  	// TODO(prateek): once the main struct is OSS-ed, parse M3DB configuration,
   543  	// and ensure the following fields are correctly overridden/line up from dtest|m3em configs
   544  	// - kv (env|zone)
   545  	// - data directory
   546  	// - seed data configuration for block size, retention
   547  	return conf
   548  }
   549  
   550  func defaultNamespaceProtoValue() (proto.Message, error) {
   551  	md, err := namespace.NewMetadata(
   552  		ident.StringID("metrics"),
   553  		namespace.NewOptions().
   554  			SetBootstrapEnabled(true).
   555  			SetCleanupEnabled(true).
   556  			SetFlushEnabled(true).
   557  			SetRepairEnabled(true).
   558  			SetWritesToCommitLog(true).
   559  			SetRetentionOptions(
   560  				retention.NewOptions().
   561  					SetBlockSize(2*time.Hour).
   562  					SetRetentionPeriod(48*time.Hour)))
   563  	if err != nil {
   564  		return nil, err
   565  	}
   566  	nsMap, err := namespace.NewMap([]namespace.Metadata{md})
   567  	if err != nil {
   568  		return nil, err
   569  	}
   570  
   571  	registry, err := namespace.ToProto(nsMap)
   572  	if err != nil {
   573  		return nil, err
   574  	}
   575  
   576  	return registry, nil
   577  }