You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ma...@apache.org on 2011/06/16 01:38:44 UTC

svn commit: r1136249 - in /hadoop/common/trunk/common: ./ src/java/org/apache/hadoop/ipc/ src/java/org/apache/hadoop/security/ src/java/org/apache/hadoop/security/authorize/ src/test/core/org/apache/hadoop/ipc/ src/test/core/org/apache/hadoop/metrics2/...

Author: mahadev
Date: Wed Jun 15 23:38:44 2011
New Revision: 1136249

URL: http://svn.apache.org/viewvc?rev=1136249&view=rev
Log:
HADOOP-6929. RPC should have a way to pass Security information other than protocol annotations. (sharad and omalley via mahadev)

Added:
    hadoop/common/trunk/common/src/java/org/apache/hadoop/security/AnnotatedSecurityInfo.java
    hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityInfo.java
Modified:
    hadoop/common/trunk/common/CHANGES.txt
    hadoop/common/trunk/common/build.xml
    hadoop/common/trunk/common/src/java/org/apache/hadoop/ipc/Client.java
    hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/trunk/common/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
    hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestAvroRpc.java
    hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestSaslRPC.java
    hadoop/common/trunk/common/src/test/core/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java

Modified: hadoop/common/trunk/common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/CHANGES.txt?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/CHANGES.txt (original)
+++ hadoop/common/trunk/common/CHANGES.txt Wed Jun 15 23:38:44 2011
@@ -222,6 +222,9 @@ Trunk (unreleased changes)
 
     HADOOP-7384. Allow test-patch to be more flexible about patch format. (todd)
 
+    HADOOP-6929. RPC should have a way to pass Security information other than 
+    protocol annotations. (sharad and omalley via mahadev)
+
   OPTIMIZATIONS
   
     HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole

Modified: hadoop/common/trunk/common/build.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/build.xml?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/build.xml (original)
+++ hadoop/common/trunk/common/build.xml Wed Jun 15 23:38:44 2011
@@ -228,7 +228,7 @@
     <pathelement path="${clover.jar}"/>
     <path refid="ivy-common.classpath"/>
     <path refid="ivy-test.classpath"/>
-    <pathelement location="${build.classes}"/>
+    <pathelement location="${hadoop-common.jar}"/>
     <pathelement location="${test.conf.dir}"/>
   </path>
 <!--
@@ -489,6 +489,10 @@
     <property name="jar.properties.list" value="commons-logging.properties, log4j.properties, hadoop-metrics.properties" />
     <jar jarfile="${build.dir}/${final.name}.jar"
          basedir="${build.classes}">
+      <service type="org.apache.hadoop.security.SecurityInfo">
+        <provider 
+           classname="org.apache.hadoop.security.AnnotatedSecurityInfo"/>
+      </service>
       <manifest>
         <section name="org/apache/hadoop">
           <attribute name="Implementation-Title" value="${ant.project.name}"/>
@@ -562,7 +566,7 @@
   <target name="-classes-compilation"
     depends="compile-core-classes, compile-core-test"/> 
 
-  <target name="compile-core-test" depends="compile-core-classes, ivy-retrieve-test, generate-test-records, generate-avro-records, generate-avro-protocols">
+  <target name="compile-core-test" depends="jar, ivy-retrieve-test, generate-test-records, generate-avro-records, generate-avro-protocols">
     <mkdir dir="${test.core.build.classes}"/>
     <javac 
      encoding="${build.encoding}" 

Modified: hadoop/common/trunk/common/src/java/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/java/org/apache/hadoop/ipc/Client.java?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/src/java/org/apache/hadoop/ipc/Client.java (original)
+++ hadoop/common/trunk/common/src/java/org/apache/hadoop/ipc/Client.java Wed Jun 15 23:38:44 2011
@@ -19,10 +19,8 @@
 package org.apache.hadoop.ipc;
 
 import java.net.InetAddress;
