You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2015/10/31 06:51:40 UTC

[4/4] hbase git commit: HBASE-14700 Support a permissive mode for secure clusters to allow SIMPLE auth clients

HBASE-14700 Support a permissive mode for secure clusters to allow SIMPLE auth clients


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

Branch: refs/heads/hbase-12439
Commit: 683f84e6a217dfd872e5f1be82c7aa4cdf232ec1
Parents: 4534f8e
Author: Gary Helmling <ga...@apache.org>
Authored: Fri Oct 30 19:45:46 2015 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Fri Oct 30 19:45:46 2015 -0700

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        | 11 ++++
 .../hbase/ipc/MetricsHBaseServerSource.java     |  5 ++
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java |  8 +++
 .../hadoop/hbase/ipc/MetricsHBaseServer.java    |  4 ++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 65 +++++++++++++++++---
 .../hbase/regionserver/HRegionServer.java       |  1 +
 .../hbase/regionserver/RSRpcServices.java       | 11 +++-
 .../hadoop/hbase/security/TestSecureRPC.java    | 65 +++++++++++++++++---
 8 files changed, 151 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index c64873f..97d09d6 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1022,6 +1022,17 @@ possible configurations would overwhelm and obscure the important.
       authentication, and will abort the connection.</description>
   </property>
   <property>
