@@ -100,14 +100,29 @@ type Series struct {
100
100
Node cluster.Node
101
101
}
102
102
103
- func (s * Server ) findSeries (ctx context.Context , orgId uint32 , patterns []string , seenAfter int64 ) ([]Series , error ) {
104
- data := models.IndexFind {
105
- Patterns : patterns ,
106
- OrgId : orgId ,
107
- From : seenAfter ,
103
+ func (s * Server ) findSeries (ctx context.Context , orgId uint32 , patterns []string , seenAfter int64 , maxSeries int ) ([]Series , error ) {
104
+
105
+ fetchFn := func (reqCtx context.Context , peer cluster.Node , peerGroups map [int32 ][]cluster.Node ) (interface {}, error ) {
106
+ ourParts := len (peer .GetPartitions ())
107
+
108
+ // assign a fractional maxSeries limit (not global, but relative to how much data the peer has)
109
+ // look at each shardgroup and check how many partitions it has
110
+ // (we assume each shardgroup is consistent across different peers for that shardgroup)
111
+ var totalParts int
112
+ for _ , otherPeers := range peerGroups {
113
+ if len (otherPeers ) > 0 {
114
+ totalParts += len (otherPeers [0 ].GetPartitions ())
115
+ }
116
+ }
117
+ data := models.IndexFind {
118
+ Patterns : patterns ,
119
+ OrgId : orgId ,
120
+ From : seenAfter ,
121
+ Limit : int64 (maxSeries * ourParts / totalParts ),
122
+ }
123
+ return peer .Post (reqCtx , "findSeriesRemote" , "/index/find" , data )
108
124
}
109
-
110
- resps , err := s .queryAllShards (ctx , data , "findSeriesRemote" , "/index/find" )
125
+ resps , err := s .queryAllShards (ctx , "findSeriesRemote" , fetchFn )
111
126
if err != nil {
112
127
return nil , err
113
128
}
@@ -121,12 +136,23 @@ func (s *Server) findSeries(ctx context.Context, orgId uint32, patterns []string
121
136
series := make ([]Series , 0 )
122
137
resp := models.IndexFindResp {}
123
138
for _ , r := range resps {
139
+ if len (series ) == maxSeries {
140
+ return nil , response .NewError (
141
+ http .StatusRequestEntityTooLarge ,
142
+ fmt .Sprintf ("Request exceeds max-series-per-req limit (%d). Reduce the number of targets or ask your admin to increase the limit." , maxSeriesPerReq ))
143
+ }
124
144
_ , err = resp .UnmarshalMsg (r .buf )
125
145
if err != nil {
126
146
return nil , err
127
147
}
128
148
129
149
for pattern , nodes := range resp .Nodes {
150
+ if len (series ) == maxSeries {
151
+ return nil , response .NewError (
152
+ http .StatusRequestEntityTooLarge ,
153
+ fmt .Sprintf ("Request exceeds max-series-per-req limit (%d). Reduce the number of targets or ask your admin to increase the limit." , maxSeriesPerReq ))
154
+ }
155
+
130
156
series = append (series , Series {
131
157
Pattern : pattern ,
132
158
Node : r .peer ,
@@ -330,7 +356,7 @@ func (s *Server) metricsFind(ctx *middleware.Context, request models.GraphiteFin
330
356
}
331
357
nodes := make ([]idx.Node , 0 )
332
358
reqCtx := ctx .Req .Context ()
333
- series , err := s .findSeries (reqCtx , ctx .OrgId , []string {request .Query }, int64 (fromUnix ))
359
+ series , err := s .findSeries (reqCtx , ctx .OrgId , []string {request .Query }, int64 (fromUnix ), maxSeriesPerReq )
334
360
if err != nil {
335
361
response .Write (ctx , response .WrapError (err ))
336
362
return
@@ -378,7 +404,7 @@ func (s *Server) metricsExpand(ctx *middleware.Context, request models.GraphiteE
378
404
for i , query := range request .Query {
379
405
i , query := i , query
380
406
g .Go (func () error {
381
- series , err := s .findSeries (errGroupCtx , ctx .OrgId , []string {query }, 0 )
407
+ series , err := s .findSeries (errGroupCtx , ctx .OrgId , []string {query }, 0 , maxSeriesPerReq )
382
408
if err != nil {
383
409
return err
384
410
}
@@ -795,7 +821,10 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan *expr.Plan)
795
821
}
796
822
series , err = s .clusterFindByTag (ctx , orgId , exprs , int64 (r .From ), maxSeriesPerReq - int (reqs .cnt ), false )
797
823
} else {
798
- series , err = s .findSeries (ctx , orgId , []string {r .Query }, int64 (r .From ))
824
+ // find limit is the limit minus what we already consumed for other targets, adjusted for how many rawReqs we folded into this resolveSeriesRequest
825
+ // note that this doesn't account for duplicate requests like target=foo&target=foo because those are both represented by the same rawReq (see NewPlan)
826
+ findLimit := (maxSeriesPerReq - int (reqs .cnt )) / len (rawReqs )
827
+ series , err = s .findSeries (ctx , orgId , []string {r .Query }, int64 (r .From ), findLimit )
799
828
}
800
829
if err != nil {
801
830
return nil , meta , err
@@ -1046,7 +1075,7 @@ func (s *Server) clusterTagDetails(ctx context.Context, orgId uint32, tag, filte
1046
1075
result := make (map [string ]uint64 )
1047
1076
1048
1077
data := models.IndexTagDetails {OrgId : orgId , Tag : tag , Filter : filter }
1049
- resps , err := s .queryAllShards (ctx , data , "clusterTagDetails" , "/index/tag_details" )
1078
+ resps , err := s .queryAllShards (ctx , "clusterTagDetails" , fetchFuncPost ( data , "clusterTagDetails" , "/index/tag_details" ) )
1050
1079
if err != nil {
1051
1080
return nil , err
1052
1081
}
@@ -1147,7 +1176,7 @@ func (s *Server) clusterFindByTag(ctx context.Context, orgId uint32, expressions
1147
1176
newCtx , cancel := context .WithCancel (ctx )
1148
1177
defer cancel ()
1149
1178
responseChan , errorChan := s .queryAllShardsGeneric (newCtx , "clusterFindByTag" ,
1150
- func (reqCtx context.Context , peer cluster.Node ) (interface {}, error ) {
1179
+ func (reqCtx context.Context , peer cluster.Node , peerGroups map [ int32 ][]cluster. Node ) (interface {}, error ) {
1151
1180
resp := models.IndexFindByTagResp {}
1152
1181
body , err := peer .PostRaw (reqCtx , "clusterFindByTag" , "/index/find_by_tag" , data )
1153
1182
if body == nil || err != nil {
@@ -1222,7 +1251,7 @@ func (s *Server) graphiteTags(ctx *middleware.Context, request models.GraphiteTa
1222
1251
1223
1252
func (s * Server ) clusterTags (ctx context.Context , orgId uint32 , filter string ) ([]string , error ) {
1224
1253
data := models.IndexTags {OrgId : orgId , Filter : filter }
1225
- resps , err := s .queryAllShards (ctx , data , "clusterTags" , "/index/tags" )
1254
+ resps , err := s .queryAllShards (ctx , "clusterTags" , fetchFuncPost ( data , "clusterTags" , "/index/tags" ) )
1226
1255
if err != nil {
1227
1256
return nil , err
1228
1257
}
@@ -1274,7 +1303,7 @@ func (s *Server) clusterAutoCompleteTags(ctx context.Context, orgId uint32, pref
1274
1303
tagSet := make (map [string ]struct {})
1275
1304
1276
1305
data := models.IndexAutoCompleteTags {OrgId : orgId , Prefix : prefix , Expr : expressions , Limit : limit }
1277
- responses , err := s .queryAllShards (ctx , data , "clusterAutoCompleteTags" , "/index/tags/autoComplete/tags" )
1306
+ responses , err := s .queryAllShards (ctx , "clusterAutoCompleteTags" , fetchFuncPost ( data , "clusterAutoCompleteTags" , "/index/tags/autoComplete/tags" ) )
1278
1307
if err != nil {
1279
1308
return nil , err
1280
1309
}
@@ -1321,7 +1350,7 @@ func (s *Server) clusterAutoCompleteTagValues(ctx context.Context, orgId uint32,
1321
1350
valSet := make (map [string ]struct {})
1322
1351
1323
1352
data := models.IndexAutoCompleteTagValues {OrgId : orgId , Tag : tag , Prefix : prefix , Expr : expressions , Limit : limit }
1324
- responses , err := s .queryAllShards (ctx , data , "clusterAutoCompleteValues" , "/index/tags/autoComplete/values" )
1353
+ responses , err := s .queryAllShards (ctx , "clusterAutoCompleteValues" , fetchFuncPost ( data , "clusterAutoCompleteValues" , "/index/tags/autoComplete/values" ) )
1325
1354
if err != nil {
1326
1355
return nil , err
1327
1356
}
@@ -1352,7 +1381,7 @@ func (s *Server) clusterAutoCompleteTagValues(ctx context.Context, orgId uint32,
1352
1381
1353
1382
func (s * Server ) graphiteTagTerms (ctx * middleware.Context , request models.GraphiteTagTerms ) {
1354
1383
data := models.IndexTagTerms {OrgId : ctx .OrgId , Tags : request .Tags , Expr : request .Expr }
1355
- responses , err := s .queryAllShards (ctx .Req .Context (), data , "graphiteTagTerms" , "/index/tags/terms" )
1384
+ responses , err := s .queryAllShards (ctx .Req .Context (), "graphiteTagTerms" , fetchFuncPost ( data , "graphiteTagTerms" , "/index/tags/terms" ) )
1356
1385
if err != nil {
1357
1386
response .Write (ctx , response .WrapErrorForTagDB (err ))
1358
1387
return
0 commit comments