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 sz...@apache.org on 2012/10/19 04:27:38 UTC

svn commit: r1399950 [2/17] - in /hadoop/common/branches/HDFS-2802/hadoop-common-project: hadoop-annotations/ hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/ hadoop-auth-examples/ hadoop-auth/ hadoop-auth/src/main/java/org/apac...

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java Fri Oct 19 02:25:55 2012
@@ -19,6 +19,8 @@ import org.ietf.jgss.GSSContext;
 import org.ietf.jgss.GSSManager;
 import org.ietf.jgss.GSSName;
 import org.ietf.jgss.Oid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.security.auth.Subject;
 import javax.security.auth.login.AppConfigurationEntry;
@@ -44,6 +46,9 @@ import java.util.Map;
  * sequence.
  */
 public class KerberosAuthenticator implements Authenticator {
+  
+  private static Logger LOG = LoggerFactory.getLogger(
+      KerberosAuthenticator.class);
 
   /**
    * HTTP header used by the SPNEGO server endpoint during an authentication sequence.
@@ -113,6 +118,18 @@ public class KerberosAuthenticator imple
   private URL url;
   private HttpURLConnection conn;
   private Base64 base64;
+  private ConnectionConfigurator connConfigurator;
+
+  /**
+   * Sets a {@link ConnectionConfigurator} instance to use for
+   * configuring connections.
+   *
+   * @param configurator the {@link ConnectionConfigurator} instance.
+   */
+  @Override
+  public void setConnectionConfigurator(ConnectionConfigurator configurator) {
+    connConfigurator = configurator;
+  }
 
   /**
    * Performs SPNEGO authentication against the specified URL.
@@ -135,11 +152,23 @@ public class KerberosAuthenticator imple
       this.url = url;
       base64 = new Base64(0);
       conn = (HttpURLConnection) url.openConnection();
+      if (connConfigurator != null) {
+        conn = connConfigurator.configure(conn);
+      }
       conn.setRequestMethod(AUTH_HTTP_METHOD);
       conn.connect();
-      if (isNegotiate()) {
+      
+      if (conn.getRequestProperty(AUTHORIZATION) != null && conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
+        LOG.debug("JDK performed authentication on our behalf.");
+        // If the JDK already did the SPNEGO back-and-forth for
+        // us, just pull out the token.
+        AuthenticatedURL.extractToken(conn, token);
+        return;
+      } else if (isNegotiate()) {
+        LOG.debug("Performing our own SPNEGO sequence.");
         doSpnegoSequence(token);
       } else {
+        LOG.debug("Using fallback authenticator sequence.");
         getFallBackAuthenticator().authenticate(url, token);
       }
     }
@@ -153,7 +182,11 @@ public class KerberosAuthenticator imple
    * @return the fallback {@link Authenticator}.
    */
   protected Authenticator getFallBackAuthenticator() {
-    return new PseudoAuthenticator();
+    Authenticator auth = new PseudoAuthenticator();
+    if (connConfigurator != null) {
+      auth.setConnectionConfigurator(connConfigurator);
+    }
+    return auth;
   }
 
   /*
@@ -182,11 +215,16 @@ public class KerberosAuthenticator imple
       AccessControlContext context = AccessController.getContext();
       Subject subject = Subject.getSubject(context);
       if (subject == null) {
+        LOG.debug("No subject in context, logging in");
         subject = new Subject();
         LoginContext login = new LoginContext("", subject,
             null, new KerberosConfiguration());
         login.login();
       }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using subject: " + subject);
+      }
       Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
 
         @Override
@@ -194,7 +232,8 @@ public class KerberosAuthenticator imple
           GSSContext gssContext = null;
           try {
             GSSManager gssManager = GSSManager.getInstance();
-            String servicePrincipal = "HTTP/" + KerberosAuthenticator.this.url.getHost();
+            String servicePrincipal = KerberosUtil.getServicePrincipal("HTTP",
+                KerberosAuthenticator.this.url.getHost());
             Oid oid = KerberosUtil.getOidInstance("NT_GSS_KRB5_PRINCIPAL");
             GSSName serviceName = gssManager.createName(servicePrincipal,
                                                         oid);
@@ -244,6 +283,9 @@ public class KerberosAuthenticator imple
   private void sendToken(byte[] outToken) throws IOException, AuthenticationException {
     String token = base64.encodeToString(outToken);
     conn = (HttpURLConnection) url.openConnection();
+    if (connConfigurator != null) {
+      conn = connConfigurator.configure(conn);
+    }
     conn.setRequestMethod(AUTH_HTTP_METHOD);
     conn.setRequestProperty(AUTHORIZATION, NEGOTIATE + " " + token);
     conn.connect();

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java Fri Oct 19 02:25:55 2012
@@ -32,6 +32,19 @@ public class PseudoAuthenticator impleme
 
   private static final String USER_NAME_EQ = USER_NAME + "=";
 
+  private ConnectionConfigurator connConfigurator;
+
+  /**
+   * Sets a {@link ConnectionConfigurator} instance to use for
+   * configuring connections.
+   *
+   * @param configurator the {@link ConnectionConfigurator} instance.
+   */
+  @Override
+  public void setConnectionConfigurator(ConnectionConfigurator configurator) {
+    connConfigurator = configurator;
+  }
+
   /**
    * Performs simple authentication against the specified URL.
    * <p/>
@@ -56,6 +69,9 @@ public class PseudoAuthenticator impleme
     strUrl += paramSeparator + USER_NAME_EQ + getUserName();
     url = new URL(strUrl);
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    if (connConfigurator != null) {
+      conn = connConfigurator.configure(conn);
+    }
     conn.setRequestMethod("OPTIONS");
     conn.connect();
     AuthenticatedURL.extractToken(conn, token);

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java Fri Oct 19 02:25:55 2012
@@ -20,6 +20,9 @@ package org.apache.hadoop.security.authe
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Locale;
 
 import org.ietf.jgss.GSSException;
 import org.ietf.jgss.Oid;
@@ -65,4 +68,33 @@ public class KerberosUtil {
          new Class[0]);
     return (String)getDefaultRealmMethod.invoke(kerbConf, new Object[0]);
   }
+  
+  /* Return fqdn of the current host */
+  static String getLocalHostName() throws UnknownHostException {
+    return InetAddress.getLocalHost().getCanonicalHostName();
+  }
+  
+  /**
+   * Create Kerberos principal for a given service and hostname. It converts
+   * hostname to lower case. If hostname is null or "0.0.0.0", it uses
+   * dynamically looked-up fqdn of the current host instead.
+   * 
+   * @param service
+   *          Service for which you want to generate the principal.
+   * @param hostname
+   *          Fully-qualified domain name.
+   * @return Converted Kerberos principal name.
+   * @throws UnknownHostException
+   *           If no IP address for the local host could be found.
+   */
+  public static final String getServicePrincipal(String service, String hostname)
+      throws UnknownHostException {
+    String fqdn = hostname;
+    if (null == fqdn || fqdn.equals("") || fqdn.equals("0.0.0.0")) {
+      fqdn = getLocalHostName();
+    }
+    // convert hostname to lowercase as kerberos does not work with hostnames
+    // with uppercase characters.
+    return service + "/" + fqdn.toLowerCase(Locale.US);
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java Fri Oct 19 02:25:55 2012
@@ -13,8 +13,10 @@
  */
 package org.apache.hadoop.security.authentication.client;
 
+import junit.framework.Assert;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import junit.framework.TestCase;
+import org.mockito.Mockito;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.FilterHolder;
@@ -113,6 +115,18 @@ public abstract class AuthenticatorTestC
     return "http://" + host + ":" + port + "/foo/bar";
   }
 
+  private static class TestConnectionConfigurator
+      implements ConnectionConfigurator {
+    boolean invoked;
+
+    @Override
+    public HttpURLConnection configure(HttpURLConnection conn)
+        throws IOException {
+      invoked = true;
+      return conn;
+    }
+  }
+
   private String POST = "test";
 
   protected void _testAuthentication(Authenticator authenticator, boolean doPost) throws Exception {
@@ -120,8 +134,12 @@ public abstract class AuthenticatorTestC
     try {
       URL url = new URL(getBaseURL());
       AuthenticatedURL.Token token = new AuthenticatedURL.Token();
-      AuthenticatedURL aUrl = new AuthenticatedURL(authenticator);
+      Assert.assertFalse(token.isSet());
+      TestConnectionConfigurator connConf = new TestConnectionConfigurator();
+      AuthenticatedURL aUrl = new AuthenticatedURL(authenticator, connConf);
       HttpURLConnection conn = aUrl.openConnection(url, token);
+      Assert.assertTrue(token.isSet());
+      Assert.assertTrue(connConf.invoked);
       String tokenStr = token.toString();
       if (doPost) {
         conn.setRequestMethod("POST");

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,7 @@ import junit.framework.TestCase;
 import org.mockito.Mockito;
 
 import java.net.HttpURLConnection;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -114,4 +115,21 @@ public class TestAuthenticatedURL extend
     }
   }
 
+  public void testConnectionConfigurator() throws Exception {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getResponseCode()).
+        thenReturn(HttpURLConnection.HTTP_UNAUTHORIZED);
+
+    ConnectionConfigurator connConf =
+        Mockito.mock(ConnectionConfigurator.class);
+    Mockito.when(connConf.configure(Mockito.<HttpURLConnection>any())).
+        thenReturn(conn);
+
+    Authenticator authenticator = Mockito.mock(Authenticator.class);
+
+    AuthenticatedURL aURL = new AuthenticatedURL(authenticator, connConf);
+    aURL.openConnection(new URL("http://foo"), new AuthenticatedURL.Token());
+    Mockito.verify(connConf).configure(Mockito.<HttpURLConnection>any());
+  }
+
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java Fri Oct 19 02:25:55 2012
@@ -43,6 +43,14 @@ public class TestKerberosAuthenticator e
     _testAuthentication(new KerberosAuthenticator(), false);
   }
 
