github.com/siglens/siglens@v0.0.0-20240328180423-f7ce9ae441ed/pkg/segment/results/blockresults/blockresult.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 blockresults
    18  
    19  import (
    20  	"bytes"
    21  	"encoding/base64"
    22  	"fmt"
    23  	"sort"
    24  	"strings"
    25  
    26  	"github.com/axiomhq/hyperloglog"
    27  	"github.com/siglens/siglens/pkg/segment/aggregations"
    28  	"github.com/siglens/siglens/pkg/segment/structs"
    29  	"github.com/siglens/siglens/pkg/segment/utils"
    30  	toputils "github.com/siglens/siglens/pkg/utils"
    31  	log "github.com/sirupsen/logrus"
    32  )
    33  
    34  type GroupByBuckets struct {
    35  	AllRunningBuckets   []*RunningBucketResults
    36  	StringBucketIdx     map[string]int
    37  	internalMeasureFns  []*structs.MeasureAggregator // all converted measure requests in order they exist in running stats
    38  	allMeasureCols      map[string][]int             // maps col name to all indices that it exist in internalMeasureFns
    39  	reverseMeasureIndex []int                        // reverse index, so idx of original measure will store the index in internalMeasureFns. -1 is reserved for count
    40  	maxBuckets          int                          // maximum number of buckets to create
    41  	GroupByColValCnt    map[string]int               // calculate freq for group by col val
    42  }
    43  
    44  type TimeBuckets struct {
    45  	AllRunningBuckets []*RunningBucketResults
    46  	UnsignedBucketIdx map[uint64]int
    47  }
    48  
    49  type BlockResults struct {
    50  	SortedResults      *SortResults
    51  	UnsortedResults    []*utils.RecordResultContainer
    52  	TimeAggregation    *TimeBuckets
    53  	GroupByAggregation *GroupByBuckets
    54  	aggs               *structs.QueryAggregators
    55  
    56  	MatchedCount uint64
    57  
    58  	nextUnsortedIdx uint64 // next index to put result in
    59  	sortResults     bool
    60  	sizeLimit       uint64
    61  }
    62  
    63  // json exportable and mergeable results for query
    64  type GroupByBucketsJSON struct {
    65  	AllGroupbyBuckets map[string]*RunningBucketResultsJSON `json:"allGroupbyBuckets"`
    66  }
    67  
    68  type TimeBucketsJSON struct {
    69  	AllTimeBuckets map[uint64]*RunningBucketResultsJSON `json:"allTimeBuckets"`
    70  }
    71  
    72  type RunningBucketResultsJSON struct {
    73  	RunningStats []interface{}                `json:"runningStats"`
    74  	CurrStats    []*structs.MeasureAggregator `json:"currStats"`
    75  	Count        uint64                       `json:"count"`
    76  }
    77  
    78  func InitBlockResults(count uint64, aggs *structs.QueryAggregators, qid uint64) (*BlockResults, error) {
    79  
    80  	blockRes := &BlockResults{aggs: aggs}
    81  	if aggs != nil && aggs.TimeHistogram != nil {
    82  		blockRes.TimeAggregation = &TimeBuckets{
    83  			AllRunningBuckets: make([]*RunningBucketResults, 0),
    84  			UnsignedBucketIdx: make(map[uint64]int),
    85  		}
    86  	}
    87  
    88  	if aggs != nil && aggs.GroupByRequest != nil {
    89  		if len(aggs.GroupByRequest.GroupByColumns) > 0 {
    90  			usedByTimechart := aggs.UsedByTimechart()
    91  			mCols, mFuns, revIndex := convertRequestToInternalStats(aggs.GroupByRequest, usedByTimechart)
    92  			blockRes.GroupByAggregation = &GroupByBuckets{
    93  				AllRunningBuckets:   make([]*RunningBucketResults, 0),
    94  				StringBucketIdx:     make(map[string]int),
    95  				allMeasureCols:      mCols,
    96  				internalMeasureFns:  mFuns,
    97  				reverseMeasureIndex: revIndex,
    98  				maxBuckets:          aggs.GroupByRequest.BucketCount,
    99  				GroupByColValCnt:    make(map[string]int),
   100  			}
   101  		}
   102  	}
   103  
   104  	if aggs != nil && aggs.Sort != nil {
   105  		sortedRes, err := InitializeSort(count, aggs.Sort)
   106  		if err != nil {
   107  			log.Errorf("qid=%d, Initialize block results failed: %v", qid, err)
   108  			return nil, err
   109  		}
   110  		blockRes.sortResults = true
   111  		blockRes.SortedResults = sortedRes
   112  	} else {
   113  		blockRes.sortResults = false
   114  		blockRes.UnsortedResults = make([]*utils.RecordResultContainer, count)
   115  		blockRes.nextUnsortedIdx = 0
   116  	}
   117  	blockRes.sizeLimit = count
   118  	blockRes.MatchedCount = 0
   119  	return blockRes, nil
   120  }
   121  
   122  // This function will map[colName] -> []idx of measure functions, converted measure ops, and the reverse index of original to converted op
   123  // Converted Measure Ops: for example, to calculate average the block will need to track sum
   124  // Count is always tracked for each bucket
   125  func convertRequestToInternalStats(req *structs.GroupByRequest, usedByTimechart bool) (map[string][]int, []*structs.MeasureAggregator, []int) {
   126  	colToIdx := make(map[string][]int) // maps a column name to all indices in allConvertedMeasureOps it relates to
   127  	allConvertedMeasureOps := make([]*structs.MeasureAggregator, 0)
   128  	allReverseIndex := make([]int, 0)
   129  	idx := 0
   130  	for _, m := range req.MeasureOperations {
   131  		measureColStr := m.MeasureCol
   132  		var mFunc utils.AggregateFunctions
   133  		var overrodeMeasureAgg *structs.MeasureAggregator
   134  		switch m.MeasureFunc {
   135  		case utils.Sum:
   136  			fallthrough
   137  		case utils.Max:
   138  			fallthrough
   139  		case utils.Min:
   140  			if m.ValueColRequest != nil {
   141  				fields := m.ValueColRequest.GetFields()
   142  				if len(fields) != 1 {
   143  					log.Errorf("convertRequestToInternalStats: Incorrect number of fields for aggCol: %v", m.String())
   144  					continue
   145  				}
   146  				measureColStr = fields[0]
   147  			}
   148  			mFunc = m.MeasureFunc
   149  		case utils.Range:
   150  			curId, err := aggregations.AddMeasureAggInRunningStatsForRange(m, &allConvertedMeasureOps, &allReverseIndex, colToIdx, idx)
   151  			if err != nil {
   152  				log.Errorf("convertRequestToInternalStats: %v", err)
   153  			}
   154  			idx = curId
   155  			continue
   156  		case utils.Count:
   157  			if m.ValueColRequest != nil {
   158  				curId, err := aggregations.AddMeasureAggInRunningStatsForCount(m, &allConvertedMeasureOps, &allReverseIndex, colToIdx, idx)
   159  				if err != nil {
   160  					log.Errorf("convertRequestToInternalStats: %v", err)
   161  				}
   162  				idx = curId
   163  			} else {
   164  				if usedByTimechart {
   165  					aggregations.AddAggCountToTimechartRunningStats(m, &allConvertedMeasureOps, &allReverseIndex, colToIdx, idx)
   166  					idx++
   167  					continue
   168  				}
   169  				allReverseIndex = append(allReverseIndex, -1)
   170  			}
   171  			continue
   172  		case utils.Avg:
   173  			if m.ValueColRequest != nil {
   174  				curId, err := aggregations.AddMeasureAggInRunningStatsForAvg(m, &allConvertedMeasureOps, &allReverseIndex, colToIdx, idx)
   175  				if err != nil {
   176  					log.Errorf("convertRequestToInternalStats: %v", err)
   177  				}
   178  				idx = curId
   179  				continue
   180  			} else {
   181  				if usedByTimechart {
   182  					aggregations.AddAggAvgToTimechartRunningStats(m, &allConvertedMeasureOps, &allReverseIndex, colToIdx, idx)
   183  					idx += 2
   184  					continue
   185  				}
   186  				mFunc = utils.Sum
   187  				overrodeMeasureAgg = m
   188  			}
   189  		case utils.Cardinality:
   190  			fallthrough
   191  		case utils.Values:
   192  			if m.ValueColRequest != nil {
   193  				curId, err := aggregations.AddMeasureAggInRunningStatsForValuesOrCardinality(m, &allConvertedMeasureOps, &allReverseIndex, colToIdx, idx)
   194  				if err != nil {
   195  					log.Errorf("convertRequestToInternalStats: %v", err)
   196  				}
   197  				idx = curId
   198  				continue
   199  			} else {
   200  				mFunc = m.MeasureFunc
   201  			}
   202  		default:
   203  			mFunc = m.MeasureFunc
   204  		}
   205  
   206  		if _, ok := colToIdx[measureColStr]; !ok {
   207  			colToIdx[measureColStr] = make([]int, 0)
   208  		}
   209  		allReverseIndex = append(allReverseIndex, idx)
   210  		colToIdx[measureColStr] = append(colToIdx[measureColStr], idx)
   211  		allConvertedMeasureOps = append(allConvertedMeasureOps, &structs.MeasureAggregator{
   212  			MeasureCol:         m.MeasureCol,
   213  			MeasureFunc:        mFunc,
   214  			ValueColRequest:    m.ValueColRequest,
   215  			StrEnc:             m.StrEnc,
   216  			OverrodeMeasureAgg: overrodeMeasureAgg,
   217  		})
   218  		idx++
   219  	}
   220  	allConvertedMeasureOps = allConvertedMeasureOps[:idx]
   221  	return colToIdx, allConvertedMeasureOps, allReverseIndex
   222  }
   223  
   224  /*
   225  Returns:
   226    - bool if this record was added
   227    - string for any remote records that were removed
   228  */
   229  func (b *BlockResults) Add(rrc *utils.RecordResultContainer) (bool, string) {
   230  	if b.sortResults {
   231  		return b.SortedResults.Add(rrc)
   232  	}
   233  
   234  	if b.nextUnsortedIdx < b.sizeLimit {
   235  		b.UnsortedResults[b.nextUnsortedIdx] = rrc
   236  		b.nextUnsortedIdx++
   237  		if rrc.SegKeyInfo.IsRemote {
   238  			return true, rrc.SegKeyInfo.RecordId
   239  		}
   240  		return true, ""
   241  	}
   242  	return false, ""
   243  }
   244  
   245  func (b *BlockResults) MergeBuckets(blockRes *BlockResults) {
   246  	if b.TimeAggregation != nil && blockRes.TimeAggregation != nil && !blockRes.aggs.UsedByTimechart() {
   247  		b.TimeAggregation.MergeBuckets(blockRes.TimeAggregation)
   248  	}
   249  	if b.GroupByAggregation != nil && blockRes.GroupByAggregation != nil {
   250  		b.GroupByAggregation.MergeBuckets(blockRes.GroupByAggregation)
   251  	}
   252  }
   253  
   254  func (b *BlockResults) MergeRemoteBuckets(grpBuckets *GroupByBucketsJSON, timeBuckets *TimeBucketsJSON) error {
   255  
   256  	if timeBuckets != nil {
   257  		remoteBuckets, err := timeBuckets.ToTimeBuckets()
   258  		if err != nil {
   259  			return err
   260  		}
   261  		if b.TimeAggregation == nil {
   262  			b.TimeAggregation = remoteBuckets
   263  		} else {
   264  			b.TimeAggregation.MergeBuckets(remoteBuckets)
   265  		}
   266  	}
   267  	if grpBuckets != nil {
   268  		remoteBuckets, err := grpBuckets.ToGroupByBucket(b.aggs.GroupByRequest)
   269  		if err != nil {
   270  			return err
   271  		}
   272  		if b.GroupByAggregation == nil {
   273  			b.GroupByAggregation = remoteBuckets
   274  		} else {
   275  			b.GroupByAggregation.MergeBuckets(remoteBuckets)
   276  		}
   277  	}
   278  	return nil
   279  }
   280  
   281  // if sort is enabled, will call heap.Pop on the underlying results
   282  func (b *BlockResults) GetResults() []*utils.RecordResultContainer {
   283  	if b.sortResults {
   284  		return b.SortedResults.GetSortedResults()
   285  	} else {
   286  		return b.UnsortedResults[:b.nextUnsortedIdx]
   287  	}
   288  }
   289  
   290  // if sort is enabled, will call heap.Pop on the underlying results
   291  func (b *BlockResults) GetResultsCopy() []*utils.RecordResultContainer {
   292  	if b.sortResults {
   293  		return b.SortedResults.GetSortedResultsCopy()
   294  	} else {
   295  		res := make([]*utils.RecordResultContainer, b.nextUnsortedIdx)
   296  		copy(res, b.UnsortedResults[:b.nextUnsortedIdx])
   297  		return res
   298  	}
   299  }
   300  
   301  func (b *BlockResults) AddMatchedCount(c uint64) {
   302  	b.MatchedCount += c
   303  }
   304  
   305  func (b *BlockResults) ShouldAddMore() bool {
   306  	if !b.sortResults {
   307  		return b.nextUnsortedIdx < b.sizeLimit
   308  	} else {
   309  		return true
   310  	}
   311  }
   312  
   313  func (b *BlockResults) WillValueBeAdded(valToAdd float64) bool {
   314  	if !b.sortResults {
   315  		return b.nextUnsortedIdx < b.sizeLimit
   316  	} else {
   317  		if b.sizeLimit == 0 {
   318  			return false
   319  		}
   320  		if b.SortedResults.Results.Len() < int(b.sizeLimit) {
   321  			return true
   322  		}
   323  		if b.SortedResults.Ascending {
   324  			if valToAdd < b.SortedResults.LastValue {
   325  				return true
   326  			}
   327  		} else {
   328  			if valToAdd > b.SortedResults.LastValue {
   329  				return true
   330  			}
   331  		}
   332  		return false
   333  	}
   334  }
   335  
   336  // return true if:
   337  // 1.   if block not fuly enclosed
   338  // 2  if time-HT but we did not use the rollup info to add time-HT
   339  // 3.   if sort present and low/high ts can be added
   340  // 4.   if rrcs left to be filled
   341  func (b *BlockResults) ShouldIterateRecords(aggsHasTimeHt bool, isBlkFullyEncosed bool,
   342  	lowTs uint64, highTs uint64, addedTimeHt bool) bool {
   343  
   344  	// case 1
   345  	if !isBlkFullyEncosed {
   346  		return true
   347  	}
   348  
   349  	if aggsHasTimeHt && !addedTimeHt {
   350  		return true // case 2
   351  	}
   352  
   353  	// case 3
   354  	if b.aggs != nil && b.aggs.Sort != nil {
   355  		if b.aggs.Sort.Ascending {
   356  			return b.WillValueBeAdded(float64(lowTs))
   357  		} else {
   358  			return b.WillValueBeAdded(float64(highTs))
   359  		}
   360  	}
   361  
   362  	// case 4
   363  	return b.nextUnsortedIdx < b.sizeLimit
   364  
   365  }
   366  
   367  func (b *BlockResults) AddMeasureResultsToKey(currKey bytes.Buffer, measureResults []utils.CValueEnclosure, groupByColVal string, usedByTimechart bool, qid uint64) {
   368  
   369  	if b.GroupByAggregation == nil {
   370  		return
   371  	}
   372  	bKey := toputils.UnsafeByteSliceToString(currKey.Bytes())
   373  	bucketIdx, ok := b.GroupByAggregation.StringBucketIdx[bKey]
   374  
   375  	var bucket *RunningBucketResults
   376  	if !ok {
   377  		nBuckets := len(b.GroupByAggregation.AllRunningBuckets)
   378  		if nBuckets >= b.GroupByAggregation.maxBuckets {
   379  			return
   380  		}
   381  		bucket = initRunningGroupByBucket(b.GroupByAggregation.internalMeasureFns)
   382  		b.GroupByAggregation.AllRunningBuckets = append(b.GroupByAggregation.AllRunningBuckets, bucket)
   383  		b.GroupByAggregation.StringBucketIdx[bKey] = nBuckets
   384  	} else {
   385  		bucket = b.GroupByAggregation.AllRunningBuckets[bucketIdx]
   386  	}
   387  
   388  	if usedByTimechart {
   389  		var gRunningStats []runningStats
   390  		_, exists := bucket.groupedRunningStats[groupByColVal]
   391  		if !exists {
   392  			gRunningStats = initRunningStats(b.GroupByAggregation.internalMeasureFns)
   393  			bucket.groupedRunningStats[groupByColVal] = gRunningStats
   394  		}
   395  		gRunningStats = bucket.groupedRunningStats[groupByColVal]
   396  		bucket.AddMeasureResults(&gRunningStats, measureResults, qid, 1, true)
   397  	} else {
   398  		bucket.AddMeasureResults(&bucket.runningStats, measureResults, qid, 1, false)
   399  	}
   400  
   401  }
   402  
   403  func (b *BlockResults) AddMeasureResultsToKeyAgileTree(bKey string,
   404  	measureResults []utils.CValueEnclosure, qid uint64, cnt uint64) {
   405  	if b.GroupByAggregation == nil {
   406  		return
   407  	}
   408  	bucketIdx, ok := b.GroupByAggregation.StringBucketIdx[bKey]
   409  
   410  	var bucket *RunningBucketResults
   411  	if !ok {
   412  		nBuckets := len(b.GroupByAggregation.AllRunningBuckets)
   413  		if nBuckets >= b.GroupByAggregation.maxBuckets {
   414  			return
   415  		}
   416  		bucket = initRunningGroupByBucket(b.GroupByAggregation.internalMeasureFns)
   417  		b.GroupByAggregation.AllRunningBuckets = append(b.GroupByAggregation.AllRunningBuckets, bucket)
   418  		b.GroupByAggregation.StringBucketIdx[bKey] = nBuckets
   419  	} else {
   420  		bucket = b.GroupByAggregation.AllRunningBuckets[bucketIdx]
   421  	}
   422  	bucket.AddMeasureResults(&bucket.runningStats, measureResults, qid, cnt, false)
   423  }
   424  
   425  func (b *BlockResults) AddKeyToTimeBucket(bucketKey uint64, count uint16) {
   426  	if b.TimeAggregation == nil {
   427  		return
   428  	}
   429  	bucketIdx, ok := b.TimeAggregation.UnsignedBucketIdx[bucketKey]
   430  	var bucket *RunningBucketResults
   431  	if !ok {
   432  		bucket = initRunningTimeBucket()
   433  		b.TimeAggregation.AllRunningBuckets = append(b.TimeAggregation.AllRunningBuckets, bucket)
   434  		b.TimeAggregation.UnsignedBucketIdx[bucketKey] = len(b.TimeAggregation.AllRunningBuckets) - 1
   435  	} else {
   436  		bucket = b.TimeAggregation.AllRunningBuckets[bucketIdx]
   437  	}
   438  	bucket.AddTimeToBucketStats(count)
   439  }
   440  
   441  func (b *BlockResults) GetTimeBuckets() *structs.AggregationResult {
   442  	if b.TimeAggregation == nil {
   443  		return &structs.AggregationResult{IsDateHistogram: true}
   444  	}
   445  	return b.TimeAggregation.ConvertToAggregationResult()
   446  }
   447  
   448  // returns a map[cName] -> []idx for measure cols and num measure functions
   449  func (b *BlockResults) GetConvertedMeasureInfo() (map[string][]int, []*structs.MeasureAggregator) {
   450  	if b.GroupByAggregation == nil {
   451  		return nil, nil
   452  	}
   453  	return b.GroupByAggregation.allMeasureCols, b.GroupByAggregation.internalMeasureFns
   454  }
   455  
   456  func (tb *TimeBuckets) MergeBuckets(toMerge *TimeBuckets) {
   457  	for key, idx := range toMerge.UnsignedBucketIdx {
   458  		bucket := toMerge.AllRunningBuckets[idx]
   459  		if idx, ok := tb.UnsignedBucketIdx[key]; !ok {
   460  			tb.AllRunningBuckets = append(tb.AllRunningBuckets, bucket)
   461  			tb.UnsignedBucketIdx[key] = len(tb.AllRunningBuckets) - 1
   462  		} else {
   463  			tb.AllRunningBuckets[idx].MergeRunningBuckets(bucket)
   464  		}
   465  	}
   466  }
   467  
   468  func (tb *TimeBuckets) ConvertToAggregationResult() *structs.AggregationResult {
   469  	results := make([]*structs.BucketResult, len(tb.AllRunningBuckets))
   470  	bucketNum := 0
   471  	for key, idx := range tb.UnsignedBucketIdx {
   472  		bucket := tb.AllRunningBuckets[idx]
   473  		results[bucketNum] = &structs.BucketResult{
   474  			ElemCount: bucket.count,
   475  			BucketKey: key,
   476  		}
   477  		bucketNum++
   478  	}
   479  	results = results[:bucketNum]
   480  	return &structs.AggregationResult{
   481  		IsDateHistogram: true,
   482  		Results:         results,
   483  	}
   484  }
   485  
   486  func (b *BlockResults) GetGroupByBuckets() *structs.AggregationResult {
   487  	if b.GroupByAggregation == nil {
   488  		return &structs.AggregationResult{IsDateHistogram: false}
   489  	}
   490  
   491  	var timechart *structs.TimechartExpr
   492  	if b.aggs.UsedByTimechart() {
   493  		timechart = b.aggs.TimeHistogram.Timechart
   494  	}
   495  	return b.GroupByAggregation.ConvertToAggregationResult(b.aggs.GroupByRequest, timechart)
   496  }
   497  
   498  // If the current GroupByBuckets are used by timechart, and timechart has a limit option, there are two different methods to add results at this point. This is because the limit option has two different ways of calculating scores, and we only return the top or bottom N results, with the remaining ones merged and placed into the 'other' col
   499  // 1. Single Agg: The score is based on the sum of the values in the aggregation. It requires two iterations. In the first iteration, sum up scores for each groupVal
   500  // 2. Multiple Aggs: The score is based on the frequency of each value of <field>. It only requires one iteration because we already have the frep for groupVal before the iteration begins
   501  func (gb *GroupByBuckets) ConvertToAggregationResult(req *structs.GroupByRequest, timechart *structs.TimechartExpr) *structs.AggregationResult {
   502  
   503  	tmLimitResult := &structs.TMLimitResult{
   504  		GroupValScoreMap: aggregations.InitialScoreMap(timechart, gb.GroupByColValCnt),
   505  	}
   506  	isRankBySum := aggregations.IsRankBySum(timechart)
   507  
   508  	// Get scores for ranking
   509  	if isRankBySum {
   510  		for _, idx := range gb.StringBucketIdx {
   511  			bucket := gb.AllRunningBuckets[idx]
   512  			currRes := make(map[string]utils.CValueEnclosure)
   513  			// Add results for group by cols inside the time range bucket
   514  			if len(bucket.groupedRunningStats) > 0 {
   515  				for groupByColVal, gRunningStats := range bucket.groupedRunningStats {
   516  					gb.AddResultToStatRes(req, bucket, gRunningStats, currRes, groupByColVal, timechart, tmLimitResult)
   517  				}
   518  			}
   519  		}
   520  	}
   521  
   522  	bucketNum := 0
   523  	results := make([]*structs.BucketResult, len(gb.AllRunningBuckets))
   524  	tmLimitResult.Hll = hyperloglog.New14()
   525  	tmLimitResult.StrSet = make(map[string]struct{}, 0)
   526  	tmLimitResult.ValIsInLimit = aggregations.CheckGroupByColValsAgainstLimit(timechart, gb.GroupByColValCnt, tmLimitResult.GroupValScoreMap, req.MeasureOperations)
   527  	for key, idx := range gb.StringBucketIdx {
   528  		bucket := gb.AllRunningBuckets[idx]
   529  		currRes := make(map[string]utils.CValueEnclosure)
   530  
   531  		// Add results for group by cols inside the time range bucket
   532  		if len(bucket.groupedRunningStats) > 0 {
   533  			// Every measure operator needs to check whether the current groupByColVal is within the limit
   534  			// If it's not, its col name should be displayed as [aggOp: otherstr]
   535  			otherCValArr := make([]*utils.CValueEnclosure, len(req.MeasureOperations))
   536  			for i := 0; i < len(req.MeasureOperations); i++ {
   537  				otherCValArr[i] = &utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   538  			}
   539  
   540  			tmLimitResult.OtherCValArr = otherCValArr
   541  			for groupByColVal, gRunningStats := range bucket.groupedRunningStats {
   542  				gb.AddResultToStatRes(req, bucket, gRunningStats, currRes, groupByColVal, timechart, tmLimitResult)
   543  			}
   544  
   545  			if timechart.LimitExpr != nil && timechart.LimitExpr.Num < len(bucket.groupedRunningStats) {
   546  				for index, mInfo := range req.MeasureOperations {
   547  					// To be modified: user can customize otherstr
   548  					mInfoStr := mInfo.String() + ": other"
   549  					currRes[mInfoStr] = *tmLimitResult.OtherCValArr[index]
   550  				}
   551  			}
   552  
   553  		} else {
   554  			gb.AddResultToStatRes(req, bucket, bucket.runningStats, currRes, "", nil, tmLimitResult)
   555  		}
   556  
   557  		var bucketKey interface{}
   558  		bucketKey, err := utils.ConvertGroupByKey([]byte(key))
   559  		if len(bucketKey.([]string)) == 1 {
   560  			bucketKey = bucketKey.([]string)[0]
   561  		}
   562  		if err != nil {
   563  			log.Errorf("ConvertToAggregationResult: failed to extract raw key: %v", err)
   564  		}
   565  		results[bucketNum] = &structs.BucketResult{
   566  			ElemCount:   bucket.count,
   567  			BucketKey:   bucketKey,
   568  			StatRes:     currRes,
   569  			GroupByKeys: req.GroupByColumns,
   570  		}
   571  		bucketNum++
   572  	}
   573  
   574  	aggregations.SortTimechartRes(timechart, &results)
   575  	return &structs.AggregationResult{
   576  		IsDateHistogram: false,
   577  		Results:         results,
   578  	}
   579  }
   580  
   581  func (gb *GroupByBuckets) AddResultToStatRes(req *structs.GroupByRequest, bucket *RunningBucketResults, runningStats []runningStats, currRes map[string]utils.CValueEnclosure,
   582  	groupByColVal string, timechart *structs.TimechartExpr, tmLimitResult *structs.TMLimitResult) {
   583  	// Some aggregate functions require multiple measure funcs or raw field values to calculate the result. For example, range() needs both max() and min(), and aggregates with eval statements may require multiple raw field values
   584  	// Therefore, it is essential to assign a value to 'idx' appropriately to skip the intermediate results generated during the computation from runningStats bucket
   585  	idx := 0
   586  
   587  	// If current col should be merged into the other col
   588  	isOtherCol := aggregations.IsOtherCol(tmLimitResult.ValIsInLimit, groupByColVal)
   589  	usedByTimechart := (timechart != nil)
   590  	usedByTimechartGroupByCol := len(groupByColVal) > 0
   591  	for index, mInfo := range req.MeasureOperations {
   592  		mInfoStr := mInfo.String()
   593  		if usedByTimechartGroupByCol {
   594  			if !isOtherCol {
   595  				mInfoStr = mInfoStr + ": " + groupByColVal
   596  			}
   597  		}
   598  
   599  		var hllToMerge *hyperloglog.Sketch
   600  		var strSetToMerge map[string]struct{}
   601  		var eVal utils.CValueEnclosure
   602  		switch mInfo.MeasureFunc {
   603  		case utils.Count:
   604  			if mInfo.ValueColRequest != nil || usedByTimechart {
   605  				if !usedByTimechart && len(mInfo.ValueColRequest.GetFields()) == 0 {
   606  					log.Errorf("AddResultToStatRes: Incorrect number of fields for aggCol: %v", mInfoStr)
   607  					continue
   608  				}
   609  
   610  				countIdx := gb.reverseMeasureIndex[idx]
   611  				countVal, err := runningStats[countIdx].rawVal.GetUIntValue()
   612  				if err != nil {
   613  					currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   614  					continue
   615  				}
   616  				eVal = utils.CValueEnclosure{CVal: countVal, Dtype: utils.SS_DT_UNSIGNED_NUM}
   617  			} else {
   618  				eVal = utils.CValueEnclosure{CVal: bucket.count, Dtype: utils.SS_DT_UNSIGNED_NUM}
   619  			}
   620  			idx++
   621  		case utils.Avg:
   622  			sumIdx := gb.reverseMeasureIndex[idx]
   623  			sumRawVal, err := runningStats[sumIdx].rawVal.GetFloatValue()
   624  			if err != nil {
   625  				currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   626  				continue
   627  			}
   628  
   629  			var avg float64
   630  			if mInfo.ValueColRequest != nil || usedByTimechart {
   631  				countIdx := gb.reverseMeasureIndex[idx+1]
   632  				countRawVal, err := runningStats[countIdx].rawVal.GetFloatValue()
   633  				if err != nil {
   634  					currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   635  					continue
   636  				}
   637  				eVal = utils.CValueEnclosure{CVal: sumRawVal / countRawVal, Dtype: utils.SS_DT_FLOAT}
   638  				idx += 2
   639  			} else {
   640  				if bucket.count == 0 {
   641  					avg = 0
   642  				} else {
   643  					avg = sumRawVal / float64(bucket.count)
   644  				}
   645  				eVal = utils.CValueEnclosure{CVal: avg, Dtype: utils.SS_DT_FLOAT}
   646  				idx++
   647  			}
   648  		case utils.Range:
   649  			minIdx := gb.reverseMeasureIndex[idx]
   650  			minRawVal, err := runningStats[minIdx].rawVal.GetFloatValue()
   651  			if err != nil {
   652  				currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   653  				continue
   654  			}
   655  
   656  			maxIdx := gb.reverseMeasureIndex[idx+1]
   657  			maxRawVal, err := runningStats[maxIdx].rawVal.GetFloatValue()
   658  			if err != nil {
   659  				currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   660  				continue
   661  			}
   662  
   663  			eVal = utils.CValueEnclosure{CVal: maxRawVal - minRawVal, Dtype: utils.SS_DT_FLOAT}
   664  			idx += 2
   665  		case utils.Cardinality:
   666  			valIdx := gb.reverseMeasureIndex[idx]
   667  			if mInfo.ValueColRequest != nil {
   668  				if len(mInfo.ValueColRequest.GetFields()) == 0 {
   669  					log.Errorf("AddResultToStatRes: Incorrect number of fields for aggCol: %v", mInfoStr)
   670  					continue
   671  				}
   672  				strSet, ok := runningStats[valIdx].rawVal.CVal.(map[string]struct{})
   673  				if !ok {
   674  					currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   675  					continue
   676  				}
   677  				eVal = utils.CValueEnclosure{CVal: uint64(len(strSet)), Dtype: utils.SS_DT_UNSIGNED_NUM}
   678  			} else {
   679  				finalVal := runningStats[valIdx].hll.Estimate()
   680  				eVal = utils.CValueEnclosure{CVal: finalVal, Dtype: utils.SS_DT_UNSIGNED_NUM}
   681  				hllToMerge = runningStats[valIdx].hll
   682  			}
   683  
   684  			idx++
   685  		case utils.Values:
   686  			if mInfo.ValueColRequest != nil {
   687  				if len(mInfo.ValueColRequest.GetFields()) == 0 {
   688  					log.Errorf("AddResultToStatRes: Incorrect number of fields for aggCol: %v", mInfoStr)
   689  					continue
   690  				}
   691  			}
   692  
   693  			valIdx := gb.reverseMeasureIndex[idx]
   694  			strSet, ok := runningStats[valIdx].rawVal.CVal.(map[string]struct{})
   695  			if !ok {
   696  				currRes[mInfoStr] = utils.CValueEnclosure{CVal: nil, Dtype: utils.SS_INVALID}
   697  				continue
   698  			}
   699  			strSetToMerge = strSet
   700  
   701  			uniqueStrings := make([]string, 0)
   702  			for str := range strSet {
   703  				uniqueStrings = append(uniqueStrings, str)
   704  			}
   705  
   706  			sort.Strings(uniqueStrings)
   707  
   708  			strVal := strings.Join(uniqueStrings, "&nbsp")
   709  			eVal = utils.CValueEnclosure{
   710  				Dtype: utils.SS_DT_STRING,
   711  				CVal:  strVal,
   712  			}
   713  
   714  			idx++
   715  		default:
   716  			valIdx := gb.reverseMeasureIndex[idx]
   717  			eVal = runningStats[valIdx].rawVal
   718  			idx++
   719  		}
   720  		shouldAddRes := aggregations.ShouldAddRes(timechart, tmLimitResult, index, eVal, hllToMerge, strSetToMerge, mInfo.MeasureFunc, groupByColVal, isOtherCol)
   721  		if shouldAddRes {
   722  			currRes[mInfoStr] = eVal
   723  		}
   724  	}
   725  }
   726  
   727  func (gb *GroupByBuckets) MergeBuckets(toMerge *GroupByBuckets) {
   728  
   729  	if len(gb.GroupByColValCnt) > 0 {
   730  		aggregations.MergeMap(gb.GroupByColValCnt, toMerge.GroupByColValCnt)
   731  	} else {
   732  		gb.GroupByColValCnt = toMerge.GroupByColValCnt
   733  	}
   734  
   735  	for key, idx := range toMerge.StringBucketIdx {
   736  		bucket := toMerge.AllRunningBuckets[idx]
   737  		if idx, ok := gb.StringBucketIdx[key]; !ok {
   738  			if len(gb.AllRunningBuckets) >= gb.maxBuckets {
   739  				continue
   740  			}
   741  			gb.AllRunningBuckets = append(gb.AllRunningBuckets, bucket)
   742  			gb.StringBucketIdx[key] = len(gb.AllRunningBuckets) - 1
   743  		} else {
   744  			gb.AllRunningBuckets[idx].MergeRunningBuckets(bucket)
   745  		}
   746  	}
   747  }
   748  
   749  func (gb *GroupByBuckets) ConvertToJson() (*GroupByBucketsJSON, error) {
   750  	retVal := &GroupByBucketsJSON{
   751  		AllGroupbyBuckets: make(map[string]*RunningBucketResultsJSON, len(gb.AllRunningBuckets)),
   752  	}
   753  	for key, idx := range gb.StringBucketIdx {
   754  		bucket := gb.AllRunningBuckets[idx]
   755  		newBucket := &RunningBucketResultsJSON{
   756  			Count:     bucket.count,
   757  			CurrStats: bucket.currStats,
   758  		}
   759  		retVals := make([]interface{}, 0, len(bucket.currStats))
   760  		for idx, rs := range bucket.runningStats {
   761  			if bucket.currStats[idx].MeasureFunc == utils.Cardinality {
   762  				encoded, err := rs.hll.MarshalBinary()
   763  				if err != nil {
   764  					log.Errorf("GroupByBuckets.ConvertToJson: failed to marshal hll: %v", err)
   765  					return nil, err
   766  				}
   767  				retVals = append(retVals, encoded)
   768  			} else {
   769  				retVals = append(retVals, rs.rawVal.CVal)
   770  			}
   771  		}
   772  		newBucket.RunningStats = retVals
   773  		retVal.AllGroupbyBuckets[key] = newBucket
   774  	}
   775  	return retVal, nil
   776  }
   777  
   778  func (tb *TimeBuckets) ConvertToJson() (*TimeBucketsJSON, error) {
   779  	retVal := &TimeBucketsJSON{
   780  		AllTimeBuckets: make(map[uint64]*RunningBucketResultsJSON, len(tb.AllRunningBuckets)),
   781  	}
   782  	for key, idx := range tb.UnsignedBucketIdx {
   783  		bucket := tb.AllRunningBuckets[idx]
   784  		newBucket := &RunningBucketResultsJSON{
   785  			Count:     bucket.count,
   786  			CurrStats: bucket.currStats,
   787  		}
   788  		retVals := make([]interface{}, 0, len(bucket.currStats))
   789  		for idx, rs := range bucket.runningStats {
   790  			if bucket.currStats[idx].MeasureFunc == utils.Cardinality {
   791  				encoded, err := rs.hll.MarshalBinary()
   792  				if err != nil {
   793  					log.Errorf("TimeBuckets.ConvertToJson: failed to marshal hll: %v", err)
   794  					return nil, err
   795  				}
   796  				retVals = append(retVals, encoded)
   797  			} else {
   798  				retVals = append(retVals, rs.rawVal.CVal)
   799  			}
   800  		}
   801  		newBucket.RunningStats = retVals
   802  		retVal.AllTimeBuckets[key] = newBucket
   803  	}
   804  	return retVal, nil
   805  }
   806  
   807  func (tb *TimeBucketsJSON) ToTimeBuckets() (*TimeBuckets, error) {
   808  	retVal := &TimeBuckets{
   809  		AllRunningBuckets: make([]*RunningBucketResults, 0, len(tb.AllTimeBuckets)),
   810  		UnsignedBucketIdx: make(map[uint64]int, len(tb.AllTimeBuckets)),
   811  	}
   812  	reverseIndex := 0
   813  	for key, runningBucket := range tb.AllTimeBuckets {
   814  		newBucket, err := runningBucket.Convert()
   815  		if err != nil {
   816  			return nil, err
   817  		}
   818  		retVal.AllRunningBuckets = append(retVal.AllRunningBuckets, newBucket)
   819  		retVal.UnsignedBucketIdx[key] = reverseIndex
   820  		reverseIndex++
   821  	}
   822  	return retVal, nil
   823  }
   824  
   825  func (gb *GroupByBucketsJSON) ToGroupByBucket(req *structs.GroupByRequest) (*GroupByBuckets, error) {
   826  	mCols, mFuns, revIndex := convertRequestToInternalStats(req, false)
   827  	retVal := &GroupByBuckets{
   828  		AllRunningBuckets:   make([]*RunningBucketResults, 0, len(gb.AllGroupbyBuckets)),
   829  		StringBucketIdx:     make(map[string]int, len(gb.AllGroupbyBuckets)),
   830  		allMeasureCols:      mCols,
   831  		internalMeasureFns:  mFuns,
   832  		reverseMeasureIndex: revIndex,
   833  		maxBuckets:          req.BucketCount,
   834  	}
   835  	reverseIndex := 0
   836  	for key, runningBucket := range gb.AllGroupbyBuckets {
   837  		newBucket, err := runningBucket.Convert()
   838  		if err != nil {
   839  			return nil, err
   840  		}
   841  		retVal.AllRunningBuckets = append(retVal.AllRunningBuckets, newBucket)
   842  		retVal.StringBucketIdx[key] = reverseIndex
   843  		reverseIndex++
   844  	}
   845  	return retVal, nil
   846  }
   847  
   848  func (rb *RunningBucketResultsJSON) Convert() (*RunningBucketResults, error) {
   849  	newBucket := &RunningBucketResults{
   850  		count:     rb.Count,
   851  		currStats: rb.CurrStats,
   852  	}
   853  	currRunningStats := make([]runningStats, 0, len(rb.RunningStats))
   854  	for statsIdx, rs := range rb.RunningStats {
   855  		if rb.CurrStats[statsIdx].MeasureFunc == utils.Cardinality {
   856  			hll := hyperloglog.New()
   857  			hllString, ok := rs.(string)
   858  			if !ok {
   859  				log.Errorf("RunningBucketResultsJSON.Convert: failed to convert hll to byte array %+v %T", rs, rs)
   860  				return nil, fmt.Errorf("failed to convert hll to byte array")
   861  			}
   862  			hllBytes, err := base64.StdEncoding.DecodeString(hllString)
   863  			if err != nil {
   864  				log.Errorf("RunningBucketResultsJSON.Convert: failed to decode hll: %v", err)
   865  				return nil, err
   866  			}
   867  			err = hll.UnmarshalBinary(hllBytes)
   868  			if err != nil {
   869  				log.Errorf("RunningBucketResultsJSON.Convert: failed to unmarshal hll: %v", err)
   870  				return nil, err
   871  			}
   872  			currRunningStats = append(currRunningStats, runningStats{hll: hll})
   873  		} else {
   874  			newVal := utils.CValueEnclosure{}
   875  			err := newVal.ConvertValue(rs)
   876  			if err != nil {
   877  				log.Errorf("RunningBucketResultsJSON.Convert: failed to convert value: %v", err)
   878  				return nil, err
   879  			}
   880  			currRunningStats = append(currRunningStats, runningStats{rawVal: newVal})
   881  		}
   882  	}
   883  	newBucket.runningStats = currRunningStats
   884  	return newBucket, nil
   885  }