github.com/siglens/siglens@v0.0.0-20240328180423-f7ce9ae441ed/pkg/segment/aggregations/segaggs.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 aggregations 18 19 import ( 20 "encoding/json" 21 "errors" 22 "fmt" 23 "regexp" 24 "sort" 25 "strconv" 26 27 "github.com/siglens/siglens/pkg/common/dtypeutils" 28 "github.com/siglens/siglens/pkg/segment/structs" 29 segutils "github.com/siglens/siglens/pkg/segment/utils" 30 "github.com/siglens/siglens/pkg/utils" 31 log "github.com/sirupsen/logrus" 32 ) 33 34 func applyTimeRangeHistogram(nodeResult *structs.NodeResult, rangeHistogram *structs.TimeBucket, aggName string) { 35 36 if nodeResult.Histogram == nil || rangeHistogram.Timechart != nil { 37 return 38 } 39 res, ok := nodeResult.Histogram[aggName] 40 if !ok || res == nil { 41 return 42 } 43 44 nodeResult.Histogram[aggName].IsDateHistogram = true 45 if rangeHistogram.EndTime != 0 || rangeHistogram.StartTime != 0 { // default values of uint64 46 finalList := make([]*structs.BucketResult, 0) 47 for _, recs := range nodeResult.Histogram[aggName].Results { 48 bucketTime, ok := recs.BucketKey.(uint64) 49 if !ok { 50 log.Errorf("time for bucket aggregation is not uint64!") 51 continue 52 } 53 if rangeHistogram.EndTime != 0 && bucketTime > rangeHistogram.EndTime { 54 continue 55 } 56 if rangeHistogram.StartTime != 0 && bucketTime < rangeHistogram.StartTime { 57 continue 58 } 59 finalList = append(finalList, recs) 60 } 61 nodeResult.Histogram[aggName].Results = finalList 62 } 63 sort.Slice(nodeResult.Histogram[aggName].Results, func(i, j int) bool { 64 iVal, ok := nodeResult.Histogram[aggName].Results[i].BucketKey.(uint64) 65 if !ok { 66 return false 67 } 68 jVal, ok := nodeResult.Histogram[aggName].Results[j].BucketKey.(uint64) 69 if !ok { 70 return true 71 } 72 return iVal < jVal 73 }) 74 } 75 76 // Function to clean up results based on input query aggregations. 77 // This will make sure all buckets respect the minCount & is returned in a sorted order 78 func PostQueryBucketCleaning(nodeResult *structs.NodeResult, post *structs.QueryAggregators, recs map[string]map[string]interface{}, 79 recordIndexInFinal map[string]int, finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) *structs.NodeResult { 80 if post == nil { 81 return nodeResult 82 } 83 84 if post.TimeHistogram != nil { 85 applyTimeRangeHistogram(nodeResult, post.TimeHistogram, post.TimeHistogram.AggName) 86 } 87 88 if post.GroupByRequest != nil { 89 nodeResult.GroupByCols = post.GroupByRequest.GroupByColumns 90 nodeResult.GroupByRequest = post.GroupByRequest 91 } 92 93 if post.TransactionArguments != nil && len(recs) == 0 { 94 return nodeResult 95 } 96 97 // For the query without groupby, skip the first aggregator without a QueryAggergatorBlock 98 // For the query that has a groupby, groupby block's aggregation is in the post.Next. Therefore, we should start from the groupby's aggregation. 99 if !post.HasQueryAggergatorBlock() && post.TransactionArguments == nil { 100 post = post.Next 101 } 102 103 for agg := post; agg != nil; agg = agg.Next { 104 err := performAggOnResult(nodeResult, agg, recs, recordIndexInFinal, finalCols, numTotalSegments, finishesSegment) 105 106 if len(nodeResult.TransactionEventRecords) > 0 { 107 nodeResult.NextQueryAgg = agg 108 return nodeResult 109 } else if nodeResult.PerformAggsOnRecs && len(recs) > 0 { 110 nodeResult.NextQueryAgg = agg 111 return nodeResult 112 } 113 114 if err != nil { 115 log.Errorf("PostQueryBucketCleaning: %v", err) 116 nodeResult.ErrList = append(nodeResult.ErrList, err) 117 } 118 } 119 120 return nodeResult 121 } 122 123 func performAggOnResult(nodeResult *structs.NodeResult, agg *structs.QueryAggregators, recs map[string]map[string]interface{}, 124 recordIndexInFinal map[string]int, finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) error { 125 switch agg.PipeCommandType { 126 case structs.OutputTransformType: 127 if agg.OutputTransforms == nil { 128 return errors.New("performAggOnResult: expected non-nil OutputTransforms") 129 } 130 131 colReq := agg.OutputTransforms.OutputColumns 132 if colReq != nil { 133 err := performColumnsRequest(nodeResult, colReq, recs, finalCols) 134 135 if err != nil { 136 return fmt.Errorf("performAggOnResult: %v", err) 137 } 138 } 139 140 if agg.OutputTransforms.LetColumns != nil { 141 err := performLetColumnsRequest(nodeResult, agg, agg.OutputTransforms.LetColumns, recs, recordIndexInFinal, finalCols, numTotalSegments, finishesSegment) 142 143 if err != nil { 144 return fmt.Errorf("performAggOnResult: %v", err) 145 } 146 } 147 148 if agg.OutputTransforms.FilterRows != nil { 149 err := performFilterRows(nodeResult, agg.OutputTransforms.FilterRows) 150 151 if err != nil { 152 return fmt.Errorf("performAggOnResult: %v", err) 153 } 154 } 155 156 if agg.OutputTransforms.MaxRows > 0 { 157 err := performMaxRows(nodeResult, agg, agg.OutputTransforms.MaxRows, recs) 158 159 if err != nil { 160 return fmt.Errorf("performAggOnResult: %v", err) 161 } 162 } 163 case structs.GroupByType: 164 nodeResult.PerformAggsOnRecs = true 165 nodeResult.RecsAggsType = structs.GroupByType 166 nodeResult.GroupByCols = agg.GroupByRequest.GroupByColumns 167 nodeResult.GroupByRequest = agg.GroupByRequest 168 case structs.MeasureAggsType: 169 nodeResult.PerformAggsOnRecs = true 170 nodeResult.RecsAggsType = structs.MeasureAggsType 171 nodeResult.MeasureOperations = agg.MeasureOperations 172 case structs.TransactionType: 173 performTransactionCommandRequest(nodeResult, agg, recs, finalCols, numTotalSegments, finishesSegment) 174 default: 175 return errors.New("performAggOnResult: multiple QueryAggregators is currently only supported for OutputTransformType") 176 } 177 178 return nil 179 } 180 181 func performMaxRows(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, maxRows uint64, recs map[string]map[string]interface{}) error { 182 183 if maxRows == 0 { 184 return nil 185 } 186 187 if recs != nil { 188 // If the number of records plus the already added Rows is less than the maxRows, we don't need to do anything. 189 if (uint64(len(recs)) + aggs.OutputTransforms.RowsAdded) <= maxRows { 190 aggs.OutputTransforms.RowsAdded += uint64(len(recs)) 191 return nil 192 } 193 194 // If the number of records is greater than the maxRows, we need to remove the extra records. 195 for key := range recs { 196 if aggs.OutputTransforms.RowsAdded >= maxRows { 197 delete(recs, key) 198 continue 199 } 200 aggs.OutputTransforms.RowsAdded++ 201 } 202 203 return nil 204 } 205 206 // Follow group by 207 if nodeResult.Histogram != nil { 208 for _, aggResult := range nodeResult.Histogram { 209 if (uint64(len(aggResult.Results)) + aggs.OutputTransforms.RowsAdded) <= maxRows { 210 aggs.OutputTransforms.RowsAdded += uint64(len(aggResult.Results)) 211 continue 212 } 213 214 // If the number of records is greater than the maxRows, we need to remove the extra records. 215 aggResult.Results = aggResult.Results[:maxRows-aggs.OutputTransforms.RowsAdded] 216 aggs.OutputTransforms.RowsAdded = maxRows 217 break 218 } 219 return nil 220 } 221 222 return nil 223 } 224 225 func performColumnsRequestWithoutGroupby(nodeResult *structs.NodeResult, colReq *structs.ColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 226 if colReq.RenameAggregationColumns != nil { 227 for oldCName, newCName := range colReq.RenameAggregationColumns { 228 if _, exists := finalCols[oldCName]; !exists { 229 log.Errorf("performColumnsRequestWithoutGroupby: column %v does not exist", oldCName) 230 continue 231 } 232 finalCols[newCName] = true 233 delete(finalCols, oldCName) 234 235 for _, record := range recs { 236 if val, exists := record[oldCName]; exists { 237 record[newCName] = val 238 delete(record, oldCName) 239 } 240 } 241 } 242 } 243 244 if colReq.RenameColumns != nil { 245 for oldCName, newCName := range colReq.RenameColumns { 246 if _, exists := finalCols[oldCName]; !exists { 247 log.Errorf("performColumnsRequestWithoutGroupby: column %v does not exist", oldCName) 248 continue 249 } 250 finalCols[newCName] = true 251 delete(finalCols, oldCName) 252 253 for _, record := range recs { 254 if val, exists := record[oldCName]; exists { 255 record[newCName] = val 256 delete(record, oldCName) 257 } 258 } 259 } 260 } 261 262 if colReq.ExcludeColumns != nil { 263 // Remove the specified columns, which may have wildcards. 264 matchingCols := getMatchingColumns(colReq.ExcludeColumns, finalCols) 265 for _, matchingCol := range matchingCols { 266 delete(finalCols, matchingCol) 267 } 268 } 269 270 if colReq.IncludeColumns != nil { 271 // Remove all columns except the specified ones, which may have wildcards. 272 if finalCols == nil { 273 return errors.New("performColumnsRequest: finalCols is nil") 274 } 275 276 matchingCols := getMatchingColumns(colReq.IncludeColumns, finalCols) 277 278 // First remove everything. 279 for col := range finalCols { 280 delete(finalCols, col) 281 } 282 283 // Add the matching columns. 284 for _, matchingCol := range matchingCols { 285 finalCols[matchingCol] = true 286 } 287 } 288 289 return nil 290 } 291 292 func performColumnsRequest(nodeResult *structs.NodeResult, colReq *structs.ColumnsRequest, recs map[string]map[string]interface{}, 293 finalCols map[string]bool) error { 294 295 if recs != nil { 296 if err := performColumnsRequestWithoutGroupby(nodeResult, colReq, recs, finalCols); err != nil { 297 return fmt.Errorf("performColumnsRequest: %v", err) 298 } 299 } 300 301 nodeResult.RenameColumns = colReq.RenameAggregationColumns 302 RenamingLoop: 303 for oldCName, newCName := range colReq.RenameAggregationColumns { 304 for i, cName := range nodeResult.MeasureFunctions { 305 if cName == oldCName { 306 nodeResult.MeasureFunctions[i] = newCName 307 308 // Change the name in MeasureResults. 309 for _, bucketHolder := range nodeResult.MeasureResults { 310 bucketHolder.MeasureVal[newCName] = bucketHolder.MeasureVal[oldCName] 311 delete(bucketHolder.MeasureVal, oldCName) 312 } 313 314 // Change the name in Histogram. 315 for _, aggResult := range nodeResult.Histogram { 316 for _, bucketResult := range aggResult.Results { 317 for cName, value := range bucketResult.StatRes { 318 if cName == oldCName { 319 bucketResult.StatRes[newCName] = value 320 delete(bucketResult.StatRes, oldCName) 321 } 322 } 323 } 324 } 325 326 continue RenamingLoop 327 } 328 } 329 330 log.Warnf("performColumnsRequest: column %v does not exist or is not an aggregation column", oldCName) 331 } 332 333 if colReq.RenameColumns != nil { 334 335 for oldCName, newCName := range colReq.RenameColumns { 336 // Rename in MeasureFunctions 337 for i, cName := range nodeResult.MeasureFunctions { 338 if cName == oldCName { 339 nodeResult.MeasureFunctions[i] = newCName 340 } 341 } 342 343 // Rename in MeasureResults 344 for _, bucketHolder := range nodeResult.MeasureResults { 345 if _, exists := bucketHolder.MeasureVal[oldCName]; exists { 346 bucketHolder.MeasureVal[newCName] = bucketHolder.MeasureVal[oldCName] 347 delete(bucketHolder.MeasureVal, oldCName) 348 } 349 } 350 351 // Rename in Histogram 352 for _, aggResult := range nodeResult.Histogram { 353 for _, bucketResult := range aggResult.Results { 354 if value, exists := bucketResult.StatRes[oldCName]; exists { 355 bucketResult.StatRes[newCName] = value 356 delete(bucketResult.StatRes, oldCName) 357 } 358 } 359 } 360 } 361 362 return nil 363 } 364 365 if colReq.ExcludeColumns != nil { 366 if nodeResult.GroupByRequest == nil { 367 return errors.New("performColumnsRequest: expected non-nil GroupByRequest while handling ExcludeColumns") 368 } 369 370 groupByColIndicesToKeep, groupByColNamesToKeep, _ := getColumnsToKeepAndRemove(nodeResult.GroupByRequest.GroupByColumns, colReq.ExcludeColumns, false) 371 _, _, measureColNamesToRemove := getColumnsToKeepAndRemove(nodeResult.MeasureFunctions, colReq.ExcludeColumns, false) 372 373 err := removeAggColumns(nodeResult, groupByColIndicesToKeep, groupByColNamesToKeep, measureColNamesToRemove) 374 if err != nil { 375 return fmt.Errorf("performColumnsRequest: error handling ExcludeColumns: %v", err) 376 } 377 } 378 if colReq.IncludeColumns != nil { 379 if nodeResult.GroupByRequest == nil { 380 return errors.New("performColumnsRequest: expected non-nil GroupByRequest while handling IncludeColumns") 381 } 382 383 groupByColIndicesToKeep, groupByColNamesToKeep, _ := getColumnsToKeepAndRemove(nodeResult.GroupByRequest.GroupByColumns, colReq.IncludeColumns, true) 384 _, _, measureColNamesToRemove := getColumnsToKeepAndRemove(nodeResult.MeasureFunctions, colReq.IncludeColumns, true) 385 386 err := removeAggColumns(nodeResult, groupByColIndicesToKeep, groupByColNamesToKeep, measureColNamesToRemove) 387 if err != nil { 388 return fmt.Errorf("performColumnsRequest: error handling IncludeColumns: %v", err) 389 } 390 } 391 if colReq.IncludeValues != nil { 392 return errors.New("performColumnsRequest: processing ColumnsRequest.IncludeValues is not implemented") 393 } 394 if colReq.Logfmt { 395 return errors.New("performColumnsRequest: processing ColumnsRequest for Logfmt is not implemented") 396 } 397 398 return nil 399 } 400 401 // Return all the columns in finalCols that match any of the wildcardCols, 402 // which may or may not contain wildcards. 403 // Note that the results may have duplicates if a column in finalCols matches 404 // multiple wildcardCols. 405 func getMatchingColumns(wildcardCols []string, finalCols map[string]bool) []string { 406 currentCols := make([]string, len(finalCols)) 407 i := 0 408 for col := range finalCols { 409 currentCols[i] = col 410 i++ 411 } 412 413 matchingCols := make([]string, 0) 414 for _, wildcardCol := range wildcardCols { 415 matchingCols = append(matchingCols, utils.SelectMatchingStringsWithWildcard(wildcardCol, currentCols)...) 416 } 417 418 return matchingCols 419 } 420 421 // This function finds which columns in `cols` match any of the wildcardCols, 422 // which may or may not contain wildcards. It returns the indices and the names 423 // of the columns to keep, as well as the names of the columns to remove. 424 // When keepMatches is true, a column is kept only if it matches at least one 425 // wildcardCol. When keepMatches is false, a column is kept only if it matches 426 // no wildcardCol. 427 // The results are returned in the same order as the input `cols`. 428 func getColumnsToKeepAndRemove(cols []string, wildcardCols []string, keepMatches bool) ([]int, []string, []string) { 429 indicesToKeep := make([]int, 0) 430 colsToKeep := make([]string, 0) 431 colsToRemove := make([]string, 0) 432 433 for i, col := range cols { 434 keep := !keepMatches 435 for _, wildcardCol := range wildcardCols { 436 isMatch := len(utils.SelectMatchingStringsWithWildcard(wildcardCol, []string{col})) > 0 437 if isMatch { 438 keep = keepMatches 439 break 440 } 441 } 442 443 if keep { 444 indicesToKeep = append(indicesToKeep, i) 445 colsToKeep = append(colsToKeep, col) 446 } else { 447 colsToRemove = append(colsToRemove, col) 448 } 449 } 450 451 return indicesToKeep, colsToKeep, colsToRemove 452 } 453 454 func removeAggColumns(nodeResult *structs.NodeResult, groupByColIndicesToKeep []int, groupByColNamesToKeep []string, measureColNamesToRemove []string) error { 455 // Remove columns from Histogram. 456 for _, aggResult := range nodeResult.Histogram { 457 for _, bucketResult := range aggResult.Results { 458 bucketResult.GroupByKeys = groupByColNamesToKeep 459 460 // Update the BucketKey. 461 bucketKeySlice, err := decodeBucketKey(bucketResult.BucketKey) 462 if err != nil { 463 return fmt.Errorf("removeAggColumns: failed to decode bucket key %v, err=%v", bucketResult.BucketKey, err) 464 } 465 bucketKeySlice = utils.SelectIndicesFromSlice(bucketKeySlice, groupByColIndicesToKeep) 466 bucketResult.BucketKey = encodeBucketKey(bucketKeySlice) 467 468 // Remove measure columns. 469 for _, bucketResult := range aggResult.Results { 470 for _, measureColName := range measureColNamesToRemove { 471 delete(bucketResult.StatRes, measureColName) 472 } 473 } 474 } 475 } 476 477 // Remove columns from MeasureResults. 478 for _, bucketHolder := range nodeResult.MeasureResults { 479 // Remove groupby columns. 480 bucketHolder.GroupByValues = utils.SelectIndicesFromSlice(bucketHolder.GroupByValues, groupByColIndicesToKeep) 481 482 // Remove measure columns. 483 for _, measureColName := range measureColNamesToRemove { 484 delete(bucketHolder.MeasureVal, measureColName) 485 } 486 } 487 488 if nodeResult.GroupByRequest == nil { 489 return fmt.Errorf("removeAggColumns: expected non-nil GroupByRequest") 490 } else { 491 nodeResult.GroupByRequest.GroupByColumns = groupByColNamesToKeep 492 } 493 494 return nil 495 } 496 497 func performLetColumnsRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, 498 recordIndexInFinal map[string]int, finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) error { 499 500 if letColReq.NewColName == "" && !aggs.HasQueryAggergatorBlock() && letColReq.StatisticColRequest == nil { 501 return errors.New("performLetColumnsRequest: expected non-empty NewColName") 502 } 503 504 // Exactly one of MultiColsRequest, SingleColRequest, ValueColRequest, RexColRequest, RenameColRequest should contain data. 505 if letColReq.MultiColsRequest != nil { 506 return errors.New("performLetColumnsRequest: processing LetColumnsRequest.MultiColsRequest is not implemented") 507 } else if letColReq.SingleColRequest != nil { 508 return errors.New("performLetColumnsRequest: processing LetColumnsRequest.SingleColRequest is not implemented") 509 } else if letColReq.ValueColRequest != nil { 510 if err := performValueColRequest(nodeResult, aggs, letColReq, recs, finalCols); err != nil { 511 return fmt.Errorf("performLetColumnsRequest: %v", err) 512 } 513 } else if letColReq.RexColRequest != nil { 514 if err := performRexColRequest(nodeResult, aggs, letColReq, recs, finalCols); err != nil { 515 return fmt.Errorf("performLetColumnsRequest: %v", err) 516 } 517 } else if letColReq.RenameColRequest != nil { 518 if err := performRenameColRequest(nodeResult, aggs, letColReq, recs, finalCols); err != nil { 519 return fmt.Errorf("performLetColumnsRequest: %v", err) 520 } 521 } else if letColReq.StatisticColRequest != nil { 522 if err := performStatisticColRequest(nodeResult, aggs, letColReq, recs); err != nil { 523 return fmt.Errorf("performLetColumnsRequest: %v", err) 524 } 525 } else if letColReq.DedupColRequest != nil { 526 if err := performDedupColRequest(nodeResult, aggs, letColReq, recs, finalCols, numTotalSegments, finishesSegment); err != nil { 527 return fmt.Errorf("performLetColumnsRequest: %v", err) 528 } 529 } else if letColReq.SortColRequest != nil { 530 if err := performSortColRequest(nodeResult, aggs, letColReq, recs, recordIndexInFinal, finalCols, numTotalSegments, finishesSegment); err != nil { 531 return fmt.Errorf("performLetColumnsRequest: %v", err) 532 } 533 } else { 534 return errors.New("performLetColumnsRequest: expected one of MultiColsRequest, SingleColRequest, ValueColRequest, RexColRequest to have a value") 535 } 536 537 return nil 538 } 539 540 func performRenameColRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 541 //Without following group by 542 if recs != nil { 543 if err := performRenameColRequestWithoutGroupby(nodeResult, letColReq, recs, finalCols); err != nil { 544 return fmt.Errorf("performRenameColRequest: %v", err) 545 } 546 return nil 547 } 548 549 //Follow group by 550 if err := performRenameColRequestOnHistogram(nodeResult, letColReq); err != nil { 551 return fmt.Errorf("performRenameColRequest: %v", err) 552 } 553 if err := performRenameColRequestOnMeasureResults(nodeResult, letColReq); err != nil { 554 return fmt.Errorf("performRenameColRequest: %v", err) 555 } 556 557 return nil 558 } 559 560 func performRenameColRequestWithoutGroupby(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 561 562 fieldsToAdd := make([]string, 0) 563 fieldsToRemove := make([]string, 0) 564 565 switch letColReq.RenameColRequest.RenameExprMode { 566 case structs.REMPhrase: 567 fallthrough 568 case structs.REMOverride: 569 570 // Suppose you rename fieldA to fieldB, but fieldA does not exist. 571 // If fieldB does not exist, nothing happens. 572 // If fieldB does exist, the result of the rename is that the data in fieldB is removed. The data in fieldB will contain null values. 573 if _, exist := finalCols[letColReq.RenameColRequest.OriginalPattern]; !exist { 574 if _, exist := finalCols[letColReq.RenameColRequest.NewPattern]; !exist { 575 return nil 576 } 577 } 578 579 fieldsToAdd = append(fieldsToAdd, letColReq.RenameColRequest.NewPattern) 580 fieldsToRemove = append(fieldsToRemove, letColReq.RenameColRequest.OriginalPattern) 581 case structs.REMRegex: 582 for colName := range finalCols { 583 newColName, err := letColReq.RenameColRequest.ProcessRenameRegexExpression(colName) 584 if err != nil { 585 return fmt.Errorf("performRenameColRequestWithoutGroupby: %v", err) 586 } 587 if len(newColName) == 0 { 588 continue 589 } 590 fieldsToAdd = append(fieldsToAdd, newColName) 591 fieldsToRemove = append(fieldsToRemove, colName) 592 } 593 default: 594 return fmt.Errorf("performRenameColRequestWithoutGroupby: RenameColRequest has an unexpected type") 595 } 596 597 for _, record := range recs { 598 for index, newColName := range fieldsToAdd { 599 record[newColName] = record[fieldsToRemove[index]] 600 } 601 } 602 for index, newColName := range fieldsToAdd { 603 finalCols[newColName] = true 604 delete(finalCols, fieldsToRemove[index]) 605 } 606 607 return nil 608 } 609 610 func performRenameColRequestOnHistogram(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 611 612 for _, aggregationResult := range nodeResult.Histogram { 613 for _, bucketResult := range aggregationResult.Results { 614 switch letColReq.RenameColRequest.RenameExprMode { 615 case structs.REMPhrase: 616 fallthrough 617 case structs.REMOverride: 618 619 // The original pattern should be a field, and the field may come from GroupByCol or the Stat Res. The same rule applies to the new pattern 620 // We should delete new pattern key-val pair, and override the original field to new col name 621 622 // If new pattern comes from GroupByCols, we should delete it in the GroupByCols 623 for index, groupByCol := range bucketResult.GroupByKeys { 624 if groupByCol == letColReq.RenameColRequest.NewPattern { 625 letColReq.RenameColRequest.RemoveBucketResGroupByColumnsByIndex(bucketResult, []int{index}) 626 break 627 } 628 } 629 630 // If new pattern comes from Stat Res, its key-value pair will be deleted 631 delete(bucketResult.StatRes, letColReq.RenameColRequest.NewPattern) 632 633 // After delete new pattern in GroupByCols or Stat Res, we should override the name of original field to new field 634 635 // If original pattern comes from Stat Res 636 val, exists := bucketResult.StatRes[letColReq.RenameColRequest.OriginalPattern] 637 if exists { 638 bucketResult.StatRes[letColReq.RenameColRequest.NewPattern] = val 639 delete(bucketResult.StatRes, letColReq.RenameColRequest.OriginalPattern) 640 continue 641 } 642 643 // If original pattern comes from GroupByCol, just override its name 644 for index, groupByCol := range bucketResult.GroupByKeys { 645 if letColReq.RenameColRequest.OriginalPattern == groupByCol { 646 // The GroupByKeys in the aggregationResult.Results array is a reference slice. 647 // If we just modify GroupByKeys in one bucket, the GroupByKeys in other buckets will also be updated 648 groupByKeys := make([]string, len(bucketResult.GroupByKeys)) 649 copy(groupByKeys, bucketResult.GroupByKeys) 650 groupByKeys[index] = letColReq.RenameColRequest.NewPattern 651 bucketResult.GroupByKeys = groupByKeys 652 break 653 } 654 } 655 656 case structs.REMRegex: 657 658 // If we override original field to a new field, we should remove new field key-val pair and just modify the key name of original field to new field 659 //Rename statistic functions name 660 for statColName, val := range bucketResult.StatRes { 661 newColName, err := letColReq.RenameColRequest.ProcessRenameRegexExpression(statColName) 662 663 if err != nil { 664 return fmt.Errorf("performRenameColRequestOnHistogram: %v", err) 665 } 666 if len(newColName) == 0 { 667 continue 668 } 669 bucketResult.StatRes[newColName] = val 670 delete(bucketResult.StatRes, statColName) 671 } 672 673 indexToRemove := make([]int, 0) 674 //Rename Group by column name 675 for index, groupByColName := range bucketResult.GroupByKeys { 676 newColName, err := letColReq.RenameColRequest.ProcessRenameRegexExpression(groupByColName) 677 if err != nil { 678 return fmt.Errorf("performRenameColRequestOnHistogram: %v", err) 679 } 680 if len(newColName) == 0 { 681 continue 682 } 683 684 for i, groupByCol := range bucketResult.GroupByKeys { 685 if groupByCol == newColName { 686 indexToRemove = append(indexToRemove, i) 687 break 688 } 689 } 690 691 groupByKeys := make([]string, len(bucketResult.GroupByKeys)) 692 copy(groupByKeys, bucketResult.GroupByKeys) 693 groupByKeys[index] = newColName 694 bucketResult.GroupByKeys = groupByKeys 695 } 696 697 letColReq.RenameColRequest.RemoveBucketResGroupByColumnsByIndex(bucketResult, indexToRemove) 698 699 default: 700 return fmt.Errorf("performRenameColRequestOnHistogram: RenameColRequest has an unexpected type") 701 } 702 } 703 } 704 705 return nil 706 } 707 708 func performRenameColRequestOnMeasureResults(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 709 710 // Compute the value for each row. 711 for _, bucketHolder := range nodeResult.MeasureResults { 712 713 switch letColReq.RenameColRequest.RenameExprMode { 714 case structs.REMPhrase: 715 fallthrough 716 case structs.REMOverride: 717 718 // If new pattern comes from GroupByCols, we should delete it in the GroupByCols 719 for index, groupByCol := range nodeResult.GroupByCols { 720 if groupByCol == letColReq.RenameColRequest.NewPattern { 721 letColReq.RenameColRequest.RemoveBucketHolderGroupByColumnsByIndex(bucketHolder, nodeResult.GroupByCols, []int{index}) 722 break 723 } 724 } 725 726 // If new pattern comes from Stat Res, its key-value pair will be deleted 727 delete(bucketHolder.MeasureVal, letColReq.RenameColRequest.NewPattern) 728 729 // After delete new pattern in GroupByCols or MeasureVal, we should override the name of original field to new field 730 731 // If original pattern comes from MeasureVal 732 val, exists := bucketHolder.MeasureVal[letColReq.RenameColRequest.OriginalPattern] 733 if exists { 734 bucketHolder.MeasureVal[letColReq.RenameColRequest.NewPattern] = val 735 delete(bucketHolder.MeasureVal, letColReq.RenameColRequest.OriginalPattern) 736 continue 737 } 738 739 // If original pattern comes from GroupByCol, just override its name 740 // There is no GroupByKeys in bucketHolder, so we can skip this step 741 case structs.REMRegex: 742 743 //Rename MeasurVal name 744 for measureName, val := range bucketHolder.MeasureVal { 745 newColName, err := letColReq.RenameColRequest.ProcessRenameRegexExpression(measureName) 746 if err != nil { 747 return fmt.Errorf("performRenameColRequestOnMeasureResults: %v", err) 748 } 749 if len(newColName) == 0 { 750 continue 751 } 752 // Being able to match indicates that the original field comes from MeasureVal 753 bucketHolder.MeasureVal[newColName] = val 754 delete(bucketHolder.MeasureVal, measureName) 755 } 756 757 indexToRemove := make([]int, 0) 758 //Rename Group by column name 759 for _, groupByColName := range nodeResult.GroupByCols { 760 newColName, err := letColReq.RenameColRequest.ProcessRenameRegexExpression(groupByColName) 761 if err != nil { 762 return fmt.Errorf("performRenameColRequestOnMeasureResults: %v", err) 763 } 764 if len(newColName) == 0 { 765 continue 766 } 767 768 for i, groupByCol := range nodeResult.GroupByCols { 769 if groupByCol == newColName { 770 indexToRemove = append(indexToRemove, i) 771 break 772 } 773 } 774 } 775 letColReq.RenameColRequest.RemoveBucketHolderGroupByColumnsByIndex(bucketHolder, nodeResult.GroupByCols, indexToRemove) 776 } 777 } 778 return nil 779 } 780 781 func performDedupColRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, 782 finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) error { 783 // Without following a group by 784 if recs != nil { 785 if err := performDedupColRequestWithoutGroupby(nodeResult, letColReq, recs, finalCols, numTotalSegments, finishesSegment); err != nil { 786 return fmt.Errorf("performDedupColRequest: %v", err) 787 } 788 return nil 789 } 790 791 // Following a group by 792 if err := performDedupColRequestOnHistogram(nodeResult, letColReq); err != nil { 793 return fmt.Errorf("performDedupColRequest: %v", err) 794 } 795 796 // Reset DedupCombinations so we can use it for computing dedup on the 797 // MeasureResults without the deduped records from the Histogram 798 // interfering. 799 // Note that this is only ok because we never again need the dedup buckets 800 // from the Histogram, and this is ok even when there's multiple segments 801 // because this post-processing logic is run on group by data only after 802 // the data from all the segments has been compiled into one NodeResult. 803 letColReq.DedupColRequest.DedupCombinations = make(map[string]map[int][]structs.SortValue, 0) 804 805 if err := performDedupColRequestOnMeasureResults(nodeResult, letColReq); err != nil { 806 return fmt.Errorf("performDedupColRequest: %v", err) 807 } 808 809 return nil 810 } 811 812 func performDedupColRequestWithoutGroupby(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, 813 finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) error { 814 815 letColReq.DedupColRequest.ProcessedSegmentsLock.Lock() 816 defer letColReq.DedupColRequest.ProcessedSegmentsLock.Unlock() 817 if finishesSegment { 818 letColReq.DedupColRequest.NumProcessedSegments++ 819 } 820 821 // Keep track of all the matched records across all segments, and only run 822 // the dedup logic once all the records are gathered. 823 if letColReq.DedupColRequest.NumProcessedSegments < numTotalSegments { 824 for k, v := range recs { 825 letColReq.DedupColRequest.DedupRecords[k] = v 826 delete(recs, k) 827 } 828 829 return nil 830 } 831 832 fieldList := letColReq.DedupColRequest.FieldList 833 combinationSlice := make([]interface{}, len(fieldList)) 834 sortbyValues := make([]structs.SortValue, len(letColReq.DedupColRequest.DedupSortEles)) 835 sortbyFields := make([]string, len(letColReq.DedupColRequest.DedupSortEles)) 836 837 for i, sortEle := range letColReq.DedupColRequest.DedupSortEles { 838 sortbyFields[i] = sortEle.Field 839 sortbyValues[i] = structs.SortValue{ 840 InterpretAs: sortEle.Op, 841 } 842 } 843 844 for k, v := range letColReq.DedupColRequest.DedupRecords { 845 recs[k] = v 846 } 847 848 recsIndexToKey := make([]string, len(recs)) 849 recsIndex := 0 850 for key, record := range recs { 851 // Initialize combination for current row 852 for index, field := range fieldList { 853 val, exists := record[field] 854 if !exists { 855 combinationSlice[index] = nil 856 } else { 857 combinationSlice[index] = val 858 } 859 } 860 861 for i, field := range sortbyFields { 862 val, exists := record[field] 863 if !exists { 864 val = nil 865 } 866 867 sortbyValues[i].Val = fmt.Sprintf("%v", val) 868 } 869 870 passes, evictionIndex, err := combinationPassesDedup(combinationSlice, recsIndex, sortbyValues, letColReq.DedupColRequest) 871 if err != nil { 872 return fmt.Errorf("performDedupColRequestWithoutGroupby: %v", err) 873 } 874 875 if evictionIndex != -1 { 876 // Evict the item at evictionIndex. 877 delete(recs, recsIndexToKey[evictionIndex]) 878 } 879 880 if !passes { 881 if !letColReq.DedupColRequest.DedupOptions.KeepEvents { 882 delete(recs, key) 883 } else { 884 // Keep this record, but clear all the values for the fieldList fields. 885 for _, field := range fieldList { 886 if _, exists := record[field]; exists { 887 record[field] = nil 888 } 889 } 890 } 891 } 892 893 recsIndexToKey[recsIndex] = key 894 recsIndex++ 895 } 896 897 return nil 898 } 899 900 func performDedupColRequestOnHistogram(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 901 fieldList := letColReq.DedupColRequest.FieldList 902 dedupRawValues := make(map[string]segutils.CValueEnclosure, len(fieldList)) 903 combinationSlice := make([]interface{}, len(fieldList)) 904 sortbyRawValues := make(map[string]segutils.CValueEnclosure, len(letColReq.DedupColRequest.DedupSortEles)) 905 sortbyValues := make([]structs.SortValue, len(letColReq.DedupColRequest.DedupSortEles)) 906 sortbyFields := make([]string, len(letColReq.DedupColRequest.DedupSortEles)) 907 908 for i, sortEle := range letColReq.DedupColRequest.DedupSortEles { 909 sortbyFields[i] = sortEle.Field 910 sortbyValues[i] = structs.SortValue{ 911 InterpretAs: sortEle.Op, 912 } 913 } 914 915 for _, aggregationResult := range nodeResult.Histogram { 916 newResults := make([]*structs.BucketResult, 0) 917 evictedFromNewResults := make([]bool, 0) // Only used when dedup has a sortby. 918 numEvicted := 0 919 920 for bucketIndex, bucketResult := range aggregationResult.Results { 921 err := getAggregationResultFieldValues(dedupRawValues, fieldList, aggregationResult, bucketIndex) 922 if err != nil { 923 return fmt.Errorf("performDedupColRequestOnHistogram: error getting dedup values: %v", err) 924 } 925 926 for i, field := range fieldList { 927 combinationSlice[i] = dedupRawValues[field] 928 } 929 930 // If the dedup has a sortby, get the sort values. 931 if len(letColReq.DedupColRequest.DedupSortEles) > 0 { 932 err = getAggregationResultFieldValues(sortbyRawValues, sortbyFields, aggregationResult, bucketIndex) 933 if err != nil { 934 return fmt.Errorf("performDedupColRequestOnHistogram: error getting sort values: %v", err) 935 } 936 937 for i, field := range sortbyFields { 938 enclosure := sortbyRawValues[field] 939 sortbyValues[i].Val, err = enclosure.GetString() 940 if err != nil { 941 return fmt.Errorf("performDedupColRequestOnHistogram: error converting sort values: %v", err) 942 } 943 } 944 } 945 946 recordIndex := len(newResults) 947 passes, evictionIndex, err := combinationPassesDedup(combinationSlice, recordIndex, sortbyValues, letColReq.DedupColRequest) 948 if err != nil { 949 return fmt.Errorf("performDedupColRequestOnHistogram: %v", err) 950 } 951 952 if evictionIndex != -1 { 953 // Evict the item at evictionIndex. 954 evictedFromNewResults[evictionIndex] = true 955 numEvicted++ 956 } 957 958 if passes { 959 newResults = append(newResults, bucketResult) 960 evictedFromNewResults = append(evictedFromNewResults, false) 961 } else if letColReq.DedupColRequest.DedupOptions.KeepEvents { 962 // Keep this bucketResult, but clear all the values for the fieldList fields. 963 964 // Decode the bucketKey into a slice of strings. 965 var bucketKeySlice []string 966 switch bucketKey := bucketResult.BucketKey.(type) { 967 case []string: 968 bucketKeySlice = bucketKey 969 case string: 970 bucketKeySlice = []string{bucketKey} 971 default: 972 return fmt.Errorf("performDedupColRequestOnHistogram: unexpected type for bucketKey %v", bucketKey) 973 } 974 975 for _, field := range fieldList { 976 if _, exists := bucketResult.StatRes[field]; exists { 977 bucketResult.StatRes[field] = segutils.CValueEnclosure{ 978 Dtype: segutils.SS_DT_BACKFILL, 979 } 980 } else { 981 for i, groupByCol := range bucketResult.GroupByKeys { 982 if groupByCol == field { 983 bucketKeySlice[i] = "" 984 break 985 } 986 } 987 } 988 } 989 990 // Set the bucketKey. 991 if len(bucketKeySlice) == 1 { 992 bucketResult.BucketKey = bucketKeySlice[0] 993 } else { 994 bucketResult.BucketKey = bucketKeySlice 995 } 996 997 newResults = append(newResults, bucketResult) 998 evictedFromNewResults = append(evictedFromNewResults, false) 999 } 1000 } 1001 1002 // Get the final results by removing the evicted items. 1003 finalResults := make([]*structs.BucketResult, len(newResults)-numEvicted) 1004 finalResultsIndex := 0 1005 for i, bucketResult := range newResults { 1006 if !evictedFromNewResults[i] { 1007 finalResults[finalResultsIndex] = bucketResult 1008 finalResultsIndex++ 1009 } 1010 } 1011 1012 aggregationResult.Results = finalResults 1013 } 1014 1015 return nil 1016 } 1017 1018 func performDedupColRequestOnMeasureResults(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1019 fieldList := letColReq.DedupColRequest.FieldList 1020 dedupRawValues := make(map[string]segutils.CValueEnclosure, len(fieldList)) 1021 combinationSlice := make([]interface{}, len(fieldList)) 1022 sortbyRawValues := make(map[string]segutils.CValueEnclosure, len(letColReq.DedupColRequest.DedupSortEles)) 1023 sortbyValues := make([]structs.SortValue, len(letColReq.DedupColRequest.DedupSortEles)) 1024 sortbyFields := make([]string, len(letColReq.DedupColRequest.DedupSortEles)) 1025 1026 for i, sortEle := range letColReq.DedupColRequest.DedupSortEles { 1027 sortbyFields[i] = sortEle.Field 1028 sortbyValues[i] = structs.SortValue{ 1029 InterpretAs: sortEle.Op, 1030 } 1031 } 1032 1033 newResults := make([]*structs.BucketHolder, 0) 1034 evictedFromNewResults := make([]bool, 0) // Only used when dedup has a sortby. 1035 numEvicted := 0 1036 1037 for bucketIndex, bucketHolder := range nodeResult.MeasureResults { 1038 err := getMeasureResultsFieldValues(dedupRawValues, fieldList, nodeResult, bucketIndex) 1039 if err != nil { 1040 return fmt.Errorf("performDedupColRequestOnMeasureResults: error getting dedup values: %v", err) 1041 } 1042 1043 for i, field := range fieldList { 1044 combinationSlice[i] = dedupRawValues[field] 1045 } 1046 1047 // If the dedup has a sortby, get the sort values. 1048 if len(letColReq.DedupColRequest.DedupSortEles) > 0 { 1049 err = getMeasureResultsFieldValues(sortbyRawValues, sortbyFields, nodeResult, bucketIndex) 1050 if err != nil { 1051 return fmt.Errorf("performDedupColRequestOnMeasureResults: error getting sort values: %v", err) 1052 } 1053 1054 for i, field := range sortbyFields { 1055 enclosure := sortbyRawValues[field] 1056 sortbyValues[i].Val, err = enclosure.GetString() 1057 if err != nil { 1058 return fmt.Errorf("performDedupColRequestOnMeasureResults: error converting sort values: %v", err) 1059 } 1060 } 1061 } 1062 1063 recordIndex := len(newResults) 1064 passes, evictionIndex, err := combinationPassesDedup(combinationSlice, recordIndex, sortbyValues, letColReq.DedupColRequest) 1065 if err != nil { 1066 return fmt.Errorf("performDedupColRequestOnMeasureResults: %v", err) 1067 } 1068 1069 if evictionIndex != -1 { 1070 // Evict the item at evictionIndex. 1071 evictedFromNewResults[evictionIndex] = true 1072 numEvicted++ 1073 } 1074 1075 if passes { 1076 newResults = append(newResults, bucketHolder) 1077 evictedFromNewResults = append(evictedFromNewResults, false) 1078 } else if letColReq.DedupColRequest.DedupOptions.KeepEvents { 1079 // Keep this bucketHolder, but clear all the values for the fieldList fields. 1080 for _, field := range fieldList { 1081 if _, exists := bucketHolder.MeasureVal[field]; exists { 1082 bucketHolder.MeasureVal[field] = nil 1083 } else { 1084 for i, groupByCol := range nodeResult.GroupByCols { 1085 if groupByCol == field { 1086 bucketHolder.GroupByValues[i] = "" 1087 break 1088 } 1089 } 1090 } 1091 } 1092 1093 newResults = append(newResults, bucketHolder) 1094 evictedFromNewResults = append(evictedFromNewResults, false) 1095 } 1096 } 1097 1098 // Get the final results by removing the evicted items. 1099 finalResults := make([]*structs.BucketHolder, len(newResults)-numEvicted) 1100 finalResultsIndex := 0 1101 for i, bucketHolder := range newResults { 1102 if !evictedFromNewResults[i] { 1103 finalResults[finalResultsIndex] = bucketHolder 1104 finalResultsIndex++ 1105 } 1106 } 1107 1108 nodeResult.MeasureResults = finalResults 1109 nodeResult.BucketCount = len(newResults) 1110 1111 return nil 1112 } 1113 1114 // Return whether the combination should be kept, and the index of the record 1115 // that should be evicted if the combination is kept. The returned record index 1116 // is only useful when the dedup has a sortby, and the record index to evict 1117 // will be -1 if nothing should be evicted. 1118 // 1119 // Note: this will update dedupExpr.DedupCombinations if the combination is kept. 1120 // Note: this ignores the dedupExpr.DedupOptions.KeepEvents option; the caller 1121 // is responsible for the extra logic when that is set. 1122 func combinationPassesDedup(combinationSlice []interface{}, recordIndex int, sortValues []structs.SortValue, dedupExpr *structs.DedupExpr) (bool, int, error) { 1123 // If the keepempty option is set, keep every combination will a nil value. 1124 // Otherwise, discard every combination with a nil value. 1125 for _, val := range combinationSlice { 1126 if val == nil { 1127 return dedupExpr.DedupOptions.KeepEmpty, -1, nil 1128 } 1129 } 1130 1131 combinationBytes, err := json.Marshal(combinationSlice) 1132 if err != nil { 1133 return false, -1, fmt.Errorf("checkDedupCombination: failed to marshal combintion %v: %v", combinationSlice, err) 1134 } 1135 1136 combination := string(combinationBytes) 1137 combinations := dedupExpr.DedupCombinations 1138 1139 if dedupExpr.DedupOptions.Consecutive { 1140 // Only remove consecutive duplicates. 1141 passes := combination != dedupExpr.PrevCombination 1142 dedupExpr.PrevCombination = combination 1143 return passes, -1, nil 1144 } 1145 1146 recordsMap, exists := combinations[combination] 1147 if !exists { 1148 recordsMap = make(map[int][]structs.SortValue, 0) 1149 combinations[combination] = recordsMap 1150 } 1151 1152 if !exists || uint64(len(recordsMap)) < dedupExpr.Limit { 1153 sortValuesCopy := make([]structs.SortValue, len(sortValues)) 1154 copy(sortValuesCopy, sortValues) 1155 recordsMap[recordIndex] = sortValuesCopy 1156 1157 return true, -1, nil 1158 } else if len(dedupExpr.DedupSortEles) > 0 { 1159 1160 // Check if this record gets sorted higher than another record with 1161 // this combination, so it should evict the lowest sorted record. 1162 foundLower := false 1163 indexOfLowest := recordIndex 1164 sortValuesOfLowest := sortValues 1165 for index, otherSortValues := range recordsMap { 1166 comparison, err := structs.CompareSortValueSlices(sortValuesOfLowest, otherSortValues, dedupExpr.DedupSortAscending) 1167 if err != nil { 1168 err := fmt.Errorf("checkDedupCombination: failed to compare sort values %v and %v: with ascending %v: %v", 1169 sortValuesOfLowest, otherSortValues, dedupExpr.DedupSortAscending, err) 1170 return false, -1, err 1171 } 1172 1173 if comparison > 0 { 1174 foundLower = true 1175 indexOfLowest = index 1176 sortValuesOfLowest = otherSortValues 1177 } 1178 } 1179 1180 if foundLower { 1181 delete(recordsMap, indexOfLowest) 1182 1183 sortValuesCopy := make([]structs.SortValue, len(sortValues)) 1184 copy(sortValuesCopy, sortValues) 1185 recordsMap[recordIndex] = sortValuesCopy 1186 1187 return true, indexOfLowest, nil 1188 } else { 1189 return false, -1, nil 1190 } 1191 } 1192 1193 return false, -1, nil 1194 } 1195 1196 func performSortColRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, 1197 recordIndexInFinal map[string]int, finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) error { 1198 // Without following a group by 1199 if recs != nil { 1200 if err := performSortColRequestWithoutGroupby(nodeResult, letColReq, recs, recordIndexInFinal, finalCols, numTotalSegments, finishesSegment); err != nil { 1201 return fmt.Errorf("performSortColRequest: %v", err) 1202 } 1203 return nil 1204 } 1205 1206 // Following a group by 1207 if err := performSortColRequestOnHistogram(nodeResult, letColReq); err != nil { 1208 return fmt.Errorf("performSortColRequest: %v", err) 1209 } 1210 1211 if err := performSortColRequestOnMeasureResults(nodeResult, letColReq); err != nil { 1212 return fmt.Errorf("performSortColRequest: %v", err) 1213 } 1214 1215 return nil 1216 } 1217 1218 func performSortColRequestWithoutGroupby(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, 1219 recordIndexInFinal map[string]int, finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) error { 1220 1221 letColReq.SortColRequest.ProcessedSegmentsLock.Lock() 1222 defer letColReq.SortColRequest.ProcessedSegmentsLock.Unlock() 1223 if finishesSegment { 1224 letColReq.SortColRequest.NumProcessedSegments++ 1225 } 1226 1227 if letColReq.SortColRequest.NumProcessedSegments < numTotalSegments { 1228 for k, v := range recs { 1229 letColReq.SortColRequest.SortRecords[k] = v 1230 delete(recs, k) 1231 } 1232 1233 return nil 1234 } 1235 1236 for k, v := range letColReq.SortColRequest.SortRecords { 1237 recs[k] = v 1238 } 1239 1240 recKeys := make([]string, 0) 1241 keyToSortByValues := make(map[string][]structs.SortValue, 0) 1242 for recInden, record := range recs { 1243 recKeys = append(recKeys, recInden) 1244 sortValue := make([]structs.SortValue, len(letColReq.SortColRequest.SortEles)) 1245 for i, sortEle := range letColReq.SortColRequest.SortEles { 1246 val, exists := record[sortEle.Field] 1247 if !exists { 1248 val = nil 1249 } 1250 1251 sortValue[i].Val = fmt.Sprintf("%v", val) 1252 sortValue[i].InterpretAs = sortEle.Op 1253 } 1254 keyToSortByValues[recInden] = sortValue 1255 } 1256 1257 // Sort the recKeys array to ensure that keys with higher priority appear first 1258 sort.Slice(recKeys, func(i, j int) bool { 1259 key1 := recKeys[i] 1260 key2 := recKeys[j] 1261 comparisonRes, err := structs.CompareSortValueSlices(keyToSortByValues[key1], keyToSortByValues[key2], letColReq.SortColRequest.SortAscending) 1262 if err != nil { 1263 return true 1264 } 1265 return comparisonRes == -1 1266 }) 1267 1268 for index, recInden := range recKeys { 1269 recordIndexInFinal[recInden] = index 1270 } 1271 return nil 1272 } 1273 1274 func performSortColRequestOnHistogram(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1275 1276 // Setup a map from each of the fields used in this expression to its value for a certain row. 1277 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 1278 1279 sortbyFields := make([]string, len(letColReq.SortColRequest.SortEles)) 1280 for i, sortEle := range letColReq.SortColRequest.SortEles { 1281 sortbyFields[i] = sortEle.Field 1282 } 1283 1284 for _, aggregationResult := range nodeResult.Histogram { 1285 recKeys := make([]int, 0) 1286 keyToSortByValues := make(map[int][]structs.SortValue, 0) 1287 for rowIndex := range aggregationResult.Results { 1288 recKeys = append(recKeys, rowIndex) 1289 1290 // Get the values of all the necessary fields. 1291 err := getAggregationResultFieldValues(fieldToValue, sortbyFields, aggregationResult, rowIndex) 1292 if err != nil { 1293 return fmt.Errorf("performSortColRequestOnHistogram: %v", err) 1294 } 1295 sortValue := make([]structs.SortValue, len(letColReq.SortColRequest.SortEles)) 1296 for i, sortEle := range letColReq.SortColRequest.SortEles { 1297 enclosure := fieldToValue[sortEle.Field] 1298 sortValue[i].Val, err = enclosure.GetString() 1299 if err != nil { 1300 return fmt.Errorf("performSortColRequestOnHistogram: error converting sort values: %v", err) 1301 } 1302 sortValue[i].InterpretAs = sortEle.Op 1303 } 1304 keyToSortByValues[rowIndex] = sortValue 1305 } 1306 1307 // Sort aggregationResult.Results' keys and map results to the correct order 1308 sort.Slice(recKeys, func(i, j int) bool { 1309 key1 := recKeys[i] 1310 key2 := recKeys[j] 1311 comparisonRes, err := structs.CompareSortValueSlices(keyToSortByValues[key1], keyToSortByValues[key2], letColReq.SortColRequest.SortAscending) 1312 if err != nil { 1313 return true 1314 } 1315 return comparisonRes == -1 1316 }) 1317 1318 resInOrder := make([]*structs.BucketResult, len(aggregationResult.Results)) 1319 for index, key := range recKeys { 1320 resInOrder[index] = aggregationResult.Results[key] 1321 } 1322 1323 aggregationResult.Results = resInOrder 1324 } 1325 1326 return nil 1327 } 1328 1329 func performSortColRequestOnMeasureResults(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1330 1331 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 1332 1333 sortbyFields := make([]string, len(letColReq.SortColRequest.SortEles)) 1334 for i, sortEle := range letColReq.SortColRequest.SortEles { 1335 sortbyFields[i] = sortEle.Field 1336 } 1337 1338 recKeys := make([]int, 0) 1339 keyToSortByValues := make(map[int][]structs.SortValue, 0) 1340 for rowIndex := range nodeResult.MeasureResults { 1341 recKeys = append(recKeys, rowIndex) 1342 1343 // Get the values of all the necessary fields. 1344 err := getMeasureResultsFieldValues(fieldToValue, sortbyFields, nodeResult, rowIndex) 1345 if err != nil { 1346 return fmt.Errorf("performSortColRequestOnMeasureResults: %v", err) 1347 } 1348 1349 sortValue := make([]structs.SortValue, len(letColReq.SortColRequest.SortEles)) 1350 for i, sortEle := range letColReq.SortColRequest.SortEles { 1351 enclosure := fieldToValue[sortEle.Field] 1352 sortValue[i].Val, err = enclosure.GetString() 1353 if err != nil { 1354 return fmt.Errorf("performSortColRequestOnMeasureResults: error converting sort values: %v", err) 1355 } 1356 sortValue[i].InterpretAs = sortEle.Op 1357 } 1358 keyToSortByValues[rowIndex] = sortValue 1359 } 1360 1361 sort.Slice(recKeys, func(i, j int) bool { 1362 key1 := recKeys[i] 1363 key2 := recKeys[j] 1364 comparisonRes, err := structs.CompareSortValueSlices(keyToSortByValues[key1], keyToSortByValues[key2], letColReq.SortColRequest.SortAscending) 1365 if err != nil { 1366 return true 1367 } 1368 return comparisonRes == -1 1369 }) 1370 1371 resInOrder := make([]*structs.BucketHolder, len(nodeResult.MeasureResults)) 1372 for index, key := range recKeys { 1373 resInOrder[index] = nodeResult.MeasureResults[key] 1374 } 1375 1376 nodeResult.MeasureResults = resInOrder 1377 return nil 1378 } 1379 1380 func performStatisticColRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}) error { 1381 1382 if err := performStatisticColRequestOnHistogram(nodeResult, letColReq); err != nil { 1383 return fmt.Errorf("performStatisticColRequest: %v", err) 1384 } 1385 if err := performStatisticColRequestOnMeasureResults(nodeResult, letColReq); err != nil { 1386 return fmt.Errorf("performStatisticColRequest: %v", err) 1387 } 1388 1389 return nil 1390 } 1391 1392 func performStatisticColRequestOnHistogram(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1393 1394 countIsGroupByCol := utils.SliceContainsString(letColReq.StatisticColRequest.GetGroupByCols(), letColReq.StatisticColRequest.StatisticOptions.CountField) 1395 percentIsGroupByCol := utils.SliceContainsString(letColReq.StatisticColRequest.GetGroupByCols(), letColReq.StatisticColRequest.StatisticOptions.PercentField) 1396 1397 for _, aggregationResult := range nodeResult.Histogram { 1398 1399 if len(aggregationResult.Results) == 0 { 1400 continue 1401 } 1402 resTotal := uint64(0) 1403 for _, bucketResult := range aggregationResult.Results { 1404 resTotal += (bucketResult.ElemCount) 1405 } 1406 //Sort results according to requirements 1407 err := letColReq.StatisticColRequest.SortBucketResult(&aggregationResult.Results) 1408 if err != nil { 1409 return fmt.Errorf("performStatisticColRequestOnHistogram: %v", err) 1410 } 1411 //Process bucket result 1412 otherCnt := resTotal 1413 for _, bucketResult := range aggregationResult.Results { 1414 1415 countName := "count(*)" 1416 newCountName, exists := nodeResult.RenameColumns["count(*)"] 1417 if exists { 1418 countName = newCountName 1419 } 1420 countIsStatisticGroupByCol := utils.SliceContainsString(letColReq.StatisticColRequest.GetGroupByCols(), countName) 1421 //Delete count generated by the stats groupby block 1422 if !countIsStatisticGroupByCol { 1423 delete(bucketResult.StatRes, countName) 1424 } 1425 1426 //Delete fields not in statistic expr 1427 err := letColReq.StatisticColRequest.RemoveFieldsNotInExprForBucketRes(bucketResult) 1428 if err != nil { 1429 return fmt.Errorf("performStatisticColRequestOnHistogram: %v", err) 1430 } 1431 1432 otherCnt -= (bucketResult.ElemCount) 1433 1434 // Set the appropriate column to the computed value 1435 if countIsGroupByCol || percentIsGroupByCol { 1436 err := letColReq.StatisticColRequest.OverrideGroupByCol(bucketResult, resTotal) 1437 if err != nil { 1438 return fmt.Errorf("performStatisticColRequestOnHistogram: %v", err) 1439 } 1440 } 1441 1442 if letColReq.StatisticColRequest.StatisticOptions.ShowCount && !countIsGroupByCol { 1443 //Set Count to StatResult 1444 letColReq.StatisticColRequest.SetCountToStatRes(bucketResult.StatRes, bucketResult.ElemCount) 1445 } 1446 1447 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc && !percentIsGroupByCol { 1448 //Set Percent to StatResult 1449 letColReq.StatisticColRequest.SetPercToStatRes(bucketResult.StatRes, bucketResult.ElemCount, resTotal) 1450 } 1451 } 1452 1453 //If useother=true, a row representing all other values is added to the results. 1454 if letColReq.StatisticColRequest.StatisticOptions.UseOther { 1455 statRes := make(map[string]segutils.CValueEnclosure) 1456 groupByKeys := aggregationResult.Results[0].GroupByKeys 1457 bucketKey := make([]string, len(groupByKeys)) 1458 otherEnclosure := segutils.CValueEnclosure{ 1459 Dtype: segutils.SS_DT_STRING, 1460 CVal: letColReq.StatisticColRequest.StatisticOptions.OtherStr, 1461 } 1462 for i := 0; i < len(groupByKeys); i++ { 1463 if groupByKeys[i] == letColReq.StatisticColRequest.StatisticOptions.CountField || groupByKeys[i] == letColReq.StatisticColRequest.StatisticOptions.PercentField { 1464 continue 1465 } 1466 bucketKey[i] = letColReq.StatisticColRequest.StatisticOptions.OtherStr 1467 } 1468 1469 for key := range aggregationResult.Results[0].StatRes { 1470 if key == letColReq.StatisticColRequest.StatisticOptions.CountField || key == letColReq.StatisticColRequest.StatisticOptions.PercentField { 1471 continue 1472 } 1473 statRes[key] = otherEnclosure 1474 } 1475 1476 otherBucketRes := &structs.BucketResult{ 1477 ElemCount: otherCnt, 1478 StatRes: statRes, 1479 BucketKey: bucketKey, 1480 GroupByKeys: groupByKeys, 1481 } 1482 1483 if countIsGroupByCol || percentIsGroupByCol { 1484 err := letColReq.StatisticColRequest.OverrideGroupByCol(otherBucketRes, resTotal) 1485 if err != nil { 1486 return fmt.Errorf("performStatisticColRequestOnHistogram: %v", err) 1487 } 1488 } 1489 1490 if letColReq.StatisticColRequest.StatisticOptions.ShowCount && !countIsGroupByCol { 1491 letColReq.StatisticColRequest.SetCountToStatRes(statRes, otherCnt) 1492 } 1493 1494 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc && !percentIsGroupByCol { 1495 letColReq.StatisticColRequest.SetPercToStatRes(statRes, otherCnt, resTotal) 1496 } 1497 1498 aggregationResult.Results = append(aggregationResult.Results, otherBucketRes) 1499 } 1500 } 1501 1502 return nil 1503 } 1504 1505 func performStatisticColRequestOnMeasureResults(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1506 1507 // Because the position of GroupByVals inside the bucketholder is related to nodeResult.GroupByCols 1508 // If there is a stats groupby block before the statistic block, that mapping relationship is based on the stats groupby cols 1509 // So we should update it 1510 preGroupByColToIndex := make(map[string]int, len(nodeResult.GroupByCols)) 1511 for index, groupByCol := range nodeResult.GroupByCols { 1512 preGroupByColToIndex[groupByCol] = index 1513 } 1514 1515 var countIsGroupByCol, percentIsGroupByCol bool 1516 countColIndex := -1 1517 percentColIndex := -1 1518 for i, measureCol := range nodeResult.MeasureFunctions { 1519 if letColReq.StatisticColRequest.StatisticOptions.ShowCount && letColReq.StatisticColRequest.StatisticOptions.CountField == measureCol { 1520 // We'll write over this existing column. 1521 countIsGroupByCol = false 1522 countColIndex = i 1523 } 1524 1525 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc && letColReq.StatisticColRequest.StatisticOptions.PercentField == measureCol { 1526 // We'll write over this existing column. 1527 percentIsGroupByCol = false 1528 percentColIndex = i 1529 } 1530 } 1531 1532 for i, groupByCol := range nodeResult.GroupByCols { 1533 if letColReq.StatisticColRequest.StatisticOptions.ShowCount && letColReq.StatisticColRequest.StatisticOptions.CountField == groupByCol { 1534 // We'll write over this existing column. 1535 countIsGroupByCol = true 1536 countColIndex = i 1537 } 1538 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc && letColReq.StatisticColRequest.StatisticOptions.PercentField == groupByCol { 1539 // We'll write over this existing column. 1540 percentIsGroupByCol = true 1541 percentColIndex = i 1542 } 1543 } 1544 1545 if letColReq.StatisticColRequest.StatisticOptions.ShowCount && countColIndex == -1 { 1546 nodeResult.MeasureFunctions = append(nodeResult.MeasureFunctions, letColReq.StatisticColRequest.StatisticOptions.CountField) 1547 } 1548 1549 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc && percentColIndex == -1 { 1550 nodeResult.MeasureFunctions = append(nodeResult.MeasureFunctions, letColReq.StatisticColRequest.StatisticOptions.PercentField) 1551 } 1552 1553 countName := "count(*)" 1554 newCountName, exists := nodeResult.RenameColumns["count(*)"] 1555 if exists { 1556 countName = newCountName 1557 } 1558 1559 resTotal := uint64(0) 1560 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc { 1561 for _, bucketHolder := range nodeResult.MeasureResults { 1562 resTotal += bucketHolder.MeasureVal[countName].(uint64) 1563 } 1564 } 1565 1566 statisticGroupByCols := letColReq.StatisticColRequest.GetGroupByCols() 1567 // Compute the value for each row. 1568 for _, bucketHolder := range nodeResult.MeasureResults { 1569 1570 countVal := bucketHolder.MeasureVal[countName] 1571 1572 if letColReq.StatisticColRequest.StatisticOptions.ShowCount { 1573 // Set the appropriate column to the computed value. 1574 if countIsGroupByCol { 1575 count, ok := countVal.(uint64) 1576 if !ok { 1577 return fmt.Errorf("performStatisticColRequestOnMeasureResults: Can not convert count to uint64") 1578 } 1579 bucketHolder.GroupByValues[countColIndex] = strconv.FormatUint(count, 10) 1580 } else { 1581 bucketHolder.MeasureVal[letColReq.StatisticColRequest.StatisticOptions.CountField] = countVal 1582 } 1583 } 1584 1585 //Delete count generated by the stats groupby block 1586 countIsStatisticGroupByCol := utils.SliceContainsString(letColReq.StatisticColRequest.GetGroupByCols(), countName) 1587 if !countIsStatisticGroupByCol { 1588 delete(bucketHolder.MeasureVal, countName) 1589 } 1590 1591 if letColReq.StatisticColRequest.StatisticOptions.ShowPerc { 1592 count, ok := countVal.(uint64) 1593 if !ok { 1594 return fmt.Errorf("performStatisticColRequestOnMeasureResults: Can not convert count to uint64") 1595 } 1596 percent := float64(count) / float64(resTotal) * 100 1597 if percentIsGroupByCol { 1598 bucketHolder.GroupByValues[percentColIndex] = fmt.Sprintf("%.6f", percent) 1599 } else { 1600 bucketHolder.MeasureVal[letColReq.StatisticColRequest.StatisticOptions.PercentField] = fmt.Sprintf("%.6f", percent) 1601 } 1602 } 1603 1604 //Put groupByVals to the correct position 1605 groupByVals := make([]string, 0) 1606 for i := 0; i < len(statisticGroupByCols); i++ { 1607 colName := statisticGroupByCols[i] 1608 val, exists := bucketHolder.MeasureVal[colName] 1609 if exists { 1610 str := "" 1611 switch v := val.(type) { 1612 case string: 1613 str = v 1614 case []byte: 1615 str = string(v) 1616 } 1617 groupByVals = append(groupByVals, str) 1618 continue 1619 } 1620 index, exists := preGroupByColToIndex[colName] 1621 if exists { 1622 groupByVals = append(groupByVals, bucketHolder.GroupByValues[index]) 1623 } 1624 } 1625 bucketHolder.GroupByValues = groupByVals 1626 } 1627 return nil 1628 } 1629 1630 func performRexColRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 1631 1632 //Without following group by 1633 if recs != nil { 1634 if err := performRexColRequestWithoutGroupby(nodeResult, letColReq, recs, finalCols); err != nil { 1635 return fmt.Errorf("performRexColRequest: %v", err) 1636 } 1637 return nil 1638 } 1639 1640 //Follow group by 1641 if err := performRexColRequestOnHistogram(nodeResult, letColReq); err != nil { 1642 return fmt.Errorf("performRexColRequest: %v", err) 1643 } 1644 if err := performRexColRequestOnMeasureResults(nodeResult, letColReq); err != nil { 1645 return fmt.Errorf("performRexColRequest: %v", err) 1646 } 1647 1648 return nil 1649 } 1650 1651 func performRexColRequestWithoutGroupby(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 1652 1653 rexExp, err := regexp.Compile(letColReq.RexColRequest.Pattern) 1654 if err != nil { 1655 return fmt.Errorf("performRexColRequestWithoutGroupby: There are some errors in the pattern: %v", err) 1656 } 1657 1658 fieldName := letColReq.RexColRequest.FieldName 1659 for _, record := range recs { 1660 fieldValue := fmt.Sprintf("%v", record[fieldName]) 1661 if len(fieldValue) == 0 { 1662 return fmt.Errorf("performRexColRequestWithoutGroupby: Field does not exist: %v", fieldName) 1663 } 1664 1665 rexResultMap, err := structs.MatchAndExtractGroups(fieldValue, rexExp) 1666 if err != nil { 1667 log.Errorf("performRexColRequestWithoutGroupby: %v", err) 1668 continue 1669 } 1670 1671 for rexColName, Value := range rexResultMap { 1672 record[rexColName] = Value 1673 } 1674 } 1675 1676 for _, rexColName := range letColReq.RexColRequest.RexColNames { 1677 finalCols[rexColName] = true 1678 } 1679 1680 return nil 1681 } 1682 1683 func performRexColRequestOnHistogram(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1684 1685 rexExp, err := regexp.Compile(letColReq.RexColRequest.Pattern) 1686 if err != nil { 1687 return fmt.Errorf("performRexColRequestOnHistogram: There are some errors in the pattern: %v", err) 1688 } 1689 1690 fieldsInExpr := letColReq.RexColRequest.GetFields() 1691 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 1692 1693 for _, aggregationResult := range nodeResult.Histogram { 1694 for rowIndex, bucketResult := range aggregationResult.Results { 1695 err := getAggregationResultFieldValues(fieldToValue, fieldsInExpr, aggregationResult, rowIndex) 1696 if err != nil { 1697 return err 1698 } 1699 1700 rexColResult, err := letColReq.RexColRequest.Evaluate(fieldToValue, rexExp) 1701 if err != nil { 1702 return err 1703 } 1704 for rexColName, rexColVal := range rexColResult { 1705 // Set the appropriate column to the computed value. 1706 if utils.SliceContainsString(nodeResult.GroupByCols, rexColName) { 1707 for keyIndex, groupByCol := range bucketResult.GroupByKeys { 1708 if rexColName != groupByCol { 1709 continue 1710 } 1711 1712 // Set the appropriate element of BucketKey to cellValueStr. 1713 switch bucketKey := bucketResult.BucketKey.(type) { 1714 case []string: 1715 bucketKey[keyIndex] = rexColVal 1716 bucketResult.BucketKey = bucketKey 1717 case string: 1718 if keyIndex != 0 { 1719 return fmt.Errorf("performRexColRequestOnHistogram: expected keyIndex to be 0, not %v", keyIndex) 1720 } 1721 bucketResult.BucketKey = rexColVal 1722 default: 1723 return fmt.Errorf("performRexColRequestOnHistogram: bucket key has unexpected type: %T", bucketKey) 1724 } 1725 1726 } 1727 } else { 1728 aggregationResult.Results[rowIndex].StatRes[rexColName] = segutils.CValueEnclosure{ 1729 Dtype: segutils.SS_DT_STRING, 1730 CVal: rexColVal, 1731 } 1732 } 1733 } 1734 } 1735 } 1736 1737 return nil 1738 } 1739 1740 func performRexColRequestOnMeasureResults(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1741 // Check if the column already exists. 1742 rexColNameInMeasureCol := make([]string, 0) 1743 rexColNameToGroupByColIndex := make(map[string]int) 1744 rexColNameInExistingField := make(map[string]bool) 1745 1746 for _, rexColName := range letColReq.RexColRequest.RexColNames { 1747 for _, measureCol := range nodeResult.MeasureFunctions { 1748 if rexColName == measureCol { 1749 rexColNameInMeasureCol = append(rexColNameInMeasureCol, rexColName) 1750 rexColNameInExistingField[rexColName] = true 1751 } 1752 } 1753 1754 for i, groupByCol := range nodeResult.GroupByCols { 1755 if rexColName == groupByCol { 1756 rexColNameToGroupByColIndex[rexColName] = i 1757 rexColNameInExistingField[rexColName] = true 1758 } 1759 } 1760 } 1761 1762 //Append new fields which not in groupby or measurecol to MeasureFunctions 1763 for _, rexColName := range letColReq.RexColRequest.RexColNames { 1764 _, exists := rexColNameInExistingField[rexColName] 1765 if exists { 1766 nodeResult.MeasureFunctions = append(nodeResult.MeasureFunctions, rexColName) 1767 } 1768 } 1769 1770 // Setup a map from each of the fields used in this expression to its value for a certain row. 1771 fieldsInExpr := letColReq.RexColRequest.GetFields() 1772 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 1773 rexExp, err := regexp.Compile(letColReq.RexColRequest.Pattern) 1774 if err != nil { 1775 return fmt.Errorf("performRexColRequestOnMeasureResults: There are some errors in the pattern: %v", err) 1776 } 1777 // Compute the value for each row. 1778 for rowIndex, bucketHolder := range nodeResult.MeasureResults { 1779 // Get the values of all the necessary fields. 1780 err := getMeasureResultsFieldValues(fieldToValue, fieldsInExpr, nodeResult, rowIndex) 1781 if err != nil { 1782 return fmt.Errorf("performRexColRequestOnMeasureResults: %v", err) 1783 } 1784 1785 rexColResult, err := letColReq.RexColRequest.Evaluate(fieldToValue, rexExp) 1786 1787 // Evaluate the rex pattern to a value. 1788 if err != nil { 1789 return fmt.Errorf("performRexColRequestOnMeasureResults: %v", err) 1790 } 1791 1792 for rexColName, index := range rexColNameToGroupByColIndex { 1793 bucketHolder.GroupByValues[index] = rexColResult[rexColName] 1794 } 1795 1796 for _, rexColName := range rexColNameInMeasureCol { 1797 bucketHolder.MeasureVal[rexColName] = rexColResult[rexColName] 1798 } 1799 } 1800 return nil 1801 } 1802 1803 func performValueColRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 1804 if recs != nil { 1805 if err := performValueColRequestWithoutGroupBy(nodeResult, letColReq, recs, finalCols); err != nil { 1806 return fmt.Errorf("performValueColRequest: %v", err) 1807 } 1808 return nil 1809 } 1810 1811 if len(nodeResult.AllRecords) > 0 { 1812 return errors.New("performValueColRequest: ValueColRequest is only implemented for aggregation fields") 1813 } 1814 1815 if err := performValueColRequestOnHistogram(nodeResult, letColReq); err != nil { 1816 return fmt.Errorf("performValueColRequest: %v", err) 1817 } 1818 if err := performValueColRequestOnMeasureResults(nodeResult, letColReq); err != nil { 1819 return fmt.Errorf("performValueColRequest: %v", err) 1820 } 1821 1822 return nil 1823 } 1824 1825 func getRecordFieldValues(fieldToValue map[string]segutils.CValueEnclosure, fieldsInExpr []string, record map[string]interface{}) error { 1826 for _, field := range fieldsInExpr { 1827 value, exists := record[field] 1828 if !exists { 1829 return fmt.Errorf("getRecordFieldValues: field %v does not exist in record", field) 1830 } 1831 1832 dVal, err := segutils.CreateDtypeEnclosure(value, 0) 1833 if err != nil { 1834 log.Errorf("failed to create dtype enclosure for field %s, err=%v", field, err) 1835 dVal = &segutils.DtypeEnclosure{Dtype: segutils.SS_DT_STRING, StringVal: fmt.Sprintf("%v", value), StringValBytes: []byte(fmt.Sprintf("%v", value))} 1836 value = fmt.Sprintf("%v", value) 1837 } 1838 1839 fieldToValue[field] = segutils.CValueEnclosure{Dtype: dVal.Dtype, CVal: value} 1840 } 1841 1842 return nil 1843 } 1844 1845 func performValueColRequestWithoutGroupBy(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest, recs map[string]map[string]interface{}, finalCols map[string]bool) error { 1846 fieldsInExpr := letColReq.ValueColRequest.GetFields() 1847 1848 for _, record := range recs { 1849 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 1850 err := getRecordFieldValues(fieldToValue, fieldsInExpr, record) 1851 if err != nil { 1852 log.Errorf("performValueColRequestWithoutGroupBy: %v", err) 1853 continue 1854 } 1855 1856 value, err := performValueColRequestOnRawRecord(letColReq, fieldToValue) 1857 if err != nil { 1858 log.Errorf("performValueColRequestWithoutGroupBy: %v", err) 1859 continue 1860 } 1861 1862 record[letColReq.NewColName] = value 1863 finalCols[letColReq.NewColName] = true 1864 } 1865 1866 return nil 1867 } 1868 1869 func performValueColRequestOnRawRecord(letColReq *structs.LetColumnsRequest, fieldToValue map[string]segutils.CValueEnclosure) (interface{}, error) { 1870 if letColReq == nil || letColReq.ValueColRequest == nil { 1871 return nil, fmt.Errorf("invalid letColReq") 1872 } 1873 1874 switch letColReq.ValueColRequest.ValueExprMode { 1875 case structs.VEMConditionExpr: 1876 value, err := letColReq.ValueColRequest.ConditionExpr.EvaluateCondition(fieldToValue) 1877 if err != nil { 1878 log.Errorf("failed to evaluate condition expr, err=%v", err) 1879 return nil, err 1880 } 1881 return value, nil 1882 case structs.VEMStringExpr: 1883 value, err := letColReq.ValueColRequest.EvaluateValueExprAsString(fieldToValue) 1884 if err != nil { 1885 log.Errorf("failed to evaluate string expr, err=%v", err) 1886 return nil, err 1887 } 1888 return value, nil 1889 case structs.VEMNumericExpr: 1890 value, err := letColReq.ValueColRequest.EvaluateToFloat(fieldToValue) 1891 if err != nil { 1892 log.Errorf("failed to evaluate numeric expr, err=%v", err) 1893 return nil, err 1894 } 1895 return value, nil 1896 case structs.VEMBooleanExpr: 1897 value, err := letColReq.ValueColRequest.EvaluateToString(fieldToValue) 1898 if err != nil { 1899 log.Errorf(" failed to evaluate boolean expr, err=%v", err) 1900 return nil, err 1901 } 1902 return value, nil 1903 default: 1904 return nil, fmt.Errorf("unknown value expr mode %v", letColReq.ValueColRequest.ValueExprMode) 1905 } 1906 } 1907 1908 func performValueColRequestOnHistogram(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 1909 // Check if the column to create already exists and is a GroupBy column. 1910 isGroupByCol := utils.SliceContainsString(nodeResult.GroupByCols, letColReq.NewColName) 1911 1912 // Setup a map from each of the fields used in this expression to its value for a certain row. 1913 fieldsInExpr := letColReq.ValueColRequest.GetFields() 1914 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 1915 1916 for _, aggregationResult := range nodeResult.Histogram { 1917 for rowIndex, bucketResult := range aggregationResult.Results { 1918 // Get the values of all the necessary fields. 1919 err := getAggregationResultFieldValues(fieldToValue, fieldsInExpr, aggregationResult, rowIndex) 1920 if err != nil { 1921 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1922 } 1923 1924 // Evaluate the expression to a value. We do not know this expression represent a number or str 1925 //Firstly, try to evaluate it as a float, if it fail. Try to evaluate it as a str 1926 var cellValueStr string 1927 var cellValueFloat float64 1928 switch letColReq.ValueColRequest.ValueExprMode { 1929 case structs.VEMConditionExpr: 1930 err := getAggregationResultFieldValues(fieldToValue, fieldsInExpr, aggregationResult, rowIndex) 1931 if err != nil { 1932 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1933 } 1934 // Evaluate the condition expression to a value. 1935 cellValueStr, err = letColReq.ValueColRequest.ConditionExpr.EvaluateCondition(fieldToValue) 1936 if err != nil { 1937 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1938 } 1939 case structs.VEMStringExpr: 1940 cellValueStr, err = letColReq.ValueColRequest.EvaluateValueExprAsString(fieldToValue) 1941 if err != nil { 1942 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1943 } 1944 case structs.VEMNumericExpr: 1945 cellValueFloat, err = letColReq.ValueColRequest.EvaluateToFloat(fieldToValue) 1946 if err != nil { 1947 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1948 } 1949 case structs.VEMBooleanExpr: 1950 cellValueStr, err = letColReq.ValueColRequest.EvaluateToString(fieldToValue) 1951 if err != nil { 1952 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1953 } 1954 } 1955 1956 if err != nil { 1957 return fmt.Errorf("performValueColRequestOnHistogram: %v", err) 1958 } 1959 1960 // Set the appropriate column to the computed value. 1961 if isGroupByCol { 1962 for keyIndex, groupByCol := range bucketResult.GroupByKeys { 1963 if letColReq.NewColName != groupByCol { 1964 continue 1965 } 1966 1967 if len(cellValueStr) == 0 { 1968 cellValueStr = fmt.Sprintf("%v", cellValueFloat) 1969 } 1970 1971 // Set the appropriate element of BucketKey to cellValueStr. 1972 switch bucketKey := bucketResult.BucketKey.(type) { 1973 case []string: 1974 bucketKey[keyIndex] = cellValueStr 1975 bucketResult.BucketKey = bucketKey 1976 case string: 1977 if keyIndex != 0 { 1978 return fmt.Errorf("performValueColRequestOnHistogram: expected keyIndex to be 0, not %v", keyIndex) 1979 } 1980 bucketResult.BucketKey = cellValueStr 1981 default: 1982 return fmt.Errorf("performValueColRequestOnHistogram: bucket key has unexpected type: %T", bucketKey) 1983 } 1984 } 1985 } else { 1986 if len(cellValueStr) > 0 { 1987 aggregationResult.Results[rowIndex].StatRes[letColReq.NewColName] = segutils.CValueEnclosure{ 1988 Dtype: segutils.SS_DT_STRING, 1989 CVal: cellValueStr, 1990 } 1991 } else { 1992 aggregationResult.Results[rowIndex].StatRes[letColReq.NewColName] = segutils.CValueEnclosure{ 1993 Dtype: segutils.SS_DT_FLOAT, 1994 CVal: cellValueFloat, 1995 } 1996 } 1997 } 1998 } 1999 } 2000 2001 return nil 2002 } 2003 2004 func performValueColRequestOnMeasureResults(nodeResult *structs.NodeResult, letColReq *structs.LetColumnsRequest) error { 2005 // Check if the column already exists. 2006 var isGroupByCol bool // If false, it should be a MeasureFunctions column. 2007 colIndex := -1 // Index in GroupByCols or MeasureFunctions. 2008 for i, measureCol := range nodeResult.MeasureFunctions { 2009 if letColReq.NewColName == measureCol { 2010 // We'll write over this existing column. 2011 isGroupByCol = false 2012 colIndex = i 2013 break 2014 } 2015 } 2016 2017 for i, groupByCol := range nodeResult.GroupByCols { 2018 if letColReq.NewColName == groupByCol { 2019 // We'll write over this existing column. 2020 isGroupByCol = true 2021 colIndex = i 2022 break 2023 } 2024 } 2025 2026 if colIndex == -1 { 2027 // Append the column as a MeasureFunctions column. 2028 isGroupByCol = false 2029 colIndex = len(nodeResult.MeasureFunctions) 2030 nodeResult.MeasureFunctions = append(nodeResult.MeasureFunctions, letColReq.NewColName) 2031 } 2032 2033 // Setup a map from each of the fields used in this expression to its value for a certain row. 2034 fieldsInExpr := letColReq.ValueColRequest.GetFields() 2035 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 2036 2037 // Compute the value for each row. 2038 for rowIndex, bucketHolder := range nodeResult.MeasureResults { 2039 // Get the values of all the necessary fields. 2040 err := getMeasureResultsFieldValues(fieldToValue, fieldsInExpr, nodeResult, rowIndex) 2041 if err != nil { 2042 return fmt.Errorf("performValueColRequestOnMeasureResults: %v", err) 2043 } 2044 2045 // Evaluate the expression to a value. 2046 cellValueStr, err := letColReq.ValueColRequest.EvaluateToString(fieldToValue) 2047 if err != nil { 2048 return fmt.Errorf("performValueColRequestOnMeasureResults: %v", err) 2049 } 2050 2051 // Set the appropriate column to the computed value. 2052 if isGroupByCol { 2053 bucketHolder.GroupByValues[colIndex] = cellValueStr 2054 } else { 2055 bucketHolder.MeasureVal[letColReq.NewColName] = cellValueStr 2056 } 2057 } 2058 return nil 2059 } 2060 2061 func performFilterRows(nodeResult *structs.NodeResult, filterRows *structs.BoolExpr) error { 2062 // Ensure all referenced columns are valid. 2063 for _, field := range filterRows.GetFields() { 2064 if !utils.SliceContainsString(nodeResult.GroupByCols, field) && 2065 !utils.SliceContainsString(nodeResult.MeasureFunctions, field) { 2066 2067 return fmt.Errorf("performFilterRows: invalid field: %v", field) 2068 } 2069 } 2070 2071 if err := performFilterRowsOnHistogram(nodeResult, filterRows); err != nil { 2072 return fmt.Errorf("performFilterRows: %v", err) 2073 } 2074 if err := performFilterRowsOnMeasureResults(nodeResult, filterRows); err != nil { 2075 return fmt.Errorf("performFilterRows: %v", err) 2076 } 2077 2078 return nil 2079 } 2080 2081 func performFilterRowsOnHistogram(nodeResult *structs.NodeResult, filterRows *structs.BoolExpr) error { 2082 fieldsInExpr := filterRows.GetFields() 2083 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 2084 2085 for _, aggregationResult := range nodeResult.Histogram { 2086 newResults := make([]*structs.BucketResult, 0, len(aggregationResult.Results)) 2087 2088 for rowIndex, bucketResult := range aggregationResult.Results { 2089 // Get the values of all the necessary fields. 2090 err := getAggregationResultFieldValues(fieldToValue, fieldsInExpr, aggregationResult, rowIndex) 2091 if err != nil { 2092 return fmt.Errorf("performFilterRowsOnHistogram: %v", err) 2093 } 2094 2095 // Evaluate the expression to a value. 2096 shouldKeep, err := filterRows.Evaluate(fieldToValue) 2097 if err != nil { 2098 return fmt.Errorf("performFilterRowsOnHistogram: failed to evaluate condition: %v", err) 2099 } 2100 2101 if shouldKeep { 2102 newResults = append(newResults, bucketResult) 2103 } 2104 } 2105 2106 aggregationResult.Results = newResults 2107 } 2108 2109 return nil 2110 } 2111 2112 func performFilterRowsOnMeasureResults(nodeResult *structs.NodeResult, filterRows *structs.BoolExpr) error { 2113 fieldsInExpr := filterRows.GetFields() 2114 fieldToValue := make(map[string]segutils.CValueEnclosure, 0) 2115 newMeasureResults := make([]*structs.BucketHolder, 0, len(nodeResult.MeasureResults)) 2116 2117 for rowIndex, bucketHolder := range nodeResult.MeasureResults { 2118 // Get the values of all the necessary fields. 2119 err := getMeasureResultsFieldValues(fieldToValue, fieldsInExpr, nodeResult, rowIndex) 2120 if err != nil { 2121 return fmt.Errorf("performFilterRowsOnMeasureResults: %v", err) 2122 } 2123 2124 // Evaluate the expression to a value. 2125 shouldKeep, err := filterRows.Evaluate(fieldToValue) 2126 if err != nil { 2127 return fmt.Errorf("performFilterRowsOnMeasureResults: failed to evaluate condition: %v", err) 2128 } 2129 2130 if shouldKeep { 2131 newMeasureResults = append(newMeasureResults, bucketHolder) 2132 } 2133 } 2134 2135 nodeResult.MeasureResults = newMeasureResults 2136 return nil 2137 } 2138 2139 func getMeasureResultsCell(nodeResult *structs.NodeResult, rowIndex int, col string) (interface{}, bool) { 2140 if value, ok := getMeasureResultsMeasureFunctionCell(nodeResult, rowIndex, col); ok { 2141 return value, true 2142 } 2143 if value, ok := getMeasureResultsGroupByCell(nodeResult, rowIndex, col); ok { 2144 return value, true 2145 } 2146 2147 return nil, false 2148 } 2149 2150 func getMeasureResultsMeasureFunctionCell(nodeResult *structs.NodeResult, rowIndex int, measureCol string) (interface{}, bool) { 2151 value, ok := nodeResult.MeasureResults[rowIndex].MeasureVal[measureCol] 2152 return value, ok 2153 } 2154 2155 func getMeasureResultsGroupByCell(nodeResult *structs.NodeResult, rowIndex int, groupByCol string) (string, bool) { 2156 for i, col := range nodeResult.GroupByCols { 2157 if groupByCol == col { 2158 return nodeResult.MeasureResults[rowIndex].GroupByValues[i], true 2159 } 2160 } 2161 2162 return "", false 2163 } 2164 2165 func getAggregationResultCell(aggResult *structs.AggregationResult, rowIndex int, col string) (interface{}, bool) { 2166 if value, ok := getAggregationResultMeasureFunctionCell(aggResult, rowIndex, col); ok { 2167 return value, true 2168 } 2169 if value, ok := getAggregationResultGroupByCell(aggResult, rowIndex, col); ok { 2170 return value, true 2171 } 2172 2173 return nil, false 2174 } 2175 2176 func getAggregationResultMeasureFunctionCell(aggResult *structs.AggregationResult, rowIndex int, measureCol string) (segutils.CValueEnclosure, bool) { 2177 value, ok := aggResult.Results[rowIndex].StatRes[measureCol] 2178 return value, ok 2179 } 2180 2181 func getAggregationResultGroupByCell(aggResult *structs.AggregationResult, rowIndex int, groupByCol string) (string, bool) { 2182 for keyIndex, groupByKey := range aggResult.Results[rowIndex].GroupByKeys { 2183 if groupByCol != groupByKey { 2184 continue 2185 } 2186 2187 // Index into BucketKey. 2188 switch bucketKey := aggResult.Results[rowIndex].BucketKey.(type) { 2189 case []string: 2190 return bucketKey[keyIndex], true 2191 case string: 2192 if keyIndex != 0 { 2193 log.Errorf("getAggregationResultGroupByCell: expected keyIndex to be 0, not %v", keyIndex) 2194 return "", false 2195 } 2196 return bucketKey, true 2197 default: 2198 log.Errorf("getAggregationResultGroupByCell: bucket key has unexpected type: %T", bucketKey) 2199 return "", false 2200 } 2201 } 2202 2203 return "", false 2204 } 2205 2206 // Replaces values in `fieldToValue` for the specified `fields`, but doesn't 2207 // remove the extra entries in `fieldToValue`. 2208 func getMeasureResultsFieldValues(fieldToValue map[string]segutils.CValueEnclosure, fields []string, 2209 nodeResult *structs.NodeResult, rowIndex int) error { 2210 2211 for _, field := range fields { 2212 var enclosure segutils.CValueEnclosure 2213 2214 value, ok := getMeasureResultsCell(nodeResult, rowIndex, field) 2215 if !ok { 2216 return fmt.Errorf("getMeasureResultsFieldValues: failed to extract field %v from row %v of MeasureResults", field, rowIndex) 2217 } 2218 2219 switch value := value.(type) { 2220 case string: 2221 enclosure.Dtype = segutils.SS_DT_STRING 2222 enclosure.CVal = value 2223 case float64: 2224 enclosure.Dtype = segutils.SS_DT_FLOAT 2225 enclosure.CVal = value 2226 case uint64: 2227 enclosure.Dtype = segutils.SS_DT_UNSIGNED_NUM 2228 enclosure.CVal = value 2229 case int64: 2230 enclosure.Dtype = segutils.SS_DT_SIGNED_NUM 2231 enclosure.CVal = value 2232 default: 2233 return fmt.Errorf("getMeasureResultsFieldValues: expected field to have a string or float value but got %T", value) 2234 } 2235 2236 fieldToValue[field] = enclosure 2237 } 2238 2239 return nil 2240 } 2241 2242 // Replaces values in `fieldToValue` for the specified `fields`, but doesn't 2243 // remove the extra entries in `fieldToValue`. 2244 func getAggregationResultFieldValues(fieldToValue map[string]segutils.CValueEnclosure, fields []string, 2245 aggResult *structs.AggregationResult, rowIndex int) error { 2246 2247 for _, field := range fields { 2248 var enclosure segutils.CValueEnclosure 2249 value, ok := getAggregationResultCell(aggResult, rowIndex, field) 2250 if !ok { 2251 return fmt.Errorf("getAggregationResultFieldValues: failed to extract field %v from row %v of AggregationResult", field, rowIndex) 2252 } 2253 2254 switch value := value.(type) { 2255 case string: 2256 enclosure.Dtype = segutils.SS_DT_STRING 2257 enclosure.CVal = value 2258 case segutils.CValueEnclosure: 2259 enclosure = value 2260 default: 2261 return fmt.Errorf("getAggregationResultFieldValues: expected field to have a string or float value but got %T", value) 2262 } 2263 2264 fieldToValue[field] = enclosure 2265 } 2266 2267 return nil 2268 } 2269 2270 func performTransactionCommandRequest(nodeResult *structs.NodeResult, aggs *structs.QueryAggregators, recs map[string]map[string]interface{}, finalCols map[string]bool, numTotalSegments uint64, finishesSegment bool) { 2271 2272 if recs != nil { 2273 2274 if nodeResult.TransactionEventRecords == nil { 2275 nodeResult.TransactionEventRecords = make(map[string]map[string]interface{}) 2276 } 2277 2278 if nodeResult.TransactionsProcessed == nil { 2279 nodeResult.TransactionsProcessed = make(map[string]map[string]interface{}, 0) 2280 } 2281 2282 if aggs.TransactionArguments.SortedRecordsSlice == nil { 2283 aggs.TransactionArguments.SortedRecordsSlice = make([]map[string]interface{}, 0) 2284 } 2285 2286 for k, v := range recs { 2287 nodeResult.TransactionEventRecords[k] = recs[k] 2288 aggs.TransactionArguments.SortedRecordsSlice = append(aggs.TransactionArguments.SortedRecordsSlice, map[string]interface{}{"key": k, "timestamp": v["timestamp"]}) 2289 delete(recs, k) 2290 } 2291 2292 var cols []string 2293 var err error 2294 2295 if finishesSegment { 2296 nodeResult.RecsAggsProcessedSegments++ 2297 2298 // Sort the records by timestamp. The records in the segment may not be sorted. We need to sort them before processing. 2299 // This method also assumes that all records in the segment will come before the records in the next segment(Segments are Sorted). 2300 sort.Slice(aggs.TransactionArguments.SortedRecordsSlice, func(i, j int) bool { 2301 return aggs.TransactionArguments.SortedRecordsSlice[i]["timestamp"].(uint64) < aggs.TransactionArguments.SortedRecordsSlice[j]["timestamp"].(uint64) 2302 }) 2303 2304 cols, err = processTransactionsOnRecords(nodeResult.TransactionEventRecords, nodeResult.TransactionsProcessed, nil, aggs.TransactionArguments, nodeResult.RecsAggsProcessedSegments == numTotalSegments) 2305 if err != nil { 2306 log.Errorf("performTransactionCommandRequest: %v", err) 2307 return 2308 } 2309 2310 nodeResult.TransactionEventRecords = nil // Clear the transaction records. Release the memory. 2311 nodeResult.TransactionEventRecords = make(map[string]map[string]interface{}) 2312 2313 // Creating a single Map after processing the segment. 2314 // This tells the PostBucketQueryCleaning function to return to the rrcreader.go to process the further segments. 2315 nodeResult.TransactionEventRecords["PROCESSED_SEGMENT_"+fmt.Sprint(nodeResult.RecsAggsProcessedSegments)] = make(map[string]interface{}) 2316 2317 aggs.TransactionArguments.SortedRecordsSlice = nil // Clear the sorted records slice. 2318 } 2319 2320 if nodeResult.RecsAggsProcessedSegments == numTotalSegments { 2321 nodeResult.TransactionEventRecords = nil 2322 nodeResult.TransactionEventRecords = make(map[string]map[string]interface{}) 2323 nodeResult.TransactionEventRecords["CHECK_NEXT_AGG"] = make(map[string]interface{}) // All segments have been processed. Check the next aggregation. 2324 2325 // Clear the Open/Pending Transactions 2326 aggs.TransactionArguments.OpenTransactionEvents = nil 2327 aggs.TransactionArguments.OpenTransactionsState = nil 2328 2329 // Assign the final processed transactions to the recs. 2330 for i, record := range nodeResult.TransactionsProcessed { 2331 recs[i] = record 2332 delete(nodeResult.TransactionsProcessed, i) 2333 } 2334 2335 for k := range finalCols { 2336 delete(finalCols, k) 2337 } 2338 2339 for _, col := range cols { 2340 finalCols[col] = true 2341 } 2342 } 2343 2344 return 2345 2346 } 2347 2348 } 2349 2350 // Evaluate a boolean expression 2351 func evaluateBoolExpr(boolExpr *structs.BoolExpr, record map[string]interface{}) bool { 2352 // Terminal condition 2353 if boolExpr.IsTerminal { 2354 return evaluateSimpleCondition(boolExpr, record) 2355 } 2356 2357 // Recursive evaluation 2358 leftResult := evaluateBoolExpr(boolExpr.LeftBool, record) 2359 rightResult := evaluateBoolExpr(boolExpr.RightBool, record) 2360 2361 // Combine results based on the boolean operation 2362 switch boolExpr.BoolOp { 2363 case structs.BoolOpAnd: 2364 return leftResult && rightResult 2365 case structs.BoolOpOr: 2366 return leftResult || rightResult 2367 default: 2368 // Handle other cases or throw an error 2369 return false 2370 } 2371 } 2372 2373 // Evaluate a simple condition (terminal node) 2374 func evaluateSimpleCondition(term *structs.BoolExpr, record map[string]interface{}) bool { 2375 leftVal, err := getValuesFromValueExpr(term.LeftValue, record) 2376 if err != nil { 2377 return false 2378 } 2379 2380 rightVal, err := getValuesFromValueExpr(term.RightValue, record) 2381 if err != nil { 2382 return false 2383 } 2384 2385 // If the left or right value is nil, return false 2386 if leftVal == nil || rightVal == nil { 2387 return false 2388 } 2389 2390 return conditionMatch(leftVal, term.ValueOp, rightVal) 2391 } 2392 2393 func getValuesFromValueExpr(valueExpr *structs.ValueExpr, record map[string]interface{}) (interface{}, error) { 2394 if valueExpr == nil { 2395 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr is nil") 2396 } 2397 2398 switch valueExpr.ValueExprMode { 2399 case structs.VEMNumericExpr: 2400 if valueExpr.NumericExpr == nil { 2401 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr.NumericExpr is nil") 2402 } 2403 if valueExpr.NumericExpr.ValueIsField { 2404 fieldValue, exists := record[valueExpr.NumericExpr.Value] 2405 if !exists { 2406 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr.NumericExpr.Value does not exist in record") 2407 } 2408 floatFieldVal, err := dtypeutils.ConvertToFloat(fieldValue, 64) 2409 if err != nil { 2410 return fieldValue, nil 2411 } 2412 return floatFieldVal, nil 2413 } else { 2414 floatVal, err := dtypeutils.ConvertToFloat(valueExpr.NumericExpr.Value, 64) 2415 return floatVal, err 2416 } 2417 case structs.VEMStringExpr: 2418 if valueExpr.StringExpr == nil { 2419 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr.StringExpr is nil") 2420 } 2421 switch valueExpr.StringExpr.StringExprMode { 2422 case structs.SEMRawString: 2423 return valueExpr.StringExpr.RawString, nil 2424 case structs.SEMField: 2425 fieldValue, exists := record[valueExpr.StringExpr.FieldName] 2426 if !exists { 2427 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr.StringExpr.Field does not exist in record") 2428 } 2429 return fieldValue, nil 2430 default: 2431 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr.StringExpr.StringExprMode is invalid") 2432 } 2433 default: 2434 return nil, fmt.Errorf("getValuesFromValueExpr: valueExpr.ValueExprMode is invalid") 2435 } 2436 } 2437 2438 func conditionMatch(fieldValue interface{}, Op string, searchValue interface{}) bool { 2439 switch Op { 2440 case "=", "eq": 2441 return fmt.Sprint(fieldValue) == fmt.Sprint(searchValue) 2442 case "!=", "neq": 2443 return fmt.Sprint(fieldValue) != fmt.Sprint(searchValue) 2444 default: 2445 fieldValFloat, err := dtypeutils.ConvertToFloat(fieldValue, 64) 2446 if err != nil { 2447 return false 2448 } 2449 searchValFloat, err := dtypeutils.ConvertToFloat(searchValue, 64) 2450 if err != nil { 2451 return false 2452 } 2453 switch Op { 2454 case ">", "gt": 2455 return fieldValFloat > searchValFloat 2456 case ">=", "gte": 2457 return fieldValFloat >= searchValFloat 2458 case "<", "lt": 2459 return fieldValFloat < searchValFloat 2460 case "<=", "lte": 2461 return fieldValFloat <= searchValFloat 2462 default: 2463 return false 2464 } 2465 } 2466 } 2467 2468 func evaluateASTNode(node *structs.ASTNode, record map[string]interface{}, recordMapStr string) bool { 2469 if node.AndFilterCondition != nil && !evaluateCondition(node.AndFilterCondition, record, recordMapStr, segutils.And) { 2470 return false 2471 } 2472 2473 if node.OrFilterCondition != nil && !evaluateCondition(node.OrFilterCondition, record, recordMapStr, segutils.Or) { 2474 return false 2475 } 2476 2477 // If the node has an exclusion filter, and the exclusion filter matches, return false. 2478 if node.ExclusionFilterCondition != nil && evaluateCondition(node.ExclusionFilterCondition, record, recordMapStr, segutils.Exclusion) { 2479 return false 2480 } 2481 2482 return true 2483 } 2484 2485 func evaluateCondition(condition *structs.Condition, record map[string]interface{}, recordMapStr string, logicalOp segutils.LogicalOperator) bool { 2486 for _, nestedNode := range condition.NestedNodes { 2487 if !evaluateASTNode(nestedNode, record, recordMapStr) { 2488 return false 2489 } 2490 } 2491 2492 for _, criteria := range condition.FilterCriteria { 2493 validMatch := false 2494 if criteria.MatchFilter != nil { 2495 validMatch = evaluateMatchFilter(criteria.MatchFilter, record, recordMapStr) 2496 } else if criteria.ExpressionFilter != nil { 2497 validMatch = evaluateExpressionFilter(criteria.ExpressionFilter, record, recordMapStr) 2498 } 2499 2500 // If the logical operator is Or and at least one of the criteria matches, return true. 2501 if logicalOp == segutils.Or && validMatch { 2502 return true 2503 } else if logicalOp == segutils.And && !validMatch { // If the logical operator is And and at least one of the criteria does not match, return false. 2504 return false 2505 } 2506 } 2507 2508 return logicalOp == segutils.And 2509 } 2510 2511 func evaluateMatchFilter(matchFilter *structs.MatchFilter, record map[string]interface{}, recordMapStr string) bool { 2512 var fieldValue interface{} 2513 var exists bool 2514 2515 if matchFilter.MatchColumn == "*" { 2516 fieldValue = recordMapStr 2517 } else { 2518 fieldValue, exists = record[matchFilter.MatchColumn] 2519 if !exists { 2520 return false 2521 } 2522 } 2523 2524 dVal, err := segutils.CreateDtypeEnclosure(fieldValue, 0) 2525 if err != nil { 2526 return false 2527 } 2528 2529 switch matchFilter.MatchType { 2530 case structs.MATCH_WORDS: 2531 return evaluateMatchWords(matchFilter, dVal.StringVal) 2532 case structs.MATCH_PHRASE: 2533 return evaluateMatchPhrase(string(matchFilter.MatchPhrase), dVal.StringVal) 2534 default: 2535 return false 2536 } 2537 } 2538 2539 func evaluateMatchWords(matchFilter *structs.MatchFilter, fieldValueStr string) bool { 2540 for _, word := range matchFilter.MatchWords { 2541 if evaluateMatchPhrase(string(word), fieldValueStr) { 2542 if matchFilter.MatchOperator == segutils.Or { 2543 return true 2544 } 2545 } else if matchFilter.MatchOperator == segutils.And { 2546 return false 2547 } 2548 } 2549 2550 return matchFilter.MatchOperator == segutils.And 2551 } 2552 2553 func evaluateMatchPhrase(matchPhrase string, fieldValueStr string) bool { 2554 // Create a regular expression to match the whole word, using \b for word boundaries 2555 pattern := `\b` + regexp.QuoteMeta(string(matchPhrase)) + `\b` 2556 r, err := regexp.Compile(pattern) 2557 if err != nil { 2558 return false 2559 } 2560 2561 // Use the regular expression to find a match 2562 return r.MatchString(fieldValueStr) 2563 } 2564 2565 func evaluateExpressionFilter(expressionFilter *structs.ExpressionFilter, record map[string]interface{}, recordMapStr string) bool { 2566 leftValue, errL := evaluateFilterInput(expressionFilter.LeftInput, record, recordMapStr) 2567 if errL != nil { 2568 return false 2569 } 2570 rightValue, errR := evaluateFilterInput(expressionFilter.RightInput, record, recordMapStr) 2571 if errR != nil { 2572 return false 2573 } 2574 2575 return conditionMatch(leftValue, expressionFilter.FilterOperator.ToString(), rightValue) 2576 } 2577 2578 func evaluateFilterInput(filterInput *structs.FilterInput, record map[string]interface{}, recordMapStr string) (interface{}, error) { 2579 if filterInput.SubTree != nil { 2580 return evaluateASTNode(filterInput.SubTree, record, recordMapStr), nil 2581 } else if filterInput.Expression != nil { 2582 return evaluateExpression(filterInput.Expression, record) 2583 } 2584 2585 return nil, fmt.Errorf("evaluateFilterInput: filterInput is invalid") 2586 } 2587 2588 func evaluateExpression(expr *structs.Expression, record map[string]interface{}) (interface{}, error) { 2589 var leftValue, rightValue, err interface{} 2590 2591 if expr.LeftInput != nil { 2592 leftValue, err = getInputValueFromExpression(expr.LeftInput, record) 2593 if err != nil { 2594 return nil, err.(error) 2595 } 2596 } 2597 2598 if expr.RightInput != nil { 2599 rightValue, err = getInputValueFromExpression(expr.RightInput, record) 2600 if err != nil { 2601 return nil, err.(error) 2602 } 2603 } 2604 2605 if leftValue != nil && rightValue != nil { 2606 return performArithmeticOperation(leftValue, rightValue, expr.ExpressionOp) 2607 } 2608 2609 return leftValue, nil 2610 } 2611 2612 func performArithmeticOperation(leftValue interface{}, rightValue interface{}, Op segutils.ArithmeticOperator) (interface{}, error) { 2613 switch Op { 2614 case segutils.Add: 2615 // Handle the case where both operands are strings 2616 if lv, ok := leftValue.(string); ok { 2617 if rv, ok := rightValue.(string); ok { 2618 return lv + rv, nil 2619 } 2620 return nil, fmt.Errorf("rightValue is not a string") 2621 } 2622 // Continue to handle the case where both operands are numbers 2623 fallthrough 2624 case segutils.Subtract, segutils.Multiply, segutils.Divide, segutils.Modulo, segutils.BitwiseAnd, segutils.BitwiseOr, segutils.BitwiseExclusiveOr: 2625 lv, errL := dtypeutils.ConvertToFloat(leftValue, 64) 2626 rv, errR := dtypeutils.ConvertToFloat(rightValue, 64) 2627 if errL != nil || errR != nil { 2628 return nil, fmt.Errorf("performArithmeticOperation: leftValue or rightValue is not a number") 2629 } 2630 switch Op { 2631 case segutils.Add: 2632 return lv + rv, nil 2633 case segutils.Subtract: 2634 return lv - rv, nil 2635 case segutils.Multiply: 2636 return lv * rv, nil 2637 case segutils.Divide: 2638 if rv == 0 { 2639 return nil, fmt.Errorf("performArithmeticOperation: cannot divide by zero") 2640 } 2641 return lv / rv, nil 2642 case segutils.Modulo: 2643 return int64(lv) % int64(rv), nil 2644 case segutils.BitwiseAnd: 2645 return int64(lv) & int64(rv), nil 2646 case segutils.BitwiseOr: 2647 return int64(lv) | int64(rv), nil 2648 case segutils.BitwiseExclusiveOr: 2649 return int64(lv) ^ int64(rv), nil 2650 default: 2651 return nil, fmt.Errorf("performArithmeticOperation: invalid arithmetic operator") 2652 } 2653 default: 2654 return nil, fmt.Errorf("performArithmeticOperation: invalid arithmetic operator") 2655 } 2656 } 2657 2658 func getInputValueFromExpression(expr *structs.ExpressionInput, record map[string]interface{}) (interface{}, error) { 2659 if expr.ColumnName != "" { 2660 value, exists := record[expr.ColumnName] 2661 if !exists { 2662 return nil, fmt.Errorf("getInputValueFromExpression: expr.ColumnName does not exist in record") 2663 } 2664 dval, err := segutils.CreateDtypeEnclosure(value, 0) 2665 if err != nil { 2666 return value, nil 2667 } else { 2668 value, _ = dval.GetValue() 2669 } 2670 return value, nil 2671 } else if expr.ColumnValue != nil { 2672 return expr.ColumnValue.GetValue() 2673 } 2674 2675 return nil, fmt.Errorf("getInputValueFromExpression: expr is invalid") 2676 } 2677 2678 func isTransactionMatchedWithTheFliterStringCondition(with *structs.FilterStringExpr, recordMapStr string, record map[string]interface{}) bool { 2679 if with.StringValue != "" { 2680 return evaluateMatchPhrase(with.StringValue, recordMapStr) 2681 } else if with.EvalBoolExpr != nil { 2682 return evaluateBoolExpr(with.EvalBoolExpr, record) 2683 } else if with.SearchNode != nil { 2684 return evaluateASTNode(with.SearchNode.(*structs.ASTNode), record, recordMapStr) 2685 } 2686 2687 return false 2688 } 2689 2690 // Splunk Transaction command based on the TransactionArguments on the JSON records. map[string]map[string]interface{} 2691 func processTransactionsOnRecords(records map[string]map[string]interface{}, processedTransactions map[string]map[string]interface{}, allCols []string, transactionArgs *structs.TransactionArguments, closeAllTransactions bool) ([]string, error) { 2692 2693 if transactionArgs == nil { 2694 return allCols, nil 2695 } 2696 2697 transactionFields := transactionArgs.Fields 2698 2699 if len(transactionFields) == 0 { 2700 transactionFields = []string{"timestamp"} 2701 } 2702 2703 transactionStartsWith := transactionArgs.StartsWith 2704 transactionEndsWith := transactionArgs.EndsWith 2705 2706 if transactionArgs.OpenTransactionEvents == nil { 2707 transactionArgs.OpenTransactionEvents = make(map[string][]map[string]interface{}) 2708 } 2709 2710 if transactionArgs.OpenTransactionsState == nil { 2711 transactionArgs.OpenTransactionsState = make(map[string]*structs.TransactionGroupState) 2712 } 2713 2714 appendGroupedRecords := func(currentState *structs.TransactionGroupState, transactionKey string) { 2715 2716 records, exists := transactionArgs.OpenTransactionEvents[transactionKey] 2717 2718 if !exists || len(records) == 0 { 2719 return 2720 } 2721 2722 groupedRecord := make(map[string]interface{}) 2723 groupedRecord["timestamp"] = currentState.Timestamp 2724 groupedRecord["event"] = records 2725 lastRecord := records[len(transactionArgs.OpenTransactionEvents[transactionKey])-1] 2726 groupedRecord["duration"] = uint64(lastRecord["timestamp"].(uint64)) - currentState.Timestamp 2727 groupedRecord["eventcount"] = uint64(len(records)) 2728 groupedRecord["transactionKey"] = transactionKey 2729 2730 for _, key := range transactionFields { 2731 groupedRecord[key] = lastRecord[key] 2732 } 2733 2734 processedTransactions[currentState.RecInden] = groupedRecord 2735 2736 // Clear the group records and state 2737 delete(transactionArgs.OpenTransactionEvents, transactionKey) 2738 delete(transactionArgs.OpenTransactionsState, transactionKey) 2739 } 2740 2741 for _, sortedRecord := range transactionArgs.SortedRecordsSlice { 2742 2743 recInden := sortedRecord["key"].(string) 2744 record := records[recInden] 2745 2746 recordMapStr := fmt.Sprintf("%v", record) 2747 2748 // Generate the transaction key from the record. 2749 transactionKey := "" 2750 2751 for _, field := range transactionFields { 2752 if record[field] != nil { 2753 transactionKey += "_" + fmt.Sprintf("%v", record[field]) 2754 } 2755 } 2756 2757 // If the transaction key is empty, then skip this record. 2758 if transactionKey == "" { 2759 continue 2760 } 2761 2762 // Initialize the group state for new transaction keys 2763 if _, exists := transactionArgs.OpenTransactionsState[transactionKey]; !exists { 2764 transactionArgs.OpenTransactionsState[transactionKey] = &structs.TransactionGroupState{ 2765 Key: transactionKey, 2766 Open: false, 2767 RecInden: recInden, 2768 Timestamp: 0, 2769 } 2770 } 2771 2772 currentState := transactionArgs.OpenTransactionsState[transactionKey] 2773 2774 // If StartsWith is given, then the transaction Should only Open when the record matches the StartsWith. OR 2775 // if StartsWith not present, then the transaction should open for all records. 2776 if !currentState.Open { 2777 openState := false 2778 2779 if transactionStartsWith != nil { 2780 openState = isTransactionMatchedWithTheFliterStringCondition(transactionStartsWith, recordMapStr, record) 2781 } else { 2782 openState = true 2783 } 2784 2785 if openState { 2786 currentState.Open = true 2787 currentState.Timestamp = uint64(record["timestamp"].(uint64)) 2788 2789 transactionArgs.OpenTransactionsState[transactionKey] = currentState 2790 transactionArgs.OpenTransactionEvents[transactionKey] = make([]map[string]interface{}, 0) 2791 } 2792 2793 } else if currentState.Open && transactionEndsWith == nil && transactionStartsWith != nil { 2794 // If StartsWith is given, but endsWith is not given, then the startswith will be the end of the transaction. 2795 // So close with last record and open a new transaction. 2796 2797 closeAndOpenState := isTransactionMatchedWithTheFliterStringCondition(transactionStartsWith, recordMapStr, record) 2798 2799 if closeAndOpenState { 2800 appendGroupedRecords(currentState, transactionKey) 2801 2802 currentState.Timestamp = uint64(record["timestamp"].(uint64)) 2803 currentState.Open = true 2804 currentState.RecInden = recInden 2805 2806 transactionArgs.OpenTransactionsState[transactionKey] = currentState 2807 transactionArgs.OpenTransactionEvents[transactionKey] = make([]map[string]interface{}, 0) 2808 } 2809 2810 } 2811 2812 // If the transaction is open, then append the record to the group. 2813 if currentState.Open { 2814 transactionArgs.OpenTransactionEvents[transactionKey] = append(transactionArgs.OpenTransactionEvents[transactionKey], record) 2815 } 2816 2817 if transactionEndsWith != nil { 2818 if currentState.Open { 2819 closeState := isTransactionMatchedWithTheFliterStringCondition(transactionEndsWith, recordMapStr, record) 2820 2821 if closeState { 2822 appendGroupedRecords(currentState, transactionKey) 2823 2824 currentState.Open = false 2825 currentState.Timestamp = 0 2826 currentState.RecInden = recInden 2827 transactionArgs.OpenTransactionsState[transactionKey] = currentState 2828 } 2829 } 2830 } 2831 } 2832 2833 // Transaction EndsWith is not given In this case, most or all of the transactionArgs.OpenTransactionEvents will not be appended to the groupedRecords. 2834 // Even if we are appending the transactionArgs.OpenTransactionEvents at StartsWith, not all the transactionArgs.OpenTransactionEvents will be appended to the groupedRecords. 2835 // So we need to append them here. 2836 if transactionEndsWith == nil && closeAllTransactions { 2837 for key := range transactionArgs.OpenTransactionEvents { 2838 appendGroupedRecords(transactionArgs.OpenTransactionsState[key], key) 2839 } 2840 } 2841 2842 allCols = make([]string, 0) 2843 allCols = append(allCols, "timestamp") 2844 allCols = append(allCols, "duration") 2845 allCols = append(allCols, "eventcount") 2846 allCols = append(allCols, "event") 2847 allCols = append(allCols, transactionFields...) 2848 2849 return allCols, nil 2850 } 2851 2852 // Decode the bucketKey into a slice of strings. 2853 func decodeBucketKey(bucketKey interface{}) ([]string, error) { 2854 switch castedKey := bucketKey.(type) { 2855 case []string: 2856 return castedKey, nil 2857 case string: 2858 return []string{castedKey}, nil 2859 default: 2860 return nil, fmt.Errorf("decodeBucketKey: unexpected type %T for bucketKey %v", castedKey, bucketKey) 2861 } 2862 } 2863 2864 // Return a string if the slice has length 1, otherwise return the slice. 2865 func encodeBucketKey(bucketKeySlice []string) interface{} { 2866 if len(bucketKeySlice) == 1 { 2867 return bucketKeySlice[0] 2868 } 2869 2870 return bucketKeySlice 2871 }