+  public void testFallbacktoPseudoAuthenticatorAnonymous() throws Exception {
+    Properties props = new Properties();
+    props.setProperty(AuthenticationFilter.AUTH_TYPE, "simple");
+    props.setProperty(PseudoAuthenticationHandler.ANONYMOUS_ALLOWED, "true");
+    setAuthenticationHandlerConfig(props);
+    _testAuthentication(new KerberosAuthenticator(), false);
+  }
+
   public void testNotAuthenticated() throws Exception {
     setAuthenticationHandlerConfig(getAuthenticationHandlerConfiguration());
     start();

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt Fri Oct 19 02:25:55 2012
@@ -1,12 +1,14 @@
 Hadoop Change Log
 
-Trunk (unreleased changes)
+Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
 
     HADOOP-8124. Remove the deprecated FSDataOutputStream constructor,
     FSDataOutputStream.sync() and Syncable.sync().  (szetszwo)
 
+    HADOOP-8886. Remove KFS support. (eli)
+
   NEW FEATURES
     
     HADOOP-8469. Make NetworkTopology class pluggable.  (Junping Du via
@@ -70,8 +72,8 @@ Trunk (unreleased changes)
     HADOOP-8360. empty-configuration.xml fails xml validation
     (Radim Kolar via harsh)
 
-    HADOOP-8367 Improve documentation of declaringClassProtocolName in rpc headers 
-                (Sanjay Radia)
+    HADOOP-8367 Improve documentation of declaringClassProtocolName in 
+    rpc headers. (Sanjay Radia)
 
     HADOOP-8415. Add getDouble() and setDouble() in
     org.apache.hadoop.conf.Configuration (Jan van der Lugt via harsh)
@@ -88,6 +90,45 @@ Trunk (unreleased changes)
     HADOOP-8523. test-patch.sh doesn't validate patches before building
     (Jack Dintruff via jeagles)
 
+    HADOOP-8624. ProtobufRpcEngine should log all RPCs if TRACE logging is
+    enabled (todd)
+
+    HADOOP-8711. IPC Server supports adding exceptions for which
+    the message is printed and the stack trace is not printed to avoid chatter.
+    (Brandon Li via Suresh)
+
+    HADOOP-8719. Workaround for kerberos-related log errors upon running any
+    hadoop command on OSX. (Jianbin Wei via harsh)
+
+    HADOOP-8619. WritableComparator must implement no-arg constructor.
+    (Chris Douglas via Suresh)
+
+    HADOOP-8814. Replace string equals "" by String#isEmpty().
+    (Brandon Li via suresh)
+
+    HADOOP-8588. SerializationFactory shouldn't throw a
+    NullPointerException if the serializations list is empty.
+    (Sho Shimauchi via harsh)
+
+    HADOOP-7930. Kerberos relogin interval in UserGroupInformation
+    should be configurable (Robert Kanter via harsh)
+
+    HADOOP-8838. Colorize the test-patch output sent to JIRA (Harsh J via
+    bobby)
+
+    HADOOP-8840. Fix the test-patch colorizer to cover all sorts of +1 lines.
+    (Harsh J via bobby)
+
+    HADOOP-8864. Addendum to HADOOP-8840: Add a coloring case for +0 results
+    too. (harsh)
+
+    HADOOP-8910. Add examples to GlobExpander#expand method. (suresh)
+
+    HADOOP-8920. Add more javadoc to metrics2 related classes. (suresh)
+
+    HADOOP-8776. Provide an option in test-patch that can enable/disable
+    compiling native code. (Chris Nauroth via suresh)
+
   BUG FIXES
 
     HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.
@@ -178,17 +219,179 @@ Trunk (unreleased changes)
     HADOOP-8521. Port StreamInputFormat to new Map Reduce API (madhukara
     phatak via bobby)
 
+    HADOOP-8593. Add missed @Override annotations in Metric/Metrics2 package.
+    (Brandon Li via suresh)
+
+    HADOOP-8623. hadoop jar command should respect HADOOP_OPTS.
+    (Steven Willis via suresh)
+
+    HADOOP-8684. Deadlock between WritableComparator and WritableComparable.
+    (Jing Zhao via suresh)
+
+    HADOOP-8786. HttpServer continues to start even if AuthenticationFilter
+    fails to init (todd)
+
+    HADOOP-8767. Secondary namenode is started on slave nodes instead of
+    master nodes. (Giovanni Delussu via suresh)
+
+    HADOOP-8818. Use equals instead == in MD5MD5CRC32FileChecksum
+    and TFileDumper. (Brandon Li via suresh)
+
+    HADOOP-8821. Fix findbugs warning related to concatenating string in a 
+    for loop in Configuration#dumpDeprecatedKeys(). (suresh)
+
+    HADOOP-7256. Resource leak during failure scenario of closing
+    of resources. (Ramkrishna S. Vasudevan via harsh)
+
+    HADOOP-8151. Error handling in snappy decompressor throws invalid
+    exceptions. (Matt Foley via harsh)
+
+    HADOOP-8813. Add InterfaceAudience and InterfaceStability annotations
+    to RPC Server and Client classes. (Brandon Li via suresh)
+
+    HADOOP-8815. RandomDatum needs to override hashCode().
+    (Brandon Li via suresh)
+
+    HADOOP-8436. NPE In getLocalPathForWrite ( path, conf ) when the
+    required context item is not configured
+    (Brahma Reddy Battula via harsh)
+
+    HADOOP-3957. Change MutableQuantiles to use a shared thread for rolling
+    over metrics. (Andrew Wang via todd)
+
+    HADOOP-8386. hadoop script doesn't work if 'cd' prints to stdout
+    (default behavior in some bash setups (esp. Ubuntu))
+    (Chiristopher Berner and Andy Isaacson via harsh)
+
+    HADOOP-8839. test-patch's -1 on @author tag presence doesn't cause
+    a -1 to the overall result (harsh)
+
+    HADOOP-8918. test-patch.sh is parsing modified files wrong.
+    (Raja Aluri via suresh)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
 
-Branch-2 ( Unreleased changes )
+Release 2.0.3-alpha - Unreleased 
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+    HADOOP-8597. Permit FsShell's text command to read Avro files.
+    (Ivan Vladimirov Ivanov via cutting)
+
+  IMPROVEMENTS
+
+    HADOOP-8789. Tests setLevel(Level.OFF) should be Level.ERROR.
+    (Andy Isaacson via eli)
+
+    HADOOP-8755. Print thread dump when tests fail due to timeout. (Andrey
+    Klochkov via atm)
+
+    HADOOP-8806. libhadoop.so: dlopen should be better at locating
+    libsnappy.so, etc. (Colin Patrick McCabe via eli)
+
+    HADOOP-8812. ExitUtil#terminate should print Exception#toString. (eli)
+
+    HADOOP-8736. Add Builder for building RPC server. (Brandon Li via Suresh)
+
+    HADOOP-8851. Use -XX:+HeapDumpOnOutOfMemoryError JVM option in the forked
+    tests. (Ivan A. Veselovsky via atm)
+
+    HADOOP-8783. Improve RPC.Server's digest auth (daryn)
+
+    HADOOP-8889. Upgrade to Surefire 2.12.3 (todd)
+
+    HADOOP-8804. Improve Web UIs when the wildcard address is used.
+    (Senthil Kumar via eli)
+
+    HADOOP-8894. GenericTestUtils.waitFor should dump thread stacks on timeout
+    (todd)
+
+    HADOOP-8909. Hadoop Common Maven protoc calls must not depend on external
+    sh script. (Chris Nauroth via suresh)
+
+    HADOOP-8911. CRLF characters in source and text files.
+    (Raja Aluri via suresh)
+
+    HADOOP-8912. Add .gitattributes file to prevent CRLF and LF mismatches
+    for source and text files. (Raja Aluri via suresh)
+
+    HADOOP-8784. Improve IPC.Client's token use (daryn)
+
+    HADOOP-8929. Add toString, other improvements for SampleQuantiles (todd)
+
+    HADOOP-8922. Provide alternate JSONP output for JMXJsonServlet to allow
+    javascript in browser dashboard (Damien Hardy via bobby)
+
+    HADOOP-8931. Add Java version to startup message. (eli)
+
+    HADOOP-8925. Remove the packaging. (eli)
+
+    HADOOP-8887. Use a Maven plugin to build the native code using CMake. (cmccabe via tucu)
+
+  OPTIMIZATIONS
+
+    HADOOP-8866. SampleQuantiles#query is O(N^2) instead of O(N). (Andrew Wang
+    via atm)
+
+    HADOOP-8926. hadoop.util.PureJavaCrc32 cache hit-ratio is low for static
+    data (Gopal V via bobby)
+
+  BUG FIXES
+
+    HADOOP-8795. BASH tab completion doesn't look in PATH, assumes path to
+    executable is specified. (Sean Mackrory via atm)
+
+    HADOOP-8780. Update DeprecatedProperties apt file. (Ahmed Radwan via
+    tomwhite)
+
+    HADOOP-8833. fs -text should make sure to call inputstream.seek(0)
+    before using input stream. (tomwhite and harsh)
+
+    HADOOP-8791. Fix rm command documentation to indicte it deletes
+    files and not directories. (Jing Zhao via suresh)
+
+    HADOOP-8616. ViewFS configuration requires a trailing slash. (Sandy Ryza
+    via atm)
+
+    HADOOP-8756. Fix SEGV when libsnappy is in java.library.path but
+    not LD_LIBRARY_PATH. (Colin Patrick McCabe via eli)
+
+    HADOOP-8881. FileBasedKeyStoresFactory initialization logging should
+    be debug not info. (tucu)
+
+    HADOOP-8913. hadoop-metrics2.properties should give units in comment 
+    for sampling period. (Sandy Ryza via suresh)
+
+    HADOOP-8878. Uppercase namenode hostname causes hadoop dfs calls with 
+    webhdfs filesystem and fsck to fail when security is on.
+    (Arpit Gupta via suresh)
+
+    HADOOP-8901. GZip and Snappy support may not work without unversioned
+    libraries (Colin Patrick McCabe via todd)
+
+    HADOOP-8883. Anonymous fallback in KerberosAuthenticator is broken.
+    (rkanter via tucu)
+
+    HADOOP-8900. BuiltInGzipDecompressor throws IOException - stored gzip size
+    doesn't match decompressed size. (Slavik Krassovsky via suresh)
+
+Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
 
     HADOOP-8388. Remove unused BlockLocation serialization.
     (Colin Patrick McCabe via eli)
 
+    HADOOP-8689. Make trash a server side configuration option. (eli)
+
+    HADOOP-8710. Remove ability for users to easily run the trash emptire. (eli)
+    
+    HADOOP-8794. Rename YARN_HOME to HADOOP_YARN_HOME. (vinodkv via acmurthy)
+
   NEW FEATURES
  
     HDFS-3042. Automatic failover support for NameNode HA (todd)
@@ -202,6 +405,16 @@ Branch-2 ( Unreleased changes )
 
     HADOOP-8465. hadoop-auth should support ephemeral authentication (tucu)
 
+    HADOOP-8644. AuthenticatedURL should be able to use SSLFactory. (tucu)
+
+    HADOOP-8581. add support for HTTPS to the web UIs. (tucu)
+
+    HADOOP-7754. Expose file descriptors from Hadoop-wrapped local 
+    FileSystems (todd and ahmed via tucu)
+
+    HADOOP-8240. Add a new API to allow users to specify a checksum type
+    on FileSystem.create(..).  (Kihwal Lee via szetszwo)
+
   IMPROVEMENTS
 
     HADOOP-8340. SNAPSHOT build versions should compare as less than their eventual
@@ -252,6 +465,52 @@ Branch-2 ( Unreleased changes )
     EOFException on Snappy or LZO block-compressed data
     (todd via harsh)
 
+    HADOOP-8541. Better high-percentile latency metrics. (Andrew Wang via atm)
+
+    HADOOP-8362. Improve exception message when Configuration.set() is
+    called with a null key or value. (Madhukara Phatak
+    and Suresh Srinivas via harsh)
+
+    HADOOP-7818. DiskChecker#checkDir should fail if the directory is
+    not executable. (Madhukara Phatak via harsh)
+
+    HADOOP-8531. SequenceFile Writer can throw out a better error if a
+    serializer or deserializer isn't available
+    (Madhukara Phatak via harsh)
+
+    HADOOP-8609. IPC server logs a useless message when shutting down socket.
+    (Jon Zuanich via atm)
+
+    HADOOP-8620. Add -Drequire.fuse and -Drequire.snappy. (Colin
+    Patrick McCabe via eli)
+
+    HADOOP-8687. Upgrade log4j to 1.2.17. (eli)
+
+    HADOOP-8278. Make sure components declare correct set of dependencies.
+    (tomwhite)
+
+    HADOOP-8700.  Use enum to define the checksum constants in DataChecksum.
+    (szetszwo)
+
+    HADOOP-8686. Fix warnings in native code. (Colin Patrick McCabe via eli)
+
+    HADOOP-8239. Add subclasses of MD5MD5CRC32FileChecksum to support file
+    checksum with CRC32C.  (Kihwal Lee via szetszwo)
+
+    HADOOP-8075. Lower native-hadoop library log from info to debug.
+    (Hızır Sefa İrken via eli)
+
+    HADOOP-8748. Refactor DFSClient retry utility methods to a new class
+    in org.apache.hadoop.io.retry.  (Arun C Murthy via szetszwo)
+
+    HADOOP-8754. Deprecate all the RPC.getServer() variants.  (Brandon Li
+    via szetszwo)
+
+    HADOOP-8801. ExitUtil#terminate should capture the exception stack trace. (eli)
+
+    HADOOP-8819. Incorrectly & is used instead of && in some file system 
+    implementations. (Brandon Li via suresh)
+
   BUG FIXES
 
     HADOOP-8372. NetUtils.normalizeHostName() incorrectly handles hostname
@@ -331,6 +590,83 @@ Branch-2 ( Unreleased changes )
     HADOOP-3886. Error in javadoc of Reporter, Mapper and Progressable
     (Jingguo Yao via harsh)
 
+    HADOOP-8587. HarFileSystem access of harMetaCache isn't threadsafe. (eli)
+
+    HADOOP-8585. Fix initialization circularity between UserGroupInformation
+    and HadoopConfiguration. (Colin Patrick McCabe via atm)
+
+    HADOOP-8552. Conflict: Same security.log.file for multiple users. 
+    (kkambatl via tucu)
+
+    HADOOP-8537. Fix TFile tests to pass even when native zlib support is not
+    compiled. (todd)
+
+    HADOOP-8626. Typo in default setting for
+    hadoop.security.group.mapping.ldap.search.filter.user. (Jonathan Natkins
+    via atm)
+
+    HADOOP-8480. The native build should honor -DskipTests.
+    (Colin Patrick McCabe via eli)
+
+    HADOOP-8659. Native libraries must build with soft-float ABI for Oracle JVM
+    on ARM. (Trevor Robinson via todd)
+
+    HADOOP-8654. TextInputFormat delimiter bug (Gelesh and Jason Lowe via
+    bobby)
+
+    HADOOP-8614. IOUtils#skipFully hangs forever on EOF. 
+    (Colin Patrick McCabe via eli)
+
+    HADOOP-8720. TestLocalFileSystem should use test root subdirectory.
+    (Vlad Rozov via eli)
+
+    HADOOP-8721. ZKFC should not retry 45 times when attempting a graceful
+    fence during a failover. (Vinayakumar B via atm)
+
+    HADOOP-8632. Configuration leaking class-loaders (Costin Leau via bobby)
+
+    HADOOP-4572. Can not access user logs - Jetty is not configured by default 
+    to serve aliases/symlinks (ahmed via tucu)
+
+    HADOOP-8660. TestPseudoAuthenticator failing with NPE. (tucu)
+
+    HADOOP-8699. some common testcases create core-site.xml in test-classes
+    making other testcases to fail. (tucu)
+
+    HADOOP-8031. Configuration class fails to find embedded .jar resources; 
+    should use URL.openStream() (genman via tucu)
+
+    HADOOP-8737. cmake: always use JAVA_HOME to find libjvm.so, jni.h, jni_md.h.
+    (Colin Patrick McCabe via eli)
+
+    HADOOP-8747. Syntax error on cmake version 2.6 patch 2 in JNIFlags.cmake. (cmccabe via tucu)
+
+    HADOOP-8722. Update BUILDING.txt with latest snappy info.
+    (Colin Patrick McCabe via eli)
+
+    HADOOP-8764. CMake: HADOOP-8737 broke ARM build. (Trevor Robinson via eli)
+
+    HADOOP-8770. NN should not RPC to self to find trash defaults. (eli)
+
+    HADOOP-8648. libhadoop: native CRC32 validation crashes when
+    io.bytes.per.checksum=1. (Colin Patrick McCabe via eli)
+
+    HADOOP-8766. FileContextMainOperationsBaseTest should randomize the root
+    dir. (Colin Patrick McCabe via atm)
+
+    HADOOP-8749. HADOOP-8031 changed the way in which relative xincludes are handled in 
+    Configuration. (ahmed via tucu)
+
+    HADOOP-8431. Running distcp wo args throws IllegalArgumentException.
+    (Sandy Ryza via eli)
+
+    HADOOP-8775. MR2 distcp permits non-positive value to -bandwidth option
+    which causes job never to complete. (Sandy Ryza via atm)
+
+    HADOOP-8781. hadoop-config.sh should add JAVA_LIBRARY_PATH to LD_LIBRARY_PATH. (tucu)
+
+    HADOOP-8855. SSL-based image transfer does not work when Kerberos is disabled. (todd via eli)
+
   BREAKDOWN OF HDFS-3042 SUBTASKS
 
     HADOOP-8220. ZKFailoverController doesn't handle failure to become active
@@ -730,7 +1066,10 @@ Release 2.0.0-alpha - 05-23-2012
     HADOOP-7868. Hadoop native fails to compile when default linker
     option is -Wl,--as-needed. (Trevor Robinson via eli)
 
-Release 0.23.3 - UNRELEASED
+    HADOOP-8655. Fix TextInputFormat for large deliminators. (Gelesh via
+    bobby) 
+
+Release 0.23.5 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
@@ -738,6 +1077,46 @@ Release 0.23.3 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-8923. JNI-based user-group mapping modules can be too chatty on 
+    lookup failures. (Kihwal Lee via suresh)
+
+    HADOOP-8930. Cumulative code coverage calculation (Andrey Klochkov via
+    bobby)
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HADOOP-8906. paths with multiple globs are unreliable. (Daryn Sharp via
+    jlowe)
+
+Release 0.23.4 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+    HADOOP-8822. relnotes.py was deleted post mavenization (bobby)
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HADOOP-8843. Old trash directories are never deleted on upgrade
+    from 1.x (jlowe)
+
+Release 0.23.3
+
+  INCOMPATIBLE CHANGES
+
+    HADOOP-7967. Need generalized multi-token filesystem support (daryn)
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
     HADOOP-8108. Move method getHostPortString() from NameNode to NetUtils.
     (Brandon Li via jitendra)
 
@@ -805,6 +1184,71 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8573. Configuration tries to read from an inputstream resource 
     multiple times (Robert Evans via tgraves)
 
+    HADOOP-8599. Non empty response from FileSystem.getFileBlockLocations when
+    asking for data beyond the end of file. (Andrey Klochkov via todd)
+
+    HADOOP-8606. FileSystem.get may return the wrong filesystem (Daryn Sharp
+    via bobby)
+
+    HADOOP-8551. fs -mkdir creates parent directories without the -p option
+    (John George via bobby)
+
+    HADOOP-8613. AbstractDelegationTokenIdentifier#getUser() should set token
+    auth type. (daryn)
+
+    HADOOP-8627. FS deleteOnExit may delete the wrong path (daryn via bobby)
+
+    HADOOP-8634. Ensure FileSystem#close doesn't squawk for deleteOnExit paths 
+    (daryn via bobby)
+
+    HADOOP-8550. hadoop fs -touchz automatically created parent directories
+    (John George via bobby)
+
+    HADOOP-8635. Cannot cancel paths registered deleteOnExit (daryn via bobby)
+
+    HADOOP-8637. FilterFileSystem#setWriteChecksum is broken (daryn via bobby)
+
+    HADOOP-8370. Native build failure: javah: class file for 
+    org.apache.hadoop.classification.InterfaceAudience not found  (Trevor
+    Robinson via tgraves)
+
+    HADOOP-8633. Interrupted FsShell copies may leave tmp files (Daryn Sharp
+    via tgraves)
+
+    HADOOP-8703. distcpV2: turn CRC checking off for 0 byte size (Dave
+    Thompson via bobby)
+
+    HADOOP-8390. TestFileSystemCanonicalization fails with JDK7  (Trevor
+    Robinson via tgraves)
+
+    HADOOP-8692. TestLocalDirAllocator fails intermittently with JDK7 
+    (Trevor Robinson via tgraves)
+
+    HADOOP-8693. TestSecurityUtil fails intermittently with JDK7 (Trevor
+    Robinson via tgraves)
+
+    HADOOP-8697. TestWritableName fails intermittently with JDK7 (Trevor
+    Robinson via tgraves)
+
+    HADOOP-8695. TestPathData fails intermittently with JDK7 (Trevor
+    Robinson via tgraves)
+
+    HADOOP-8611. Allow fall-back to the shell-based implementation when 
+    JNI-based users-group mapping fails (Robert Parker via bobby) 
+
+    HADOOP-8225. DistCp fails when invoked by Oozie (daryn via bobby)
+
+    HADOOP-8709. globStatus changed behavior from 0.20/1.x (Jason Lowe via
+    bobby)
+
+    HADOOP-8725. MR is broken when security is off (daryn via bobby)
+
+    HADOOP-8726. The Secrets in Credentials are not available to MR tasks
+    (daryn and Benoy Antony via bobby)
+
+    HADOOP-8727. Gracefully deprecate dfs.umaskmode in 2.x onwards (Harsh J
+    via bobby)
+
 Release 0.23.2 - UNRELEASED 
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-3077/hadoop-common-project/hadoop-common/CHANGES.txt:r1363593-1396941
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:r1360400-1399945

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml Fri Oct 19 02:25:55 2012
@@ -176,18 +176,6 @@
      </Match>
 
      <Match>
-       <Class name="org.apache.hadoop.fs.kfs.KFSOutputStream" />
-       <Field name="path" />
-       <Bug pattern="URF_UNREAD_FIELD" />
-     </Match>
-
-     <Match>
-       <Class name="org.apache.hadoop.fs.kfs.KosmosFileSystem" />
-       <Method name="initialize" />
-       <Bug pattern="DM_EXIT" />
-     </Match>
-
-     <Match>
        <Class name="org.apache.hadoop.io.Closeable" />
        <Bug pattern="NM_SAME_SIMPLE_NAME_AS_INTERFACE" />
      </Match>
@@ -295,4 +283,13 @@
       <Class name="~org\.apache\.hadoop\.ha\.proto\.ZKFCProtocolProtos.*"/>
     </Match>
 
+    <!--
+       Manually checked, misses child thread manually syncing on parent's intrinsic lock.
+    -->
+     <Match>
+       <Class name="org.apache.hadoop.metrics2.lib.MutableQuantiles" />
+       <Field name="previousSnapshot" />
+       <Bug pattern="IS2_INCONSISTENT_SYNC" />
+     </Match>
+
  </FindBugsFilter>

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/pom.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/pom.xml?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/pom.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/pom.xml Fri Oct 19 02:25:55 2012
@@ -31,9 +31,6 @@
   <packaging>jar</packaging>
 
   <properties>
-    <snappy.prefix>/usr/local</snappy.prefix>
-    <snappy.lib>${snappy.prefix}/lib</snappy.lib>
-    <bundle.snappy>false</bundle.snappy>
     <kdc.resource.dir>src/test/resources/kdc</kdc.resource.dir>
     <hadoop.component>common</hadoop.component>
     <is.hadoop.component>true</is.hadoop.component>
@@ -44,7 +41,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>
-      <scope>provided</scope>
+      <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>
@@ -77,13 +74,13 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>commons-net</groupId>
-      <artifactId>commons-net</artifactId>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
+      <groupId>commons-net</groupId>
+      <artifactId>commons-net</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
@@ -102,16 +99,12 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>asm</groupId>
-      <artifactId>asm</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>com.sun.jersey</groupId>
       <artifactId>jersey-core</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <!-- Used, even though 'mvn dependency:analyze' doesn't find it -->
       <groupId>com.sun.jersey</groupId>
       <artifactId>jersey-json</artifactId>
       <scope>compile</scope>
@@ -124,22 +117,22 @@
     <dependency>
       <groupId>tomcat</groupId>
       <artifactId>jasper-compiler</artifactId>
-      <scope>compile</scope>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>tomcat</groupId>
       <artifactId>jasper-runtime</artifactId>
-      <scope>compile</scope>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>javax.servlet.jsp</groupId>
       <artifactId>jsp-api</artifactId>
-      <scope>compile</scope>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>commons-el</groupId>
       <artifactId>commons-el</artifactId>
-      <scope>compile</scope>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>commons-logging</groupId>
@@ -147,11 +140,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging-api</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
       <scope>compile</scope>
@@ -162,26 +150,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.mina</groupId>
-      <artifactId>mina-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.ftpserver</groupId>
-      <artifactId>ftplet-api</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.ftpserver</groupId>
-      <artifactId>ftpserver-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.ftpserver</groupId>
-      <artifactId>ftpserver-deprecated</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -192,11 +160,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>commons-collections</groupId>
-      <artifactId>commons-collections</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>commons-configuration</groupId>
       <artifactId>commons-configuration</artifactId>
       <scope>compile</scope>
@@ -209,16 +172,11 @@
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jdt</groupId>
-      <artifactId>core</artifactId>
-      <scope>compile</scope>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>oro</groupId>
-      <artifactId>oro</artifactId>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
@@ -227,11 +185,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.aspectj</groupId>
-      <artifactId>aspectjrt</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
@@ -242,11 +195,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>net.sf.kosmosfs</groupId>
-      <artifactId>kfs</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.ant</groupId>
       <artifactId>ant</artifactId>
       <scope>provided</scope>
@@ -262,11 +210,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>com.googlecode.json-simple</groupId>
-      <artifactId>json-simple</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>com.jcraft</groupId>
       <artifactId>jsch</artifactId>
     </dependency>
@@ -333,72 +276,15 @@
         <artifactId>maven-antrun-plugin</artifactId>
         <executions>
           <execution>
-            <id>compile-proto</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <target>
-                <echo file="target/compile-proto.sh">
-                    PROTO_DIR=src/main/proto
-                    JAVA_DIR=target/generated-sources/java
-                    which cygpath 2&gt; /dev/null
-                    if [ $? = 1 ]; then
-                      IS_WIN=false
-                    else
-                      IS_WIN=true
-                      WIN_PROTO_DIR=`cygpath --windows $PROTO_DIR`
-                      WIN_JAVA_DIR=`cygpath --windows $JAVA_DIR`
-                    fi
-                    mkdir -p $JAVA_DIR 2&gt; /dev/null
-                    for PROTO_FILE in `ls $PROTO_DIR/*.proto 2&gt; /dev/null`
-                    do
-                        if [ "$IS_WIN" = "true" ]; then
-                          protoc -I$WIN_PROTO_DIR --java_out=$WIN_JAVA_DIR $PROTO_FILE
-                        else
-                          protoc -I$PROTO_DIR --java_out=$JAVA_DIR $PROTO_FILE
-                        fi
-                    done
-                </echo>
-                <exec executable="sh" dir="${basedir}" failonerror="true">
-                  <arg line="target/compile-proto.sh"/>
-                </exec>
-              </target>
-            </configuration>
-          </execution>
-          <execution>
-            <id>compile-test-proto</id>
-            <phase>generate-test-sources</phase>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
             <goals>
               <goal>run</goal>
             </goals>
             <configuration>
               <target>
-                <echo file="target/compile-test-proto.sh">
-                    PROTO_DIR=src/test/proto
-                    JAVA_DIR=target/generated-test-sources/java
-                    which cygpath 2&gt; /dev/null
-                    if [ $? = 1 ]; then
-                      IS_WIN=false
-                    else
-                      IS_WIN=true
-                      WIN_PROTO_DIR=`cygpath --windows $PROTO_DIR`
-                      WIN_JAVA_DIR=`cygpath --windows $JAVA_DIR`
-                    fi
-                    mkdir -p $JAVA_DIR 2&gt; /dev/null
-                    for PROTO_FILE in `ls $PROTO_DIR/*.proto 2&gt; /dev/null`
-                    do
-                        if [ "$IS_WIN" = "true" ]; then
-                          protoc -I$WIN_PROTO_DIR --java_out=$WIN_JAVA_DIR $PROTO_FILE
-                        else
-                          protoc -I$PROTO_DIR --java_out=$JAVA_DIR $PROTO_FILE
-                        fi
-                    done
-                </echo>
-                <exec executable="sh" dir="${basedir}" failonerror="true">
-                  <arg line="target/compile-test-proto.sh"/>
-                </exec>
+                <mkdir dir="target/generated-sources/java" />
+                <mkdir dir="target/generated-test-sources/java" />
               </target>
             </configuration>
           </execution>
@@ -476,6 +362,48 @@
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>compile-proto</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--java_out=target/generated-sources/java</argument>
+                <argument>src/main/proto/HAServiceProtocol.proto</argument>
+                <argument>src/main/proto/IpcConnectionContext.proto</argument>
+                <argument>src/main/proto/ProtocolInfo.proto</argument>
+                <argument>src/main/proto/RpcPayloadHeader.proto</argument>
+                <argument>src/main/proto/ZKFCProtocol.proto</argument>
+                <argument>src/main/proto/hadoop_rpc.proto</argument>
+              </arguments>
+            </configuration>
+          </execution>
+          <execution>
+            <id>compile-test-proto</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/test/proto/</argument>
+                <argument>--java_out=target/generated-test-sources/java</argument>
+                <argument>src/test/proto/test.proto</argument>
+                <argument>src/test/proto/test_rpc_service.proto</argument>
+              </arguments>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
         <artifactId>build-helper-maven-plugin</artifactId>
         <executions>
           <execution>
@@ -533,10 +461,10 @@
         <activeByDefault>false</activeByDefault>
       </activation>
       <properties>
-        <snappy.prefix>/usr/local</snappy.prefix>
-        <snappy.lib>${snappy.prefix}/lib</snappy.lib>
-        <snappy.include>${snappy.prefix}/include</snappy.include>
-        <runas.home></runas.home>
+        <snappy.prefix></snappy.prefix>
+        <snappy.lib></snappy.lib>
+        <snappy.include></snappy.include>
+        <require.snappy>false</require.snappy>
       </properties>
       <build>
         <plugins>
@@ -568,25 +496,34 @@
               </execution>
             </executions>
           </plugin>
+
           <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.apache.hadoop.cmake.maven.ng</groupId>
+            <artifactId>cmake-ng</artifactId>
             <executions>
               <execution>
-                <id>make</id>
-                <phase>compile</phase>
-                <goals><goal>run</goal></goals>
+                <id>cmake-compile</id>
+                <goals><goal>compile</goal></goals>
                 <configuration>
-                  <target>
-                    <exec executable="cmake" dir="${project.build.directory}/native" failonerror="true">
-                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model}"/>
-                      <env key="CFLAGS" value="-I${snappy.include}"/>
-                      <env key="LDFLAGS" value="-L${snappy.lib}"/>
-                    </exec>
-                    <exec executable="make" dir="${project.build.directory}/native" failonerror="true">
-                      <arg line="VERBOSE=1"/>
-                    </exec>
-                  </target>
+                  <target>all</target>
+                  <source>${basedir}/src</source>
+                  <vars>
+                    <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
+                    <JVM_ARCH_DATA_MODEL>${sun.arch.data.model}</JVM_ARCH_DATA_MODEL>
+                    <REQUIRE_SNAPPY>${require.snappy}</REQUIRE_SNAPPY>
+                    <CUSTOM_SNAPPY_PREFIX>${snappy.prefix}</CUSTOM_SNAPPY_PREFIX>
+                    <CUSTOM_SNAPPY_LIB>${snappy.lib}</CUSTOM_SNAPPY_LIB>
+                    <CUSTOM_SNAPPY_INCLUDE>${snappy.include}</CUSTOM_SNAPPY_INCLUDE>
+                  </vars>
+                </configuration>
+              </execution>
+              <execution>
+                <id>test_bulk_crc32</id>
+                <goals><goal>test</goal></goals>
+                <configuration>
+                  <binary>${project.build.directory}/native/test_bulk_crc32</binary>
+                  <timeout>300</timeout>
+                  <results>${project.build.directory}/results</results>
                 </configuration>
               </execution>
             </executions>

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/CMakeLists.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/CMakeLists.txt?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/CMakeLists.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/CMakeLists.txt Fri Oct 19 02:25:55 2012
@@ -21,18 +21,7 @@ cmake_minimum_required(VERSION 2.6 FATAL
 # Default to release builds
 set(CMAKE_BUILD_TYPE, Release)
 
-# If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
-# This variable is set by maven.
-if (JVM_ARCH_DATA_MODEL EQUAL 32)
-    # force 32-bit code generation on amd64/x86_64, ppc64, sparc64
-    if (CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
-        set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -m32")
-        set(CMAKE_LD_FLAGS "${CMAKE_LD_FLAGS} -m32")
-    endif ()
-    if (CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-        set(CMAKE_SYSTEM_PROCESSOR "i686")
-    endif ()
-endif (JVM_ARCH_DATA_MODEL EQUAL 32)
+include(JNIFlags.cmake NO_POLICY_SCOPE)
 
 # Compile a library with both shared and static variants
 function(add_dual_library LIBNAME)
@@ -61,16 +50,46 @@ function(dual_output_directory TGT DIR)
     output_directory(${TGT}_static "${DIR}")
 endfunction(dual_output_directory TGT DIR)
 
+#
+# This macro alters the behavior of find_package and find_library.
+# It does this by setting the CMAKE_FIND_LIBRARY_SUFFIXES global variable. 
+# You should save that variable before calling this function and restore it
+# after you have accomplished your goal.
+#
+# The behavior is altered in two ways:
+# 1. We always find shared libraries, never static;
+# 2. We find shared libraries with the given version number.
+#
+# On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+#
+macro(set_find_shared_library_version LVERS)
+    IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
+    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # Windows doesn't support finding shared libraries by version.
+    ELSE()
+        # Most UNIX variants use .so
+        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
+    ENDIF()
+endmacro(set_find_shared_library_version LVERS)
+
 if (NOT GENERATED_JAVAH)
     # Must identify where the generated headers have been placed
     MESSAGE(FATAL_ERROR "You must set the cmake variable GENERATED_JAVAH")
 endif (NOT GENERATED_JAVAH)
 find_package(JNI REQUIRED)
+
+SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
+set_find_shared_library_version("1")
 find_package(ZLIB REQUIRED)
+SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
 
 set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
 set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
 set(D main/native/src/org/apache/hadoop)
+set(T main/native/src/test/org/apache/hadoop)
 
 GET_FILENAME_COMPONENT(HADOOP_ZLIB_LIBRARY ${ZLIB_LIBRARIES} NAME)
 
@@ -79,17 +98,29 @@ INCLUDE(CheckCSourceCompiles)
 CHECK_FUNCTION_EXISTS(sync_file_range HAVE_SYNC_FILE_RANGE)
 CHECK_FUNCTION_EXISTS(posix_fadvise HAVE_POSIX_FADVISE)
 
-find_library(SNAPPY_LIBRARY NAMES snappy PATHS)
-find_path(SNAPPY_INCLUDE_DIR NAMES snappy.h PATHS)
-if (SNAPPY_LIBRARY)
+SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
+set_find_shared_library_version("1")
+find_library(SNAPPY_LIBRARY 
+    NAMES snappy
+    PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/lib
+          ${CUSTOM_SNAPPY_PREFIX}/lib64 ${CUSTOM_SNAPPY_LIB})
+SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
+find_path(SNAPPY_INCLUDE_DIR 
+    NAMES snappy.h
+    PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/include
+          ${CUSTOM_SNAPPY_INCLUDE})
+if (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
     GET_FILENAME_COMPONENT(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
     set(SNAPPY_SOURCE_FILES
         "${D}/io/compress/snappy/SnappyCompressor.c"
         "${D}/io/compress/snappy/SnappyDecompressor.c")
-else (${SNAPPY_LIBRARY})
+else (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
     set(SNAPPY_INCLUDE_DIR "")
     set(SNAPPY_SOURCE_FILES "")
-endif (SNAPPY_LIBRARY)
+    IF(REQUIRE_SNAPPY)
+        MESSAGE(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
+    ENDIF(REQUIRE_SNAPPY)
+endif (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
 
 include_directories(
     ${GENERATED_JAVAH}
@@ -100,9 +131,17 @@ include_directories(
     ${JNI_INCLUDE_DIRS}
     ${ZLIB_INCLUDE_DIRS}
     ${SNAPPY_INCLUDE_DIR}
+    ${D}/util
 )
 CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
+add_executable(test_bulk_crc32
+    ${D}/util/bulk_crc32.c
+    ${T}/util/test_bulk_crc32.c
+)
+set_property(SOURCE main.cpp PROPERTY INCLUDE_DIRECTORIES "\"-Werror\" \"-Wall\"")
+
+SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
 add_dual_library(hadoop
     ${D}/io/compress/lz4/Lz4Compressor.c
     ${D}/io/compress/lz4/Lz4Decompressor.c
@@ -116,9 +155,21 @@ add_dual_library(hadoop
     ${D}/security/JniBasedUnixGroupsMapping.c
     ${D}/security/JniBasedUnixGroupsNetgroupMapping.c
     ${D}/security/getGroup.c
+    ${D}/util/NativeCodeLoader.c
     ${D}/util/NativeCrc32.c
     ${D}/util/bulk_crc32.c
 )
+
+IF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
+    #
+    # By embedding '$ORIGIN' into the RPATH of libhadoop.so,
+    # dlopen will look in the directory containing libhadoop.so.
+    # However, $ORIGIN is not supported by all operating systems.
+    #
+    SET_TARGET_PROPERTIES(hadoop 
+        PROPERTIES INSTALL_RPATH "\$ORIGIN/")
+ENDIF()
+
 target_link_dual_libraries(hadoop
     dl
     ${JAVA_JVM_LIBRARY}

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/config.h.cmake
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/config.h.cmake?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/config.h.cmake (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/config.h.cmake Fri Oct 19 02:25:55 2012
@@ -2,7 +2,6 @@
 #define CONFIG_H
 
 #cmakedefine HADOOP_ZLIB_LIBRARY "@HADOOP_ZLIB_LIBRARY@"
-#cmakedefine HADOOP_RUNAS_HOME "@HADOOP_RUNAS_HOME@"
 #cmakedefine HADOOP_SNAPPY_LIBRARY "@HADOOP_SNAPPY_LIBRARY@"
 #cmakedefine HAVE_SYNC_FILE_RANGE
 #cmakedefine HAVE_POSIX_FADVISE

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/contrib/bash-tab-completion/hadoop.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/contrib/bash-tab-completion/hadoop.sh?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/contrib/bash-tab-completion/hadoop.sh (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/contrib/bash-tab-completion/hadoop.sh Fri Oct 19 02:25:55 2012
@@ -26,7 +26,7 @@ _hadoop() {
   COMPREPLY=()
   cur=${COMP_WORDS[COMP_CWORD]}
   prev=${COMP_WORDS[COMP_CWORD-1]}  
-  script=${COMP_WORDS[0]}  
+  script=`which ${COMP_WORDS[0]}`
   
   # Bash lets you tab complete things even if the script doesn't
   # exist (or isn't executable). Check to make sure it is, as we

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop Fri Oct 19 02:25:55 2012
@@ -19,7 +19,7 @@
 
 bin=`which $0`
 bin=`dirname ${bin}`
-bin=`cd "$bin"; pwd`
+bin=`cd "$bin" > /dev/null; pwd`
  
 DEFAULT_LIBEXEC_DIR="$bin"/../libexec
 HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
@@ -96,33 +96,30 @@ case $COMMAND in
     # the core commands
     if [ "$COMMAND" = "fs" ] ; then
       CLASS=org.apache.hadoop.fs.FsShell
-      HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
     elif [ "$COMMAND" = "version" ] ; then
       CLASS=org.apache.hadoop.util.VersionInfo
-      HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
     elif [ "$COMMAND" = "jar" ] ; then
       CLASS=org.apache.hadoop.util.RunJar
     elif [ "$COMMAND" = "distcp" ] ; then
       CLASS=org.apache.hadoop.tools.DistCp
       CLASSPATH=${CLASSPATH}:${TOOL_PATH}
-      HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
     elif [ "$COMMAND" = "daemonlog" ] ; then
       CLASS=org.apache.hadoop.log.LogLevel
-      HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
     elif [ "$COMMAND" = "archive" ] ; then
       CLASS=org.apache.hadoop.tools.HadoopArchives
       CLASSPATH=${CLASSPATH}:${TOOL_PATH}
-      HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
     elif [[ "$COMMAND" = -*  ]] ; then
         # class and package names cannot begin with a -
         echo "Error: No command named \`$COMMAND' was found. Perhaps you meant \`hadoop ${COMMAND#-}'"
         exit 1
     else
-      HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
       CLASS=$COMMAND
     fi
     shift
     
+    # Always respect HADOOP_OPTS and HADOOP_CLIENT_OPTS
+    HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
+
     #make sure security appender is turned off
     HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}"
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh Fri Oct 19 02:25:55 2012
@@ -42,8 +42,8 @@ HADOOP_COMMON_LIB_JARS_DIR=${HADOOP_COMM
 HADOOP_COMMON_LIB_NATIVE_DIR=${HADOOP_COMMON_LIB_NATIVE_DIR:-"lib/native"}
 HDFS_DIR=${HDFS_DIR:-"share/hadoop/hdfs"}
 HDFS_LIB_JARS_DIR=${HDFS_LIB_JARS_DIR:-"share/hadoop/hdfs/lib"}
-YARN_DIR=${YARN_DIR:-"share/hadoop/mapreduce"}
-YARN_LIB_JARS_DIR=${YARN_LIB_JARS_DIR:-"share/hadoop/mapreduce/lib"}
+YARN_DIR=${YARN_DIR:-"share/hadoop/yarn"}
+YARN_LIB_JARS_DIR=${YARN_LIB_JARS_DIR:-"share/hadoop/yarn/lib"}
 MAPRED_DIR=${MAPRED_DIR:-"share/hadoop/mapreduce"}
 MAPRED_LIB_JARS_DIR=${MAPRED_LIB_JARS_DIR:-"share/hadoop/mapreduce/lib"}
 
@@ -74,6 +74,10 @@ fi
 
 export HADOOP_CONF_DIR="${HADOOP_CONF_DIR:-$HADOOP_PREFIX/$DEFAULT_CONF_DIR}"
 
+if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
+  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
+fi
+
 # User can specify hostnames or a file where the hostnames are (not both)
 if [[ ( "$HADOOP_SLAVES" != '' ) && ( "$HADOOP_SLAVE_NAMES" != '' ) ]] ; then
   echo \
@@ -113,9 +117,6 @@ case "`uname`" in
 CYGWIN*) cygwin=true;;
 esac
 
-if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
-  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
-fi
 
 # check if net.ipv6.bindv6only is set to 1
 bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null)
@@ -243,6 +244,7 @@ HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.id.st
 HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.root.logger=${HADOOP_ROOT_LOGGER:-INFO,console}"
 if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
   HADOOP_OPTS="$HADOOP_OPTS -Djava.library.path=$JAVA_LIBRARY_PATH"
+  export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$JAVA_LIBRARY_PATH
 fi  
 HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.policy.file=$HADOOP_POLICYFILE"
 
@@ -267,21 +269,21 @@ fi
 CLASSPATH=${CLASSPATH}:$HADOOP_HDFS_HOME/$HDFS_DIR'/*'
 
 # put yarn in classpath if present
-if [ "$YARN_HOME" = "" ]; then
+if [ "$HADOOP_YARN_HOME" = "" ]; then
   if [ -d "${HADOOP_PREFIX}/$YARN_DIR" ]; then
-    export YARN_HOME=$HADOOP_PREFIX
+    export HADOOP_YARN_HOME=$HADOOP_PREFIX
   fi
 fi
 
-if [ -d "$YARN_HOME/$YARN_DIR/webapps" ]; then
-  CLASSPATH=${CLASSPATH}:$YARN_HOME/$YARN_DIR
+if [ -d "$HADOOP_YARN_HOME/$YARN_DIR/webapps" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/$YARN_DIR
 fi
 
-if [ -d "$YARN_HOME/$YARN_LIB_JARS_DIR" ]; then
-  CLASSPATH=${CLASSPATH}:$YARN_HOME/$YARN_LIB_JARS_DIR'/*'
+if [ -d "$HADOOP_YARN_HOME/$YARN_LIB_JARS_DIR" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/$YARN_LIB_JARS_DIR'/*'
 fi
 
-CLASSPATH=${CLASSPATH}:$YARN_HOME/$YARN_DIR'/*'
+CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/$YARN_DIR'/*'
 
 # put mapred in classpath if present AND different from YARN
 if [ "$HADOOP_MAPRED_HOME" = "" ]; then
@@ -290,7 +292,7 @@ if [ "$HADOOP_MAPRED_HOME" = "" ]; then
   fi
 fi
 
-if [ "$HADOOP_MAPRED_HOME/$MAPRED_DIR" != "$YARN_HOME/$YARN_DIR" ] ; then
+if [ "$HADOOP_MAPRED_HOME/$MAPRED_DIR" != "$HADOOP_YARN_HOME/$YARN_DIR" ] ; then
   if [ -d "$HADOOP_MAPRED_HOME/$MAPRED_DIR/webapps" ]; then
     CLASSPATH=${CLASSPATH}:$HADOOP_MAPRED_HOME/$MAPRED_DIR
   fi

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh Fri Oct 19 02:25:55 2012
@@ -141,7 +141,7 @@ case $startStop in
     echo starting $command, logging to $log
     cd "$HADOOP_PREFIX"
     case $command in
-      namenode|secondarynamenode|datanode|dfs|dfsadmin|fsck|balancer|zkfc)
+      namenode|secondarynamenode|datanode|journalnode|dfs|dfsadmin|fsck|balancer|zkfc)
         if [ -z "$HADOOP_HDFS_HOME" ]; then
           hdfsScript="$HADOOP_PREFIX"/bin/hdfs
         else

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/slaves.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/slaves.sh?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/slaves.sh (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/slaves.sh Fri Oct 19 02:25:55 2012
@@ -42,9 +42,6 @@ DEFAULT_LIBEXEC_DIR="$bin"/../libexec
 HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
 . $HADOOP_LIBEXEC_DIR/hadoop-config.sh
 
-if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
-  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
-fi
 
 # Where to start the script, see hadoop-config.sh
 # (it set up the variables based on command line options)

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/start-all.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/start-all.sh?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/start-all.sh (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/bin/start-all.sh Fri Oct 19 02:25:55 2012
@@ -33,6 +33,6 @@ if [ -f "${HADOOP_HDFS_HOME}"/sbin/start
 fi
 
 # start yarn daemons if yarn is present
-if [ -f "${YARN_HOME}"/sbin/start-yarn.sh ]; then
-  "${YARN_HOME}"/sbin/start-yarn.sh --config $HADOOP_CONF_DIR
+if [ -f "${HADOOP_YARN_HOME}"/sbin/start-yarn.sh ]; then
+  "${HADOOP_YARN_HOME}"/sbin/start-yarn.sh --config $HADOOP_CONF_DIR
 fi

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh Fri Oct 19 02:25:55 2012
@@ -47,6 +47,14 @@ done
 # Extra Java runtime options.  Empty by default.
 export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true $HADOOP_CLIENT_OPTS"
 
+MAC_OSX=false
+case "`uname`" in
+Darwin*) MAC_OSX=true;;
+esac
+if $MAC_OSX; then
+    export HADOOP_OPTS="$HADOOP_OPTS -Djava.security.krb5.realm= -Djava.security.krb5.kdc="
+fi
+
 # Command specific options appended to HADOOP_OPTS when specified
 export HADOOP_NAMENODE_OPTS="-Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,RFAS} -Dhdfs.audit.logger=${HDFS_AUDIT_LOGGER:-INFO,NullAppender} $HADOOP_NAMENODE_OPTS"
 export HADOOP_DATANODE_OPTS="-Dhadoop.security.logger=ERROR,RFAS $HADOOP_DATANODE_OPTS"

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-metrics2.properties
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-metrics2.properties?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-metrics2.properties (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/hadoop-metrics2.properties Fri Oct 19 02:25:55 2012
@@ -2,7 +2,7 @@
 # See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
 
 *.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
-# default sampling period
+# default sampling period, in seconds
 *.period=10
 
 # The namenode-metrics.out will contain metrics from all context

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties Fri Oct 19 02:25:55 2012
@@ -106,7 +106,7 @@ hadoop.security.logger=INFO,NullAppender
 hadoop.security.log.maxfilesize=256MB
 hadoop.security.log.maxbackupindex=20
 log4j.category.SecurityLogger=${hadoop.security.logger}
-hadoop.security.log.file=SecurityAuth.audit
+hadoop.security.log.file=SecurityAuth-${user.name}.audit
 log4j.appender.RFAS=org.apache.log4j.RollingFileAppender 
 log4j.appender.RFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
 log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-client.xml.example
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-client.xml.example?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-client.xml.example (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-client.xml.example Fri Oct 19 02:25:55 2012
@@ -1,6 +1,21 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
+<!--
+   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.
+-->
 <configuration>
 
 <property>
@@ -21,7 +36,15 @@
 <property>
   <name>ssl.client.truststore.type</name>
   <value>jks</value>
-  <description>Optional. Default value is "jks".
+  <description>Optional. The keystore file format, default value is "jks".
+  </description>
+</property>
+
+<property>
+  <name>ssl.client.truststore.reload.interval</name>
+  <value>10000</value>
+  <description>Truststore reload check interval, in milliseconds.
+  Default value is 10000 (10 seconds).
   </description>
 </property>
 
@@ -50,7 +73,7 @@
 <property>
   <name>ssl.client.keystore.type</name>
   <value>jks</value>
-  <description>Optional. Default value is "jks".
+  <description>Optional. The keystore file format, default value is "jks".
   </description>
 </property>
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-server.xml.example
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-server.xml.example?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-server.xml.example (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/conf/ssl-server.xml.example Fri Oct 19 02:25:55 2012
@@ -1,6 +1,21 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
+<!--
+   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.
+-->
 <configuration>
 
 <property>
@@ -20,11 +35,18 @@
 <property>
   <name>ssl.server.truststore.type</name>
   <value>jks</value>
-  <description>Optional. Default value is "jks".
+  <description>Optional. The keystore file format, default value is "jks".
   </description>
 </property>
 
 <property>
+  <name>ssl.server.truststore.reload.interval</name>
+  <value>10000</value>
+  <description>Truststore reload check interval, in milliseconds.
+  Default value is 10000 (10 seconds).
+</property>
+
+<property>
   <name>ssl.server.keystore.location</name>
   <value></value>
   <description>Keystore to be used by NN and DN. Must be specified.
@@ -48,7 +70,7 @@
 <property>
   <name>ssl.server.keystore.type</name>
   <value>jks</value>
-  <description>Optional. Default value is "jks".
+  <description>Optional. The keystore file format, default value is "jks".
   </description>
 </property>
 

Propchange: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/docs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/docs:r1360400-1399945
  Merged /hadoop/common/branches/HDFS-3077/hadoop-common-project/hadoop-common/src/main/docs:r1363593-1396941