+    <name>hbase.ipc.server.fallback-to-simple-auth-allowed</name>
+    <value>false</value>
+    <description>When a server is configured to require secure connections, it will
+      reject connection attempts from clients using SASL SIMPLE (unsecure) authentication.
+      This setting allows secure servers to accept SASL SIMPLE connections from clients
+      when the client requests.  When false (the default), the server will not allow the fallback
+      to SIMPLE authentication, and will reject the connection.  WARNING: This setting should ONLY
+      be used as a temporary measure while converting clients over to secure authentication.  It
+      MUST BE DISABLED for secure operation.</description>
+  </property>
+  <property>
     <name>hbase.display.keys</name>
     <value>true</value>
     <description>When this is set to true the webUI and such will display all start/end keys

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
index 36bd643..5cf71f3 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
@@ -34,6 +34,9 @@ public interface MetricsHBaseServerSource extends BaseSource {
   String AUTHENTICATION_FAILURES_NAME = "authenticationFailures";
   String AUTHENTICATION_FAILURES_DESC =
       "Number of authentication failures.";
+  String AUTHENTICATION_FALLBACKS_NAME = "authenticationFallbacks";
+  String AUTHENTICATION_FALLBACKS_DESC =
+      "Number of fallbacks to insecure authentication.";
   String SENT_BYTES_NAME = "sentBytes";
   String SENT_BYTES_DESC = "Number of bytes sent.";
   String RECEIVED_BYTES_NAME = "receivedBytes";
@@ -80,6 +83,8 @@ public interface MetricsHBaseServerSource extends BaseSource {
 
   void authenticationFailure();
 
+  void authenticationFallback();
+
   void exception();
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
index fe84733..78b1c66 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
@@ -36,6 +36,7 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
   private final MutableCounterLong authorizationFailures;
   private final MutableCounterLong authenticationSuccesses;
   private final MutableCounterLong authenticationFailures;
+  private final MutableCounterLong authenticationFallbacks;
   private final MutableCounterLong sentBytes;
   private final MutableCounterLong receivedBytes;
 
@@ -85,6 +86,8 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
         AUTHENTICATION_SUCCESSES_NAME, AUTHENTICATION_SUCCESSES_DESC, 0L);
     this.authenticationFailures = this.getMetricsRegistry().newCounter(AUTHENTICATION_FAILURES_NAME,
         AUTHENTICATION_FAILURES_DESC, 0L);
+    this.authenticationFallbacks = this.getMetricsRegistry().newCounter(
+        AUTHENTICATION_FALLBACKS_NAME, AUTHENTICATION_FALLBACKS_DESC, 0L);
     this.sentBytes = this.getMetricsRegistry().newCounter(SENT_BYTES_NAME,
         SENT_BYTES_DESC, 0L);
     this.receivedBytes = this.getMetricsRegistry().newCounter(RECEIVED_BYTES_NAME,
@@ -117,6 +120,11 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
   }
 
   @Override
+  public void authenticationFallback() {
+    authenticationFallbacks.incr();
+  }
+
+  @Override
   public void exception() {
     exceptions.incr();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
index ceef0df..d276503 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
@@ -53,6 +53,10 @@ public class MetricsHBaseServer {
     source.authenticationSuccess();
   }
 
+  void authenticationFallback() {
+    source.authenticationFallback();
+  }
+
   void sentBytes(long count) {
     source.sentBytes(count);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/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 6cbce95..f588df0 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
@@ -80,6 +80,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Operation;
 import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
@@ -156,7 +157,7 @@ import com.google.protobuf.TextFormat;
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving
-public class RpcServer implements RpcServerInterface {
+public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   // LOG is being used in CallRunner and the log level is being changed in tests
   public static final Log LOG = LogFactory.getLog(RpcServer.class);
   private static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
@@ -168,6 +169,12 @@ public class RpcServer implements RpcServerInterface {
   public static final byte CURRENT_VERSION = 0;
 
   /**
+   * Whether we allow a fallback to SIMPLE auth for insecure clients when security is enabled.
+   */
+  public static final String FALLBACK_TO_INSECURE_CLIENT_AUTH =
+          "hbase.ipc.server.fallback-to-simple-auth-allowed";
+
+  /**
    * How many calls/handler are allowed in the queue.
    */
   static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
@@ -276,6 +283,7 @@ public class RpcServer implements RpcServerInterface {
 
   private final BoundedByteBufferPool reservoir;
 
+  private volatile boolean allowFallbackToSimpleAuth;
 
   /**
    * Datastructure that holds all necessary to a method invocation and then afterward, carries
@@ -1253,6 +1261,9 @@ public class RpcServer implements RpcServerInterface {
     private final Call saslCall =
       new Call(SASL_CALLID, this.service, null, null, null, null, this, null, 0, null, null);
 
+    // was authentication allowed with a fallback to simple auth
+    private boolean authenticatedWithFallback;
+
     public UserGroupInformation attemptingUser = null; // user name before auth
     protected User user = null;
     protected UserGroupInformation ugi = null;
@@ -1329,19 +1340,21 @@ public class RpcServer implements RpcServerInterface {
 
     private UserGroupInformation getAuthorizedUgi(String authorizedId)
         throws IOException {
+      UserGroupInformation authorizedUgi;
       if (authMethod == AuthMethod.DIGEST) {
         TokenIdentifier tokenId = HBaseSaslRpcServer.getIdentifier(authorizedId,
             secretManager);
-        UserGroupInformation ugi = tokenId.getUser();
-        if (ugi == null) {
+        authorizedUgi = tokenId.getUser();
+        if (authorizedUgi == null) {
           throw new AccessDeniedException(
               "Can't retrieve username from tokenIdentifier.");
         }
-        ugi.addTokenIdentifier(tokenId);
-        return ugi;
+        authorizedUgi.addTokenIdentifier(tokenId);
       } else {
-        return UserGroupInformation.createRemoteUser(authorizedId);
+        authorizedUgi = UserGroupInformation.createRemoteUser(authorizedId);
       }
+      authorizedUgi.setAuthenticationMethod(authMethod.authenticationMethod.getAuthMethod());
+      return authorizedUgi;
     }
 
     private void saslReadAndProcess(byte[] saslToken) throws IOException,
@@ -1520,10 +1533,15 @@ public class RpcServer implements RpcServerInterface {
         return doBadPreambleHandling(msg, new BadAuthException(msg));
       }
       if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
-        AccessDeniedException ae = new AccessDeniedException("Authentication is required");
-        setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
-        responder.doRespond(authFailedCall);
-        throw ae;
+        if (allowFallbackToSimpleAuth) {
+          metrics.authenticationFallback();
+          authenticatedWithFallback = true;
+        } else {
+          AccessDeniedException ae = new AccessDeniedException("Authentication is required");
+          setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
+          responder.doRespond(authFailedCall);
+          throw ae;
+        }
       }
       if (!isSecurityEnabled && authMethod != AuthMethod.SIMPLE) {
         doRawSaslReply(SaslStatus.SUCCESS, new IntWritable(
@@ -1670,6 +1688,12 @@ public class RpcServer implements RpcServerInterface {
         if (ugi != null) {
           ugi.setAuthenticationMethod(AuthMethod.SIMPLE.authenticationMethod);
         }
+        // audit logging for SASL authenticated users happens in saslReadAndProcess()
+        if (authenticatedWithFallback) {
+          LOG.warn("Allowed fallback to SIMPLE auth for " + ugi
+              + " connecting from " + getHostAddress());
+        }
+        AUDITLOG.info(AUTH_SUCCESSFUL_FOR + ugi);
       } else {
         // user is authenticated
         ugi.setAuthenticationMethod(authMethod.authenticationMethod);
@@ -2024,10 +2048,31 @@ public class RpcServer implements RpcServerInterface {
     if (isSecurityEnabled) {
       HBaseSaslRpcServer.init(conf);
     }
+    initReconfigurable(conf);
+
     this.scheduler = scheduler;
     this.scheduler.init(new RpcSchedulerContext(this));
   }
 
+  @Override
+  public void onConfigurationChange(Configuration newConf) {
+    initReconfigurable(newConf);
+  }
+
+  private void initReconfigurable(Configuration confToLoad) {
+    this.allowFallbackToSimpleAuth = confToLoad.getBoolean(FALLBACK_TO_INSECURE_CLIENT_AUTH, false);
+    if (isSecurityEnabled && allowFallbackToSimpleAuth) {
+      LOG.warn("********* WARNING! *********");
+      LOG.warn("This server is configured to allow connections from INSECURE clients");
+      LOG.warn("(" + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = true).");
+      LOG.warn("While this option is enabled, client identities cannot be secured, and user");
+      LOG.warn("impersonation is possible!");
+      LOG.warn("For secure operation, please disable SIMPLE authentication as soon as possible,");
+      LOG.warn("by setting " + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = false in hbase-site.xml");
+      LOG.warn("****************************");
+    }
+  }
+
   /**
    * Subclasses of HBaseServer can override this to provide their own
    * Connection implementations.

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0234769..2ce2193 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -885,6 +885,7 @@ public class HRegionServer extends HasThread implements
   private void registerConfigurationObservers() {
     // Registering the compactSplitThread object with the ConfigurationManager.
     configurationManager.registerObserver(this.compactSplitThread);
+    configurationManager.registerObserver(this.rpcServices);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 28bf069..5729334 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.exceptions.OperationConflictException;
@@ -192,7 +193,8 @@ import com.google.protobuf.TextFormat;
 @InterfaceAudience.Private
 @SuppressWarnings("deprecation")
 public class RSRpcServices implements HBaseRPCErrorHandler,
-    AdminService.BlockingInterface, ClientService.BlockingInterface, PriorityFunction {
+    AdminService.BlockingInterface, ClientService.BlockingInterface, PriorityFunction,
+    ConfigurationObserver {
   protected static final Log LOG = LogFactory.getLog(RSRpcServices.class);
 
   /** RPC scheduler to use for the region server. */
@@ -982,6 +984,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     rs.setName(name);
   }
 
+  @Override
+  public void onConfigurationChange(Configuration newConf) {
+    if (rpcServer instanceof ConfigurationObserver) {
+      ((ConfigurationObserver)rpcServer).onConfigurationChange(newConf);
+    }
+  }
+
   protected PriorityFunction createPriority() {
     return new AnnotationReadingPriorityFunction(this);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/683f84e6/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java
index 66b8c75..769e014 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileF
 import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting;
 import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertSame;
 
 import java.io.File;
@@ -101,20 +102,26 @@ public class TestSecureRPC {
   }
 
   @Test
+  public void testRpcWithInsecureFallback() throws Exception {
+    testRpcFallbackToSimpleAuth(RpcClientImpl.class);
+  }
+
+  @Test
   public void testAsyncRpc() throws Exception {
     testRpcCallWithEnabledKerberosSaslAuth(AsyncRpcClient.class);
   }
 
+  @Test
+  public void testAsyncRpcWithInsecureFallback() throws Exception {
+    testRpcFallbackToSimpleAuth(AsyncRpcClient.class);
+  }
+
   private void testRpcCallWithEnabledKerberosSaslAuth(Class<? extends RpcClient> rpcImplClass)
       throws Exception {
     String krbKeytab = getKeytabFileForTesting();
     String krbPrincipal = getPrincipalForTesting();
 
-    Configuration cnf = new Configuration();
-    cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(cnf);
-    UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab);
-    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    UserGroupInformation ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal);
     UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser();
 
     // check that the login user is okay:
@@ -122,8 +129,28 @@ public class TestSecureRPC {
     assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
     assertEquals(krbPrincipal, ugi.getUserName());
 
+    Configuration clientConf = getSecuredConfiguration();
+    callRpcService(rpcImplClass, User.create(ugi2), clientConf, false);
+  }
+
+  private UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal)
+      throws Exception {
+    Configuration cnf = new Configuration();
+    cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    UserGroupInformation.setConfiguration(cnf);
+    UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab);
+    return UserGroupInformation.getLoginUser();
+  }
+
+  private void callRpcService(Class<? extends RpcClient> rpcImplClass, User clientUser,
+                              Configuration clientConf, boolean allowInsecureFallback)
+      throws Exception {
+    Configuration clientConfCopy = new Configuration(clientConf);
+    clientConfCopy.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, rpcImplClass.getName());
+
     Configuration conf = getSecuredConfiguration();
-    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, rpcImplClass.getName());
+    conf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, allowInsecureFallback);
+
     SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class);
     Mockito.when(securityInfoMock.getServerPrincipal())
         .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL);
