Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Initial query limit overriding #3090

Merged
merged 89 commits into from
Jan 21, 2021
Merged

Initial query limit overriding #3090

merged 89 commits into from
Jan 21, 2021

Conversation

rallen090
Copy link
Collaborator

@rallen090 rallen090 commented Jan 14, 2021

What this PR does / why we need it:

To support dynamically overriding query limit values.

E.g. when running

curl -vvvsSf -X POST 0.0.0.0:7201/kvstore -d '{"ry.limits","value":{"maxRecentlyQueriedSeriesBlocks":{"limit":1,"lookbackSeconds":15,"forceExceeded":true}},"commit":true}'

this then results in forced query abandoning
image

Special notes for your reviewer:

Does this PR introduce a user-facing and/or backwards incompatible change?:


Does this PR require updating code package or user-facing documentation?:


Copy link
Collaborator

@ryanhall07 ryanhall07 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add a test?

src/dbnode/network/server/tchannelthrift/node/service.go Outdated Show resolved Hide resolved
src/dbnode/storage/limits/query_limits.go Outdated Show resolved Hide resolved
src/dbnode/storage/limits/query_limits.go Outdated Show resolved Hide resolved
@codecov
Copy link

codecov bot commented Jan 14, 2021

Codecov Report

Merging #3090 (e07440d) into master (9387826) will increase coverage by 0.3%.
The diff coverage is 39.4%.

Impacted file tree graph

@@            Coverage Diff            @@
##           master    #3090     +/-   ##
=========================================
+ Coverage    71.9%    72.2%   +0.3%     
=========================================
  Files        1083     1084      +1     
  Lines      100592   100162    -430     
=========================================
+ Hits        72371    72399     +28     
+ Misses      23200    22722    -478     
- Partials     5021     5041     +20     
Flag Coverage Δ
aggregator 75.8% <ø> (+3.7%) ⬆️
cluster 84.8% <ø> (ø)
collector 84.3% <ø> (ø)
dbnode 78.7% <49.6%> (-0.2%) ⬇️
m3em 74.4% <ø> (ø)
m3ninx 73.2% <ø> (+<0.1%) ⬆️
metrics 20.0% <ø> (ø)
msg 74.0% <ø> (ø)
query 67.2% <23.4%> (+0.2%) ⬆️
x 80.5% <ø> (+0.2%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.


Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 9387826...e07440d. Read the comment docs.

@@ -595,7 +595,6 @@ func (r *blockRetriever) streamRequest(
shard uint32,
id ident.ID,
startTime time.Time,
nsCtx namespace.Context,
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Linter was complaining

kvStore kv.Store,
update *KeyValueUpdate,
) (*KeyValueUpdateResult, error) {
old, err := kvStore.Get(update.Key)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Allowing arbitrary kv updates here is a little scary; seems the footgunning potential may be quite high, although forcing the Commit flag may be enough for safety

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well so they have to conform to the type of input (e.g. the key for querylimits must be as kvpb.QueryLimits{}). And yeah I think the Commit allows you to safely confirm the diff which we return in the result before applying.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we use CheckAndSet instead to require the client to know what version is being overridden?

Copy link
Collaborator

@arnikola arnikola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm

Copy link
Collaborator

@wesleyk wesleyk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Awesome! Still LG, couple more suggestions from the latest changes but feel free to address separately

@@ -53,6 +53,16 @@ per second safely with your deployment and you want to use the default lookback
of `15s` then you would multiply 10,000 by 15 to get 150,000 as a max value with
a 15s lookback.

The third limit `maxRecentlyQueriedSeriesDiskRead` caps the bytes associated with
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we make each of the limits a subsection so it's clear what each section pertains to?

`maxRecentlyQueriedSeriesBlocks`, which also limits the memory cost of specific series
matched in-memory, because of an inefficiency in how allocations would occur even for series
known to not be present on disk for a given shard. This inefficiency has been resolved
https://github.com/m3db/m3/pull/3103 and therefore this limit should be tracking memory cost
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we make this a link like "resolved with this PR and..."

site/content/operational_guide/resource_limits.md Outdated Show resolved Hide resolved
src/dbnode/server/server.go Outdated Show resolved Hide resolved
kvStore kv.Store,
update *KeyValueUpdate,
) (*KeyValueUpdateResult, error) {
old, err := kvStore.Get(update.Key)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we use CheckAndSet instead to require the client to know what version is being overridden?

},
},
commit: true,
expectedJSON: `maxRecentlyQueriedSeriesBlocks:<limit:1 lookbackSeconds:15 forceExceeded:true > `,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

out of curiosity what's up with the formatting at the end (true > )

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh so the true is the val of forceExceeded and the > is the close bracket for the json.RawMessage

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(was more curious why the spacing ends up like that instead of the beginning, which doesn't have any spaces, not really important)

@rallen090 rallen090 merged commit 188655d into master Jan 21, 2021
@rallen090 rallen090 deleted the ra/dynamic-limits branch January 21, 2021 16:04
soundvibe added a commit that referenced this pull request Jan 22, 2021
* master:
  [DOCS] Update to cluster docs (#3084)
  [dbnode][coordinator] Ensure docs limit is propagated for search and aggregate RPCs (#3108)
  [query] Take bounds into account for list endpoints (#3110)
  Add warning to changing blocksize (#3096)
  Add support for dynamic query limit overriding (#3090)
  [tests] test setups exported to allow us to use it from other packages (#3042)
  [query] Implemented Graphite's pow function (#3048)
  [dbnode] Direct conversion of encoded tags to doc.Metadata (#3087)
  [tests] Skip flaky TestWatchNoLeader (#3106)
  [dbnode] Faster search of tag bytes in convert.FromSeriesIDAndTags (#3075)
  Replace bytes.Compare() == 0 with bytes.Equal() (#3101)
  Capture seekerMgr instead Rlock (#3104)
  [m3db] Check bloom filter before stream request allocation (#3103)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants