github.com/siglens/siglens@v0.0.0-20240328180423-f7ce9ae441ed/pkg/segment/query/segquery.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 query 18 19 import ( 20 "errors" 21 "fmt" 22 "sort" 23 "time" 24 25 dtu "github.com/siglens/siglens/pkg/common/dtypeutils" 26 "github.com/siglens/siglens/pkg/config" 27 "github.com/siglens/siglens/pkg/grpc/grpc_query" 28 "github.com/siglens/siglens/pkg/instrumentation" 29 "github.com/siglens/siglens/pkg/querytracker" 30 "github.com/siglens/siglens/pkg/segment/pqmr" 31 "github.com/siglens/siglens/pkg/segment/query/metadata" 32 "github.com/siglens/siglens/pkg/segment/query/pqs" 33 pqsmeta "github.com/siglens/siglens/pkg/segment/query/pqs/meta" 34 "github.com/siglens/siglens/pkg/segment/query/summary" 35 "github.com/siglens/siglens/pkg/segment/reader/segread" 36 "github.com/siglens/siglens/pkg/segment/results/segresults" 37 "github.com/siglens/siglens/pkg/segment/search" 38 "github.com/siglens/siglens/pkg/segment/structs" 39 segutils "github.com/siglens/siglens/pkg/segment/utils" 40 "github.com/siglens/siglens/pkg/segment/writer" 41 "github.com/siglens/siglens/pkg/utils" 42 log "github.com/sirupsen/logrus" 43 ) 44 45 const QUERY_INFO_REFRESH_LOOP_SECS = 300 46 47 var ExtractKibanaRequestsFn func([]string, uint64) map[string]*structs.SegmentSearchRequest 48 49 // Inits metadata layer and search limiter 50 func InitQueryNode(getMyIds func() []uint64, extractKibanaRequestsFn func([]string, uint64) map[string]*structs.SegmentSearchRequest) error { 51 ExtractKibanaRequestsFn = extractKibanaRequestsFn 52 ticker := time.NewTicker(30 * time.Second) 53 done := make(chan bool) 54 startTime := time.Now() 55 go func() { 56 for { 57 select { 58 case <-done: 59 log.Infof("Query server has successfully been initialized in %+v", time.Since(startTime)) 60 return 61 case <-ticker.C: 62 log.Infof("Query server is still being initialized with metadata. Total elapsed time %+v", time.Since(startTime)) 63 } 64 } 65 }() 66 initMetadataRefresh() 67 initGlobalMetadataRefresh(getMyIds) 68 go runQueryInfoRefreshLoop(getMyIds) 69 70 // Init specific writer components for kibana requests 71 if !config.IsIngestNode() { 72 writer.HostnameDir() 73 writer.InitKibanaInternalData() 74 } 75 ticker.Stop() 76 done <- true 77 78 return nil 79 } 80 81 // ingest only nodes should call this to be fetching remote pqs information 82 func InitQueryInfoRefresh(getMyIds func() []uint64) { 83 go runQueryInfoRefreshLoop(getMyIds) 84 } 85 86 func InitQueryMetrics() { 87 go queryMetricsLooper() 88 } 89 90 func queryMetricsLooper() { 91 for { 92 time.Sleep(1 * time.Minute) 93 go func() { 94 instrumentation.SetSegmentMicroindexCountGauge(metadata.GetTotalSMICount()) 95 }() 96 } 97 } 98 99 func getQueryType(sNode *structs.SearchNode, aggs *structs.QueryAggregators) (structs.SearchNodeType, structs.QueryType) { 100 if aggs != nil && aggs.GroupByRequest != nil { 101 if aggs.GroupByRequest.MeasureOperations != nil && aggs.GroupByRequest.GroupByColumns == nil { 102 return sNode.NodeType, structs.SegmentStatsCmd 103 } 104 if aggs != nil && aggs.GroupByRequest.MeasureOperations != nil && aggs.GroupByRequest.GroupByColumns != nil { 105 return sNode.NodeType, structs.GroupByCmd 106 } 107 } 108 if aggs != nil && aggs.MeasureOperations != nil && aggs.GroupByRequest == nil { 109 return sNode.NodeType, structs.SegmentStatsCmd 110 } 111 return sNode.NodeType, structs.RRCCmd 112 } 113 114 func ApplyFilterOperator(node *structs.ASTNode, timeRange *dtu.TimeRange, aggs *structs.QueryAggregators, 115 qid uint64, qc *structs.QueryContext) *structs.NodeResult { 116 117 sTime := time.Now() 118 searchNode := ConvertASTNodeToSearchNode(node, qid) 119 kibanaIndices := qc.TableInfo.GetKibanaIndices() 120 nonKibanaIndices := qc.TableInfo.GetQueryTables() 121 containsKibana := false 122 if len(kibanaIndices) != 0 { 123 containsKibana = true 124 } 125 querytracker.UpdateQTUsage(nonKibanaIndices, searchNode, aggs) 126 parallelismPerFile := int64(1) 127 _, qType := getQueryType(searchNode, aggs) 128 querySummary := summary.InitQuerySummary(summary.LOGS, qid) 129 pqid := querytracker.GetHashForQuery(searchNode) 130 defer querySummary.LogSummaryAndEmitMetrics(qid, pqid, containsKibana, qc.Orgid) 131 allSegFileResults, err := segresults.InitSearchResults(qc.SizeLimit, aggs, qType, qid) 132 if err != nil { 133 log.Errorf("qid=%d Failed to InitSearchResults! error %+v", qid, err) 134 return &structs.NodeResult{ 135 ErrList: []error{err}, 136 } 137 } 138 dqs := InitDistQueryService(querySummary, allSegFileResults) 139 queryInfo, err := InitQueryInformation(searchNode, aggs, timeRange, qc.TableInfo, 140 qc.SizeLimit, parallelismPerFile, qid, dqs, qc.Orgid) 141 if err != nil { 142 log.Errorf("qid=%d Failed to InitQueryInformation! error %+v", qid, err) 143 return &structs.NodeResult{ 144 ErrList: []error{err}, 145 } 146 } 147 err = associateSearchInfoWithQid(qid, allSegFileResults, aggs, dqs, qType) 148 if err != nil { 149 log.Errorf("qid=%d Failed to associate search results with qid! Error: %+v", qid, err) 150 } 151 152 log.Infof("qid=%d, Extracted node type %v for query. ParallelismPerFile=%v. Starting search...", 153 qid, searchNode.NodeType, parallelismPerFile) 154 155 // Kibana requests will not honor time range sent in the query 156 // TODO: distibuted kibana requests? 157 applyKibanaFilterOperator(kibanaIndices, allSegFileResults, parallelismPerFile, searchNode, 158 qc.SizeLimit, aggs, qid, querySummary) 159 switch qType { 160 case structs.SegmentStatsCmd: 161 return getNodeResultsForSegmentStatsCmd(queryInfo, sTime, allSegFileResults, nil, querySummary, false, qc.Orgid) 162 case structs.RRCCmd, structs.GroupByCmd: 163 bucketLimit := MAX_GRP_BUCKS 164 if aggs != nil { 165 if aggs.BucketLimit != 0 && aggs.BucketLimit < MAX_GRP_BUCKS { 166 bucketLimit = aggs.BucketLimit 167 } 168 aggs.BucketLimit = bucketLimit 169 } else { 170 aggs = structs.InitDefaultQueryAggregations() 171 aggs.BucketLimit = bucketLimit 172 queryInfo.aggs = aggs 173 } 174 return getNodeResultsForRRCCmd(queryInfo, sTime, allSegFileResults, querySummary, false, qc.Orgid) 175 default: 176 err := errors.New("unsupported query type") 177 log.Errorf("qid=%d Failed to apply search! error %+v", qid, err) 178 return &structs.NodeResult{ 179 ErrList: []error{err}, 180 } 181 } 182 } 183 184 /* 185 Fast path for GRPC requests for unrotated requests 186 187 Assumptions: 188 - no kibana indices 189 - sNode may have regexes that needs to be compiled 190 */ 191 func ApplyUnrotatedQuery(sNode *structs.SearchNode, timeRange *dtu.TimeRange, aggs *structs.QueryAggregators, 192 indexInfo *structs.TableInfo, sizeLimit uint64, qid uint64, orgid uint64) *structs.NodeResult { 193 194 sTime := time.Now() 195 sNode.AddQueryInfoForNode() 196 197 querytracker.UpdateQTUsage(indexInfo.GetQueryTables(), sNode, aggs) 198 _, qType := getQueryType(sNode, aggs) 199 allSegFileResults, err := segresults.InitSearchResults(sizeLimit, aggs, qType, qid) 200 if err != nil { 201 log.Errorf("qid=%d Failed to InitSearchResults! error %+v", qid, err) 202 return &structs.NodeResult{ 203 ErrList: []error{err}, 204 } 205 } 206 querySummary := summary.InitQuerySummary(summary.LOGS, qid) 207 parallelismPerFile := int64(1) 208 queryInfo, err := InitQueryInformation(sNode, aggs, timeRange, indexInfo, 209 sizeLimit, parallelismPerFile, qid, nil, orgid) 210 defer querySummary.LogSummaryAndEmitMetrics(qid, queryInfo.pqid, false, orgid) 211 if err != nil { 212 return &structs.NodeResult{ 213 ErrList: []error{err}, 214 } 215 } 216 err = associateSearchInfoWithQid(qid, allSegFileResults, aggs, nil, queryInfo.qType) 217 if err != nil { 218 log.Errorf("qid=%d Failed to associate search results with qid! Error: %+v", qid, err) 219 } 220 221 log.Infof("qid=%d, Extracted node type %v for query. ParallelismPerFile=%v. Starting search...", 222 qid, sNode.NodeType, parallelismPerFile) 223 switch queryInfo.qType { 224 case structs.RRCCmd, structs.GroupByCmd: 225 return getNodeResultsForRRCCmd(queryInfo, sTime, allSegFileResults, querySummary, true, orgid) 226 case structs.SegmentStatsCmd: 227 return getNodeResultsForSegmentStatsCmd(queryInfo, sTime, allSegFileResults, nil, querySummary, true, orgid) 228 default: 229 err := errors.New("unsupported query type") 230 log.Errorf("qid=%d Failed to apply search! error %+v", qid, err) 231 return &structs.NodeResult{ 232 ErrList: []error{err}, 233 } 234 } 235 } 236 237 /* 238 Fast path for GRPC requests for rotated requests 239 240 Assumptions: 241 - no kibana indices 242 - sNode may have regexes that needs to be compiled 243 */ 244 func ApplyRotatedQuery(reqs []grpc_query.SegkeyRequest, sNode *structs.SearchNode, timeRange *dtu.TimeRange, aggs *structs.QueryAggregators, 245 indexInfo *structs.TableInfo, sizeLimit uint64, qid uint64, orgid uint64) *structs.NodeResult { 246 247 sTime := time.Now() 248 sNode.AddQueryInfoForNode() 249 250 querytracker.UpdateQTUsage(indexInfo.GetQueryTables(), sNode, aggs) 251 _, qType := getQueryType(sNode, aggs) 252 allSegFileResults, err := segresults.InitSearchResults(sizeLimit, aggs, qType, qid) 253 if err != nil { 254 log.Errorf("qid=%d Failed to InitSearchResults! error %+v", qid, err) 255 return &structs.NodeResult{ 256 ErrList: []error{err}, 257 } 258 } 259 querySummary := summary.InitQuerySummary(summary.LOGS, qid) 260 parallelismPerFile := int64(1) 261 queryInfo, err := InitQueryInformation(sNode, aggs, timeRange, indexInfo, 262 sizeLimit, parallelismPerFile, qid, nil, orgid) 263 defer querySummary.LogSummaryAndEmitMetrics(qid, queryInfo.pqid, false, orgid) 264 if err != nil { 265 return &structs.NodeResult{ 266 ErrList: []error{err}, 267 } 268 } 269 err = associateSearchInfoWithQid(qid, allSegFileResults, aggs, nil, queryInfo.qType) 270 if err != nil { 271 log.Errorf("qid=%d Failed to associate search results with qid! Error: %+v", qid, err) 272 } 273 274 log.Infof("qid=%d, Extracted node type %v for query. ParallelismPerFile=%v. Starting search...", 275 qid, sNode.NodeType, parallelismPerFile) 276 switch queryInfo.qType { 277 case structs.RRCCmd, structs.GroupByCmd: 278 qsrs := convertSegKeysToQSR(queryInfo, reqs) 279 qsrs, raw, pqs := filterSegKeysToQueryResults(queryInfo, qsrs) 280 log.Infof("qid=%d, QueryType %+v Filtered %d segkeys to raw %d and %d pqs keys", qid, queryInfo.qType.String(), len(qsrs), raw, pqs) 281 return getNodeResultsFromQSRS(qsrs, queryInfo, sTime, allSegFileResults, querySummary) 282 case structs.SegmentStatsCmd: 283 return getNodeResultsForSegmentStatsCmd(queryInfo, sTime, allSegFileResults, reqs, 284 querySummary, false, orgid) 285 default: 286 err := errors.New("unsupported query type") 287 log.Errorf("qid=%d Failed to apply search! error %+v", qid, err) 288 return &structs.NodeResult{ 289 ErrList: []error{err}, 290 } 291 } 292 } 293 294 func convertSegKeysToQSR(qI *queryInformation, segReqs []grpc_query.SegkeyRequest) []*querySegmentRequest { 295 qsrs := make([]*querySegmentRequest, 0, len(segReqs)) 296 for _, segReq := range segReqs { 297 qsrs = append(qsrs, &querySegmentRequest{ 298 queryInformation: *qI, 299 segKey: segReq.GetSegmentKey(), 300 tableName: segReq.GetTableName(), 301 segKeyTsRange: &dtu.TimeRange{StartEpochMs: segReq.GetStartEpochMs(), EndEpochMs: segReq.GetEndEpochMs()}, 302 }) 303 } 304 return qsrs 305 } 306 307 func convertSegStatKeysToQSR(qI *queryInformation, segReqs []grpc_query.SegkeyRequest) []*querySegmentRequest { 308 qsrs := make([]*querySegmentRequest, 0, len(segReqs)) 309 for _, segReq := range segReqs { 310 qsrs = append(qsrs, &querySegmentRequest{ 311 queryInformation: *qI, 312 segKey: segReq.GetSegmentKey(), 313 tableName: segReq.GetTableName(), 314 sType: structs.SEGMENT_STATS_SEARCH, 315 segKeyTsRange: &dtu.TimeRange{StartEpochMs: segReq.GetStartEpochMs(), EndEpochMs: segReq.GetEndEpochMs()}, 316 }) 317 } 318 return qsrs 319 } 320 321 // Base function to apply operators on query segment requests 322 func getNodeResultsFromQSRS(sortedQSRSlice []*querySegmentRequest, queryInfo *queryInformation, sTime time.Time, 323 allSegFileResults *segresults.SearchResults, querySummary *summary.QuerySummary) *structs.NodeResult { 324 applyFopAllRequests(sortedQSRSlice, queryInfo, allSegFileResults, querySummary) 325 err := queryInfo.Wait(querySummary) 326 if err != nil { 327 log.Errorf("qid=%d Failed to wait for all query segment requests to finish! Error: %+v", queryInfo.qid, err) 328 return &structs.NodeResult{ 329 ErrList: []error{err}, 330 } 331 } 332 querySummary.UpdateQueryTotalTime(time.Since(sTime), allSegFileResults.GetNumBuckets()) 333 setQidAsFinished(queryInfo.qid) 334 queryType := GetQueryType(queryInfo.qid) 335 bucketLimit := MAX_GRP_BUCKS 336 if queryInfo.aggs != nil { 337 if queryInfo.aggs.BucketLimit != 0 && queryInfo.aggs.BucketLimit < MAX_GRP_BUCKS { 338 bucketLimit = queryInfo.aggs.BucketLimit 339 } 340 } 341 aggMeasureRes, aggMeasureFunctions, aggGroupByCols, bucketCount := allSegFileResults.GetGroupyByBuckets(bucketLimit) 342 return &structs.NodeResult{ 343 AllRecords: allSegFileResults.GetResults(), 344 ErrList: allSegFileResults.GetAllErrors(), 345 TotalResults: allSegFileResults.GetQueryCount(), 346 Histogram: allSegFileResults.GetBucketResults(), 347 SegEncToKey: allSegFileResults.SegEncToKey, 348 MeasureResults: aggMeasureRes, 349 MeasureFunctions: aggMeasureFunctions, 350 GroupByCols: aggGroupByCols, 351 Qtype: queryType.String(), 352 BucketCount: bucketCount, 353 } 354 } 355 356 func getNodeResultsForRRCCmd(queryInfo *queryInformation, sTime time.Time, allSegFileResults *segresults.SearchResults, 357 querySummary *summary.QuerySummary, unrotatedGRPC bool, orgid uint64) *structs.NodeResult { 358 359 sortedQSRSlice, numRawSearch, distributedQueries, numPQS, err := getAllSegmentsInQuery(queryInfo, unrotatedGRPC, sTime, orgid) 360 if err != nil { 361 log.Errorf("qid=%d Failed to get all segments in query! Error: %+v", queryInfo.qid, err) 362 return &structs.NodeResult{ 363 ErrList: []error{err}, 364 } 365 } 366 log.Infof("qid=%d, Received %+v query segment requests. %+v raw search %+v pqs and %+v distribued query elapsed time: %+v", 367 queryInfo.qid, len(sortedQSRSlice), numRawSearch, numPQS, distributedQueries, time.Since(sTime)) 368 err = setTotalSegmentsToSearch(queryInfo.qid, numRawSearch+numPQS+distributedQueries) 369 if err != nil { 370 log.Errorf("qid=%d Failed to set total segments to search! Error: %+v", queryInfo.qid, err) 371 } 372 querySummary.UpdateRemainingDistributedQueries(distributedQueries) 373 return getNodeResultsFromQSRS(sortedQSRSlice, queryInfo, sTime, allSegFileResults, querySummary) 374 } 375 376 func getNodeResultsForSegmentStatsCmd(queryInfo *queryInformation, sTime time.Time, allSegFileResults *segresults.SearchResults, 377 reqs []grpc_query.SegkeyRequest, querySummary *summary.QuerySummary, unrotatedOnly bool, orgid uint64) *structs.NodeResult { 378 sortedQSRSlice, numRawSearch, numDistributed := getAllSegmentsInAggs(queryInfo, reqs, queryInfo.aggs, queryInfo.queryRange, queryInfo.indexInfo.GetQueryTables(), 379 queryInfo.qid, unrotatedOnly, sTime, orgid) 380 err := setTotalSegmentsToSearch(queryInfo.qid, numRawSearch) 381 if err != nil { 382 log.Errorf("qid=%d Failed to set total segments to search! Error: %+v", queryInfo.qid, err) 383 } 384 querySummary.UpdateRemainingDistributedQueries(numDistributed) 385 log.Infof("qid=%d, Received %+v query segment aggs, with %+v raw search %v distributed, query elapsed time: %+v", 386 queryInfo.qid, len(sortedQSRSlice), numRawSearch, numDistributed, time.Since(sTime)) 387 if queryInfo.aggs.MeasureOperations != nil { 388 allSegFileResults.InitSegmentStatsResults(queryInfo.aggs.MeasureOperations) 389 applyAggOpOnSegments(sortedQSRSlice, allSegFileResults, queryInfo.qid, querySummary, queryInfo.sNodeType, queryInfo.aggs.MeasureOperations) 390 } 391 querySummary.UpdateQueryTotalTime(time.Since(sTime), allSegFileResults.GetNumBuckets()) 392 queryType := GetQueryType(queryInfo.qid) 393 aggMeasureRes, aggMeasureFunctions, aggGroupByCols, bucketCount := allSegFileResults.GetSegmentStatsResults(0) 394 err = queryInfo.Wait(querySummary) 395 if err != nil { 396 log.Errorf("qid=%d getNodeResultsForSegmentStatsCmd: Failed to wait for all query segment requests to finish! Error: %+v", queryInfo.qid, err) 397 return &structs.NodeResult{ 398 ErrList: []error{err}, 399 } 400 } 401 setQidAsFinished(queryInfo.qid) 402 return &structs.NodeResult{ 403 ErrList: allSegFileResults.GetAllErrors(), 404 TotalResults: allSegFileResults.GetQueryCount(), 405 SegEncToKey: allSegFileResults.SegEncToKey, 406 MeasureResults: aggMeasureRes, 407 MeasureFunctions: aggMeasureFunctions, 408 GroupByCols: aggGroupByCols, 409 Qtype: queryType.String(), 410 BucketCount: bucketCount, 411 } 412 } 413 414 func getSortedQSRResult(aggs *structs.QueryAggregators, allQSRs []*querySegmentRequest) []*querySegmentRequest { 415 if aggs != nil && aggs.Sort != nil { 416 if aggs.Sort.Ascending { 417 // index 0 should have the latest time 418 sort.Slice(allQSRs, func(i, j int) bool { 419 return allQSRs[i].segKeyTsRange.StartEpochMs < allQSRs[j].segKeyTsRange.StartEpochMs 420 }) 421 } else { 422 // index 0 should have the earliest time 423 sort.Slice(allQSRs, func(i, j int) bool { 424 return allQSRs[i].segKeyTsRange.StartEpochMs > allQSRs[j].segKeyTsRange.StartEpochMs 425 }) 426 } 427 } 428 return allQSRs 429 } 430 431 // Gets special kibana SSRs and applies raw search 432 func applyKibanaFilterOperator(kibanaIndices []string, allSegFileResults *segresults.SearchResults, parallelismPerFile int64, searchNode *structs.SearchNode, 433 sizeLimit uint64, aggs *structs.QueryAggregators, qid uint64, qs *summary.QuerySummary) { 434 if len(kibanaIndices) == 0 { 435 return 436 } 437 kibanaSearchRequests := ExtractKibanaRequestsFn(kibanaIndices, qid) 438 log.Infof("qid=%d, applyKibanaFilterOperator: Kibana request has %+v SSRs", qid, len(kibanaSearchRequests)) 439 440 tRange := &dtu.TimeRange{ 441 StartEpochMs: 0, 442 EndEpochMs: utils.GetCurrentTimeInMs(), 443 } 444 err := applyFilterOperatorInternal(allSegFileResults, kibanaSearchRequests, parallelismPerFile, searchNode, tRange, 445 sizeLimit, aggs, qid, qs) 446 if err != nil { 447 log.Errorf("qid=%d, applyKibanaFilterOperator failed to apply filter opterator for kibana requests! %+v", qid, err) 448 allSegFileResults.AddError(err) 449 } 450 } 451 452 func reverseSortedQSRSlice(sortedQSRSlice []*querySegmentRequest) { 453 lenSortedQSRSlice := len(sortedQSRSlice) 454 455 for i := 0; i < lenSortedQSRSlice/2; i++ { 456 sortedQSRSlice[i], sortedQSRSlice[lenSortedQSRSlice-i-1] = sortedQSRSlice[lenSortedQSRSlice-i-1], sortedQSRSlice[i] 457 } 458 } 459 460 // loops over all inputted querySegmentRequests and apply search for each file. This function may exit early 461 func applyFopAllRequests(sortedQSRSlice []*querySegmentRequest, queryInfo *queryInformation, 462 allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) { 463 464 // In order, search segKeys (either raw or pqs depending on above sType). 465 // If no aggs, early exit at utils.QUERY_EARLY_EXIT_LIMIT 466 // If sort, check if next segkey's time range will overlap with the recent best results 467 // If there's aggs and they can be computed fully by agile trees, limit the number of 468 // buckets to utils.QUERY_MAX_BUCKETS unless a sort or computation follows the aggs. 469 470 limitAgileAggsTreeBuckets := canUseBucketLimitedAgileAggsTree(sortedQSRSlice, queryInfo) 471 var agileTreeBuckets map[string]struct{} 472 var agileTreeBuf []byte 473 if config.IsAggregationsEnabled() && queryInfo.qType == structs.GroupByCmd && 474 queryInfo.sNodeType == structs.MatchAllQuery { 475 agileTreeBuf = make([]byte, 300_000_000) 476 } 477 478 doBuckPull := false 479 rrcsCompleted := false 480 segsNotSent := int(0) 481 recsSearchedSinceLastUpdate := uint64(0) 482 allEmptySegsForPqid := map[string]bool{} 483 var err error 484 if len(sortedQSRSlice) > 0 && queryInfo.persistentQuery { 485 allEmptySegsForPqid, err = pqsmeta.GetAllEmptySegmentsForPqid(sortedQSRSlice[0].pqid) 486 if err != nil { 487 log.Errorf("qid=%d, Failed to get empty segments for pqid %+v! Error: %v", queryInfo.qid, sortedQSRSlice[0].pqid, err) 488 } 489 } 490 491 // If we have a Transaction command, we want to search the segments from the oldest to the newest 492 if queryInfo.aggs != nil && queryInfo.aggs.HasTransactionArgumentsInChain() { 493 reverseSortedQSRSlice(sortedQSRSlice) 494 } 495 496 for idx, segReq := range sortedQSRSlice { 497 498 isCancelled, err := checkForCancelledQuery(queryInfo.qid) 499 if err != nil { 500 log.Errorf("qid=%d, Failed to checkForCancelledQuery. Error: %v", queryInfo.qid, err) 501 } 502 if isCancelled { 503 return 504 } 505 otherAggsPresent, timeAggs := checkAggTypes(segReq.aggs) 506 eeType := allSegFileResults.ShouldSearchSegKey(segReq.segKeyTsRange, segReq.sNode.NodeType, otherAggsPresent, timeAggs) 507 if eeType == segresults.EetEarlyExit { 508 allSegFileResults.SetEarlyExit(true) 509 } else if eeType == segresults.EetMatchAllAggs { 510 allSegFileResults.SetEarlyExit(true) 511 err := applyFopFastPathSingleRequest(segReq, allSegFileResults, qs) 512 if err != nil { 513 log.Errorf("qid=%d, Failed to apply fastpath for segKey %+v! Error: %v", queryInfo.qid, segReq.segKey, err) 514 allSegFileResults.AddError(err) 515 } 516 } else { 517 doAgileTree, str := canUseAgileTree(segReq, queryInfo) 518 519 if doAgileTree { 520 sTime := time.Now() 521 522 if limitAgileAggsTreeBuckets { 523 // Reuse the bucket keys from the previous segments so we 524 // sync which buckets we're using across segments. 525 str.SetBuckets(agileTreeBuckets) 526 str.SetBucketLimit(segutils.QUERY_MAX_BUCKETS) 527 } 528 529 search.ApplyAgileTree(str, segReq.aggs, allSegFileResults, segReq.sizeLimit, queryInfo.qid, 530 agileTreeBuf) 531 532 if limitAgileAggsTreeBuckets { 533 // Get the buckets so we can use its keys for the next 534 // segment so that we sync which buckets we're using across 535 // segments. 536 agileTreeBuckets = str.GetBuckets() 537 } 538 539 str.Close() 540 timeElapsed := time.Since(sTime) 541 queryMetrics := &structs.QueryProcessingMetrics{} 542 numRecs := metadata.GetNumOfSearchedRecordsRotated(segReq.segKey) 543 queryMetrics.SetNumRecordsToRawSearch(numRecs) 544 queryMetrics.SetNumRecordsMatched(numRecs) 545 qs.UpdateSummary(summary.STREE, timeElapsed, queryMetrics) 546 } else { 547 if segReq.sType == structs.PQS { 548 _, ok := allEmptySegsForPqid[segReq.segKey] 549 if ok { 550 log.Debugf("Skipping segKey %v for pqid %v", segReq.segKey, segReq.queryInformation.pqid) 551 continue 552 } 553 } 554 // else we continue with rawsearch 555 err = applyFilterOperatorSingleRequest(segReq, allSegFileResults, qs) 556 if err != nil { 557 log.Errorf("qid=%d, Failed to apply filter operator for segKey %+v! Error: %v", queryInfo.qid, segReq.segKey, err) 558 allSegFileResults.AddError(err) 559 } 560 } 561 } 562 var recsSearched uint64 563 if segReq.sType == structs.RAW_SEARCH || segReq.sType == structs.PQS { 564 recsSearched = metadata.GetNumOfSearchedRecordsRotated(segReq.segKey) 565 } else { 566 recsSearched = writer.GetNumOfSearchedRecordsUnRotated(segReq.segKey) 567 } 568 if idx == len(sortedQSRSlice)-1 { 569 doBuckPull = true 570 } 571 recsSearchedSinceLastUpdate += recsSearched 572 if segReq.HasMatchedRrc { 573 segsNotSent++ 574 segenc := allSegFileResults.SegKeyToEnc[segReq.segKey] 575 incrementNumFinishedSegments(segsNotSent, queryInfo.qid, recsSearchedSinceLastUpdate, segenc, 576 doBuckPull, nil) 577 segsNotSent = 0 578 recsSearchedSinceLastUpdate = 0 579 } else { 580 segsNotSent++ 581 } 582 if !rrcsCompleted && areAllRRCsFound(allSegFileResults, sortedQSRSlice[idx+1:], segReq.aggs) { 583 qs.SetRRCFinishTime() 584 rrcsCompleted = true 585 } 586 } 587 588 if segsNotSent > 0 { 589 doBucketPull := true // This is the last update, so flush the buckets. 590 incrementNumFinishedSegments(segsNotSent, queryInfo.qid, recsSearchedSinceLastUpdate, 0, doBucketPull, nil) 591 } 592 593 if !rrcsCompleted { 594 qs.SetRRCFinishTime() 595 } 596 if len(sortedQSRSlice) == 0 { 597 incrementNumFinishedSegments(0, queryInfo.qid, recsSearchedSinceLastUpdate, 0, false, nil) 598 } 599 } 600 601 // Return true if we can use AgileAggsTrees for all the segments and we can 602 // limit the number of buckets. 603 func canUseBucketLimitedAgileAggsTree(sortedQSRSlice []*querySegmentRequest, queryInfo *queryInformation) bool { 604 if !queryInfo.aggs.CanLimitBuckets() { 605 return false 606 } 607 608 for _, segReq := range sortedQSRSlice { 609 canUse, agileTree := canUseAgileTree(segReq, queryInfo) 610 611 if agileTree != nil { 612 agileTree.Close() 613 } 614 615 if !canUse { 616 return false 617 } 618 } 619 620 return true 621 } 622 623 func canUseAgileTree(segReq *querySegmentRequest, queryInfo *queryInformation) (bool, *segread.AgileTreeReader) { 624 isSegFullyEncosed := segReq.queryRange.AreTimesFullyEnclosed(segReq.segKeyTsRange.StartEpochMs, 625 segReq.segKeyTsRange.EndEpochMs) 626 _, timeAggs := checkAggTypes(segReq.aggs) 627 628 if config.IsAggregationsEnabled() && isSegFullyEncosed && queryInfo.qType == structs.GroupByCmd && 629 queryInfo.sNodeType == structs.MatchAllQuery && !timeAggs { 630 return search.CanDoStarTree(segReq.segKey, segReq.aggs, queryInfo.qid) 631 } 632 633 return false, nil 634 } 635 636 // returns true if any element in qsrs would displace any of the RRCs 637 // displacing of RRCs will only happen if not all RRCs exist or if sort conditions will displace 638 // Returns true if no more raw search will to be performed 639 func areAllRRCsFound(sr *segresults.SearchResults, qsrs []*querySegmentRequest, aggs *structs.QueryAggregators) bool { 640 641 if sr.ShouldContinueRRCSearch() { 642 return false 643 } 644 645 if aggs == nil || aggs.Sort == nil { 646 return true 647 } 648 649 for _, r := range qsrs { 650 var willValBeAdded bool 651 if aggs.Sort.Ascending { 652 willValBeAdded = sr.BlockResults.WillValueBeAdded(float64(r.segKeyTsRange.StartEpochMs)) 653 } else { 654 willValBeAdded = sr.BlockResults.WillValueBeAdded(float64(r.segKeyTsRange.EndEpochMs)) 655 } 656 if willValBeAdded { 657 return false 658 } 659 } 660 return true 661 } 662 663 // Returns query segment requests, count of keys to raw search, count of distributed queries, count of keys in PQS 664 func getAllUnrotatedSegments(queryInfo *queryInformation, unrotatedGRPC bool, sTime time.Time, orgid uint64) ([]*querySegmentRequest, uint64, uint64, uint64, error) { 665 allUnrotatedKeys, totalChecked, totalCount := writer.FilterUnrotatedSegmentsInQuery(queryInfo.queryRange, queryInfo.indexInfo.GetQueryTables(), orgid) 666 log.Infof("qid=%d, Unrotated query time filtering returned %v segment keys to search out of %+v. query elapsed time: %+v", queryInfo.qid, totalCount, 667 totalChecked, time.Since(sTime)) 668 669 var distCount uint64 670 var err error 671 if !unrotatedGRPC { 672 distCount, err = queryInfo.dqs.DistributeUnrotatedQuery(queryInfo) 673 if err != nil { 674 log.Errorf("qid=%d, Failed to send unrotated query request! Error: %v", queryInfo.qid, err) 675 return nil, 0, 0, 0, err 676 } 677 } 678 qsr, raw, pqs := filterUnrotatedSegKeysToQueryRequests(queryInfo, allUnrotatedKeys) 679 return qsr, raw, distCount, pqs, nil 680 } 681 682 // returns query segment requests, count of keys to raw search, and distributed query count 683 func getAllSegmentsInAggs(queryInfo *queryInformation, reqs []grpc_query.SegkeyRequest, aggs *structs.QueryAggregators, timeRange *dtu.TimeRange, indexNames []string, 684 qid uint64, unrotatedGRPC bool, sTime time.Time, orgid uint64) ([]*querySegmentRequest, uint64, uint64) { 685 686 if len(reqs) != 0 { 687 qsrs := convertSegStatKeysToQSR(queryInfo, reqs) 688 return qsrs, uint64(len(qsrs)), 0 689 } 690 if unrotatedGRPC { 691 unrotatedQSR, unrotatedRawCount, unrotatedDistQueries := getAllUnrotatedSegmentsInAggs(queryInfo, aggs, timeRange, indexNames, qid, sTime, orgid, unrotatedGRPC) 692 return unrotatedQSR, unrotatedRawCount, unrotatedDistQueries 693 } 694 695 // Do rotated time & index name filtering 696 rotatedQSR, rotatedRawCount, rotatedDistQueries := getAllRotatedSegmentsInAggs(queryInfo, aggs, timeRange, indexNames, qid, sTime, orgid) 697 unrotatedQSR, unrotatedRawCount, unrotatedDistQueries := getAllUnrotatedSegmentsInAggs(queryInfo, aggs, timeRange, indexNames, qid, sTime, orgid, unrotatedGRPC) 698 allSegRequests := append(rotatedQSR, unrotatedQSR...) 699 //get seg stats for allPossibleKeys 700 return allSegRequests, rotatedRawCount + unrotatedRawCount, rotatedDistQueries + unrotatedDistQueries 701 } 702 703 func getAllUnrotatedSegmentsInAggs(queryInfo *queryInformation, aggs *structs.QueryAggregators, timeRange *dtu.TimeRange, indexNames []string, 704 qid uint64, sTime time.Time, orgid uint64, unrotatedGRPC bool) ([]*querySegmentRequest, uint64, uint64) { 705 allUnrotatedKeys, totalChecked, totalCount := writer.FilterUnrotatedSegmentsInQuery(timeRange, indexNames, orgid) 706 log.Infof("qid=%d, Unrotated query time filtering returned %v segment keys to search out of %+v. query elapsed time: %+v", qid, totalCount, 707 totalChecked, time.Since(sTime)) 708 var distCount uint64 709 var err error 710 if !unrotatedGRPC { 711 distCount, err = queryInfo.dqs.DistributeUnrotatedQuery(queryInfo) 712 if err != nil { 713 log.Errorf("qid=%d, Failed to send unrotated query request! Error: %v", queryInfo.qid, err) 714 return nil, 0, 0 715 } 716 } 717 qsrs, rawSearch := filterAggSegKeysToQueryResults(queryInfo, allUnrotatedKeys, aggs, structs.UNROTATED_SEGMENT_STATS_SEARCH) 718 return qsrs, rawSearch, distCount 719 } 720 721 func getAllRotatedSegmentsInAggs(queryInfo *queryInformation, aggs *structs.QueryAggregators, timeRange *dtu.TimeRange, indexNames []string, 722 qid uint64, sTime time.Time, orgid uint64) ([]*querySegmentRequest, uint64, uint64) { 723 // 1. metadata.FilterSegmentsByTime gives epoch range 724 allPossibleKeys, tsPassedCount, totalPossible := metadata.FilterSegmentsByTime(timeRange, indexNames, orgid) 725 log.Infof("qid=%d, Rotated query time filtering returned %v segment keys to search out of %+v. query elapsed time: %+v", qid, tsPassedCount, 726 totalPossible, time.Since(sTime)) 727 728 qsrs, totalQsr := filterAggSegKeysToQueryResults(queryInfo, allPossibleKeys, aggs, structs.SEGMENT_STATS_SEARCH) 729 currNodeQsrs, distributedRequests, err := queryInfo.dqs.DistributeRotatedRequests(queryInfo, qsrs) 730 if err != nil { 731 log.Errorf("qid=%d, Error in distributing rotated requests %+v", queryInfo.qid, err) 732 return nil, 0, 0 733 } 734 return currNodeQsrs, totalQsr - distributedRequests, distributedRequests 735 } 736 737 func applyAggOpOnSegments(sortedQSRSlice []*querySegmentRequest, allSegFileResults *segresults.SearchResults, qid uint64, qs *summary.QuerySummary, 738 searchType structs.SearchNodeType, measureOperations []*structs.MeasureAggregator) { 739 // Use a global variable to store data that meets the conditions during the process of traversing segments 740 runningEvalStats := make(map[string]interface{}, 0) 741 //assuming we will allow 100 measure Operations 742 for _, segReq := range sortedQSRSlice { 743 isCancelled, err := checkForCancelledQuery(qid) 744 if err != nil { 745 log.Errorf("qid=%d, Failed to checkForCancelledQuery. Error: %v", qid, err) 746 } 747 if isCancelled { 748 break 749 } 750 isSegmentFullyEnclosed := segReq.segKeyTsRange.AreTimesFullyEnclosed(segReq.segKeyTsRange.StartEpochMs, segReq.segKeyTsRange.EndEpochMs) 751 752 // Because segment only store statistical data such as min, max..., for some functions we should recompute raw data to get the results 753 // If agg has evaluation functions, we should recompute raw data instead of using the previously stored statistical data in the segment 754 aggHasEvalFunc := segReq.aggs.HasValueColRequest() 755 aggHasValuesFunc := segReq.aggs.HasValuesFunc() 756 var sstMap map[string]*structs.SegStats 757 if searchType == structs.MatchAllQuery && isSegmentFullyEnclosed && !aggHasEvalFunc && !aggHasValuesFunc { 758 sstMap, err = segread.ReadSegStats(segReq.segKey, segReq.qid) 759 if err != nil { 760 log.Errorf("qid=%d, applyAggOpOnSegments : ReadSegStats: Failed to get segment level stats for segKey %+v! Error: %v", qid, segReq.segKey, err) 761 allSegFileResults.AddError(err) 762 continue 763 } 764 } else { 765 // run through micro index check for block tracker & generate SSR 766 blocksToRawSearch, err := segReq.GetMicroIndexFilter() 767 if err != nil { 768 log.Errorf("qid=%d, failed to get blocks to raw search! Defaulting to searching all blocks. SegKey %+v", segReq.qid, segReq.segKey) 769 blocksToRawSearch = segReq.GetEntireFileMicroIndexFilter() 770 } 771 sTime := time.Now() 772 isQueryPersistent, err := querytracker.IsQueryPersistent([]string{segReq.tableName}, segReq.sNode) 773 if err != nil { 774 log.Errorf("qid=%d, applyAggOpOnSegments: Failed to check if query is persistent! Error: %v", qid, err) 775 } 776 var rawSearchSSR map[string]*structs.SegmentSearchRequest 777 if segReq.sType == structs.SEGMENT_STATS_SEARCH { 778 rawSearchSSR = ExtractSSRFromSearchNode(segReq.sNode, blocksToRawSearch, segReq.queryRange, segReq.indexInfo.GetQueryTables(), qs, segReq.qid, isQueryPersistent, segReq.pqid) 779 } else { 780 rawSearchSSR = metadata.ExtractUnrotatedSSRFromSearchNode(segReq.sNode, segReq.queryRange, segReq.indexInfo.GetQueryTables(), blocksToRawSearch, qs, segReq.qid) 781 } 782 qs.UpdateExtractSSRTime(time.Since(sTime)) 783 784 // rawSearchSSR should be of size 1 or 0 785 for _, req := range rawSearchSSR { 786 sstMap, err = search.RawComputeSegmentStats(req, segReq.parallelismPerFile, segReq.sNode, segReq.segKeyTsRange, segReq.aggs.MeasureOperations, allSegFileResults, qid, qs) 787 if err != nil { 788 log.Errorf("qid=%d, applyAggOpOnSegments : ReadSegStats: Failed to get segment level stats for segKey %+v! Error: %v", qid, segReq.segKey, err) 789 allSegFileResults.AddError(err) 790 } 791 } 792 } 793 err = allSegFileResults.UpdateSegmentStats(sstMap, measureOperations, runningEvalStats) 794 if err != nil { 795 log.Errorf("qid=%d, applyAggOpOnSegments : ReadSegStats: Failed to update segment stats for segKey %+v! Error: %v", qid, segReq.segKey, err) 796 allSegFileResults.AddError(err) 797 continue 798 } 799 totalRecsSearched := uint64(0) 800 if segReq.sType == structs.SEGMENT_STATS_SEARCH { 801 totalRecsSearched = metadata.GetNumOfSearchedRecordsRotated(segReq.segKey) 802 } else if segReq.sType == structs.UNROTATED_SEGMENT_STATS_SEARCH { 803 totalRecsSearched = writer.GetNumOfSearchedRecordsUnRotated(segReq.segKey) 804 } 805 segenc := allSegFileResults.GetAddSegEnc(segReq.segKey) 806 incrementNumFinishedSegments(1, qid, totalRecsSearched, segenc, true, sstMap) 807 } 808 809 if len(sortedQSRSlice) == 0 { 810 incrementNumFinishedSegments(0, qid, 0, 0, true, nil) 811 } 812 } 813 814 // return sorted slice of querySegmentRequests, count of raw search requests, distributed queries, and count of pqs request 815 func getAllSegmentsInQuery(queryInfo *queryInformation, unrotatedGRPC bool, sTime time.Time, orgid uint64) ([]*querySegmentRequest, uint64, uint64, uint64, error) { 816 if unrotatedGRPC { 817 unrotatedQSR, unrotatedRawCount, unrotatedDistQueries, unrotatedPQSCount, err := getAllUnrotatedSegments(queryInfo, unrotatedGRPC, sTime, orgid) 818 if err != nil { 819 return nil, 0, 0, 0, err 820 } 821 sortedQSRSlice := getSortedQSRResult(queryInfo.aggs, unrotatedQSR) 822 return sortedQSRSlice, unrotatedRawCount, unrotatedDistQueries, unrotatedPQSCount, nil 823 } 824 825 unrotatedQSR, unrotatedRawCount, unrotatedDistQueries, unrotatedPQSCount, err := getAllUnrotatedSegments(queryInfo, unrotatedGRPC, sTime, orgid) 826 if err != nil { 827 return nil, 0, 0, 0, err 828 } 829 rotatedQSR, rotatedRawCount, rotatedDistQueries, rotatedPQS, err := getAllRotatedSegmentsInQuery(queryInfo, sTime, orgid) 830 if err != nil { 831 return nil, 0, 0, 0, err 832 } 833 allSegRequests := append(rotatedQSR, unrotatedQSR...) 834 // Sort query segment results depending on aggs 835 sortedQSRSlice := getSortedQSRResult(queryInfo.aggs, allSegRequests) 836 return sortedQSRSlice, rotatedRawCount + unrotatedRawCount, unrotatedDistQueries + rotatedDistQueries, unrotatedPQSCount + rotatedPQS, nil 837 } 838 839 // returns sorted order of querySegmentRequests, count of keys to raw search, count of distributed queries, and count of pqs keys to raw search 840 func getAllRotatedSegmentsInQuery(queryInfo *queryInformation, sTime time.Time, orgid uint64) ([]*querySegmentRequest, uint64, uint64, uint64, error) { 841 // 1. metadata.FilterSegmentsByTime gives epoch range 842 allPossibleKeys, tsPassedCount, totalPossible := metadata.FilterSegmentsByTime(queryInfo.queryRange, queryInfo.indexInfo.GetQueryTables(), orgid) 843 log.Infof("qid=%d, Rotated query time filtering returned %v segment keys to search out of %+v. query elapsed time: %+v", queryInfo.qid, tsPassedCount, 844 totalPossible, time.Since(sTime)) 845 846 qsrs := convertSegKeysToQueryRequests(queryInfo, allPossibleKeys) 847 currNodeQsrs, distributedRequests, err := queryInfo.dqs.DistributeRotatedRequests(queryInfo, qsrs) 848 if err != nil { 849 log.Errorf("qid=%d, Error in distributing rotated requests %+v", queryInfo.qid, err) 850 return nil, 0, 0, 0, err 851 } 852 853 // 2. Whatever needed sorting of segKeys based on sorts & generation into querySegmentRequest 854 qsr, raw, pqs := filterSegKeysToQueryResults(queryInfo, currNodeQsrs) 855 return qsr, raw, distributedRequests, pqs, nil 856 } 857 858 func applyFilterOperatorSingleRequest(qsr *querySegmentRequest, allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) error { 859 switch qsr.sType { 860 case structs.PQS: 861 return applyFilterOperatorPQSRequest(qsr, allSegFileResults, qs) 862 case structs.RAW_SEARCH: 863 return applyFilterOperatorRawSearchRequest(qsr, allSegFileResults, qs) 864 case structs.UNROTATED_PQS: 865 return applyFilterOperatorUnrotatedPQSRequest(qsr, allSegFileResults, qs) 866 case structs.UNROTATED_RAW_SEARCH: 867 return applyFilterOperatorUnrotatedRawSearchRequest(qsr, allSegFileResults, qs) 868 case structs.UNKNOWN: 869 log.Errorf("qid=%d, Got a unknown query segment request! SegKey %+v", qsr.qid, qsr.segKey) 870 } 871 return fmt.Errorf("unsupported segment type %+v", qsr.sType) 872 } 873 874 func applyFilterOperatorPQSRequest(qsr *querySegmentRequest, allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) error { 875 spqmr, err := pqs.GetAllPersistentQueryResults(qsr.segKey, qsr.queryInformation.pqid) 876 if err != nil { 877 qsr.sType = structs.RAW_SEARCH 878 qsr.blkTracker = structs.InitEntireFileBlockTracker() 879 return applyFilterOperatorRawSearchRequest(qsr, allSegFileResults, qs) 880 } 881 err = applyPQSToRotatedRequest(qsr, allSegFileResults, spqmr, qs) 882 if err != nil { 883 qsr.sType = structs.RAW_SEARCH 884 qsr.blkTracker = structs.InitEntireFileBlockTracker() 885 return applyFilterOperatorRawSearchRequest(qsr, allSegFileResults, qs) 886 } 887 888 // Get time range/blocks missing from sqpmr from metadata layer. 889 missingTRange := metadata.GetTSRangeForMissingBlocks(qsr.segKey, qsr.segKeyTsRange, spqmr) 890 if missingTRange == nil || !allSegFileResults.ShouldSearchRange(missingTRange.StartEpochMs, missingTRange.EndEpochMs) { 891 return nil 892 } 893 qsr.sType = structs.RAW_SEARCH 894 qsr.blkTracker = structs.InitExclusionBlockTracker(spqmr) // blocks not found in pqs, that we need to raw search for a key 895 return applyFilterOperatorRawSearchRequest(qsr, allSegFileResults, qs) 896 } 897 898 func applyFilterOperatorRawSearchRequest(qsr *querySegmentRequest, allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) error { 899 // run through micro index check for block tracker & generate SSR 900 blocksToRawSearch, err := qsr.GetMicroIndexFilter() 901 if err != nil { 902 log.Errorf("qid=%d, failed to get blocks to raw search! Defaulting to searching all blocks. SegKey %+v", qsr.qid, qsr.segKey) 903 blocksToRawSearch = qsr.GetEntireFileMicroIndexFilter() 904 } 905 906 isQueryPersistent, err := querytracker.IsQueryPersistent([]string{qsr.tableName}, qsr.sNode) 907 if err != nil { 908 log.Errorf("qid=%d, failed to check if query is persistent", qsr.qid) 909 } 910 911 sTime := time.Now() 912 rawSearchSSR := ExtractSSRFromSearchNode(qsr.sNode, blocksToRawSearch, qsr.queryRange, qsr.indexInfo.GetQueryTables(), qs, qsr.qid, isQueryPersistent, qsr.pqid) 913 qs.UpdateExtractSSRTime(time.Since(sTime)) 914 for _, req := range rawSearchSSR { 915 req.SType = qsr.sType 916 } 917 err = applyFilterOperatorInternal(allSegFileResults, rawSearchSSR, qsr.parallelismPerFile, qsr.sNode, qsr.queryRange, 918 qsr.sizeLimit, qsr.aggs, qsr.qid, qs) 919 920 for _, req := range rawSearchSSR { 921 if req.HasMatchedRrc { 922 qsr.HasMatchedRrc = true 923 break 924 } 925 } 926 return err 927 } 928 929 func applyFilterOperatorUnrotatedPQSRequest(qsr *querySegmentRequest, allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) error { 930 spqmr, err := writer.GetAllPersistentQueryResults(qsr.segKey, qsr.queryInformation.pqid) 931 if err != nil { 932 qsr.sType = structs.UNROTATED_RAW_SEARCH 933 qsr.blkTracker = structs.InitEntireFileBlockTracker() 934 return applyFilterOperatorUnrotatedRawSearchRequest(qsr, allSegFileResults, qs) 935 } 936 err = applyPQSToUnrotatedRequest(qsr, allSegFileResults, spqmr, qs) 937 if err != nil { 938 qsr.sType = structs.UNROTATED_RAW_SEARCH 939 qsr.blkTracker = structs.InitEntireFileBlockTracker() 940 return applyFilterOperatorUnrotatedRawSearchRequest(qsr, allSegFileResults, qs) 941 } 942 943 // If all possible blocks we need to search are in spqmr, no need to raw search anything 944 missingTRange := writer.GetTSRangeForMissingBlocks(qsr.segKey, qsr.segKeyTsRange, spqmr) 945 if missingTRange == nil || !allSegFileResults.ShouldSearchRange(missingTRange.StartEpochMs, missingTRange.EndEpochMs) { 946 return nil 947 } 948 qsr.sType = structs.UNROTATED_RAW_SEARCH 949 qsr.blkTracker = structs.InitExclusionBlockTracker(spqmr) // blocks not found in pqs, that we need to raw search for a key 950 return applyFilterOperatorUnrotatedRawSearchRequest(qsr, allSegFileResults, qs) 951 } 952 953 func applyFilterOperatorUnrotatedRawSearchRequest(qsr *querySegmentRequest, allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) error { 954 // run through micro index check for block tracker & generate SSR 955 blocksToRawSearch, err := qsr.GetMicroIndexFilter() 956 if err != nil { 957 log.Errorf("qid=%d, failed to get blocks to raw search! Defaulting to searching all blocks. SegKey %+v", qsr.qid, qsr.segKey) 958 blocksToRawSearch = qsr.GetEntireFileMicroIndexFilter() 959 } 960 sTime := time.Now() 961 rawSearchSSR := metadata.ExtractUnrotatedSSRFromSearchNode(qsr.sNode, qsr.queryRange, qsr.indexInfo.GetQueryTables(), blocksToRawSearch, qs, qsr.qid) 962 qs.UpdateExtractSSRTime(time.Since(sTime)) 963 for _, req := range rawSearchSSR { 964 req.SType = qsr.sType 965 } 966 err = applyFilterOperatorInternal(allSegFileResults, rawSearchSSR, qsr.parallelismPerFile, qsr.sNode, qsr.queryRange, 967 qsr.sizeLimit, qsr.aggs, qsr.qid, qs) 968 969 for _, req := range rawSearchSSR { 970 if req.HasMatchedRrc { 971 qsr.HasMatchedRrc = true 972 break 973 } 974 } 975 return err 976 } 977 978 // loops over all segrequests and performs raw search 979 func applyFilterOperatorInternal(allSegFileResults *segresults.SearchResults, allSegRequests map[string]*structs.SegmentSearchRequest, 980 parallelismPerFile int64, searchNode *structs.SearchNode, timeRange *dtu.TimeRange, sizeLimit uint64, aggs *structs.QueryAggregators, 981 qid uint64, qs *summary.QuerySummary) error { 982 for _, req := range allSegRequests { 983 search.RawSearchSegmentFileWrapper(req, parallelismPerFile, searchNode, timeRange, sizeLimit, aggs, allSegFileResults, qid, qs) 984 } 985 986 return nil 987 } 988 989 // Returns sorted order of query segment requests, count of keys to raw search 990 func filterAggSegKeysToQueryResults(qInfo *queryInformation, allPossibleKeys map[string]map[string]*dtu.TimeRange, 991 aggs *structs.QueryAggregators, segType structs.SegType) ([]*querySegmentRequest, uint64) { 992 993 allAggSegmentRequests := make([]*querySegmentRequest, 0) 994 aggSearchCount := uint64(0) 995 for tableName, segKeys := range allPossibleKeys { 996 for segKey, tsRange := range segKeys { 997 if tsRange == nil { 998 log.Errorf("qid=%d, filterAggSegKeysToQueryResults reieved an empty segment time range. SegKey %+v", qInfo.qid, segKey) 999 continue 1000 } 1001 qReq := &querySegmentRequest{ 1002 queryInformation: *qInfo, 1003 segKey: segKey, 1004 segKeyTsRange: tsRange, 1005 tableName: tableName, 1006 } 1007 1008 qReq.sType = segType 1009 qReq.blkTracker = structs.InitEntireFileBlockTracker() 1010 aggSearchCount++ 1011 1012 allAggSegmentRequests = append(allAggSegmentRequests, qReq) 1013 } 1014 } 1015 1016 return allAggSegmentRequests, aggSearchCount 1017 } 1018 1019 // Returns sorted order of query segment requests, count of keys to raw search, count of keys in PQS 1020 func filterSegKeysToQueryResults(qInfo *queryInformation, qsegs []*querySegmentRequest) ([]*querySegmentRequest, uint64, uint64) { 1021 pqsCount := uint64(0) 1022 rawSearchCount := uint64(0) 1023 for _, qReq := range qsegs { 1024 if pqs.DoesSegKeyHavePqidResults(qReq.segKey, qInfo.pqid) { 1025 qReq.sType = structs.PQS 1026 pqsCount++ 1027 } else { 1028 qReq.sType = structs.RAW_SEARCH 1029 qReq.blkTracker = structs.InitEntireFileBlockTracker() 1030 rawSearchCount++ 1031 } 1032 } 1033 1034 return qsegs, rawSearchCount, pqsCount 1035 } 1036 1037 func convertSegKeysToQueryRequests(qInfo *queryInformation, allPossibleKeys map[string]map[string]*dtu.TimeRange) []*querySegmentRequest { 1038 allSegRequests := make([]*querySegmentRequest, 0) 1039 for tableName, segKeys := range allPossibleKeys { 1040 for segKey, tsRange := range segKeys { 1041 if tsRange == nil { 1042 log.Errorf("qid=%d, FilterSegKeysToQueryResults reieved an empty segment time range. SegKey %+v", qInfo.qid, segKey) 1043 continue 1044 } 1045 qReq := &querySegmentRequest{ 1046 queryInformation: *qInfo, 1047 segKey: segKey, 1048 segKeyTsRange: tsRange, 1049 tableName: tableName, 1050 } 1051 allSegRequests = append(allSegRequests, qReq) 1052 } 1053 } 1054 1055 return allSegRequests 1056 } 1057 1058 // Returns query segment requests, count of keys to raw search, count of keys in PQS 1059 func filterUnrotatedSegKeysToQueryRequests(qInfo *queryInformation, allPossibleKeys map[string]map[string]*dtu.TimeRange) ([]*querySegmentRequest, uint64, uint64) { 1060 1061 allSegRequests := make([]*querySegmentRequest, 0) 1062 pqsCount := uint64(0) 1063 rawSearchCount := uint64(0) 1064 for tableName, segKeys := range allPossibleKeys { 1065 for segKey, tsRange := range segKeys { 1066 if tsRange == nil { 1067 log.Errorf("qid=%d, FilterSegKeysToQueryResults reieved an empty segment time range. SegKey %+v", qInfo.qid, segKey) 1068 continue 1069 } 1070 qReq := &querySegmentRequest{ 1071 queryInformation: *qInfo, 1072 segKey: segKey, 1073 segKeyTsRange: tsRange, 1074 tableName: tableName, 1075 } 1076 if writer.DoesSegKeyHavePqidResults(segKey, qInfo.pqid) { 1077 qReq.sType = structs.UNROTATED_PQS 1078 pqsCount++ 1079 } else { 1080 qReq.sType = structs.UNROTATED_RAW_SEARCH 1081 qReq.blkTracker = structs.InitEntireFileBlockTracker() 1082 rawSearchCount++ 1083 } 1084 allSegRequests = append(allSegRequests, qReq) 1085 } 1086 } 1087 return allSegRequests, rawSearchCount, pqsCount 1088 } 1089 1090 // gets search metadata for a segKey and runs raw search 1091 func applyPQSToRotatedRequest(qsr *querySegmentRequest, allSearchResults *segresults.SearchResults, spqmr *pqmr.SegmentPQMRResults, qs *summary.QuerySummary) error { 1092 1093 searchMetadata, blkSummaries, err := metadata.GetSearchInfoForPQSQuery(qsr.segKey, spqmr) 1094 if err != nil { 1095 log.Errorf("qid=%d, applyRawSearchToPQSMatches: failed to get search info for pqs query %+v. Error: %+v", 1096 qsr.qid, qsr.segKey, err) 1097 return err 1098 } 1099 1100 return applySinglePQSRawSearch(qsr, allSearchResults, spqmr, searchMetadata, blkSummaries, qs) 1101 } 1102 1103 // gets search metadata for a segKey and runs raw search 1104 func applyPQSToUnrotatedRequest(qsr *querySegmentRequest, allSearchResults *segresults.SearchResults, spqmr *pqmr.SegmentPQMRResults, qs *summary.QuerySummary) error { 1105 1106 searchMetadata, blkSummaries, err := writer.GetSearchInfoForPQSQuery(qsr.segKey, spqmr) 1107 if err != nil { 1108 log.Errorf("qid=%d, applyRawSearchToPQSMatches: failed to get search info for pqs query %+v. Error: %+v", 1109 qsr.qid, qsr.segKey, err) 1110 return err 1111 } 1112 return applySinglePQSRawSearch(qsr, allSearchResults, spqmr, searchMetadata, blkSummaries, qs) 1113 } 1114 1115 func applySinglePQSRawSearch(qsr *querySegmentRequest, allSearchResults *segresults.SearchResults, spqmr *pqmr.SegmentPQMRResults, searchMetadata map[uint16]*structs.BlockMetadataHolder, 1116 blkSummaries []*structs.BlockSummary, qs *summary.QuerySummary) error { 1117 if len(searchMetadata) == 0 { 1118 log.Infof("qid=%d, applyRawSearchToPQSMatches: segKey %+v has 0 blocks in segment PQMR results", qsr.qid, qsr.segKey) 1119 return nil 1120 } 1121 req := &structs.SegmentSearchRequest{ 1122 SegmentKey: qsr.segKey, 1123 VirtualTableName: qsr.tableName, 1124 AllPossibleColumns: qsr.colsToSearch, 1125 AllBlocksToSearch: searchMetadata, 1126 SearchMetadata: &structs.SearchMetadataHolder{ 1127 BlockSummaries: blkSummaries, 1128 SearchTotalMemory: uint64(len(blkSummaries) * 16), // TODO: add bitset size here 1129 }, 1130 } 1131 search.RawSearchPQMResults(req, qsr.parallelismPerFile, qsr.queryRange, qsr.aggs, qsr.sizeLimit, spqmr, allSearchResults, qsr.qid, qs) 1132 1133 if req.HasMatchedRrc { 1134 qsr.HasMatchedRrc = true 1135 } 1136 return nil 1137 } 1138 1139 func applyFopFastPathSingleRequest(qsr *querySegmentRequest, allSegFileResults *segresults.SearchResults, qs *summary.QuerySummary) error { 1140 1141 // run through micro index check for block tracker & generate SSR 1142 blocksToRawSearch, err := qsr.GetMicroIndexFilter() 1143 if err != nil { 1144 log.Errorf("qid=%d, applyFopFastPathSingleRequest failed to get blocks, Defaulting to searching all blocks. SegKey %+v", qsr.qid, qsr.segKey) 1145 blocksToRawSearch = qsr.GetEntireFileMicroIndexFilter() 1146 } 1147 1148 sTime := time.Now() 1149 isQueryPersistent, err := querytracker.IsQueryPersistent([]string{qsr.tableName}, qsr.sNode) 1150 if err != nil { 1151 log.Errorf("qid=%d, applyFopFastPathSingleRequest: Failed to check if query is persistent!", qsr.qid) 1152 } 1153 rawSearchSSR := ExtractSSRFromSearchNode(qsr.sNode, blocksToRawSearch, qsr.queryRange, qsr.indexInfo.GetQueryTables(), qs, qsr.qid, isQueryPersistent, qsr.pqid) 1154 qs.UpdateExtractSSRTime(time.Since(sTime)) 1155 for _, req := range rawSearchSSR { 1156 req.SType = qsr.sType 1157 } 1158 1159 err = applyFopFastPathInternal(allSegFileResults, rawSearchSSR, qsr.parallelismPerFile, qsr.sNode, qsr.queryRange, 1160 qsr.sizeLimit, qsr.aggs, qsr.qid, qs) 1161 1162 for _, req := range rawSearchSSR { 1163 if req.HasMatchedRrc { 1164 qsr.HasMatchedRrc = true 1165 break 1166 } 1167 } 1168 1169 return err 1170 } 1171 1172 // loops over all segrequests and performs raw search 1173 func applyFopFastPathInternal(allSegFileResults *segresults.SearchResults, 1174 allSegRequests map[string]*structs.SegmentSearchRequest, 1175 parallelismPerFile int64, searchNode *structs.SearchNode, timeRange *dtu.TimeRange, 1176 sizeLimit uint64, aggs *structs.QueryAggregators, 1177 qid uint64, qs *summary.QuerySummary) error { 1178 1179 for _, req := range allSegRequests { 1180 search.AggsFastPathWrapper(req, parallelismPerFile, searchNode, timeRange, sizeLimit, 1181 aggs, allSegFileResults, qid, qs) 1182 } 1183 return nil 1184 } 1185 1186 // first bool is existience of non time aggs, second bool is existience of time agg 1187 func checkAggTypes(aggs *structs.QueryAggregators) (bool, bool) { 1188 nonTime := false 1189 timeAgg := false 1190 if aggs != nil { 1191 if aggs.TimeHistogram != nil { 1192 timeAgg = true 1193 } 1194 if aggs.GroupByRequest != nil { 1195 nonTime = true 1196 } 1197 } 1198 return nonTime, timeAgg 1199 }