@@ -141,7 +168,7 @@ public class TestSecureRPC {
             conf, new FifoRpcScheduler(conf, 1));
     rpcServer.start();
     RpcClient rpcClient =
-        RpcClientFactory.createClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString());
+        RpcClientFactory.createClient(clientConfCopy, HConstants.DEFAULT_CLUSTER_ID.toString());
     try {
       InetSocketAddress address = rpcServer.getListenerAddress();
       if (address == null) {
@@ -150,7 +177,7 @@ public class TestSecureRPC {
       BlockingRpcChannel channel =
           rpcClient.createBlockingRpcChannel(
             ServerName.valueOf(address.getHostName(), address.getPort(),
-            System.currentTimeMillis()), User.getCurrent(), 5000);
+            System.currentTimeMillis()), clientUser, 5000);
       TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
           TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
       List<Integer> results = new ArrayList<Integer>();
@@ -164,4 +191,26 @@ public class TestSecureRPC {
       rpcServer.stop();
     }
   }
+
+  public void testRpcFallbackToSimpleAuth(Class<? extends RpcClient> rpcImplClass) throws Exception {
+    String krbKeytab = getKeytabFileForTesting();
+    String krbPrincipal = getPrincipalForTesting();
+
+    UserGroupInformation ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal);
+    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
+    assertEquals(krbPrincipal, ugi.getUserName());
+
+    String clientUsername = "testuser";
+    UserGroupInformation clientUgi = UserGroupInformation.createUserForTesting(clientUsername,
+        new String[]{clientUsername});
+
+    // check that the client user is insecure
+    assertNotSame(ugi, clientUgi);
+    assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod());
+    assertEquals(clientUsername, clientUgi.getUserName());
+
+    Configuration clientConf = new Configuration();
+    clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
+    callRpcService(rpcImplClass, User.create(clientUgi), clientConf, true);
+  }
 }
\ No newline at end of file