[ 
https://issues.apache.org/jira/browse/BOOKKEEPER-675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13761972#comment-13761972
 ] 

Ivan Kelly commented on BOOKKEEPER-675:
---------------------------------------

{quote}
Please create a jira for the TODO in the patch;
{quote}
Will do

{quote}
This change "+ channels.clear();" seems to be unrelated to this patch;
{quote}
If channels isn't cleared after all channels are closed, the bookie watcher 
will try to close them when it looses all bookies (when zkclient goes down). 
The executor may be shutdown by that point, so this spams the logs with junk.

{quote}
You have changed some error messages to debug, like those in 
PerChannelBookieClient. They seem to represent some error condition or at least 
an error condition in a bookie. As such, they might impact the execution of the 
client and I would think that the severity level should be at least WARN.{quote}
In production logs, these messages are almost meaningless, and they spam the 
logs. Lets say you're writing at 300 requests per second and a bookie dies. It 
takes 5 seconds to detect. Thats 1500 log messages. And what do they tell you? 
That a bookie died. Not that a write failed. You have many other less spammy 
means to detect bookie death. It this shouldn't affect the client. The client 
is built to withstand such failures. So really you've written 1500 WARN log 
messages, and possibly woken up the service engineer for nothing. In practice, 
messages like this just get put on an exclude list. If the write of the entry 
fails to write to quorum we should log it. But then we should only log the 
first entry to fail for that ledger, as I've done in the patch, and only when 
it fails completely, not just to one replica.
                
> Log noise fixup before cutting 4.2.2
> ------------------------------------
>
>                 Key: BOOKKEEPER-675
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-675
>             Project: Bookkeeper
>          Issue Type: Wish
>            Reporter: Ivan Kelly
>            Assignee: Ivan Kelly
>             Fix For: 4.2.2, 4.3.0
>
>         Attachments: 
> 0001-BOOKKEEPER-675-Log-noise-fixup-before-cutting-4.2.2.patch, 
> 0001-BOOKKEEPER-675-Log-noise-fixup-before-cutting-4.2.2.patch, 
> 0001-BOOKKEEPER-675-Log-noise-fixup-trunk.patch, 
> 0001-BOOKKEEPER-675-Log-noise-fixup-trunk.patch
>
>
> The message isn't a warning, but an information message that tells you that 
> you initiated your cluster before instance ids existed, as such it should be 
> info level so it doesn't show up on monitoring systems.
> WARN  org.apache.bookkeeper.bookie.Bookie  - INSTANCEID not exists in
> zookeeper. Not considering it for data verification 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to