github.com/siglens/siglens@v0.0.0-20240328180423-f7ce9ae441ed/pkg/segment/results/segresults/segresults.go (about)

     1  /*
     2  Copyright 2023.
     3  
     4  Licensed under the Apache License, Version 2.0 (the "License");
     5  you may not use this file except in compliance with the License.
     6  You may obtain a copy of the License at
     7  
     8      http://www.apache.org/licenses/LICENSE-2.0
     9  
    10  Unless required by applicable law or agreed to in writing, software
    11  distributed under the License is distributed on an "AS IS" BASIS,
    12  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    13  See the License for the specific language governing permissions and
    14  limitations under the License.
    15  */
    16  
    17  package segresults
    18  
    19  import (
    20  	"encoding/json"
    21  	"fmt"
    22  	"sort"
    23  	"strconv"
    24  	"strings"
    25  	"sync"
    26  
    27  	"github.com/dustin/go-humanize"
    28  	dtu "github.com/siglens/siglens/pkg/common/dtypeutils"
    29  	"github.com/siglens/siglens/pkg/segment/aggregations"
    30  	"github.com/siglens/siglens/pkg/segment/reader/segread"
    31  	"github.com/siglens/siglens/pkg/segment/results/blockresults"
    32  	"github.com/siglens/siglens/pkg/segment/structs"
    33  	"github.com/siglens/siglens/pkg/segment/utils"
    34  	"github.com/siglens/siglens/pkg/segment/writer/stats"
    35  	log "github.com/sirupsen/logrus"
    36  )
    37  
    38  type EarlyExitType uint8
    39  
    40  const EMPTY_GROUPBY_KEY = "*"
    41  
    42  const (
    43  	EetContSearch EarlyExitType = iota + 1
    44  	EetMatchAllAggs
    45  	EetEarlyExit
    46  )
    47  
    48  func (e EarlyExitType) String() string {
    49  	switch e {
    50  	case EetContSearch:
    51  		return "Continue search"
    52  	case EetMatchAllAggs:
    53  		return "Match all aggs"
    54  	case EetEarlyExit:
    55  		return "Early exit"
    56  	default:
    57  		return fmt.Sprintf("Unknown early exit type %d", e)
    58  	}
    59  }
    60  
    61  // Stores information received by remote nodes for a query
    62  type remoteSearchResult struct {
    63  
    64  	// for RRCs in BlockResults that come from remote nodes, this map stores the raw logs
    65  	remoteLogs map[string]map[string]interface{}
    66  
    67  	// all columns that are present in the remote logs
    68  	remoteColumns map[string]struct{}
    69  }
    70  
    71  // exposes a struct to manage and maintain thread safe addition of results
    72  type SearchResults struct {
    73  	updateLock *sync.Mutex
    74  	queryType  structs.QueryType
    75  	qid        uint64
    76  	sAggs      *structs.QueryAggregators
    77  	sizeLimit  uint64
    78  
    79  	resultCount  uint64 // total count of results
    80  	EarlyExit    bool
    81  	BlockResults *blockresults.BlockResults // stores information about the matched RRCs
    82  	remoteInfo   *remoteSearchResult        // stores information about remote raw logs and columns
    83  
    84  	runningSegStat   []*structs.SegStats
    85  	segStatsResults  *segStatsResults
    86  	convertedBuckets map[string]*structs.AggregationResult
    87  	allSSTS          map[uint16]map[string]*structs.SegStats // maps segKeyEnc to a map of segstats
    88  	AllErrors        []error
    89  	SegKeyToEnc      map[string]uint16
    90  	SegEncToKey      map[uint16]string
    91  	MaxSegKeyEnc     uint16
    92  
    93  	statsAreFinal bool // If true, segStatsResults and convertedBuckets must not change.
    94  }
    95  
    96  type segStatsResults struct {
    97  	measureResults   map[string]utils.CValueEnclosure // maps agg function to result
    98  	measureFunctions []string
    99  	groupByCols      []string
   100  }
   101  
   102  func InitSearchResults(sizeLimit uint64, aggs *structs.QueryAggregators, qType structs.QueryType, qid uint64) (*SearchResults, error) {
   103  	lock := &sync.Mutex{}
   104  	blockResults, err := blockresults.InitBlockResults(sizeLimit, aggs, qid)
   105  	if err != nil {
   106  		log.Errorf("InitSearchResults: failed to initialize blockResults: %v", err)
   107  		return nil, err
   108  	}
   109  
   110  	allErrors := make([]error, 0)
   111  	var runningSegStat []*structs.SegStats
   112  	if aggs != nil && aggs.MeasureOperations != nil {
   113  		runningSegStat = make([]*structs.SegStats, len(aggs.MeasureOperations))
   114  	}
   115  	return &SearchResults{
   116  		queryType:    qType,
   117  		updateLock:   lock,
   118  		sizeLimit:    sizeLimit,
   119  		resultCount:  uint64(0),
   120  		BlockResults: blockResults,
   121  		remoteInfo: &remoteSearchResult{
   122  			remoteLogs:    make(map[string]map[string]interface{}),
   123  			remoteColumns: make(map[string]struct{}),
   124  		},
   125  		qid:            qid,
   126  		sAggs:          aggs,
   127  		allSSTS:        make(map[uint16]map[string]*structs.SegStats),
   128  		runningSegStat: runningSegStat,
   129  		AllErrors:      allErrors,
   130  		SegKeyToEnc:    make(map[string]uint16),
   131  		SegEncToKey:    make(map[uint16]string),
   132  		MaxSegKeyEnc:   1,
   133  	}, nil
   134  }
   135  
   136  func (sr *SearchResults) InitSegmentStatsResults(mOps []*structs.MeasureAggregator) {
   137  	sr.updateLock.Lock()
   138  	mFuncs := make([]string, len(mOps))
   139  	for i, op := range mOps {
   140  		mFuncs[i] = op.String()
   141  	}
   142  	retVal := make(map[string]utils.CValueEnclosure, len(mOps))
   143  	sr.segStatsResults = &segStatsResults{
   144  		measureResults:   retVal,
   145  		measureFunctions: mFuncs,
   146  	}
   147  	sr.updateLock.Unlock()
   148  }
   149  
   150  // checks if total count has been set and if any more raw records are needed
   151  // if retruns true, then only aggregations / sorts are needed
   152  func (sr *SearchResults) ShouldContinueRRCSearch() bool {
   153  	return sr.resultCount <= sr.sizeLimit
   154  }
   155  
   156  // Adds local results to the search results
   157  func (sr *SearchResults) AddBlockResults(blockRes *blockresults.BlockResults) {
   158  	sr.updateLock.Lock()
   159  	for _, rec := range blockRes.GetResults() {
   160  		_, removedID := sr.BlockResults.Add(rec)
   161  		sr.removeLog(removedID)
   162  	}
   163  	sr.resultCount += blockRes.MatchedCount
   164  	sr.BlockResults.MergeBuckets(blockRes)
   165  	sr.updateLock.Unlock()
   166  }
   167  
   168  // returns the raw, running buckets that have been created. This is used to merge with remote results
   169  func (sr *SearchResults) GetRunningBuckets() (*blockresults.TimeBuckets, *blockresults.GroupByBuckets) {
   170  	return sr.BlockResults.TimeAggregation, sr.BlockResults.GroupByAggregation
   171  }
   172  
   173  /*
   174  adds an entry to the remote logs
   175  
   176  the caller is responsible for ensuring that
   177  */
   178  func (sr *SearchResults) addRawLog(id string, log map[string]interface{}) {
   179  	sr.remoteInfo.remoteLogs[id] = log
   180  }
   181  
   182  /*
   183  Removes an entry from the remote logs
   184  */
   185  func (sr *SearchResults) removeLog(id string) {
   186  	if id == "" {
   187  		return
   188  	}
   189  	delete(sr.remoteInfo.remoteLogs, id)
   190  }
   191  
   192  func (sr *SearchResults) AddSSTMap(sstMap map[string]*structs.SegStats, skEnc uint16) {
   193  	sr.updateLock.Lock()
   194  	sr.allSSTS[skEnc] = sstMap
   195  	sr.updateLock.Unlock()
   196  }
   197  
   198  // deletes segKeyEnc from the map of allSSTS and any errors associated with it
   199  func (sr *SearchResults) GetEncodedSegStats(segKeyEnc uint16) ([]byte, error) {
   200  	sr.updateLock.Lock()
   201  	retVal, ok := sr.allSSTS[segKeyEnc]
   202  	delete(sr.allSSTS, segKeyEnc)
   203  	sr.updateLock.Unlock()
   204  	if !ok {
   205  		return nil, nil
   206  	}
   207  
   208  	allSegStatJson := make(map[string]*structs.SegStatsJSON, len(retVal))
   209  	for k, v := range retVal {
   210  		rawJson, err := v.ToJSON()
   211  		if err != nil {
   212  			log.Errorf("GetEncodedSegStats: failed to convert segstats to json: %v", err)
   213  			continue
   214  		}
   215  		allSegStatJson[k] = rawJson
   216  	}
   217  	allJSON := &structs.AllSegStatsJSON{
   218  		AllSegStats: allSegStatJson,
   219  	}
   220  	jsonBytes, err := json.Marshal(allJSON)
   221  	if err != nil {
   222  		log.Errorf("GetEncodedSegStats: failed to marshal allSegStatJson: %v", err)
   223  		return nil, err
   224  	}
   225  	return jsonBytes, nil
   226  }
   227  
   228  func (sr *SearchResults) AddError(err error) {
   229  	sr.updateLock.Lock()
   230  	sr.AllErrors = append(sr.AllErrors, err)
   231  	sr.updateLock.Unlock()
   232  }
   233  
   234  func (sr *SearchResults) UpdateSegmentStats(sstMap map[string]*structs.SegStats, measureOps []*structs.MeasureAggregator,
   235  	runningEvalStats map[string]interface{}) error {
   236  	sr.updateLock.Lock()
   237  	defer sr.updateLock.Unlock()
   238  	for idx, measureAgg := range measureOps {
   239  		if len(sstMap) == 0 {
   240  			continue
   241  		}
   242  
   243  		aggOp := measureAgg.MeasureFunc
   244  		aggCol := measureAgg.MeasureCol
   245  
   246  		if aggOp == utils.Count && aggCol == "*" {
   247  			// Choose the first column.
   248  			for key := range sstMap {
   249  				aggCol = key
   250  				break
   251  			}
   252  		}
   253  		currSst, ok := sstMap[aggCol]
   254  		if !ok && measureAgg.ValueColRequest == nil {
   255  			log.Debugf("applyAggOpOnSegments sstMap was nil for aggCol %v", aggCol)
   256  			continue
   257  		}
   258  		var err error
   259  		var sstResult *utils.NumTypeEnclosure
   260  		// For eval statements in aggregate functions, there should be only one field for min and max
   261  		switch aggOp {
   262  		case utils.Min:
   263  			if measureAgg.ValueColRequest != nil {
   264  				err := aggregations.ComputeAggEvalForMinOrMax(measureAgg, sstMap, sr.segStatsResults.measureResults, true)
   265  				if err != nil {
   266  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   267  				}
   268  				continue
   269  			}
   270  			sstResult, err = segread.GetSegMin(sr.runningSegStat[idx], currSst)
   271  		case utils.Max:
   272  			if measureAgg.ValueColRequest != nil {
   273  				err := aggregations.ComputeAggEvalForMinOrMax(measureAgg, sstMap, sr.segStatsResults.measureResults, false)
   274  				if err != nil {
   275  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   276  				}
   277  				continue
   278  			}
   279  			sstResult, err = segread.GetSegMax(sr.runningSegStat[idx], currSst)
   280  		case utils.Range:
   281  			if measureAgg.ValueColRequest != nil {
   282  				err := aggregations.ComputeAggEvalForRange(measureAgg, sstMap, sr.segStatsResults.measureResults, runningEvalStats)
   283  				if err != nil {
   284  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   285  				}
   286  				continue
   287  			}
   288  			sstResult, err = segread.GetSegRange(sr.runningSegStat[idx], currSst)
   289  		case utils.Cardinality:
   290  			if measureAgg.ValueColRequest != nil {
   291  				err := aggregations.ComputeAggEvalForCardinality(measureAgg, sstMap, sr.segStatsResults.measureResults, runningEvalStats)
   292  				if err != nil {
   293  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   294  				}
   295  				continue
   296  			}
   297  			sstResult, err = segread.GetSegCardinality(sr.runningSegStat[idx], currSst)
   298  		case utils.Count:
   299  			if measureAgg.ValueColRequest != nil {
   300  				err := aggregations.ComputeAggEvalForCount(measureAgg, sstMap, sr.segStatsResults.measureResults)
   301  				if err != nil {
   302  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   303  				}
   304  				continue
   305  			}
   306  			sstResult, err = segread.GetSegCount(sr.runningSegStat[idx], currSst)
   307  		case utils.Sum:
   308  			if measureAgg.ValueColRequest != nil {
   309  				err := aggregations.ComputeAggEvalForSum(measureAgg, sstMap, sr.segStatsResults.measureResults)
   310  				if err != nil {
   311  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   312  				}
   313  				continue
   314  			}
   315  			sstResult, err = segread.GetSegSum(sr.runningSegStat[idx], currSst)
   316  		case utils.Avg:
   317  			if measureAgg.ValueColRequest != nil {
   318  				err := aggregations.ComputeAggEvalForAvg(measureAgg, sstMap, sr.segStatsResults.measureResults, runningEvalStats)
   319  				if err != nil {
   320  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   321  				}
   322  				continue
   323  			}
   324  			sstResult, err = segread.GetSegAvg(sr.runningSegStat[idx], currSst)
   325  		case utils.Values:
   326  			strSet := make(map[string]struct{}, 0)
   327  			valuesStrSetVal, exists := runningEvalStats[measureAgg.String()]
   328  			if !exists {
   329  				runningEvalStats[measureAgg.String()] = make(map[string]struct{}, 0)
   330  			} else {
   331  				strSet, ok = valuesStrSetVal.(map[string]struct{})
   332  				if !ok {
   333  					return fmt.Errorf("UpdateSegmentStats: can not convert strSet for aggCol: %v", measureAgg.String())
   334  				}
   335  			}
   336  
   337  			if measureAgg.ValueColRequest != nil {
   338  				err := aggregations.ComputeAggEvalForValues(measureAgg, sstMap, sr.segStatsResults.measureResults, strSet)
   339  				if err != nil {
   340  					return fmt.Errorf("UpdateSegmentStats: %v", err)
   341  				}
   342  				continue
   343  			}
   344  
   345  			// Merge two SegStat
   346  			if currSst != nil && currSst.StringStats != nil && currSst.StringStats.StrSet != nil {
   347  				for str := range currSst.StringStats.StrSet {
   348  					strSet[str] = struct{}{}
   349  				}
   350  			}
   351  			if sr.runningSegStat[idx] != nil {
   352  
   353  				for str := range sr.runningSegStat[idx].StringStats.StrSet {
   354  					strSet[str] = struct{}{}
   355  				}
   356  
   357  				sr.runningSegStat[idx].StringStats.StrSet = strSet
   358  			}
   359  
   360  			uniqueStrings := make([]string, 0)
   361  			for str := range strSet {
   362  				uniqueStrings = append(uniqueStrings, str)
   363  			}
   364  			sort.Strings(uniqueStrings)
   365  			strVal := strings.Join(uniqueStrings, "&nbsp")
   366  			sr.segStatsResults.measureResults[measureAgg.String()] = utils.CValueEnclosure{
   367  				Dtype: utils.SS_DT_STRING,
   368  				CVal:  strVal,
   369  			}
   370  			continue
   371  		}
   372  		if err != nil {
   373  			log.Errorf("UpdateSegmentStats: error getting segment level stats %+v", err)
   374  			return err
   375  		}
   376  
   377  		// if this is the first segment, then set the running segment stat to the current segment stat
   378  		// else, segread.GetN will update the running segment stat
   379  		if sr.runningSegStat[idx] == nil {
   380  			sr.runningSegStat[idx] = currSst
   381  		}
   382  
   383  		enclosure, err := sstResult.ToCValueEnclosure()
   384  		if err != nil {
   385  			log.Errorf("UpdateSegmentStats: cannot convert sstResult: %v", err)
   386  			return err
   387  		}
   388  		sr.segStatsResults.measureResults[measureAgg.String()] = *enclosure
   389  	}
   390  	return nil
   391  }
   392  
   393  func (sr *SearchResults) GetQueryCount() *structs.QueryCount {
   394  	sr.updateLock.Lock()
   395  	defer sr.updateLock.Unlock()
   396  	var shouldEarlyExit bool
   397  	if sr.sAggs != nil {
   398  		shouldEarlyExit = sr.sAggs.EarlyExit
   399  	} else {
   400  		shouldEarlyExit = true
   401  	}
   402  	qc := &structs.QueryCount{TotalCount: sr.resultCount, EarlyExit: shouldEarlyExit}
   403  	if sr.EarlyExit {
   404  		qc.Op = utils.GreaterThanOrEqualTo
   405  	} else {
   406  		qc.Op = utils.Equals
   407  	}
   408  	return qc
   409  }
   410  
   411  func (sr *SearchResults) GetTotalCount() uint64 {
   412  	sr.updateLock.Lock()
   413  	defer sr.updateLock.Unlock()
   414  	return sr.resultCount
   415  }
   416  
   417  // Adds remote rrc results to the search results
   418  func (sr *SearchResults) MergeRemoteRRCResults(rrcs []*utils.RecordResultContainer, grpByBuckets *blockresults.GroupByBucketsJSON,
   419  	timeBuckets *blockresults.TimeBucketsJSON, allCols map[string]struct{}, rawLogs []map[string]interface{},
   420  	remoteCount uint64, earlyExit bool) error {
   421  	sr.updateLock.Lock()
   422  	defer sr.updateLock.Unlock()
   423  	for cName := range allCols {
   424  		sr.remoteInfo.remoteColumns[cName] = struct{}{}
   425  	}
   426  
   427  	for idx, rrc := range rrcs {
   428  		addedRRC, removedID := sr.BlockResults.Add(rrc)
   429  		if addedRRC {
   430  			sr.addRawLog(rrc.SegKeyInfo.RecordId, rawLogs[idx])
   431  		}
   432  		sr.removeLog(removedID)
   433  	}
   434  	if earlyExit {
   435  		sr.EarlyExit = true
   436  	}
   437  	err := sr.BlockResults.MergeRemoteBuckets(grpByBuckets, timeBuckets)
   438  	if err != nil {
   439  		log.Errorf("MergeRemoteRRCResults: Error merging remote buckets: %v", err)
   440  		return err
   441  	}
   442  	sr.resultCount += remoteCount
   443  	return nil
   444  }
   445  
   446  func (sr *SearchResults) AddSegmentStats(allJSON *structs.AllSegStatsJSON) error {
   447  	sstMap := make(map[string]*structs.SegStats, len(allJSON.AllSegStats))
   448  	for k, v := range allJSON.AllSegStats {
   449  		rawStats, err := v.ToStats()
   450  		if err != nil {
   451  			return err
   452  		}
   453  		sstMap[k] = rawStats
   454  	}
   455  	return sr.UpdateSegmentStats(sstMap, sr.sAggs.MeasureOperations, nil)
   456  }
   457  
   458  // Get remote raw logs and columns based on the remoteID and all RRCs
   459  func (sr *SearchResults) GetRemoteInfo(remoteID string, inrrcs []*utils.RecordResultContainer) ([]map[string]interface{}, []string, error) {
   460  	sr.updateLock.Lock()
   461  	defer sr.updateLock.Unlock()
   462  	if sr.remoteInfo == nil {
   463  		return nil, nil, fmt.Errorf("log does not have remote info")
   464  	}
   465  	finalLogs := make([]map[string]interface{}, 0, len(inrrcs))
   466  	rawLogs := sr.remoteInfo.remoteLogs
   467  	remoteCols := sr.remoteInfo.remoteColumns
   468  	count := 0
   469  	for i := 0; i < len(inrrcs); i++ {
   470  		if inrrcs[i].SegKeyInfo.IsRemote && strings.HasPrefix(inrrcs[i].SegKeyInfo.RecordId, remoteID) {
   471  			finalLogs = append(finalLogs, rawLogs[inrrcs[i].SegKeyInfo.RecordId])
   472  			count++
   473  		}
   474  	}
   475  	finalLogs = finalLogs[:count]
   476  
   477  	allCols := make([]string, 0, len(remoteCols))
   478  	idx := 0
   479  	for col := range remoteCols {
   480  		allCols = append(allCols, col)
   481  		idx++
   482  	}
   483  	allCols = allCols[:idx]
   484  	sort.Strings(allCols)
   485  	return finalLogs, allCols, nil
   486  }
   487  
   488  func (sr *SearchResults) GetSegmentStatsResults(skEnc uint16) ([]*structs.BucketHolder, []string, []string, int) {
   489  	sr.updateLock.Lock()
   490  	defer sr.updateLock.Unlock()
   491  
   492  	if sr.segStatsResults == nil {
   493  		return nil, nil, nil, 0
   494  	}
   495  	delete(sr.allSSTS, skEnc)
   496  	bucketHolder := &structs.BucketHolder{}
   497  	bucketHolder.MeasureVal = make(map[string]interface{})
   498  	bucketHolder.GroupByValues = []string{EMPTY_GROUPBY_KEY}
   499  	for mfName, aggVal := range sr.segStatsResults.measureResults {
   500  		switch aggVal.Dtype {
   501  		case utils.SS_DT_FLOAT:
   502  			bucketHolder.MeasureVal[mfName] = humanize.CommafWithDigits(aggVal.CVal.(float64), 3)
   503  		case utils.SS_DT_SIGNED_NUM:
   504  			bucketHolder.MeasureVal[mfName] = humanize.Comma(aggVal.CVal.(int64))
   505  		case utils.SS_DT_STRING:
   506  			bucketHolder.MeasureVal[mfName] = aggVal.CVal
   507  		}
   508  	}
   509  	aggMeasureResult := []*structs.BucketHolder{bucketHolder}
   510  	return aggMeasureResult, sr.segStatsResults.measureFunctions, sr.segStatsResults.groupByCols, len(sr.segStatsResults.measureResults)
   511  }
   512  
   513  func (sr *SearchResults) GetSegmentStatsMeasureResults() map[string]utils.CValueEnclosure {
   514  	sr.updateLock.Lock()
   515  	defer sr.updateLock.Unlock()
   516  	return sr.segStatsResults.measureResults
   517  }
   518  
   519  func (sr *SearchResults) GetSegmentRunningStats() []*structs.SegStats {
   520  	sr.updateLock.Lock()
   521  	defer sr.updateLock.Unlock()
   522  	return sr.runningSegStat
   523  }
   524  
   525  func (sr *SearchResults) GetGroupyByBuckets(limit int) ([]*structs.BucketHolder, []string, []string, int) {
   526  	sr.updateLock.Lock()
   527  	defer sr.updateLock.Unlock()
   528  
   529  	if sr.convertedBuckets != nil && !sr.statsAreFinal {
   530  		sr.loadBucketsInternal()
   531  	}
   532  	bucketHolderArr := make([]*structs.BucketHolder, 0)
   533  	added := int(0)
   534  	internalMFuncs := make(map[string]bool)
   535  	for _, agg := range sr.convertedBuckets {
   536  		for _, aggVal := range agg.Results {
   537  			measureVal := make(map[string]interface{})
   538  			groupByValues := make([]string, 0)
   539  			for mName, mVal := range aggVal.StatRes {
   540  				rawVal, err := mVal.GetValue()
   541  				if err != nil {
   542  					log.Errorf("GetGroupyByBuckets: failed to get raw value for measurement %+v", err)
   543  					continue
   544  				}
   545  				internalMFuncs[mName] = true
   546  				measureVal[mName] = rawVal
   547  
   548  			}
   549  			if added >= limit {
   550  				break
   551  			}
   552  			switch bKey := aggVal.BucketKey.(type) {
   553  			case float64, uint64, int64:
   554  				bKeyConv := fmt.Sprintf("%+v", bKey)
   555  				groupByValues = append(groupByValues, bKeyConv)
   556  				added++
   557  			case []string:
   558  
   559  				for _, bk := range aggVal.BucketKey.([]string) {
   560  					groupByValues = append(groupByValues, bk)
   561  					added++
   562  				}
   563  			case string:
   564  				groupByValues = append(groupByValues, bKey)
   565  				added++
   566  			default:
   567  				log.Errorf("Received an unknown type for bucket key! %+v", bKey)
   568  			}
   569  			bucketHolder := &structs.BucketHolder{
   570  				GroupByValues: groupByValues,
   571  				MeasureVal:    measureVal,
   572  			}
   573  			bucketHolderArr = append(bucketHolderArr, bucketHolder)
   574  		}
   575  	}
   576  
   577  	retMFuns := make([]string, len(internalMFuncs))
   578  	idx := 0
   579  	for mName := range internalMFuncs {
   580  		retMFuns[idx] = mName
   581  		idx++
   582  	}
   583  
   584  	if sr.sAggs == nil || sr.sAggs.GroupByRequest == nil {
   585  		return bucketHolderArr, retMFuns, nil, added
   586  	} else {
   587  		return bucketHolderArr, retMFuns, sr.sAggs.GroupByRequest.GroupByColumns, added
   588  	}
   589  }
   590  
   591  // If agg.GroupByRequest.GroupByColumns == StatisticExpr.GroupByCols, which means there is only one groupby block in query
   592  func (sr *SearchResults) IsOnlyStatisticGroupBy() bool {
   593  	for agg := sr.sAggs; agg != nil; agg = agg.Next {
   594  		if agg.GroupByRequest != nil && agg.GroupByRequest.GroupByColumns != nil {
   595  			for _, groupByCol1 := range agg.GroupByRequest.GroupByColumns {
   596  				for _, groupByCol2 := range sr.GetStatisticGroupByCols() {
   597  					if groupByCol1 != groupByCol2 {
   598  						return false
   599  					}
   600  				}
   601  			}
   602  			return true
   603  		}
   604  	}
   605  	return false
   606  }
   607  
   608  func (sr *SearchResults) GetStatisticGroupByCols() []string {
   609  	groupByCols := make([]string, 0)
   610  	for agg := sr.sAggs; agg != nil; agg = agg.Next {
   611  		if agg.OutputTransforms != nil && agg.OutputTransforms.LetColumns != nil && agg.OutputTransforms.LetColumns.StatisticColRequest != nil {
   612  			groupByCols = append(agg.OutputTransforms.LetColumns.StatisticColRequest.FieldList, agg.OutputTransforms.LetColumns.StatisticColRequest.ByClause...)
   613  			return groupByCols
   614  		}
   615  	}
   616  	return groupByCols
   617  }
   618  
   619  // For Rename or top/rare block, we may need to delete some groupby columns while processing them
   620  func (sr *SearchResults) RemoveUnusedGroupByCols(aggGroupByCols []string) []string {
   621  	for agg := sr.sAggs; agg != nil; agg = agg.Next {
   622  		// Rename block
   623  		aggGroupByCols = sr.GetRenameGroupByCols(aggGroupByCols, agg)
   624  		// Statistic block: to be finished
   625  	}
   626  	return aggGroupByCols
   627  }
   628  
   629  // Rename field A to field B. If A and B are groupby columns, field B should be removed from groupby columns, and rename A to B
   630  func (sr *SearchResults) GetRenameGroupByCols(aggGroupByCols []string, agg *structs.QueryAggregators) []string {
   631  	if agg.OutputTransforms != nil && agg.OutputTransforms.LetColumns != nil && agg.OutputTransforms.LetColumns.RenameColRequest != nil {
   632  
   633  		// Except for regex, other RenameExprModes will only rename one column
   634  		renameIndex := -1
   635  		indexToRemove := make([]int, 0)
   636  
   637  		for index, groupByCol := range aggGroupByCols {
   638  			switch agg.OutputTransforms.LetColumns.RenameColRequest.RenameExprMode {
   639  			case structs.REMPhrase:
   640  				fallthrough
   641  			case structs.REMOverride:
   642  
   643  				if groupByCol == agg.OutputTransforms.LetColumns.RenameColRequest.OriginalPattern {
   644  					renameIndex = index
   645  				}
   646  				if groupByCol == agg.OutputTransforms.LetColumns.RenameColRequest.NewPattern {
   647  					indexToRemove = append(indexToRemove, index)
   648  				}
   649  
   650  			case structs.REMRegex:
   651  				newColName, err := agg.OutputTransforms.LetColumns.RenameColRequest.ProcessRenameRegexExpression(groupByCol)
   652  				if err != nil {
   653  					return []string{}
   654  				}
   655  				if len(newColName) == 0 {
   656  					continue
   657  				}
   658  				for i, colName := range aggGroupByCols {
   659  					if colName == newColName {
   660  						indexToRemove = append(indexToRemove, i)
   661  						break
   662  					}
   663  				}
   664  				aggGroupByCols[index] = newColName
   665  			}
   666  		}
   667  		if renameIndex != -1 {
   668  			aggGroupByCols[renameIndex] = agg.OutputTransforms.LetColumns.RenameColRequest.NewPattern
   669  		}
   670  		aggGroupByCols = agg.OutputTransforms.LetColumns.RenameColRequest.RemoveColsByIndex(aggGroupByCols, indexToRemove)
   671  	}
   672  	return aggGroupByCols
   673  }
   674  
   675  // Subsequent calls may not return the same result as the previous may clean up the underlying heap used. Use GetResultsCopy to prevent this
   676  func (sr *SearchResults) GetResults() []*utils.RecordResultContainer {
   677  	sr.updateLock.Lock()
   678  	defer sr.updateLock.Unlock()
   679  	return sr.BlockResults.GetResults()
   680  }
   681  
   682  func (sr *SearchResults) GetResultsCopy() []*utils.RecordResultContainer {
   683  	sr.updateLock.Lock()
   684  	defer sr.updateLock.Unlock()
   685  	return sr.BlockResults.GetResultsCopy()
   686  }
   687  
   688  func (sr *SearchResults) GetBucketResults() map[string]*structs.AggregationResult {
   689  	sr.updateLock.Lock()
   690  	defer sr.updateLock.Unlock()
   691  
   692  	if !sr.statsAreFinal {
   693  		sr.loadBucketsInternal()
   694  	}
   695  
   696  	return sr.convertedBuckets
   697  }
   698  
   699  func (sr *SearchResults) SetFinalStatsFromNodeResult(nodeResult *structs.NodeResult) error {
   700  	sr.updateLock.Lock()
   701  	defer sr.updateLock.Unlock()
   702  
   703  	if sr.statsAreFinal {
   704  		return fmt.Errorf("SetFinalStatsFromNodeResult: stats are already final")
   705  	}
   706  
   707  	if len(nodeResult.GroupByCols) > 0 {
   708  		sr.convertedBuckets = nodeResult.Histogram
   709  	} else {
   710  		if length := len(nodeResult.MeasureResults); length != 1 {
   711  			err := fmt.Errorf("Unexpected MeasureResults length")
   712  			log.Errorf("%v", err)
   713  			return err
   714  		}
   715  
   716  		sr.segStatsResults.measureFunctions = nodeResult.MeasureFunctions
   717  		sr.segStatsResults.measureResults = make(map[string]utils.CValueEnclosure, len(nodeResult.MeasureFunctions))
   718  		sr.segStatsResults.groupByCols = nil
   719  
   720  		for _, measureFunc := range sr.segStatsResults.measureFunctions {
   721  			value, ok := nodeResult.MeasureResults[0].MeasureVal[measureFunc]
   722  			if !ok {
   723  				err := fmt.Errorf("SetFinalStatsFromNodeResult: %v not found in MeasureVal", measureFunc)
   724  				log.Errorf("%v", err)
   725  				return err
   726  			}
   727  
   728  			// Create a CValueEnclosure for `value`.
   729  			var valueAsEnclosure utils.CValueEnclosure
   730  			valueStr, ok := value.(string)
   731  			if !ok {
   732  				err := fmt.Errorf("SetFinalStatsFromNodeResult: unexpected type: %T", value)
   733  				log.Errorf("%v", err)
   734  				return err
   735  			}
   736  
   737  			// Remove any commas.
   738  			valueStr = strings.ReplaceAll(valueStr, ",", "")
   739  
   740  			if valueFloat, err := strconv.ParseFloat(valueStr, 64); err == nil {
   741  				valueAsEnclosure.Dtype = utils.SS_DT_FLOAT
   742  				valueAsEnclosure.CVal = valueFloat
   743  			} else if valueInt, err := strconv.ParseInt(valueStr, 10, 64); err == nil {
   744  				valueAsEnclosure.Dtype = utils.SS_DT_SIGNED_NUM
   745  				valueAsEnclosure.CVal = valueInt
   746  			} else {
   747  				valueAsEnclosure.Dtype = utils.SS_DT_STRING
   748  				valueAsEnclosure.CVal = valueStr
   749  			}
   750  
   751  			sr.segStatsResults.measureResults[measureFunc] = valueAsEnclosure
   752  		}
   753  	}
   754  
   755  	sr.statsAreFinal = true
   756  
   757  	return nil
   758  }
   759  
   760  func (sr *SearchResults) GetNumBuckets() int {
   761  	sr.updateLock.Lock()
   762  	defer sr.updateLock.Unlock()
   763  	retVal := 0
   764  	if sr.BlockResults == nil {
   765  		return 0
   766  	}
   767  	if sr.BlockResults.TimeAggregation != nil {
   768  		retVal += len(sr.BlockResults.TimeAggregation.AllRunningBuckets)
   769  	}
   770  	if sr.BlockResults.GroupByAggregation != nil {
   771  		retVal += len(sr.BlockResults.GroupByAggregation.AllRunningBuckets)
   772  	}
   773  	return retVal
   774  }
   775  
   776  func (sr *SearchResults) loadBucketsInternal() {
   777  	if sr.statsAreFinal {
   778  		log.Errorf("loadBucketsInternal: cannot update convertedBuckets because stats are final")
   779  		return
   780  	}
   781  
   782  	retVal := make(map[string]*structs.AggregationResult)
   783  	if sr.BlockResults.TimeAggregation != nil {
   784  		retVal[sr.sAggs.TimeHistogram.AggName] = sr.BlockResults.GetTimeBuckets()
   785  	}
   786  	if sr.BlockResults.GroupByAggregation != nil {
   787  		retVal[sr.sAggs.GroupByRequest.AggName] = sr.BlockResults.GetGroupByBuckets()
   788  	}
   789  	sr.convertedBuckets = retVal
   790  }
   791  
   792  func (sr *SearchResults) GetAllErrors() []error {
   793  	sr.updateLock.Lock()
   794  	defer sr.updateLock.Unlock()
   795  	return sr.AllErrors
   796  }
   797  
   798  // returns if the segkey needs to be searched or if we have hit an early exit
   799  func (sr *SearchResults) ShouldSearchSegKey(tRange *dtu.TimeRange,
   800  	snt structs.SearchNodeType, otherAggsPresent bool, timeAggs bool) EarlyExitType {
   801  
   802  	// do we have enough RRCs?
   803  	if sr.ShouldContinueRRCSearch() {
   804  		return EetContSearch
   805  	}
   806  	if sr.queryType == structs.GroupByCmd {
   807  		if sr.GetNumBuckets() < sr.sAggs.GroupByRequest.BucketCount {
   808  			return EetContSearch
   809  		} else {
   810  			return EetEarlyExit
   811  		}
   812  	} else if sr.queryType != structs.RRCCmd {
   813  		return EetContSearch
   814  	}
   815  
   816  	// do the RRCs we have pass the sort check?
   817  	if sr.sAggs != nil && sr.sAggs.Sort != nil {
   818  		var willValBeAdded bool
   819  		if sr.sAggs.Sort.Ascending {
   820  			willValBeAdded = sr.BlockResults.WillValueBeAdded(float64(tRange.StartEpochMs))
   821  		} else {
   822  			willValBeAdded = sr.BlockResults.WillValueBeAdded(float64(tRange.EndEpochMs))
   823  		}
   824  		if willValBeAdded {
   825  			return EetContSearch
   826  		}
   827  	}
   828  
   829  	// do we have all sorted RRCs and now need to only run date histogram?
   830  	if snt == structs.MatchAllQuery && timeAggs && !otherAggsPresent {
   831  		return EetMatchAllAggs
   832  	}
   833  
   834  	// do we have all sorted RRCs but still need raw search to complete the rest of the buckets?
   835  	if snt != structs.MatchAllQuery && (timeAggs || otherAggsPresent) {
   836  		return EetContSearch
   837  	}
   838  
   839  	// do we have all sorted RRCs with no aggs?
   840  	if sr.sAggs == nil {
   841  		return EetEarlyExit
   842  	}
   843  
   844  	// do we have all sorted RRCs but should not early exit?
   845  	if !sr.sAggs.EarlyExit {
   846  		return EetContSearch
   847  	}
   848  
   849  	// do we have all sorted RRCs and now need to run aggregations?
   850  	if sr.sAggs.TimeHistogram != nil {
   851  		return EetContSearch
   852  	}
   853  
   854  	// do we have all sorted RRCs and now need to run aggregations?
   855  	if sr.sAggs.GroupByRequest != nil {
   856  		return EetContSearch
   857  	}
   858  
   859  	return EetEarlyExit
   860  }
   861  
   862  // returns true in following cases:
   863  // 1. search is not rrc
   864  // 1. if any value in a block will be added based on highTs and lowTs
   865  // 2. if time buckets exist
   866  func (sr *SearchResults) ShouldSearchRange(lowTs, highTs uint64) bool {
   867  	if sr.queryType != structs.RRCCmd {
   868  		return true
   869  	}
   870  
   871  	if sr.ShouldContinueRRCSearch() {
   872  		return true
   873  	}
   874  	if sr.sAggs == nil {
   875  		return false
   876  	}
   877  	if !sr.sAggs.EarlyExit {
   878  		return true
   879  	}
   880  	if sr.sAggs.TimeHistogram != nil {
   881  		return true
   882  	}
   883  
   884  	if sr.sAggs.Sort != nil {
   885  		if sr.sAggs.Sort.Ascending {
   886  			return sr.BlockResults.WillValueBeAdded(float64(lowTs))
   887  		} else {
   888  			return sr.BlockResults.WillValueBeAdded(float64(highTs))
   889  		}
   890  	}
   891  	return false
   892  }
   893  
   894  // sets early exit to value
   895  func (sr *SearchResults) SetEarlyExit(exited bool) {
   896  	sr.EarlyExit = exited
   897  }
   898  
   899  func (sr *SearchResults) GetAddSegEnc(sk string) uint16 {
   900  
   901  	sr.updateLock.Lock()
   902  	defer sr.updateLock.Unlock()
   903  
   904  	retval, ok := sr.SegKeyToEnc[sk]
   905  	if ok {
   906  		return retval
   907  	}
   908  
   909  	retval = sr.MaxSegKeyEnc
   910  	sr.SegEncToKey[sr.MaxSegKeyEnc] = sk
   911  	sr.SegKeyToEnc[sk] = sr.MaxSegKeyEnc
   912  	sr.MaxSegKeyEnc++
   913  	return retval
   914  }
   915  
   916  // helper struct to coordinate parallel segstats results
   917  type StatsResults struct {
   918  	rwLock  *sync.RWMutex
   919  	ssStats map[string]*structs.SegStats // maps column name to segstats
   920  }
   921  
   922  func InitStatsResults() *StatsResults {
   923  	return &StatsResults{
   924  		rwLock:  &sync.RWMutex{},
   925  		ssStats: make(map[string]*structs.SegStats),
   926  	}
   927  }
   928  
   929  func (sr *StatsResults) MergeSegStats(m1 map[string]*structs.SegStats) {
   930  	sr.rwLock.Lock()
   931  	sr.ssStats = stats.MergeSegStats(sr.ssStats, m1)
   932  	sr.rwLock.Unlock()
   933  }
   934  
   935  func (sr *StatsResults) GetSegStats() map[string]*structs.SegStats {
   936  	sr.rwLock.Lock()
   937  	retVal := sr.ssStats
   938  	sr.rwLock.Unlock()
   939  	return retVal
   940  }