-import java.net.NetworkInterface;
 import java.net.Socket;
 import java.net.InetSocketAddress;
-import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
 import java.net.ConnectException;
@@ -254,7 +252,7 @@ public class Client {
       Class<?> protocol = remoteId.getProtocol();
       this.useSasl = UserGroupInformation.isSecurityEnabled();
       if (useSasl && protocol != null) {
-        TokenInfo tokenInfo = protocol.getAnnotation(TokenInfo.class);
+        TokenInfo tokenInfo = SecurityUtil.getTokenInfo(protocol);
         if (tokenInfo != null) {
           TokenSelector<? extends TokenIdentifier> tokenSelector = null;
           try {
@@ -269,7 +267,7 @@ public class Client {
               .getHostAddress() + ":" + addr.getPort()), 
               ticket.getTokens());
         }
-        KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
+        KerberosInfo krbInfo = SecurityUtil.getKerberosInfo(protocol);
         if (krbInfo != null) {
           serverPrincipal = remoteId.getServerPrincipal();
           if (LOG.isDebugEnabled()) {
@@ -1283,7 +1281,7 @@ public class Client {
       if (!UserGroupInformation.isSecurityEnabled() || protocol == null) {
         return null;
       }
-      KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
+      KerberosInfo krbInfo = SecurityUtil.getKerberosInfo(protocol);
       if (krbInfo != null) {
         String serverKey = krbInfo.serverPrincipal();
         if (serverKey == null) {

Added: hadoop/common/trunk/common/src/java/org/apache/hadoop/security/AnnotatedSecurityInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/java/org/apache/hadoop/security/AnnotatedSecurityInfo.java?rev=1136249&view=auto
==============================================================================
--- hadoop/common/trunk/common/src/java/org/apache/hadoop/security/AnnotatedSecurityInfo.java (added)
+++ hadoop/common/trunk/common/src/java/org/apache/hadoop/security/AnnotatedSecurityInfo.java Wed Jun 15 23:38:44 2011
@@ -0,0 +1,39 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.security.token.TokenInfo;
+
+/**
+ * Constructs SecurityInfo from Annotations provided in protocol interface.
+ */
+public class AnnotatedSecurityInfo extends SecurityInfo {
+
+  @Override
+  public KerberosInfo getKerberosInfo(Class<?> protocol) {
+    return protocol.getAnnotation(KerberosInfo.class);
+  }
+
+  @Override
+  public TokenInfo getTokenInfo(Class<?> protocol) {
+    return protocol.getAnnotation(TokenInfo.class);
+  }
+
+  
+}

Added: hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityInfo.java?rev=1136249&view=auto
==============================================================================
--- hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityInfo.java (added)
+++ hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityInfo.java Wed Jun 15 23:38:44 2011
@@ -0,0 +1,43 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.security.token.TokenInfo;
+
+/**
+ * Interface used by RPC to get the Security information for a given 
+ * protocol.
+ */
+public abstract class SecurityInfo {
+
+  /**
+   * Get the KerberosInfo for a given protocol.
+   * @param protocol interface class
+   * @return KerberosInfo
+   */
+  public abstract KerberosInfo getKerberosInfo(Class<?> protocol);
+
+  /**
+   * Get the TokenInfo for a given protocol.
+   * @param protocol interface class
+   * @return TokenInfo instance
+   */
+  public abstract TokenInfo getTokenInfo(Class<?> protocol);
+
+}

Modified: hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityUtil.java?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/trunk/common/src/java/org/apache/hadoop/security/SecurityUtil.java Wed Jun 15 23:38:44 2011
@@ -22,6 +22,7 @@ import java.net.URI;
 import java.net.URL;
 import java.net.UnknownHostException;
 import java.security.AccessController;
+import java.util.ServiceLoader;
 import java.util.Set;
 
 import javax.security.auth.Subject;
@@ -33,8 +34,8 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.TokenInfo;
 
 import sun.security.jgss.krb5.Krb5Util;
 import sun.security.krb5.Credentials;
@@ -291,4 +292,62 @@ public class SecurityUtil {
   public static String getHostFromPrincipal(String principalName) {
     return new KerberosName(principalName).getHostName();
   }
+
+  private static ServiceLoader<SecurityInfo> securityInfoProviders = 
+    ServiceLoader.load(SecurityInfo.class);
+  private static SecurityInfo[] testProviders = new SecurityInfo[0];
+
+  /**
+   * Test setup method to register additional providers.
+   * @param providers a list of high priority providers to use
+   */
+  @InterfaceAudience.Private
+  public static void setSecurityInfoProviders(SecurityInfo... providers) {
+    testProviders = providers;
+  }
+  
+  /**
+   * Look up the KerberosInfo for a given protocol. It searches all known
+   * SecurityInfo providers.
+   * @param protocol the protocol class to get the information for
+   * @return the KerberosInfo or null if it has no KerberosInfo defined
+   */
+  public static KerberosInfo getKerberosInfo(Class<?> protocol) {
+    for(SecurityInfo provider: testProviders) {
+      KerberosInfo result = provider.getKerberosInfo(protocol);
+      if (result != null) {
+        return result;
+      }
+    }
+    for(SecurityInfo provider: securityInfoProviders) {
+      KerberosInfo result = provider.getKerberosInfo(protocol);
+      if (result != null) {
+        return result;
+      }
+    }
+    return null;
+  }
+ 
+  /**
+   * Look up the TokenInfo for a given protocol. It searches all known
+   * SecurityInfo providers.
+   * @param protocol The protocol class to get the information for.
+   * @return the TokenInfo or null if it has no KerberosInfo defined
+   */
+  public static TokenInfo getTokenInfo(Class<?> protocol) {
+    for(SecurityInfo provider: testProviders) {
+      TokenInfo result = provider.getTokenInfo(protocol);
+      if (result != null) {
+        return result;
+      }      
+    }
+    for(SecurityInfo provider: securityInfoProviders) {
+      TokenInfo result = provider.getTokenInfo(protocol);
+      if (result != null) {
+        return result;
+      }
+    } 
+    return null;
+  }
+
 }

Modified: hadoop/common/trunk/common/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java (original)
+++ hadoop/common/trunk/common/src/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java Wed Jun 15 23:38:44 2011
@@ -41,8 +41,6 @@ import org.apache.hadoop.security.UserGr
 @InterfaceStability.Evolving
 public class ServiceAuthorizationManager {
   private static final String HADOOP_POLICY_FILE = "hadoop-policy.xml";
-  private static final Log LOG = LogFactory
-  .getLog(ServiceAuthorizationManager.class);
 
   private Map<Class<?>, AccessControlList> protocolToAcl =
     new IdentityHashMap<Class<?>, AccessControlList>();
@@ -86,7 +84,7 @@ public class ServiceAuthorizationManager
     }
     
     // get client principal key to verify (if available)
-    KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
+    KerberosInfo krbInfo = SecurityUtil.getKerberosInfo(protocol);
     String clientPrincipal = null; 
     if (krbInfo != null) {
       String clientKey = krbInfo.clientPrincipal();

Modified: hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestAvroRpc.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestAvroRpc.java?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestAvroRpc.java (original)
+++ hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestAvroRpc.java Wed Jun 15 23:38:44 2011
@@ -18,8 +18,14 @@
 
 package org.apache.hadoop.ipc;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+
+import java.io.IOException;
 import java.net.InetSocketAddress;
 
+import javax.security.sasl.Sasl;
+
+import junit.framework.Assert;
 import junit.framework.TestCase;
 
 import org.apache.avro.ipc.AvroRemoteException;
@@ -27,7 +33,16 @@ import org.apache.avro.util.Utf8;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.TestSaslRPC.CustomSecurityInfo;
+import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier;
+import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityInfo;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 
 /** Unit tests for AvroRpc. */
 public class TestAvroRpc extends TestCase {
@@ -36,8 +51,6 @@ public class TestAvroRpc extends TestCas
   public static final Log LOG =
     LogFactory.getLog(TestAvroRpc.class);
   
-  private static Configuration conf = new Configuration();
-
   int datasize = 1024*100;
   int numThreads = 50;
 
@@ -56,19 +69,47 @@ public class TestAvroRpc extends TestCas
     }
   }
 
-  public void testCalls() throws Exception {
+  public void testReflect() throws Exception {
+    testReflect(false);
+  }
+
+  public void testSecureReflect() throws Exception {
+    testReflect(true);
+  }
+
+  public void testSpecific() throws Exception {
+    testSpecific(false);
+  }
+
+  public void testSecureSpecific() throws Exception {
+    testSpecific(true);
+  }
+
+  private void testReflect(boolean secure) throws Exception {
     Configuration conf = new Configuration();
+    TestTokenSecretManager sm = null;
+    if (secure) {
+      makeSecure(conf);
+      sm = new TestTokenSecretManager();
+    }
+    UserGroupInformation.setConfiguration(conf);
     RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
     Server server = RPC.getServer(AvroTestProtocol.class,
-                                  new TestImpl(), ADDRESS, 0, conf);
-    AvroTestProtocol proxy = null;
+                                  new TestImpl(), ADDRESS, 0, 5, true, 
+                                  conf, sm);
     try {
       server.start();
-
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
-      proxy =
+
+      if (secure) {
+        addToken(sm, addr);
+        //QOP must be auth
+        Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
+      }
+
+      AvroTestProtocol proxy =
         (AvroTestProtocol)RPC.getProxy(AvroTestProtocol.class, 0, addr, conf);
-      
+
       proxy.ping();
 
       String echo = proxy.echo("hello world");
@@ -89,23 +130,62 @@ public class TestAvroRpc extends TestCas
       assertTrue(caught);
 
     } finally {
+      resetSecurity();
       server.stop();
     }
   }
 
-  public void testAvroSpecificRpc() throws Exception {
+  private void makeSecure(Configuration conf) {
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    conf.set("hadoop.rpc.socket.factory.class.default", "");
+    //Avro doesn't work with security annotations on protocol.
+    //Avro works ONLY with custom security context
+    SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
+  }
+  
+  private void resetSecurity() {
+    SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
+  }
+
+  private void addToken(TestTokenSecretManager sm, 
+      InetSocketAddress addr) throws IOException {
+    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    
+    TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
+        .getUserName()));
+    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
+        sm);
+    Text host = new Text(addr.getAddress().getHostAddress() + ":"
+        + addr.getPort());
+    token.setService(host);
+    LOG.info("Service IP address for token is " + host);
+    current.addToken(token);
+  }
+
+  private void testSpecific(boolean secure) throws Exception {
     Configuration conf = new Configuration();
+    TestTokenSecretManager sm = null;
+    if (secure) {
+      makeSecure(conf);
+      sm = new TestTokenSecretManager();
+    }
+    UserGroupInformation.setConfiguration(conf);
     RPC.setProtocolEngine(conf, AvroSpecificTestProtocol.class, 
         AvroSpecificRpcEngine.class);
     Server server = RPC.getServer(AvroSpecificTestProtocol.class,
-                                  new AvroSpecificTestProtocolImpl(), 
-                                  ADDRESS, 0, conf);
-    AvroSpecificTestProtocol proxy = null;
+        new AvroSpecificTestProtocolImpl(), ADDRESS, 0, 5, true, 
+        conf, sm);
     try {
       server.start();
-
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
-      proxy =
+
+      if (secure) {
+        addToken(sm, addr);
+        //QOP must be auth
+        Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
+      }
+
+      AvroSpecificTestProtocol proxy =
         (AvroSpecificTestProtocol)RPC.getProxy(AvroSpecificTestProtocol.class, 
             0, addr, conf);
       
@@ -116,6 +196,7 @@ public class TestAvroRpc extends TestCas
       assertEquals(3, intResult);
 
     } finally {
+      resetSecurity();
       server.stop();
     }
   }
@@ -134,5 +215,5 @@ public class TestAvroRpc extends TestCas
     }
     
   }
-  
+
 }

Modified: hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestSaslRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestSaslRPC.java?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestSaslRPC.java (original)
+++ hadoop/common/trunk/common/src/test/core/org/apache/hadoop/ipc/TestSaslRPC.java Wed Jun 15 23:38:44 2011
@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.lang.annotation.Annotation;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
@@ -33,28 +36,28 @@ import javax.security.sasl.Sasl;
 
 import junit.framework.Assert;
 
-import org.apache.commons.logging.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenInfo;
-import org.apache.hadoop.security.token.TokenSelector;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.SaslInputStream;
 import org.apache.hadoop.security.SaslRpcClient;
 import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityInfo;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.TestUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenInfo;
+import org.apache.hadoop.security.token.TokenSelector;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -187,6 +190,42 @@ public class TestSaslRPC {
     }
   }
 
