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 st...@apache.org on 2009/08/26 18:56:56 UTC

svn commit: r808123 - in /hadoop/common/branches/HADOOP-6194: ./ bin/ ivy/ src/java/org/apache/hadoop/conf/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/kfs/ src/java/org/apache/hadoop/fs/s3/ src/java/org/apache/hadoop/fs/s3native/ src/...

Author: stevel
Date: Wed Aug 26 16:56:55 2009
New Revision: 808123

URL: http://svn.apache.org/viewvc?rev=808123&view=rev
Log:
HADOOP-6194 service base class - updating branch to svn trunk

Modified:
    hadoop/common/branches/HADOOP-6194/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/HADOOP-6194/bin/hadoop-config.sh
    hadoop/common/branches/HADOOP-6194/bin/hdfs
    hadoop/common/branches/HADOOP-6194/bin/mapred
    hadoop/common/branches/HADOOP-6194/bin/rcc
    hadoop/common/branches/HADOOP-6194/ivy/hadoop-core.pom
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/conf/Configuration.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FileSystem.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FsShell.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/Trash.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/conf/TestConfiguration.java
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/FileSystemContractBaseTest.java
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java

Modified: hadoop/common/branches/HADOOP-6194/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/CHANGES.txt?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/CHANGES.txt (original)
+++ hadoop/common/branches/HADOOP-6194/CHANGES.txt Wed Aug 26 16:56:55 2009
@@ -74,6 +74,10 @@
     MAPREDUCE-711. Removed Distributed Cache from Common, to move it
     under Map/Reduce. (Vinod Kumar Vavilapalli via yhemanth)
 
+    HADOOP-6201. Change FileSystem::listStatus contract to throw
+    FileNotFoundException if the directory does not exist, rather than letting
+    this be implementation-specific. (Jakob Homan via cdouglas)
+
   NEW FEATURES
 
     HADOOP-4268. Change fsck to use ClientProtocol methods so that the
@@ -500,6 +504,9 @@
 
     HADOOP-6173. Change src/native/packageNativeHadoop.sh to package all
     native library files.  (Hong Tang via szetszwo)
+
+    HADOOP-6184. Provide an API to dump Configuration in a JSON format.
+    (V.V.Chaitanya Krishna via yhemanth)
  
   OPTIMIZATIONS
 
@@ -931,6 +938,9 @@
     HADOOP-6103. Clones the classloader as part of Configuration clone.
     (Amareshwari Sriramadasu via ddas)
 
+    HADOOP-6152. Fix classpath variables in bin/hadoop-config.sh and some
+    other scripts.  (Aaron Kimball via szetszwo)
+
 Release 0.20.1 - Unreleased
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HADOOP-6194/CHANGES.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Aug 26 16:56:55 2009
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/CHANGES.txt:804966-806688
+/hadoop/common/trunk/CHANGES.txt:804966-807681
 /hadoop/core/branches/branch-0.18/CHANGES.txt:727226
 /hadoop/core/branches/branch-0.19/CHANGES.txt:713112
 /hadoop/core/trunk/CHANGES.txt:776175-785643,785929-786278

Modified: hadoop/common/branches/HADOOP-6194/bin/hadoop-config.sh
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/bin/hadoop-config.sh?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/bin/hadoop-config.sh (original)
+++ hadoop/common/branches/HADOOP-6194/bin/hadoop-config.sh Wed Aug 26 16:56:55 2009
@@ -115,9 +115,6 @@
 if [ -d "$HADOOP_CORE_HOME/build/test/classes" ]; then
   CLASSPATH=${CLASSPATH}:$HADOOP_CORE_HOME/build/test/classes
 fi
