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

feat: add support for dml_rate_limit #19679

Merged
merged 8 commits into from
Dec 5, 2024
Merged

feat: add support for dml_rate_limit #19679

merged 8 commits into from
Dec 5, 2024

Conversation

zwang28
Copy link
Contributor

@zwang28 zwang28 commented Dec 4, 2024

I hereby agree to the terms of the RisingWave Labs, Inc. Contributor License Agreement.

What's changed and what's your intention?

close #13801

The implementation is similar to SourceExecutor's rate limit.

The key difference is, when altering the DML rate limit, i.e. Mutation::Throttle, this PR updates the data stream's rate limiter in place, rather than recreating it as done in SourceExecutor.

  • The problem with recreating data stream is, the old data stream may still contain data chunks resutled from previous chunk splitting, recreating will lose them.
  • This won't cause problem for SourceExecutor because it can still read those data from persistent store after recreating.
  • This is a problem for DmlExecutor because those data chunks have already been removed from the DML data channel.

Checklist

  • I have written necessary rustdoc comments
  • I have added necessary unit tests and integration tests
  • I have added test labels as necessary. See details.
  • I have added fuzzing tests or opened an issue to track them. (Optional, recommended for new SQL features Sqlsmith: Sql feature generation #7934).
  • My PR contains breaking changes. (If it deprecates some features, please create a tracking issue to remove them in the future).
  • All checks passed in ./risedev check (or alias, ./risedev c)
  • My PR changes performance-critical code. (Please run macro/micro-benchmarks and show the results.)
  • My PR contains critical fixes that are necessary to be merged into the latest release. (Please check out the details)

Documentation

  • My PR needs documentation updates. (Please use the Release note section below to summarize the impact on users)

Release note

If this PR includes changes that directly affect users or other significant modifications relevant to the community, kindly draft a release note to provide a concise summary of these changes. Please prioritize highlighting the impact these changes will have on users.

@zwang28 zwang28 requested review from kwannoel and chenzl25 December 4, 2024 15:47
@zwang28 zwang28 force-pushed the wangzheng/dml_rate_limit branch from be07576 to 4627f11 Compare December 4, 2024 15:50
Copy link
Contributor

@kwannoel kwannoel left a comment

Choose a reason for hiding this comment

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

Can we add a test for DML rate limit?

Something like:

  1. Set dml rate limit on table + parallelism=1.

  2. Trigger DML statement in background

    system ok
    risedev psql -c 'insert into ... values ...' &
    
  3. Sleep 10s

  4. check number of rows in table ~10.

Run the test only in non-madsim context.

Copy link
Contributor

@chenzl25 chenzl25 left a comment

Choose a reason for hiding this comment

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

Rest LGTM!

Comment on lines +171 to +176
let reader = apply_dml_rate_limit(
handle.stream_reader().into_stream(),
self.rate_limiter.clone(),
)
.boxed()
.map_err(StreamExecutorError::from);
Copy link
Contributor

Choose a reason for hiding this comment

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

Just note that the current approach puts the rate limiter in the input of dml executor instead of the output. There are some differences between these two places. Limiting the input means some transaction End or Rollback messages might be delayed, and the dml executor actually will buffer the active (uncommitted) transaction (unless it is found that the transaction is too large to buffer). Anyway, it is acceptable to me.

} else {
false
};
if chunk_size == limit + 1 && ends_with_update {
Copy link
Contributor

Choose a reason for hiding this comment

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

What if the limit is 1 and the chunk is [-U, +U]

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The chunk of size 2 can successfully obtain 1 permit from the rate limiter.
Actually this is the code snippet from #16407. It's been moved to utils.rs for reusability.

@zwang28 zwang28 requested a review from kwannoel December 5, 2024 04:44
Copy link
Contributor

@kwannoel kwannoel left a comment

Choose a reason for hiding this comment

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

LGTM for now. But IMO dml rate limit should be able to break the atomicity. In some cases, the user may insert a large number of records in a single insert statement.

After all persistence is only guaranteed by flush.

@zwang28 zwang28 enabled auto-merge December 5, 2024 06:07
@zwang28 zwang28 added this pull request to the merge queue Dec 5, 2024
Merged via the queue into main with commit f0a91df Dec 5, 2024
33 of 34 checks passed
@zwang28 zwang28 deleted the wangzheng/dml_rate_limit branch December 5, 2024 07:48
zwang28 added a commit that referenced this pull request Dec 6, 2024
@zwang28
Copy link
Contributor Author

zwang28 commented Dec 11, 2024

LGTM for now. But IMO dml rate limit should be able to break the atomicity.

Agree.

In some cases, the user may insert a large number of records in a single insert statement.

The atomicity will be broken after processing MAX_CHUNK_FOR_ATOMICITY chunks. Since MAX_CHUNK_FOR_ATOMICITY is hard-coded to 32, I think it's good for now.

Copy link
Contributor

Hi, there.

📝 Telemetry Reminder:
If you're implementing this feature, please consider adding telemetry metrics to track its usage. This helps us understand how the feature is being used and improve it further.
You can find the function report_event of telemetry reporting in the following files. Feel free to ask questions if you need any guidance!

  • src/frontend/src/telemetry.rs
  • src/meta/src/telemetry.rs
  • src/stream/src/telemetry.rs
  • src/storage/compactor/src/telemetry.rs
    Or calling report_event_common (src/common/telemetry_event/src/lib.rs) as if finding it hard to implement.
    ✨ Thank you for your contribution to RisingWave! ✨

This is an automated comment created by the peaceiris/actions-label-commenter. Responding to the bot or mentioning it won't have any effect.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Add rate limit for DML executor
5 participants