You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2014/09/24 04:54:06 UTC

git commit: HBASE-12056 RPC logging too much in DEBUG mode

Repository: hbase
Updated Branches:
  refs/heads/master a5bd93168 -> c31625767


HBASE-12056 RPC logging too much in DEBUG mode


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c3162576
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c3162576
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c3162576

Branch: refs/heads/master
Commit: c316257671554b2243342e8f3c15ebf909355212
Parents: a5bd931
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Sep 23 19:53:57 2014 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Sep 23 19:53:57 2014 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcClient.java  | 39 +++++++++++---------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  7 +++-
 2 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c3162576/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index c0002fd..07c7fb3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -27,7 +27,6 @@ import com.google.protobuf.Message.Builder;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -114,8 +113,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
 @InterfaceAudience.Private
 public class RpcClient {
-  // The LOG key is intentionally not from this package to avoid ipc logging at DEBUG (all under
-  // o.a.h.hbase is set to DEBUG as default).
   public static final Log LOG = LogFactory.getLog(RpcClient.class);
   protected final PoolMap<ConnectionId, Connection> connections;
 
@@ -436,6 +433,7 @@ public class RpcClient {
         return cts;
       }
 
+      @Override
       public void close(){
         assert shouldCloseConnection.get();
         callsToWrite.offer(CallFuture.DEATH_PILL);
@@ -769,8 +767,8 @@ public class RpcClient {
 
     @Override
     public void run() {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(getName() + ": starting, connections " + connections.size());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(getName() + ": starting, connections " + connections.size());
       }
 
       try {
@@ -778,17 +776,21 @@ public class RpcClient {
           readResponse();
         }
       } catch (InterruptedException t) {
-        LOG.debug(getName() + ": interrupted while waiting for call responses");
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(getName() + ": interrupted while waiting for call responses");
+        }
         markClosed(ExceptionUtil.asInterrupt(t));
       } catch (Throwable t) {
-        LOG.debug(getName() + ": unexpected throwable while waiting for call responses", t);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(getName() + ": unexpected throwable while waiting for call responses", t);
+        }
         markClosed(new IOException("Unexpected throwable while waiting call responses", t));
       }
 
       close();
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(getName() + ": stopped, connections " + connections.size());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(getName() + ": stopped, connections " + connections.size());
       }
     }
 
@@ -849,12 +851,15 @@ public class RpcClient {
         final UserGroupInformation user)
     throws IOException, InterruptedException{
       user.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
         public Object run() throws IOException, InterruptedException {
           closeConnection();
           if (shouldAuthenticateOverKrb()) {
             if (currRetries < maxRetries) {
-              LOG.debug("Exception encountered while connecting to " +
-                  "the server : " + ex);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Exception encountered while connecting to " +
+                    "the server : " + ex);
+              }
               //try re-login
               if (UserGroupInformation.isLoginKeytabBased()) {
                 UserGroupInformation.getLoginUser().reloginFromKeytab();
@@ -1048,14 +1053,14 @@ public class RpcClient {
       disposeSasl();
 
       // log the info
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(getName() + ": closing ipc connection to " + server);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(getName() + ": closing ipc connection to " + server);
       }
 
       cleanupCalls(true);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(getName() + ": ipc connection to " + server + " closed");
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(getName() + ": ipc connection to " + server + " closed");
       }
     }
 
@@ -1222,8 +1227,8 @@ public class RpcClient {
       if (e == null) throw new NullPointerException();
 
       if (shouldCloseConnection.compareAndSet(false, true)) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(getName() + ": marking at should close, reason: " + e.getMessage());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(getName() + ": marking at should close, reason: " + e.getMessage());
         }
         if (callSender != null) {
           callSender.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3162576/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 2912d60..01a1d3f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -555,6 +555,7 @@ public class RpcServer implements RpcServerInterface {
       Reader() throws IOException {
         this.readSelector = Selector.open();
       }
+      @Override
       public void run() {
         try {
           doRunLoop();
@@ -1252,8 +1253,8 @@ public class RpcServer implements RpcServerInterface {
     private void saslReadAndProcess(byte[] saslToken) throws IOException,
         InterruptedException {
       if (saslContextEstablished) {
-        if (LOG.isDebugEnabled())
-          LOG.debug("Have read input token of size " + saslToken.length
+        if (LOG.isTraceEnabled())
+          LOG.trace("Have read input token of size " + saslToken.length
               + " for processing by saslServer.unwrap()");
 
         if (!useWrap) {
@@ -2004,6 +2005,7 @@ public class RpcServer implements RpcServerInterface {
    * the return response has protobuf response payload. On failure, the
    * exception name and the stack trace are returned in the protobuf response.
    */
+  @Override
   public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
       Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
   throws IOException {
@@ -2158,6 +2160,7 @@ public class RpcServer implements RpcServerInterface {
   /**
    * Returns the metrics instance for reporting RPC call statistics
    */
+  @Override
   public MetricsHBaseServer getMetrics() {
     return metrics;
   }