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

[1/9] git commit: HBASE-11706 Set versions for VerifyReplication (cuijianwei)

Repository: hbase
Updated Branches:
  refs/heads/0.98 800000c1e -> 67c232326
  refs/heads/branch-1 1669bc44c -> b8f2f67d4
  refs/heads/master fa46724f3 -> 12d4a42e7


HBASE-11706 Set versions for VerifyReplication (cuijianwei)


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

Branch: refs/heads/master
Commit: 3c03dcb1b9422c8afa06fc427c5de1d3dca5ae6c
Parents: fa46724
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:30:47 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:30:47 2014 -0700

----------------------------------------------------------------------
 .../mapreduce/replication/VerifyReplication.java      | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3c03dcb1/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index d4ac8f7..7748675 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -69,6 +69,7 @@ public class VerifyReplication extends Configured implements Tool {
   public final static String NAME = "verifyrep";
   static long startTime = 0;
   static long endTime = Long.MAX_VALUE;
+  static int versions = -1;
   static String tableName = null;
   static String families = null;
   static String peerId = null;
@@ -109,6 +110,9 @@ public class VerifyReplication extends Configured implements Tool {
           }
         }
         scan.setTimeRange(startTime, endTime);
+        if (versions >= 0) {
+          scan.setMaxVersions(versions);
+        }
         HConnectionManager.execute(new HConnectable<Void>(conf) {
           @Override
           public Void connect(HConnection conn) throws IOException {
@@ -208,6 +212,9 @@ public class VerifyReplication extends Configured implements Tool {
 
     Scan scan = new Scan();
     scan.setTimeRange(startTime, endTime);
+    if (versions >= 0) {
+      scan.setMaxVersions(versions);
+    }
     if(families != null) {
       String[] fams = families.split(",");
       for(String fam : fams) {
@@ -250,6 +257,12 @@ public class VerifyReplication extends Configured implements Tool {
           continue;
         }
 
+        final String versionsArgKey = "--versions=";
+        if (cmd.startsWith(versionsArgKey)) {
+          versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
+          continue;
+        }
+
         final String familiesArgKey = "--families=";
         if (cmd.startsWith(familiesArgKey)) {
           families = cmd.substring(familiesArgKey.length());
@@ -286,6 +299,7 @@ public class VerifyReplication extends Configured implements Tool {
     System.err.println(" starttime    beginning of the time range");
     System.err.println("              without endtime means from starttime to forever");
     System.err.println(" endtime      end of the time range");
+    System.err.println(" versions     number of cell versions to verify");
     System.err.println(" families     comma-separated list of families to copy");
     System.err.println();
     System.err.println("Args:");


[2/9] git commit: HBASE-11706 Set versions for VerifyReplication (cuijianwei)

Posted by ap...@apache.org.
HBASE-11706 Set versions for VerifyReplication (cuijianwei)


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

Branch: refs/heads/branch-1
Commit: 59a49f3149413c5655f6a69f4c2d896abe7c160d
Parents: 1669bc4
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:30:47 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:31:12 2014 -0700

----------------------------------------------------------------------
 .../mapreduce/replication/VerifyReplication.java      | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59a49f31/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index d4ac8f7..7748675 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -69,6 +69,7 @@ public class VerifyReplication extends Configured implements Tool {
   public final static String NAME = "verifyrep";
   static long startTime = 0;
   static long endTime = Long.MAX_VALUE;
+  static int versions = -1;
   static String tableName = null;
   static String families = null;
   static String peerId = null;
@@ -109,6 +110,9 @@ public class VerifyReplication extends Configured implements Tool {
           }
         }
         scan.setTimeRange(startTime, endTime);
+        if (versions >= 0) {
+          scan.setMaxVersions(versions);
+        }
         HConnectionManager.execute(new HConnectable<Void>(conf) {
           @Override
           public Void connect(HConnection conn) throws IOException {
@@ -208,6 +212,9 @@ public class VerifyReplication extends Configured implements Tool {
 
     Scan scan = new Scan();
     scan.setTimeRange(startTime, endTime);
+    if (versions >= 0) {
+      scan.setMaxVersions(versions);
+    }
     if(families != null) {
       String[] fams = families.split(",");
       for(String fam : fams) {
@@ -250,6 +257,12 @@ public class VerifyReplication extends Configured implements Tool {
           continue;
         }
 
+        final String versionsArgKey = "--versions=";
+        if (cmd.startsWith(versionsArgKey)) {
+          versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
+          continue;
+        }
+
         final String familiesArgKey = "--families=";
         if (cmd.startsWith(familiesArgKey)) {
           families = cmd.substring(familiesArgKey.length());
@@ -286,6 +299,7 @@ public class VerifyReplication extends Configured implements Tool {
     System.err.println(" starttime    beginning of the time range");
     System.err.println("              without endtime means from starttime to forever");
     System.err.println(" endtime      end of the time range");
+    System.err.println(" versions     number of cell versions to verify");
     System.err.println(" families     comma-separated list of families to copy");
     System.err.println();
     System.err.println("Args:");


[6/9] git commit: HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)

Posted by ap...@apache.org.
HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)


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

Branch: refs/heads/0.98
Commit: cb4ac0d397a77f03715e431b87c8b67bff7aa18c
Parents: b7b1d01
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:41:18 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:41:33 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cb4ac0d3/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 31484bb..8a34a82 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -110,7 +110,6 @@ public class CallRunner {
         RequestContext.clear();
       }
       RpcServer.CurCall.set(null);
-      this.rpcServer.addCallSize(call.getSize() * -1);
       // Set the response for undelayed calls and delayed calls with
       // undelayed responses.
       if (!call.isDelayed() || !call.isReturnValueDelayed()) {
@@ -139,6 +138,9 @@ public class CallRunner {
     } catch (Exception e) {
       RpcServer.LOG.warn(Thread.currentThread().getName()
           + ": caught: " + StringUtils.stringifyException(e));
+    } finally {
+      // regardless if succesful or not we need to reset the callQueueSize
+      this.rpcServer.addCallSize(call.getSize() * -1);
     }
   }
 


[3/9] git commit: HBASE-11706 Set versions for VerifyReplication (cuijianwei)

Posted by ap...@apache.org.
HBASE-11706 Set versions for VerifyReplication (cuijianwei)


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

Branch: refs/heads/0.98
Commit: b7b1d01e05ec5d966b2bf6d82831137f48159bfb
Parents: 800000c
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:30:47 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:38:35 2014 -0700

----------------------------------------------------------------------
 .../mapreduce/replication/VerifyReplication.java      | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b7b1d01e/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 87fa157..bb17e64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -67,6 +67,7 @@ public class VerifyReplication extends Configured implements Tool {
   public final static String NAME = "verifyrep";
   static long startTime = 0;
   static long endTime = Long.MAX_VALUE;
+  static int versions = -1;
   static String tableName = null;
   static String families = null;
   static String peerId = null;
@@ -107,6 +108,9 @@ public class VerifyReplication extends Configured implements Tool {
           }
         }
         scan.setTimeRange(startTime, endTime);
+        if (versions >= 0) {
+          scan.setMaxVersions(versions);
+        }
         HConnectionManager.execute(new HConnectable<Void>(conf) {
           @Override
           public Void connect(HConnection conn) throws IOException {
@@ -205,6 +209,9 @@ public class VerifyReplication extends Configured implements Tool {
 
     Scan scan = new Scan();
     scan.setTimeRange(startTime, endTime);
+    if (versions >= 0) {
+      scan.setMaxVersions(versions);
+    }
     if(families != null) {
       String[] fams = families.split(",");
       for(String fam : fams) {
@@ -247,6 +254,12 @@ public class VerifyReplication extends Configured implements Tool {
           continue;
         }
 
+        final String versionsArgKey = "--versions=";
+        if (cmd.startsWith(versionsArgKey)) {
+          versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
+          continue;
+        }
+
         final String familiesArgKey = "--families=";
         if (cmd.startsWith(familiesArgKey)) {
           families = cmd.substring(familiesArgKey.length());
@@ -283,6 +296,7 @@ public class VerifyReplication extends Configured implements Tool {
     System.err.println(" starttime    beginning of the time range");
     System.err.println("              without endtime means from starttime to forever");
     System.err.println(" endtime      end of the time range");
+    System.err.println(" versions     number of cell versions to verify");
     System.err.println(" families     comma-separated list of families to copy");
     System.err.println();
     System.err.println("Args:");


[8/9] git commit: HBASE-11589 AccessControlException should be a not retriable exception (Qiang Tian)

Posted by ap...@apache.org.
HBASE-11589 AccessControlException should be a not retriable exception (Qiang Tian)

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1
Commit: b8f2f67d4c79138b94dc6c330944cc709049ce05
Parents: 2df6b05
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 19:10:56 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 19:12:09 2014 -0700

----------------------------------------------------------------------
 .../hbase/security/AccessDeniedException.java   |  5 ++++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 24 ++++++++++----------
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  6 ++---
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 12 +++++-----
 4 files changed, 26 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b8f2f67d/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
index 482faef..f7b07e5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
@@ -41,4 +41,9 @@ public class AccessDeniedException extends DoNotRetryIOException {
   public AccessDeniedException(String s) {
     super(s);
   }
+
+  public AccessDeniedException(Throwable cause) {
+    super(cause);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8f2f67d/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 ef4df3e..7d9c569 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
+import io.netty.util.internal.ConcurrentSet;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -63,8 +64,6 @@ import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
 
-import io.netty.util.internal.ConcurrentSet;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -91,6 +90,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
@@ -108,7 +108,6 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -119,8 +118,8 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
-import org.htrace.TraceInfo;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.htrace.TraceInfo;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
@@ -1235,7 +1234,7 @@ public class RpcServer implements RpcServerInterface {
             secretManager);
         UserGroupInformation ugi = tokenId.getUser();
         if (ugi == null) {
-          throw new AccessControlException(
+          throw new AccessDeniedException(
               "Can't retrieve username from tokenIdentifier.");
         }
         ugi.addTokenIdentifier(tokenId);
@@ -1265,7 +1264,7 @@ public class RpcServer implements RpcServerInterface {
             switch (authMethod) {
             case DIGEST:
               if (secretManager == null) {
-                throw new AccessControlException(
+                throw new AccessDeniedException(
                     "Server is not configured to do DIGEST authentication.");
               }
               saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
@@ -1282,7 +1281,7 @@ public class RpcServer implements RpcServerInterface {
               }
               final String names[] = SaslUtil.splitKerberosName(fullName);
               if (names.length != 3) {
-                throw new AccessControlException(
+                throw new AccessDeniedException(
                     "Kerberos principal name does NOT have the expected "
                         + "hostname part: " + fullName);
               }
@@ -1297,7 +1296,7 @@ public class RpcServer implements RpcServerInterface {
               });
             }
             if (saslServer == null)
-              throw new AccessControlException(
+              throw new AccessDeniedException(
                   "Unable to find SASL server implementation for "
                       + authMethod.getMechanismName());
             if (LOG.isDebugEnabled()) {
@@ -1421,7 +1420,7 @@ public class RpcServer implements RpcServerInterface {
         return doBadPreambleHandling(msg, new BadAuthException(msg));
       }
       if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
-        AccessControlException ae = new AccessControlException("Authentication is required");
+        AccessDeniedException ae = new AccessDeniedException("Authentication is required");
         setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
         responder.doRespond(authFailedCall);
         throw ae;
@@ -1581,7 +1580,7 @@ public class RpcServer implements RpcServerInterface {
             && (!protocolUser.getUserName().equals(user.getUserName()))) {
           if (authMethod == AuthMethod.DIGEST) {
             // Not allowed to doAs if token authentication is used
-            throw new AccessControlException("Authenticated user (" + user
+            throw new AccessDeniedException("Authenticated user (" + user
                 + ") doesn't match what the client claims to be ("
                 + protocolUser + ")");
           } else {
@@ -1669,7 +1668,7 @@ public class RpcServer implements RpcServerInterface {
         if (!authorizeConnection()) {
           // Throw FatalConnectionException wrapping ACE so client does right thing and closes
           // down the connection instead of trying to read non-existent retun.
-          throw new AccessControlException("Connection from " + this + " for service " +
+          throw new AccessDeniedException("Connection from " + this + " for service " +
             connectionHeader.getServiceName() + " is unauthorized for user: " + user);
         }
       }
@@ -1778,7 +1777,8 @@ public class RpcServer implements RpcServerInterface {
           LOG.debug("Connection authorization failed: " + ae.getMessage(), ae);
         }
         metrics.authorizationFailure();
-        setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
+        setupResponse(authFailedResponse, authFailedCall,
+          new AccessDeniedException(ae), ae.getMessage());
         responder.doRespond(authFailedCall);
         return false;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8f2f67d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 53f5874..b569f4b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionPlacementMaintainer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -73,7 +74,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -1669,7 +1669,7 @@ public abstract class FSUtils {
    *          the action
    */
   public static void checkAccess(UserGroupInformation ugi, FileStatus file,
-      FsAction action) throws AccessControlException {
+      FsAction action) throws AccessDeniedException {
     if (ugi.getShortUserName().equals(file.getOwner())) {
       if (file.getPermission().getUserAction().implies(action)) {
         return;
@@ -1681,7 +1681,7 @@ public abstract class FSUtils {
     } else if (file.getPermission().getOtherAction().implies(action)) {
       return;
     }
-    throw new AccessControlException("Permission denied:" + " action=" + action
+    throw new AccessDeniedException("Permission denied:" + " action=" + action
         + " path=" + file.getPath() + " user=" + ugi.getShortUserName());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8f2f67d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 3450516..982417e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -106,8 +106,8 @@ import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
@@ -1565,7 +1565,7 @@ public class HBaseFsck extends Configured {
     }
   }
 
-  private void preCheckPermission() throws IOException, AccessControlException {
+  private void preCheckPermission() throws IOException, AccessDeniedException {
     if (shouldIgnorePreCheckPermission()) {
       return;
     }
@@ -1578,12 +1578,12 @@ public class HBaseFsck extends Configured {
     for (FileStatus file : files) {
       try {
         FSUtils.checkAccess(ugi, file, FsAction.WRITE);
-      } catch (AccessControlException ace) {
-        LOG.warn("Got AccessControlException when preCheckPermission ", ace);
+      } catch (AccessDeniedException ace) {
+        LOG.warn("Got AccessDeniedException when preCheckPermission ", ace);
         errors.reportError(ERROR_CODE.WRONG_USAGE, "Current user " + ugi.getUserName()
           + " does not have write perms to " + file.getPath()
           + ". Please rerun hbck as hdfs user " + file.getOwner());
-        throw new AccessControlException(ace);
+        throw ace;
       }
     }
   }
@@ -4014,7 +4014,7 @@ public class HBaseFsck extends Configured {
     // pre-check current user has FS write permission or not
     try {
       preCheckPermission();
-    } catch (AccessControlException ace) {
+    } catch (AccessDeniedException ace) {
       Runtime.getRuntime().exit(-1);
     } catch (IOException ioe) {
       Runtime.getRuntime().exit(-1);


[7/9] git commit: HBASE-11589 AccessControlException should be a not retriable exception (Qiang Tian)

Posted by ap...@apache.org.
HBASE-11589 AccessControlException should be a not retriable exception (Qiang Tian)


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

Branch: refs/heads/master
Commit: 12d4a42e712036dc0b8843cec5c56e366cc157c7
Parents: c2bfc64
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 19:10:56 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 19:10:56 2014 -0700

----------------------------------------------------------------------
 .../hbase/security/AccessDeniedException.java   |  5 ++++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 24 ++++++++++----------
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  6 ++---
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 12 +++++-----
 4 files changed, 26 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/12d4a42e/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
index 482faef..f7b07e5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
@@ -41,4 +41,9 @@ public class AccessDeniedException extends DoNotRetryIOException {
   public AccessDeniedException(String s) {
     super(s);
   }
+
+  public AccessDeniedException(Throwable cause) {
+    super(cause);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12d4a42e/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 ef4df3e..7d9c569 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
+import io.netty.util.internal.ConcurrentSet;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -63,8 +64,6 @@ import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
 
-import io.netty.util.internal.ConcurrentSet;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -91,6 +90,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
@@ -108,7 +108,6 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -119,8 +118,8 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
-import org.htrace.TraceInfo;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.htrace.TraceInfo;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
@@ -1235,7 +1234,7 @@ public class RpcServer implements RpcServerInterface {
             secretManager);
         UserGroupInformation ugi = tokenId.getUser();
         if (ugi == null) {
-          throw new AccessControlException(
+          throw new AccessDeniedException(
               "Can't retrieve username from tokenIdentifier.");
         }
         ugi.addTokenIdentifier(tokenId);
@@ -1265,7 +1264,7 @@ public class RpcServer implements RpcServerInterface {
             switch (authMethod) {
             case DIGEST:
               if (secretManager == null) {
-                throw new AccessControlException(
+                throw new AccessDeniedException(
                     "Server is not configured to do DIGEST authentication.");
               }
               saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
@@ -1282,7 +1281,7 @@ public class RpcServer implements RpcServerInterface {
               }
               final String names[] = SaslUtil.splitKerberosName(fullName);
               if (names.length != 3) {
-                throw new AccessControlException(
+                throw new AccessDeniedException(
                     "Kerberos principal name does NOT have the expected "
                         + "hostname part: " + fullName);
               }
@@ -1297,7 +1296,7 @@ public class RpcServer implements RpcServerInterface {
               });
             }
             if (saslServer == null)
-              throw new AccessControlException(
+              throw new AccessDeniedException(
                   "Unable to find SASL server implementation for "
                       + authMethod.getMechanismName());
             if (LOG.isDebugEnabled()) {
@@ -1421,7 +1420,7 @@ public class RpcServer implements RpcServerInterface {
         return doBadPreambleHandling(msg, new BadAuthException(msg));
       }
       if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
-        AccessControlException ae = new AccessControlException("Authentication is required");
+        AccessDeniedException ae = new AccessDeniedException("Authentication is required");
         setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
         responder.doRespond(authFailedCall);
         throw ae;
@@ -1581,7 +1580,7 @@ public class RpcServer implements RpcServerInterface {
             && (!protocolUser.getUserName().equals(user.getUserName()))) {
           if (authMethod == AuthMethod.DIGEST) {
             // Not allowed to doAs if token authentication is used
-            throw new AccessControlException("Authenticated user (" + user
+            throw new AccessDeniedException("Authenticated user (" + user
                 + ") doesn't match what the client claims to be ("
                 + protocolUser + ")");
           } else {
@@ -1669,7 +1668,7 @@ public class RpcServer implements RpcServerInterface {
         if (!authorizeConnection()) {
           // Throw FatalConnectionException wrapping ACE so client does right thing and closes
           // down the connection instead of trying to read non-existent retun.
-          throw new AccessControlException("Connection from " + this + " for service " +
+          throw new AccessDeniedException("Connection from " + this + " for service " +
             connectionHeader.getServiceName() + " is unauthorized for user: " + user);
         }
       }
@@ -1778,7 +1777,8 @@ public class RpcServer implements RpcServerInterface {
           LOG.debug("Connection authorization failed: " + ae.getMessage(), ae);
         }
         metrics.authorizationFailure();
-        setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
+        setupResponse(authFailedResponse, authFailedCall,
+          new AccessDeniedException(ae), ae.getMessage());
         responder.doRespond(authFailedCall);
         return false;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12d4a42e/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 7b3b636..0f21b72 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionPlacementMaintainer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -73,7 +74,6 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -1670,7 +1670,7 @@ public abstract class FSUtils {
    *          the action
    */
   public static void checkAccess(UserGroupInformation ugi, FileStatus file,
-      FsAction action) throws AccessControlException {
+      FsAction action) throws AccessDeniedException {
     if (ugi.getShortUserName().equals(file.getOwner())) {
       if (file.getPermission().getUserAction().implies(action)) {
         return;
@@ -1682,7 +1682,7 @@ public abstract class FSUtils {
     } else if (file.getPermission().getOtherAction().implies(action)) {
       return;
     }
-    throw new AccessControlException("Permission denied:" + " action=" + action
+    throw new AccessDeniedException("Permission denied:" + " action=" + action
         + " path=" + file.getPath() + " user=" + ugi.getShortUserName());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/12d4a42e/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index e5365da..efcd7cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -107,8 +107,8 @@ import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
@@ -1580,7 +1580,7 @@ public class HBaseFsck extends Configured {
     setCheckHdfs(prevHdfsCheck);
   }
 
-  private void preCheckPermission() throws IOException, AccessControlException {
+  private void preCheckPermission() throws IOException, AccessDeniedException {
     if (shouldIgnorePreCheckPermission()) {
       return;
     }
@@ -1593,12 +1593,12 @@ public class HBaseFsck extends Configured {
     for (FileStatus file : files) {
       try {
         FSUtils.checkAccess(ugi, file, FsAction.WRITE);
-      } catch (AccessControlException ace) {
-        LOG.warn("Got AccessControlException when preCheckPermission ", ace);
+      } catch (AccessDeniedException ace) {
+        LOG.warn("Got AccessDeniedException when preCheckPermission ", ace);
         errors.reportError(ERROR_CODE.WRONG_USAGE, "Current user " + ugi.getUserName()
           + " does not have write perms to " + file.getPath()
           + ". Please rerun hbck as hdfs user " + file.getOwner());
-        throw new AccessControlException(ace);
+        throw ace;
       }
     }
   }
@@ -4139,7 +4139,7 @@ public class HBaseFsck extends Configured {
     // pre-check current user has FS write permission or not
     try {
       preCheckPermission();
-    } catch (AccessControlException ace) {
+    } catch (AccessDeniedException ace) {
       Runtime.getRuntime().exit(-1);
     } catch (IOException ioe) {
       Runtime.getRuntime().exit(-1);


[4/9] git commit: HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)

Posted by ap...@apache.org.
HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)


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

Branch: refs/heads/master
Commit: c2bfc64719718a8d2743cad91a25a842389062df
Parents: 3c03dcb
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:41:18 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:41:18 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2bfc647/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 33474d4..c364786 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -110,7 +110,6 @@ public class CallRunner {
         RequestContext.clear();
       }
       RpcServer.CurCall.set(null);
-      this.rpcServer.addCallSize(call.getSize() * -1);
       // Set the response for undelayed calls and delayed calls with
       // undelayed responses.
       if (!call.isDelayed() || !call.isReturnValueDelayed()) {
@@ -139,6 +138,9 @@ public class CallRunner {
     } catch (Exception e) {
       RpcServer.LOG.warn(Thread.currentThread().getName()
           + ": caught: " + StringUtils.stringifyException(e));
+    } finally {
+      // regardless if succesful or not we need to reset the callQueueSize
+      this.rpcServer.addCallSize(call.getSize() * -1);
     }
   }
 


[5/9] git commit: HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)

Posted by ap...@apache.org.
HBASE-11705 callQueueSize should be decremented in a fail-fast scenario (Esteban Gutierrez)


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

Branch: refs/heads/branch-1
Commit: 2df6b0562f0df6e4d1c5eafb9acbd486f3ed3b66
Parents: 59a49f3
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 18:41:18 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 18:41:28 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2df6b056/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 33474d4..c364786 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -110,7 +110,6 @@ public class CallRunner {
         RequestContext.clear();
       }
       RpcServer.CurCall.set(null);
-      this.rpcServer.addCallSize(call.getSize() * -1);
       // Set the response for undelayed calls and delayed calls with
       // undelayed responses.
       if (!call.isDelayed() || !call.isReturnValueDelayed()) {
@@ -139,6 +138,9 @@ public class CallRunner {
     } catch (Exception e) {
       RpcServer.LOG.warn(Thread.currentThread().getName()
           + ": caught: " + StringUtils.stringifyException(e));
+    } finally {
+      // regardless if succesful or not we need to reset the callQueueSize
+      this.rpcServer.addCallSize(call.getSize() * -1);
     }
   }
 


[9/9] git commit: HBASE-11589 AccessControlException should be a not retriable exception (Qiang Tian)

Posted by ap...@apache.org.
HBASE-11589 AccessControlException should be a not retriable exception (Qiang Tian)

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 67c232326d16d64bbd7551ec2ef3c796f365d16e
Parents: cb4ac0d
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 8 19:10:56 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 8 19:17:28 2014 -0700

----------------------------------------------------------------------
 .../hbase/security/AccessDeniedException.java    |  5 +++++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java   | 19 ++++++++++---------
 .../org/apache/hadoop/hbase/util/FSUtils.java    |  6 +++---
 .../org/apache/hadoop/hbase/util/HBaseFsck.java  | 12 ++++++------
 4 files changed, 24 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/67c23232/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
index 482faef..f7b07e5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
@@ -41,4 +41,9 @@ public class AccessDeniedException extends DoNotRetryIOException {
   public AccessDeniedException(String s) {
     super(s);
   }
+
+  public AccessDeniedException(Throwable cause) {
+    super(cause);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/67c23232/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 8ff70fe..6f5b6a7 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
@@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
@@ -100,7 +101,6 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -1247,7 +1247,7 @@ public class RpcServer implements RpcServerInterface {
             secretManager);
         UserGroupInformation ugi = tokenId.getUser();
         if (ugi == null) {
-          throw new AccessControlException(
+          throw new AccessDeniedException(
               "Can't retrieve username from tokenIdentifier.");
         }
         ugi.addTokenIdentifier(tokenId);
@@ -1277,7 +1277,7 @@ public class RpcServer implements RpcServerInterface {
             switch (authMethod) {
             case DIGEST:
               if (secretManager == null) {
-                throw new AccessControlException(
+                throw new AccessDeniedException(
                     "Server is not configured to do DIGEST authentication.");
               }
               saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
@@ -1294,7 +1294,7 @@ public class RpcServer implements RpcServerInterface {
               }
               final String names[] = SaslUtil.splitKerberosName(fullName);
               if (names.length != 3) {
-                throw new AccessControlException(
+                throw new AccessDeniedException(
                     "Kerberos principal name does NOT have the expected "
                         + "hostname part: " + fullName);
               }
@@ -1309,7 +1309,7 @@ public class RpcServer implements RpcServerInterface {
               });
             }
             if (saslServer == null)
-              throw new AccessControlException(
+              throw new AccessDeniedException(
                   "Unable to find SASL server implementation for "
                       + authMethod.getMechanismName());
             if (LOG.isDebugEnabled()) {
@@ -1453,7 +1453,7 @@ public class RpcServer implements RpcServerInterface {
             return doBadPreambleHandling(msg, new BadAuthException(msg));
           }
           if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
-            AccessControlException ae = new AccessControlException("Authentication is required");
+            AccessDeniedException ae = new AccessDeniedException("Authentication is required");
             setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
             responder.doRespond(authFailedCall);
             throw ae;
@@ -1566,7 +1566,7 @@ public class RpcServer implements RpcServerInterface {
             && (!protocolUser.getUserName().equals(user.getUserName()))) {
           if (authMethod == AuthMethod.DIGEST) {
             // Not allowed to doAs if token authentication is used
-            throw new AccessControlException("Authenticated user (" + user
+            throw new AccessDeniedException("Authenticated user (" + user
                 + ") doesn't match what the client claims to be ("
                 + protocolUser + ")");
           } else {
@@ -1655,7 +1655,7 @@ public class RpcServer implements RpcServerInterface {
         if (!authorizeConnection()) {
           // Throw FatalConnectionException wrapping ACE so client does right thing and closes
           // down the connection instead of trying to read non-existent retun.
-          throw new AccessControlException("Connection from " + this + " for service " +
+          throw new AccessDeniedException("Connection from " + this + " for service " +
             connectionHeader.getServiceName() + " is unauthorized for user: " + user);
         }
       }
@@ -1765,7 +1765,8 @@ public class RpcServer implements RpcServerInterface {
       } catch (AuthorizationException ae) {
         LOG.debug("Connection authorization failed: " + ae.getMessage(), ae);
         metrics.authorizationFailure();
-        setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
+        setupResponse(authFailedResponse, authFailedCall,
+          new AccessDeniedException(ae), ae.getMessage());
         responder.doRespond(authFailedCall);
         return false;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/67c23232/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index c418ec3..ade9940 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionPlacementMaintainer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -73,7 +74,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -1684,7 +1684,7 @@ public abstract class FSUtils {
    *          the action
    */
   public static void checkAccess(UserGroupInformation ugi, FileStatus file,
-      FsAction action) throws AccessControlException {
+      FsAction action) throws AccessDeniedException {
     if (ugi.getShortUserName().equals(file.getOwner())) {
       if (file.getPermission().getUserAction().implies(action)) {
         return;
@@ -1696,7 +1696,7 @@ public abstract class FSUtils {
     } else if (file.getPermission().getOtherAction().implies(action)) {
       return;
     }
-    throw new AccessControlException("Permission denied:" + " action=" + action
+    throw new AccessDeniedException("Permission denied:" + " action=" + action
         + " path=" + file.getPath() + " user=" + ugi.getShortUserName());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/67c23232/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 783b3da..1ca7f39 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -104,8 +104,8 @@ import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
@@ -1560,7 +1560,7 @@ public class HBaseFsck extends Configured {
     }
   }
 
-  private void preCheckPermission() throws IOException, AccessControlException {
+  private void preCheckPermission() throws IOException, AccessDeniedException {
     if (shouldIgnorePreCheckPermission()) {
       return;
     }
@@ -1573,12 +1573,12 @@ public class HBaseFsck extends Configured {
     for (FileStatus file : files) {
       try {
         FSUtils.checkAccess(ugi, file, FsAction.WRITE);
-      } catch (AccessControlException ace) {
-        LOG.warn("Got AccessControlException when preCheckPermission ", ace);
+      } catch (AccessDeniedException ace) {
+        LOG.warn("Got AccessDeniedException when preCheckPermission ", ace);
         errors.reportError(ERROR_CODE.WRONG_USAGE, "Current user " + ugi.getUserName()
           + " does not have write perms to " + file.getPath()
           + ". Please rerun hbck as hdfs user " + file.getOwner());
-        throw new AccessControlException(ace);
+        throw ace;
       }
     }
   }
@@ -3999,7 +3999,7 @@ public class HBaseFsck extends Configured {
     // pre-check current user has FS write permission or not
     try {
       preCheckPermission();
-    } catch (AccessControlException ace) {
+    } catch (AccessDeniedException ace) {
       Runtime.getRuntime().exit(-1);
     } catch (IOException ioe) {
       Runtime.getRuntime().exit(-1);