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 ss...@apache.org on 2012/10/16 02:03:31 UTC

svn commit: r1398581 [11/14] - in /hadoop/common/branches/MR-3902/hadoop-common-project: hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/ hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/ hadoop-auth/sr...

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java Tue Oct 16 00:02:55 2012
@@ -63,7 +63,7 @@ public class HostsFileReader {
               // Everything from now on is a comment
               break;
             }
-            if (!nodes[i].equals("")) {
+            if (!nodes[i].isEmpty()) {
               LOG.info("Adding " + nodes[i] + " to the list of hosts from " + filename);
               set.add(nodes[i]);  // might need to add canonical name
             }
@@ -80,13 +80,13 @@ public class HostsFileReader {
 
   public synchronized void refresh() throws IOException {
     LOG.info("Refreshing hosts (include/exclude) list");
-    if (!includesFile.equals("")) {
+    if (!includesFile.isEmpty()) {
       Set<String> newIncludes = new HashSet<String>();
       readFileToSet(includesFile, newIncludes);
       // switch the new hosts that are to be included
       includes = newIncludes;
     }
-    if (!excludesFile.equals("")) {
+    if (!excludesFile.isEmpty()) {
       Set<String> newExcludes = new HashSet<String>();
       readFileToSet(excludesFile, newExcludes);
       // switch the excluded hosts

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java Tue Oct 16 00:02:55 2012
@@ -204,11 +204,13 @@ public class LineReader {
       int startPosn = bufferPosn; //starting from where we left off the last time
       if (bufferPosn >= bufferLength) {
         startPosn = bufferPosn = 0;
-        if (prevCharCR)
+        if (prevCharCR) {
           ++bytesConsumed; //account for CR from previous read
+        }
         bufferLength = in.read(buffer);
-        if (bufferLength <= 0)
+        if (bufferLength <= 0) {
           break; // EOF
+        }
       }
       for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
         if (buffer[bufferPosn] == LF) {
@@ -223,8 +225,9 @@ public class LineReader {
         prevCharCR = (buffer[bufferPosn] == CR);
       }
       int readLength = bufferPosn - startPosn;
-      if (prevCharCR && newlineLength == 0)
+      if (prevCharCR && newlineLength == 0) {
         --readLength; //CR at the end of the buffer
+      }
       bytesConsumed += readLength;
       int appendLength = readLength - newlineLength;
       if (appendLength > maxLineLength - txtLength) {
@@ -236,8 +239,9 @@ public class LineReader {
       }
     } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
 
-    if (bytesConsumed > (long)Integer.MAX_VALUE)
-      throw new IOException("Too many bytes before newline: " + bytesConsumed);    
+    if (bytesConsumed > (long)Integer.MAX_VALUE) {
+      throw new IOException("Too many bytes before newline: " + bytesConsumed);
+    }
     return (int)bytesConsumed;
   }
 
@@ -246,18 +250,56 @@ public class LineReader {
    */
   private int readCustomLine(Text str, int maxLineLength, int maxBytesToConsume)
       throws IOException {
+   /* We're reading data from inputStream, but the head of the stream may be
+    *  already captured in the previous buffer, so we have several cases:
+    * 
+    * 1. The buffer tail does not contain any character sequence which
+    *    matches with the head of delimiter. We count it as a 
+    *    ambiguous byte count = 0
+    *    
+    * 2. The buffer tail contains a X number of characters,
+    *    that forms a sequence, which matches with the
+    *    head of delimiter. We count ambiguous byte count = X
+    *    
+    *    // ***  eg: A segment of input file is as follows
+    *    
+    *    " record 1792: I found this bug very interesting and
+    *     I have completely read about it. record 1793: This bug
+    *     can be solved easily record 1794: This ." 
+    *    
+    *    delimiter = "record";
+    *        
+    *    supposing:- String at the end of buffer =
+    *    "I found this bug very interesting and I have completely re"
+    *    There for next buffer = "ad about it. record 179       ...."           
+    *     
+    *     The matching characters in the input
+    *     buffer tail and delimiter head = "re" 
+    *     Therefore, ambiguous byte count = 2 ****   //
+    *     
+    *     2.1 If the following bytes are the remaining characters of
+    *         the delimiter, then we have to capture only up to the starting 
+    *         position of delimiter. That means, we need not include the 
+    *         ambiguous characters in str.
+    *     
+    *     2.2 If the following bytes are not the remaining characters of
+    *         the delimiter ( as mentioned in the example ), 
+    *         then we have to include the ambiguous characters in str. 
+    */
     str.clear();
     int txtLength = 0; // tracks str.getLength(), as an optimization
     long bytesConsumed = 0;
     int delPosn = 0;
+    int ambiguousByteCount=0; // To capture the ambiguous characters count
     do {
-      int startPosn = bufferPosn; // starting from where we left off the last
-      // time
+      int startPosn = bufferPosn; // Start from previous end position
       if (bufferPosn >= bufferLength) {
         startPosn = bufferPosn = 0;
         bufferLength = in.read(buffer);
-        if (bufferLength <= 0)
+        if (bufferLength <= 0) {
+          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
           break; // EOF
+        }
       }
       for (; bufferPosn < bufferLength; ++bufferPosn) {
         if (buffer[bufferPosn] == recordDelimiterBytes[delPosn]) {
@@ -267,7 +309,7 @@ public class LineReader {
             break;
           }
         } else if (delPosn != 0) {
-          bufferPosn--; // recheck if bufferPosn matches start of delimiter
+          bufferPosn--;
           delPosn = 0;
         }
       }
@@ -278,14 +320,27 @@ public class LineReader {
         appendLength = maxLineLength - txtLength;
       }
       if (appendLength > 0) {
+        if (ambiguousByteCount > 0) {
+          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
+          //appending the ambiguous characters (refer case 2.2)
+          bytesConsumed += ambiguousByteCount;
+          ambiguousByteCount=0;
+        }
         str.append(buffer, startPosn, appendLength);
         txtLength += appendLength;
       }
-    } while (delPosn < recordDelimiterBytes.length
+      if (bufferPosn >= bufferLength) {
+        if (delPosn > 0 && delPosn < recordDelimiterBytes.length) {
+          ambiguousByteCount = delPosn;
+          bytesConsumed -= ambiguousByteCount; //to be consumed in next
+        }
+      }
+    } while (delPosn < recordDelimiterBytes.length 
         && bytesConsumed < maxBytesToConsume);
-    if (bytesConsumed > (long) Integer.MAX_VALUE)
+    if (bytesConsumed > (long) Integer.MAX_VALUE) {
       throw new IOException("Too many bytes before delimiter: " + bytesConsumed);
-    return (int) bytesConsumed;
+    }
+    return (int) bytesConsumed; 
   }
 
   /**
@@ -297,7 +352,7 @@ public class LineReader {
    */
   public int readLine(Text str, int maxLineLength) throws IOException {
     return readLine(str, maxLineLength, Integer.MAX_VALUE);
-}
+  }
 
   /**
    * Read from the InputStream into the given Text.
@@ -308,5 +363,4 @@ public class LineReader {
   public int readLine(Text str) throws IOException {
     return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
   }
-
 }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java Tue Oct 16 00:02:55 2012
@@ -47,7 +47,7 @@ public class NativeCodeLoader {
     }
     try {
       System.loadLibrary("hadoop");
-      LOG.info("Loaded the native-hadoop library");
+      LOG.debug("Loaded the native-hadoop library");
       nativeCodeLoaded = true;
     } catch (Throwable t) {
       // Ignore failure to load
@@ -75,6 +75,11 @@ public class NativeCodeLoader {
   }
 
   /**
+   * Returns true only if this build was compiled with support for snappy.
+   */
+  public static native boolean buildSupportsSnappy();
+
+  /**
    * Return if native hadoop libraries, if present, can be used for this job.
    * @param conf configuration
    * 

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java Tue Oct 16 00:02:55 2012
@@ -214,6 +214,7 @@ public class Progress {
     this.status = status;
   }
 
+  @Override
   public String toString() {
     StringBuilder result = new StringBuilder();
     toString(result);

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32.java Tue Oct 16 00:02:55 2012
@@ -46,17 +46,17 @@ public class PureJavaCrc32 implements Ch
     reset();
   }
 
-  /** {@inheritDoc} */
+  @Override
   public long getValue() {
     return (~crc) & 0xffffffffL;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void reset() {
     crc = 0xffffffff;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void update(byte[] b, int off, int len) {
     int localCrc = crc;
     while(len > 7) {
@@ -81,7 +81,7 @@ public class PureJavaCrc32 implements Ch
     crc = localCrc;
   }
 
-  /** {@inheritDoc} */
+  @Override
   final public void update(int b) {
     crc = (crc >>> 8) ^ T8_0[(crc ^ b) & 0xff];
   }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java Tue Oct 16 00:02:55 2012
@@ -42,18 +42,18 @@ public class PureJavaCrc32C implements C
     reset();
   }
 
-  /** {@inheritDoc} */
+  @Override
   public long getValue() {
     long ret = crc;
     return (~ret) & 0xffffffffL;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void reset() {
     crc = 0xffffffff;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void update(byte[] b, int off, int len) {
     int localCrc = crc;
     while(len > 7) {
@@ -78,7 +78,7 @@ public class PureJavaCrc32C implements C
     crc = localCrc;
   }
 
-  /** {@inheritDoc} */
+  @Override
   final public void update(int b) {
     crc = (crc >>> 8) ^ T8_0[(crc ^ b) & 0xff];
   }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java Tue Oct 16 00:02:55 2012
@@ -52,13 +52,12 @@ public final class QuickSort implements 
    * {@inheritDoc} If the recursion depth falls below {@link #getMaxDepth},
    * then switch to {@link HeapSort}.
    */
+  @Override
   public void sort(IndexedSortable s, int p, int r) {
     sort(s, p, r, null);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  @Override
   public void sort(final IndexedSortable s, int p, int r,
       final Progressable rep) {
     sortInternal(s, p, r, rep, getMaxDepth(r - p));

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java Tue Oct 16 00:02:55 2012
@@ -257,6 +257,7 @@ public class ReflectionUtils {
    */
   private static ThreadLocal<CopyInCopyOutBuffer> cloneBuffers
       = new ThreadLocal<CopyInCopyOutBuffer>() {
+      @Override
       protected synchronized CopyInCopyOutBuffer initialValue() {
         return new CopyInCopyOutBuffer();
       }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java Tue Oct 16 00:02:55 2012
@@ -30,7 +30,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 
 /** 
  * A base class for running a Unix command.
@@ -323,10 +322,12 @@ abstract public class Shell {
       this.run();    
     }
 
+    @Override
     public String[] getExecString() {
       return command;
     }
 
+    @Override
     protected void parseExecResult(BufferedReader lines) throws IOException {
       output = new StringBuffer();
       char[] buf = new char[512];
@@ -348,6 +349,7 @@ abstract public class Shell {
      *
      * @return a string representation of the object.
      */
+    @Override
     public String toString() {
       StringBuilder builder = new StringBuilder();
       String[] args = getExecString();

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java Tue Oct 16 00:02:55 2012
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.StringTokenizer;
 
+import com.google.common.net.InetAddresses;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
@@ -77,6 +78,9 @@ public class StringUtils {
    * @return the hostname to the first dot
    */
   public static String simpleHostname(String fullHostname) {
+    if (InetAddresses.isInetAddress(fullHostname)) {
+      return fullHostname;
+    }
     int offset = fullHostname.indexOf('.');
     if (offset != -1) {
       return fullHostname.substring(0, offset);
@@ -348,7 +352,7 @@ public class StringUtils {
    * @return an array of <code>String</code> values
    */
   public static String[] getTrimmedStrings(String str){
-    if (null == str || "".equals(str.trim())) {
+    if (null == str || str.trim().isEmpty()) {
       return emptyStringArray;
     }
 
@@ -408,7 +412,7 @@ public class StringUtils {
       String str, char separator) {
     // String.split returns a single empty result for splitting the empty
     // string.
-    if ("".equals(str)) {
+    if (str.isEmpty()) {
       return new String[]{""};
     }
     ArrayList<String> strList = new ArrayList<String>();

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Filter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Filter.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Filter.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Filter.java Tue Oct 16 00:02:55 2012
@@ -193,6 +193,7 @@ public abstract class Filter implements 
   
   // Writable interface
   
+  @Override
   public void write(DataOutput out) throws IOException {
     out.writeInt(VERSION);
     out.writeInt(this.nbHash);
@@ -200,6 +201,7 @@ public abstract class Filter implements 
     out.writeInt(this.vectorSize);
   }
 
+  @Override
   public void readFields(DataInput in) throws IOException {
     int ver = in.readInt();
     if (ver > 0) { // old unversioned format

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java Tue Oct 16 00:02:55 2012
@@ -154,12 +154,14 @@ public class Key implements WritableComp
 
   // Writable
 
+  @Override
   public void write(DataOutput out) throws IOException {
     out.writeInt(bytes.length);
     out.write(bytes);
     out.writeDouble(weight);
   }
   
+  @Override
   public void readFields(DataInput in) throws IOException {
     this.bytes = new byte[in.readInt()];
     in.readFully(this.bytes);

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java Tue Oct 16 00:02:55 2012
@@ -81,6 +81,7 @@ public class JenkinsHash extends Hash {
    * <p>Use for hash table lookup, or anything where one collision in 2^^32 is
    * acceptable.  Do NOT use for cryptographic purposes.
   */
+  @Override
   @SuppressWarnings("fallthrough")
   public int hash(byte[] key, int nbytes, int initval) {
     int length = nbytes;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/MurmurHash.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/MurmurHash.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/MurmurHash.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/MurmurHash.java Tue Oct 16 00:02:55 2012
@@ -37,6 +37,7 @@ public class MurmurHash extends Hash {
     return _instance;
   }
   
+  @Override
   public int hash(byte[] data, int length, int seed) {
     int m = 0x5bd1e995;
     int r = 24;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c Tue Oct 16 00:02:55 2012
@@ -88,7 +88,7 @@ JNIEXPORT jint JNICALL Java_org_apache_h
 
   compressed_direct_buf_len = LZ4_compress(uncompressed_bytes, compressed_bytes, uncompressed_direct_buf_len);
   if (compressed_direct_buf_len < 0){
-    THROW(env, "Ljava/lang/InternalError", "LZ4_compress failed");
+    THROW(env, "java/lang/InternalError", "LZ4_compress failed");
   }
 
   (*env)->SetIntField(env, thisj, Lz4Compressor_uncompressedDirectBufLen, 0);

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c Tue Oct 16 00:02:55 2012
@@ -85,7 +85,7 @@ JNIEXPORT jint JNICALL Java_org_apache_h
 
   uncompressed_direct_buf_len = LZ4_uncompress_unknownOutputSize(compressed_bytes, uncompressed_bytes, compressed_direct_buf_len, uncompressed_direct_buf_len);
   if (uncompressed_direct_buf_len < 0) {
-    THROW(env, "Ljava/lang/InternalError", "LZ4_uncompress_unknownOutputSize failed.");
+    THROW(env, "java/lang/InternalError", "LZ4_uncompress_unknownOutputSize failed.");
   }
 
   (*env)->SetIntField(env, thisj, Lz4Decompressor_compressedDirectBufLen, 0);

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c Tue Oct 16 00:02:55 2012
@@ -98,11 +98,11 @@ JNIEXPORT jint JNICALL Java_org_apache_h
   snappy_status ret = dlsym_snappy_compress(uncompressed_bytes,
         uncompressed_direct_buf_len, compressed_bytes, &buf_len);
   if (ret != SNAPPY_OK){
-    THROW(env, "Ljava/lang/InternalError", "Could not compress data. Buffer length is too small.");
+    THROW(env, "java/lang/InternalError", "Could not compress data. Buffer length is too small.");
     return 0;
   }
   if (buf_len > JINT_MAX) {
-    THROW(env, "Ljava/lang/InternalError", "Invalid return buffer length.");
+    THROW(env, "java/lang/InternalError", "Invalid return buffer length.");
     return 0;
   }
 

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c Tue Oct 16 00:02:55 2012
@@ -92,11 +92,11 @@ JNIEXPORT jint JNICALL Java_org_apache_h
 
   snappy_status ret = dlsym_snappy_uncompress(compressed_bytes, compressed_direct_buf_len, uncompressed_bytes, &uncompressed_direct_buf_len);
   if (ret == SNAPPY_BUFFER_TOO_SMALL){
-    THROW(env, "Ljava/lang/InternalError", "Could not decompress data. Buffer length is too small.");
+    THROW(env, "java/lang/InternalError", "Could not decompress data. Buffer length is too small.");
   } else if (ret == SNAPPY_INVALID_INPUT){
-    THROW(env, "Ljava/lang/InternalError", "Could not decompress data. Input is invalid.");
+    THROW(env, "java/lang/InternalError", "Could not decompress data. Input is invalid.");
   } else if (ret != SNAPPY_OK){
-    THROW(env, "Ljava/lang/InternalError", "Could not decompress data.");
+    THROW(env, "java/lang/InternalError", "Could not decompress data.");
   }
 
   (*env)->SetIntField(env, thisj, SnappyDecompressor_compressedDirectBufLen, 0);

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c Tue Oct 16 00:02:55 2012
@@ -23,6 +23,7 @@
  */
 #include <assert.h>
 #include <arpa/inet.h>
+#include <errno.h>
 #include <stdint.h>
 #include <unistd.h>
 
@@ -33,9 +34,10 @@
 
 #define USE_PIPELINED
 
+#define CRC_INITIAL_VAL 0xffffffff
+
 typedef uint32_t (*crc_update_func_t)(uint32_t, const uint8_t *, size_t);
-static uint32_t crc_init();
-static uint32_t crc_val(uint32_t crc);
+static inline uint32_t crc_val(uint32_t crc);
 static uint32_t crc32_zlib_sb8(uint32_t crc, const uint8_t *buf, size_t length);
 static uint32_t crc32c_sb8(uint32_t crc, const uint8_t *buf, size_t length);
 
@@ -45,6 +47,35 @@ static void pipelined_crc32c(uint32_t *c
 static int cached_cpu_supports_crc32; // initialized by constructor below
 static uint32_t crc32c_hardware(uint32_t crc, const uint8_t* data, size_t length);
 
+int bulk_calculate_crc(const uint8_t *data, size_t data_len,
+                    uint32_t *sums, int checksum_type,
+                    int bytes_per_checksum) {
+  uint32_t crc;
+  crc_update_func_t crc_update_func;
+
+  switch (checksum_type) {
+    case CRC32_ZLIB_POLYNOMIAL:
+      crc_update_func = crc32_zlib_sb8;
+      break;
+    case CRC32C_POLYNOMIAL:
+      crc_update_func = crc32c_sb8;
+      break;
+    default:
+      return -EINVAL;
+      break;
+  }
+  while (likely(data_len > 0)) {
+    int len = likely(data_len >= bytes_per_checksum) ? bytes_per_checksum : data_len;
+    crc = CRC_INITIAL_VAL;
+    crc = crc_update_func(crc, data, len);
+    *sums = ntohl(crc_val(crc));
+    data += len;
+    data_len -= len;
+    sums++;
+  }
+  return 0;
+}
+
 int bulk_verify_crc(const uint8_t *data, size_t data_len,
                     const uint32_t *sums, int checksum_type,
                     int bytes_per_checksum,
@@ -80,7 +111,7 @@ int bulk_verify_crc(const uint8_t *data,
   if (do_pipelined) {
     /* Process three blocks at a time */
     while (likely(n_blocks >= 3)) {
-      crc1 = crc2 = crc3 = crc_init();  
+      crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
       pipelined_crc32c(&crc1, &crc2, &crc3, data, bytes_per_checksum, 3);
 
       crc = ntohl(crc_val(crc1));
@@ -101,7 +132,7 @@ int bulk_verify_crc(const uint8_t *data,
 
     /* One or two blocks */
     if (n_blocks) {
-      crc1 = crc2 = crc_init();
+      crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
       pipelined_crc32c(&crc1, &crc2, &crc3, data, bytes_per_checksum, n_blocks);
 
       if ((crc = ntohl(crc_val(crc1))) != *sums)
@@ -118,7 +149,7 @@ int bulk_verify_crc(const uint8_t *data,
  
     /* For something smaller than a block */
     if (remainder) {
-      crc1 = crc_init();
+      crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
       pipelined_crc32c(&crc1, &crc2, &crc3, data, remainder, 1);
 
       if ((crc = ntohl(crc_val(crc1))) != *sums)
@@ -130,7 +161,7 @@ int bulk_verify_crc(const uint8_t *data,
 
   while (likely(data_len > 0)) {
     int len = likely(data_len >= bytes_per_checksum) ? bytes_per_checksum : data_len;
-    crc = crc_init();
+    crc = CRC_INITIAL_VAL;
     crc = crc_update_func(crc, data, len);
     crc = ntohl(crc_val(crc));
     if (unlikely(crc != *sums)) {
@@ -151,18 +182,10 @@ return_crc_error:
   return INVALID_CHECKSUM_DETECTED;
 }
 
-
-/**
- * Initialize a CRC
- */
-static uint32_t crc_init() {
-  return 0xffffffff;
-}
-
 /**
  * Extract the final result of a CRC
  */
-static uint32_t crc_val(uint32_t crc) {
+static inline uint32_t crc_val(uint32_t crc) {
   return ~crc;
 }
 
@@ -398,7 +421,7 @@ static void pipelined_crc32c(uint32_t *c
         counter--;
       }
 
-      /* Take care of the remainder. They are only up to three bytes,
+      /* Take care of the remainder. They are only up to seven bytes,
        * so performing byte-level crc32 won't take much time.
        */
       bdata = (uint8_t*)data;
@@ -433,7 +456,7 @@ static void pipelined_crc32c(uint32_t *c
         "crc32b (%5), %0;\n\t"
         "crc32b (%5,%4,1), %1;\n\t"
          : "=r"(c1), "=r"(c2) 
-         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(bdata)
+         : "r"(c1), "r"(c2), "r"(block_size), "r"(bdata)
         );
         bdata++;
         remainder--;
@@ -593,7 +616,7 @@ static void pipelined_crc32c(uint32_t *c
         "crc32b (%5), %0;\n\t"
         "crc32b (%5,%4,1), %1;\n\t"
          : "=r"(c1), "=r"(c2) 
-         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(bdata)
+         : "r"(c1), "r"(c2), "r"(block_size), "r"(bdata)
         );
         bdata++;
         remainder--;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.h?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.h (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.h Tue Oct 16 00:02:55 2012
@@ -19,6 +19,7 @@
 #define BULK_CRC32_H_INCLUDED
 
 #include <stdint.h>
+#include <unistd.h> /* for size_t */
 
 // Constants for different CRC algorithms
 #define CRC32C_POLYNOMIAL 1
@@ -42,16 +43,45 @@ typedef struct crc32_error {
  * of bytes_per_checksum bytes. The checksums are each 32 bits
  * and are stored in sequential indexes of the 'sums' array.
  *
- *  checksum_type - one of the CRC32 constants defined above
- *  error_info - if non-NULL, will be filled in if an error
- *               is detected
+ * @param data                  The data to checksum
+ * @param dataLen               Length of the data buffer
+ * @param sums                  (out param) buffer to write checksums into.
+ *                              It must contain at least dataLen * 4 bytes.
+ * @param checksum_type         One of the CRC32 algorithm constants defined 
+ *                              above
+ * @param bytes_per_checksum    How many bytes of data to process per checksum.
+ * @param error_info            If non-NULL, will be filled in if an error
+ *                              is detected
  *
- * Returns: 0 for success, non-zero for an error, result codes
- *          for which are defined above
+ * @return                      0 for success, non-zero for an error, result codes
+ *                              for which are defined above
  */
 extern int bulk_verify_crc(const uint8_t *data, size_t data_len,
     const uint32_t *sums, int checksum_type,
     int bytes_per_checksum,
     crc32_error_t *error_info);
 
+/**
+ * Calculate checksums for some data.
+ *
+ * The checksums are each 32 bits and are stored in sequential indexes of the
+ * 'sums' array.
+ *
+ * This function is not (yet) optimized.  It is provided for testing purposes
+ * only.
+ *
+ * @param data                  The data to checksum
+ * @param dataLen               Length of the data buffer
+ * @param sums                  (out param) buffer to write checksums into.
+ *                              It must contain at least dataLen * 4 bytes.
+ * @param checksum_type         One of the CRC32 algorithm constants defined 
+ *                              above
+ * @param bytesPerChecksum      How many bytes of data to process per checksum.
+ *
+ * @return                      0 for success, non-zero for an error
+ */
+int bulk_calculate_crc(const uint8_t *data, size_t data_len,
+                    uint32_t *sums, int checksum_type,
+                    int bytes_per_checksum);
+
 #endif

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml Tue Oct 16 00:02:55 2012
@@ -239,5 +239,12 @@
        group list is separated by a blank. For e.g. "alice,bob users,wheel".
        A special value of "*" means all users are allowed.</description>
     </property>
+  
+  <property>
+    <name>security.qjournal.service.protocol.acl</name>
+    <value>${HADOOP_HDFS_USER}</value>
+    <description>ACL for QJournalProtocol, used by the NN to communicate with
+    JNs when using the QuorumJournalManager for edit logs.</description>
+  </property>
 
 </configuration>

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem Tue Oct 16 00:02:55 2012
@@ -17,6 +17,5 @@ org.apache.hadoop.fs.LocalFileSystem
 org.apache.hadoop.fs.viewfs.ViewFileSystem
 org.apache.hadoop.fs.s3.S3FileSystem
 org.apache.hadoop.fs.s3native.NativeS3FileSystem
-org.apache.hadoop.fs.kfs.KosmosFileSystem
 org.apache.hadoop.fs.ftp.FTPFileSystem
 org.apache.hadoop.fs.HarFileSystem

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml Tue Oct 16 00:02:55 2012
@@ -251,6 +251,14 @@
 </property>
 
 <property>
+    <name>hadoop.kerberos.min.seconds.before.relogin</name>
+    <value>60</value>
+    <description>The minimum time between relogin attempts for Kerberos, in
+    seconds.
+    </description>
+</property>
+
+<property>
   <name>hadoop.security.auth_to_local</name>
   <value></value>
   <description>Maps kerberos principals to local user names</description>
@@ -766,42 +774,6 @@
   <description>Replication factor</description>
 </property>
 
-<!-- Kosmos File System -->
-
-<property>
-  <name>kfs.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>kfs.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  kfs.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>kfs.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>kfs.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>kfs.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
 <!-- FTP file system -->
 <property>
   <name>ftp.stream-buffer-size</name>
@@ -1088,4 +1060,23 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.jetty.logs.serve.aliases</name>
+  <value>true</value>
+  <description>
+    Enable/Disable aliases serving from jetty
+  </description>
+</property>
+
+<property>
+  <name>fs.permissions.umask-mode</name>
+  <value>022</value>
+  <description>
+    The umask used when creating files and directories.
+    Can be in octal or in symbolic. Examples are:
+    "022" (octal for u=rwx,g=r-x,o=r-x in symbolic),
+    or "u=rwx,g=rwx,o=" (symbolic for 007 in octal).
+  </description>
+</property>
+
 </configuration>

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/site/apt/DeprecatedProperties.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/site/apt/DeprecatedProperties.apt.vm?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/site/apt/DeprecatedProperties.apt.vm (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/site/apt/DeprecatedProperties.apt.vm Tue Oct 16 00:02:55 2012
@@ -24,8 +24,6 @@ Deprecated Properties
 *-------------------------------+-----------------------+
 || <<Deprecated property name>> || <<New property name>>|
 *-------------------------------+-----------------------+
-|StorageId | dfs.datanode.StorageId
-*---+---+
 |create.empty.dir.if.nonexist | mapreduce.jobcontrol.createdir.ifnotexist
 *---+---+
 |dfs.access.time.precision | dfs.namenode.accesstime.precision
@@ -38,14 +36,16 @@ Deprecated Properties
 *---+---+
 |dfs.block.size | dfs.blocksize
 *---+---+
-|dfs.client.buffer.dir | fs.client.buffer.dir
-*---+---+
 |dfs.data.dir | dfs.datanode.data.dir
 *---+---+
 |dfs.datanode.max.xcievers | dfs.datanode.max.transfer.threads
 *---+---+
 |dfs.df.interval | fs.df.interval
 *---+---+
+|dfs.federation.nameservice.id | dfs.nameservice.id
+*---+---+
+|dfs.federation.nameservices | dfs.nameservices
+*---+---+
 |dfs.http.address | dfs.namenode.http-address
 *---+---+
 |dfs.https.address | dfs.namenode.https-address
@@ -54,10 +54,10 @@ Deprecated Properties
 *---+---+
 |dfs.https.need.client.auth | dfs.client.https.need-auth
 *---+---+
-|dfs.max-repl-streams | dfs.namenode.replication.max-streams
-*---+---+
 |dfs.max.objects | dfs.namenode.max.objects
 *---+---+
+|dfs.max-repl-streams | dfs.namenode.replication.max-streams
+*---+---+
 |dfs.name.dir | dfs.namenode.name.dir
 *---+---+
 |dfs.name.dir.restore | dfs.namenode.name.dir.restore
@@ -86,6 +86,8 @@ Deprecated Properties
 *---+---+
 |dfs.socket.timeout | dfs.client.socket-timeout
 *---+---+
+|dfs.umaskmode | fs.permissions.umask-mode
+*---+---+
 |dfs.write.packet.size | dfs.client-write-packet-size
 *---+---+
 |fs.checkpoint.dir | dfs.namenode.checkpoint.dir
@@ -106,10 +108,10 @@ Deprecated Properties
 *---+---+
 |hadoop.pipes.command-file.keep | mapreduce.pipes.commandfile.preserve
 *---+---+
-|hadoop.pipes.executable | mapreduce.pipes.executable
-*---+---+
 |hadoop.pipes.executable.interpretor | mapreduce.pipes.executable.interpretor
 *---+---+
+|hadoop.pipes.executable | mapreduce.pipes.executable
+*---+---+
 |hadoop.pipes.java.mapper | mapreduce.pipes.isjavamapper
 *---+---+
 |hadoop.pipes.java.recordreader | mapreduce.pipes.isjavarecordreader
@@ -130,6 +132,12 @@ Deprecated Properties
 *---+---+
 |io.sort.spill.percent | mapreduce.map.sort.spill.percent
 *---+---+
+|jobclient.completion.poll.interval | mapreduce.client.completion.pollinterval
+*---+---+
+|jobclient.output.filter | mapreduce.client.output.filter
+*---+---+
+|jobclient.progress.monitor.poll.interval | mapreduce.client.progressmonitor.pollinterval
+*---+---+
 |job.end.notification.url | mapreduce.job.end-notification.url
 *---+---+
 |job.end.retry.attempts | mapreduce.job.end-notification.retry.attempts
@@ -138,12 +146,6 @@ Deprecated Properties
 *---+---+
 |job.local.dir | mapreduce.job.local.dir
 *---+---+
-|jobclient.completion.poll.interval | mapreduce.client.completion.pollinterval
-*---+---+
-|jobclient.output.filter | mapreduce.client.output.filter
-*---+---+
-|jobclient.progress.monitor.poll.interval | mapreduce.client.progressmonitor.pollinterval
-*---+---+
 |keep.failed.task.files | mapreduce.task.files.preserve.failedtasks
 *---+---+
 |keep.task.files.pattern | mapreduce.task.files.preserve.filepattern
@@ -196,10 +198,6 @@ Deprecated Properties
 *---+---+
 |mapred.compress.map.output | mapreduce.map.output.compress
 *---+---+
-|mapred.create.symlink | NONE - symlinking is always on
-*---+---+
-|mapreduce.job.cache.symlink.create | NONE - symlinking is always on
-*---+---+
 |mapred.data.field.separator | mapreduce.fieldsel.data.field.separator
 *---+---+
 |mapred.debug.out.lines | mapreduce.task.debugout.lines
@@ -214,18 +212,18 @@ Deprecated Properties
 *---+---+
 |mapred.heartbeats.in.second | mapreduce.jobtracker.heartbeats.in.second
 *---+---+
-|mapred.hosts | mapreduce.jobtracker.hosts.filename
-*---+---+
 |mapred.hosts.exclude | mapreduce.jobtracker.hosts.exclude.filename
 *---+---+
-|mapred.inmem.merge.threshold | mapreduce.reduce.merge.inmem.threshold
+|mapred.hosts | mapreduce.jobtracker.hosts.filename
 *---+---+
-|mapred.input.dir | mapreduce.input.fileinputformat.inputdir
+|mapred.inmem.merge.threshold | mapreduce.reduce.merge.inmem.threshold
 *---+---+
 |mapred.input.dir.formats | mapreduce.input.multipleinputs.dir.formats
 *---+---+
 |mapred.input.dir.mappers | mapreduce.input.multipleinputs.dir.mappers
 *---+---+
+|mapred.input.dir | mapreduce.input.fileinputformat.inputdir
+*---+---+
 |mapred.input.pathFilter.class | mapreduce.input.pathFilter.class
 *---+---+
 |mapred.jar | mapreduce.job.jar
@@ -236,6 +234,8 @@ Deprecated Properties
 *---+---+
 |mapred.job.id | mapreduce.job.id
 *---+---+
+|mapred.jobinit.threads | mapreduce.jobtracker.jobinit.threads
+*---+---+
 |mapred.job.map.memory.mb | mapreduce.map.memory.mb
 *---+---+
 |mapred.job.name | mapreduce.job.name
@@ -258,42 +258,40 @@ Deprecated Properties
 *---+---+
 |mapred.job.shuffle.merge.percent | mapreduce.reduce.shuffle.merge.percent
 *---+---+
-|mapred.job.tracker | mapreduce.jobtracker.address
-*---+---+
 |mapred.job.tracker.handler.count | mapreduce.jobtracker.handler.count
 *---+---+
 |mapred.job.tracker.history.completed.location | mapreduce.jobtracker.jobhistory.completed.location
 *---+---+
 |mapred.job.tracker.http.address | mapreduce.jobtracker.http.address
 *---+---+
+|mapred.jobtracker.instrumentation | mapreduce.jobtracker.instrumentation
+*---+---+
+|mapred.jobtracker.job.history.block.size | mapreduce.jobtracker.jobhistory.block.size
+*---+---+
 |mapred.job.tracker.jobhistory.lru.cache.size | mapreduce.jobtracker.jobhistory.lru.cache.size
 *---+---+
+|mapred.job.tracker | mapreduce.jobtracker.address
+*---+---+
+|mapred.jobtracker.maxtasks.per.job | mapreduce.jobtracker.maxtasks.perjob
+*---+---+
 |mapred.job.tracker.persist.jobstatus.active | mapreduce.jobtracker.persist.jobstatus.active
 *---+---+
 |mapred.job.tracker.persist.jobstatus.dir | mapreduce.jobtracker.persist.jobstatus.dir
 *---+---+
 |mapred.job.tracker.persist.jobstatus.hours | mapreduce.jobtracker.persist.jobstatus.hours
 *---+---+
-|mapred.job.tracker.retire.jobs | mapreduce.jobtracker.retirejobs
+|mapred.jobtracker.restart.recover | mapreduce.jobtracker.restart.recover
 *---+---+
 |mapred.job.tracker.retiredjobs.cache.size | mapreduce.jobtracker.retiredjobs.cache.size
 *---+---+
-|mapred.jobinit.threads | mapreduce.jobtracker.jobinit.threads
-*---+---+
-|mapred.jobtracker.instrumentation | mapreduce.jobtracker.instrumentation
-*---+---+
-|mapred.jobtracker.job.history.block.size | mapreduce.jobtracker.jobhistory.block.size
-*---+---+
-|mapred.jobtracker.maxtasks.per.job | mapreduce.jobtracker.maxtasks.perjob
+|mapred.job.tracker.retire.jobs | mapreduce.jobtracker.retirejobs
 *---+---+
-|mapred.jobtracker.restart.recover | mapreduce.jobtracker.restart.recover
+|mapred.jobtracker.taskalloc.capacitypad | mapreduce.jobtracker.taskscheduler.taskalloc.capacitypad
 *---+---+
 |mapred.jobtracker.taskScheduler | mapreduce.jobtracker.taskscheduler
 *---+---+
 |mapred.jobtracker.taskScheduler.maxRunningTasksPerJob | mapreduce.jobtracker.taskscheduler.maxrunningtasks.perjob
 *---+---+
-|mapred.jobtracker.taskalloc.capacitypad | mapreduce.jobtracker.taskscheduler.taskalloc.capacitypad
-*---+---+
 |mapred.join.expr | mapreduce.join.expr
 *---+---+
 |mapred.join.keycomparator | mapreduce.join.keycomparator
@@ -320,19 +318,19 @@ Deprecated Properties
 *---+---+
 |mapred.map.output.compression.codec | mapreduce.map.output.compress.codec
 *---+---+
-|mapred.map.task.debug.script | mapreduce.map.debug.script
-*---+---+
-|mapred.map.tasks | mapreduce.job.maps
-*---+---+
-|mapred.map.tasks.speculative.execution | mapreduce.map.speculative
-*---+---+
 |mapred.mapoutput.key.class | mapreduce.map.output.key.class
 *---+---+
 |mapred.mapoutput.value.class | mapreduce.map.output.value.class
 *---+---+
+|mapred.mapper.regex.group | mapreduce.mapper.regexmapper..group
+*---+---+
 |mapred.mapper.regex | mapreduce.mapper.regex
 *---+---+
-|mapred.mapper.regex.group | mapreduce.mapper.regexmapper..group
+|mapred.map.task.debug.script | mapreduce.map.debug.script
+*---+---+
+|mapred.map.tasks | mapreduce.job.maps
+*---+---+
+|mapred.map.tasks.speculative.execution | mapreduce.map.speculative
 *---+---+
 |mapred.max.map.failures.percent | mapreduce.map.failures.maxpercent
 *---+---+
@@ -352,12 +350,12 @@ Deprecated Properties
 *---+---+
 |mapred.min.split.size.per.rack | mapreduce.input.fileinputformat.split.minsize.per.rack
 *---+---+
-|mapred.output.compress | mapreduce.output.fileoutputformat.compress
-*---+---+
 |mapred.output.compression.codec | mapreduce.output.fileoutputformat.compress.codec
 *---+---+
 |mapred.output.compression.type | mapreduce.output.fileoutputformat.compress.type
 *---+---+
+|mapred.output.compress | mapreduce.output.fileoutputformat.compress
+*---+---+
 |mapred.output.dir | mapreduce.output.fileoutputformat.outputdir
 *---+---+
 |mapred.output.key.class | mapreduce.job.output.key.class
@@ -440,12 +438,6 @@ Deprecated Properties
 *---+---+
 |mapred.task.timeout | mapreduce.task.timeout
 *---+---+
-|mapred.task.tracker.http.address | mapreduce.tasktracker.http.address
-*---+---+
-|mapred.task.tracker.report.address | mapreduce.tasktracker.report.address
-*---+---+
-|mapred.task.tracker.task-controller | mapreduce.tasktracker.taskcontroller
-*---+---+
 |mapred.tasktracker.dns.interface | mapreduce.tasktracker.dns.interface
 *---+---+
 |mapred.tasktracker.dns.nameserver | mapreduce.tasktracker.dns.nameserver
@@ -454,6 +446,8 @@ Deprecated Properties
 *---+---+
 |mapred.tasktracker.expiry.interval | mapreduce.jobtracker.expire.trackers.interval
 *---+---+
+|mapred.task.tracker.http.address | mapreduce.tasktracker.http.address
+*---+---+
 |mapred.tasktracker.indexcache.mb | mapreduce.tasktracker.indexcache.mb
 *---+---+
 |mapred.tasktracker.instrumentation | mapreduce.tasktracker.instrumentation
@@ -466,6 +460,10 @@ Deprecated Properties
 *---+---+
 |mapred.tasktracker.reduce.tasks.maximum | mapreduce.tasktracker.reduce.tasks.maximum
 *---+---+
+|mapred.task.tracker.report.address | mapreduce.tasktracker.report.address
+*---+---+
+|mapred.task.tracker.task-controller | mapreduce.tasktracker.taskcontroller
+*---+---+
 |mapred.tasktracker.taskmemorymanager.monitoring-interval | mapreduce.tasktracker.taskmemorymanager.monitoringinterval
 *---+---+
 |mapred.tasktracker.tasks.sleeptime-before-sigkill | mapreduce.tasktracker.tasks.sleeptimebeforesigkill
@@ -480,20 +478,12 @@ Deprecated Properties
 *---+---+
 |mapred.tip.id | mapreduce.task.id
 *---+---+
-|mapred.used.genericoptionsparser | mapreduce.client.genericoptionsparser.used
-*---+---+
-|mapred.userlog.limit.kb | mapreduce.task.userlog.limit.kb
-*---+---+
-|mapred.userlog.retain.hours | mapreduce.job.userlog.retain.hours
-*---+---+
-|mapred.work.output.dir | mapreduce.task.output.dir
-*---+---+
-|mapred.working.dir | mapreduce.job.working.dir
-*---+---+
 |mapreduce.combine.class | mapreduce.job.combine.class
 *---+---+
 |mapreduce.inputformat.class | mapreduce.job.inputformat.class
 *---+---+
+|mapreduce.job.counters.limit | mapreduce.job.counters.max
+*---+---+
 |mapreduce.jobtracker.permissions.supergroup | mapreduce.cluster.permissions.supergroup
 *---+---+
 |mapreduce.map.class | mapreduce.job.map.class
@@ -504,6 +494,16 @@ Deprecated Properties
 *---+---+
 |mapreduce.reduce.class | mapreduce.job.reduce.class
 *---+---+
+|mapred.used.genericoptionsparser | mapreduce.client.genericoptionsparser.used
+*---+---+
+|mapred.userlog.limit.kb | mapreduce.task.userlog.limit.kb
+*---+---+
+|mapred.userlog.retain.hours | mapreduce.job.userlog.retain.hours
+*---+---+
+|mapred.working.dir | mapreduce.job.working.dir
+*---+---+
+|mapred.work.output.dir | mapreduce.task.output.dir
+*---+---+
 |min.num.spills.for.combine | mapreduce.map.combine.minspills
 *---+---+
 |reduce.output.key.value.fields.spec | mapreduce.fieldsel.reduce.output.key.value.fields.spec
@@ -538,3 +538,13 @@ Deprecated Properties
 *---+---+
 |webinterface.private.actions | mapreduce.jobtracker.webinterface.trusted
 *---+---+
+
+  The following table lists additional changes to some configuration properties:
+
+*-------------------------------+-----------------------+
+|| <<Deprecated property name>> || <<New property name>>|
+*-------------------------------+-----------------------+
+|mapred.create.symlink | NONE - symlinking is always on
+*---+---+
+|mapreduce.job.cache.symlink.create | NONE - symlinking is always on
+*---+---+

Propchange: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/core/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-3077/hadoop-common-project/hadoop-common/src/test/core:r1363593-1396941

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLICommand.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLICommand.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLICommand.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLICommand.java Tue Oct 16 00:02:55 2012
@@ -24,5 +24,6 @@ public interface CLICommand {
   public CommandExecutor getExecutor(String tag) throws IllegalArgumentException;
   public CLICommandTypes getType();
   public String getCmd();
+  @Override
   public String toString();
 }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLITestCmd.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLITestCmd.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLITestCmd.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CLITestCmd.java Tue Oct 16 00:02:55 2012
@@ -31,6 +31,7 @@ public class CLITestCmd implements CLICo
     this.type = type;
   }
 
+  @Override
   public CommandExecutor getExecutor(String tag) throws IllegalArgumentException {
     if (getType() instanceof CLICommandFS)
       return new FSCmdExecutor(tag, new FsShell());
@@ -38,12 +39,17 @@ public class CLITestCmd implements CLICo
         IllegalArgumentException("Unknown type of test command: " + getType());
   }
 
+  @Override
   public CLICommandTypes getType() {
     return type;
   }
+  
+  @Override
   public String getCmd() {
     return cmd;
   }
+  
+  @Override
   public String toString() {
     return cmd;
   }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/FSCmdExecutor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/FSCmdExecutor.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/FSCmdExecutor.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/FSCmdExecutor.java Tue Oct 16 00:02:55 2012
@@ -29,6 +29,7 @@ public class FSCmdExecutor extends Comma
     this.shell = shell;
   }
 
+  @Override
   protected void execute(final String cmd) throws Exception{
     String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
     ToolRunner.run(shell, args);

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java Tue Oct 16 00:02:55 2012
@@ -24,7 +24,6 @@ import javax.xml.parsers.DocumentBuilder
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.mortbay.util.ajax.JSON;
-import org.mortbay.util.ajax.JSON.Output;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java Tue Oct 16 00:02:55 2012
@@ -39,7 +39,6 @@ import java.util.regex.Pattern;
 
 import junit.framework.TestCase;
 import static org.junit.Assert.assertArrayEquals;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.Path;
@@ -359,6 +358,36 @@ public class TestConfiguration extends T
     tearDown();
   }
 
+  public void testRelativeIncludes() throws Exception {
+    tearDown();
+    String relConfig = new File("./tmp/test-config.xml").getAbsolutePath();
+    String relConfig2 = new File("./tmp/test-config2.xml").getAbsolutePath();
+
+    new File(new File(relConfig).getParent()).mkdirs();
+    out = new BufferedWriter(new FileWriter(relConfig2));
+    startConfig();
+    appendProperty("a", "b");
+    endConfig();
+
+    out = new BufferedWriter(new FileWriter(relConfig));
+    startConfig();
+    // Add the relative path instead of the absolute one.
+    addInclude(new File(relConfig2).getName());
+    appendProperty("c", "d");
+    endConfig();
+
+    // verify that the includes file contains all properties
+    Path fileResource = new Path(relConfig);
+    conf.addResource(fileResource);
+    assertEquals(conf.get("a"), "b");
+    assertEquals(conf.get("c"), "d");
+
+    // Cleanup
+    new File(relConfig).delete();
+    new File(relConfig2).delete();
+    new File(new File(relConfig).getParent()).delete();
+  }
+
   BufferedWriter out;
 	
   public void testIntegerRanges() {
@@ -1157,6 +1186,12 @@ public class TestConfiguration extends T
         configuration.getPattern("testPattern", Pattern.compile("")).pattern());
   }
   
+  public void testGetClassByNameOrNull() throws Exception {
+   Configuration config = new Configuration();
+   Class<?> clazz = config.getClassByNameOrNull("java.lang.Object");
+   assertNotNull(clazz);
+  }
+  
   public static void main(String[] argv) throws Exception {
     junit.textui.TestRunner.main(new String[]{
       TestConfiguration.class.getName()

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java Tue Oct 16 00:02:55 2012
@@ -19,8 +19,6 @@
 package org.apache.hadoop.conf;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestDeprecatedKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestDeprecatedKeys.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestDeprecatedKeys.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestDeprecatedKeys.java Tue Oct 16 00:02:55 2012
@@ -18,9 +18,6 @@
 
 package org.apache.hadoop.conf;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import java.io.ByteArrayOutputStream;
 import java.util.Map;
 

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java Tue Oct 16 00:02:55 2012
@@ -99,17 +99,11 @@ public class TestReconfiguration {
       super(conf);
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override 
     public Collection<String> getReconfigurableProperties() {
       return Arrays.asList(PROP1, PROP2, PROP4);
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public synchronized void reconfigurePropertyImpl(String property, 
                                                      String newVal) {

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java Tue Oct 16 00:02:55 2012
@@ -67,6 +67,7 @@ public abstract class FSMainOperationsBa
   protected static FileSystem fSys;
   
   final private static PathFilter DEFAULT_FILTER = new PathFilter() {
+    @Override
     public boolean accept(final Path file) {
       return true;
     }
@@ -74,6 +75,7 @@ public abstract class FSMainOperationsBa
 
   //A test filter with returns any path containing a "b" 
   final private static PathFilter TEST_X_FILTER = new PathFilter() {
+    @Override
     public boolean accept(Path file) {
       if(file.getName().contains("x") || file.getName().contains("X"))
         return true;
@@ -364,15 +366,17 @@ public abstract class FSMainOperationsBa
   }
   
   @Test
-  public void testGlobStatusThrowsExceptionForNonExistentFile() throws Exception {
-    try {
-      // This should throw a FileNotFoundException
-      fSys.globStatus(
-          getTestRootPath(fSys, "test/hadoopfsdf/?"));
-      Assert.fail("Should throw FileNotFoundException");
-    } catch (FileNotFoundException fnfe) {
-      // expected
-    }
+  public void testGlobStatusNonExistentFile() throws Exception {
+    FileStatus[] paths = fSys.globStatus(
+        getTestRootPath(fSys, "test/hadoopfsdf"));
+    Assert.assertNull(paths);
+
+    paths = fSys.globStatus(
+        getTestRootPath(fSys, "test/hadoopfsdf/?"));
+    Assert.assertEquals(0, paths.length);
+    paths = fSys.globStatus(
+        getTestRootPath(fSys, "test/hadoopfsdf/xyz*/?"));
+    Assert.assertEquals(0, paths.length);
   }
   
   @Test

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java Tue Oct 16 00:02:55 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs;
 
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
@@ -61,12 +62,20 @@ public abstract class FileContextMainOpe
   private static String TEST_DIR_AXX = "test/hadoop/axx";
   private static int numBlocks = 2;
   
-  static  final String LOCAL_FS_ROOT_URI = "file:///tmp/test";
-  
+  public static final Path LOCAL_FS_ROOT_PATH;
+      
+  static {
+    File testBuildData = new File(System.getProperty("test.build.data",
+                                    "build/test/data"));
+    Path localFsRootPath = new Path(testBuildData.getAbsolutePath(), 
+                                    "root-uri");
+    LOCAL_FS_ROOT_PATH = localFsRootPath.makeQualified(LocalFileSystem.NAME, null);
+  }
   
   protected static FileContext fc;
   
   final private static PathFilter DEFAULT_FILTER = new PathFilter() {
+    @Override
     public boolean accept(final Path file) {
       return true;
     }
@@ -74,6 +83,7 @@ public abstract class FileContextMainOpe
 
   //A test filter with returns any path containing a "b" 
   final private static PathFilter TEST_X_FILTER = new PathFilter() {
+    @Override
     public boolean accept(Path file) {
       if(file.getName().contains("x") || file.getName().contains("X"))
         return true;
@@ -93,7 +103,7 @@ public abstract class FileContextMainOpe
   @After
   public void tearDown() throws Exception {
     fc.delete(new Path(getAbsoluteTestRootPath(fc), new Path("test")), true);
-    fc.delete(new Path(LOCAL_FS_ROOT_URI), true);
+    fc.delete(LOCAL_FS_ROOT_PATH, true);
   }
   
   
@@ -174,7 +184,7 @@ public abstract class FileContextMainOpe
     
     // Try a URI
 
-    absoluteDir = new Path(LOCAL_FS_ROOT_URI + "/existingDir");
+    absoluteDir = new Path(LOCAL_FS_ROOT_PATH, "existingDir");
     fc.mkdir(absoluteDir, FileContext.DEFAULT_PERM, true);
     fc.setWorkingDirectory(absoluteDir);
     Assert.assertEquals(absoluteDir, fc.getWorkingDirectory());
@@ -360,15 +370,17 @@ public abstract class FileContextMainOpe
   }
   
   @Test
-  public void testGlobStatusThrowsExceptionForNonExistentFile() throws Exception {
-    try {
-      // This should throw a FileNotFoundException
-      fc.util().globStatus(
-          getTestRootPath(fc, "test/hadoopfsdf/?"));
-      Assert.fail("Should throw FileNotFoundException");
-    } catch (FileNotFoundException fnfe) {
-      // expected
-    }
+  public void testGlobStatusNonExistentFile() throws Exception {
+    FileStatus[] paths = fc.util().globStatus(
+          getTestRootPath(fc, "test/hadoopfsdf"));
+    Assert.assertNull(paths);
+
+    paths = fc.util().globStatus(
+        getTestRootPath(fc, "test/hadoopfsdf/?"));
+    Assert.assertEquals(0, paths.length);
+    paths = fc.util().globStatus(
+        getTestRootPath(fc, "test/hadoopfsdf/xyz*/?"));
+    Assert.assertEquals(0, paths.length);
   }
   
   @Test

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java Tue Oct 16 00:02:55 2012
@@ -176,6 +176,7 @@ public abstract class FileContextPermiss
         .createRemoteUser("otherUser");
     FileContext newFc = otherUser.doAs(new PrivilegedExceptionAction<FileContext>() {
 
+      @Override
       public FileContext run() throws Exception {
         FileContext newFc = FileContext.getFileContext();
         return newFc;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java Tue Oct 16 00:02:55 2012
@@ -20,8 +20,6 @@ package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.ArrayList;
-import java.util.Iterator;
-
 import junit.framework.Assert;
 
 import org.apache.hadoop.fs.permission.FsPermission;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAvroFSInput.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAvroFSInput.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAvroFSInput.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAvroFSInput.java Tue Oct 16 00:02:55 2012
@@ -19,7 +19,6 @@
 package org.apache.hadoop.fs;
 
 import java.io.BufferedWriter;
-import java.io.IOException;
 import java.io.OutputStreamWriter;
 
 import org.apache.hadoop.conf.Configuration;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java Tue Oct 16 00:02:55 2012
@@ -29,11 +29,13 @@ public class TestDU extends TestCase {
   final static private File DU_DIR = new File(
       System.getProperty("test.build.data","/tmp"), "dutmp");
 
+  @Override
   public void setUp() {
       FileUtil.fullyDelete(DU_DIR);
       assertTrue(DU_DIR.mkdirs());
   }
 
+  @Override
   public void tearDown() throws IOException {
       FileUtil.fullyDelete(DU_DIR);
   }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFSMainOperationsLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFSMainOperationsLocalFileSystem.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFSMainOperationsLocalFileSystem.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFSMainOperationsLocalFileSystem.java Tue Oct 16 00:02:55 2012
@@ -28,6 +28,7 @@ import org.junit.Test;
 
 public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTest {
 
+  @Override
   @Before
   public void setUp() throws Exception {
     fSys = FileSystem.getLocal(new Configuration());
@@ -35,12 +36,14 @@ public class TestFSMainOperationsLocalFi
   }
   
   static Path wd = null;
+  @Override
   protected Path getDefaultWorkingDirectory() throws IOException {
     if (wd == null)
       wd = FileSystem.getLocal(new Configuration()).getWorkingDirectory();
     return wd;
   }
   
+  @Override
   @After
   public void tearDown() throws Exception {
     super.tearDown();

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsPermission.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsPermission.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsPermission.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsPermission.java Tue Oct 16 00:02:55 2012
@@ -26,12 +26,14 @@ import org.junit.Before;
 public class TestFcLocalFsPermission extends 
   FileContextPermissionBase {
 
+  @Override
   @Before
   public void setUp() throws Exception {
     fc = FileContext.getLocalFSFileContext();
     super.setUp();
   }
 
+  @Override
   @After
   public void tearDown() throws Exception {
     super.tearDown();

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsUtil.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsUtil.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFcLocalFsUtil.java Tue Oct 16 00:02:55 2012
@@ -25,6 +25,7 @@ import org.junit.Before;
 public class TestFcLocalFsUtil extends
   FileContextUtilBase {
 
+  @Override
   @Before
   public void setUp() throws Exception {
     fc = FileContext.getLocalFSFileContext();

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCaching.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCaching.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCaching.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCaching.java Tue Oct 16 00:02:55 2012
@@ -110,6 +110,7 @@ public class TestFileSystemCaching {
   
   public static class InitializeForeverFileSystem extends LocalFileSystem {
     final static Semaphore sem = new Semaphore(0);
+    @Override
     public void initialize(URI uri, Configuration conf) throws IOException {
       // notify that InitializeForeverFileSystem started initialization
       sem.release();
@@ -127,6 +128,7 @@ public class TestFileSystemCaching {
   public void testCacheEnabledWithInitializeForeverFS() throws Exception {
     final Configuration conf = new Configuration();
     Thread t = new Thread() {
+      @Override
       public void run() {
         conf.set("fs.localfs1.impl", "org.apache.hadoop.fs." +
          "TestFileSystemCaching$InitializeForeverFileSystem");
@@ -167,11 +169,13 @@ public class TestFileSystemCaching {
     UserGroupInformation ugiA = UserGroupInformation.createRemoteUser("foo");
     UserGroupInformation ugiB = UserGroupInformation.createRemoteUser("bar");
     FileSystem fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
     });
     FileSystem fsA1 = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
@@ -180,6 +184,7 @@ public class TestFileSystemCaching {
     assertSame(fsA, fsA1);
     
     FileSystem fsB = ugiB.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
@@ -192,6 +197,7 @@ public class TestFileSystemCaching {
     UserGroupInformation ugiA2 = UserGroupInformation.createRemoteUser("foo");
     
     fsA = ugiA2.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
@@ -203,6 +209,7 @@ public class TestFileSystemCaching {
     ugiA.addToken(t1);
     
     fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
@@ -245,12 +252,14 @@ public class TestFileSystemCaching {
     conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
     UserGroupInformation ugiA = UserGroupInformation.createRemoteUser("foo");
     FileSystem fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
     });
     //Now we should get the cached filesystem
     FileSystem fsA1 = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }
@@ -261,6 +270,7 @@ public class TestFileSystemCaching {
     
     //Now we should get a different (newly created) filesystem
     fsA1 = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
       public FileSystem run() throws Exception {
         return FileSystem.get(new URI("cachedfile://a"), conf);
       }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsOptions.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsOptions.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsOptions.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsOptions.java Tue Oct 16 00:02:55 2012
@@ -19,8 +19,6 @@ package org.apache.hadoop.fs;
 
 import static org.junit.Assert.*;
 
-import java.io.IOException;
-
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.util.DataChecksum;
 

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java Tue Oct 16 00:02:55 2012
@@ -411,6 +411,7 @@ public class TestFsShellReturnCode {
   }
   
   static class MyFsShell extends FsShell {
+    @Override
     protected void registerCommands(CommandFactory factory) {
       factory.addClass(InterruptCommand.class, "-testInterrupt");
     }

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java Tue Oct 16 00:02:55 2012
@@ -18,7 +18,6 @@
 package org.apache.hadoop.fs;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java Tue Oct 16 00:02:55 2012
@@ -293,6 +293,23 @@ public class TestLocalDirAllocator {
     }
   }
 
+  /*
+   * Test when mapred.local.dir not configured and called
+   * getLocalPathForWrite
+   */
+  @Test
+  public void testShouldNotthrowNPE() throws Exception {
+    Configuration conf1 = new Configuration();
+    try {
+      dirAllocator.getLocalPathForWrite("/test", conf1);
+      fail("Exception not thrown when " + CONTEXT + " is not set");
+    } catch (IOException e) {
+      assertEquals(CONTEXT + " not configured", e.getMessage());
+    } catch (NullPointerException e) {
+      fail("Lack of configuration should not have thrown an NPE.");
+    }
+  }
+
   /** Test no side effect files are left over. After creating a temp
    * temp file, remove both the temp file and its parent. Verify that
    * no files or directories are left over as can happen when File objects

Modified: hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextCreateMkdir.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextCreateMkdir.java?rev=1398581&r1=1398580&r2=1398581&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextCreateMkdir.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextCreateMkdir.java Tue Oct 16 00:02:55 2012
@@ -23,6 +23,7 @@ import org.junit.Before;
 public class TestLocalFSFileContextCreateMkdir extends
   FileContextCreateMkdirBaseTest {
 
+  @Override
   @Before
   public void setUp() throws Exception {
     fc = FileContext.getLocalFSFileContext();