-
Notifications
You must be signed in to change notification settings - Fork 7.3k
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
ZOOKEEPER-4801: Add payload size limitation for committed log #2130
base: master
Are you sure you want to change the base?
ZOOKEEPER-4801: Add payload size limitation for committed log #2130
Conversation
# Conflicts: # zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java # zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java
@kezhuw This one if waiting for 3.9.3 release. Please help in review. |
@@ -92,9 +93,16 @@ public class ZKDatabase { | |||
public static final String COMMIT_LOG_COUNT = "zookeeper.commitLogCount"; | |||
public static final int DEFAULT_COMMIT_LOG_COUNT = 500; | |||
public int commitLogCount; | |||
public static final String COMMIT_LOG_SIZE = "zookeeper.commitLogSize"; | |||
public static final double DEFAULT_COMMIT_LOG_SIZE = Runtime.getRuntime().maxMemory() * 0.2; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it evaluated as bytes ? Should it better to be long
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we are going to have two metrics to limit committedLog
. Should we design official approach to turn one of them but not all ? For example, 0
to disable limitation. But enable count limitation in case of all got disabled. We cloud use the maximum to disable them currently, but that might be unfriendly. I am ok for it to be leftover.
Besides, should we add metrics to track varying commit log count and commit log bytes ? Also, I propose to change the name from commitLogSize
to commitLogBytes
. I checked ServerMetrics
that all _SIZE
s are size of some kind of collections.
@@ -151,7 +158,24 @@ public ZKDatabase(FileTxnSnapLog snapLog) { | |||
DEFAULT_COMMIT_LOG_COUNT); | |||
commitLogCount = DEFAULT_COMMIT_LOG_COUNT; | |||
} | |||
LOG.info("{}={}", COMMIT_LOG_COUNT, commitLogCount); | |||
try { | |||
commitLogSize = Double.parseDouble( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
backlog: I think we cloud introduce something similar to Time::parseTimeInterval in a later time. The size may look large in case of bytes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense to me and I think it's doable in this PR too since we're already touching this part. The parser is already merged and could be easily shared.
@@ -137,12 +145,11 @@ public ZKDatabase(FileTxnSnapLog snapLog) { | |||
commitLogCount = Integer.parseInt( | |||
System.getProperty(COMMIT_LOG_COUNT, | |||
Integer.toString(DEFAULT_COMMIT_LOG_COUNT))); | |||
if (commitLogCount < DEFAULT_COMMIT_LOG_COUNT) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The default value is 500 which is the recommended minimum. (https://zookeeper.apache.org/doc/r3.9.2/zookeeperAdmin.html)
I do think 500
is pretty small for ZooKeeper already. It is meaningless to support 1
as minimum. Is it better to rename DEFAULT_COMMIT_LOG_COUNT
to MIN_COMMIT_LOG_COUNT
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What would be the default then?
@@ -92,9 +93,16 @@ public class ZKDatabase { | |||
public static final String COMMIT_LOG_COUNT = "zookeeper.commitLogCount"; | |||
public static final int DEFAULT_COMMIT_LOG_COUNT = 500; | |||
public int commitLogCount; | |||
public static final String COMMIT_LOG_SIZE = "zookeeper.commitLogSize"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should document this in zookeeperAdmin.md
.
/** | ||
* committedLog bytes size. | ||
*/ | ||
private AtomicLong totalCommittLogSize = new AtomicLong(0); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
typo: Should be totalCommitLogSize
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
And long
is enough ?
minCommittedLog = 0; | ||
maxCommittedLog = 0; | ||
} else { | ||
minCommittedLog = p.getZxid(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should be peek.getZxid()
?
} | ||
PureRequestProposal p = new PureRequestProposal(request); | ||
committedLog.add(p); | ||
maxCommittedLog = p.getZxid(); | ||
totalCommittLogSize.addAndGet(request.getApproximateSize()); | ||
while (committedLog.size() > 0 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
!committedLog.isEmpty()
?
@@ -151,7 +158,24 @@ public ZKDatabase(FileTxnSnapLog snapLog) { | |||
DEFAULT_COMMIT_LOG_COUNT); | |||
commitLogCount = DEFAULT_COMMIT_LOG_COUNT; | |||
} | |||
LOG.info("{}={}", COMMIT_LOG_COUNT, commitLogCount); | |||
try { | |||
commitLogSize = Double.parseDouble( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will Double.parseDouble()
accept floating point numbers from the config?
Shouldn't we prevent that?
This PR is based on #2115, please merge it first.