-if [ -d "$HADOOP_CORE_HOME/build/tools" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_CORE_HOME/build/tools
-fi
 
 # so that filenames w/ spaces are handled correctly in loops below
 IFS=
@@ -157,13 +154,6 @@
   CLASSPATH=${CLASSPATH}:$f;
 done
 
-for f in $HADOOP_CORE_HOME/hadoop-*-tools.jar; do
-  TOOL_PATH=${TOOL_PATH}:$f;
-done
-for f in $HADOOP_CORE_HOME/build/hadoop-*-tools.jar; do
-  TOOL_PATH=${TOOL_PATH}:$f;
-done
-
 # add user-specified CLASSPATH last
 if [ "$HADOOP_CLASSPATH" != "" ]; then
   CLASSPATH=${CLASSPATH}:${HADOOP_CLASSPATH}
@@ -189,7 +179,6 @@
 if $cygwin; then
   HADOOP_CORE_HOME=`cygpath -w "$HADOOP_CORE_HOME"`
   HADOOP_LOG_DIR=`cygpath -w "$HADOOP_LOG_DIR"`
-  TOOL_PATH=`cygpath -p -w "$TOOL_PATH"`
 fi
 # setup 'java.library.path' for native-hadoop code if necessary
 JAVA_LIBRARY_PATH=''
@@ -228,12 +217,12 @@
 if [ "$HADOOP_HDFS_HOME" = "" ]; then
   if [ -d "${HADOOP_HOME}/hdfs" ]; then
     HADOOP_HDFS_HOME=$HADOOP_HOME/hdfs
-    echo Found HDFS installed at $HADOOP_HDFS_HOME
+    #echo Found HDFS installed at $HADOOP_HDFS_HOME
   fi
 fi
 
 if [ -d "${HADOOP_HDFS_HOME}" ]; then
-  for f in $HADOOP_HDFS_HOME/hadoop-*-hdfs.jar; do
+  for f in $HADOOP_HDFS_HOME/hadoop-hdfs-*.jar; do
     CLASSPATH=${CLASSPATH}:$f;
   done
 
@@ -247,14 +236,51 @@
   fi
 fi
 
+# cygwin path translation
+if $cygwin; then
+  HADOOP_HDFS_HOME=`cygpath -w "$HADOOP_HDFS_HOME"`
+fi
+
 # set mapred home if mapred is present
 if [ "$HADOOP_MAPRED_HOME" = "" ]; then
   if [ -d "${HADOOP_HOME}/mapred" ]; then
     HADOOP_MAPRED_HOME=$HADOOP_HOME/mapred
-    echo Found MAPRED installed at $HADOOP_MAPRED_HOME
+    #echo Found MAPRED installed at $HADOOP_MAPRED_HOME
   fi
 fi
 
+if [ -d "${HADOOP_MAPRED_HOME}" ]; then
+  for f in $HADOOP_MAPRED_HOME/hadoop-mapred-*.jar; do
+    CLASSPATH=${CLASSPATH}:$f
+  done
+
+  for f in $HADOOP_MAPRED_HOME/lib/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f
+  done
+
+  if [ -d "$HADOOP_MAPRED_HOME/build/classes" ]; then
+    CLASSPATH=${CLASSPATH}:$HADOOP_MAPRED_HOME/build/classes
+  fi
+
+  if [ -d "$HADOOP_MAPRED_HOME/build/tools" ]; then
+    CLASSPATH=${CLASSPATH}:$HADOOP_MAPRED_HOME/build/tools
+  fi
+
+  for f in $HADOOP_MAPRED_HOME/hadoop-mapred-tools-*.jar; do
+    TOOL_PATH=${TOOL_PATH}:$f;
+  done
+  for f in $HADOOP_MAPRED_HOME/build/hadoop-mapred-tools-*.jar; do
+    TOOL_PATH=${TOOL_PATH}:$f;
+  done
+fi
+
+# cygwin path translation
+if $cygwin; then
+  HADOOP_MAPRED_HOME=`cygpath -w "$HADOOP_MAPRED_HOME"`
+  TOOL_PATH=`cygpath -p -w "$TOOL_PATH"`
+fi
+
+
 # TODO:remove this when dir structure is changed
-export HADOOP_HDFS_HOME=$HADOOP_HOME
-export HADOOP_MAPRED_HOME=$HADOOP_HOME
+#export HADOOP_HDFS_HOME=$HADOOP_HOME
+#export HADOOP_MAPRED_HOME=$HADOOP_HOME

Modified: hadoop/common/branches/HADOOP-6194/bin/hdfs
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/bin/hdfs?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/bin/hdfs (original)
+++ hadoop/common/branches/HADOOP-6194/bin/hdfs Wed Aug 26 16:56:55 2009
@@ -94,7 +94,7 @@
 if [ -d "$HADOOP_HDFS_HOME/webapps" ]; then
   CLASSPATH=${CLASSPATH}:$HADOOP_HDFS_HOME
 fi
-for f in $HADOOP_HDFS_HOME/hadoop-*-hdfs.jar; do
+for f in $HADOOP_HDFS_HOME/hadoop-hdfs-*.jar; do
   CLASSPATH=${CLASSPATH}:$f;
 done
 

Modified: hadoop/common/branches/HADOOP-6194/bin/mapred
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/bin/mapred?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/bin/mapred (original)
+++ hadoop/common/branches/HADOOP-6194/bin/mapred Wed Aug 26 16:56:55 2009
@@ -84,7 +84,7 @@
 if [ -d "$HADOOP_MAPRED_HOME/webapps" ]; then
   CLASSPATH=${CLASSPATH}:$HADOOP_MAPRED_HOME
 fi
-for f in $HADOOP_MAPRED_HOME/hadoop-*-mapred.jar; do
+for f in $HADOOP_MAPRED_HOME/hadoop-mapred-*.jar; do
   CLASSPATH=${CLASSPATH}:$f;
 done
 

Modified: hadoop/common/branches/HADOOP-6194/bin/rcc
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/bin/rcc?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/bin/rcc (original)
+++ hadoop/common/branches/HADOOP-6194/bin/rcc Wed Aug 26 16:56:55 2009
@@ -72,7 +72,7 @@
 if [ -d "$HADOOP_HOME/webapps" ]; then
   CLASSPATH=${CLASSPATH}:$HADOOP_HOME
 fi
-for f in $HADOOP_HOME/hadoop-*-core.jar; do
+for f in $HADOOP_HOME/hadoop-core-*.jar; do
   CLASSPATH=${CLASSPATH}:$f;
 done
 

Modified: hadoop/common/branches/HADOOP-6194/ivy/hadoop-core.pom
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/ivy/hadoop-core.pom?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/ivy/hadoop-core.pom (original)
+++ hadoop/common/branches/HADOOP-6194/ivy/hadoop-core.pom Wed Aug 26 16:56:55 2009
@@ -36,22 +36,6 @@
   </licenses>
   <dependencies>
 
-    <!-- Avro-->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>avro</artifactId>
-      <version>${avro.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-      <version>${jackson-mapper-asl.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.thoughtworks.paranamer</groupId>
-      <artifactId>paranamer</artifactId>
-      <version>${paranamer.version}</version>
-    </dependency>
 
     <!-- always include commons-logging and log4J -->
     <dependency>
@@ -132,7 +116,7 @@
     <dependency>
       <groupId>commons-httpclient</groupId>
       <artifactId>commons-httpclient</artifactId>
-      <version>${commons-httpclient.version}</version>
+      <version>3.1</version>
       <scope>optional</scope>
       <exclusions>
         <exclusion>
@@ -148,15 +132,15 @@
     <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
-      <version>${commons-codec.version}</version>
+      <version>1.3</version>
       <scope>optional</scope>
     </dependency>
 
-    <!--CLI is needed to scan the command line -->
+    <!--CLI is needed to scan the command line, but only the 1.0 branch is released -->
     <dependency>
       <groupId>commons-cli</groupId>
       <artifactId>commons-cli</artifactId>
-      <version>${commons-cli.version}</version>
+      <version>2.0-20070823</version>
       <scope>optional</scope>
     </dependency>
 
@@ -165,7 +149,7 @@
     <dependency>
       <groupId>commons-net</groupId>
       <artifactId>commons-net</artifactId>
-      <version>${commons-net.version}</version>
+      <version>1.4.1</version>
       <scope>optional</scope>
     </dependency>
 
@@ -248,14 +232,16 @@
 
     <!--Kosmos filesystem
     http://kosmosfs.sourceforge.net/
+    This is not in the central repository
+    -->
+    <!--
+        <dependency>
+          <groupId>org.kosmix</groupId>
+          <artifactId>kfs</artifactId>
+          <version>0.1</version>
+          <scope>optional</scope>
+        </dependency>
     -->
-    <dependency>
-      <groupId>net.sf.kosmosfs</groupId>
-      <artifactId>kfs</artifactId>
-      <version>${kfs.version}</version>
-      <scope>optional</scope>
-    </dependency>
-
 
     <!--
      http://xmlenc.sourceforge.net/
@@ -264,7 +250,7 @@
     <dependency>
       <groupId>xmlenc</groupId>
       <artifactId>xmlenc</artifactId>
-      <version>${xmlenc.version}</version>
+      <version>0.52</version>
       <scope>optional</scope>
     </dependency>
   </dependencies>

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/conf/Configuration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/conf/Configuration.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/conf/Configuration.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/conf/Configuration.java Wed Aug 26 16:56:55 2009
@@ -28,6 +28,7 @@
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.Reader;
+import java.io.Writer;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -62,6 +63,8 @@
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerator;
 import org.w3c.dom.DOMException;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
@@ -155,7 +158,7 @@
   private boolean loadDefaults = true;
   
   /**
-   * Configurtion objects
+   * Configuration objects
    */
   private static final WeakHashMap<Configuration,Object> REGISTRY = 
     new WeakHashMap<Configuration,Object>();
@@ -167,6 +170,18 @@
   private static final ArrayList<String> defaultResources = 
     new ArrayList<String>();
   
+  /**
+   * Flag to indicate if the storage of resource which updates a key needs 
+   * to be stored for each key
+   */
+  private boolean storeResource;
+  
+  /**
+   * Stores the mapping of key to the resource which modifies or loads 
+   * the key most recently
+   */
+  private HashMap<String, String> updatingResource;
+  
   static{
     //print deprecation warning if hadoop-site.xml is found in classpath
     ClassLoader cL = Thread.currentThread().getContextClassLoader();
@@ -214,6 +229,23 @@
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
+    this.storeResource = false;
+  }
+  
+  /**
+   * A new configuration with the same settings and additional facility for
+   * storage of resource to each key which loads or updates 
+   * the key most recently
+   * @param other the configuration from which to clone settings
+   * @param storeResource flag to indicate if the storage of resource to 
+   * each key is to be stored
+   */
+  private Configuration(Configuration other, boolean storeResource) {
+    this(other);
+    this.storeResource = storeResource;
+    if (storeResource) {
+      updatingResource = new HashMap<String, String>();
+    }
   }
   
   /** 
@@ -1081,8 +1113,14 @@
     if (properties == null) {
       properties = new Properties();
       loadResources(properties, resources, quietmode);
-      if (overlay!= null)
+      if (overlay!= null) {
         properties.putAll(overlay);
+        if (storeResource) {
+          for (Map.Entry<Object,Object> item: overlay.entrySet()) {
+            updatingResource.put((String) item.getKey(), "Unknown");
+          }
+        }
+      }
     }
     return properties;
   }
@@ -1251,6 +1289,9 @@
         if (attr != null && value != null) {
           if (!finalParameters.contains(attr)) {
             properties.setProperty(attr, value);
+            if (storeResource) {
+              updatingResource.put(attr, name.toString());
+            }
             if (finalParameter)
               finalParameters.add(attr);
           } else {
@@ -1323,6 +1364,43 @@
   }
 
   /**
+   *  Writes out all the parameters and their properties (final and resource) to
+   *  the given {@link Writer}
+   *  The format of the output would be 
+   *  { "properties" : [ {key1,value1,key1.isFinal,key1.resource}, {key2,value2,
+   *  key2.isFinal,key2.resource}... ] } 
+   *  It does not output the parameters of the configuration object which is 
+   *  loaded from an input stream.
+   * @param out the Writer to write to
+   * @throws IOException
+   */
+  public static void dumpConfiguration(Configuration conf, 
+      Writer out) throws IOException {
+    Configuration config = new Configuration(conf,true);
+    config.reloadConfiguration();
+    JsonFactory dumpFactory = new JsonFactory();
+    JsonGenerator dumpGenerator = dumpFactory.createJsonGenerator(out);
+    dumpGenerator.writeStartObject();
+    dumpGenerator.writeFieldName("properties");
+    dumpGenerator.writeStartArray();
+    dumpGenerator.flush();
+    for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
+      dumpGenerator.writeStartObject();
+      dumpGenerator.writeStringField("key", (String) item.getKey());
+      dumpGenerator.writeStringField("value", 
+          config.get((String) item.getKey()));
+      dumpGenerator.writeBooleanField("isFinal",
+          config.finalParameters.contains(item.getKey()));
+      dumpGenerator.writeStringField("resource",
+          config.updatingResource.get(item.getKey()));
+      dumpGenerator.writeEndObject();
+    }
+    dumpGenerator.writeEndArray();
+    dumpGenerator.writeEndObject();
+    dumpGenerator.flush();
+  }
+  
+  /**
    * Get the {@link ClassLoader} for this job.
    * 
    * @return the correct class loader.

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FileSystem.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FileSystem.java Wed Aug 26 16:56:55 2009
@@ -731,25 +731,25 @@
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
    * 
-   * @param f
-   *          given path
+   * @param f given path
    * @return the statuses of the files/directories in the given patch
-   * @throws IOException
+   * @throws FileNotFoundException when the path does not exist;
+   *         IOException see specific implementation
    */
-  public abstract FileStatus[] listStatus(Path f) throws IOException;
+  public abstract FileStatus[] listStatus(Path f) throws FileNotFoundException, 
+                                                         IOException;
     
   /*
    * Filter files/directories in the given path using the user-supplied path
    * filter. Results are added to the given array <code>results</code>.
    */
   private void listStatus(ArrayList<FileStatus> results, Path f,
-      PathFilter filter) throws IOException {
+      PathFilter filter) throws FileNotFoundException, IOException {
     FileStatus listing[] = listStatus(f);
-    if (listing != null) {
-      for (int i = 0; i < listing.length; i++) {
-        if (filter.accept(listing[i].getPath())) {
-          results.add(listing[i]);
-        }
+
+    for (int i = 0; i < listing.length; i++) {
+      if (filter.accept(listing[i].getPath())) {
+        results.add(listing[i]);
       }
     }
   }
@@ -764,10 +764,11 @@
    *          the user-supplied path filter
    * @return an array of FileStatus objects for the files under the given path
    *         after applying the filter
-   * @throws IOException
-   *           if encounter any problem while fetching the status
+   * @throws FileNotFoundException when the path does not exist;
+   *         IOException see specific implementation   
    */
-  public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
+  public FileStatus[] listStatus(Path f, PathFilter filter) 
+                                   throws FileNotFoundException, IOException {
     ArrayList<FileStatus> results = new ArrayList<FileStatus>();
     listStatus(results, f, filter);
     return results.toArray(new FileStatus[results.size()]);
@@ -781,10 +782,11 @@
    *          a list of paths
    * @return a list of statuses for the files under the given paths after
    *         applying the filter default Path filter
-   * @exception IOException
+   * @throws FileNotFoundException when the path does not exist;
+   *         IOException see specific implementation
    */
   public FileStatus[] listStatus(Path[] files)
-      throws IOException {
+      throws FileNotFoundException, IOException {
     return listStatus(files, DEFAULT_FILTER);
   }
 
@@ -798,10 +800,11 @@
    *          the user-supplied path filter
    * @return a list of statuses for the files under the given paths after
    *         applying the filter
-   * @exception IOException
+   * @throws FileNotFoundException when the path does not exist;
+   *         IOException see specific implementation
    */
   public FileStatus[] listStatus(Path[] files, PathFilter filter)
-      throws IOException {
+      throws FileNotFoundException, IOException {
     ArrayList<FileStatus> results = new ArrayList<FileStatus>();
     for (int i = 0; i < files.length; i++) {
       listStatus(results, files[i], filter);

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FsShell.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FsShell.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/FsShell.java Wed Aug 26 16:56:55 2009
@@ -532,17 +532,18 @@
       return;
     }
     FileStatus items[] = srcFs.listStatus(src);
-    if (items == null) {
+    try {
+      items = srcFs.listStatus(src);
+    } catch (FileNotFoundException fnfe) {
       throw new IOException("Could not get listing for " + src);
-    } else {
+    }
 
-      for (int i = 0; i < items.length; i++) {
-        if (!items[i].isDir()) {
-          setFileReplication(items[i].getPath(), srcFs, newRep, waitingList);
-        } else if (recursive) {
-          setReplication(newRep, srcFs, items[i].getPath(), recursive, 
-                         waitingList);
-        }
+    for (int i = 0; i < items.length; i++) {
+      if (!items[i].isDir()) {
+        setFileReplication(items[i].getPath(), srcFs, newRep, waitingList);
+      } else if (recursive) {
+        setReplication(newRep, srcFs, items[i].getPath(), recursive, 
+                       waitingList);
       }
     }
   }
@@ -706,7 +707,11 @@
         statusToPrint = globStatus;
       } else {
         Path statPaths[] = FileUtil.stat2Paths(globStatus, srcPath);
-        statusToPrint = srcFs.listStatus(statPaths);
+        try {
+          statusToPrint = srcFs.listStatus(statPaths);
+        } catch(FileNotFoundException fnfe) {
+          statusToPrint = null;
+        }
       }
       if ((statusToPrint == null) || ((statusToPrint.length == 0) &&
                                       (!srcFs.exists(srcPath)))){
@@ -1234,11 +1239,10 @@
     Path path = src.getPath();
     try {
       FileStatus[] files = srcFs.listStatus(path);
-      if ( files == null ) {
-        System.err.println(cmd + 
-                           ": could not get listing for '" + path + "'");
-      }
+
       return files;
+    } catch(FileNotFoundException fnfe) {
+      System.err.println(cmd + ": could not get listing for '" + path + "'");
     } catch (IOException e) {
       System.err.println(cmd + 
                          ": could not get get listing for '" + path + "' : " +

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java Wed Aug 26 16:56:55 2009
@@ -285,7 +285,7 @@
     FileStatus[] results;
 
     if (!localf.exists()) {
-      return null;
+      throw new FileNotFoundException("File " + f + " does not exist.");
     }
     if (localf.isFile()) {
       return new FileStatus[] {

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/Trash.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/Trash.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/Trash.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/Trash.java Wed Aug 26 16:56:55 2009
@@ -170,10 +170,14 @@
 
   /** Delete old checkpoints. */
   public void expunge() throws IOException {
-    FileStatus[] dirs = fs.listStatus(trash);            // scan trash sub-directories
-    if( dirs == null){
+    FileStatus[] dirs = null;
+    
+    try {
+      dirs = fs.listStatus(trash);            // scan trash sub-directories
+    } catch (FileNotFoundException fnfe) {
       return;
     }
+
     long now = System.currentTimeMillis();
     for (int i = 0; i < dirs.length; i++) {
       Path path = dirs[i].getPath();
@@ -253,9 +257,6 @@
               continue;
             }
 
-            if (homes == null)
-              continue;
-
             for (FileStatus home : homes) {         // dump each trash
               if (!home.isDir())
                 continue;

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java Wed Aug 26 16:56:55 2009
@@ -142,6 +142,9 @@
         Path absolute = makeAbsolute(path);
         String srep = absolute.toUri().getPath();
 
+        if(!kfsImpl.exists(srep))
+          throw new FileNotFoundException("File " + path + " does not exist.");
+
         if (kfsImpl.isFile(srep))
                 return new FileStatus[] { getFileStatus(path) } ;
 
@@ -249,15 +252,13 @@
         return kfsImpl.remove(srep) == 0;
 
       FileStatus[] dirEntries = listStatus(absolute);
-      if ((!recursive) && (dirEntries != null) && 
-            (dirEntries.length != 0)) {
+      if (!recursive && (dirEntries.length != 0)) {
         throw new IOException("Directory " + path.toString() + 
         " is not empty.");
       }
-      if (dirEntries != null) {
-        for (int i = 0; i < dirEntries.length; i++) {
-          delete(new Path(absolute, dirEntries[i].getPath()), recursive);
-        }
+
+      for (int i = 0; i < dirEntries.length; i++) {
+        delete(new Path(absolute, dirEntries[i].getPath()), recursive);
       }
       return kfsImpl.rmdir(srep) == 0;
     }

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java Wed Aug 26 16:56:55 2009
@@ -178,7 +178,7 @@
     Path absolutePath = makeAbsolute(f);
     INode inode = store.retrieveINode(absolutePath);
     if (inode == null) {
-      return null;
+      throw new FileNotFoundException("File " + f + " does not exist.");
     }
     if (inode.isFile()) {
       return new FileStatus[] {
@@ -303,10 +303,13 @@
        store.deleteBlock(block);
      }
    } else {
-     FileStatus[] contents = listStatus(absolutePath);
-     if (contents == null) {
+     FileStatus[] contents = null; 
+     try {
+       contents = listStatus(absolutePath);
+     } catch(FileNotFoundException fnfe) {
        return false;
      }
+
      if ((contents.length !=0) && (!recursive)) {
        throw new IOException("Directory " + path.toString() 
            + " is not empty.");

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java Wed Aug 26 16:56:55 2009
@@ -456,7 +456,7 @@
     
     if (status.isEmpty() &&
         store.retrieveMetadata(key + FOLDER_SUFFIX) == null) {
-      return null;
+      throw new FileNotFoundException("File " + f + " does not exist.");
     }
     
     return status.toArray(new FileStatus[status.size()]);

Modified: hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/conf/TestConfiguration.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/conf/TestConfiguration.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/conf/TestConfiguration.java Wed Aug 26 16:56:55 2009
@@ -21,12 +21,15 @@
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.StringWriter;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Random;
 
 import junit.framework.TestCase;
 
 import org.apache.hadoop.fs.Path;
+import org.codehaus.jackson.map.ObjectMapper;
 
 
 public class TestConfiguration extends TestCase {
@@ -408,6 +411,182 @@
     assertTrue(other.getClassLoader() instanceof Fake_ClassLoader);
   }
   
+  static class JsonConfiguration {
+    JsonProperty[] properties;
+
+    public JsonProperty[] getProperties() {
+      return properties;
+    }
+
+    public void setProperties(JsonProperty[] properties) {
+      this.properties = properties;
+    }
+  }
+  
+  static class JsonProperty {
+    String key;
+    public String getKey() {
+      return key;
+    }
+    public void setKey(String key) {
+      this.key = key;
+    }
+    public String getValue() {
+      return value;
+    }
+    public void setValue(String value) {
+      this.value = value;
+    }
+    public boolean getIsFinal() {
+      return isFinal;
+    }
+    public void setIsFinal(boolean isFinal) {
+      this.isFinal = isFinal;
+    }
+    public String getResource() {
+      return resource;
+    }
+    public void setResource(String resource) {
+      this.resource = resource;
+    }
+    String value;
+    boolean isFinal;
+    String resource;
+  }
+  
+  public void testDumpConfiguration () throws IOException {
+    StringWriter outWriter = new StringWriter();
+    Configuration.dumpConfiguration(conf, outWriter);
+    String jsonStr = outWriter.toString();
+    ObjectMapper mapper = new ObjectMapper();
+    JsonConfiguration jconf = 
+      mapper.readValue(jsonStr, JsonConfiguration.class);
+    int defaultLength = jconf.getProperties().length;
+    
+    // add 3 keys to the existing configuration properties
+    out=new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendProperty("test.key1", "value1");
+    appendProperty("test.key2", "value2",true);
+    appendProperty("test.key3", "value3");
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    out.close();
+    
+    outWriter = new StringWriter();
+    Configuration.dumpConfiguration(conf, outWriter);
+    jsonStr = outWriter.toString();
+    mapper = new ObjectMapper();
+    jconf = mapper.readValue(jsonStr, JsonConfiguration.class);
+    int length = jconf.getProperties().length;
+    // check for consistency in the number of properties parsed in Json format.
+    assertEquals(length, defaultLength+3);
+    
+    //change few keys in another resource file
+    out=new BufferedWriter(new FileWriter(CONFIG2));
+    startConfig();
+    appendProperty("test.key1", "newValue1");
+    appendProperty("test.key2", "newValue2");
+    endConfig();
+    Path fileResource1 = new Path(CONFIG2);
+    conf.addResource(fileResource1);
+    out.close();
+    
+    outWriter = new StringWriter();
+    Configuration.dumpConfiguration(conf, outWriter);
+    jsonStr = outWriter.toString();
+    mapper = new ObjectMapper();
+    jconf = mapper.readValue(jsonStr, JsonConfiguration.class);
+    
+    // put the keys and their corresponding attributes into a hashmap for their 
+    // efficient retrieval
+    HashMap<String,JsonProperty> confDump = new HashMap<String,JsonProperty>();
+    for(JsonProperty prop : jconf.getProperties()) {
+      confDump.put(prop.getKey(), prop);
+    }
+    // check if the value and resource of test.key1 is changed
+    assertEquals("newValue1", confDump.get("test.key1").getValue());
+    assertEquals(false, confDump.get("test.key1").getIsFinal());
+    assertEquals(fileResource1.toString(),
+        confDump.get("test.key1").getResource());
+    // check if final parameter test.key2 is not changed, since it is first 
+    // loaded as final parameter
+    assertEquals("value2", confDump.get("test.key2").getValue());
+    assertEquals(true, confDump.get("test.key2").getIsFinal());
+    assertEquals(fileResource.toString(),
+        confDump.get("test.key2").getResource());
+    // check for other keys which are not modified later
+    assertEquals("value3", confDump.get("test.key3").getValue());
+    assertEquals(false, confDump.get("test.key3").getIsFinal());
+    assertEquals(fileResource.toString(),
+        confDump.get("test.key3").getResource());
+    // check for resource to be "Unknown" for keys which are loaded using 'set' 
+    // and expansion of properties
+    conf.set("test.key4", "value4");
+    conf.set("test.key5", "value5");
+    conf.set("test.key6", "${test.key5}");
+    outWriter = new StringWriter();
+    Configuration.dumpConfiguration(conf, outWriter);
+    jsonStr = outWriter.toString();
+    mapper = new ObjectMapper();
+    jconf = mapper.readValue(jsonStr, JsonConfiguration.class);
+    confDump = new HashMap<String, JsonProperty>();
+    for(JsonProperty prop : jconf.getProperties()) {
+      confDump.put(prop.getKey(), prop);
+    }
+    assertEquals("value5",confDump.get("test.key6").getValue());
+    assertEquals("Unknown", confDump.get("test.key4").getResource());
+    outWriter.close();
+  }
+  
+  public void testDumpConfiguratioWithoutDefaults() throws IOException {
+    // check for case when default resources are not loaded
+    Configuration config = new Configuration(false);
+    StringWriter outWriter = new StringWriter();
+    Configuration.dumpConfiguration(config, outWriter);
+    String jsonStr = outWriter.toString();
+    ObjectMapper mapper = new ObjectMapper();
+    JsonConfiguration jconf = 
+      mapper.readValue(jsonStr, JsonConfiguration.class);
+    
+    //ensure that no properties are loaded.
+    assertEquals(0, jconf.getProperties().length);
+    
+    // add 2 keys
+    out=new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendProperty("test.key1", "value1");
+    appendProperty("test.key2", "value2",true);
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    config.addResource(fileResource);
+    out.close();
+    
+    outWriter = new StringWriter();
+    Configuration.dumpConfiguration(config, outWriter);
+    jsonStr = outWriter.toString();
+    mapper = new ObjectMapper();
+    jconf = mapper.readValue(jsonStr, JsonConfiguration.class);
+    
+    HashMap<String, JsonProperty>confDump = new HashMap<String, JsonProperty>();
+    for (JsonProperty prop : jconf.getProperties()) {
+      confDump.put(prop.getKey(), prop);
+    }
+    //ensure only 2 keys are loaded
+    assertEquals(2,jconf.getProperties().length);
+    //ensure the values are consistent
+    assertEquals(confDump.get("test.key1").getValue(),"value1");
+    assertEquals(confDump.get("test.key2").getValue(),"value2");
+    //check the final tag
+    assertEquals(false, confDump.get("test.key1").getIsFinal());
+    assertEquals(true, confDump.get("test.key2").getIsFinal());
+    //check the resource for each property
+    for (JsonProperty prop : jconf.getProperties()) {
+      assertEquals(fileResource.toString(),prop.getResource());
+    }
+  }
+  
   public static void main(String[] argv) throws Exception {
     junit.textui.TestRunner.main(new String[]{
       TestConfiguration.class.getName()

Modified: hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/FileSystemContractBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/FileSystemContractBaseTest.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/FileSystemContractBaseTest.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/FileSystemContractBaseTest.java Wed Aug 26 16:56:55 2009
@@ -162,8 +162,13 @@
     }
   }
   
-  public void testListStatusReturnsNullForNonExistentFile() throws Exception {
-    assertNull(fs.listStatus(path("/test/hadoop/file")));
+  public void testListStatusThrowsExceptionForNonExistentFile() throws Exception {
+    try {
+      fs.listStatus(path("/test/hadoop/file"));
+      fail("Should throw FileNotFoundException");
+    } catch (FileNotFoundException fnfe) {
+      // expected
+    }
   }
   
   public void testListStatus() throws Exception {

Modified: hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java Wed Aug 26 16:56:55 2009
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.fs.kfs;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -50,11 +51,12 @@
 
     public String[] readdir(String path) throws IOException {
         FileStatus[] p = localFS.listStatus(new Path(path));
-        String[] entries = null;
-
-        if (p == null) {
-            return null;
+        try {
+          p = localFS.listStatus(new Path(path));
+        } catch ( FileNotFoundException fnfe ) {
+          return null;
         }
+        String[] entries = null;
 
         entries = new String[p.length];
         for (int i = 0; i < p.length; i++)

Modified: hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java?rev=808123&r1=808122&r2=808123&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java Wed Aug 26 16:56:55 2009
@@ -545,16 +545,15 @@
    */
   private void initFileDirTables(Path path) throws IOException {
     FileStatus[] stats = fs.listStatus(path);
-    if (stats != null) { 
-      for (FileStatus stat : stats) {
-        if (stat.isDir()) {
-          dirs.add(stat.getPath().toString());
-          initFileDirTables(stat.getPath());
-        } else {
-          Path filePath = stat.getPath();
-          if (filePath.getName().startsWith(StructureGenerator.FILE_NAME_PREFIX)) {
-            files.add(filePath.toString());
-          }
+
+    for (FileStatus stat : stats) {
+      if (stat.isDir()) {
+        dirs.add(stat.getPath().toString());
+        initFileDirTables(stat.getPath());
+      } else {
+        Path filePath = stat.getPath();
+        if (filePath.getName().startsWith(StructureGenerator.FILE_NAME_PREFIX)) {
+          files.add(filePath.toString());
         }
       }
     }