-
Notifications
You must be signed in to change notification settings - Fork 3.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
HBASE-24603: Make Zookeeper sync() call synchronous #1945
Conversation
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
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.
A couple of comments. Once resolved, the patch seems good to go unless some stress testing it planned.
Regardless, this change is required anyways.
this.recoverableZooKeeper.sync(path, (i, s, o) -> latch.countDown(), null); | ||
try { | ||
if (!latch.await(zkSyncTimeout, TimeUnit.MILLISECONDS)) { | ||
LOG.error("sync() operation to ZK timed out. Configured timeout: {}ms. This usually points " |
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.
IMHO error
is fine but better to keep it warn
. We anyways have warn
for InterruptedException
also.
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.
This should be WARN.
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.
Any particular reason this has to be WARN? This can potentially be a correctness issue at this point and hence I chose the higher log level.
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.
WARN implies to me as an operator that something bad happened that is noteworthy but not an emergency. ERROR implies a serious problem that probably should cause someone to be paged.
sync is going to occasionally time out. We should be handling such without correctness problems. ERROR logging doesn't substitute for handling the timeout properly. So, assuming we are handling timeouts correctly, WARN level logging is most appropriate.
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.
Fair point. Changed.
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Show resolved
Hide resolved
Test failures are relevant, I just ran |
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Show resolved
Hide resolved
this.recoverableZooKeeper.sync(path, (i, s, o) -> latch.countDown(), null); | ||
try { | ||
if (!latch.await(zkSyncTimeout, TimeUnit.MILLISECONDS)) { | ||
LOG.error("sync() operation to ZK timed out. Configured timeout: {}ms. This usually points " |
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.
This should be WARN.
💔 -1 overall
This message was automatically generated. |
1 similar comment
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
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.
Rebased to include the docker fix that went in overnight.
🎊 +1 overall
This message was automatically generated. |
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.
Looks good overall, using SingleThreadExecutor should be good as long as ZK client already serializes events:
while (true) {
Object event = waitingEvents.take();
if (event == eventOfDeath) {
wasKilled = true;
} else {
processEvent(event);
}
....
....
}
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Show resolved
Hide resolved
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Outdated
Show resolved
Hide resolved
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Show resolved
Hide resolved
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread.
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Show resolved
Hide resolved
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
Show resolved
Hide resolved
Approved, assuming a clean precommit prior to merge. |
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.
+1
🎊 +1 overall
This message was automatically generated. |
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f)
🎊 +1 overall
This message was automatically generated. |
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f) (cherry picked from commit 2379a25)
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f) (cherry picked from commit 2379a25)
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f)
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f)
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f) (cherry picked from commit 2379a25)
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f)
Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f)
…ache#1975) Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f)
…ache#1975) Author: Andrew Purtell Reason: Improvement Ref: CDPD-15964 Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> Change-Id: Iea2a2e7ae9d64d5e5349fe7c4ba7eda92da74a65 (cherry picked from commit 9e06a48)
…ache#1975) Writing a test for this is tricky. There is enough coverage for functional tests. Only concern is performance, but there is enough logging for it to detect timed out/badly performing sync calls. Additionally, this patch decouples the ZK event processing into it's own thread rather than doing it in the EventThread's context. That avoids deadlocks and stalls of the event thread. Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> (cherry picked from commit 84e246f) (cherry picked from commit 100d794) Change-Id: Iea2a2e7ae9d64d5e5349fe7c4ba7eda92da74a65
Writing a test for this is tricky. There is enough coverage for
functional tests. Only concern is performance, but there is enough
logging for it to detect timed out/badly performing sync calls.