+  public static class CustomSecurityInfo extends SecurityInfo {
+
+    @Override
+    public KerberosInfo getKerberosInfo(Class<?> protocol) {
+      return new KerberosInfo() {
+        @Override
+        public Class<? extends Annotation> annotationType() {
+          return null;
+        }
+        @Override
+        public String serverPrincipal() {
+          return SERVER_PRINCIPAL_KEY;
+        }
+        @Override
+        public String clientPrincipal() {
+          return null;
+        }
+      };
+    }
+
+    @Override
+    public TokenInfo getTokenInfo(Class<?> protocol) {
+      return new TokenInfo() {
+        @Override
+        public Class<? extends TokenSelector<? extends 
+            TokenIdentifier>> value() {
+          return TestTokenSelector.class;
+        }
+        @Override
+        public Class<? extends Annotation> annotationType() {
+          return null;
+        }
+      };
+    }
+  }
+
   @Test
   public void testDigestRpc() throws Exception {
     TestTokenSecretManager sm = new TestTokenSecretManager();
@@ -195,7 +234,21 @@ public class TestSaslRPC {
 
     doDigestRpc(server, sm);
   }
-  
+
+  @Test
+  public void testDigestRpcWithoutAnnotation() throws Exception {
+    TestTokenSecretManager sm = new TestTokenSecretManager();
+    try {
+      SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
+      final Server server = RPC.getServer(TestSaslProtocol.class,
+                                          new TestSaslImpl(), ADDRESS, 0, 5, 
+                                          true, conf, sm);
+      doDigestRpc(server, sm);
+    } finally {
+      SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
+    }
+  }
+
   @Test
   public void testSecureToInsecureRpc() throws Exception {
     Server server = RPC.getServer(TestSaslProtocol.class,
@@ -223,8 +276,8 @@ public class TestSaslRPC {
     assertTrue(succeeded);
   }
   
-  private void doDigestRpc(Server server, TestTokenSecretManager sm)
-      throws Exception {
+  private void doDigestRpc(Server server, TestTokenSecretManager sm
+                           ) throws Exception {
     server.start();
 
     final UserGroupInformation current = UserGroupInformation.getCurrentUser();

Modified: hadoop/common/trunk/common/src/test/core/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/common/src/test/core/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java?rev=1136249&r1=1136248&r2=1136249&view=diff
==============================================================================
--- hadoop/common/trunk/common/src/test/core/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java (original)
+++ hadoop/common/trunk/common/src/test/core/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java Wed Jun 15 23:38:44 2011
@@ -138,6 +138,6 @@ public class TestMetricsConfig {
    * @return the filename
    */
   public static String getTestFilename(String basename) {
-    return "build/classes/"+ basename +".properties";
+    return "build/test/"+ basename +".properties";
   }
 }