github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/query/storage/m3/cluster.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 m3
    22  
    23  import (
    24  	"errors"
    25  	"fmt"
    26  	"io"
    27  	"sync"
    28  	"time"
    29  
    30  	"github.com/m3db/m3/src/dbnode/client"
    31  	"github.com/m3db/m3/src/query/storage/m3/storagemetadata"
    32  	xerrors "github.com/m3db/m3/src/x/errors"
    33  	"github.com/m3db/m3/src/x/ident"
    34  )
    35  
    36  var (
    37  	errNamespaceIDNotSet   = errors.New("namespace ID not set")
    38  	errSessionNotSet       = errors.New("session not set")
    39  	errRetentionNotSet     = errors.New("retention not set")
    40  	errResolutionNotSet    = errors.New("resolution not set")
    41  	errNegativeDataLatency = errors.New("negative dataLatency")
    42  
    43  	// DefaultClusterNamespaceDownsampleOptions is a default options.
    44  	// NB(antanas): this was made public to access it in promremote storage.
    45  	// Ideally downsampling could be decoupled from m3 storage.
    46  	DefaultClusterNamespaceDownsampleOptions = ClusterNamespaceDownsampleOptions{
    47  		All: true,
    48  	}
    49  )
    50  
    51  // ClusterConfigType is an enum representing the configuration used
    52  // to create a Clusters interface
    53  type ClusterConfigType int
    54  
    55  const (
    56  	// ClusterConfigTypeStatic is for static configuration.
    57  	ClusterConfigTypeStatic = iota
    58  	// ClusterConfigTypeDynamic is for dynamic configuration.
    59  	ClusterConfigTypeDynamic
    60  )
    61  
    62  // Clusters is a flattened collection of local storage clusters and namespaces.
    63  type Clusters interface {
    64  	io.Closer
    65  
    66  	// ClusterNamespaces returns all known and ready cluster namespaces.
    67  	ClusterNamespaces() ClusterNamespaces
    68  
    69  	// NonReadyClusterNamespaces returns all cluster namespaces not in the ready state.
    70  	NonReadyClusterNamespaces() ClusterNamespaces
    71  
    72  	// UnaggregatedClusterNamespace returns the valid unaggregated
    73  	// cluster namespace. If the namespace is not yet initialized, returns false.
    74  	UnaggregatedClusterNamespace() (ClusterNamespace, bool)
    75  
    76  	// AggregatedClusterNamespace returns an aggregated cluster namespace
    77  	// at a specific retention and resolution.
    78  	AggregatedClusterNamespace(attrs RetentionResolution) (ClusterNamespace, bool)
    79  
    80  	// ConfigType returns the type of configuration used to create this Clusters
    81  	// object.
    82  	ConfigType() ClusterConfigType
    83  }
    84  
    85  // RetentionResolution is a tuple of retention and resolution that describes
    86  // an aggregated metrics policy.
    87  type RetentionResolution struct {
    88  	Retention  time.Duration
    89  	Resolution time.Duration
    90  }
    91  
    92  // ClusterNamespace is a local storage cluster namespace.
    93  type ClusterNamespace interface {
    94  	NamespaceID() ident.ID
    95  	Options() ClusterNamespaceOptions
    96  	Session() client.Session
    97  }
    98  
    99  // ClusterNamespaceOptions is a set of options
   100  type ClusterNamespaceOptions struct {
   101  	// Note: Don't allow direct access, as we want to provide defaults
   102  	// and/or error if call to access a field is not relevant/correct.
   103  	attributes  storagemetadata.Attributes
   104  	downsample  *ClusterNamespaceDownsampleOptions
   105  	dataLatency time.Duration
   106  	readOnly    bool
   107  }
   108  
   109  // NewClusterNamespaceOptions creates new cluster namespace options.
   110  func NewClusterNamespaceOptions(
   111  	attributes storagemetadata.Attributes,
   112  	downsample *ClusterNamespaceDownsampleOptions,
   113  ) ClusterNamespaceOptions {
   114  	return ClusterNamespaceOptions{
   115  		attributes: attributes,
   116  		downsample: downsample,
   117  	}
   118  }
   119  
   120  // Attributes returns the storage attributes of the cluster namespace.
   121  func (o ClusterNamespaceOptions) Attributes() storagemetadata.Attributes {
   122  	return o.attributes
   123  }
   124  
   125  // DataLatency returns the duration after which the data is available in this cluster namespace.
   126  func (o ClusterNamespaceOptions) DataLatency() time.Duration {
   127  	return o.dataLatency
   128  }
   129  
   130  // ReadOnly returns the value of ReadOnly option for a cluster namespace.
   131  func (o ClusterNamespaceOptions) ReadOnly() bool {
   132  	return o.readOnly
   133  }
   134  
   135  // DownsampleOptions returns the downsample options for a cluster namespace,
   136  // which is only valid if the namespace is an aggregated cluster namespace.
   137  func (o ClusterNamespaceOptions) DownsampleOptions() (
   138  	ClusterNamespaceDownsampleOptions,
   139  	error,
   140  ) {
   141  	if o.attributes.MetricsType != storagemetadata.AggregatedMetricsType {
   142  		return ClusterNamespaceDownsampleOptions{}, errNotAggregatedClusterNamespace
   143  	}
   144  	if o.downsample == nil {
   145  		return DefaultClusterNamespaceDownsampleOptions, nil
   146  	}
   147  	return *o.downsample, nil
   148  }
   149  
   150  // ClusterNamespaceDownsampleOptions is the downsample options for
   151  // a cluster namespace.
   152  type ClusterNamespaceDownsampleOptions struct {
   153  	All bool
   154  }
   155  
   156  // ClusterNamespaces is a slice of ClusterNamespace instances.
   157  type ClusterNamespaces []ClusterNamespace
   158  
   159  // NumAggregatedClusterNamespaces returns the number of aggregated
   160  // cluster namespaces.
   161  func (n ClusterNamespaces) NumAggregatedClusterNamespaces() int {
   162  	count := 0
   163  	for _, namespace := range n {
   164  		if namespace.Options().Attributes().MetricsType == storagemetadata.AggregatedMetricsType {
   165  			count++
   166  		}
   167  	}
   168  	return count
   169  }
   170  
   171  // UnaggregatedClusterNamespaceDefinition is the definition for the
   172  // cluster namespace that holds unaggregated metrics data.
   173  type UnaggregatedClusterNamespaceDefinition struct {
   174  	NamespaceID ident.ID
   175  	Session     client.Session
   176  	Retention   time.Duration
   177  }
   178  
   179  // Validate will validate the cluster namespace definition.
   180  func (def UnaggregatedClusterNamespaceDefinition) Validate() error {
   181  	if def.NamespaceID == nil || len(def.NamespaceID.String()) == 0 {
   182  		return errNamespaceIDNotSet
   183  	}
   184  	if def.Session == nil {
   185  		return errSessionNotSet
   186  	}
   187  	if def.Retention <= 0 {
   188  		return errRetentionNotSet
   189  	}
   190  	return nil
   191  }
   192  
   193  // AggregatedClusterNamespaceDefinition is a definition for a
   194  // cluster namespace that holds aggregated metrics data at a
   195  // specific retention and resolution.
   196  type AggregatedClusterNamespaceDefinition struct {
   197  	NamespaceID ident.ID
   198  	Session     client.Session
   199  	Retention   time.Duration
   200  	Resolution  time.Duration
   201  	Downsample  *ClusterNamespaceDownsampleOptions
   202  	DataLatency time.Duration
   203  	ReadOnly    bool
   204  }
   205  
   206  // Validate validates the cluster namespace definition.
   207  func (def AggregatedClusterNamespaceDefinition) Validate() error {
   208  	if def.NamespaceID == nil || len(def.NamespaceID.String()) == 0 {
   209  		return errNamespaceIDNotSet
   210  	}
   211  	if def.Session == nil {
   212  		return errSessionNotSet
   213  	}
   214  	if def.Retention <= 0 {
   215  		return errRetentionNotSet
   216  	}
   217  	if def.Resolution <= 0 {
   218  		return errResolutionNotSet
   219  	}
   220  	if def.DataLatency < 0 {
   221  		return errNegativeDataLatency
   222  	}
   223  	return nil
   224  }
   225  
   226  type clusters struct {
   227  	namespaces            []ClusterNamespace
   228  	unaggregatedNamespace ClusterNamespace
   229  	aggregatedNamespaces  map[RetentionResolution]ClusterNamespace
   230  }
   231  
   232  // NewClusters instantiates a new Clusters instance.
   233  func NewClusters(
   234  	unaggregatedClusterNamespace UnaggregatedClusterNamespaceDefinition,
   235  	aggregatedClusterNamespaces ...AggregatedClusterNamespaceDefinition,
   236  ) (Clusters, error) {
   237  	expectedAggregated := len(aggregatedClusterNamespaces)
   238  	expectedAll := 1 + expectedAggregated
   239  	namespaces := make(ClusterNamespaces, 0, expectedAll)
   240  	aggregatedNamespaces := make(map[RetentionResolution]ClusterNamespace,
   241  		expectedAggregated)
   242  
   243  	def := unaggregatedClusterNamespace
   244  	unaggregatedNamespace, err := newUnaggregatedClusterNamespace(def)
   245  	if err != nil {
   246  		return nil, err
   247  	}
   248  
   249  	namespaces = append(namespaces, unaggregatedNamespace)
   250  	for _, def := range aggregatedClusterNamespaces {
   251  		namespace, err := newAggregatedClusterNamespace(def)
   252  		if err != nil {
   253  			return nil, err
   254  		}
   255  
   256  		namespaces = append(namespaces, namespace)
   257  		key := RetentionResolution{
   258  			Retention:  namespace.Options().Attributes().Retention,
   259  			Resolution: namespace.Options().Attributes().Resolution,
   260  		}
   261  
   262  		_, exists := aggregatedNamespaces[key]
   263  		if exists {
   264  			return nil, fmt.Errorf("duplicate aggregated namespace exists for: "+
   265  				"retention=%s, resolution=%s",
   266  				key.Retention.String(), key.Resolution.String())
   267  		}
   268  
   269  		aggregatedNamespaces[key] = namespace
   270  	}
   271  
   272  	return &clusters{
   273  		namespaces:            namespaces,
   274  		unaggregatedNamespace: unaggregatedNamespace,
   275  		aggregatedNamespaces:  aggregatedNamespaces,
   276  	}, nil
   277  }
   278  
   279  func (c *clusters) ClusterNamespaces() ClusterNamespaces {
   280  	return c.namespaces
   281  }
   282  
   283  func (c *clusters) NonReadyClusterNamespaces() ClusterNamespaces {
   284  	// statically configured cluster namespaces are always considered ready.
   285  	return nil
   286  }
   287  
   288  func (c *clusters) UnaggregatedClusterNamespace() (ClusterNamespace, bool) {
   289  	return c.unaggregatedNamespace, true
   290  }
   291  
   292  func (c *clusters) AggregatedClusterNamespace(
   293  	attrs RetentionResolution,
   294  ) (ClusterNamespace, bool) {
   295  	namespace, ok := c.aggregatedNamespaces[attrs]
   296  	return namespace, ok
   297  }
   298  
   299  func (c *clusters) ConfigType() ClusterConfigType {
   300  	return ClusterConfigTypeStatic
   301  }
   302  
   303  func (c *clusters) Close() error {
   304  	var (
   305  		wg             sync.WaitGroup
   306  		syncMultiErrs  syncMultiErrs
   307  		uniqueSessions []client.Session
   308  	)
   309  	// Collect unique sessions, some namespaces may share same
   310  	// client session (same cluster)
   311  	uniqueSessions = append(uniqueSessions, c.unaggregatedNamespace.Session())
   312  	for _, namespace := range c.aggregatedNamespaces {
   313  		unique := true
   314  		for _, session := range uniqueSessions {
   315  			if namespace.Session() == session {
   316  				unique = false
   317  				break
   318  			}
   319  		}
   320  		if unique {
   321  			uniqueSessions = append(uniqueSessions, namespace.Session())
   322  		}
   323  	}
   324  
   325  	for _, session := range uniqueSessions {
   326  		session := session // Capture for lambda
   327  		wg.Add(1)
   328  		go func() {
   329  			defer wg.Done()
   330  			err := session.Close()
   331  			syncMultiErrs.add(err)
   332  		}()
   333  	}
   334  
   335  	wg.Wait()
   336  
   337  	return syncMultiErrs.lastError()
   338  }
   339  
   340  type clusterNamespace struct {
   341  	namespaceID ident.ID
   342  	options     ClusterNamespaceOptions
   343  	session     client.Session
   344  }
   345  
   346  func newUnaggregatedClusterNamespace(
   347  	def UnaggregatedClusterNamespaceDefinition,
   348  ) (ClusterNamespace, error) {
   349  	if err := def.Validate(); err != nil {
   350  		return nil, err
   351  	}
   352  
   353  	ns := def.NamespaceID
   354  	// Set namespace to NoFinalize to avoid cloning it in write operations
   355  	ns.NoFinalize()
   356  	return &clusterNamespace{
   357  		namespaceID: ns,
   358  		options: ClusterNamespaceOptions{
   359  			attributes: storagemetadata.Attributes{
   360  				MetricsType: storagemetadata.UnaggregatedMetricsType,
   361  				Retention:   def.Retention,
   362  			},
   363  		},
   364  		session: def.Session,
   365  	}, nil
   366  }
   367  
   368  func newAggregatedClusterNamespace(
   369  	def AggregatedClusterNamespaceDefinition,
   370  ) (ClusterNamespace, error) {
   371  	if err := def.Validate(); err != nil {
   372  		return nil, err
   373  	}
   374  
   375  	ns := def.NamespaceID
   376  	// Set namespace to NoFinalize to avoid cloning it in write operations
   377  	ns.NoFinalize()
   378  	return &clusterNamespace{
   379  		namespaceID: ns,
   380  		options: ClusterNamespaceOptions{
   381  			attributes: storagemetadata.Attributes{
   382  				MetricsType: storagemetadata.AggregatedMetricsType,
   383  				Retention:   def.Retention,
   384  				Resolution:  def.Resolution,
   385  			},
   386  			downsample:  def.Downsample,
   387  			dataLatency: def.DataLatency,
   388  			readOnly:    def.ReadOnly,
   389  		},
   390  		session: def.Session,
   391  	}, nil
   392  }
   393  
   394  func (n *clusterNamespace) NamespaceID() ident.ID {
   395  	return n.namespaceID
   396  }
   397  
   398  func (n *clusterNamespace) Options() ClusterNamespaceOptions {
   399  	return n.options
   400  }
   401  
   402  func (n *clusterNamespace) Session() client.Session {
   403  	return n.session
   404  }
   405  
   406  type syncMultiErrs struct {
   407  	sync.Mutex
   408  	multiErr xerrors.MultiError
   409  }
   410  
   411  func (errs *syncMultiErrs) add(err error) {
   412  	errs.Lock()
   413  	errs.multiErr = errs.multiErr.Add(err)
   414  	errs.Unlock()
   415  }
   416  
   417  func (errs *syncMultiErrs) lastError() error {
   418  	errs.Lock()
   419  	defer errs.Unlock()
   420  	// TODO: consider taking a debug param when building a syncMultiErrs
   421  	// which would determine wether to return only the last error message
   422  	// or the consolidated list of errors.
   423  	return errs.multiErr.LastError()
   424  }