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

Disable backlog quota check by default. #4320

Merged
merged 5 commits into from
May 29, 2019

Conversation

codelipenghui
Copy link
Contributor

Motivation

Previously we used producer_request_hold as the default backlog quota retention policy. But producer_request_hold is unfriendly for online business, even online business faces big risks. When users ask us to apply pulsar to the online environment, what should be paid attention to? Change the default backlog quota retention policy often mentioned by the first, and initially at zhaopin.com, we have also had such problems.

So, I propose to change the default backlog quota retention policy to consumer_backlog_eviction, from our practice shows that this is a choice for most situations and the most important thing is that this option allows users to avoid taking greater risks.

This change will affect the way our existing users use it. I want to discuss it together to make some trade-offs. If necessary, i can start a email thread.

Modifications

Change broker level default backlog quota retention policy to consumer_backlog_eviction.

If yes was chosen, please highlight the changes

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API: (no)
  • The schema: (no)
  • The default values of configurations: (yes)
  • The wire protocol: (no)
  • The rest endpoints: (no)
  • The admin cli options: (no)
  • Anything that affects deployment: (no)

Documentation

  • Does this pull request introduce a new feature? (no)

Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

LGTM. IMO - I would prefer always writable for an event streaming system.

Copy link
Contributor

@merlimat merlimat left a comment

Choose a reason for hiding this comment

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

All the defaults we have are based on not deleting data unless explicitely asked to do so.

@codelipenghui
Copy link
Contributor Author

@merlimat

If I didn’t understand it wrong, this is just to guarantee that the data that has been written is not be lost. If we look at the user's entire system, pulsar is the important part, serving real-time streaming data. If it becomes unwritable, this is also data loss for the user.

The core of the problem is how the user should handle when the backlog has exceeded the threshold, delete old data or reject new data. For streaming data platforms, i think rejecting new data is basically unacceptable. Users can use the retention mechanism to confirm how long they retain data.

For the backlog, this is for a subscriber. There will be one subscriber's backlog cause all new data to be rejected.

If the default configuration is inclined to not lost data. reject new data should also be considered.

If do not consider resource limitations, not limit the backlog by default can match both two . My suggestion is to prefer the configuration with the least loss to the user as the default configuration.

@srkukarni
Copy link
Contributor

echoing @sijie thoughts, I believe that streaming systems usually prefer writable system. However at the same time I believe queuing systems prefer non-loss of any transactions. Since Pulsar supports both, this is a tricky issue.

@merlimat
Copy link
Contributor

@codelipenghui

If I didn’t understand it wrong, this is just to guarantee that the data that has been written is not be lost. If we look at the user's entire system, pulsar is the important part, serving real-time streaming data. If it becomes unwritable, this is also data loss for the user.

Of course if the system is not-writable, the application will be impacted, though the main difference is that what was "acked" before is not lost.

Applications can apply fallback strategies when a downstream system is unavailable, from the simplest (fail user request) to more complex (degraded functioning mode), but they won't be able to know if "acked" data was dropped by the system. There would be no immediate way to communicate that to users, or to apply other strategies.

The core of the problem is how the user should handle when the backlog has exceeded the threshold, delete old data or reject new data. For streaming data platforms, i think rejecting new data is basically unacceptable. Users can use the retention mechanism to confirm how long they retain data.

The default option can be set in broker.conf and overridden per namespace, to adjust to needs.

For the backlog, this is for a subscriber. There will be one subscriber's backlog cause all new data to be rejected.

Without information around what's the importance of the subscriptions, it's difficult to generalize the judgement of which subscription should retain data.

Subscriptions are a way to explicitly retain data in the system.

If the default configuration is inclined to not lost data. reject new data should also be considered.

That's what the default setting is.

@srkukarni

I believe that streaming systems usually prefer writable system. However at the same time I believe queuing systems prefer non-loss of any transactions. Since Pulsar supports both, this is a tricky issue.

That also comes with the dichotomy between Consumer vs Reader.
In streaming mode, to me, a Reader makes more sense in that it does imply data retention, therefore it would never incur in a backlog quota violation.

@codelipenghui

If do not consider resource limitations, not limit the backlog by default can match both two . My suggestion is to prefer the configuration with the least loss to the user as the default configuration.

Yes, the default quota setting is very arbitrary (and very low). I'd be fine to disable backlog quota check by default. It will simplify many "gotchas", though that won't remove the reality that disk space is finite (even if most likely > 10GB), and applications need to reason about what to do when the limit is reached.

@sijie
Copy link
Member

sijie commented May 25, 2019

I'd be fine to disable backlog quota check by default.

I think this is a good tradeoff for satisfying both sides. @codelipenghui ?

@codelipenghui
Copy link
Contributor Author

@sijie yes, agree.

@sijie
Copy link
Member

sijie commented May 27, 2019

@codelipenghui can you make the change based on Matteo's comment?

@sijie sijie added area/config doc Your PR contains doc changes, no matter whether the changes are in markdown or code files. labels May 27, 2019
@codelipenghui
Copy link
Contributor Author

@sijie I'm already disable backlog quota check by default.

@codelipenghui codelipenghui changed the title Change default backlog quota retention policy to consumer_backlog_eviction Disable backlog quota check by default. May 28, 2019
@codelipenghui
Copy link
Contributor Author

run Java8 Tests
run Integration Tests

conf/broker.conf Outdated
@@ -68,13 +68,13 @@ zooKeeperOperationTimeoutSeconds=30
brokerShutdownTimeoutMs=60000

# Enable backlog quota check. Enforces action on topic when the quota is reached
backlogQuotaCheckEnabled=true
backlogQuotaCheckEnabled=false
Copy link
Contributor

Choose a reason for hiding this comment

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

This will also disable custom quota checks on namespaces. I think we should have a way to set backlogQuotaDefaultLimitGB to -1 for disabling the quota check by default. That would require some small adjustment in the code check.

@codelipenghui
Copy link
Contributor Author

@merlimat I'm already address your comment. PTAL.

@merlimat merlimat added this to the 2.4.0 milestone May 29, 2019
@merlimat merlimat merged commit 84abb6c into apache:master May 29, 2019
@codelipenghui codelipenghui deleted the config_broker_backlog_quota branch May 30, 2019 10:25
sijie pushed a commit that referenced this pull request Jan 18, 2020
### Motivation
Some parameters are added in the `broker.conf` and `standalone.conf` files. However, those parameters are not updated in the docs.
See the following PRs for details: #4150, #4066, #4197, #3819, #4261, #4273, #4320.

### Modifications
Add those parameter info, and sync docs with the code.

Does not update the description quite much, there are two reasons for this:
1. Keep doc content consistent with code. We need to update the description for those parameters in the code first, and then sync them in docs.
2. Will adopt a generator to generate those content automatically in the near future.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
### Motivation
Some parameters are added in the `broker.conf` and `standalone.conf` files. However, those parameters are not updated in the docs.
See the following PRs for details: apache#4150, apache#4066, apache#4197, apache#3819, apache#4261, apache#4273, apache#4320.

### Modifications
Add those parameter info, and sync docs with the code.

Does not update the description quite much, there are two reasons for this:
1. Keep doc content consistent with code. We need to update the description for those parameters in the code first, and then sync them in docs.
2. Will adopt a generator to generate those content automatically in the near future.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/config doc Your PR contains doc changes, no matter whether the changes are in markdown or code files.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants