You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by dd...@apache.org on 2013/11/01 18:50:19 UTC

svn commit: r1538003 - in /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase: ipc/TestDelayedRpc.java security/TestSecureRPC.java

Author: ddas
Date: Fri Nov  1 17:50:19 2013
New Revision: 1538003

URL: http://svn.apache.org/r1538003
Log:
HBASE-8569. Addendum patch (for secure RPC tests).

Added:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java
Modified:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java?rev=1538003&r1=1538002&r2=1538003&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java Fri Nov  1 17:50:19 2013
@@ -28,7 +28,6 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -47,6 +46,7 @@ import org.apache.log4j.spi.LoggingEvent
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.common.collect.Lists;
 import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.RpcController;
@@ -122,7 +122,7 @@ public class TestDelayedRpc {
   }
 
   private static class ListAppender extends AppenderSkeleton {
-    private List<String> messages = new ArrayList<String>();
+    private final List<String> messages = new ArrayList<String>();
 
     @Override
     protected void append(LoggingEvent event) {
@@ -203,13 +203,13 @@ public class TestDelayedRpc {
     }
   }
 
-  static class TestDelayedImplementation
+  public static class TestDelayedImplementation
   implements TestDelayedRpcProtos.TestDelayedService.BlockingInterface {
     /**
      * Should the return value of delayed call be set at the end of the delay
      * or at call return.
      */
-    private boolean delayReturnValue;
+    private final boolean delayReturnValue;
 
     /**
      * @param delayReturnValue Should the response to the delayed call be set
@@ -231,11 +231,12 @@ public class TestDelayedRpc {
       final Delayable call = RpcServer.getCurrentCall();
       call.startDelay(delayReturnValue);
       new Thread() {
+        @Override
         public void run() {
           try {
             Thread.sleep(500);
             TestResponse.Builder responseBuilder = TestResponse.newBuilder();
-            call.endDelay(delayReturnValue ? 
+            call.endDelay(delayReturnValue ?
                 responseBuilder.setResponse(DELAYED).build() : null);
           } catch (Exception e) {
             e.printStackTrace();
@@ -249,10 +250,10 @@ public class TestDelayedRpc {
     }
   }
 
-  private static class TestThread extends Thread {
-    private TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub;
-    private boolean delay;
-    private List<Integer> results;
+  public static class TestThread extends Thread {
+    private final TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub;
+    private final boolean delay;
+    private final List<Integer> results;
 
     public TestThread(TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub,
         boolean delay, List<Integer> results) {

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java?rev=1538003&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java Fri Nov  1 17:50:19 2013
@@ -0,0 +1,117 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.security;
+
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting;
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting;
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration;
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.isKerberosPropertySetted;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assume.assumeTrue;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.ipc.TestDelayedRpc.TestDelayedImplementation;
+import org.apache.hadoop.hbase.ipc.TestDelayedRpc.TestThread;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.BlockingService;
+
+public class TestSecureRPC {
+  public static RpcServerInterface rpcServer;
+  /**
+   * To run this test, we must specify the following system properties:
+   *<p>
+   * <b> hbase.regionserver.kerberos.principal </b>
+   * <p>
+   * <b> hbase.regionserver.keytab.file </b>
+   */
+  @Test
+  public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception {
+    assumeTrue(isKerberosPropertySetted());
+    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 ugi2 = UserGroupInformation.getCurrentUser();
+
+    // check that the login user is okay:
+    assertSame(ugi, ugi2);
+    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
+    assertEquals(krbPrincipal, ugi.getUserName());
+
+    Configuration conf = getSecuredConfiguration();
+
+    SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class);
+    Mockito.when(securityInfoMock.getServerPrincipal())
+      .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL);
+    SecurityInfo.addInfo("TestDelayedService", securityInfoMock);
+
+    boolean delayReturnValue = false;
+    InetSocketAddress isa = new InetSocketAddress("localhost", 0);
+    TestDelayedImplementation instance = new TestDelayedImplementation(delayReturnValue);
+    BlockingService service =
+        TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
+
+    rpcServer = new RpcServer(null, "testSecuredDelayedRpc",
+        Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
+          isa, conf, new FifoRpcScheduler(conf, 1));
+    rpcServer.start();
+    RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString());
+    try {
+      BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
+          new ServerName(rpcServer.getListenerAddress().getHostName(),
+              rpcServer.getListenerAddress().getPort(), System.currentTimeMillis()),
+          User.getCurrent(), 1000);
+      TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
+        TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
+      List<Integer> results = new ArrayList<Integer>();
+      TestThread th1 = new TestThread(stub, true, results);
+      th1.start();
+      Thread.sleep(100);
+      th1.join();
+
+      assertEquals(0xDEADBEEF, results.get(0).intValue());
+    } finally {
+      rpcClient.stop();
+    }
+  }
+}
\ No newline at end of file