github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/query/storage/fanout/storage.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 fanout
    22  
    23  import (
    24  	"bytes"
    25  	"context"
    26  	"fmt"
    27  	"sync"
    28  	"time"
    29  
    30  	"github.com/m3db/m3/src/query/block"
    31  	"github.com/m3db/m3/src/query/errors"
    32  	"github.com/m3db/m3/src/query/models"
    33  	"github.com/m3db/m3/src/query/policy/filter"
    34  	"github.com/m3db/m3/src/query/storage"
    35  	"github.com/m3db/m3/src/query/storage/m3"
    36  	"github.com/m3db/m3/src/query/storage/m3/consolidators"
    37  	"github.com/m3db/m3/src/query/storage/m3/storagemetadata"
    38  	"github.com/m3db/m3/src/query/util/execution"
    39  	xerrors "github.com/m3db/m3/src/x/errors"
    40  	"github.com/m3db/m3/src/x/instrument"
    41  
    42  	"go.uber.org/zap"
    43  )
    44  
    45  const (
    46  	initMetricMapSize     = 10
    47  	fetchDataWarningError = "fetch_data_error"
    48  )
    49  
    50  type fanoutStorage struct {
    51  	stores             []storage.Storage
    52  	fetchFilter        filter.Storage
    53  	writeFilter        filter.Storage
    54  	completeTagsFilter filter.StorageCompleteTags
    55  	tagOptions         models.TagOptions
    56  	opts               m3.Options
    57  	instrumentOpts     instrument.Options
    58  }
    59  
    60  // NewStorage creates a new fanout Storage instance.
    61  func NewStorage(
    62  	stores []storage.Storage,
    63  	fetchFilter filter.Storage,
    64  	writeFilter filter.Storage,
    65  	completeTagsFilter filter.StorageCompleteTags,
    66  	tagOptions models.TagOptions,
    67  	opts m3.Options,
    68  	instrumentOpts instrument.Options,
    69  ) storage.Storage {
    70  	return &fanoutStorage{
    71  		stores:             stores,
    72  		fetchFilter:        fetchFilter,
    73  		writeFilter:        writeFilter,
    74  		completeTagsFilter: completeTagsFilter,
    75  		tagOptions:         tagOptions,
    76  		opts:               opts,
    77  		instrumentOpts:     instrumentOpts,
    78  	}
    79  }
    80  
    81  func (s *fanoutStorage) QueryStorageMetadataAttributes(
    82  	ctx context.Context,
    83  	queryStart, queryEnd time.Time,
    84  	opts *storage.FetchOptions,
    85  ) ([]storagemetadata.Attributes, error) {
    86  	// Optimization for the single store case
    87  	if len(s.stores) == 1 {
    88  		return s.stores[0].QueryStorageMetadataAttributes(ctx, queryStart, queryEnd, opts)
    89  	}
    90  
    91  	found := make(map[storagemetadata.Attributes]bool)
    92  	for _, store := range s.stores {
    93  		attrs, err := store.QueryStorageMetadataAttributes(ctx, queryStart, queryEnd, opts)
    94  		if err != nil {
    95  			return nil, err
    96  		}
    97  		for _, attr := range attrs {
    98  			found[attr] = true
    99  		}
   100  	}
   101  
   102  	attrs := make([]storagemetadata.Attributes, 0, len(found))
   103  	for attr := range found {
   104  		attrs = append(attrs, attr)
   105  	}
   106  
   107  	return attrs, nil
   108  }
   109  
   110  func (s *fanoutStorage) FetchProm(
   111  	ctx context.Context,
   112  	query *storage.FetchQuery,
   113  	options *storage.FetchOptions,
   114  ) (storage.PromResult, error) {
   115  	stores := filterStores(s.stores, s.fetchFilter, query)
   116  	// Optimization for the single store case
   117  	if len(stores) == 1 {
   118  		return stores[0].FetchProm(ctx, query, options)
   119  	}
   120  
   121  	var (
   122  		mu         sync.Mutex
   123  		wg         sync.WaitGroup
   124  		multiErr   xerrors.MultiError
   125  		numWarning int
   126  	)
   127  
   128  	wg.Add(len(stores))
   129  
   130  	var (
   131  		fanout    = consolidators.NamespaceCoversAllQueryRange
   132  		matchOpts = s.opts.SeriesConsolidationMatchOptions()
   133  		tagOpts   = s.opts.TagOptions()
   134  		limitOpts = consolidators.LimitOptions{
   135  			Limit:             options.SeriesLimit,
   136  			RequireExhaustive: options.RequireExhaustive,
   137  		}
   138  
   139  		accumulator = consolidators.NewMultiFetchResult(fanout, matchOpts, tagOpts, limitOpts)
   140  	)
   141  
   142  	defer func() {
   143  		_ = accumulator.Close()
   144  	}()
   145  	for _, store := range stores {
   146  		store := store
   147  		go func() {
   148  			defer wg.Done()
   149  
   150  			storeResult, err := store.FetchCompressed(ctx, query, options)
   151  
   152  			mu.Lock()
   153  			defer mu.Unlock()
   154  
   155  			if err != nil {
   156  				warning, err := storage.IsWarning(store, err)
   157  				if !warning {
   158  					multiErr = multiErr.Add(err)
   159  					s.instrumentOpts.Logger().Error(
   160  						"fanout to store returned error",
   161  						zap.Error(err),
   162  						zap.String("store", store.Name()),
   163  						zap.String("function", "FetchProm"))
   164  					return
   165  				}
   166  
   167  				// Is warning, add to accumulator but also process any results.
   168  				accumulator.AddWarnings(block.Warning{
   169  					Name:    store.Name(),
   170  					Message: fetchDataWarningError,
   171  				})
   172  				numWarning++
   173  				s.instrumentOpts.Logger().Warn(
   174  					"partial results: fanout to store returned warning",
   175  					zap.Error(err),
   176  					zap.String("store", store.Name()),
   177  					zap.String("function", "FetchProm"))
   178  			}
   179  
   180  			if storeResult == nil {
   181  				return
   182  			}
   183  
   184  			for _, r := range storeResult.Results() {
   185  				accumulator.Add(r)
   186  			}
   187  		}()
   188  	}
   189  
   190  	wg.Wait()
   191  	// NB: Check multiError first; if any hard error storages errored, the entire
   192  	// query must be errored.
   193  	if err := multiErr.FinalError(); err != nil {
   194  		return storage.PromResult{}, err
   195  	}
   196  
   197  	// If there were no successful results at all, return a normal error.
   198  	if numWarning > 0 && numWarning == len(stores) {
   199  		return storage.PromResult{}, errors.ErrNoValidResults
   200  	}
   201  
   202  	result, attrs, err := accumulator.FinalResultWithAttrs()
   203  	if err != nil {
   204  		return storage.PromResult{}, err
   205  	}
   206  
   207  	resolutions := make([]time.Duration, 0, len(attrs))
   208  	for _, attr := range attrs {
   209  		resolutions = append(resolutions, attr.Resolution)
   210  	}
   211  
   212  	result.Metadata.Resolutions = resolutions
   213  	return storage.SeriesIteratorsToPromResult(ctx, result,
   214  		s.opts.ReadWorkerPool(), s.opts.TagOptions(), s.opts.PromConvertOptions(), options)
   215  }
   216  
   217  func (s *fanoutStorage) FetchCompressed(
   218  	ctx context.Context,
   219  	query *storage.FetchQuery,
   220  	options *storage.FetchOptions,
   221  ) (consolidators.MultiFetchResult, error) {
   222  	stores := filterStores(s.stores, s.fetchFilter, query)
   223  	// Optimization for the single store case
   224  	if len(stores) == 1 {
   225  		return stores[0].FetchCompressed(ctx, query, options)
   226  	}
   227  
   228  	var (
   229  		mu       sync.Mutex
   230  		wg       sync.WaitGroup
   231  		multiErr xerrors.MultiError
   232  	)
   233  
   234  	wg.Add(len(stores))
   235  
   236  	var (
   237  		fanout    = consolidators.NamespaceCoversAllQueryRange
   238  		matchOpts = s.opts.SeriesConsolidationMatchOptions()
   239  		tagOpts   = s.opts.TagOptions()
   240  		limitOpts = consolidators.LimitOptions{
   241  			Limit:             options.SeriesLimit,
   242  			RequireExhaustive: options.RequireExhaustive,
   243  		}
   244  
   245  		accumulator = consolidators.NewMultiFetchResult(fanout, matchOpts, tagOpts, limitOpts)
   246  	)
   247  
   248  	defer func() {
   249  		_ = accumulator.Close()
   250  	}()
   251  	for _, store := range stores {
   252  		store := store
   253  		go func() {
   254  			defer wg.Done()
   255  			storeResult, err := store.FetchCompressed(ctx, query, options)
   256  			mu.Lock()
   257  			defer mu.Unlock()
   258  
   259  			if err != nil {
   260  				multiErr = multiErr.Add(err)
   261  				s.instrumentOpts.Logger().Error(
   262  					"fanout to store returned error",
   263  					zap.Error(err),
   264  					zap.String("store", store.Name()),
   265  					zap.String("function", "FetchProm"))
   266  				return
   267  			}
   268  
   269  			for _, r := range storeResult.Results() {
   270  				accumulator.Add(r)
   271  			}
   272  		}()
   273  	}
   274  
   275  	wg.Wait()
   276  	// NB: Check multiError first; if any hard error storages errored, the entire
   277  	// query must be errored.
   278  	if err := multiErr.FinalError(); err != nil {
   279  		return nil, err
   280  	}
   281  
   282  	return accumulator, nil
   283  }
   284  
   285  func (s *fanoutStorage) FetchBlocks(
   286  	ctx context.Context,
   287  	query *storage.FetchQuery,
   288  	options *storage.FetchOptions,
   289  ) (block.Result, error) {
   290  	stores := filterStores(s.stores, s.fetchFilter, query)
   291  	// Optimization for the single store case
   292  	if len(stores) == 1 {
   293  		return stores[0].FetchBlocks(ctx, query, options)
   294  	}
   295  
   296  	var (
   297  		mu         sync.Mutex
   298  		wg         sync.WaitGroup
   299  		multiErr   xerrors.MultiError
   300  		numWarning int
   301  	)
   302  
   303  	// TODO(arnikola): update this to use a genny map
   304  	blockResult := make(map[string]block.Block, len(stores))
   305  	wg.Add(len(stores))
   306  	resultMeta := block.NewResultMetadata()
   307  	for _, store := range stores {
   308  		store := store
   309  		go func() {
   310  			defer wg.Done()
   311  
   312  			result, err := store.FetchBlocks(ctx, query, options)
   313  
   314  			mu.Lock()
   315  			defer mu.Unlock()
   316  
   317  			if err != nil {
   318  				if warning, err := storage.IsWarning(store, err); warning {
   319  					resultMeta.AddWarning(store.Name(), fetchDataWarningError)
   320  					numWarning++
   321  					s.instrumentOpts.Logger().Warn(
   322  						"partial results: fanout to store returned warning",
   323  						zap.Error(err),
   324  						zap.String("store", store.Name()),
   325  						zap.String("function", "FetchBlocks"))
   326  					return
   327  				}
   328  
   329  				multiErr = multiErr.Add(err)
   330  				s.instrumentOpts.Logger().Error(
   331  					"fanout to store returned error",
   332  					zap.Error(err),
   333  					zap.String("store", store.Name()),
   334  					zap.String("function", "FetchBlocks"))
   335  				return
   336  			}
   337  
   338  			resultMeta = resultMeta.CombineMetadata(result.Metadata)
   339  			for _, bl := range result.Blocks {
   340  				key := bl.Meta().String()
   341  				foundBlock, found := blockResult[key]
   342  				if !found {
   343  					blockResult[key] = bl
   344  					continue
   345  				}
   346  
   347  				// This block exists. Check to see if it's already an appendable block.
   348  				blockType := foundBlock.Info().Type()
   349  				if blockType != block.BlockContainer {
   350  					var err error
   351  					blockResult[key], err = block.NewContainerBlock(foundBlock, bl)
   352  					if err != nil {
   353  						multiErr = multiErr.Add(err)
   354  						return
   355  					}
   356  
   357  					continue
   358  				}
   359  
   360  				accumulator, ok := foundBlock.(block.AccumulatorBlock)
   361  				if !ok {
   362  					multiErr = multiErr.Add(fmt.Errorf("container block has incorrect type"))
   363  					return
   364  				}
   365  
   366  				// Already an accumulator block, add current block.
   367  				if err := accumulator.AddBlock(bl); err != nil {
   368  					multiErr = multiErr.Add(err)
   369  					return
   370  				}
   371  			}
   372  		}()
   373  	}
   374  
   375  	wg.Wait()
   376  	// NB: Check multiError first; if any hard error storages errored, the entire
   377  	// query must be errored.
   378  	if err := multiErr.FinalError(); err != nil {
   379  		return block.Result{}, err
   380  	}
   381  
   382  	// If there were no successful results at all, return a normal error.
   383  	if numWarning > 0 && numWarning == len(stores) {
   384  		return block.Result{}, errors.ErrNoValidResults
   385  	}
   386  
   387  	blocks := make([]block.Block, 0, len(blockResult))
   388  	updateResultMeta := func(meta block.Metadata) block.Metadata {
   389  		meta.ResultMetadata = meta.ResultMetadata.CombineMetadata(resultMeta)
   390  		return meta
   391  	}
   392  
   393  	lazyOpts := block.NewLazyOptions().SetMetaTransform(updateResultMeta)
   394  	for _, bl := range blockResult {
   395  		// Update constituent blocks with combined resultMetadata if it has been
   396  		// changed.
   397  		if !resultMeta.IsDefault() {
   398  			bl = block.NewLazyBlock(bl, lazyOpts)
   399  		}
   400  
   401  		blocks = append(blocks, bl)
   402  	}
   403  
   404  	return block.Result{
   405  		Blocks:   blocks,
   406  		Metadata: resultMeta,
   407  	}, nil
   408  }
   409  
   410  func (s *fanoutStorage) SearchSeries(
   411  	ctx context.Context,
   412  	query *storage.FetchQuery,
   413  	options *storage.FetchOptions,
   414  ) (*storage.SearchResults, error) {
   415  	// TODO: arnikola use a genny map here instead, or better yet, hide this
   416  	// behind an accumulator.
   417  	metricMap := make(map[string]models.Metric, initMetricMapSize)
   418  	stores := filterStores(s.stores, s.fetchFilter, query)
   419  	metadata := block.NewResultMetadata()
   420  	for _, store := range stores {
   421  		results, err := store.SearchSeries(ctx, query, options)
   422  		if err != nil {
   423  			if warning, err := storage.IsWarning(store, err); warning {
   424  				metadata.AddWarning(store.Name(), fetchDataWarningError)
   425  				s.instrumentOpts.Logger().Warn(
   426  					"partial results: fanout to store returned warning",
   427  					zap.Error(err),
   428  					zap.String("store", store.Name()),
   429  					zap.String("function", "SearchSeries"))
   430  				continue
   431  			}
   432  
   433  			s.instrumentOpts.Logger().Error(
   434  				"fanout to store returned error",
   435  				zap.Error(err),
   436  				zap.String("store", store.Name()),
   437  				zap.String("function", "SearchSeries"))
   438  			return nil, err
   439  		}
   440  
   441  		metadata = metadata.CombineMetadata(results.Metadata)
   442  		for _, metric := range results.Metrics {
   443  			id := string(metric.ID)
   444  			if existing, found := metricMap[id]; found {
   445  				existing.Tags = existing.Tags.AddTagsIfNotExists(metric.Tags.Tags)
   446  				metricMap[id] = existing
   447  			} else {
   448  				metricMap[id] = metric
   449  			}
   450  		}
   451  	}
   452  
   453  	metrics := make(models.Metrics, 0, len(metricMap))
   454  	for _, v := range metricMap {
   455  		metrics = append(metrics, v)
   456  	}
   457  
   458  	result := &storage.SearchResults{
   459  		Metrics:  metrics,
   460  		Metadata: metadata,
   461  	}
   462  
   463  	return result, nil
   464  }
   465  
   466  func (s *fanoutStorage) CompleteTags(
   467  	ctx context.Context,
   468  	query *storage.CompleteTagsQuery,
   469  	options *storage.FetchOptions,
   470  ) (*consolidators.CompleteTagsResult, error) {
   471  	stores := filterCompleteTagsStores(s.stores, s.completeTagsFilter, *query)
   472  
   473  	var completeTagsResult consolidators.CompleteTagsResult
   474  
   475  	// short circuit complete tags
   476  	if len(stores) == 1 {
   477  		result, err := stores[0].CompleteTags(ctx, query, options)
   478  		if err != nil {
   479  			return result, err
   480  		}
   481  		completeTagsResult = *result
   482  	} else {
   483  		accumulatedTags := consolidators.NewCompleteTagsResultBuilder(
   484  			query.CompleteNameOnly, s.tagOptions)
   485  		metadata := block.NewResultMetadata()
   486  		for _, store := range stores {
   487  			result, err := store.CompleteTags(ctx, query, options)
   488  			if err != nil {
   489  				if warning, err := storage.IsWarning(store, err); warning {
   490  					metadata.AddWarning(store.Name(), fetchDataWarningError)
   491  					s.instrumentOpts.Logger().Warn(
   492  						"partial results: fanout to store returned warning",
   493  						zap.Error(err),
   494  						zap.String("store", store.Name()),
   495  						zap.String("function", "CompleteTags"))
   496  					continue
   497  				}
   498  
   499  				s.instrumentOpts.Logger().Error(
   500  					"fanout to store returned error",
   501  					zap.Error(err),
   502  					zap.String("store", store.Name()),
   503  					zap.String("function", "CompleteTags"))
   504  
   505  				return nil, err
   506  			}
   507  
   508  			metadata = metadata.CombineMetadata(result.Metadata)
   509  			err = accumulatedTags.Add(result)
   510  			if err != nil {
   511  				return nil, err
   512  			}
   513  		}
   514  
   515  		completeTagsResult = accumulatedTags.Build()
   516  		completeTagsResult.Metadata = metadata
   517  	}
   518  
   519  	completeTagsResult = applyOptions(completeTagsResult, options)
   520  	return &completeTagsResult, nil
   521  }
   522  
   523  func applyOptions(
   524  	result consolidators.CompleteTagsResult,
   525  	opts *storage.FetchOptions,
   526  ) consolidators.CompleteTagsResult {
   527  	if opts.RestrictQueryOptions == nil {
   528  		return result
   529  	}
   530  
   531  	filter := opts.RestrictQueryOptions.GetRestrictByTag().GetFilterByNames()
   532  	if len(filter) > 0 {
   533  		// Filter out unwanted tags inplace.
   534  		filteredList := result.CompletedTags[:0]
   535  		for _, s := range result.CompletedTags {
   536  			skip := false
   537  			for _, name := range filter {
   538  				if bytes.Equal(s.Name, name) {
   539  					skip = true
   540  					break
   541  				}
   542  			}
   543  
   544  			if skip {
   545  				continue
   546  			}
   547  
   548  			filteredList = append(filteredList, s)
   549  		}
   550  
   551  		result.CompletedTags = filteredList
   552  	}
   553  
   554  	return result
   555  }
   556  
   557  func (s *fanoutStorage) Write(ctx context.Context,
   558  	query *storage.WriteQuery) error {
   559  	// TODO: Consider removing this lookup on every write by maintaining
   560  	//  different read/write lists
   561  	stores := filterStores(s.stores, s.writeFilter, query)
   562  	// short circuit writes
   563  	if len(stores) == 1 {
   564  		return stores[0].Write(ctx, query)
   565  	}
   566  
   567  	requests := make([]execution.Request, 0, len(stores))
   568  	for _, store := range stores {
   569  		requests = append(requests, newWriteRequest(store, query))
   570  	}
   571  
   572  	return execution.ExecuteParallel(ctx, requests)
   573  }
   574  
   575  func (s *fanoutStorage) ErrorBehavior() storage.ErrorBehavior {
   576  	return storage.BehaviorFail
   577  }
   578  
   579  func (s *fanoutStorage) Type() storage.Type {
   580  	return storage.TypeMultiDC
   581  }
   582  
   583  func (s *fanoutStorage) Name() string {
   584  	inner := make([]string, 0, len(s.stores))
   585  	for _, store := range s.stores {
   586  		inner = append(inner, store.Name())
   587  	}
   588  
   589  	return fmt.Sprintf("fanout_store, inner: %v", inner)
   590  }
   591  
   592  func (s *fanoutStorage) Close() error {
   593  	var lastErr error
   594  	for idx, store := range s.stores {
   595  		// Keep going on error to close all storages
   596  		if err := store.Close(); err != nil {
   597  			s.instrumentOpts.Logger().Error("unable to close storage",
   598  				zap.Int("store", int(store.Type())), zap.Int("index", idx))
   599  			lastErr = err
   600  		}
   601  	}
   602  
   603  	return lastErr
   604  }
   605  
   606  func filterStores(
   607  	stores []storage.Storage,
   608  	filterPolicy filter.Storage,
   609  	query storage.Query,
   610  ) []storage.Storage {
   611  	filtered := make([]storage.Storage, 0, len(stores))
   612  	for _, s := range stores {
   613  		if filterPolicy(query, s) {
   614  			filtered = append(filtered, s)
   615  		}
   616  	}
   617  
   618  	return filtered
   619  }
   620  
   621  func filterCompleteTagsStores(
   622  	stores []storage.Storage,
   623  	filterPolicy filter.StorageCompleteTags,
   624  	query storage.CompleteTagsQuery,
   625  ) []storage.Storage {
   626  	filtered := make([]storage.Storage, 0, len(stores))
   627  	for _, s := range stores {
   628  		if filterPolicy(query, s) {
   629  			filtered = append(filtered, s)
   630  		}
   631  	}
   632  
   633  	return filtered
   634  }
   635  
   636  type writeRequest struct {
   637  	store storage.Storage
   638  	query *storage.WriteQuery
   639  }
   640  
   641  func newWriteRequest(store storage.Storage, query *storage.WriteQuery) execution.Request {
   642  	return &writeRequest{
   643  		store: store,
   644  		query: query,
   645  	}
   646  }
   647  
   648  func (f *writeRequest) Process(ctx context.Context) error {
   649  	return f.store.Write(ctx, f.query)
   650  }