Bläddra i källkod

HADOOP-4687 Moving src directories on branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/HADOOP-4687/core@776174 13f79535-47bb-0310-9956-ffa450edef68
Owen O'Malley 16 år sedan
incheckning
5128a9a453
100 ändrade filer med 21610 tillägg och 0 borttagningar
  1. 444 0
      src/java/core-default.xml
  2. 69 0
      src/java/org/apache/hadoop/HadoopVersionAnnotation.java
  3. 29 0
      src/java/org/apache/hadoop/conf/Configurable.java
  4. 1326 0
      src/java/org/apache/hadoop/conf/Configuration.java
  5. 46 0
      src/java/org/apache/hadoop/conf/Configured.java
  6. 23 0
      src/java/org/apache/hadoop/conf/package.html
  7. 879 0
      src/java/org/apache/hadoop/filecache/DistributedCache.java
  8. 241 0
      src/java/org/apache/hadoop/fs/BlockLocation.java
  9. 96 0
      src/java/org/apache/hadoop/fs/BufferedFSInputStream.java
  10. 35 0
      src/java/org/apache/hadoop/fs/ChecksumException.java
  11. 547 0
      src/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  12. 164 0
      src/java/org/apache/hadoop/fs/ContentSummary.java
  13. 193 0
      src/java/org/apache/hadoop/fs/DF.java
  14. 198 0
      src/java/org/apache/hadoop/fs/DU.java
  15. 62 0
      src/java/org/apache/hadoop/fs/FSDataInputStream.java
  16. 100 0
      src/java/org/apache/hadoop/fs/FSDataOutputStream.java
  17. 29 0
      src/java/org/apache/hadoop/fs/FSError.java
  18. 432 0
      src/java/org/apache/hadoop/fs/FSInputChecker.java
  19. 78 0
      src/java/org/apache/hadoop/fs/FSInputStream.java
  20. 176 0
      src/java/org/apache/hadoop/fs/FSOutputSummer.java
  21. 53 0
      src/java/org/apache/hadoop/fs/FileChecksum.java
  22. 252 0
      src/java/org/apache/hadoop/fs/FileStatus.java
  23. 1648 0
      src/java/org/apache/hadoop/fs/FileSystem.java
  24. 794 0
      src/java/org/apache/hadoop/fs/FileUtil.java
  25. 278 0
      src/java/org/apache/hadoop/fs/FilterFileSystem.java
  26. 1925 0
      src/java/org/apache/hadoop/fs/FsShell.java
  27. 315 0
      src/java/org/apache/hadoop/fs/FsShellPermissions.java
  28. 70 0
      src/java/org/apache/hadoop/fs/FsStatus.java
  29. 61 0
      src/java/org/apache/hadoop/fs/FsUrlConnection.java
  30. 47 0
      src/java/org/apache/hadoop/fs/FsUrlStreamHandler.java
  31. 78 0
      src/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java
  32. 166 0
      src/java/org/apache/hadoop/fs/GlobExpander.java
  33. 892 0
      src/java/org/apache/hadoop/fs/HarFileSystem.java
  34. 0 0
      src/java/org/apache/hadoop/fs/LengthFileChecksum.java
  35. 418 0
      src/java/org/apache/hadoop/fs/LocalDirAllocator.java
  36. 115 0
      src/java/org/apache/hadoop/fs/LocalFileSystem.java
  37. 113 0
      src/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java
  38. 298 0
      src/java/org/apache/hadoop/fs/Path.java
  39. 32 0
      src/java/org/apache/hadoop/fs/PathFilter.java
  40. 47 0
      src/java/org/apache/hadoop/fs/PositionedReadable.java
  41. 496 0
      src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  42. 41 0
      src/java/org/apache/hadoop/fs/Seekable.java
  43. 30 0
      src/java/org/apache/hadoop/fs/Syncable.java
  44. 291 0
      src/java/org/apache/hadoop/fs/Trash.java
  45. 38 0
      src/java/org/apache/hadoop/fs/ftp/FTPException.java
  46. 576 0
      src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
  47. 126 0
      src/java/org/apache/hadoop/fs/ftp/FTPInputStream.java
  48. 60 0
      src/java/org/apache/hadoop/fs/kfs/IFSImpl.java
  49. 151 0
      src/java/org/apache/hadoop/fs/kfs/KFSImpl.java
  50. 130 0
      src/java/org/apache/hadoop/fs/kfs/KFSInputStream.java
  51. 97 0
      src/java/org/apache/hadoop/fs/kfs/KFSOutputStream.java
  52. 340 0
      src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
  53. 98 0
      src/java/org/apache/hadoop/fs/kfs/package.html
  54. 23 0
      src/java/org/apache/hadoop/fs/package.html
  55. 61 0
      src/java/org/apache/hadoop/fs/permission/AccessControlException.java
  56. 67 0
      src/java/org/apache/hadoop/fs/permission/FsAction.java
  57. 232 0
      src/java/org/apache/hadoop/fs/permission/FsPermission.java
  58. 118 0
      src/java/org/apache/hadoop/fs/permission/PermissionStatus.java
  59. 47 0
      src/java/org/apache/hadoop/fs/s3/Block.java
  60. 63 0
      src/java/org/apache/hadoop/fs/s3/FileSystemStore.java
  61. 117 0
      src/java/org/apache/hadoop/fs/s3/INode.java
  62. 390 0
      src/java/org/apache/hadoop/fs/s3/Jets3tFileSystemStore.java
  63. 280 0
      src/java/org/apache/hadoop/fs/s3/MigrationTool.java
  64. 99 0
      src/java/org/apache/hadoop/fs/s3/S3Credentials.java
  65. 34 0
      src/java/org/apache/hadoop/fs/s3/S3Exception.java
  66. 361 0
      src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
  67. 31 0
      src/java/org/apache/hadoop/fs/s3/S3FileSystemException.java
  68. 211 0
      src/java/org/apache/hadoop/fs/s3/S3InputStream.java
  69. 231 0
      src/java/org/apache/hadoop/fs/s3/S3OutputStream.java
  70. 32 0
      src/java/org/apache/hadoop/fs/s3/VersionMismatchException.java
  71. 55 0
      src/java/org/apache/hadoop/fs/s3/package.html
  72. 54 0
      src/java/org/apache/hadoop/fs/s3native/FileMetadata.java
  73. 255 0
      src/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java
  74. 65 0
      src/java/org/apache/hadoop/fs/s3native/NativeFileSystemStore.java
  75. 578 0
      src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
  76. 59 0
      src/java/org/apache/hadoop/fs/s3native/PartialListing.java
  77. 32 0
      src/java/org/apache/hadoop/fs/s3native/package.html
  78. 86 0
      src/java/org/apache/hadoop/fs/shell/Command.java
  79. 75 0
      src/java/org/apache/hadoop/fs/shell/CommandFormat.java
  80. 28 0
      src/java/org/apache/hadoop/fs/shell/CommandUtils.java
  81. 77 0
      src/java/org/apache/hadoop/fs/shell/Count.java
  82. 40 0
      src/java/org/apache/hadoop/http/FilterContainer.java
  83. 29 0
      src/java/org/apache/hadoop/http/FilterInitializer.java
  84. 519 0
      src/java/org/apache/hadoop/http/HttpServer.java
  85. 207 0
      src/java/org/apache/hadoop/io/AbstractMapWritable.java
  86. 94 0
      src/java/org/apache/hadoop/io/ArrayFile.java
  87. 103 0
      src/java/org/apache/hadoop/io/ArrayWritable.java
  88. 76 0
      src/java/org/apache/hadoop/io/BinaryComparable.java
  89. 259 0
      src/java/org/apache/hadoop/io/BloomMapFile.java
  90. 111 0
      src/java/org/apache/hadoop/io/BooleanWritable.java
  91. 87 0
      src/java/org/apache/hadoop/io/ByteWritable.java
  92. 216 0
      src/java/org/apache/hadoop/io/BytesWritable.java
  93. 24 0
      src/java/org/apache/hadoop/io/Closeable.java
  94. 86 0
      src/java/org/apache/hadoop/io/CompressedWritable.java
  95. 91 0
      src/java/org/apache/hadoop/io/DataInputBuffer.java
  96. 108 0
      src/java/org/apache/hadoop/io/DataOutputBuffer.java
  97. 199 0
      src/java/org/apache/hadoop/io/DefaultStringifier.java
  98. 60 0
      src/java/org/apache/hadoop/io/DeprecatedUTF8.java
  99. 95 0
      src/java/org/apache/hadoop/io/DoubleWritable.java
  100. 202 0
      src/java/org/apache/hadoop/io/EnumSetWritable.java

+ 444 - 0
src/java/core-default.xml

@@ -0,0 +1,444 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Do not modify this file directly.  Instead, copy entries that you -->
+<!-- wish to modify from this file into core-site.xml and change them -->
+<!-- there.  If core-site.xml does not already exist, create it.      -->
+
+<configuration>
+
+<!--- global properties -->
+
+<property>
+  <name>hadoop.tmp.dir</name>
+  <value>/tmp/hadoop-${user.name}</value>
+  <description>A base for other temporary directories.</description>
+</property>
+
+<property>
+  <name>hadoop.native.lib</name>
+  <value>true</value>
+  <description>Should native hadoop libraries, if present, be used.</description>
+</property>
+
+<property>
+  <name>hadoop.http.filter.initializers</name>
+  <value></value>
+  <description>A comma separated list of class names. Each class in the list 
+  must extend org.apache.hadoop.http.FilterInitializer. The corresponding 
+  Filter will be initialized. Then, the Filter will be applied to all user 
+  facing jsp and servlet web pages.  The ordering of the list defines the 
+  ordering of the filters.</description>
+</property>
+
+<property>
+  <name>hadoop.security.authorization</name>
+  <value>false</value>
+  <description>Is service-level authorization enabled?</description>
+</property>
+
+<!--- logging properties -->
+
+<property>
+  <name>hadoop.logfile.size</name>
+  <value>10000000</value>
+  <description>The max size of each log file</description>
+</property>
+
+<property>
+  <name>hadoop.logfile.count</name>
+  <value>10</value>
+  <description>The max number of log files</description>
+</property>
+
+<!-- i/o properties -->
+<property>
+  <name>io.file.buffer.size</name>
+  <value>4096</value>
+  <description>The size of buffer for use in sequence 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>io.bytes.per.checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  io.file.buffer.size.</description>
+</property>
+
+<property>
+  <name>io.skip.checksum.errors</name>
+  <value>false</value>
+  <description>If true, when a checksum error is encountered while
+  reading a sequence file, entries are skipped, instead of throwing an
+  exception.</description>
+</property>
+
+<property>
+  <name>io.compression.codecs</name>
+  <value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value>
+  <description>A list of the compression codec classes that can be used 
+               for compression/decompression.</description>
+</property>
+
+<property>
+  <name>io.serializations</name>
+  <value>org.apache.hadoop.io.serializer.WritableSerialization</value>
+  <description>A list of serialization classes that can be used for
+  obtaining serializers and deserializers.</description>
+</property>
+
+<property>
+  <name>io.seqfile.local.dir</name>
+  <value>${hadoop.tmp.dir}/io/local</value>
+  <description>The local directory where sequence file stores intermediate
+  data files during merge.  May be a comma-separated list of
+  directories on different devices in order to spread disk i/o.
+  Directories that do not exist are ignored.
+  </description>
+</property>
+
+<!-- file system properties -->
+
+<property>
+  <name>fs.default.name</name>
+  <value>file:///</value>
+  <description>The name of the default file system.  A URI whose
+  scheme and authority determine the FileSystem implementation.  The
+  uri's scheme determines the config property (fs.SCHEME.impl) naming
+  the FileSystem implementation class.  The uri's authority is used to
+  determine the host, port, etc. for a filesystem.</description>
+</property>
+
+<property>
+  <name>fs.trash.interval</name>
+  <value>0</value>
+  <description>Number of minutes between trash checkpoints.
+  If zero, the trash feature is disabled.
+  </description>
+</property>
+
+<property>
+  <name>fs.file.impl</name>
+  <value>org.apache.hadoop.fs.LocalFileSystem</value>
+  <description>The FileSystem for file: uris.</description>
+</property>
+
+<property>
+  <name>fs.hdfs.impl</name>
+  <value>org.apache.hadoop.hdfs.DistributedFileSystem</value>
+  <description>The FileSystem for hdfs: uris.</description>
+</property>
+
+<property>
+  <name>fs.s3.impl</name>
+  <value>org.apache.hadoop.fs.s3.S3FileSystem</value>
+  <description>The FileSystem for s3: uris.</description>
+</property>
+
+<property>
+  <name>fs.s3n.impl</name>
+  <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
+  <description>The FileSystem for s3n: (Native S3) uris.</description>
+</property>
+
+<property>
+  <name>fs.kfs.impl</name>
+  <value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value>
+  <description>The FileSystem for kfs: uris.</description>
+</property>
+
+<property>
+  <name>fs.hftp.impl</name>
+  <value>org.apache.hadoop.hdfs.HftpFileSystem</value>
+</property>
+
+<property>
+  <name>fs.hsftp.impl</name>
+  <value>org.apache.hadoop.hdfs.HsftpFileSystem</value>
+</property>
+
+<property>
+  <name>fs.ftp.impl</name>
+  <value>org.apache.hadoop.fs.ftp.FTPFileSystem</value>
+  <description>The FileSystem for ftp: uris.</description>
+</property>
+
+<property>
+  <name>fs.ramfs.impl</name>
+  <value>org.apache.hadoop.fs.InMemoryFileSystem</value>
+  <description>The FileSystem for ramfs: uris.</description>
+</property>
+
+<property>
+  <name>fs.har.impl</name>
+  <value>org.apache.hadoop.fs.HarFileSystem</value>
+  <description>The filesystem for Hadoop archives. </description>
+</property>
+
+<property>
+  <name>fs.checkpoint.dir</name>
+  <value>${hadoop.tmp.dir}/dfs/namesecondary</value>
+  <description>Determines where on the local filesystem the DFS secondary
+      name node should store the temporary images to merge.
+      If this is a comma-delimited list of directories then the image is
+      replicated in all of the directories for redundancy.
+  </description>
+</property>
+
+<property>
+  <name>fs.checkpoint.edits.dir</name>
+  <value>${fs.checkpoint.dir}</value>
+  <description>Determines where on the local filesystem the DFS secondary
+      name node should store the temporary edits to merge.
+      If this is a comma-delimited list of directoires then teh edits is
+      replicated in all of the directoires for redundancy.
+      Default value is same as fs.checkpoint.dir
+  </description>
+</property>
+
+<property>
+  <name>fs.checkpoint.period</name>
+  <value>3600</value>
+  <description>The number of seconds between two periodic checkpoints.
+  </description>
+</property>
+
+<property>
+  <name>fs.checkpoint.size</name>
+  <value>67108864</value>
+  <description>The size of the current edit log (in bytes) that triggers
+       a periodic checkpoint even if the fs.checkpoint.period hasn't expired.
+  </description>
+</property>
+
+
+
+<property>
+  <name>fs.s3.block.size</name>
+  <value>67108864</value>
+  <description>Block size to use when writing files to S3.</description>
+</property>
+
+<property>
+  <name>fs.s3.buffer.dir</name>
+  <value>${hadoop.tmp.dir}/s3</value>
+  <description>Determines where on the local filesystem the S3 filesystem
+  should store files before sending them to S3
+  (or after retrieving them from S3).
+  </description>
+</property>
+
+<property>
+  <name>fs.s3.maxRetries</name>
+  <value>4</value>
+  <description>The maximum number of retries for reading or writing files to S3, 
+  before we signal failure to the application.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3.sleepTimeSeconds</name>
+  <value>10</value>
+  <description>The number of seconds to sleep between each S3 retry.
+  </description>
+</property>
+
+
+<property>
+  <name>local.cache.size</name>
+  <value>10737418240</value>
+  <description>The limit on the size of cache you want to keep, set by default
+  to 10GB. This will act as a soft limit on the cache directory for out of band data.
+  </description>
+</property>
+            
+<property>
+  <name>io.seqfile.compress.blocksize</name>
+  <value>1000000</value>
+  <description>The minimum block size for compression in block compressed 
+          SequenceFiles.
+  </description>
+</property>
+
+<property>
+  <name>io.seqfile.lazydecompress</name>
+  <value>true</value>
+  <description>Should values of block-compressed SequenceFiles be decompressed
+          only when necessary.
+  </description>
+</property>
+
+<property>
+  <name>io.seqfile.sorter.recordlimit</name>
+  <value>1000000</value>
+  <description>The limit on number of records to be kept in memory in a spill 
+          in SequenceFiles.Sorter
+  </description>
+</property>
+
+ <property>
+  <name>io.mapfile.bloom.size</name>
+  <value>1048576</value>
+  <description>The size of BloomFilter-s used in BloomMapFile. Each time this many
+  keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter).
+  Larger values minimize the number of filters, which slightly increases the performance,
+  but may waste too much space if the total number of keys is usually much smaller
+  than this number.
+  </description>
+</property>
+
+<property>
+  <name>io.mapfile.bloom.error.rate</name>
+  <value>0.005</value>
+  <description>The rate of false positives in BloomFilter-s used in BloomMapFile.
+  As this value decreases, the size of BloomFilter-s increases exponentially. This
+  value is the probability of encountering false positives (default is 0.5%).
+  </description>
+</property>
+
+<property>
+  <name>hadoop.util.hash.type</name>
+  <value>murmur</value>
+  <description>The default implementation of Hash. Currently this can take one of the
+  two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash.
+  </description>
+</property>
+
+
+<!-- ipc properties -->
+
+<property>
+  <name>ipc.client.idlethreshold</name>
+  <value>4000</value>
+  <description>Defines the threshold number of connections after which
+               connections will be inspected for idleness.
+  </description>
+</property>
+
+<property>
+  <name>ipc.client.kill.max</name>
+  <value>10</value>
+  <description>Defines the maximum number of clients to disconnect in one go.
+  </description>
+</property>
+
+<property>
+  <name>ipc.client.connection.maxidletime</name>
+  <value>10000</value>
+  <description>The maximum time in msec after which a client will bring down the
+               connection to the server.
+  </description>
+</property>
+
+<property>
+  <name>ipc.client.connect.max.retries</name>
+  <value>10</value>
+  <description>Indicates the number of retries a client will make to establish
+               a server connection.
+  </description>
+</property>
+
+<property>
+  <name>ipc.server.listen.queue.size</name>
+  <value>128</value>
+  <description>Indicates the length of the listen queue for servers accepting
+               client connections.
+  </description>
+</property>
+
+<property>
+  <name>ipc.server.tcpnodelay</name>
+  <value>false</value>
+  <description>Turn on/off Nagle's algorithm for the TCP socket connection on 
+  the server. Setting to true disables the algorithm and may decrease latency
+  with a cost of more/smaller packets. 
+  </description>
+</property>
+
+<property>
+  <name>ipc.client.tcpnodelay</name>
+  <value>false</value>
+  <description>Turn on/off Nagle's algorithm for the TCP socket connection on 
+  the client. Setting to true disables the algorithm and may decrease latency
+  with a cost of more/smaller packets. 
+  </description>
+</property>
+
+
+<!-- Web Interface Configuration -->
+
+<property>
+  <name>webinterface.private.actions</name>
+  <value>false</value>
+  <description> If set to true, the web interfaces of JT and NN may contain 
+                actions, such as kill job, delete file, etc., that should 
+                not be exposed to public. Enable this option if the interfaces 
+                are only reachable by those who have the right authorization.
+  </description>
+</property>
+
+<!-- Proxy Configuration -->
+
+<property>
+  <name>hadoop.rpc.socket.factory.class.default</name>
+  <value>org.apache.hadoop.net.StandardSocketFactory</value>
+  <description> Default SocketFactory to use. This parameter is expected to be
+    formatted as "package.FactoryClassName".
+  </description>
+</property>
+
+<property>
+  <name>hadoop.rpc.socket.factory.class.ClientProtocol</name>
+  <value></value>
+  <description> SocketFactory to use to connect to a DFS. If null or empty, use
+    hadoop.rpc.socket.class.default. This socket factory is also used by
+    DFSClient to create sockets to DataNodes.
+  </description>
+</property>
+
+
+
+<property>
+  <name>hadoop.socks.server</name>
+  <value></value>
+  <description> Address (host:port) of the SOCKS server to be used by the
+    SocksSocketFactory.
+  </description>
+</property>
+
+<!-- Rack Configuration -->
+
+<property>
+  <name>topology.node.switch.mapping.impl</name>
+  <value>org.apache.hadoop.net.ScriptBasedMapping</value>
+  <description> The default implementation of the DNSToSwitchMapping. It
+    invokes a script specified in topology.script.file.name to resolve
+    node names. If the value for topology.script.file.name is not set, the
+    default value of DEFAULT_RACK is returned for all node names.
+  </description>
+</property>
+
+<property>
+  <name>topology.script.file.name</name>
+  <value></value>
+  <description> The script name that should be invoked to resolve DNS names to
+    NetworkTopology names. Example: the script would take host.foo.bar as an
+    argument, and return /rack1 as the output.
+  </description>
+</property>
+
+<property>
+  <name>topology.script.number.args</name>
+  <value>100</value>
+  <description> The max number of args that the script configured with 
+    topology.script.file.name should be run with. Each arg is an
+    IP address.
+  </description>
+</property>
+
+
+
+</configuration>

+ 69 - 0
src/java/org/apache/hadoop/HadoopVersionAnnotation.java

@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop;
+
+import java.lang.annotation.*;
+
+/**
+ * A package attribute that captures the version of Hadoop that was compiled.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.PACKAGE)
+public @interface HadoopVersionAnnotation {
+ 
+  /**
+   * Get the Hadoop version
+   * @return the version string "0.6.3-dev"
+   */
+  String version();
+  
+  /**
+   * Get the username that compiled Hadoop.
+   */
+  String user();
+  
+  /**
+   * Get the date when Hadoop was compiled.
+   * @return the date in unix 'date' format
+   */
+  String date();
+    
+  /**
+   * Get the url for the subversion repository.
+   */
+  String url();
+  
+  /**
+   * Get the subversion revision.
+   * @return the revision number as a string (eg. "451451")
+   */
+  String revision();
+
+  /**
+   * Get the branch from which this was compiled.
+   * @return The branch name, e.g. "trunk" or "branches/branch-0.20"
+   */
+  String branch();
+
+  /**
+   * Get a checksum of the source files from which
+   * Hadoop was compiled.
+   * @return a string that uniquely identifies the source
+   **/
+  String srcChecksum();    
+}

+ 29 - 0
src/java/org/apache/hadoop/conf/Configurable.java

@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+/** Something that may be configured with a {@link Configuration}. */
+public interface Configurable {
+
+  /** Set the configuration to be used by this object. */
+  void setConf(Configuration conf);
+
+  /** Return the configuration used by this object. */
+  Configuration getConf();
+}

+ 1326 - 0
src/java/org/apache/hadoop/conf/Configuration.java

@@ -0,0 +1,1326 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+import java.io.BufferedInputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.WeakHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.w3c.dom.DOMException;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Text;
+import org.xml.sax.SAXException;
+
+/** 
+ * Provides access to configuration parameters.
+ *
+ * <h4 id="Resources">Resources</h4>
+ *
+ * <p>Configurations are specified by resources. A resource contains a set of
+ * name/value pairs as XML data. Each resource is named by either a 
+ * <code>String</code> or by a {@link Path}. If named by a <code>String</code>, 
+ * then the classpath is examined for a file with that name.  If named by a 
+ * <code>Path</code>, then the local filesystem is examined directly, without 
+ * referring to the classpath.
+ *
+ * <p>Unless explicitly turned off, Hadoop by default specifies two 
+ * resources, loaded in-order from the classpath: <ol>
+ * <li><tt><a href="{@docRoot}/../core-default.html">core-default.xml</a>
+ * </tt>: Read-only defaults for hadoop.</li>
+ * <li><tt>core-site.xml</tt>: Site-specific configuration for a given hadoop
+ * installation.</li>
+ * </ol>
+ * Applications may add additional resources, which are loaded
+ * subsequent to these resources in the order they are added.
+ * 
+ * <h4 id="FinalParams">Final Parameters</h4>
+ *
+ * <p>Configuration parameters may be declared <i>final</i>. 
+ * Once a resource declares a value final, no subsequently-loaded 
+ * resource can alter that value.  
+ * For example, one might define a final parameter with:
+ * <tt><pre>
+ *  &lt;property&gt;
+ *    &lt;name&gt;dfs.client.buffer.dir&lt;/name&gt;
+ *    &lt;value&gt;/tmp/hadoop/dfs/client&lt;/value&gt;
+ *    <b>&lt;final&gt;true&lt;/final&gt;</b>
+ *  &lt;/property&gt;</pre></tt>
+ *
+ * Administrators typically define parameters as final in 
+ * <tt>core-site.xml</tt> for values that user applications may not alter.
+ *
+ * <h4 id="VariableExpansion">Variable Expansion</h4>
+ *
+ * <p>Value strings are first processed for <i>variable expansion</i>. The
+ * available properties are:<ol>
+ * <li>Other properties defined in this Configuration; and, if a name is
+ * undefined here,</li>
+ * <li>Properties in {@link System#getProperties()}.</li>
+ * </ol>
+ *
+ * <p>For example, if a configuration resource contains the following property
+ * definitions: 
+ * <tt><pre>
+ *  &lt;property&gt;
+ *    &lt;name&gt;basedir&lt;/name&gt;
+ *    &lt;value&gt;/user/${<i>user.name</i>}&lt;/value&gt;
+ *  &lt;/property&gt;
+ *  
+ *  &lt;property&gt;
+ *    &lt;name&gt;tempdir&lt;/name&gt;
+ *    &lt;value&gt;${<i>basedir</i>}/tmp&lt;/value&gt;
+ *  &lt;/property&gt;</pre></tt>
+ *
+ * When <tt>conf.get("tempdir")</tt> is called, then <tt>${<i>basedir</i>}</tt>
+ * will be resolved to another property in this Configuration, while
+ * <tt>${<i>user.name</i>}</tt> would then ordinarily be resolved to the value
+ * of the System property with that name.
+ */
+public class Configuration implements Iterable<Map.Entry<String,String>>,
+                                      Writable {
+  private static final Log LOG =
+    LogFactory.getLog(Configuration.class);
+
+  private boolean quietmode = true;
+  
+  /**
+   * List of configuration resources.
+   */
+  private ArrayList<Object> resources = new ArrayList<Object>();
+
+  /**
+   * List of configuration parameters marked <b>final</b>. 
+   */
+  private Set<String> finalParameters = new HashSet<String>();
+  
+  private boolean loadDefaults = true;
+  
+  /**
+   * Configurtion objects
+   */
+  private static final WeakHashMap<Configuration,Object> REGISTRY = 
+    new WeakHashMap<Configuration,Object>();
+  
+  /**
+   * List of default Resources. Resources are loaded in the order of the list 
+   * entries
+   */
+  private static final ArrayList<String> defaultResources = 
+    new ArrayList<String>();
+  
+  static{
+    //print deprecation warning if hadoop-site.xml is found in classpath
+    ClassLoader cL = Thread.currentThread().getContextClassLoader();
+    if (cL == null) {
+      cL = Configuration.class.getClassLoader();
+    }
+    if(cL.getResource("hadoop-site.xml")!=null) {
+      LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " +
+          "Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, "
+          + "mapred-site.xml and hdfs-site.xml to override properties of " +
+          "core-default.xml, mapred-default.xml and hdfs-default.xml " +
+          "respectively");
+    }
+    addDefaultResource("core-default.xml");
+    addDefaultResource("core-site.xml");
+  }
+  
+  private Properties properties;
+  private Properties overlay;
+  private ClassLoader classLoader;
+  {
+    classLoader = Thread.currentThread().getContextClassLoader();
+    if (classLoader == null) {
+      classLoader = Configuration.class.getClassLoader();
+    }
+  }
+  
+  /** A new configuration. */
+  public Configuration() {
+    this(true);
+  }
+
+  /** A new configuration where the behavior of reading from the default 
+   * resources can be turned off.
+   * 
+   * If the parameter {@code loadDefaults} is false, the new instance
+   * will not load resources from the default files. 
+   * @param loadDefaults specifies whether to load from the default files
+   */
+  public Configuration(boolean loadDefaults) {
+    this.loadDefaults = loadDefaults;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(StringUtils.stringifyException(new IOException("config()")));
+    }
+    synchronized(Configuration.class) {
+      REGISTRY.put(this, null);
+    }
+  }
+  
+  /** 
+   * A new configuration with the same settings cloned from another.
+   * 
+   * @param other the configuration from which to clone settings.
+   */
+  @SuppressWarnings("unchecked")
+  public Configuration(Configuration other) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(StringUtils.stringifyException
+                (new IOException("config(config)")));
+    }
+   
+   this.resources = (ArrayList)other.resources.clone();
+   synchronized(other) {
+     if (other.properties != null) {
+       this.properties = (Properties)other.properties.clone();
+     }
+
+     if (other.overlay!=null) {
+       this.overlay = (Properties)other.overlay.clone();
+     }
+   }
+   
+    this.finalParameters = new HashSet<String>(other.finalParameters);
+    synchronized(Configuration.class) {
+      REGISTRY.put(this, null);
+    }
+  }
+  
+  /**
+   * Add a default resource. Resources are loaded in the order of the resources 
+   * added.
+   * @param name file name. File should be present in the classpath.
+   */
+  public static synchronized void addDefaultResource(String name) {
+    if(!defaultResources.contains(name)) {
+      defaultResources.add(name);
+      for(Configuration conf : REGISTRY.keySet()) {
+        if(conf.loadDefaults) {
+          conf.reloadConfiguration();
+        }
+      }
+    }
+  }
+
+  /**
+   * Add a configuration resource. 
+   * 
+   * The properties of this resource will override properties of previously 
+   * added resources, unless they were marked <a href="#Final">final</a>. 
+   * 
+   * @param name resource to be added, the classpath is examined for a file 
+   *             with that name.
+   */
+  public void addResource(String name) {
+    addResourceObject(name);
+  }
+
+  /**
+   * Add a configuration resource. 
+   * 
+   * The properties of this resource will override properties of previously 
+   * added resources, unless they were marked <a href="#Final">final</a>. 
+   * 
+   * @param url url of the resource to be added, the local filesystem is 
+   *            examined directly to find the resource, without referring to 
+   *            the classpath.
+   */
+  public void addResource(URL url) {
+    addResourceObject(url);
+  }
+
+  /**
+   * Add a configuration resource. 
+   * 
+   * The properties of this resource will override properties of previously 
+   * added resources, unless they were marked <a href="#Final">final</a>. 
+   * 
+   * @param file file-path of resource to be added, the local filesystem is
+   *             examined directly to find the resource, without referring to 
+   *             the classpath.
+   */
+  public void addResource(Path file) {
+    addResourceObject(file);
+  }
+
+  /**
+   * Add a configuration resource. 
+   * 
+   * The properties of this resource will override properties of previously 
+   * added resources, unless they were marked <a href="#Final">final</a>. 
+   * 
+   * @param in InputStream to deserialize the object from. 
+   */
+  public void addResource(InputStream in) {
+    addResourceObject(in);
+  }
+  
+  
+  /**
+   * Reload configuration from previously added resources.
+   *
+   * This method will clear all the configuration read from the added 
+   * resources, and final parameters. This will make the resources to 
+   * be read again before accessing the values. Values that are added
+   * via set methods will overlay values read from the resources.
+   */
+  public synchronized void reloadConfiguration() {
+    properties = null;                            // trigger reload
+    finalParameters.clear();                      // clear site-limits
+  }
+  
+  private synchronized void addResourceObject(Object resource) {
+    resources.add(resource);                      // add to resources
+    reloadConfiguration();
+  }
+  
+  private static Pattern varPat = Pattern.compile("\\$\\{[^\\}\\$\u0020]+\\}");
+  private static int MAX_SUBST = 20;
+
+  private String substituteVars(String expr) {
+    if (expr == null) {
+      return null;
+    }
+    Matcher match = varPat.matcher("");
+    String eval = expr;
+    for(int s=0; s<MAX_SUBST; s++) {
+      match.reset(eval);
+      if (!match.find()) {
+        return eval;
+      }
+      String var = match.group();
+      var = var.substring(2, var.length()-1); // remove ${ .. }
+      String val = null;
+      try {
+        val = System.getProperty(var);
+      } catch(SecurityException se) {
+        LOG.warn("Unexpected SecurityException in Configuration", se);
+      }
+      if (val == null) {
+        val = getRaw(var);
+      }
+      if (val == null) {
+        return eval; // return literal ${var}: var is unbound
+      }
+      // substitute
+      eval = eval.substring(0, match.start())+val+eval.substring(match.end());
+    }
+    throw new IllegalStateException("Variable substitution depth too large: " 
+                                    + MAX_SUBST + " " + expr);
+  }
+  
+  /**
+   * Get the value of the <code>name</code> property, <code>null</code> if
+   * no such property exists.
+   * 
+   * Values are processed for <a href="#VariableExpansion">variable expansion</a> 
+   * before being returned. 
+   * 
+   * @param name the property name.
+   * @return the value of the <code>name</code> property, 
+   *         or null if no such property exists.
+   */
+  public String get(String name) {
+    return substituteVars(getProps().getProperty(name));
+  }
+
+  /**
+   * Get the value of the <code>name</code> property, without doing
+   * <a href="#VariableExpansion">variable expansion</a>.
+   * 
+   * @param name the property name.
+   * @return the value of the <code>name</code> property, 
+   *         or null if no such property exists.
+   */
+  public String getRaw(String name) {
+    return getProps().getProperty(name);
+  }
+
+  /** 
+   * Set the <code>value</code> of the <code>name</code> property.
+   * 
+   * @param name property name.
+   * @param value property value.
+   */
+  public void set(String name, String value) {
+    getOverlay().setProperty(name, value);
+    getProps().setProperty(name, value);
+  }
+  
+  /**
+   * Sets a property if it is currently unset.
+   * @param name the property name
+   * @param value the new value
+   */
+  public void setIfUnset(String name, String value) {
+    if (get(name) == null) {
+      set(name, value);
+    }
+  }
+  
+  private synchronized Properties getOverlay() {
+    if (overlay==null){
+      overlay=new Properties();
+    }
+    return overlay;
+  }
+
+  /** 
+   * Get the value of the <code>name</code> property. If no such property 
+   * exists, then <code>defaultValue</code> is returned.
+   * 
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value, or <code>defaultValue</code> if the property 
+   *         doesn't exist.                    
+   */
+  public String get(String name, String defaultValue) {
+    return substituteVars(getProps().getProperty(name, defaultValue));
+  }
+    
+  /** 
+   * Get the value of the <code>name</code> property as an <code>int</code>.
+   *   
+   * If no such property exists, or if the specified value is not a valid
+   * <code>int</code>, then <code>defaultValue</code> is returned.
+   * 
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value as an <code>int</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public int getInt(String name, int defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      String hexString = getHexDigits(valueString);
+      if (hexString != null) {
+        return Integer.parseInt(hexString, 16);
+      }
+      return Integer.parseInt(valueString);
+    } catch (NumberFormatException e) {
+      return defaultValue;
+    }
+  }
+
+  /** 
+   * Set the value of the <code>name</code> property to an <code>int</code>.
+   * 
+   * @param name property name.
+   * @param value <code>int</code> value of the property.
+   */
+  public void setInt(String name, int value) {
+    set(name, Integer.toString(value));
+  }
+
+
+  /** 
+   * Get the value of the <code>name</code> property as a <code>long</code>.  
+   * If no such property is specified, or if the specified value is not a valid
+   * <code>long</code>, then <code>defaultValue</code> is returned.
+   * 
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>long</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public long getLong(String name, long defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      String hexString = getHexDigits(valueString);
+      if (hexString != null) {
+        return Long.parseLong(hexString, 16);
+      }
+      return Long.parseLong(valueString);
+    } catch (NumberFormatException e) {
+      return defaultValue;
+    }
+  }
+
+  private String getHexDigits(String value) {
+    boolean negative = false;
+    String str = value;
+    String hexString = null;
+    if (value.startsWith("-")) {
+      negative = true;
+      str = value.substring(1);
+    }
+    if (str.startsWith("0x") || str.startsWith("0X")) {
+      hexString = str.substring(2);
+      if (negative) {
+        hexString = "-" + hexString;
+      }
+      return hexString;
+    }
+    return null;
+  }
+  
+  /** 
+   * Set the value of the <code>name</code> property to a <code>long</code>.
+   * 
+   * @param name property name.
+   * @param value <code>long</code> value of the property.
+   */
+  public void setLong(String name, long value) {
+    set(name, Long.toString(value));
+  }
+
+  /** 
+   * Get the value of the <code>name</code> property as a <code>float</code>.  
+   * If no such property is specified, or if the specified value is not a valid
+   * <code>float</code>, then <code>defaultValue</code> is returned.
+   * 
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>float</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public float getFloat(String name, float defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return Float.parseFloat(valueString);
+    } catch (NumberFormatException e) {
+      return defaultValue;
+    }
+  }
+  /**
+   * Set the value of the <code>name</code> property to a <code>float</code>.
+   * 
+   * @param name property name.
+   * @param value property value.
+   */
+  public void setFloat(String name, float value) {
+    set(name,Float.toString(value));
+  }
+ 
+  /** 
+   * Get the value of the <code>name</code> property as a <code>boolean</code>.  
+   * If no such property is specified, or if the specified value is not a valid
+   * <code>boolean</code>, then <code>defaultValue</code> is returned.
+   * 
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>boolean</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public boolean getBoolean(String name, boolean defaultValue) {
+    String valueString = get(name);
+    if ("true".equals(valueString))
+      return true;
+    else if ("false".equals(valueString))
+      return false;
+    else return defaultValue;
+  }
+
+  /** 
+   * Set the value of the <code>name</code> property to a <code>boolean</code>.
+   * 
+   * @param name property name.
+   * @param value <code>boolean</code> value of the property.
+   */
+  public void setBoolean(String name, boolean value) {
+    set(name, Boolean.toString(value));
+  }
+
+  /**
+   * Set the given property, if it is currently unset.
+   * @param name property name
+   * @param value new value
+   */
+  public void setBooleanIfUnset(String name, boolean value) {
+    setIfUnset(name, Boolean.toString(value));
+  }
+
+  /**
+   * A class that represents a set of positive integer ranges. It parses 
+   * strings of the form: "2-3,5,7-" where ranges are separated by comma and 
+   * the lower/upper bounds are separated by dash. Either the lower or upper 
+   * bound may be omitted meaning all values up to or over. So the string 
+   * above means 2, 3, 5, and 7, 8, 9, ...
+   */
+  public static class IntegerRanges {
+    private static class Range {
+      int start;
+      int end;
+    }
+
+    List<Range> ranges = new ArrayList<Range>();
+    
+    public IntegerRanges() {
+    }
+    
+    public IntegerRanges(String newValue) {
+      StringTokenizer itr = new StringTokenizer(newValue, ",");
+      while (itr.hasMoreTokens()) {
+        String rng = itr.nextToken().trim();
+        String[] parts = rng.split("-", 3);
+        if (parts.length < 1 || parts.length > 2) {
+          throw new IllegalArgumentException("integer range badly formed: " + 
+                                             rng);
+        }
+        Range r = new Range();
+        r.start = convertToInt(parts[0], 0);
+        if (parts.length == 2) {
+          r.end = convertToInt(parts[1], Integer.MAX_VALUE);
+        } else {
+          r.end = r.start;
+        }
+        if (r.start > r.end) {
+          throw new IllegalArgumentException("IntegerRange from " + r.start + 
+                                             " to " + r.end + " is invalid");
+        }
+        ranges.add(r);
+      }
+    }
+
+    /**
+     * Convert a string to an int treating empty strings as the default value.
+     * @param value the string value
+     * @param defaultValue the value for if the string is empty
+     * @return the desired integer
+     */
+    private static int convertToInt(String value, int defaultValue) {
+      String trim = value.trim();
+      if (trim.length() == 0) {
+        return defaultValue;
+      }
+      return Integer.parseInt(trim);
+    }
+
+    /**
+     * Is the given value in the set of ranges
+     * @param value the value to check
+     * @return is the value in the ranges?
+     */
+    public boolean isIncluded(int value) {
+      for(Range r: ranges) {
+        if (r.start <= value && value <= r.end) {
+          return true;
+        }
+      }
+      return false;
+    }
+    
+    @Override
+    public String toString() {
+      StringBuffer result = new StringBuffer();
+      boolean first = true;
+      for(Range r: ranges) {
+        if (first) {
+          first = false;
+        } else {
+          result.append(',');
+        }
+        result.append(r.start);
+        result.append('-');
+        result.append(r.end);
+      }
+      return result.toString();
+    }
+  }
+
+  /**
+   * Parse the given attribute as a set of integer ranges
+   * @param name the attribute name
+   * @param defaultValue the default value if it is not set
+   * @return a new set of ranges from the configured value
+   */
+  public IntegerRanges getRange(String name, String defaultValue) {
+    return new IntegerRanges(get(name, defaultValue));
+  }
+
+  /** 
+   * Get the comma delimited values of the <code>name</code> property as 
+   * a collection of <code>String</code>s.  
+   * If no such property is specified then empty collection is returned.
+   * <p>
+   * This is an optimized version of {@link #getStrings(String)}
+   * 
+   * @param name property name.
+   * @return property value as a collection of <code>String</code>s. 
+   */
+  public Collection<String> getStringCollection(String name) {
+    String valueString = get(name);
+    return StringUtils.getStringCollection(valueString);
+  }
+
+  /** 
+   * Get the comma delimited values of the <code>name</code> property as 
+   * an array of <code>String</code>s.  
+   * If no such property is specified then <code>null</code> is returned.
+   * 
+   * @param name property name.
+   * @return property value as an array of <code>String</code>s, 
+   *         or <code>null</code>. 
+   */
+  public String[] getStrings(String name) {
+    String valueString = get(name);
+    return StringUtils.getStrings(valueString);
+  }
+
+  /** 
+   * Get the comma delimited values of the <code>name</code> property as 
+   * an array of <code>String</code>s.  
+   * If no such property is specified then default value is returned.
+   * 
+   * @param name property name.
+   * @param defaultValue The default value
+   * @return property value as an array of <code>String</code>s, 
+   *         or default value. 
+   */
+  public String[] getStrings(String name, String... defaultValue) {
+    String valueString = get(name);
+    if (valueString == null) {
+      return defaultValue;
+    } else {
+      return StringUtils.getStrings(valueString);
+    }
+  }
+
+  /** 
+   * Set the array of string values for the <code>name</code> property as 
+   * as comma delimited values.  
+   * 
+   * @param name property name.
+   * @param values The values
+   */
+  public void setStrings(String name, String... values) {
+    set(name, StringUtils.arrayToString(values));
+  }
+ 
+  /**
+   * Load a class by name.
+   * 
+   * @param name the class name.
+   * @return the class object.
+   * @throws ClassNotFoundException if the class is not found.
+   */
+  public Class<?> getClassByName(String name) throws ClassNotFoundException {
+    return Class.forName(name, true, classLoader);
+  }
+
+  /** 
+   * Get the value of the <code>name</code> property
+   * as an array of <code>Class</code>.
+   * The value of the property specifies a list of comma separated class names.  
+   * If no such property is specified, then <code>defaultValue</code> is 
+   * returned.
+   * 
+   * @param name the property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>Class[]</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public Class<?>[] getClasses(String name, Class<?> ... defaultValue) {
+    String[] classnames = getStrings(name);
+    if (classnames == null)
+      return defaultValue;
+    try {
+      Class<?>[] classes = new Class<?>[classnames.length];
+      for(int i = 0; i < classnames.length; i++) {
+        classes[i] = getClassByName(classnames[i]);
+      }
+      return classes;
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** 
+   * Get the value of the <code>name</code> property as a <code>Class</code>.  
+   * If no such property is specified, then <code>defaultValue</code> is 
+   * returned.
+   * 
+   * @param name the class name.
+   * @param defaultValue default value.
+   * @return property value as a <code>Class</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public Class<?> getClass(String name, Class<?> defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return getClassByName(valueString);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** 
+   * Get the value of the <code>name</code> property as a <code>Class</code>
+   * implementing the interface specified by <code>xface</code>.
+   *   
+   * If no such property is specified, then <code>defaultValue</code> is 
+   * returned.
+   * 
+   * An exception is thrown if the returned class does not implement the named
+   * interface. 
+   * 
+   * @param name the class name.
+   * @param defaultValue default value.
+   * @param xface the interface implemented by the named class.
+   * @return property value as a <code>Class</code>, 
+   *         or <code>defaultValue</code>.
+   */
+  public <U> Class<? extends U> getClass(String name, 
+                                         Class<? extends U> defaultValue, 
+                                         Class<U> xface) {
+    try {
+      Class<?> theClass = getClass(name, defaultValue);
+      if (theClass != null && !xface.isAssignableFrom(theClass))
+        throw new RuntimeException(theClass+" not "+xface.getName());
+      else if (theClass != null)
+        return theClass.asSubclass(xface);
+      else
+        return null;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>List</code>
+   * of objects implementing the interface specified by <code>xface</code>.
+   * 
+   * An exception is thrown if any of the classes does not exist, or if it does
+   * not implement the named interface.
+   * 
+   * @param name the property name.
+   * @param xface the interface implemented by the classes named by
+   *        <code>name</code>.
+   * @return a <code>List</code> of objects implementing <code>xface</code>.
+   */
+  @SuppressWarnings("unchecked")
+  public <U> List<U> getInstances(String name, Class<U> xface) {
+    List<U> ret = new ArrayList<U>();
+    Class<?>[] classes = getClasses(name);
+    for (Class<?> cl: classes) {
+      if (!xface.isAssignableFrom(cl)) {
+        throw new RuntimeException(cl + " does not implement " + xface);
+      }
+      ret.add((U)ReflectionUtils.newInstance(cl, this));
+    }
+    return ret;
+  }
+
+  /** 
+   * Set the value of the <code>name</code> property to the name of a 
+   * <code>theClass</code> implementing the given interface <code>xface</code>.
+   * 
+   * An exception is thrown if <code>theClass</code> does not implement the 
+   * interface <code>xface</code>. 
+   * 
+   * @param name property name.
+   * @param theClass property value.
+   * @param xface the interface implemented by the named class.
+   */
+  public void setClass(String name, Class<?> theClass, Class<?> xface) {
+    if (!xface.isAssignableFrom(theClass))
+      throw new RuntimeException(theClass+" not "+xface.getName());
+    set(name, theClass.getName());
+  }
+
+  /** 
+   * Get a local file under a directory named by <i>dirsProp</i> with
+   * the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
+   * then one is chosen based on <i>path</i>'s hash code.  If the selected
+   * directory does not exist, an attempt is made to create it.
+   * 
+   * @param dirsProp directory in which to locate the file.
+   * @param path file-path.
+   * @return local file under the directory with the given path.
+   */
+  public Path getLocalPath(String dirsProp, String path)
+    throws IOException {
+    String[] dirs = getStrings(dirsProp);
+    int hashCode = path.hashCode();
+    FileSystem fs = FileSystem.getLocal(this);
+    for (int i = 0; i < dirs.length; i++) {  // try each local dir
+      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
+      Path file = new Path(dirs[index], path);
+      Path dir = file.getParent();
+      if (fs.mkdirs(dir) || fs.exists(dir)) {
+        return file;
+      }
+    }
+    LOG.warn("Could not make " + path + 
+             " in local directories from " + dirsProp);
+    for(int i=0; i < dirs.length; i++) {
+      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
+      LOG.warn(dirsProp + "[" + index + "]=" + dirs[index]);
+    }
+    throw new IOException("No valid local directories in property: "+dirsProp);
+  }
+
+  /** 
+   * Get a local file name under a directory named in <i>dirsProp</i> with
+   * the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
+   * then one is chosen based on <i>path</i>'s hash code.  If the selected
+   * directory does not exist, an attempt is made to create it.
+   * 
+   * @param dirsProp directory in which to locate the file.
+   * @param path file-path.
+   * @return local file under the directory with the given path.
+   */
+  public File getFile(String dirsProp, String path)
+    throws IOException {
+    String[] dirs = getStrings(dirsProp);
+    int hashCode = path.hashCode();
+    for (int i = 0; i < dirs.length; i++) {  // try each local dir
+      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
+      File file = new File(dirs[index], path);
+      File dir = file.getParentFile();
+      if (dir.exists() || dir.mkdirs()) {
+        return file;
+      }
+    }
+    throw new IOException("No valid local directories in property: "+dirsProp);
+  }
+
+  /** 
+   * Get the {@link URL} for the named resource.
+   * 
+   * @param name resource name.
+   * @return the url for the named resource.
+   */
+  public URL getResource(String name) {
+    return classLoader.getResource(name);
+  }
+  
+  /** 
+   * Get an input stream attached to the configuration resource with the
+   * given <code>name</code>.
+   * 
+   * @param name configuration resource name.
+   * @return an input stream attached to the resource.
+   */
+  public InputStream getConfResourceAsInputStream(String name) {
+    try {
+      URL url= getResource(name);
+
+      if (url == null) {
+        LOG.info(name + " not found");
+        return null;
+      } else {
+        LOG.info("found resource " + name + " at " + url);
+      }
+
+      return url.openStream();
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  /** 
+   * Get a {@link Reader} attached to the configuration resource with the
+   * given <code>name</code>.
+   * 
+   * @param name configuration resource name.
+   * @return a reader attached to the resource.
+   */
+  public Reader getConfResourceAsReader(String name) {
+    try {
+      URL url= getResource(name);
+
+      if (url == null) {
+        LOG.info(name + " not found");
+        return null;
+      } else {
+        LOG.info("found resource " + name + " at " + url);
+      }
+
+      return new InputStreamReader(url.openStream());
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  protected synchronized Properties getProps() {
+    if (properties == null) {
+      properties = new Properties();
+      loadResources(properties, resources, quietmode);
+      if (overlay!= null)
+        properties.putAll(overlay);
+    }
+    return properties;
+  }
+
+  /**
+   * Return the number of keys in the configuration.
+   *
+   * @return number of keys in the configuration.
+   */
+  public int size() {
+    return getProps().size();
+  }
+
+  /**
+   * Clears all keys from the configuration.
+   */
+  public void clear() {
+    getProps().clear();
+    getOverlay().clear();
+  }
+
+  /**
+   * Get an {@link Iterator} to go through the list of <code>String</code> 
+   * key-value pairs in the configuration.
+   * 
+   * @return an iterator over the entries.
+   */
+  public Iterator<Map.Entry<String, String>> iterator() {
+    // Get a copy of just the string to string pairs. After the old object
+    // methods that allow non-strings to be put into configurations are removed,
+    // we could replace properties with a Map<String,String> and get rid of this
+    // code.
+    Map<String,String> result = new HashMap<String,String>();
+    for(Map.Entry<Object,Object> item: getProps().entrySet()) {
+      if (item.getKey() instanceof String && 
+          item.getValue() instanceof String) {
+        result.put((String) item.getKey(), (String) item.getValue());
+      }
+    }
+    return result.entrySet().iterator();
+  }
+
+  private void loadResources(Properties properties,
+                             ArrayList resources,
+                             boolean quiet) {
+    if(loadDefaults) {
+      for (String resource : defaultResources) {
+        loadResource(properties, resource, quiet);
+      }
+    
+      //support the hadoop-site.xml as a deprecated case
+      if(getResource("hadoop-site.xml")!=null) {
+        loadResource(properties, "hadoop-site.xml", quiet);
+      }
+    }
+    
+    for (Object resource : resources) {
+      loadResource(properties, resource, quiet);
+    }
+  }
+
+  private void loadResource(Properties properties, Object name, boolean quiet) {
+    try {
+      DocumentBuilderFactory docBuilderFactory 
+        = DocumentBuilderFactory.newInstance();
+      //ignore all comments inside the xml file
+      docBuilderFactory.setIgnoringComments(true);
+
+      //allow includes in the xml file
+      docBuilderFactory.setNamespaceAware(true);
+      try {
+          docBuilderFactory.setXIncludeAware(true);
+      } catch (UnsupportedOperationException e) {
+        LOG.error("Failed to set setXIncludeAware(true) for parser "
+                + docBuilderFactory
+                + ":" + e,
+                e);
+      }
+      DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+      Document doc = null;
+      Element root = null;
+
+      if (name instanceof URL) {                  // an URL resource
+        URL url = (URL)name;
+        if (url != null) {
+          if (!quiet) {
+            LOG.info("parsing " + url);
+          }
+          doc = builder.parse(url.toString());
+        }
+      } else if (name instanceof String) {        // a CLASSPATH resource
+        URL url = getResource((String)name);
+        if (url != null) {
+          if (!quiet) {
+            LOG.info("parsing " + url);
+          }
+          doc = builder.parse(url.toString());
+        }
+      } else if (name instanceof Path) {          // a file resource
+        // Can't use FileSystem API or we get an infinite loop
+        // since FileSystem uses Configuration API.  Use java.io.File instead.
+        File file = new File(((Path)name).toUri().getPath())
+          .getAbsoluteFile();
+        if (file.exists()) {
+          if (!quiet) {
+            LOG.info("parsing " + file);
+          }
+          InputStream in = new BufferedInputStream(new FileInputStream(file));
+          try {
+            doc = builder.parse(in);
+          } finally {
+            in.close();
+          }
+        }
+      } else if (name instanceof InputStream) {
+        try {
+          doc = builder.parse((InputStream)name);
+        } finally {
+          ((InputStream)name).close();
+        }
+      } else if (name instanceof Element) {
+        root = (Element)name;
+      }
+
+      if (doc == null && root == null) {
+        if (quiet)
+          return;
+        throw new RuntimeException(name + " not found");
+      }
+
+      if (root == null) {
+        root = doc.getDocumentElement();
+      }
+      if (!"configuration".equals(root.getTagName()))
+        LOG.fatal("bad conf file: top-level element not <configuration>");
+      NodeList props = root.getChildNodes();
+      for (int i = 0; i < props.getLength(); i++) {
+        Node propNode = props.item(i);
+        if (!(propNode instanceof Element))
+          continue;
+        Element prop = (Element)propNode;
+        if ("configuration".equals(prop.getTagName())) {
+          loadResource(properties, prop, quiet);
+          continue;
+        }
+        if (!"property".equals(prop.getTagName()))
+          LOG.warn("bad conf file: element not <property>");
+        NodeList fields = prop.getChildNodes();
+        String attr = null;
+        String value = null;
+        boolean finalParameter = false;
+        for (int j = 0; j < fields.getLength(); j++) {
+          Node fieldNode = fields.item(j);
+          if (!(fieldNode instanceof Element))
+            continue;
+          Element field = (Element)fieldNode;
+          if ("name".equals(field.getTagName()) && field.hasChildNodes())
+            attr = ((Text)field.getFirstChild()).getData().trim();
+          if ("value".equals(field.getTagName()) && field.hasChildNodes())
+            value = ((Text)field.getFirstChild()).getData();
+          if ("final".equals(field.getTagName()) && field.hasChildNodes())
+            finalParameter = "true".equals(((Text)field.getFirstChild()).getData());
+        }
+        
+        // Ignore this parameter if it has already been marked as 'final'
+        if (attr != null && value != null) {
+          if (!finalParameters.contains(attr)) {
+            properties.setProperty(attr, value);
+            if (finalParameter)
+              finalParameters.add(attr);
+          } else {
+            LOG.warn(name+":a attempt to override final parameter: "+attr
+                     +";  Ignoring.");
+          }
+        }
+      }
+        
+    } catch (IOException e) {
+      LOG.fatal("error parsing conf file: " + e);
+      throw new RuntimeException(e);
+    } catch (DOMException e) {
+      LOG.fatal("error parsing conf file: " + e);
+      throw new RuntimeException(e);
+    } catch (SAXException e) {
+      LOG.fatal("error parsing conf file: " + e);
+      throw new RuntimeException(e);
+    } catch (ParserConfigurationException e) {
+      LOG.fatal("error parsing conf file: " + e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** 
+   * Write out the non-default properties in this configuration to the give
+   * {@link OutputStream}.
+   * 
+   * @param out the output stream to write to.
+   */
+  public void writeXml(OutputStream out) throws IOException {
+    Properties properties = getProps();
+    try {
+      Document doc =
+        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
+      Element conf = doc.createElement("configuration");
+      doc.appendChild(conf);
+      conf.appendChild(doc.createTextNode("\n"));
+      for (Enumeration e = properties.keys(); e.hasMoreElements();) {
+        String name = (String)e.nextElement();
+        Object object = properties.get(name);
+        String value = null;
+        if (object instanceof String) {
+          value = (String) object;
+        }else {
+          continue;
+        }
+        Element propNode = doc.createElement("property");
+        conf.appendChild(propNode);
+      
+        Element nameNode = doc.createElement("name");
+        nameNode.appendChild(doc.createTextNode(name));
+        propNode.appendChild(nameNode);
+      
+        Element valueNode = doc.createElement("value");
+        valueNode.appendChild(doc.createTextNode(value));
+        propNode.appendChild(valueNode);
+
+        conf.appendChild(doc.createTextNode("\n"));
+      }
+    
+      DOMSource source = new DOMSource(doc);
+      StreamResult result = new StreamResult(out);
+      TransformerFactory transFactory = TransformerFactory.newInstance();
+      Transformer transformer = transFactory.newTransformer();
+      transformer.transform(source, result);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Get the {@link ClassLoader} for this job.
+   * 
+   * @return the correct class loader.
+   */
+  public ClassLoader getClassLoader() {
+    return classLoader;
+  }
+  
+  /**
+   * Set the class loader that will be used to load the various objects.
+   * 
+   * @param classLoader the new class loader.
+   */
+  public void setClassLoader(ClassLoader classLoader) {
+    this.classLoader = classLoader;
+  }
+  
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("Configuration: ");
+    if(loadDefaults) {
+      toString(defaultResources, sb);
+      if(resources.size()>0) {
+        sb.append(", ");
+      }
+    }
+    toString(resources, sb);
+    return sb.toString();
+  }
+
+  private void toString(ArrayList resources, StringBuffer sb) {
+    ListIterator i = resources.listIterator();
+    while (i.hasNext()) {
+      if (i.nextIndex() != 0) {
+        sb.append(", ");
+      }
+      sb.append(i.next());
+    }
+  }
+
+  /** 
+   * Set the quietness-mode. 
+   * 
+   * In the quiet-mode, error and informational messages might not be logged.
+   * 
+   * @param quietmode <code>true</code> to set quiet-mode on, <code>false</code>
+   *              to turn it off.
+   */
+  public synchronized void setQuietMode(boolean quietmode) {
+    this.quietmode = quietmode;
+  }
+
+  /** For debugging.  List non-default properties to the terminal and exit. */
+  public static void main(String[] args) throws Exception {
+    new Configuration().writeXml(System.out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    clear();
+    int size = WritableUtils.readVInt(in);
+    for(int i=0; i < size; ++i) {
+      set(org.apache.hadoop.io.Text.readString(in), 
+          org.apache.hadoop.io.Text.readString(in));
+    }
+  }
+
+  //@Override
+  public void write(DataOutput out) throws IOException {
+    Properties props = getProps();
+    WritableUtils.writeVInt(out, props.size());
+    for(Map.Entry<Object, Object> item: props.entrySet()) {
+      org.apache.hadoop.io.Text.writeString(out, (String) item.getKey());
+      org.apache.hadoop.io.Text.writeString(out, (String) item.getValue());
+    }
+  }
+
+}

+ 46 - 0
src/java/org/apache/hadoop/conf/Configured.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+/** Base class for things that may be configured with a {@link Configuration}. */
+public class Configured implements Configurable {
+
+  private Configuration conf;
+
+  /** Construct a Configured. */
+  public Configured() {
+    this(null);
+  }
+  
+  /** Construct a Configured. */
+  public Configured(Configuration conf) {
+    setConf(conf);
+  }
+
+  // inherit javadoc
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  // inherit javadoc
+  public Configuration getConf() {
+    return conf;
+  }
+
+}

+ 23 - 0
src/java/org/apache/hadoop/conf/package.html

@@ -0,0 +1,23 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+Configuration of system parameters.
+</body>
+</html>

+ 879 - 0
src/java/org/apache/hadoop/filecache/DistributedCache.java

@@ -0,0 +1,879 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.filecache;
+
+import org.apache.commons.logging.*;
+import java.io.*;
+import java.util.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.*;
+import org.apache.hadoop.fs.*;
+
+import java.net.URI;
+
+/**
+ * Distribute application-specific large, read-only files efficiently.
+ * 
+ * <p><code>DistributedCache</code> is a facility provided by the Map-Reduce
+ * framework to cache files (text, archives, jars etc.) needed by applications.
+ * </p>
+ * 
+ * <p>Applications specify the files, via urls (hdfs:// or http://) to be cached 
+ * via the {@link org.apache.hadoop.mapred.JobConf}.
+ * The <code>DistributedCache</code> assumes that the
+ * files specified via hdfs:// urls are already present on the 
+ * {@link FileSystem} at the path specified by the url.</p>
+ * 
+ * <p>The framework will copy the necessary files on to the slave node before 
+ * any tasks for the job are executed on that node. Its efficiency stems from 
+ * the fact that the files are only copied once per job and the ability to 
+ * cache archives which are un-archived on the slaves.</p> 
+ *
+ * <p><code>DistributedCache</code> can be used to distribute simple, read-only
+ * data/text files and/or more complex types such as archives, jars etc. 
+ * Archives (zip, tar and tgz/tar.gz files) are un-archived at the slave nodes. 
+ * Jars may be optionally added to the classpath of the tasks, a rudimentary 
+ * software distribution mechanism.  Files have execution permissions.
+ * Optionally users can also direct it to symlink the distributed cache file(s)
+ * into the working directory of the task.</p>
+ * 
+ * <p><code>DistributedCache</code> tracks modification timestamps of the cache 
+ * files. Clearly the cache files should not be modified by the application 
+ * or externally while the job is executing.</p>
+ * 
+ * <p>Here is an illustrative example on how to use the 
+ * <code>DistributedCache</code>:</p>
+ * <p><blockquote><pre>
+ *     // Setting up the cache for the application
+ *     
+ *     1. Copy the requisite files to the <code>FileSystem</code>:
+ *     
+ *     $ bin/hadoop fs -copyFromLocal lookup.dat /myapp/lookup.dat  
+ *     $ bin/hadoop fs -copyFromLocal map.zip /myapp/map.zip  
+ *     $ bin/hadoop fs -copyFromLocal mylib.jar /myapp/mylib.jar
+ *     $ bin/hadoop fs -copyFromLocal mytar.tar /myapp/mytar.tar
+ *     $ bin/hadoop fs -copyFromLocal mytgz.tgz /myapp/mytgz.tgz
+ *     $ bin/hadoop fs -copyFromLocal mytargz.tar.gz /myapp/mytargz.tar.gz
+ *     
+ *     2. Setup the application's <code>JobConf</code>:
+ *     
+ *     JobConf job = new JobConf();
+ *     DistributedCache.addCacheFile(new URI("/myapp/lookup.dat#lookup.dat"), 
+ *                                   job);
+ *     DistributedCache.addCacheArchive(new URI("/myapp/map.zip", job);
+ *     DistributedCache.addFileToClassPath(new Path("/myapp/mylib.jar"), job);
+ *     DistributedCache.addCacheArchive(new URI("/myapp/mytar.tar", job);
+ *     DistributedCache.addCacheArchive(new URI("/myapp/mytgz.tgz", job);
+ *     DistributedCache.addCacheArchive(new URI("/myapp/mytargz.tar.gz", job);
+ *     
+ *     3. Use the cached files in the {@link org.apache.hadoop.mapred.Mapper}
+ *     or {@link org.apache.hadoop.mapred.Reducer}:
+ *     
+ *     public static class MapClass extends MapReduceBase  
+ *     implements Mapper&lt;K, V, K, V&gt; {
+ *     
+ *       private Path[] localArchives;
+ *       private Path[] localFiles;
+ *       
+ *       public void configure(JobConf job) {
+ *         // Get the cached archives/files
+ *         localArchives = DistributedCache.getLocalCacheArchives(job);
+ *         localFiles = DistributedCache.getLocalCacheFiles(job);
+ *       }
+ *       
+ *       public void map(K key, V value, 
+ *                       OutputCollector&lt;K, V&gt; output, Reporter reporter) 
+ *       throws IOException {
+ *         // Use data from the cached archives/files here
+ *         // ...
+ *         // ...
+ *         output.collect(k, v);
+ *       }
+ *     }
+ *     
+ * </pre></blockquote></p>
+ * 
+ * @see org.apache.hadoop.mapred.JobConf
+ * @see org.apache.hadoop.mapred.JobClient
+ */
+public class DistributedCache {
+  // cacheID to cacheStatus mapping
+  private static TreeMap<String, CacheStatus> cachedArchives = new TreeMap<String, CacheStatus>();
+  
+  private static TreeMap<Path, Long> baseDirSize = new TreeMap<Path, Long>();
+  
+  // default total cache size
+  private static final long DEFAULT_CACHE_SIZE = 10737418240L;
+
+  private static final Log LOG =
+    LogFactory.getLog(DistributedCache.class);
+  
+  /**
+   * Get the locally cached file or archive; it could either be 
+   * previously cached (and valid) or copy it from the {@link FileSystem} now.
+   * 
+   * @param cache the cache to be localized, this should be specified as 
+   * new URI(hdfs://hostname:port/absolute_path_to_file#LINKNAME). If no schema 
+   * or hostname:port is provided the file is assumed to be in the filesystem
+   * being used in the Configuration
+   * @param conf The Confguration file which contains the filesystem
+   * @param baseDir The base cache Dir where you wnat to localize the files/archives
+   * @param fileStatus The file status on the dfs.
+   * @param isArchive if the cache is an archive or a file. In case it is an
+   *  archive with a .zip or .jar or .tar or .tgz or .tar.gz extension it will
+   *  be unzipped/unjarred/untarred automatically 
+   *  and the directory where the archive is unzipped/unjarred/untarred is
+   *  returned as the Path.
+   *  In case of a file, the path to the file is returned
+   * @param confFileStamp this is the hdfs file modification timestamp to verify that the 
+   * file to be cached hasn't changed since the job started
+   * @param currentWorkDir this is the directory where you would want to create symlinks 
+   * for the locally cached files/archives
+   * @return the path to directory where the archives are unjarred in case of archives,
+   * the path to the file where the file is copied locally 
+   * @throws IOException
+   */
+  public static Path getLocalCache(URI cache, Configuration conf, 
+                                   Path baseDir, FileStatus fileStatus,
+                                   boolean isArchive, long confFileStamp,
+                                   Path currentWorkDir) 
+  throws IOException {
+    return getLocalCache(cache, conf, baseDir, fileStatus, isArchive, 
+        confFileStamp, currentWorkDir, true);
+  }
+  /**
+   * Get the locally cached file or archive; it could either be 
+   * previously cached (and valid) or copy it from the {@link FileSystem} now.
+   * 
+   * @param cache the cache to be localized, this should be specified as 
+   * new URI(hdfs://hostname:port/absolute_path_to_file#LINKNAME). If no schema 
+   * or hostname:port is provided the file is assumed to be in the filesystem
+   * being used in the Configuration
+   * @param conf The Confguration file which contains the filesystem
+   * @param baseDir The base cache Dir where you wnat to localize the files/archives
+   * @param fileStatus The file status on the dfs.
+   * @param isArchive if the cache is an archive or a file. In case it is an
+   *  archive with a .zip or .jar or .tar or .tgz or .tar.gz extension it will
+   *  be unzipped/unjarred/untarred automatically 
+   *  and the directory where the archive is unzipped/unjarred/untarred is
+   *  returned as the Path.
+   *  In case of a file, the path to the file is returned
+   * @param confFileStamp this is the hdfs file modification timestamp to verify that the 
+   * file to be cached hasn't changed since the job started
+   * @param currentWorkDir this is the directory where you would want to create symlinks 
+   * for the locally cached files/archives
+   * @param honorSymLinkConf if this is false, then the symlinks are not
+   * created even if conf says so (this is required for an optimization in task
+   * launches
+   * @return the path to directory where the archives are unjarred in case of archives,
+   * the path to the file where the file is copied locally 
+   * @throws IOException
+   */
+  public static Path getLocalCache(URI cache, Configuration conf, 
+      Path baseDir, FileStatus fileStatus,
+      boolean isArchive, long confFileStamp,
+      Path currentWorkDir, boolean honorSymLinkConf) 
+  throws IOException {
+    String cacheId = makeRelative(cache, conf);
+    CacheStatus lcacheStatus;
+    Path localizedPath;
+    synchronized (cachedArchives) {
+      lcacheStatus = cachedArchives.get(cacheId);
+      if (lcacheStatus == null) {
+        // was never localized
+        lcacheStatus = new CacheStatus(baseDir, new Path(baseDir, new Path(cacheId)));
+        cachedArchives.put(cacheId, lcacheStatus);
+      }
+
+      synchronized (lcacheStatus) {
+        localizedPath = localizeCache(conf, cache, confFileStamp, lcacheStatus, 
+            fileStatus, isArchive, currentWorkDir, honorSymLinkConf);
+        lcacheStatus.refcount++;
+      }
+    }
+
+    // try deleting stuff if you can
+    long size = 0;
+    synchronized (baseDirSize) {
+      Long get = baseDirSize.get(baseDir);
+      if ( get != null ) {
+    	size = get.longValue();
+      }
+    }
+    // setting the cache size to a default of 10GB
+    long allowedSize = conf.getLong("local.cache.size", DEFAULT_CACHE_SIZE);
+    if (allowedSize < size) {
+      // try some cache deletions
+      deleteCache(conf);
+    }
+    return localizedPath;
+  }
+
+  
+  /**
+   * Get the locally cached file or archive; it could either be 
+   * previously cached (and valid) or copy it from the {@link FileSystem} now.
+   * 
+   * @param cache the cache to be localized, this should be specified as 
+   * new URI(hdfs://hostname:port/absolute_path_to_file#LINKNAME). If no schema 
+   * or hostname:port is provided the file is assumed to be in the filesystem
+   * being used in the Configuration
+   * @param conf The Confguration file which contains the filesystem
+   * @param baseDir The base cache Dir where you wnat to localize the files/archives
+   * @param isArchive if the cache is an archive or a file. In case it is an 
+   *  archive with a .zip or .jar or .tar or .tgz or .tar.gz extension it will 
+   *  be unzipped/unjarred/untarred automatically 
+   *  and the directory where the archive is unzipped/unjarred/untarred 
+   *  is returned as the Path.
+   *  In case of a file, the path to the file is returned
+   * @param confFileStamp this is the hdfs file modification timestamp to verify that the 
+   * file to be cached hasn't changed since the job started
+   * @param currentWorkDir this is the directory where you would want to create symlinks 
+   * for the locally cached files/archives
+   * @return the path to directory where the archives are unjarred in case of archives,
+   * the path to the file where the file is copied locally 
+   * @throws IOException
+
+   */
+  public static Path getLocalCache(URI cache, Configuration conf, 
+                                   Path baseDir, boolean isArchive,
+                                   long confFileStamp, Path currentWorkDir) 
+  throws IOException {
+    return getLocalCache(cache, conf, 
+                         baseDir, null, isArchive,
+                         confFileStamp, currentWorkDir);
+  }
+  
+  /**
+   * This is the opposite of getlocalcache. When you are done with
+   * using the cache, you need to release the cache
+   * @param cache The cache URI to be released
+   * @param conf configuration which contains the filesystem the cache 
+   * is contained in.
+   * @throws IOException
+   */
+  public static void releaseCache(URI cache, Configuration conf)
+    throws IOException {
+    String cacheId = makeRelative(cache, conf);
+    synchronized (cachedArchives) {
+      CacheStatus lcacheStatus = cachedArchives.get(cacheId);
+      if (lcacheStatus == null)
+        return;
+      synchronized (lcacheStatus) {
+        lcacheStatus.refcount--;
+      }
+    }
+  }
+  
+  // To delete the caches which have a refcount of zero
+  
+  private static void deleteCache(Configuration conf) throws IOException {
+    // try deleting cache Status with refcount of zero
+    synchronized (cachedArchives) {
+      for (Iterator it = cachedArchives.keySet().iterator(); it.hasNext();) {
+        String cacheId = (String) it.next();
+        CacheStatus lcacheStatus = cachedArchives.get(cacheId);
+        synchronized (lcacheStatus) {
+          if (lcacheStatus.refcount == 0) {
+            // delete this cache entry
+            FileSystem.getLocal(conf).delete(lcacheStatus.localLoadPath, true);
+            synchronized (baseDirSize) {
+              Long dirSize = baseDirSize.get(lcacheStatus.baseDir);
+              if ( dirSize != null ) {
+            	dirSize -= lcacheStatus.size;
+            	baseDirSize.put(lcacheStatus.baseDir, dirSize);
+              }
+            }
+            it.remove();
+          }
+        }
+      }
+    }
+  }
+
+  /*
+   * Returns the relative path of the dir this cache will be localized in
+   * relative path that this cache will be localized in. For
+   * hdfs://hostname:port/absolute_path -- the relative path is
+   * hostname/absolute path -- if it is just /absolute_path -- then the
+   * relative path is hostname of DFS this mapred cluster is running
+   * on/absolute_path
+   */
+  public static String makeRelative(URI cache, Configuration conf)
+    throws IOException {
+    String host = cache.getHost();
+    if (host == null) {
+      host = cache.getScheme();
+    }
+    if (host == null) {
+      URI defaultUri = FileSystem.get(conf).getUri();
+      host = defaultUri.getHost();
+      if (host == null) {
+        host = defaultUri.getScheme();
+      }
+    }
+    String path = host + cache.getPath();
+    path = path.replace(":/","/");                // remove windows device colon
+    return path;
+  }
+
+  private static Path cacheFilePath(Path p) {
+    return new Path(p, p.getName());
+  }
+
+  // the method which actually copies the caches locally and unjars/unzips them
+  // and does chmod for the files
+  private static Path localizeCache(Configuration conf, 
+                                    URI cache, long confFileStamp,
+                                    CacheStatus cacheStatus,
+                                    FileStatus fileStatus,
+                                    boolean isArchive, 
+                                    Path currentWorkDir,boolean honorSymLinkConf) 
+  throws IOException {
+    boolean doSymlink = honorSymLinkConf && getSymlink(conf);
+    if(cache.getFragment() == null) {
+    	doSymlink = false;
+    }
+    FileSystem fs = getFileSystem(cache, conf);
+    String link = currentWorkDir.toString() + Path.SEPARATOR + cache.getFragment();
+    File flink = new File(link);
+    if (ifExistsAndFresh(conf, fs, cache, confFileStamp,
+                           cacheStatus, fileStatus)) {
+      if (isArchive) {
+        if (doSymlink){
+          if (!flink.exists())
+            FileUtil.symLink(cacheStatus.localLoadPath.toString(), 
+                             link);
+        }
+        return cacheStatus.localLoadPath;
+      }
+      else {
+        if (doSymlink){
+          if (!flink.exists())
+            FileUtil.symLink(cacheFilePath(cacheStatus.localLoadPath).toString(), 
+                             link);
+        }
+        return cacheFilePath(cacheStatus.localLoadPath);
+      }
+    } else {
+      // remove the old archive
+      // if the old archive cannot be removed since it is being used by another
+      // job
+      // return null
+      if (cacheStatus.refcount > 1 && (cacheStatus.currentStatus == true))
+        throw new IOException("Cache " + cacheStatus.localLoadPath.toString()
+                              + " is in use and cannot be refreshed");
+      
+      FileSystem localFs = FileSystem.getLocal(conf);
+      localFs.delete(cacheStatus.localLoadPath, true);
+      synchronized (baseDirSize) {
+    	Long dirSize = baseDirSize.get(cacheStatus.baseDir);
+    	if ( dirSize != null ) {
+    	  dirSize -= cacheStatus.size;
+    	  baseDirSize.put(cacheStatus.baseDir, dirSize);
+    	}
+      }
+      Path parchive = new Path(cacheStatus.localLoadPath,
+                               new Path(cacheStatus.localLoadPath.getName()));
+      
+      if (!localFs.mkdirs(cacheStatus.localLoadPath)) {
+        throw new IOException("Mkdirs failed to create directory " + 
+                              cacheStatus.localLoadPath.toString());
+      }
+
+      String cacheId = cache.getPath();
+      fs.copyToLocalFile(new Path(cacheId), parchive);
+      if (isArchive) {
+        String tmpArchive = parchive.toString().toLowerCase();
+        File srcFile = new File(parchive.toString());
+        File destDir = new File(parchive.getParent().toString());
+        if (tmpArchive.endsWith(".jar")) {
+          RunJar.unJar(srcFile, destDir);
+        } else if (tmpArchive.endsWith(".zip")) {
+          FileUtil.unZip(srcFile, destDir);
+        } else if (isTarFile(tmpArchive)) {
+          FileUtil.unTar(srcFile, destDir);
+        }
+        // else will not do anyhting
+        // and copy the file into the dir as it is
+      }
+      
+      long cacheSize = FileUtil.getDU(new File(parchive.getParent().toString()));
+      cacheStatus.size = cacheSize;
+      synchronized (baseDirSize) {
+      	Long dirSize = baseDirSize.get(cacheStatus.baseDir);
+      	if( dirSize == null ) {
+      	  dirSize = Long.valueOf(cacheSize);
+      	} else {
+      	  dirSize += cacheSize;
+      	}
+      	baseDirSize.put(cacheStatus.baseDir, dirSize);
+      }
+      
+      // do chmod here 
+      try {
+        //Setting recursive permission to grant everyone read and execute
+        FileUtil.chmod(cacheStatus.baseDir.toString(), "ugo+rx",true);
+      } catch(InterruptedException e) {
+    	LOG.warn("Exception in chmod" + e.toString());
+      }
+
+      // update cacheStatus to reflect the newly cached file
+      cacheStatus.currentStatus = true;
+      cacheStatus.mtime = getTimestamp(conf, cache);
+    }
+    
+    if (isArchive){
+      if (doSymlink){
+        if (!flink.exists())
+          FileUtil.symLink(cacheStatus.localLoadPath.toString(), 
+                           link);
+      }
+      return cacheStatus.localLoadPath;
+    }
+    else {
+      if (doSymlink){
+        if (!flink.exists())
+          FileUtil.symLink(cacheFilePath(cacheStatus.localLoadPath).toString(), 
+                           link);
+      }
+      return cacheFilePath(cacheStatus.localLoadPath);
+    }
+  }
+
+  private static boolean isTarFile(String filename) {
+    return (filename.endsWith(".tgz") || filename.endsWith(".tar.gz") ||
+           filename.endsWith(".tar"));
+  }
+  
+  // Checks if the cache has already been localized and is fresh
+  private static boolean ifExistsAndFresh(Configuration conf, FileSystem fs, 
+                                          URI cache, long confFileStamp, 
+                                          CacheStatus lcacheStatus,
+                                          FileStatus fileStatus) 
+  throws IOException {
+    // check for existence of the cache
+    if (lcacheStatus.currentStatus == false) {
+      return false;
+    } else {
+      long dfsFileStamp;
+      if (fileStatus != null) {
+        dfsFileStamp = fileStatus.getModificationTime();
+      } else {
+        dfsFileStamp = getTimestamp(conf, cache);
+      }
+
+      // ensure that the file on hdfs hasn't been modified since the job started 
+      if (dfsFileStamp != confFileStamp) {
+        LOG.fatal("File: " + cache + " has changed on HDFS since job started");
+        throw new IOException("File: " + cache + 
+                              " has changed on HDFS since job started");
+      }
+      
+      if (dfsFileStamp != lcacheStatus.mtime) {
+        // needs refreshing
+        return false;
+      }
+    }
+    
+    return true;
+  }
+
+  /**
+   * Returns mtime of a given cache file on hdfs.
+   * @param conf configuration
+   * @param cache cache file 
+   * @return mtime of a given cache file on hdfs
+   * @throws IOException
+   */
+  public static long getTimestamp(Configuration conf, URI cache)
+    throws IOException {
+    FileSystem fileSystem = FileSystem.get(cache, conf);
+    Path filePath = new Path(cache.getPath());
+
+    return fileSystem.getFileStatus(filePath).getModificationTime();
+  }
+  
+  /**
+   * This method create symlinks for all files in a given dir in another directory
+   * @param conf the configuration
+   * @param jobCacheDir the target directory for creating symlinks
+   * @param workDir the directory in which the symlinks are created
+   * @throws IOException
+   */
+  public static void createAllSymlink(Configuration conf, File jobCacheDir, File workDir)
+    throws IOException{
+    if ((jobCacheDir == null || !jobCacheDir.isDirectory()) ||
+           workDir == null || (!workDir.isDirectory())) {
+      return;
+    }
+    boolean createSymlink = getSymlink(conf);
+    if (createSymlink){
+      File[] list = jobCacheDir.listFiles();
+      for (int i=0; i < list.length; i++){
+        FileUtil.symLink(list[i].getAbsolutePath(),
+                         new File(workDir, list[i].getName()).toString());
+      }
+    }  
+  }
+  
+  private static FileSystem getFileSystem(URI cache, Configuration conf)
+    throws IOException {
+    if ("hdfs".equals(cache.getScheme()))
+      return FileSystem.get(cache, conf);
+    else
+      return FileSystem.get(conf);
+  }
+
+  /**
+   * Set the configuration with the given set of archives
+   * @param archives The list of archives that need to be localized
+   * @param conf Configuration which will be changed
+   */
+  public static void setCacheArchives(URI[] archives, Configuration conf) {
+    String sarchives = StringUtils.uriToString(archives);
+    conf.set("mapred.cache.archives", sarchives);
+  }
+
+  /**
+   * Set the configuration with the given set of files
+   * @param files The list of files that need to be localized
+   * @param conf Configuration which will be changed
+   */
+  public static void setCacheFiles(URI[] files, Configuration conf) {
+    String sfiles = StringUtils.uriToString(files);
+    conf.set("mapred.cache.files", sfiles);
+  }
+
+  /**
+   * Get cache archives set in the Configuration
+   * @param conf The configuration which contains the archives
+   * @return A URI array of the caches set in the Configuration
+   * @throws IOException
+   */
+  public static URI[] getCacheArchives(Configuration conf) throws IOException {
+    return StringUtils.stringToURI(conf.getStrings("mapred.cache.archives"));
+  }
+
+  /**
+   * Get cache files set in the Configuration
+   * @param conf The configuration which contains the files
+   * @return A URI array of the files set in the Configuration
+   * @throws IOException
+   */
+
+  public static URI[] getCacheFiles(Configuration conf) throws IOException {
+    return StringUtils.stringToURI(conf.getStrings("mapred.cache.files"));
+  }
+
+  /**
+   * Return the path array of the localized caches
+   * @param conf Configuration that contains the localized archives
+   * @return A path array of localized caches
+   * @throws IOException
+   */
+  public static Path[] getLocalCacheArchives(Configuration conf)
+    throws IOException {
+    return StringUtils.stringToPath(conf
+                                    .getStrings("mapred.cache.localArchives"));
+  }
+
+  /**
+   * Return the path array of the localized files
+   * @param conf Configuration that contains the localized files
+   * @return A path array of localized files
+   * @throws IOException
+   */
+  public static Path[] getLocalCacheFiles(Configuration conf)
+    throws IOException {
+    return StringUtils.stringToPath(conf.getStrings("mapred.cache.localFiles"));
+  }
+
+  /**
+   * Get the timestamps of the archives
+   * @param conf The configuration which stored the timestamps
+   * @return a string array of timestamps 
+   * @throws IOException
+   */
+  public static String[] getArchiveTimestamps(Configuration conf) {
+    return conf.getStrings("mapred.cache.archives.timestamps");
+  }
+
+
+  /**
+   * Get the timestamps of the files
+   * @param conf The configuration which stored the timestamps
+   * @return a string array of timestamps 
+   * @throws IOException
+   */
+  public static String[] getFileTimestamps(Configuration conf) {
+    return conf.getStrings("mapred.cache.files.timestamps");
+  }
+
+  /**
+   * This is to check the timestamp of the archives to be localized
+   * @param conf Configuration which stores the timestamp's
+   * @param timestamps comma separated list of timestamps of archives.
+   * The order should be the same as the order in which the archives are added.
+   */
+  public static void setArchiveTimestamps(Configuration conf, String timestamps) {
+    conf.set("mapred.cache.archives.timestamps", timestamps);
+  }
+
+  /**
+   * This is to check the timestamp of the files to be localized
+   * @param conf Configuration which stores the timestamp's
+   * @param timestamps comma separated list of timestamps of files.
+   * The order should be the same as the order in which the files are added.
+   */
+  public static void setFileTimestamps(Configuration conf, String timestamps) {
+    conf.set("mapred.cache.files.timestamps", timestamps);
+  }
+  
+  /**
+   * Set the conf to contain the location for localized archives 
+   * @param conf The conf to modify to contain the localized caches
+   * @param str a comma separated list of local archives
+   */
+  public static void setLocalArchives(Configuration conf, String str) {
+    conf.set("mapred.cache.localArchives", str);
+  }
+
+  /**
+   * Set the conf to contain the location for localized files 
+   * @param conf The conf to modify to contain the localized caches
+   * @param str a comma separated list of local files
+   */
+  public static void setLocalFiles(Configuration conf, String str) {
+    conf.set("mapred.cache.localFiles", str);
+  }
+
+  /**
+   * Add a archives to be localized to the conf
+   * @param uri The uri of the cache to be localized
+   * @param conf Configuration to add the cache to
+   */
+  public static void addCacheArchive(URI uri, Configuration conf) {
+    String archives = conf.get("mapred.cache.archives");
+    conf.set("mapred.cache.archives", archives == null ? uri.toString()
+             : archives + "," + uri.toString());
+  }
+  
+  /**
+   * Add a file to be localized to the conf
+   * @param uri The uri of the cache to be localized
+   * @param conf Configuration to add the cache to
+   */
+  public static void addCacheFile(URI uri, Configuration conf) {
+    String files = conf.get("mapred.cache.files");
+    conf.set("mapred.cache.files", files == null ? uri.toString() : files + ","
+             + uri.toString());
+  }
+
+  /**
+   * Add an file path to the current set of classpath entries It adds the file
+   * to cache as well.
+   * 
+   * @param file Path of the file to be added
+   * @param conf Configuration that contains the classpath setting
+   */
+  public static void addFileToClassPath(Path file, Configuration conf)
+    throws IOException {
+    String classpath = conf.get("mapred.job.classpath.files");
+    conf.set("mapred.job.classpath.files", classpath == null ? file.toString()
+             : classpath + System.getProperty("path.separator") + file.toString());
+    FileSystem fs = FileSystem.get(conf);
+    URI uri = fs.makeQualified(file).toUri();
+
+    addCacheFile(uri, conf);
+  }
+
+  /**
+   * Get the file entries in classpath as an array of Path
+   * 
+   * @param conf Configuration that contains the classpath setting
+   */
+  public static Path[] getFileClassPaths(Configuration conf) {
+    String classpath = conf.get("mapred.job.classpath.files");
+    if (classpath == null)
+      return null;
+    ArrayList list = Collections.list(new StringTokenizer(classpath, System
+                                                          .getProperty("path.separator")));
+    Path[] paths = new Path[list.size()];
+    for (int i = 0; i < list.size(); i++) {
+      paths[i] = new Path((String) list.get(i));
+    }
+    return paths;
+  }
+
+  /**
+   * Add an archive path to the current set of classpath entries. It adds the
+   * archive to cache as well.
+   * 
+   * @param archive Path of the archive to be added
+   * @param conf Configuration that contains the classpath setting
+   */
+  public static void addArchiveToClassPath(Path archive, Configuration conf)
+    throws IOException {
+    String classpath = conf.get("mapred.job.classpath.archives");
+    conf.set("mapred.job.classpath.archives", classpath == null ? archive
+             .toString() : classpath + System.getProperty("path.separator")
+             + archive.toString());
+    FileSystem fs = FileSystem.get(conf);
+    URI uri = fs.makeQualified(archive).toUri();
+
+    addCacheArchive(uri, conf);
+  }
+
+  /**
+   * Get the archive entries in classpath as an array of Path
+   * 
+   * @param conf Configuration that contains the classpath setting
+   */
+  public static Path[] getArchiveClassPaths(Configuration conf) {
+    String classpath = conf.get("mapred.job.classpath.archives");
+    if (classpath == null)
+      return null;
+    ArrayList list = Collections.list(new StringTokenizer(classpath, System
+                                                          .getProperty("path.separator")));
+    Path[] paths = new Path[list.size()];
+    for (int i = 0; i < list.size(); i++) {
+      paths[i] = new Path((String) list.get(i));
+    }
+    return paths;
+  }
+
+  /**
+   * This method allows you to create symlinks in the current working directory
+   * of the task to all the cache files/archives
+   * @param conf the jobconf 
+   */
+  public static void createSymlink(Configuration conf){
+    conf.set("mapred.create.symlink", "yes");
+  }
+  
+  /**
+   * This method checks to see if symlinks are to be create for the 
+   * localized cache files in the current working directory 
+   * @param conf the jobconf
+   * @return true if symlinks are to be created- else return false
+   */
+  public static boolean getSymlink(Configuration conf){
+    String result = conf.get("mapred.create.symlink");
+    if ("yes".equals(result)){
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * This method checks if there is a conflict in the fragment names 
+   * of the uris. Also makes sure that each uri has a fragment. It 
+   * is only to be called if you want to create symlinks for 
+   * the various archives and files.
+   * @param uriFiles The uri array of urifiles
+   * @param uriArchives the uri array of uri archives
+   */
+  public static boolean checkURIs(URI[]  uriFiles, URI[] uriArchives){
+    if ((uriFiles == null) && (uriArchives == null)){
+      return true;
+    }
+    if (uriFiles != null){
+      for (int i = 0; i < uriFiles.length; i++){
+        String frag1 = uriFiles[i].getFragment();
+        if (frag1 == null)
+          return false;
+        for (int j=i+1; j < uriFiles.length; j++){
+          String frag2 = uriFiles[j].getFragment();
+          if (frag2 == null)
+            return false;
+          if (frag1.equalsIgnoreCase(frag2))
+            return false;
+        }
+        if (uriArchives != null){
+          for (int j = 0; j < uriArchives.length; j++){
+            String frag2 = uriArchives[j].getFragment();
+            if (frag2 == null){
+              return false;
+            }
+            if (frag1.equalsIgnoreCase(frag2))
+              return false;
+            for (int k=j+1; k < uriArchives.length; k++){
+              String frag3 = uriArchives[k].getFragment();
+              if (frag3 == null)
+                return false;
+              if (frag2.equalsIgnoreCase(frag3))
+                return false;
+            }
+          }
+        }
+      }
+    }
+    return true;
+  }
+
+  private static class CacheStatus {
+    // false, not loaded yet, true is loaded
+    boolean currentStatus;
+
+    // the local load path of this cache
+    Path localLoadPath;
+    
+    //the base dir where the cache lies
+    Path baseDir;
+    
+    //the size of this cache
+    long size;
+
+    // number of instances using this cache
+    int refcount;
+
+    // the cache-file modification time
+    long mtime;
+
+    public CacheStatus(Path baseDir, Path localLoadPath) {
+      super();
+      this.currentStatus = false;
+      this.localLoadPath = localLoadPath;
+      this.refcount = 0;
+      this.mtime = -1;
+      this.baseDir = baseDir;
+      this.size = 0;
+    }
+  }
+
+  /**
+   * Clear the entire contents of the cache and delete the backing files. This
+   * should only be used when the server is reinitializing, because the users
+   * are going to lose their files.
+   */
+  public static void purgeCache(Configuration conf) throws IOException {
+    synchronized (cachedArchives) {
+      FileSystem localFs = FileSystem.getLocal(conf);
+      for (Map.Entry<String,CacheStatus> f: cachedArchives.entrySet()) {
+        try {
+          localFs.delete(f.getValue().localLoadPath, true);
+        } catch (IOException ie) {
+          LOG.debug("Error cleaning up cache", ie);
+        }
+      }
+      cachedArchives.clear();
+    }
+  }
+}

+ 241 - 0
src/java/org/apache/hadoop/fs/BlockLocation.java

@@ -0,0 +1,241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+
+/*
+ * A BlockLocation lists hosts, offset and length
+ * of block. 
+ * 
+ */
+public class BlockLocation implements Writable {
+
+  static {               // register a ctor
+    WritableFactories.setFactory
+      (BlockLocation.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new BlockLocation(); }
+       });
+  }
+
+  private String[] hosts; //hostnames of datanodes
+  private String[] names; //hostname:portNumber of datanodes
+  private String[] topologyPaths; // full path name in network topology
+  private long offset;  //offset of the of the block in the file
+  private long length;
+
+  /**
+   * Default Constructor
+   */
+  public BlockLocation() {
+    this(new String[0], new String[0],  0L, 0L);
+  }
+
+  /**
+   * Constructor with host, name, offset and length
+   */
+  public BlockLocation(String[] names, String[] hosts, long offset, 
+                       long length) {
+    if (names == null) {
+      this.names = new String[0];
+    } else {
+      this.names = names;
+    }
+    if (hosts == null) {
+      this.hosts = new String[0];
+    } else {
+      this.hosts = hosts;
+    }
+    this.offset = offset;
+    this.length = length;
+    this.topologyPaths = new String[0];
+  }
+
+  /**
+   * Constructor with host, name, network topology, offset and length
+   */
+  public BlockLocation(String[] names, String[] hosts, String[] topologyPaths,
+                       long offset, long length) {
+    this(names, hosts, offset, length);
+    if (topologyPaths == null) {
+      this.topologyPaths = new String[0];
+    } else {
+      this.topologyPaths = topologyPaths;
+    }
+  }
+
+  /**
+   * Get the list of hosts (hostname) hosting this block
+   */
+  public String[] getHosts() throws IOException {
+    if ((hosts == null) || (hosts.length == 0)) {
+      return new String[0];
+    } else {
+      return hosts;
+    }
+  }
+
+  /**
+   * Get the list of names (hostname:port) hosting this block
+   */
+  public String[] getNames() throws IOException {
+    if ((names == null) || (names.length == 0)) {
+      return new String[0];
+    } else {
+      return this.names;
+    }
+  }
+
+  /**
+   * Get the list of network topology paths for each of the hosts.
+   * The last component of the path is the host.
+   */
+  public String[] getTopologyPaths() throws IOException {
+    if ((topologyPaths == null) || (topologyPaths.length == 0)) {
+      return new String[0];
+    } else {
+      return this.topologyPaths;
+    }
+  }
+  
+  /**
+   * Get the start offset of file associated with this block
+   */
+  public long getOffset() {
+    return offset;
+  }
+  
+  /**
+   * Get the length of the block
+   */
+  public long getLength() {
+    return length;
+  }
+  
+  /**
+   * Set the start offset of file associated with this block
+   */
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  /**
+   * Set the length of block
+   */
+  public void setLength(long length) {
+    this.length = length;
+  }
+
+  /**
+   * Set the hosts hosting this block
+   */
+  public void setHosts(String[] hosts) throws IOException {
+    if (hosts == null) {
+      this.hosts = new String[0];
+    } else {
+      this.hosts = hosts;
+    }
+  }
+
+  /**
+   * Set the names (host:port) hosting this block
+   */
+  public void setNames(String[] names) throws IOException {
+    if (names == null) {
+      this.names = new String[0];
+    } else {
+      this.names = names;
+    }
+  }
+
+  /**
+   * Set the network topology paths of the hosts
+   */
+  public void setTopologyPaths(String[] topologyPaths) throws IOException {
+    if (topologyPaths == null) {
+      this.topologyPaths = new String[0];
+    } else {
+      this.topologyPaths = topologyPaths;
+    }
+  }
+
+  /**
+   * Implement write of Writable
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(offset);
+    out.writeLong(length);
+    out.writeInt(names.length);
+    for (int i=0; i < names.length; i++) {
+      Text name = new Text(names[i]);
+      name.write(out);
+    }
+    out.writeInt(hosts.length);
+    for (int i=0; i < hosts.length; i++) {
+      Text host = new Text(hosts[i]);
+      host.write(out);
+    }
+    out.writeInt(topologyPaths.length);
+    for (int i=0; i < topologyPaths.length; i++) {
+      Text host = new Text(topologyPaths[i]);
+      host.write(out);
+    }
+  }
+  
+  /**
+   * Implement readFields of Writable
+   */
+  public void readFields(DataInput in) throws IOException {
+    this.offset = in.readLong();
+    this.length = in.readLong();
+    int numNames = in.readInt();
+    this.names = new String[numNames];
+    for (int i = 0; i < numNames; i++) {
+      Text name = new Text();
+      name.readFields(in);
+      names[i] = name.toString();
+    }
+    int numHosts = in.readInt();
+    for (int i = 0; i < numHosts; i++) {
+      Text host = new Text();
+      host.readFields(in);
+      hosts[i] = host.toString();
+    }
+    int numTops = in.readInt();
+    Text path = new Text();
+    for (int i = 0; i < numTops; i++) {
+      path.readFields(in);
+      topologyPaths[i] = path.toString();
+    }
+  }
+  
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(offset);
+    result.append(',');
+    result.append(length);
+    for(String h: hosts) {
+      result.append(',');
+      result.append(h);
+    }
+    return result.toString();
+  }
+}

+ 96 - 0
src/java/org/apache/hadoop/fs/BufferedFSInputStream.java

@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+
+
+/**
+ * A class optimizes reading from FSInputStream by bufferring
+ */
+
+
+public class BufferedFSInputStream extends BufferedInputStream
+implements Seekable, PositionedReadable {
+  /**
+   * Creates a <code>BufferedFSInputStream</code>
+   * with the specified buffer size,
+   * and saves its  argument, the input stream
+   * <code>in</code>, for later use.  An internal
+   * buffer array of length  <code>size</code>
+   * is created and stored in <code>buf</code>.
+   *
+   * @param   in     the underlying input stream.
+   * @param   size   the buffer size.
+   * @exception IllegalArgumentException if size <= 0.
+   */
+  public BufferedFSInputStream(FSInputStream in, int size) {
+    super(in, size);
+  }
+
+  public long getPos() throws IOException {
+    return ((FSInputStream)in).getPos()-(count-pos);
+  }
+
+  public long skip(long n) throws IOException {
+    if (n <= 0) {
+      return 0;
+    }
+
+    seek(getPos()+n);
+    return n;
+  }
+
+  public void seek(long pos) throws IOException {
+    if( pos<0 ) {
+      return;
+    }
+    // optimize: check if the pos is in the buffer
+    long end = ((FSInputStream)in).getPos();
+    long start = end - count;
+    if( pos>=start && pos<end) {
+      this.pos = (int)(pos-start);
+      return;
+    }
+
+    // invalidate buffer
+    this.pos = 0;
+    this.count = 0;
+
+    ((FSInputStream)in).seek(pos);
+  }
+
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    pos = 0;
+    count = 0;
+    return ((FSInputStream)in).seekToNewSource(targetPos);
+  }
+
+  public int read(long position, byte[] buffer, int offset, int length) throws IOException {
+    return ((FSInputStream)in).read(position, buffer, offset, length) ;
+  }
+
+  public void readFully(long position, byte[] buffer, int offset, int length) throws IOException {
+    ((FSInputStream)in).readFully(position, buffer, offset, length);
+  }
+
+  public void readFully(long position, byte[] buffer) throws IOException {
+    ((FSInputStream)in).readFully(position, buffer);
+  }
+}

+ 35 - 0
src/java/org/apache/hadoop/fs/ChecksumException.java

@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+/** Thrown for checksum errors. */
+public class ChecksumException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private long pos;
+  public ChecksumException(String description, long pos) {
+    super(description);
+    this.pos = pos;
+  }
+  
+  public long getPos() {
+    return pos;
+  }
+}

+ 547 - 0
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -0,0 +1,547 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.*;
+import java.util.Arrays;
+import java.util.zip.CRC32;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.StringUtils;
+
+/****************************************************************
+ * Abstract Checksumed FileSystem.
+ * It provide a basice implementation of a Checksumed FileSystem,
+ * which creates a checksum file for each raw file.
+ * It generates & verifies checksums at the client side.
+ *
+ *****************************************************************/
+public abstract class ChecksumFileSystem extends FilterFileSystem {
+  private static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0};
+  private int bytesPerChecksum = 512;
+  private boolean verifyChecksum = true;
+
+  public static double getApproxChkSumLength(long size) {
+    return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
+  }
+  
+  public ChecksumFileSystem(FileSystem fs) {
+    super(fs);
+  }
+
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf != null) {
+      bytesPerChecksum = conf.getInt("io.bytes.per.checksum", 512);
+    }
+  }
+  
+  /**
+   * Set whether to verify checksum.
+   */
+  public void setVerifyChecksum(boolean verifyChecksum) {
+    this.verifyChecksum = verifyChecksum;
+  }
+
+  /** get the raw file system */
+  public FileSystem getRawFileSystem() {
+    return fs;
+  }
+
+  /** Return the name of the checksum file associated with a file.*/
+  public Path getChecksumFile(Path file) {
+    return new Path(file.getParent(), "." + file.getName() + ".crc");
+  }
+
+  /** Return true iff file is a checksum file name.*/
+  public static boolean isChecksumFile(Path file) {
+    String name = file.getName();
+    return name.startsWith(".") && name.endsWith(".crc");
+  }
+
+  /** Return the length of the checksum file given the size of the 
+   * actual file.
+   **/
+  public long getChecksumFileLength(Path file, long fileSize) {
+    return getChecksumLength(fileSize, getBytesPerSum());
+  }
+
+  /** Return the bytes Per Checksum */
+  public int getBytesPerSum() {
+    return bytesPerChecksum;
+  }
+
+  private int getSumBufferSize(int bytesPerSum, int bufferSize) {
+    int defaultBufferSize = getConf().getInt("io.file.buffer.size", 4096);
+    int proportionalBufferSize = bufferSize / bytesPerSum;
+    return Math.max(bytesPerSum,
+                    Math.max(proportionalBufferSize, defaultBufferSize));
+  }
+
+  /*******************************************************
+   * For open()'s FSInputStream
+   * It verifies that data matches checksums.
+   *******************************************************/
+  private static class ChecksumFSInputChecker extends FSInputChecker {
+    public static final Log LOG 
+      = LogFactory.getLog(FSInputChecker.class);
+    
+    private ChecksumFileSystem fs;
+    private FSDataInputStream datas;
+    private FSDataInputStream sums;
+    
+    private static final int HEADER_LENGTH = 8;
+    
+    private int bytesPerSum = 1;
+    private long fileLen = -1L;
+    
+    public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
+      throws IOException {
+      this(fs, file, fs.getConf().getInt("io.file.buffer.size", 4096));
+    }
+    
+    public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
+      throws IOException {
+      super( file, fs.getFileStatus(file).getReplication() );
+      this.datas = fs.getRawFileSystem().open(file, bufferSize);
+      this.fs = fs;
+      Path sumFile = fs.getChecksumFile(file);
+      try {
+        int sumBufferSize = fs.getSumBufferSize(fs.getBytesPerSum(), bufferSize);
+        sums = fs.getRawFileSystem().open(sumFile, sumBufferSize);
+
+        byte[] version = new byte[CHECKSUM_VERSION.length];
+        sums.readFully(version);
+        if (!Arrays.equals(version, CHECKSUM_VERSION))
+          throw new IOException("Not a checksum file: "+sumFile);
+        this.bytesPerSum = sums.readInt();
+        set(fs.verifyChecksum, new CRC32(), bytesPerSum, 4);
+      } catch (FileNotFoundException e) {         // quietly ignore
+        set(fs.verifyChecksum, null, 1, 0);
+      } catch (IOException e) {                   // loudly ignore
+        LOG.warn("Problem opening checksum file: "+ file + 
+                 ".  Ignoring exception: " + 
+                 StringUtils.stringifyException(e));
+        set(fs.verifyChecksum, null, 1, 0);
+      }
+    }
+    
+    private long getChecksumFilePos( long dataPos ) {
+      return HEADER_LENGTH + 4*(dataPos/bytesPerSum);
+    }
+    
+    protected long getChunkPosition( long dataPos ) {
+      return dataPos/bytesPerSum*bytesPerSum;
+    }
+    
+    public int available() throws IOException {
+      return datas.available() + super.available();
+    }
+    
+    public int read(long position, byte[] b, int off, int len)
+      throws IOException {
+      // parameter check
+      if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
+        throw new IndexOutOfBoundsException();
+      } else if (len == 0) {
+        return 0;
+      }
+      if( position<0 ) {
+        throw new IllegalArgumentException(
+            "Parameter position can not to be negative");
+      }
+
+      ChecksumFSInputChecker checker = new ChecksumFSInputChecker(fs, file);
+      checker.seek(position);
+      int nread = checker.read(b, off, len);
+      checker.close();
+      return nread;
+    }
+    
+    public void close() throws IOException {
+      datas.close();
+      if( sums != null ) {
+        sums.close();
+      }
+      set(fs.verifyChecksum, null, 1, 0);
+    }
+    
+
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      long sumsPos = getChecksumFilePos(targetPos);
+      fs.reportChecksumFailure(file, datas, targetPos, sums, sumsPos);
+      boolean newDataSource = datas.seekToNewSource(targetPos);
+      return sums.seekToNewSource(sumsPos) || newDataSource;
+    }
+
+    @Override
+    protected int readChunk(long pos, byte[] buf, int offset, int len,
+        byte[] checksum) throws IOException {
+      boolean eof = false;
+      if(needChecksum()) {
+        try {
+          long checksumPos = getChecksumFilePos(pos); 
+          if(checksumPos != sums.getPos()) {
+            sums.seek(checksumPos);
+          }
+          sums.readFully(checksum);
+        } catch (EOFException e) {
+          eof = true;
+        }
+        len = bytesPerSum;
+      }
+      if(pos != datas.getPos()) {
+        datas.seek(pos);
+      }
+      int nread = readFully(datas, buf, offset, len);
+      if( eof && nread > 0) {
+        throw new ChecksumException("Checksum error: "+file+" at "+pos, pos);
+      }
+      return nread;
+    }
+    
+    /* Return the file length */
+    private long getFileLength() throws IOException {
+      if( fileLen==-1L ) {
+        fileLen = fs.getContentSummary(file).getLength();
+      }
+      return fileLen;
+    }
+    
+    /**
+     * Skips over and discards <code>n</code> bytes of data from the
+     * input stream.
+     *
+     *The <code>skip</code> method skips over some smaller number of bytes
+     * when reaching end of file before <code>n</code> bytes have been skipped.
+     * The actual number of bytes skipped is returned.  If <code>n</code> is
+     * negative, no bytes are skipped.
+     *
+     * @param      n   the number of bytes to be skipped.
+     * @return     the actual number of bytes skipped.
+     * @exception  IOException  if an I/O error occurs.
+     *             ChecksumException if the chunk to skip to is corrupted
+     */
+    public synchronized long skip(long n) throws IOException {
+      long curPos = getPos();
+      long fileLength = getFileLength();
+      if( n+curPos > fileLength ) {
+        n = fileLength - curPos;
+      }
+      return super.skip(n);
+    }
+    
+    /**
+     * Seek to the given position in the stream.
+     * The next read() will be from that position.
+     * 
+     * <p>This method does not allow seek past the end of the file.
+     * This produces IOException.
+     *
+     * @param      pos   the postion to seek to.
+     * @exception  IOException  if an I/O error occurs or seeks after EOF
+     *             ChecksumException if the chunk to seek to is corrupted
+     */
+
+    public synchronized void seek(long pos) throws IOException {
+      if(pos>getFileLength()) {
+        throw new IOException("Cannot seek after EOF");
+      }
+      super.seek(pos);
+    }
+
+  }
+
+  /**
+   * Opens an FSDataInputStream at the indicated Path.
+   * @param f the file name to open
+   * @param bufferSize the size of the buffer to be used.
+   */
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    return new FSDataInputStream(
+        new ChecksumFSInputChecker(this, f, bufferSize));
+  }
+
+  /** {@inheritDoc} */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Not supported");
+  }
+
+  /**
+   * Calculated the length of the checksum file in bytes.
+   * @param size the length of the data file in bytes
+   * @param bytesPerSum the number of bytes in a checksum block
+   * @return the number of bytes in the checksum file
+   */
+  public static long getChecksumLength(long size, int bytesPerSum) {
+    //the checksum length is equal to size passed divided by bytesPerSum +
+    //bytes written in the beginning of the checksum file.  
+    return ((size + bytesPerSum - 1) / bytesPerSum) * 4 +
+             CHECKSUM_VERSION.length + 4;  
+  }
+
+  /** This class provides an output stream for a checksummed file.
+   * It generates checksums for data. */
+  private static class ChecksumFSOutputSummer extends FSOutputSummer {
+    private FSDataOutputStream datas;    
+    private FSDataOutputStream sums;
+    private static final float CHKSUM_AS_FRACTION = 0.01f;
+    
+    public ChecksumFSOutputSummer(ChecksumFileSystem fs, 
+                          Path file, 
+                          boolean overwrite, 
+                          short replication,
+                          long blockSize,
+                          Configuration conf)
+      throws IOException {
+      this(fs, file, overwrite, 
+           conf.getInt("io.file.buffer.size", 4096),
+           replication, blockSize, null);
+    }
+    
+    public ChecksumFSOutputSummer(ChecksumFileSystem fs, 
+                          Path file, 
+                          boolean overwrite,
+                          int bufferSize,
+                          short replication,
+                          long blockSize,
+                          Progressable progress)
+      throws IOException {
+      super(new CRC32(), fs.getBytesPerSum(), 4);
+      int bytesPerSum = fs.getBytesPerSum();
+      this.datas = fs.getRawFileSystem().create(file, overwrite, bufferSize, 
+                                         replication, blockSize, progress);
+      int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
+      this.sums = fs.getRawFileSystem().create(fs.getChecksumFile(file), true, 
+                                               sumBufferSize, replication,
+                                               blockSize);
+      sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
+      sums.writeInt(bytesPerSum);
+    }
+    
+    public void close() throws IOException {
+      flushBuffer();
+      sums.close();
+      datas.close();
+    }
+    
+    @Override
+    protected void writeChunk(byte[] b, int offset, int len, byte[] checksum)
+    throws IOException {
+      datas.write(b, offset, len);
+      sums.write(checksum);
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    Path parent = f.getParent();
+    if (parent != null && !mkdirs(parent)) {
+      throw new IOException("Mkdirs failed to create " + parent);
+    }
+    final FSDataOutputStream out = new FSDataOutputStream(
+        new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
+            blockSize, progress), null);
+    if (permission != null) {
+      setPermission(f, permission);
+    }
+    return out;
+  }
+
+  /**
+   * Set replication for an existing file.
+   * Implement the abstract <tt>setReplication</tt> of <tt>FileSystem</tt>
+   * @param src file name
+   * @param replication new replication
+   * @throws IOException
+   * @return true if successful;
+   *         false if file does not exist or is a directory
+   */
+  public boolean setReplication(Path src, short replication) throws IOException {
+    boolean value = fs.setReplication(src, replication);
+    if (!value)
+      return false;
+
+    Path checkFile = getChecksumFile(src);
+    if (exists(checkFile))
+      fs.setReplication(checkFile, replication);
+
+    return true;
+  }
+
+  /**
+   * Rename files/dirs
+   */
+  public boolean rename(Path src, Path dst) throws IOException {
+    if (fs.isDirectory(src)) {
+      return fs.rename(src, dst);
+    } else {
+
+      boolean value = fs.rename(src, dst);
+      if (!value)
+        return false;
+
+      Path checkFile = getChecksumFile(src);
+      if (fs.exists(checkFile)) { //try to rename checksum
+        if (fs.isDirectory(dst)) {
+          value = fs.rename(checkFile, dst);
+        } else {
+          value = fs.rename(checkFile, getChecksumFile(dst));
+        }
+      }
+
+      return value;
+    }
+  }
+
+  /**
+   * Implement the delete(Path, boolean) in checksum
+   * file system.
+   */
+  public boolean delete(Path f, boolean recursive) throws IOException{
+    FileStatus fstatus = null;
+    try {
+      fstatus = fs.getFileStatus(f);
+    } catch(FileNotFoundException e) {
+      return false;
+    }
+    if(fstatus.isDir()) {
+      //this works since the crcs are in the same
+      //directories and the files. so we just delete
+      //everything in the underlying filesystem
+      return fs.delete(f, recursive);
+    } else {
+      Path checkFile = getChecksumFile(f);
+      if (fs.exists(checkFile)) {
+        fs.delete(checkFile, true);
+      }
+      return fs.delete(f, true);
+    }
+  }
+    
+  final private static PathFilter DEFAULT_FILTER = new PathFilter() {
+    public boolean accept(Path file) {
+      return !isChecksumFile(file);
+    }
+  };
+
+  /**
+   * List the statuses of the files/directories in the given path if the path is
+   * a directory.
+   * 
+   * @param f
+   *          given path
+   * @return the statuses of the files/directories in the given patch
+   * @throws IOException
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    return fs.listStatus(f, DEFAULT_FILTER);
+  }
+  
+  @Override
+  public boolean mkdirs(Path f) throws IOException {
+    return fs.mkdirs(f);
+  }
+
+  @Override
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    Configuration conf = getConf();
+    FileUtil.copy(getLocal(conf), src, this, dst, delSrc, conf);
+  }
+
+  /**
+   * The src file is under FS, and the dst is on the local disk.
+   * Copy it from FS control to the local dst name.
+   */
+  @Override
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    Configuration conf = getConf();
+    FileUtil.copy(this, src, getLocal(conf), dst, delSrc, conf);
+  }
+
+  /**
+   * The src file is under FS, and the dst is on the local disk.
+   * Copy it from FS control to the local dst name.
+   * If src and dst are directories, the copyCrc parameter
+   * determines whether to copy CRC files.
+   */
+  public void copyToLocalFile(Path src, Path dst, boolean copyCrc)
+    throws IOException {
+    if (!fs.isDirectory(src)) { // source is a file
+      fs.copyToLocalFile(src, dst);
+      FileSystem localFs = getLocal(getConf()).getRawFileSystem();
+      if (localFs.isDirectory(dst)) {
+        dst = new Path(dst, src.getName());
+      }
+      dst = getChecksumFile(dst);
+      if (localFs.exists(dst)) { //remove old local checksum file
+        localFs.delete(dst, true);
+      }
+      Path checksumFile = getChecksumFile(src);
+      if (copyCrc && fs.exists(checksumFile)) { //copy checksum file
+        fs.copyToLocalFile(checksumFile, dst);
+      }
+    } else {
+      FileStatus[] srcs = listStatus(src);
+      for (FileStatus srcFile : srcs) {
+        copyToLocalFile(srcFile.getPath(), 
+                        new Path(dst, srcFile.getPath().getName()), copyCrc);
+      }
+    }
+  }
+
+  @Override
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    return tmpLocalFile;
+  }
+
+  @Override
+  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    moveFromLocalFile(tmpLocalFile, fsOutputFile);
+  }
+
+  /**
+   * Report a checksum error to the file system.
+   * @param f the file name containing the error
+   * @param in the stream open on the file
+   * @param inPos the position of the beginning of the bad data in the file
+   * @param sums the stream open on the checksum file
+   * @param sumsPos the position of the beginning of the bad data in the checksum file
+   * @return if retry is neccessary
+   */
+  public boolean reportChecksumFailure(Path f, FSDataInputStream in,
+                                       long inPos, FSDataInputStream sums, long sumsPos) {
+    return false;
+  }
+}

+ 164 - 0
src/java/org/apache/hadoop/fs/ContentSummary.java

@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+/** Store the summary of a content (a directory or a file). */
+public class ContentSummary implements Writable{
+  private long length;
+  private long fileCount;
+  private long directoryCount;
+  private long quota;
+  private long spaceConsumed;
+  private long spaceQuota;
+  
+
+  /** Constructor */
+  public ContentSummary() {}
+  
+  /** Constructor */
+  public ContentSummary(long length, long fileCount, long directoryCount) {
+    this(length, fileCount, directoryCount, -1L, length, -1L);
+  }
+
+  /** Constructor */
+  public ContentSummary(
+      long length, long fileCount, long directoryCount, long quota,
+      long spaceConsumed, long spaceQuota) {
+    this.length = length;
+    this.fileCount = fileCount;
+    this.directoryCount = directoryCount;
+    this.quota = quota;
+    this.spaceConsumed = spaceConsumed;
+    this.spaceQuota = spaceQuota;
+  }
+
+  /** @return the length */
+  public long getLength() {return length;}
+
+  /** @return the directory count */
+  public long getDirectoryCount() {return directoryCount;}
+
+  /** @return the file count */
+  public long getFileCount() {return fileCount;}
+  
+  /** Return the directory quota */
+  public long getQuota() {return quota;}
+  
+  /** Retuns (disk) space consumed */ 
+  public long getSpaceConsumed() {return spaceConsumed;}
+
+  /** Returns (disk) space quota */
+  public long getSpaceQuota() {return spaceQuota;}
+  
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(length);
+    out.writeLong(fileCount);
+    out.writeLong(directoryCount);
+    out.writeLong(quota);
+    out.writeLong(spaceConsumed);
+    out.writeLong(spaceQuota);
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    this.length = in.readLong();
+    this.fileCount = in.readLong();
+    this.directoryCount = in.readLong();
+    this.quota = in.readLong();
+    this.spaceConsumed = in.readLong();
+    this.spaceQuota = in.readLong();
+  }
+  
+  /** 
+   * Output format:
+   * <----12----> <----12----> <-------18------->
+   *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE FILE_NAME    
+   */
+  private static final String STRING_FORMAT = "%12d %12d %18d ";
+  /** 
+   * Output format:
+   * <----12----> <----15----> <----15----> <----15----> <----12----> <----12----> <-------18------->
+   *    QUOTA   REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM DIR_COUNT   FILE_COUNT   CONTENT_SIZE     FILE_NAME    
+   */
+  private static final String QUOTA_STRING_FORMAT = "%12s %15s ";
+  private static final String SPACE_QUOTA_STRING_FORMAT = "%15s %15s ";
+  
+  /** The header string */
+  private static final String HEADER = String.format(
+      STRING_FORMAT.replace('d', 's'), "directories", "files", "bytes");
+
+  private static final String QUOTA_HEADER = String.format(
+      QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT, 
+      "quota", "remaining quota", "space quota", "reamaining quota") +
+      HEADER;
+  
+  /** Return the header of the output.
+   * if qOption is false, output directory count, file count, and content size;
+   * if qOption is true, output quota and remaining quota as well.
+   * 
+   * @param qOption a flag indicating if quota needs to be printed or not
+   * @return the header of the output
+   */
+  public static String getHeader(boolean qOption) {
+    return qOption ? QUOTA_HEADER : HEADER;
+  }
+  
+  /** {@inheritDoc} */
+  public String toString() {
+    return toString(true);
+  }
+
+  /** Return the string representation of the object in the output format.
+   * if qOption is false, output directory count, file count, and content size;
+   * if qOption is true, output quota and remaining quota as well.
+   * 
+   * @param qOption a flag indicating if quota needs to be printed or not
+   * @return the string representation of the object
+   */
+  public String toString(boolean qOption) {
+    String prefix = "";
+    if (qOption) {
+      String quotaStr = "none";
+      String quotaRem = "inf";
+      String spaceQuotaStr = "none";
+      String spaceQuotaRem = "inf";
+      
+      if (quota>0) {
+        quotaStr = Long.toString(quota);
+        quotaRem = Long.toString(quota-(directoryCount+fileCount));
+      }
+      if (spaceQuota>0) {
+        spaceQuotaStr = Long.toString(spaceQuota);
+        spaceQuotaRem = Long.toString(spaceQuota - spaceConsumed);        
+      }
+      
+      prefix = String.format(QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT, 
+                             quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
+    }
+    
+    return prefix + String.format(STRING_FORMAT, directoryCount, 
+                                  fileCount, length);
+  }
+}

+ 193 - 0
src/java/org/apache/hadoop/fs/DF.java

@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.BufferedReader;
+
+import java.util.EnumSet;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
+
+/** Filesystem disk space usage statistics.  Uses the unix 'df' program.
+ * Tested on Linux, FreeBSD, Cygwin. */
+public class DF extends Shell {
+  public static final long DF_INTERVAL_DEFAULT = 3 * 1000; // default DF refresh interval 
+  
+  private String dirPath;
+  private String filesystem;
+  private long capacity;
+  private long used;
+  private long available;
+  private int percentUsed;
+  private String mount;
+
+  enum OSType {
+    OS_TYPE_UNIX("UNIX"),
+    OS_TYPE_WIN("Windows"),
+    OS_TYPE_SOLARIS("SunOS"),
+    OS_TYPE_MAC("Mac"),
+    OS_TYPE_AIX("AIX");
+
+    private String id;
+    OSType(String id) {
+      this.id = id;
+    }
+    public boolean match(String osStr) {
+      return osStr != null && osStr.indexOf(id) >= 0;
+    }
+    String getId() {
+      return id;
+    }
+  }
+
+  private static final String OS_NAME = System.getProperty("os.name");
+  private static final OSType OS_TYPE = getOSType(OS_NAME);
+
+  protected static OSType getOSType(String osName) {
+    for (OSType ost : EnumSet.allOf(OSType.class)) {
+      if (ost.match(osName)) {
+        return ost;
+      }
+    }
+    return OSType.OS_TYPE_UNIX;
+  }
+
+  public DF(File path, Configuration conf) throws IOException {
+    this(path, conf.getLong("dfs.df.interval", DF.DF_INTERVAL_DEFAULT));
+  }
+
+  public DF(File path, long dfInterval) throws IOException {
+    super(dfInterval);
+    this.dirPath = path.getCanonicalPath();
+  }
+
+  protected OSType getOSType() {
+    return OS_TYPE;
+  }
+  
+  /// ACCESSORS
+
+  public String getDirPath() {
+    return dirPath;
+  }
+  
+  public String getFilesystem() throws IOException { 
+    run(); 
+    return filesystem; 
+  }
+  
+  public long getCapacity() throws IOException { 
+    run(); 
+    return capacity; 
+  }
+  
+  public long getUsed() throws IOException { 
+    run(); 
+    return used;
+  }
+  
+  public long getAvailable() throws IOException { 
+    run(); 
+    return available;
+  }
+  
+  public int getPercentUsed() throws IOException {
+    run();
+    return percentUsed;
+  }
+
+  public String getMount() throws IOException {
+    run();
+    return mount;
+  }
+  
+  public String toString() {
+    return
+      "df -k " + mount +"\n" +
+      filesystem + "\t" +
+      capacity / 1024 + "\t" +
+      used / 1024 + "\t" +
+      available / 1024 + "\t" +
+      percentUsed + "%\t" +
+      mount;
+  }
+
+  @Override
+  protected String[] getExecString() {
+    // ignoring the error since the exit code it enough
+    return new String[] {"bash","-c","exec 'df' '-k' '" + dirPath 
+                         + "' 2>/dev/null"};
+  }
+
+  @Override
+  protected void parseExecResult(BufferedReader lines) throws IOException {
+    lines.readLine();                         // skip headings
+  
+    String line = lines.readLine();
+    if (line == null) {
+      throw new IOException( "Expecting a line not the end of stream" );
+    }
+    StringTokenizer tokens =
+      new StringTokenizer(line, " \t\n\r\f%");
+    
+    this.filesystem = tokens.nextToken();
+    if (!tokens.hasMoreTokens()) {            // for long filesystem name
+      line = lines.readLine();
+      if (line == null) {
+        throw new IOException( "Expecting a line not the end of stream" );
+      }
+      tokens = new StringTokenizer(line, " \t\n\r\f%");
+    }
+
+    switch(getOSType()) {
+      case OS_TYPE_AIX:
+        this.capacity = Long.parseLong(tokens.nextToken()) * 1024;
+        this.available = Long.parseLong(tokens.nextToken()) * 1024;
+        this.percentUsed = Integer.parseInt(tokens.nextToken());
+        tokens.nextToken();
+        tokens.nextToken();
+        this.mount = tokens.nextToken();
+        this.used = this.capacity - this.available;
+        break;
+
+      case OS_TYPE_WIN:
+      case OS_TYPE_SOLARIS:
+      case OS_TYPE_MAC:
+      case OS_TYPE_UNIX:
+      default:
+        this.capacity = Long.parseLong(tokens.nextToken()) * 1024;
+        this.used = Long.parseLong(tokens.nextToken()) * 1024;
+        this.available = Long.parseLong(tokens.nextToken()) * 1024;
+        this.percentUsed = Integer.parseInt(tokens.nextToken());
+        this.mount = tokens.nextToken();
+        break;
+   }
+  }
+
+  public static void main(String[] args) throws Exception {
+    String path = ".";
+    if (args.length > 0)
+      path = args[0];
+
+    System.out.println(new DF(new File(path), DF_INTERVAL_DEFAULT).toString());
+  }
+}

+ 198 - 0
src/java/org/apache/hadoop/fs/DU.java

@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Filesystem disk space usage statistics.  Uses the unix 'du' program*/
+public class DU extends Shell {
+  private String  dirPath;
+
+  private AtomicLong used = new AtomicLong();
+  private volatile boolean shouldRun = true;
+  private Thread refreshUsed;
+  private IOException duException = null;
+  private long refreshInterval;
+  
+  /**
+   * Keeps track of disk usage.
+   * @param path the path to check disk usage in
+   * @param interval refresh the disk usage at this interval
+   * @throws IOException if we fail to refresh the disk usage
+   */
+  public DU(File path, long interval) throws IOException {
+    super(0);
+    
+    //we set the Shell interval to 0 so it will always run our command
+    //and use this one to set the thread sleep interval
+    this.refreshInterval = interval;
+    this.dirPath = path.getCanonicalPath();
+    
+    //populate the used variable
+    run();
+  }
+  
+  /**
+   * Keeps track of disk usage.
+   * @param path the path to check disk usage in
+   * @param conf configuration object
+   * @throws IOException if we fail to refresh the disk usage
+   */
+  public DU(File path, Configuration conf) throws IOException {
+    this(path, 600000L);
+    //10 minutes default refresh interval
+  }
+
+  /**
+   * This thread refreshes the "used" variable.
+   * 
+   * Future improvements could be to not permanently
+   * run this thread, instead run when getUsed is called.
+   **/
+  class DURefreshThread implements Runnable {
+    
+    public void run() {
+      
+      while(shouldRun) {
+
+        try {
+          Thread.sleep(refreshInterval);
+          
+          try {
+            //update the used variable
+            DU.this.run();
+          } catch (IOException e) {
+            synchronized (DU.this) {
+              //save the latest exception so we can return it in getUsed()
+              duException = e;
+            }
+            
+            LOG.warn("Could not get disk usage information", e);
+          }
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+  }
+  
+  /**
+   * Decrease how much disk space we use.
+   * @param value decrease by this value
+   */
+  public void decDfsUsed(long value) {
+    used.addAndGet(-value);
+  }
+
+  /**
+   * Increase how much disk space we use.
+   * @param value increase by this value
+   */
+  public void incDfsUsed(long value) {
+    used.addAndGet(value);
+  }
+  
+  /**
+   * @return disk space used 
+   * @throws IOException if the shell command fails
+   */
+  public long getUsed() throws IOException {
+    //if the updating thread isn't started, update on demand
+    if(refreshUsed == null) {
+      run();
+    } else {
+      synchronized (DU.this) {
+        //if an exception was thrown in the last run, rethrow
+        if(duException != null) {
+          IOException tmp = duException;
+          duException = null;
+          throw tmp;
+        }
+      }
+    }
+    
+    return used.longValue();
+  }
+
+  /**
+   * @return the path of which we're keeping track of disk usage
+   */
+  public String getDirPath() {
+    return dirPath;
+  }
+  
+  /**
+   * Start the disk usage checking thread.
+   */
+  public void start() {
+    //only start the thread if the interval is sane
+    if(refreshInterval > 0) {
+      refreshUsed = new Thread(new DURefreshThread(), 
+          "refreshUsed-"+dirPath);
+      refreshUsed.setDaemon(true);
+      refreshUsed.start();
+    }
+  }
+  
+  /**
+   * Shut down the refreshing thread.
+   */
+  public void shutdown() {
+    this.shouldRun = false;
+    
+    if(this.refreshUsed != null) {
+      this.refreshUsed.interrupt();
+    }
+  }
+  
+  public String toString() {
+    return
+      "du -sk " + dirPath +"\n" +
+      used + "\t" + dirPath;
+  }
+
+  protected String[] getExecString() {
+    return new String[] {"du", "-sk", dirPath};
+  }
+  
+  protected void parseExecResult(BufferedReader lines) throws IOException {
+    String line = lines.readLine();
+    if (line == null) {
+      throw new IOException("Expecting a line not the end of stream");
+    }
+    String[] tokens = line.split("\t");
+    if(tokens.length == 0) {
+      throw new IOException("Illegal du output");
+    }
+    this.used.set(Long.parseLong(tokens[0])*1024);
+  }
+
+  public static void main(String[] args) throws Exception {
+    String path = ".";
+    if (args.length > 0) {
+      path = args[0];
+    }
+
+    System.out.println(new DU(new File(path), new Configuration()).toString());
+  }
+}

+ 62 - 0
src/java/org/apache/hadoop/fs/FSDataInputStream.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.*;
+
+/** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
+ * and buffers input through a {@link BufferedInputStream}. */
+public class FSDataInputStream extends DataInputStream
+    implements Seekable, PositionedReadable {
+
+  public FSDataInputStream(InputStream in)
+    throws IOException {
+    super(in);
+    if( !(in instanceof Seekable) || !(in instanceof PositionedReadable) ) {
+      throw new IllegalArgumentException(
+          "In is not an instance of Seekable or PositionedReadable");
+    }
+  }
+  
+  public synchronized void seek(long desired) throws IOException {
+    ((Seekable)in).seek(desired);
+  }
+
+  public long getPos() throws IOException {
+    return ((Seekable)in).getPos();
+  }
+  
+  public int read(long position, byte[] buffer, int offset, int length)
+    throws IOException {
+    return ((PositionedReadable)in).read(position, buffer, offset, length);
+  }
+  
+  public void readFully(long position, byte[] buffer, int offset, int length)
+    throws IOException {
+    ((PositionedReadable)in).readFully(position, buffer, offset, length);
+  }
+  
+  public void readFully(long position, byte[] buffer)
+    throws IOException {
+    ((PositionedReadable)in).readFully(position, buffer, 0, buffer.length);
+  }
+  
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    return ((Seekable)in).seekToNewSource(targetPos); 
+  }
+}

+ 100 - 0
src/java/org/apache/hadoop/fs/FSDataOutputStream.java

@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.*;
+
+/** Utility that wraps a {@link OutputStream} in a {@link DataOutputStream},
+ * buffers output through a {@link BufferedOutputStream} and creates a checksum
+ * file. */
+public class FSDataOutputStream extends DataOutputStream implements Syncable {
+  private OutputStream wrappedStream;
+
+  private static class PositionCache extends FilterOutputStream {
+    private FileSystem.Statistics statistics;
+    long position;
+
+    public PositionCache(OutputStream out, 
+                         FileSystem.Statistics stats,
+                         long pos) throws IOException {
+      super(out);
+      statistics = stats;
+      position = pos;
+    }
+
+    public void write(int b) throws IOException {
+      out.write(b);
+      position++;
+      if (statistics != null) {
+        statistics.incrementBytesWritten(1);
+      }
+    }
+    
+    public void write(byte b[], int off, int len) throws IOException {
+      out.write(b, off, len);
+      position += len;                            // update position
+      if (statistics != null) {
+        statistics.incrementBytesWritten(len);
+      }
+    }
+      
+    public long getPos() throws IOException {
+      return position;                            // return cached position
+    }
+    
+    public void close() throws IOException {
+      out.close();
+    }
+  }
+
+  @Deprecated
+  public FSDataOutputStream(OutputStream out) throws IOException {
+    this(out, null);
+  }
+
+  public FSDataOutputStream(OutputStream out, FileSystem.Statistics stats)
+    throws IOException {
+    this(out, stats, 0);
+  }
+
+  public FSDataOutputStream(OutputStream out, FileSystem.Statistics stats,
+                            long startPosition) throws IOException {
+    super(new PositionCache(out, stats, startPosition));
+    wrappedStream = out;
+  }
+  
+  public long getPos() throws IOException {
+    return ((PositionCache)out).getPos();
+  }
+
+  public void close() throws IOException {
+    out.close();         // This invokes PositionCache.close()
+  }
+
+  // Returns the underlying output stream. This is used by unit tests.
+  public OutputStream getWrappedStream() {
+    return wrappedStream;
+  }
+
+  /** {@inheritDoc} */
+  public void sync() throws IOException {
+    if (wrappedStream instanceof Syncable) {
+      ((Syncable)wrappedStream).sync();
+    }
+  }
+}

+ 29 - 0
src/java/org/apache/hadoop/fs/FSError.java

@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+/** Thrown for unexpected filesystem errors, presumed to reflect disk errors
+ * in the native filesystem. */
+public class FSError extends Error {
+  private static final long serialVersionUID = 1L;
+
+  FSError(Throwable cause) {
+    super(cause);
+  }
+}

+ 432 - 0
src/java/org/apache/hadoop/fs/FSInputChecker.java

@@ -0,0 +1,432 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.zip.Checksum;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * This is a generic input stream for verifying checksums for
+ * data before it is read by a user.
+ */
+
+abstract public class FSInputChecker extends FSInputStream {
+  public static final Log LOG 
+  = LogFactory.getLog(FSInputChecker.class);
+  
+  /** The file name from which data is read from */
+  protected Path file;
+  private Checksum sum;
+  private boolean verifyChecksum = true;
+  private byte[] buf;
+  private byte[] checksum;
+  private int pos;
+  private int count;
+  
+  private int numOfRetries;
+  
+  // cached file position
+  private long chunkPos = 0;
+  
+  /** Constructor
+   * 
+   * @param file The name of the file to be read
+   * @param numOfRetries Number of read retries when ChecksumError occurs
+   */
+  protected FSInputChecker( Path file, int numOfRetries) {
+    this.file = file;
+    this.numOfRetries = numOfRetries;
+  }
+  
+  /** Constructor
+   * 
+   * @param file The name of the file to be read
+   * @param numOfRetries Number of read retries when ChecksumError occurs
+   * @param sum the type of Checksum engine
+   * @param chunkSize maximun chunk size
+   * @param checksumSize the number byte of each checksum
+   */
+  protected FSInputChecker( Path file, int numOfRetries, 
+      boolean verifyChecksum, Checksum sum, int chunkSize, int checksumSize ) {
+    this(file, numOfRetries);
+    set(verifyChecksum, sum, chunkSize, checksumSize);
+  }
+  
+  /** Reads in next checksum chunk data into <code>buf</code> at <code>offset</code>
+   * and checksum into <code>checksum</code>.
+   * The method is used for implementing read, therefore, it should be optimized
+   * for sequential reading
+   * @param pos chunkPos
+   * @param buf desitination buffer
+   * @param offset offset in buf at which to store data
+   * @param len maximun number of bytes to read
+   * @return number of bytes read
+   */
+  abstract protected int readChunk(long pos, byte[] buf, int offset, int len,
+      byte[] checksum) throws IOException;
+
+  /** Return position of beginning of chunk containing pos. 
+   *
+   * @param pos a postion in the file
+   * @return the starting position of the chunk which contains the byte
+   */
+  abstract protected long getChunkPosition(long pos);
+
+  /** Return true if there is a need for checksum verification */
+  protected synchronized boolean needChecksum() {
+    return verifyChecksum && sum != null;
+  }
+  
+  /**
+   * Read one checksum-verified byte
+   * 
+   * @return     the next byte of data, or <code>-1</code> if the end of the
+   *             stream is reached.
+   * @exception  IOException  if an I/O error occurs.
+   */
+
+  public synchronized int read() throws IOException {
+    if (pos >= count) {
+      fill();
+      if (pos >= count) {
+        return -1;
+      }
+    }
+    return buf[pos++] & 0xff;
+  }
+  
+  /**
+   * Read checksum verified bytes from this byte-input stream into 
+   * the specified byte array, starting at the given offset.
+   *
+   * <p> This method implements the general contract of the corresponding
+   * <code>{@link InputStream#read(byte[], int, int) read}</code> method of
+   * the <code>{@link InputStream}</code> class.  As an additional
+   * convenience, it attempts to read as many bytes as possible by repeatedly
+   * invoking the <code>read</code> method of the underlying stream.  This
+   * iterated <code>read</code> continues until one of the following
+   * conditions becomes true: <ul>
+   *
+   *   <li> The specified number of bytes have been read,
+   *
+   *   <li> The <code>read</code> method of the underlying stream returns
+   *   <code>-1</code>, indicating end-of-file.
+   *
+   * </ul> If the first <code>read</code> on the underlying stream returns
+   * <code>-1</code> to indicate end-of-file then this method returns
+   * <code>-1</code>.  Otherwise this method returns the number of bytes
+   * actually read.
+   *
+   * @param      b     destination buffer.
+   * @param      off   offset at which to start storing bytes.
+   * @param      len   maximum number of bytes to read.
+   * @return     the number of bytes read, or <code>-1</code> if the end of
+   *             the stream has been reached.
+   * @exception  IOException  if an I/O error occurs.
+   *             ChecksumException if any checksum error occurs
+   */
+  public synchronized int read(byte[] b, int off, int len) throws IOException {
+    // parameter check
+    if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
+      throw new IndexOutOfBoundsException();
+    } else if (len == 0) {
+      return 0;
+    }
+
+    int n = 0;
+    for (;;) {
+      int nread = read1(b, off + n, len - n);
+      if (nread <= 0) 
+        return (n == 0) ? nread : n;
+      n += nread;
+      if (n >= len)
+        return n;
+    }
+  }
+  
+  /**
+   * Fills the buffer with a chunk data. 
+   * No mark is supported.
+   * This method assumes that all data in the buffer has already been read in,
+   * hence pos > count.
+   */
+  private void fill(  ) throws IOException {
+    assert(pos>=count);
+    // fill internal buffer
+    count = readChecksumChunk(buf, 0, buf.length);
+  }
+  
+  /*
+   * Read characters into a portion of an array, reading from the underlying
+   * stream at most once if necessary.
+   */
+  private int read1(byte b[], int off, int len)
+  throws IOException {
+    int avail = count-pos;
+    if( avail <= 0 ) {
+      if(len>=buf.length) {
+        // read a chunk to user buffer directly; avoid one copy
+        int nread = readChecksumChunk(b, off, len);
+        return nread;
+      } else {
+        // read a chunk into the local buffer
+        fill();
+        if( count <= 0 ) {
+          return -1;
+        } else {
+          avail = count;
+        }
+      }
+    }
+    
+    // copy content of the local buffer to the user buffer
+    int cnt = (avail < len) ? avail : len;
+    System.arraycopy(buf, pos, b, off, cnt);
+    pos += cnt;
+    return cnt;    
+  }
+  
+  /* Read up one checksum chunk to array <i>b</i> at pos <i>off</i>
+   * It requires a checksum chunk boundary
+   * in between <cur_pos, cur_pos+len> 
+   * and it stops reading at the boundary or at the end of the stream;
+   * Otherwise an IllegalArgumentException is thrown.
+   * This makes sure that all data read are checksum verified.
+   * 
+   * @param b   the buffer into which the data is read.
+   * @param off the start offset in array <code>b</code>
+   *            at which the data is written.
+   * @param len the maximum number of bytes to read.
+   * @return    the total number of bytes read into the buffer, or
+   *            <code>-1</code> if there is no more data because the end of
+   *            the stream has been reached.
+   * @throws IOException if an I/O error occurs.
+   */ 
+  private int readChecksumChunk(byte b[], int off, int len)
+  throws IOException {
+    // invalidate buffer
+    count = pos = 0;
+          
+    int read = 0;
+    boolean retry = true;
+    int retriesLeft = numOfRetries; 
+    do {
+      retriesLeft--;
+
+      try {
+        read = readChunk(chunkPos, b, off, len, checksum);
+        if( read > 0 ) {
+          if( needChecksum() ) {
+            sum.update(b, off, read);
+            verifySum(chunkPos);
+          }
+          chunkPos += read;
+        } 
+        retry = false;
+      } catch (ChecksumException ce) {
+          LOG.info("Found checksum error: b[" + off + ", " + (off+read) + "]="
+              + StringUtils.byteToHexString(b, off, off + read), ce);
+          if (retriesLeft == 0) {
+            throw ce;
+          }
+          
+          // try a new replica
+          if (seekToNewSource(chunkPos)) {
+            // Since at least one of the sources is different, 
+            // the read might succeed, so we'll retry.
+            seek(chunkPos);
+          } else {
+            // Neither the data stream nor the checksum stream are being read
+            // from different sources, meaning we'll still get a checksum error 
+            // if we try to do the read again.  We throw an exception instead.
+            throw ce;
+          }
+        }
+    } while (retry);
+    return read;
+  }
+  
+  /* verify checksum for the chunk.
+   * @throws ChecksumException if there is a mismatch
+   */
+  private void verifySum(long errPos) throws ChecksumException {
+    long crc = getChecksum();
+    long sumValue = sum.getValue();
+    sum.reset();
+    if (crc != sumValue) {
+      throw new ChecksumException(
+          "Checksum error: "+file+" at "+errPos, errPos);
+    }
+  }
+  
+  /* calculate checksum value */
+  private long getChecksum() {
+    return checksum2long(checksum);
+  }
+
+  /** Convert a checksum byte array to a long */
+  static public long checksum2long(byte[] checksum) {
+    long crc = 0L;
+    for(int i=0; i<checksum.length; i++) {
+      crc |= (0xffL&(long)checksum[i])<<((checksum.length-i-1)*8);
+    }
+    return crc;
+  }
+  
+  @Override
+  public synchronized long getPos() throws IOException {
+    return chunkPos-(count-pos);
+  }
+
+  @Override
+  public synchronized int available() throws IOException {
+    return count-pos;
+  }
+  
+  /**
+   * Skips over and discards <code>n</code> bytes of data from the
+   * input stream.
+   *
+   * <p>This method may skip more bytes than are remaining in the backing
+   * file. This produces no exception and the number of bytes skipped
+   * may include some number of bytes that were beyond the EOF of the
+   * backing file. Attempting to read from the stream after skipping past
+   * the end will result in -1 indicating the end of the file.
+   *
+   *<p>If <code>n</code> is negative, no bytes are skipped.
+   *
+   * @param      n   the number of bytes to be skipped.
+   * @return     the actual number of bytes skipped.
+   * @exception  IOException  if an I/O error occurs.
+   *             ChecksumException if the chunk to skip to is corrupted
+   */
+  public synchronized long skip(long n) throws IOException {
+    if (n <= 0) {
+      return 0;
+    }
+
+    seek(getPos()+n);
+    return n;
+  }
+
+  /**
+   * Seek to the given position in the stream.
+   * The next read() will be from that position.
+   * 
+   * <p>This method may seek past the end of the file.
+   * This produces no exception and an attempt to read from
+   * the stream will result in -1 indicating the end of the file.
+   *
+   * @param      pos   the postion to seek to.
+   * @exception  IOException  if an I/O error occurs.
+   *             ChecksumException if the chunk to seek to is corrupted
+   */
+
+  public synchronized void seek(long pos) throws IOException {
+    if( pos<0 ) {
+      return;
+    }
+    // optimize: check if the pos is in the buffer
+    long start = chunkPos - this.count;
+    if( pos>=start && pos<chunkPos) {
+      this.pos = (int)(pos-start);
+      return;
+    }
+    
+    // reset the current state
+    resetState();
+    
+    // seek to a checksum boundary
+    chunkPos = getChunkPosition(pos);
+    
+    // scan to the desired position
+    int delta = (int)(pos - chunkPos);
+    if( delta > 0) {
+      readFully(this, new byte[delta], 0, delta);
+    }
+  }
+
+  /**
+   * A utility function that tries to read up to <code>len</code> bytes from
+   * <code>stm</code>
+   * 
+   * @param stm    an input stream
+   * @param buf    destiniation buffer
+   * @param offset offset at which to store data
+   * @param len    number of bytes to read
+   * @return actual number of bytes read
+   * @throws IOException if there is any IO error
+   */
+  protected static int readFully(InputStream stm, 
+      byte[] buf, int offset, int len) throws IOException {
+    int n = 0;
+    for (;;) {
+      int nread = stm.read(buf, offset + n, len - n);
+      if (nread <= 0) 
+        return (n == 0) ? nread : n;
+      n += nread;
+      if (n >= len)
+        return n;
+    }
+  }
+  
+  /**
+   * Set the checksum related parameters
+   * @param verifyChecksum whether to verify checksum
+   * @param sum which type of checksum to use
+   * @param maxChunkSize maximun chunk size
+   * @param checksumSize checksum size
+   */
+  final protected synchronized void set(boolean verifyChecksum,
+      Checksum sum, int maxChunkSize, int checksumSize ) {
+    this.verifyChecksum = verifyChecksum;
+    this.sum = sum;
+    this.buf = new byte[maxChunkSize];
+    this.checksum = new byte[checksumSize];
+    this.count = 0;
+    this.pos = 0;
+  }
+
+  final public boolean markSupported() {
+    return false;
+  }
+  
+  final public void mark(int readlimit) {
+  }
+  
+  final public void reset() throws IOException {
+    throw new IOException("mark/reset not supported");
+  }
+  
+
+  /* reset this FSInputChecker's state */
+  private void resetState() {
+    // invalidate buffer
+    count = 0;
+    pos = 0;
+    // reset Checksum
+    if (sum != null) {
+      sum.reset();
+    }
+  }
+}

+ 78 - 0
src/java/org/apache/hadoop/fs/FSInputStream.java

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.*;
+
+/****************************************************************
+ * FSInputStream is a generic old InputStream with a little bit
+ * of RAF-style seek ability.
+ *
+ *****************************************************************/
+public abstract class FSInputStream extends InputStream
+    implements Seekable, PositionedReadable {
+  /**
+   * Seek to the given offset from the start of the file.
+   * The next read() will be from that location.  Can't
+   * seek past the end of the file.
+   */
+  public abstract void seek(long pos) throws IOException;
+
+  /**
+   * Return the current offset from the start of the file
+   */
+  public abstract long getPos() throws IOException;
+
+  /**
+   * Seeks a different copy of the data.  Returns true if 
+   * found a new source, false otherwise.
+   */
+  public abstract boolean seekToNewSource(long targetPos) throws IOException;
+
+  public int read(long position, byte[] buffer, int offset, int length)
+    throws IOException {
+    synchronized (this) {
+      long oldPos = getPos();
+      int nread = -1;
+      try {
+        seek(position);
+        nread = read(buffer, offset, length);
+      } finally {
+        seek(oldPos);
+      }
+      return nread;
+    }
+  }
+    
+  public void readFully(long position, byte[] buffer, int offset, int length)
+    throws IOException {
+    int nread = 0;
+    while (nread < length) {
+      int nbytes = read(position+nread, buffer, offset+nread, length-nread);
+      if (nbytes < 0) {
+        throw new EOFException("End of file reached before reading fully.");
+      }
+      nread += nbytes;
+    }
+  }
+    
+  public void readFully(long position, byte[] buffer)
+    throws IOException {
+    readFully(position, buffer, 0, buffer.length);
+  }
+}

+ 176 - 0
src/java/org/apache/hadoop/fs/FSOutputSummer.java

@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.zip.Checksum;
+
+/**
+ * This is a generic output stream for generating checksums for
+ * data before it is written to the underlying stream
+ */
+
+abstract public class FSOutputSummer extends OutputStream {
+  // data checksum
+  private Checksum sum;
+  // internal buffer for storing data before it is checksumed
+  private byte buf[];
+  // internal buffer for storing checksum
+  private byte checksum[];
+  // The number of valid bytes in the buffer.
+  private int count;
+  
+  protected FSOutputSummer(Checksum sum, int maxChunkSize, int checksumSize) {
+    this.sum = sum;
+    this.buf = new byte[maxChunkSize];
+    this.checksum = new byte[checksumSize];
+    this.count = 0;
+  }
+  
+  /* write the data chunk in <code>b</code> staring at <code>offset</code> with
+   * a length of <code>len</code>, and its checksum
+   */
+  protected abstract void writeChunk(byte[] b, int offset, int len, byte[] checksum)
+  throws IOException;
+
+  /** Write one byte */
+  public synchronized void write(int b) throws IOException {
+    sum.update(b);
+    buf[count++] = (byte)b;
+    if(count == buf.length) {
+      flushBuffer();
+    }
+  }
+
+  /**
+   * Writes <code>len</code> bytes from the specified byte array 
+   * starting at offset <code>off</code> and generate a checksum for
+   * each data chunk.
+   *
+   * <p> This method stores bytes from the given array into this
+   * stream's buffer before it gets checksumed. The buffer gets checksumed 
+   * and flushed to the underlying output stream when all data 
+   * in a checksum chunk are in the buffer.  If the buffer is empty and
+   * requested length is at least as large as the size of next checksum chunk
+   * size, this method will checksum and write the chunk directly 
+   * to the underlying output stream.  Thus it avoids uneccessary data copy.
+   *
+   * @param      b     the data.
+   * @param      off   the start offset in the data.
+   * @param      len   the number of bytes to write.
+   * @exception  IOException  if an I/O error occurs.
+   */
+  public synchronized void write(byte b[], int off, int len)
+  throws IOException {
+    if (off < 0 || len < 0 || off > b.length - len) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+
+    for (int n=0;n<len;n+=write1(b, off+n, len-n)) {
+    }
+  }
+  
+  /**
+   * Write a portion of an array, flushing to the underlying
+   * stream at most once if necessary.
+   */
+  private int write1(byte b[], int off, int len) throws IOException {
+    if(count==0 && len>=buf.length) {
+      // local buffer is empty and user data has one chunk
+      // checksum and output data
+      final int length = buf.length;
+      sum.update(b, off, length);
+      writeChecksumChunk(b, off, length, false);
+      return length;
+    }
+    
+    // copy user data to local buffer
+    int bytesToCopy = buf.length-count;
+    bytesToCopy = (len<bytesToCopy) ? len : bytesToCopy;
+    sum.update(b, off, bytesToCopy);
+    System.arraycopy(b, off, buf, count, bytesToCopy);
+    count += bytesToCopy;
+    if (count == buf.length) {
+      // local buffer is full
+      flushBuffer();
+    } 
+    return bytesToCopy;
+  }
+
+  /* Forces any buffered output bytes to be checksumed and written out to
+   * the underlying output stream. 
+   */
+  protected synchronized void flushBuffer() throws IOException {
+    flushBuffer(false);
+  }
+
+  /* Forces any buffered output bytes to be checksumed and written out to
+   * the underlying output stream.  If keep is true, then the state of 
+   * this object remains intact.
+   */
+  protected synchronized void flushBuffer(boolean keep) throws IOException {
+    if (count != 0) {
+      int chunkLen = count;
+      count = 0;
+      writeChecksumChunk(buf, 0, chunkLen, keep);
+      if (keep) {
+        count = chunkLen;
+      }
+    }
+  }
+  
+  /** Generate checksum for the data chunk and output data chunk & checksum
+   * to the underlying output stream. If keep is true then keep the
+   * current checksum intact, do not reset it.
+   */
+  private void writeChecksumChunk(byte b[], int off, int len, boolean keep)
+  throws IOException {
+    int tempChecksum = (int)sum.getValue();
+    if (!keep) {
+      sum.reset();
+    }
+    int2byte(tempChecksum, checksum);
+    writeChunk(b, off, len, checksum);
+  }
+
+  /**
+   * Converts a checksum integer value to a byte stream
+   */
+  static public byte[] convertToByteStream(Checksum sum, int checksumSize) {
+    return int2byte((int)sum.getValue(), new byte[checksumSize]);
+  }
+
+  static byte[] int2byte(int integer, byte[] bytes) {
+    bytes[0] = (byte)((integer >>> 24) & 0xFF);
+    bytes[1] = (byte)((integer >>> 16) & 0xFF);
+    bytes[2] = (byte)((integer >>>  8) & 0xFF);
+    bytes[3] = (byte)((integer >>>  0) & 0xFF);
+    return bytes;
+  }
+
+  /**
+   * Resets existing buffer with a new one of the specified size.
+   */
+  protected synchronized void resetChecksumChunk(int size) {
+    sum.reset();
+    this.buf = new byte[size];
+    this.count = 0;
+  }
+}

+ 53 - 0
src/java/org/apache/hadoop/fs/FileChecksum.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.io.Writable;
+
+/** An abstract class representing file checksums for files. */
+public abstract class FileChecksum implements Writable {
+  /** The checksum algorithm name */ 
+  public abstract String getAlgorithmName();
+
+  /** The length of the checksum in bytes */ 
+  public abstract int getLength();
+
+  /** The value of the checksum in bytes */ 
+  public abstract byte[] getBytes();
+
+  /** Return true if both the algorithms and the values are the same. */
+  public boolean equals(Object other) {
+    if (other == this) {
+      return true;
+    }
+    if (other == null || !(other instanceof FileChecksum)) {
+      return false;
+    }
+
+    final FileChecksum that = (FileChecksum)other;
+    return this.getAlgorithmName().equals(that.getAlgorithmName())
+      && Arrays.equals(this.getBytes(), that.getBytes());
+  }
+  
+  /** {@inheritDoc} */
+  public int hashCode() {
+    return getAlgorithmName().hashCode() ^ Arrays.hashCode(getBytes());
+  }
+}

+ 252 - 0
src/java/org/apache/hadoop/fs/FileStatus.java

@@ -0,0 +1,252 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/** Interface that represents the client side information for a file.
+ */
+public class FileStatus implements Writable, Comparable {
+
+  private Path path;
+  private long length;
+  private boolean isdir;
+  private short block_replication;
+  private long blocksize;
+  private long modification_time;
+  private long access_time;
+  private FsPermission permission;
+  private String owner;
+  private String group;
+  
+  public FileStatus() { this(0, false, 0, 0, 0, 0, null, null, null, null); }
+  
+  //We should deprecate this soon?
+  public FileStatus(long length, boolean isdir, int block_replication,
+                    long blocksize, long modification_time, Path path) {
+
+    this(length, isdir, block_replication, blocksize, modification_time,
+         0, null, null, null, path);
+  }
+  
+  public FileStatus(long length, boolean isdir, int block_replication,
+                    long blocksize, long modification_time, long access_time,
+                    FsPermission permission, String owner, String group, 
+                    Path path) {
+    this.length = length;
+    this.isdir = isdir;
+    this.block_replication = (short)block_replication;
+    this.blocksize = blocksize;
+    this.modification_time = modification_time;
+    this.access_time = access_time;
+    this.permission = (permission == null) ? 
+                      FsPermission.getDefault() : permission;
+    this.owner = (owner == null) ? "" : owner;
+    this.group = (group == null) ? "" : group;
+    this.path = path;
+  }
+
+  /* 
+   * @return the length of this file, in blocks
+   */
+  public long getLen() {
+    return length;
+  }
+
+  /**
+   * Is this a directory?
+   * @return true if this is a directory
+   */
+  public boolean isDir() {
+    return isdir;
+  }
+
+  /**
+   * Get the block size of the file.
+   * @return the number of bytes
+   */
+  public long getBlockSize() {
+    return blocksize;
+  }
+
+  /**
+   * Get the replication factor of a file.
+   * @return the replication factor of a file.
+   */
+  public short getReplication() {
+    return block_replication;
+  }
+
+  /**
+   * Get the modification time of the file.
+   * @return the modification time of file in milliseconds since January 1, 1970 UTC.
+   */
+  public long getModificationTime() {
+    return modification_time;
+  }
+
+  /**
+   * Get the access time of the file.
+   * @return the access time of file in milliseconds since January 1, 1970 UTC.
+   */
+  public long getAccessTime() {
+    return access_time;
+  }
+
+  /**
+   * Get FsPermission associated with the file.
+   * @return permssion. If a filesystem does not have a notion of permissions
+   *         or if permissions could not be determined, then default 
+   *         permissions equivalent of "rwxrwxrwx" is returned.
+   */
+  public FsPermission getPermission() {
+    return permission;
+  }
+  
+  /**
+   * Get the owner of the file.
+   * @return owner of the file. The string could be empty if there is no
+   *         notion of owner of a file in a filesystem or if it could not 
+   *         be determined (rare).
+   */
+  public String getOwner() {
+    return owner;
+  }
+  
+  /**
+   * Get the group associated with the file.
+   * @return group for the file. The string could be empty if there is no
+   *         notion of group of a file in a filesystem or if it could not 
+   *         be determined (rare).
+   */
+  public String getGroup() {
+    return group;
+  }
+  
+  public Path getPath() {
+    return path;
+  }
+
+  /* These are provided so that these values could be loaded lazily 
+   * by a filesystem (e.g. local file system).
+   */
+  
+  /**
+   * Sets permission.
+   * @param permission if permission is null, default value is set
+   */
+  protected void setPermission(FsPermission permission) {
+    this.permission = (permission == null) ? 
+                      FsPermission.getDefault() : permission;
+  }
+  
+  /**
+   * Sets owner.
+   * @param owner if it is null, default value is set
+   */  
+  protected void setOwner(String owner) {
+    this.owner = (owner == null) ? "" : owner;
+  }
+  
+  /**
+   * Sets group.
+   * @param group if it is null, default value is set
+   */  
+  protected void setGroup(String group) {
+    this.group = (group == null) ? "" :  group;
+  }
+
+  //////////////////////////////////////////////////
+  // Writable
+  //////////////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, getPath().toString());
+    out.writeLong(length);
+    out.writeBoolean(isdir);
+    out.writeShort(block_replication);
+    out.writeLong(blocksize);
+    out.writeLong(modification_time);
+    out.writeLong(access_time);
+    permission.write(out);
+    Text.writeString(out, owner);
+    Text.writeString(out, group);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    String strPath = Text.readString(in);
+    this.path = new Path(strPath);
+    this.length = in.readLong();
+    this.isdir = in.readBoolean();
+    this.block_replication = in.readShort();
+    blocksize = in.readLong();
+    modification_time = in.readLong();
+    access_time = in.readLong();
+    permission.readFields(in);
+    owner = Text.readString(in);
+    group = Text.readString(in);
+  }
+
+  /**
+   * Compare this object to another object
+   * 
+   * @param   o the object to be compared.
+   * @return  a negative integer, zero, or a positive integer as this object
+   *   is less than, equal to, or greater than the specified object.
+   * 
+   * @throws ClassCastException if the specified object's is not of 
+   *         type FileStatus
+   */
+  public int compareTo(Object o) {
+    FileStatus other = (FileStatus)o;
+    return this.getPath().compareTo(other.getPath());
+  }
+  
+  /** Compare if this object is equal to another object
+   * @param   o the object to be compared.
+   * @return  true if two file status has the same path name; false if not.
+   */
+  public boolean equals(Object o) {
+    if (o == null) {
+      return false;
+    }
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof FileStatus)) {
+      return false;
+    }
+    FileStatus other = (FileStatus)o;
+    return this.getPath().equals(other.getPath());
+  }
+  
+  /**
+   * Returns a hash code value for the object, which is defined as
+   * the hash code of the path name.
+   *
+   * @return  a hash code value for the path name.
+   */
+  public int hashCode() {
+    return getPath().hashCode();
+  }
+}

+ 1648 - 0
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -0,0 +1,1648 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.Closeable;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/****************************************************************
+ * An abstract base class for a fairly generic filesystem.  It
+ * may be implemented as a distributed filesystem, or as a "local"
+ * one that reflects the locally-connected disk.  The local version
+ * exists for small Hadoop instances and for testing.
+ *
+ * <p>
+ *
+ * All user code that may potentially use the Hadoop Distributed
+ * File System should be written to use a FileSystem object.  The
+ * Hadoop DFS is a multi-machine system that appears as a single
+ * disk.  It's useful because of its fault tolerance and potentially
+ * very large capacity.
+ * 
+ * <p>
+ * The local implementation is {@link LocalFileSystem} and distributed
+ * implementation is DistributedFileSystem.
+ *****************************************************************/
+public abstract class FileSystem extends Configured implements Closeable {
+  private static final String FS_DEFAULT_NAME_KEY = "fs.default.name";
+
+  public static final Log LOG = LogFactory.getLog(FileSystem.class);
+
+  /** FileSystem cache */
+  private static final Cache CACHE = new Cache();
+
+  /** The key this instance is stored under in the cache. */
+  private Cache.Key key;
+
+  /** Recording statistics per a FileSystem class */
+  private static final Map<Class<? extends FileSystem>, Statistics> 
+    statisticsTable =
+      new IdentityHashMap<Class<? extends FileSystem>, Statistics>();
+  
+  /**
+   * The statistics for this file system.
+   */
+  protected Statistics statistics;
+
+  /**
+   * A cache of files that should be deleted when filsystem is closed
+   * or the JVM is exited.
+   */
+  private Set<Path> deleteOnExit = new TreeSet<Path>();
+
+  /** Returns the configured filesystem implementation.*/
+  public static FileSystem get(Configuration conf) throws IOException {
+    return get(getDefaultUri(conf), conf);
+  }
+  
+  /** Get the default filesystem URI from a configuration.
+   * @param conf the configuration to access
+   * @return the uri of the default filesystem
+   */
+  public static URI getDefaultUri(Configuration conf) {
+    return URI.create(fixName(conf.get(FS_DEFAULT_NAME_KEY, "file:///")));
+  }
+
+  /** Set the default filesystem URI in a configuration.
+   * @param conf the configuration to alter
+   * @param uri the new default filesystem uri
+   */
+  public static void setDefaultUri(Configuration conf, URI uri) {
+    conf.set(FS_DEFAULT_NAME_KEY, uri.toString());
+  }
+
+  /** Set the default filesystem URI in a configuration.
+   * @param conf the configuration to alter
+   * @param uri the new default filesystem uri
+   */
+  public static void setDefaultUri(Configuration conf, String uri) {
+    setDefaultUri(conf, URI.create(fixName(uri)));
+  }
+
+  /** Called after a new FileSystem instance is constructed.
+   * @param name a uri whose authority section names the host, port, etc.
+   *   for this FileSystem
+   * @param conf the configuration
+   */
+  public void initialize(URI name, Configuration conf) throws IOException {
+    statistics = getStatistics(name.getScheme(), getClass());    
+  }
+
+  /** Returns a URI whose scheme and authority identify this FileSystem.*/
+  public abstract URI getUri();
+  
+  /** Update old-format filesystem names, for back-compatibility.  This should
+   * eventually be replaced with a checkName() method that throws an exception
+   * for old-format names. */ 
+  private static String fixName(String name) {
+    // convert old-format name to new-format name
+    if (name.equals("local")) {         // "local" is now "file:///".
+      LOG.warn("\"local\" is a deprecated filesystem name."
+               +" Use \"file:///\" instead.");
+      name = "file:///";
+    } else if (name.indexOf('/')==-1) {   // unqualified is "hdfs://"
+      LOG.warn("\""+name+"\" is a deprecated filesystem name."
+               +" Use \"hdfs://"+name+"/\" instead.");
+      name = "hdfs://"+name;
+    }
+    return name;
+  }
+
+  /**
+   * Get the local file syste
+   * @param conf the configuration to configure the file system with
+   * @return a LocalFileSystem
+   */
+  public static LocalFileSystem getLocal(Configuration conf)
+    throws IOException {
+    return (LocalFileSystem)get(LocalFileSystem.NAME, conf);
+  }
+
+  /** Returns the FileSystem for this URI's scheme and authority.  The scheme
+   * of the URI determines a configuration property name,
+   * <tt>fs.<i>scheme</i>.class</tt> whose value names the FileSystem class.
+   * The entire URI is passed to the FileSystem instance's initialize method.
+   */
+  public static FileSystem get(URI uri, Configuration conf) throws IOException {
+    String scheme = uri.getScheme();
+    String authority = uri.getAuthority();
+
+    if (scheme == null) {                       // no scheme: use default FS
+      return get(conf);
+    }
+
+    if (authority == null) {                       // no authority
+      URI defaultUri = getDefaultUri(conf);
+      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
+          && defaultUri.getAuthority() != null) {  // & default has authority
+        return get(defaultUri, conf);              // return default
+      }
+    }
+
+    return CACHE.get(uri, conf);
+  }
+
+  /** Returns the FileSystem for this URI's scheme and authority.  The scheme
+   * of the URI determines a configuration property name,
+   * <tt>fs.<i>scheme</i>.class</tt> whose value names the FileSystem class.
+   * The entire URI is passed to the FileSystem instance's initialize method.
+   * This always returns a new FileSystem object.
+   */
+  public static FileSystem newInstance(URI uri, Configuration conf) throws IOException {
+    String scheme = uri.getScheme();
+    String authority = uri.getAuthority();
+
+    if (scheme == null) {                       // no scheme: use default FS
+      return newInstance(conf);
+    }
+
+    if (authority == null) {                       // no authority
+      URI defaultUri = getDefaultUri(conf);
+      if (scheme.equals(defaultUri.getScheme())    // if scheme matches default
+          && defaultUri.getAuthority() != null) {  // & default has authority
+        return newInstance(defaultUri, conf);              // return default
+      }
+    }
+    return CACHE.getUnique(uri, conf);
+  }
+
+  /** Returns a unique configured filesystem implementation.
+   * This always returns a new FileSystem object. */
+  public static FileSystem newInstance(Configuration conf) throws IOException {
+    return newInstance(getDefaultUri(conf), conf);
+  }
+
+  /**
+   * Get a unique local file system object
+   * @param conf the configuration to configure the file system with
+   * @return a LocalFileSystem
+   * This always returns a new FileSystem object.
+   */
+  public static LocalFileSystem newInstanceLocal(Configuration conf)
+    throws IOException {
+    return (LocalFileSystem)newInstance(LocalFileSystem.NAME, conf);
+  }
+
+  private static class ClientFinalizer extends Thread {
+    public synchronized void run() {
+      try {
+        FileSystem.closeAll();
+      } catch (IOException e) {
+        LOG.info("FileSystem.closeAll() threw an exception:\n" + e);
+      }
+    }
+  }
+  private static final ClientFinalizer clientFinalizer = new ClientFinalizer();
+
+  /**
+   * Close all cached filesystems. Be sure those filesystems are not
+   * used anymore.
+   * 
+   * @throws IOException
+   */
+  public static void closeAll() throws IOException {
+    CACHE.closeAll();
+  }
+
+  /** Make sure that a path specifies a FileSystem. */
+  public Path makeQualified(Path path) {
+    checkPath(path);
+    return path.makeQualified(this);
+  }
+    
+  /** create a file with the provided permission
+   * The permission of the file is set to be the provided permission as in
+   * setPermission, not permission&~umask
+   * 
+   * It is implemented using two RPCs. It is understood that it is inefficient,
+   * but the implementation is thread-safe. The other option is to change the
+   * value of umask in configuration to be 0, but it is not thread-safe.
+   * 
+   * @param fs file system handle
+   * @param file the name of the file to be created
+   * @param permission the permission of the file
+   * @return an output stream
+   * @throws IOException
+   */
+  public static FSDataOutputStream create(FileSystem fs,
+      Path file, FsPermission permission) throws IOException {
+    // create the file with default permission
+    FSDataOutputStream out = fs.create(file);
+    // set its permission to the supplied one
+    fs.setPermission(file, permission);
+    return out;
+  }
+
+  /** create a directory with the provided permission
+   * The permission of the directory is set to be the provided permission as in
+   * setPermission, not permission&~umask
+   * 
+   * @see #create(FileSystem, Path, FsPermission)
+   * 
+   * @param fs file system handle
+   * @param dir the name of the directory to be created
+   * @param permission the permission of the directory
+   * @return true if the directory creation succeeds; false otherwise
+   * @throws IOException
+   */
+  public static boolean mkdirs(FileSystem fs, Path dir, FsPermission permission)
+  throws IOException {
+    // create the directory using the default permission
+    boolean result = fs.mkdirs(dir);
+    // set its permission to be the supplied one
+    fs.setPermission(dir, permission);
+    return result;
+  }
+
+  ///////////////////////////////////////////////////////////////
+  // FileSystem
+  ///////////////////////////////////////////////////////////////
+
+  protected FileSystem() {
+    super(null);
+  }
+
+  /** Check that a Path belongs to this FileSystem. */
+  protected void checkPath(Path path) {
+    URI uri = path.toUri();
+    if (uri.getScheme() == null)                // fs is relative 
+      return;
+    String thisScheme = this.getUri().getScheme();
+    String thatScheme = uri.getScheme();
+    String thisAuthority = this.getUri().getAuthority();
+    String thatAuthority = uri.getAuthority();
+    //authority and scheme are not case sensitive
+    if (thisScheme.equalsIgnoreCase(thatScheme)) {// schemes match
+      if (thisAuthority == thatAuthority ||       // & authorities match
+          (thisAuthority != null && 
+           thisAuthority.equalsIgnoreCase(thatAuthority)))
+        return;
+
+      if (thatAuthority == null &&                // path's authority is null
+          thisAuthority != null) {                // fs has an authority
+        URI defaultUri = getDefaultUri(getConf()); // & is the conf default 
+        if (thisScheme.equalsIgnoreCase(defaultUri.getScheme()) &&
+            thisAuthority.equalsIgnoreCase(defaultUri.getAuthority()))
+          return;
+        try {                                     // or the default fs's uri
+          defaultUri = get(getConf()).getUri();
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        if (thisScheme.equalsIgnoreCase(defaultUri.getScheme()) &&
+            thisAuthority.equalsIgnoreCase(defaultUri.getAuthority()))
+          return;
+      }
+    }
+    throw new IllegalArgumentException("Wrong FS: "+path+
+                                       ", expected: "+this.getUri());
+  }
+
+  /**
+   * Return an array containing hostnames, offset and size of 
+   * portions of the given file.  For a nonexistent 
+   * file or regions, null will be returned.
+   *
+   * This call is most helpful with DFS, where it returns 
+   * hostnames of machines that contain the given file.
+   *
+   * The FileSystem will simply return an elt containing 'localhost'.
+   */
+  public BlockLocation[] getFileBlockLocations(FileStatus file, 
+      long start, long len) throws IOException {
+    if (file == null) {
+      return null;
+    }
+
+    if ( (start<0) || (len < 0) ) {
+      throw new IllegalArgumentException("Invalid start or len parameter");
+    }
+
+    if (file.getLen() < start) {
+      return new BlockLocation[0];
+
+    }
+    String[] name = { "localhost:50010" };
+    String[] host = { "localhost" };
+    return new BlockLocation[] { new BlockLocation(name, host, 0, file.getLen()) };
+  }
+  
+  /**
+   * Opens an FSDataInputStream at the indicated Path.
+   * @param f the file name to open
+   * @param bufferSize the size of the buffer to be used.
+   */
+  public abstract FSDataInputStream open(Path f, int bufferSize)
+    throws IOException;
+    
+  /**
+   * Opens an FSDataInputStream at the indicated Path.
+   * @param f the file to open
+   */
+  public FSDataInputStream open(Path f) throws IOException {
+    return open(f, getConf().getInt("io.file.buffer.size", 4096));
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path.
+   * Files are overwritten by default.
+   */
+  public FSDataOutputStream create(Path f) throws IOException {
+    return create(f, true);
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path.
+   */
+  public FSDataOutputStream create(Path f, boolean overwrite)
+    throws IOException {
+    return create(f, overwrite, 
+                  getConf().getInt("io.file.buffer.size", 4096),
+                  getDefaultReplication(),
+                  getDefaultBlockSize());
+  }
+
+  /**
+   * Create an FSDataOutputStream at the indicated Path with write-progress
+   * reporting.
+   * Files are overwritten by default.
+   */
+  public FSDataOutputStream create(Path f, Progressable progress) throws IOException {
+    return create(f, true, 
+                  getConf().getInt("io.file.buffer.size", 4096),
+                  getDefaultReplication(),
+                  getDefaultBlockSize(), progress);
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path.
+   * Files are overwritten by default.
+   */
+  public FSDataOutputStream create(Path f, short replication)
+    throws IOException {
+    return create(f, true, 
+                  getConf().getInt("io.file.buffer.size", 4096),
+                  replication,
+                  getDefaultBlockSize());
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path with write-progress
+   * reporting.
+   * Files are overwritten by default.
+   */
+  public FSDataOutputStream create(Path f, short replication, Progressable progress)
+    throws IOException {
+    return create(f, true, 
+                  getConf().getInt("io.file.buffer.size", 4096),
+                  replication,
+                  getDefaultBlockSize(), progress);
+  }
+
+    
+  /**
+   * Opens an FSDataOutputStream at the indicated Path.
+   * @param f the file name to open
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @param bufferSize the size of the buffer to be used.
+   */
+  public FSDataOutputStream create(Path f, 
+                                   boolean overwrite,
+                                   int bufferSize
+                                   ) throws IOException {
+    return create(f, overwrite, bufferSize, 
+                  getDefaultReplication(),
+                  getDefaultBlockSize());
+  }
+    
+  /**
+   * Opens an FSDataOutputStream at the indicated Path with write-progress
+   * reporting.
+   * @param f the file name to open
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @param bufferSize the size of the buffer to be used.
+   */
+  public FSDataOutputStream create(Path f, 
+                                   boolean overwrite,
+                                   int bufferSize,
+                                   Progressable progress
+                                   ) throws IOException {
+    return create(f, overwrite, bufferSize, 
+                  getDefaultReplication(),
+                  getDefaultBlockSize(), progress);
+  }
+    
+    
+  /**
+   * Opens an FSDataOutputStream at the indicated Path.
+   * @param f the file name to open
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @param bufferSize the size of the buffer to be used.
+   * @param replication required block replication for the file. 
+   */
+  public FSDataOutputStream create(Path f, 
+                                   boolean overwrite,
+                                   int bufferSize,
+                                   short replication,
+                                   long blockSize
+                                   ) throws IOException {
+    return create(f, overwrite, bufferSize, replication, blockSize, null);
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path with write-progress
+   * reporting.
+   * @param f the file name to open
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @param bufferSize the size of the buffer to be used.
+   * @param replication required block replication for the file. 
+   */
+  public FSDataOutputStream create(Path f,
+                                            boolean overwrite,
+                                            int bufferSize,
+                                            short replication,
+                                            long blockSize,
+                                            Progressable progress
+                                            ) throws IOException {
+    return this.create(f, FsPermission.getDefault(),
+        overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path with write-progress
+   * reporting.
+   * @param f the file name to open
+   * @param permission
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @param bufferSize the size of the buffer to be used.
+   * @param replication required block replication for the file.
+   * @param blockSize
+   * @param progress
+   * @throws IOException
+   * @see #setPermission(Path, FsPermission)
+   */
+  public abstract FSDataOutputStream create(Path f,
+      FsPermission permission,
+      boolean overwrite,
+      int bufferSize,
+      short replication,
+      long blockSize,
+      Progressable progress) throws IOException;
+
+  /**
+   * Creates the given Path as a brand-new zero-length file.  If
+   * create fails, or if it already existed, return false.
+   */
+  public boolean createNewFile(Path f) throws IOException {
+    if (exists(f)) {
+      return false;
+    } else {
+      create(f, false, getConf().getInt("io.file.buffer.size", 4096)).close();
+      return true;
+    }
+  }
+
+  /**
+   * Append to an existing file (optional operation).
+   * Same as append(f, getConf().getInt("io.file.buffer.size", 4096), null)
+   * @param f the existing file to be appended.
+   * @throws IOException
+   */
+  public FSDataOutputStream append(Path f) throws IOException {
+    return append(f, getConf().getInt("io.file.buffer.size", 4096), null);
+  }
+  /**
+   * Append to an existing file (optional operation).
+   * Same as append(f, bufferSize, null).
+   * @param f the existing file to be appended.
+   * @param bufferSize the size of the buffer to be used.
+   * @throws IOException
+   */
+  public FSDataOutputStream append(Path f, int bufferSize) throws IOException {
+    return append(f, bufferSize, null);
+  }
+
+  /**
+   * Append to an existing file (optional operation).
+   * @param f the existing file to be appended.
+   * @param bufferSize the size of the buffer to be used.
+   * @param progress for reporting progress if it is not null.
+   * @throws IOException
+   */
+  public abstract FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException;
+
+  /**
+   * Set replication for an existing file.
+   * 
+   * @param src file name
+   * @param replication new replication
+   * @throws IOException
+   * @return true if successful;
+   *         false if file does not exist or is a directory
+   */
+  public boolean setReplication(Path src, short replication)
+    throws IOException {
+    return true;
+  }
+
+  /**
+   * Renames Path src to Path dst.  Can take place on local fs
+   * or remote DFS.
+   */
+  public abstract boolean rename(Path src, Path dst) throws IOException;
+    
+  /** Delete a file.
+   *
+   * @param f the path to delete.
+   * @param recursive if path is a directory and set to 
+   * true, the directory is deleted else throws an exception. In
+   * case of a file the recursive can be set to either true or false. 
+   * @return  true if delete is successful else false. 
+   * @throws IOException
+   */
+  public abstract boolean delete(Path f, boolean recursive) throws IOException;
+
+  /**
+   * Mark a path to be deleted when FileSystem is closed.
+   * When the JVM shuts down,
+   * all FileSystem objects will be closed automatically.
+   * Then,
+   * the marked path will be deleted as a result of closing the FileSystem.
+   *
+   * The path has to exist in the file system.
+   * 
+   * @param f the path to delete.
+   * @return  true if deleteOnExit is successful, otherwise false.
+   * @throws IOException
+   */
+  public boolean deleteOnExit(Path f) throws IOException {
+    if (!exists(f)) {
+      return false;
+    }
+    synchronized (deleteOnExit) {
+      deleteOnExit.add(f);
+    }
+    return true;
+  }
+
+  /**
+   * Delete all files that were marked as delete-on-exit. This recursively
+   * deletes all files in the specified paths.
+   */
+  protected void processDeleteOnExit() {
+    synchronized (deleteOnExit) {
+      for (Iterator<Path> iter = deleteOnExit.iterator(); iter.hasNext();) {
+        Path path = iter.next();
+        try {
+          delete(path, true);
+        }
+        catch (IOException e) {
+          LOG.info("Ignoring failure to deleteOnExit for path " + path);
+        }
+        iter.remove();
+      }
+    }
+  }
+  
+  /** Check if exists.
+   * @param f source file
+   */
+  public boolean exists(Path f) throws IOException {
+    try {
+      return getFileStatus(f) != null;
+    } catch (FileNotFoundException e) {
+      return false;
+    }
+  }
+
+  /** True iff the named path is a directory.
+   * Note: Avoid using this method. Instead reuse the FileStatus 
+   * returned by getFileStatus() or listStatus() methods.
+   */
+  public boolean isDirectory(Path f) throws IOException {
+    try {
+      return getFileStatus(f).isDir();
+    } catch (FileNotFoundException e) {
+      return false;               // f does not exist
+    }
+  }
+
+  /** True iff the named path is a regular file.
+   * Note: Avoid using this method. Instead reuse the FileStatus 
+   * returned by getFileStatus() or listStatus() methods.
+   */
+  public boolean isFile(Path f) throws IOException {
+    try {
+      return !getFileStatus(f).isDir();
+    } catch (FileNotFoundException e) {
+      return false;               // f does not exist
+    }
+  }
+    
+  /** Return the {@link ContentSummary} of a given {@link Path}. */
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    FileStatus status = getFileStatus(f);
+    if (!status.isDir()) {
+      // f is a file
+      return new ContentSummary(status.getLen(), 1, 0);
+    }
+    // f is a directory
+    long[] summary = {0, 0, 1};
+    for(FileStatus s : listStatus(f)) {
+      ContentSummary c = s.isDir() ? getContentSummary(s.getPath()) :
+                                     new ContentSummary(s.getLen(), 1, 0);
+      summary[0] += c.getLength();
+      summary[1] += c.getFileCount();
+      summary[2] += c.getDirectoryCount();
+    }
+    return new ContentSummary(summary[0], summary[1], summary[2]);
+  }
+
+  final private static PathFilter DEFAULT_FILTER = new PathFilter() {
+      public boolean accept(Path file) {
+        return true;
+      }     
+    };
+    
+  /**
+   * List the statuses of the files/directories in the given path if the path is
+   * a directory.
+   * 
+   * @param f
+   *          given path
+   * @return the statuses of the files/directories in the given patch
+   * @throws IOException
+   */
+  public abstract FileStatus[] listStatus(Path f) throws 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 {
+    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]);
+        }
+      }
+    }
+  }
+
+  /**
+   * Filter files/directories in the given path using the user-supplied path
+   * filter.
+   * 
+   * @param f
+   *          a path name
+   * @param filter
+   *          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
+   */
+  public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
+    ArrayList<FileStatus> results = new ArrayList<FileStatus>();
+    listStatus(results, f, filter);
+    return results.toArray(new FileStatus[results.size()]);
+  }
+
+  /**
+   * Filter files/directories in the given list of paths using default
+   * path filter.
+   * 
+   * @param files
+   *          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
+   */
+  public FileStatus[] listStatus(Path[] files)
+      throws IOException {
+    return listStatus(files, DEFAULT_FILTER);
+  }
+
+  /**
+   * Filter files/directories in the given list of paths using user-supplied
+   * path filter.
+   * 
+   * @param files
+   *          a list of paths
+   * @param filter
+   *          the user-supplied path filter
+   * @return a list of statuses for the files under the given paths after
+   *         applying the filter
+   * @exception IOException
+   */
+  public FileStatus[] listStatus(Path[] files, PathFilter filter)
+      throws IOException {
+    ArrayList<FileStatus> results = new ArrayList<FileStatus>();
+    for (int i = 0; i < files.length; i++) {
+      listStatus(results, files[i], filter);
+    }
+    return results.toArray(new FileStatus[results.size()]);
+  }
+
+  /**
+   * <p>Return all the files that match filePattern and are not checksum
+   * files. Results are sorted by their names.
+   * 
+   * <p>
+   * A filename pattern is composed of <i>regular</i> characters and
+   * <i>special pattern matching</i> characters, which are:
+   *
+   * <dl>
+   *  <dd>
+   *   <dl>
+   *    <p>
+   *    <dt> <tt> ? </tt>
+   *    <dd> Matches any single character.
+   *
+   *    <p>
+   *    <dt> <tt> * </tt>
+   *    <dd> Matches zero or more characters.
+   *
+   *    <p>
+   *    <dt> <tt> [<i>abc</i>] </tt>
+   *    <dd> Matches a single character from character set
+   *     <tt>{<i>a,b,c</i>}</tt>.
+   *
+   *    <p>
+   *    <dt> <tt> [<i>a</i>-<i>b</i>] </tt>
+   *    <dd> Matches a single character from the character range
+   *     <tt>{<i>a...b</i>}</tt>.  Note that character <tt><i>a</i></tt> must be
+   *     lexicographically less than or equal to character <tt><i>b</i></tt>.
+   *
+   *    <p>
+   *    <dt> <tt> [^<i>a</i>] </tt>
+   *    <dd> Matches a single character that is not from character set or range
+   *     <tt>{<i>a</i>}</tt>.  Note that the <tt>^</tt> character must occur
+   *     immediately to the right of the opening bracket.
+   *
+   *    <p>
+   *    <dt> <tt> \<i>c</i> </tt>
+   *    <dd> Removes (escapes) any special meaning of character <i>c</i>.
+   *
+   *    <p>
+   *    <dt> <tt> {ab,cd} </tt>
+   *    <dd> Matches a string from the string set <tt>{<i>ab, cd</i>} </tt>
+   *    
+   *    <p>
+   *    <dt> <tt> {ab,c{de,fh}} </tt>
+   *    <dd> Matches a string from the string set <tt>{<i>ab, cde, cfh</i>}</tt>
+   *
+   *   </dl>
+   *  </dd>
+   * </dl>
+   *
+   * @param pathPattern a regular expression specifying a pth pattern
+
+   * @return an array of paths that match the path pattern
+   * @throws IOException
+   */
+  public FileStatus[] globStatus(Path pathPattern) throws IOException {
+    return globStatus(pathPattern, DEFAULT_FILTER);
+  }
+  
+  /**
+   * Return an array of FileStatus objects whose path names match pathPattern
+   * and is accepted by the user-supplied path filter. Results are sorted by
+   * their path names.
+   * Return null if pathPattern has no glob and the path does not exist.
+   * Return an empty array if pathPattern has a glob and no path matches it. 
+   * 
+   * @param pathPattern
+   *          a regular expression specifying the path pattern
+   * @param filter
+   *          a user-supplied path filter
+   * @return an array of FileStatus objects
+   * @throws IOException if any I/O error occurs when fetching file status
+   */
+  public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
+      throws IOException {
+    String filename = pathPattern.toUri().getPath();
+    List<String> filePatterns = GlobExpander.expand(filename);
+    if (filePatterns.size() == 1) {
+      return globStatusInternal(pathPattern, filter);
+    } else {
+      List<FileStatus> results = new ArrayList<FileStatus>();
+      for (String filePattern : filePatterns) {
+        FileStatus[] files = globStatusInternal(new Path(filePattern), filter);
+        for (FileStatus file : files) {
+          results.add(file);
+        }
+      }
+      return results.toArray(new FileStatus[results.size()]);
+    }
+  }
+
+  private FileStatus[] globStatusInternal(Path pathPattern, PathFilter filter)
+      throws IOException {
+    Path[] parents = new Path[1];
+    int level = 0;
+    String filename = pathPattern.toUri().getPath();
+    
+    // path has only zero component
+    if ("".equals(filename) || Path.SEPARATOR.equals(filename)) {
+      return getFileStatus(new Path[]{pathPattern});
+    }
+
+    // path has at least one component
+    String[] components = filename.split(Path.SEPARATOR);
+    // get the first component
+    if (pathPattern.isAbsolute()) {
+      parents[0] = new Path(Path.SEPARATOR);
+      level = 1;
+    } else {
+      parents[0] = new Path(Path.CUR_DIR);
+    }
+
+    // glob the paths that match the parent path, i.e., [0, components.length-1]
+    boolean[] hasGlob = new boolean[]{false};
+    Path[] parentPaths = globPathsLevel(parents, components, level, hasGlob);
+    FileStatus[] results;
+    if (parentPaths == null || parentPaths.length == 0) {
+      results = null;
+    } else {
+      // Now work on the last component of the path
+      GlobFilter fp = new GlobFilter(components[components.length - 1], filter);
+      if (fp.hasPattern()) { // last component has a pattern
+        // list parent directories and then glob the results
+        results = listStatus(parentPaths, fp);
+        hasGlob[0] = true;
+      } else { // last component does not have a pattern
+        // get all the path names
+        ArrayList<Path> filteredPaths = new ArrayList<Path>(parentPaths.length);
+        for (int i = 0; i < parentPaths.length; i++) {
+          parentPaths[i] = new Path(parentPaths[i],
+            components[components.length - 1]);
+          if (fp.accept(parentPaths[i])) {
+            filteredPaths.add(parentPaths[i]);
+          }
+        }
+        // get all their statuses
+        results = getFileStatus(
+            filteredPaths.toArray(new Path[filteredPaths.size()]));
+      }
+    }
+
+    // Decide if the pathPattern contains a glob or not
+    if (results == null) {
+      if (hasGlob[0]) {
+        results = new FileStatus[0];
+      }
+    } else {
+      if (results.length == 0 ) {
+        if (!hasGlob[0]) {
+          results = null;
+        }
+      } else {
+        Arrays.sort(results);
+      }
+    }
+    return results;
+  }
+
+  /*
+   * For a path of N components, return a list of paths that match the
+   * components [<code>level</code>, <code>N-1</code>].
+   */
+  private Path[] globPathsLevel(Path[] parents, String[] filePattern,
+      int level, boolean[] hasGlob) throws IOException {
+    if (level == filePattern.length - 1)
+      return parents;
+    if (parents == null || parents.length == 0) {
+      return null;
+    }
+    GlobFilter fp = new GlobFilter(filePattern[level]);
+    if (fp.hasPattern()) {
+      parents = FileUtil.stat2Paths(listStatus(parents, fp));
+      hasGlob[0] = true;
+    } else {
+      for (int i = 0; i < parents.length; i++) {
+        parents[i] = new Path(parents[i], filePattern[level]);
+      }
+    }
+    return globPathsLevel(parents, filePattern, level + 1, hasGlob);
+  }
+
+  /* A class that could decide if a string matches the glob or not */
+  private static class GlobFilter implements PathFilter {
+    private PathFilter userFilter = DEFAULT_FILTER;
+    private Pattern regex;
+    private boolean hasPattern = false;
+      
+    /** Default pattern character: Escape any special meaning. */
+    private static final char  PAT_ESCAPE = '\\';
+    /** Default pattern character: Any single character. */
+    private static final char  PAT_ANY = '.';
+    /** Default pattern character: Character set close. */
+    private static final char  PAT_SET_CLOSE = ']';
+      
+    GlobFilter() {
+    }
+      
+    GlobFilter(String filePattern) throws IOException {
+      setRegex(filePattern);
+    }
+      
+    GlobFilter(String filePattern, PathFilter filter) throws IOException {
+      userFilter = filter;
+      setRegex(filePattern);
+    }
+      
+    private boolean isJavaRegexSpecialChar(char pChar) {
+      return pChar == '.' || pChar == '$' || pChar == '(' || pChar == ')' ||
+             pChar == '|' || pChar == '+';
+    }
+    void setRegex(String filePattern) throws IOException {
+      int len;
+      int setOpen;
+      int curlyOpen;
+      boolean setRange;
+
+      StringBuilder fileRegex = new StringBuilder();
+
+      // Validate the pattern
+      len = filePattern.length();
+      if (len == 0)
+        return;
+
+      setOpen = 0;
+      setRange = false;
+      curlyOpen = 0;
+
+      for (int i = 0; i < len; i++) {
+        char pCh;
+          
+        // Examine a single pattern character
+        pCh = filePattern.charAt(i);
+        if (pCh == PAT_ESCAPE) {
+          fileRegex.append(pCh);
+          i++;
+          if (i >= len)
+            error("An escaped character does not present", filePattern, i);
+          pCh = filePattern.charAt(i);
+        } else if (isJavaRegexSpecialChar(pCh)) {
+          fileRegex.append(PAT_ESCAPE);
+        } else if (pCh == '*') {
+          fileRegex.append(PAT_ANY);
+          hasPattern = true;
+        } else if (pCh == '?') {
+          pCh = PAT_ANY;
+          hasPattern = true;
+        } else if (pCh == '{') {
+          fileRegex.append('(');
+          pCh = '(';
+          curlyOpen++;
+          hasPattern = true;
+        } else if (pCh == ',' && curlyOpen > 0) {
+          fileRegex.append(")|");
+          pCh = '(';
+        } else if (pCh == '}' && curlyOpen > 0) {
+          // End of a group
+          curlyOpen--;
+          fileRegex.append(")");
+          pCh = ')';
+        } else if (pCh == '[' && setOpen == 0) {
+          setOpen++;
+          hasPattern = true;
+        } else if (pCh == '^' && setOpen > 0) {
+        } else if (pCh == '-' && setOpen > 0) {
+          // Character set range
+          setRange = true;
+        } else if (pCh == PAT_SET_CLOSE && setRange) {
+          // Incomplete character set range
+          error("Incomplete character set range", filePattern, i);
+        } else if (pCh == PAT_SET_CLOSE && setOpen > 0) {
+          // End of a character set
+          if (setOpen < 2)
+            error("Unexpected end of set", filePattern, i);
+          setOpen = 0;
+        } else if (setOpen > 0) {
+          // Normal character, or the end of a character set range
+          setOpen++;
+          setRange = false;
+        }
+        fileRegex.append(pCh);
+      }
+        
+      // Check for a well-formed pattern
+      if (setOpen > 0 || setRange || curlyOpen > 0) {
+        // Incomplete character set or character range
+        error("Expecting set closure character or end of range, or }", 
+            filePattern, len);
+      }
+      regex = Pattern.compile(fileRegex.toString());
+    }
+      
+    boolean hasPattern() {
+      return hasPattern;
+    }
+      
+    public boolean accept(Path path) {
+      return regex.matcher(path.getName()).matches() && userFilter.accept(path);
+    }
+      
+    private void error(String s, String pattern, int pos) throws IOException {
+      throw new IOException("Illegal file pattern: "
+                            +s+ " for glob "+ pattern + " at " + pos);
+    }
+  }
+    
+  /** Return the current user's home directory in this filesystem.
+   * The default implementation returns "/user/$USER/".
+   */
+  public Path getHomeDirectory() {
+    return new Path("/user/"+System.getProperty("user.name"))
+      .makeQualified(this);
+  }
+
+
+  /**
+   * Set the current working directory for the given file system. All relative
+   * paths will be resolved relative to it.
+   * 
+   * @param new_dir
+   */
+  public abstract void setWorkingDirectory(Path new_dir);
+    
+  /**
+   * Get the current working directory for the given file system
+   * @return the directory pathname
+   */
+  public abstract Path getWorkingDirectory();
+
+  /**
+   * Call {@link #mkdirs(Path, FsPermission)} with default permission.
+   */
+  public boolean mkdirs(Path f) throws IOException {
+    return mkdirs(f, FsPermission.getDefault());
+  }
+
+  /**
+   * Make the given file and all non-existent parents into
+   * directories. Has the semantics of Unix 'mkdir -p'.
+   * Existence of the directory hierarchy is not an error.
+   */
+  public abstract boolean mkdirs(Path f, FsPermission permission
+      ) throws IOException;
+
+  /**
+   * The src file is on the local disk.  Add it to FS at
+   * the given dst name and the source is kept intact afterwards
+   */
+  public void copyFromLocalFile(Path src, Path dst)
+    throws IOException {
+    copyFromLocalFile(false, src, dst);
+  }
+
+  /**
+   * The src files is on the local disk.  Add it to FS at
+   * the given dst name, removing the source afterwards.
+   */
+  public void moveFromLocalFile(Path[] srcs, Path dst)
+    throws IOException {
+    copyFromLocalFile(true, true, srcs, dst);
+  }
+
+  /**
+   * The src file is on the local disk.  Add it to FS at
+   * the given dst name, removing the source afterwards.
+   */
+  public void moveFromLocalFile(Path src, Path dst)
+    throws IOException {
+    copyFromLocalFile(true, src, dst);
+  }
+
+  /**
+   * The src file is on the local disk.  Add it to FS at
+   * the given dst name.
+   * delSrc indicates if the source should be removed
+   */
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    copyFromLocalFile(delSrc, true, src, dst);
+  }
+  
+  /**
+   * The src files are on the local disk.  Add it to FS at
+   * the given dst name.
+   * delSrc indicates if the source should be removed
+   */
+  public void copyFromLocalFile(boolean delSrc, boolean overwrite, 
+                                Path[] srcs, Path dst)
+    throws IOException {
+    Configuration conf = getConf();
+    FileUtil.copy(getLocal(conf), srcs, this, dst, delSrc, overwrite, conf);
+  }
+  
+  /**
+   * The src file is on the local disk.  Add it to FS at
+   * the given dst name.
+   * delSrc indicates if the source should be removed
+   */
+  public void copyFromLocalFile(boolean delSrc, boolean overwrite, 
+                                Path src, Path dst)
+    throws IOException {
+    Configuration conf = getConf();
+    FileUtil.copy(getLocal(conf), src, this, dst, delSrc, overwrite, conf);
+  }
+    
+  /**
+   * The src file is under FS, and the dst is on the local disk.
+   * Copy it from FS control to the local dst name.
+   */
+  public void copyToLocalFile(Path src, Path dst) throws IOException {
+    copyToLocalFile(false, src, dst);
+  }
+    
+  /**
+   * The src file is under FS, and the dst is on the local disk.
+   * Copy it from FS control to the local dst name.
+   * Remove the source afterwards
+   */
+  public void moveToLocalFile(Path src, Path dst) throws IOException {
+    copyToLocalFile(true, src, dst);
+  }
+
+  /**
+   * The src file is under FS, and the dst is on the local disk.
+   * Copy it from FS control to the local dst name.
+   * delSrc indicates if the src will be removed or not.
+   */   
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    FileUtil.copy(this, src, getLocal(getConf()), dst, delSrc, getConf());
+  }
+
+  /**
+   * Returns a local File that the user can write output to.  The caller
+   * provides both the eventual FS target name and the local working
+   * file.  If the FS is local, we write directly into the target.  If
+   * the FS is remote, we write into the tmp local area.
+   */
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    return tmpLocalFile;
+  }
+
+  /**
+   * Called when we're all done writing to the target.  A local FS will
+   * do nothing, because we've written to exactly the right place.  A remote
+   * FS will copy the contents of tmpLocalFile to the correct target at
+   * fsOutputFile.
+   */
+  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    moveFromLocalFile(tmpLocalFile, fsOutputFile);
+  }
+
+  /**
+   * No more filesystem operations are needed.  Will
+   * release any held locks.
+   */
+  public void close() throws IOException {
+    // delete all files that were marked as delete-on-exit.
+    processDeleteOnExit();
+    CACHE.remove(this.key, this);
+  }
+
+  /** Return the total size of all files in the filesystem.*/
+  public long getUsed() throws IOException{
+    long used = 0;
+    FileStatus[] files = listStatus(new Path("/"));
+    for(FileStatus file:files){
+      used += file.getLen();
+    }
+    return used;
+  }
+
+  /** Return the number of bytes that large input files should be optimally
+   * be split into to minimize i/o time. */
+  public long getDefaultBlockSize() {
+    // default to 32MB: large enough to minimize the impact of seeks
+    return getConf().getLong("fs.local.block.size", 32 * 1024 * 1024);
+  }
+    
+  /**
+   * Get the default replication.
+   */
+  public short getDefaultReplication() { return 1; }
+
+  /**
+   * Return a file status object that represents the path.
+   * @param f The path we want information from
+   * @return a FileStatus object
+   * @throws FileNotFoundException when the path does not exist;
+   *         IOException see specific implementation
+   */
+  public abstract FileStatus getFileStatus(Path f) throws IOException;
+
+  /**
+   * Get the checksum of a file.
+   *
+   * @param f The file path
+   * @return The file checksum.  The default return value is null,
+   *  which indicates that no checksum algorithm is implemented
+   *  in the corresponding FileSystem.
+   */
+  public FileChecksum getFileChecksum(Path f) throws IOException {
+    return null;
+  }
+  
+  /**
+   * Set the verify checksum flag. This is only applicable if the 
+   * corresponding FileSystem supports checksum. By default doesn't do anything.
+   * @param verifyChecksum
+   */
+  public void setVerifyChecksum(boolean verifyChecksum) {
+    //doesn't do anything
+  }
+
+  /**
+   * Return a list of file status objects that corresponds to the list of paths
+   * excluding those non-existent paths.
+   * 
+   * @param paths
+   *          the list of paths we want information from
+   * @return a list of FileStatus objects
+   * @throws IOException
+   *           see specific implementation
+   */
+  private FileStatus[] getFileStatus(Path[] paths) throws IOException {
+    if (paths == null) {
+      return null;
+    }
+    ArrayList<FileStatus> results = new ArrayList<FileStatus>(paths.length);
+    for (int i = 0; i < paths.length; i++) {
+      try {
+        results.add(getFileStatus(paths[i]));
+      } catch (FileNotFoundException e) { // do nothing
+      }
+    }
+    return results.toArray(new FileStatus[results.size()]);
+  }
+  
+  /**
+   * Returns a status object describing the use and capacity of the
+   * file system. If the file system has multiple partitions, the
+   * use and capacity of the root partition is reflected.
+   * 
+   * @return a FsStatus object
+   * @throws IOException
+   *           see specific implementation
+   */
+  public FsStatus getStatus() throws IOException {
+    return getStatus(null);
+  }
+
+  /**
+   * Returns a status object describing the use and capacity of the
+   * file system. If the file system has multiple partitions, the
+   * use and capacity of the partition pointed to by the specified
+   * path is reflected.
+   * @param p Path for which status should be obtained. null means
+   * the default partition. 
+   * @return a FsStatus object
+   * @throws IOException
+   *           see specific implementation
+   */
+  public FsStatus getStatus(Path p) throws IOException {
+    return new FsStatus(Long.MAX_VALUE, 0, Long.MAX_VALUE);
+  }
+
+  /**
+   * Set permission of a path.
+   * @param p
+   * @param permission
+   */
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+  }
+
+  /**
+   * Set owner of a path (i.e. a file or a directory).
+   * The parameters username and groupname cannot both be null.
+   * @param p The path
+   * @param username If it is null, the original username remains unchanged.
+   * @param groupname If it is null, the original groupname remains unchanged.
+   */
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+  }
+
+  /**
+   * Set access time of a file
+   * @param p The path
+   * @param mtime Set the modification time of this file.
+   *              The number of milliseconds since Jan 1, 1970. 
+   *              A value of -1 means that this call should not set modification time.
+   * @param atime Set the access time of this file.
+   *              The number of milliseconds since Jan 1, 1970. 
+   *              A value of -1 means that this call should not set access time.
+   */
+  public void setTimes(Path p, long mtime, long atime
+      ) throws IOException {
+  }
+
+  private static FileSystem createFileSystem(URI uri, Configuration conf
+      ) throws IOException {
+    Class<?> clazz = conf.getClass("fs." + uri.getScheme() + ".impl", null);
+    if (clazz == null) {
+      throw new IOException("No FileSystem for scheme: " + uri.getScheme());
+    }
+    FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
+    fs.initialize(uri, conf);
+    return fs;
+  }
+
+  /** Caching FileSystem objects */
+  static class Cache {
+    private final Map<Key, FileSystem> map = new HashMap<Key, FileSystem>();
+
+    /** A variable that makes all objects in the cache unique */
+    private static AtomicLong unique = new AtomicLong(1);
+
+    synchronized FileSystem get(URI uri, Configuration conf) throws IOException{
+      Key key = new Key(uri, conf);
+      return getInternal(uri, conf, key);
+    }
+
+    /** The objects inserted into the cache using this method are all unique */
+    synchronized FileSystem getUnique(URI uri, Configuration conf) throws IOException{
+      Key key = new Key(uri, conf, unique.getAndIncrement());
+      return getInternal(uri, conf, key);
+    }
+
+    private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{
+      FileSystem fs = map.get(key);
+      if (fs == null) {
+        fs = createFileSystem(uri, conf);
+        if (map.isEmpty() && !clientFinalizer.isAlive()) {
+          Runtime.getRuntime().addShutdownHook(clientFinalizer);
+        }
+        fs.key = key;
+        map.put(key, fs);
+      }
+      return fs;
+    }
+
+    synchronized void remove(Key key, FileSystem fs) {
+      if (map.containsKey(key) && fs == map.get(key)) {
+        map.remove(key);
+        if (map.isEmpty() && !clientFinalizer.isAlive()) {
+          if (!Runtime.getRuntime().removeShutdownHook(clientFinalizer)) {
+            LOG.info("Could not cancel cleanup thread, though no " +
+                     "FileSystems are open");
+          }
+        }
+      }
+    }
+
+    synchronized void closeAll() throws IOException {
+      List<IOException> exceptions = new ArrayList<IOException>();
+      for(; !map.isEmpty(); ) {
+        Map.Entry<Key, FileSystem> e = map.entrySet().iterator().next();
+        final Key key = e.getKey();
+        final FileSystem fs = e.getValue();
+
+        //remove from cache
+        remove(key, fs);
+
+        if (fs != null) {
+          try {
+            fs.close();
+          }
+          catch(IOException ioe) {
+            exceptions.add(ioe);
+          }
+        }
+      }
+
+      if (!exceptions.isEmpty()) {
+        throw MultipleIOException.createIOException(exceptions);
+      }
+    }
+
+    /** FileSystem.Cache.Key */
+    static class Key {
+      final String scheme;
+      final String authority;
+      final String username;
+      final long unique;   // an artificial way to make a key unique
+
+      Key(URI uri, Configuration conf) throws IOException {
+        this(uri, conf, 0);
+      }
+
+      Key(URI uri, Configuration conf, long unique) throws IOException {
+        scheme = uri.getScheme()==null?"":uri.getScheme().toLowerCase();
+        authority = uri.getAuthority()==null?"":uri.getAuthority().toLowerCase();
+        this.unique = unique;
+        UserGroupInformation ugi = UserGroupInformation.readFrom(conf);
+        if (ugi == null) {
+          try {
+            ugi = UserGroupInformation.login(conf);
+          } catch(LoginException e) {
+            LOG.warn("uri=" + uri, e);
+          }
+        }
+        username = ugi == null? null: ugi.getUserName();
+      }
+
+      /** {@inheritDoc} */
+      public int hashCode() {
+        return (scheme + authority + username).hashCode() + (int)unique;
+      }
+
+      static boolean isEqual(Object a, Object b) {
+        return a == b || (a != null && a.equals(b));        
+      }
+
+      /** {@inheritDoc} */
+      public boolean equals(Object obj) {
+        if (obj == this) {
+          return true;
+        }
+        if (obj != null && obj instanceof Key) {
+          Key that = (Key)obj;
+          return isEqual(this.scheme, that.scheme)
+                 && isEqual(this.authority, that.authority)
+                 && isEqual(this.username, that.username)
+                 && (this.unique == that.unique);
+        }
+        return false;        
+      }
+
+      /** {@inheritDoc} */
+      public String toString() {
+        return username + "@" + scheme + "://" + authority;        
+      }
+    }
+  }
+  
+  public static final class Statistics {
+    private final String scheme;
+    private AtomicLong bytesRead = new AtomicLong();
+    private AtomicLong bytesWritten = new AtomicLong();
+    
+    public Statistics(String scheme) {
+      this.scheme = scheme;
+    }
+
+    /**
+     * Increment the bytes read in the statistics
+     * @param newBytes the additional bytes read
+     */
+    public void incrementBytesRead(long newBytes) {
+      bytesRead.getAndAdd(newBytes);
+    }
+    
+    /**
+     * Increment the bytes written in the statistics
+     * @param newBytes the additional bytes written
+     */
+    public void incrementBytesWritten(long newBytes) {
+      bytesWritten.getAndAdd(newBytes);
+    }
+    
+    /**
+     * Get the total number of bytes read
+     * @return the number of bytes
+     */
+    public long getBytesRead() {
+      return bytesRead.get();
+    }
+    
+    /**
+     * Get the total number of bytes written
+     * @return the number of bytes
+     */
+    public long getBytesWritten() {
+      return bytesWritten.get();
+    }
+    
+    public String toString() {
+      return bytesRead + " bytes read and " + bytesWritten + 
+             " bytes written";
+    }
+    
+    /**
+     * Reset the counts of bytes to 0.
+     */
+    public void reset() {
+      bytesWritten.set(0);
+      bytesRead.set(0);
+    }
+    
+    /**
+     * Get the uri scheme associated with this statistics object.
+     * @return the schema associated with this set of statistics
+     */
+    public String getScheme() {
+      return scheme;
+    }
+  }
+  
+  /**
+   * Get the Map of Statistics object indexed by URI Scheme.
+   * @return a Map having a key as URI scheme and value as Statistics object
+   * @deprecated use {@link #getAllStatistics} instead
+   */
+  @Deprecated
+  public static synchronized Map<String, Statistics> getStatistics() {
+    Map<String, Statistics> result = new HashMap<String, Statistics>();
+    for(Statistics stat: statisticsTable.values()) {
+      result.put(stat.getScheme(), stat);
+    }
+    return result;
+  }
+
+  /**
+   * Return the FileSystem classes that have Statistics
+   */
+  public static synchronized List<Statistics> getAllStatistics() {
+    return new ArrayList<Statistics>(statisticsTable.values());
+  }
+  
+  /**
+   * Get the statistics for a particular file system
+   * @param cls the class to lookup
+   * @return a statistics object
+   */
+  public static synchronized 
+  Statistics getStatistics(String scheme, Class<? extends FileSystem> cls) {
+    Statistics result = statisticsTable.get(cls);
+    if (result == null) {
+      result = new Statistics(scheme);
+      statisticsTable.put(cls, result);
+    }
+    return result;
+  }
+  
+  public static synchronized void clearStatistics() {
+    for(Statistics stat: statisticsTable.values()) {
+      stat.reset();
+    }
+  }
+
+  public static synchronized
+  void printStatistics() throws IOException {
+    for (Map.Entry<Class<? extends FileSystem>, Statistics> pair: 
+            statisticsTable.entrySet()) {
+      System.out.println("  FileSystem " + pair.getKey().getName() + 
+                         ": " + pair.getValue());
+    }
+  }
+}

+ 794 - 0
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -0,0 +1,794 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.*;
+import java.util.Enumeration;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.mortbay.log.Log;
+
+/**
+ * A collection of file-processing util methods
+ */
+public class FileUtil {
+  /**
+   * convert an array of FileStatus to an array of Path
+   * 
+   * @param stats
+   *          an array of FileStatus objects
+   * @return an array of paths corresponding to the input
+   */
+  public static Path[] stat2Paths(FileStatus[] stats) {
+    if (stats == null)
+      return null;
+    Path[] ret = new Path[stats.length];
+    for (int i = 0; i < stats.length; ++i) {
+      ret[i] = stats[i].getPath();
+    }
+    return ret;
+  }
+
+  /**
+   * convert an array of FileStatus to an array of Path.
+   * If stats if null, return path
+   * @param stats
+   *          an array of FileStatus objects
+   * @param path
+   *          default path to return in stats is null
+   * @return an array of paths corresponding to the input
+   */
+  public static Path[] stat2Paths(FileStatus[] stats, Path path) {
+    if (stats == null)
+      return new Path[]{path};
+    else
+      return stat2Paths(stats);
+  }
+  
+  /**
+   * Delete a directory and all its contents.  If
+   * we return false, the directory may be partially-deleted.
+   */
+  public static boolean fullyDelete(File dir) throws IOException {
+    File contents[] = dir.listFiles();
+    if (contents != null) {
+      for (int i = 0; i < contents.length; i++) {
+        if (contents[i].isFile()) {
+          if (!contents[i].delete()) {
+            return false;
+          }
+        } else {
+          //try deleting the directory
+          // this might be a symlink
+          boolean b = false;
+          b = contents[i].delete();
+          if (b){
+            //this was indeed a symlink or an empty directory
+            continue;
+          }
+          // if not an empty directory or symlink let
+          // fullydelete handle it.
+          if (!fullyDelete(contents[i])) {
+            return false;
+          }
+        }
+      }
+    }
+    return dir.delete();
+  }
+
+  /**
+   * Recursively delete a directory.
+   * 
+   * @param fs {@link FileSystem} on which the path is present
+   * @param dir directory to recursively delete 
+   * @throws IOException
+   * @deprecated Use {@link FileSystem#delete(Path, boolean)}
+   */
+  @Deprecated
+  public static void fullyDelete(FileSystem fs, Path dir) 
+  throws IOException {
+    fs.delete(dir, true);
+  }
+
+  //
+  // If the destination is a subdirectory of the source, then
+  // generate exception
+  //
+  private static void checkDependencies(FileSystem srcFS, 
+                                        Path src, 
+                                        FileSystem dstFS, 
+                                        Path dst)
+                                        throws IOException {
+    if (srcFS == dstFS) {
+      String srcq = src.makeQualified(srcFS).toString() + Path.SEPARATOR;
+      String dstq = dst.makeQualified(dstFS).toString() + Path.SEPARATOR;
+      if (dstq.startsWith(srcq)) {
+        if (srcq.length() == dstq.length()) {
+          throw new IOException("Cannot copy " + src + " to itself.");
+        } else {
+          throw new IOException("Cannot copy " + src + " to its subdirectory " +
+                                dst);
+        }
+      }
+    }
+  }
+
+  /** Copy files between FileSystems. */
+  public static boolean copy(FileSystem srcFS, Path src, 
+                             FileSystem dstFS, Path dst, 
+                             boolean deleteSource,
+                             Configuration conf) throws IOException {
+    return copy(srcFS, src, dstFS, dst, deleteSource, true, conf);
+  }
+
+  public static boolean copy(FileSystem srcFS, Path[] srcs, 
+                             FileSystem dstFS, Path dst,
+                             boolean deleteSource, 
+                             boolean overwrite, Configuration conf)
+                             throws IOException {
+    boolean gotException = false;
+    boolean returnVal = true;
+    StringBuffer exceptions = new StringBuffer();
+
+    if (srcs.length == 1)
+      return copy(srcFS, srcs[0], dstFS, dst, deleteSource, overwrite, conf);
+
+    // Check if dest is directory
+    if (!dstFS.exists(dst)) {
+      throw new IOException("`" + dst +"': specified destination directory " +
+                            "doest not exist");
+    } else {
+      FileStatus sdst = dstFS.getFileStatus(dst);
+      if (!sdst.isDir()) 
+        throw new IOException("copying multiple files, but last argument `" +
+                              dst + "' is not a directory");
+    }
+
+    for (Path src : srcs) {
+      try {
+        if (!copy(srcFS, src, dstFS, dst, deleteSource, overwrite, conf))
+          returnVal = false;
+      } catch (IOException e) {
+        gotException = true;
+        exceptions.append(e.getMessage());
+        exceptions.append("\n");
+      }
+    }
+    if (gotException) {
+      throw new IOException(exceptions.toString());
+    }
+    return returnVal;
+  }
+
+  /** Copy files between FileSystems. */
+  public static boolean copy(FileSystem srcFS, Path src, 
+                             FileSystem dstFS, Path dst, 
+                             boolean deleteSource,
+                             boolean overwrite,
+                             Configuration conf) throws IOException {
+    FileStatus fileStatus = srcFS.getFileStatus(src);
+    return copy(srcFS, fileStatus, dstFS, dst, deleteSource, overwrite, conf);
+  }
+
+  /** Copy files between FileSystems. */
+  private static boolean copy(FileSystem srcFS, FileStatus srcStatus,
+                              FileSystem dstFS, Path dst,
+                              boolean deleteSource,
+                              boolean overwrite,
+                              Configuration conf) throws IOException {
+    Path src = srcStatus.getPath();
+    dst = checkDest(src.getName(), dstFS, dst, overwrite);
+    if (srcStatus.isDir()) {
+      checkDependencies(srcFS, src, dstFS, dst);
+      if (!dstFS.mkdirs(dst)) {
+        return false;
+      }
+      FileStatus contents[] = srcFS.listStatus(src);
+      for (int i = 0; i < contents.length; i++) {
+        copy(srcFS, contents[i], dstFS,
+             new Path(dst, contents[i].getPath().getName()),
+             deleteSource, overwrite, conf);
+      }
+    } else {
+      InputStream in=null;
+      OutputStream out = null;
+      try {
+        in = srcFS.open(src);
+        out = dstFS.create(dst, overwrite);
+        IOUtils.copyBytes(in, out, conf, true);
+      } catch (IOException e) {
+        IOUtils.closeStream(out);
+        IOUtils.closeStream(in);
+        throw e;
+      }
+    }
+    if (deleteSource) {
+      return srcFS.delete(src, true);
+    } else {
+      return true;
+    }
+  
+  }
+
+  /** Copy all files in a directory to one output file (merge). */
+  public static boolean copyMerge(FileSystem srcFS, Path srcDir, 
+                                  FileSystem dstFS, Path dstFile, 
+                                  boolean deleteSource,
+                                  Configuration conf, String addString) throws IOException {
+    dstFile = checkDest(srcDir.getName(), dstFS, dstFile, false);
+
+    if (!srcFS.getFileStatus(srcDir).isDir())
+      return false;
+   
+    OutputStream out = dstFS.create(dstFile);
+    
+    try {
+      FileStatus contents[] = srcFS.listStatus(srcDir);
+      for (int i = 0; i < contents.length; i++) {
+        if (!contents[i].isDir()) {
+          InputStream in = srcFS.open(contents[i].getPath());
+          try {
+            IOUtils.copyBytes(in, out, conf, false);
+            if (addString!=null)
+              out.write(addString.getBytes("UTF-8"));
+                
+          } finally {
+            in.close();
+          } 
+        }
+      }
+    } finally {
+      out.close();
+    }
+    
+
+    if (deleteSource) {
+      return srcFS.delete(srcDir, true);
+    } else {
+      return true;
+    }
+  }  
+  
+  /** Copy local files to a FileSystem. */
+  public static boolean copy(File src,
+                             FileSystem dstFS, Path dst,
+                             boolean deleteSource,
+                             Configuration conf) throws IOException {
+    dst = checkDest(src.getName(), dstFS, dst, false);
+
+    if (src.isDirectory()) {
+      if (!dstFS.mkdirs(dst)) {
+        return false;
+      }
+      File contents[] = src.listFiles();
+      for (int i = 0; i < contents.length; i++) {
+        copy(contents[i], dstFS, new Path(dst, contents[i].getName()),
+             deleteSource, conf);
+      }
+    } else if (src.isFile()) {
+      InputStream in = null;
+      OutputStream out =null;
+      try {
+        in = new FileInputStream(src);
+        out = dstFS.create(dst);
+        IOUtils.copyBytes(in, out, conf);
+      } catch (IOException e) {
+        IOUtils.closeStream( out );
+        IOUtils.closeStream( in );
+        throw e;
+      }
+    } else {
+      throw new IOException(src.toString() + 
+                            ": No such file or directory");
+    }
+    if (deleteSource) {
+      return FileUtil.fullyDelete(src);
+    } else {
+      return true;
+    }
+  }
+
+  /** Copy FileSystem files to local files. */
+  public static boolean copy(FileSystem srcFS, Path src, 
+                             File dst, boolean deleteSource,
+                             Configuration conf) throws IOException {
+    FileStatus filestatus = srcFS.getFileStatus(src);
+    return copy(srcFS, filestatus, dst, deleteSource, conf);
+  }
+
+  /** Copy FileSystem files to local files. */
+  private static boolean copy(FileSystem srcFS, FileStatus srcStatus,
+                              File dst, boolean deleteSource,
+                              Configuration conf) throws IOException {
+    Path src = srcStatus.getPath();
+    if (srcStatus.isDir()) {
+      if (!dst.mkdirs()) {
+        return false;
+      }
+      FileStatus contents[] = srcFS.listStatus(src);
+      for (int i = 0; i < contents.length; i++) {
+        copy(srcFS, contents[i],
+             new File(dst, contents[i].getPath().getName()),
+             deleteSource, conf);
+      }
+    } else {
+      InputStream in = srcFS.open(src);
+      IOUtils.copyBytes(in, new FileOutputStream(dst), conf);
+    }
+    if (deleteSource) {
+      return srcFS.delete(src, true);
+    } else {
+      return true;
+    }
+  }
+
+  private static Path checkDest(String srcName, FileSystem dstFS, Path dst,
+      boolean overwrite) throws IOException {
+    if (dstFS.exists(dst)) {
+      FileStatus sdst = dstFS.getFileStatus(dst);
+      if (sdst.isDir()) {
+        if (null == srcName) {
+          throw new IOException("Target " + dst + " is a directory");
+        }
+        return checkDest(null, dstFS, new Path(dst, srcName), overwrite);
+      } else if (!overwrite) {
+        throw new IOException("Target " + dst + " already exists");
+      }
+    }
+    return dst;
+  }
+
+  /**
+   * This class is only used on windows to invoke the cygpath command.
+   */
+  private static class CygPathCommand extends Shell {
+    String[] command;
+    String result;
+    CygPathCommand(String path) throws IOException {
+      command = new String[]{"cygpath", "-u", path};
+      run();
+    }
+    String getResult() throws IOException {
+      return result;
+    }
+    protected String[] getExecString() {
+      return command;
+    }
+    protected void parseExecResult(BufferedReader lines) throws IOException {
+      String line = lines.readLine();
+      if (line == null) {
+        throw new IOException("Can't convert '" + command[2] + 
+                              " to a cygwin path");
+      }
+      result = line;
+    }
+  }
+
+  /**
+   * Convert a os-native filename to a path that works for the shell.
+   * @param filename The filename to convert
+   * @return The unix pathname
+   * @throws IOException on windows, there can be problems with the subprocess
+   */
+  public static String makeShellPath(String filename) throws IOException {
+    if (Path.WINDOWS) {
+      return new CygPathCommand(filename).getResult();
+    } else {
+      return filename;
+    }    
+  }
+  
+  /**
+   * Convert a os-native filename to a path that works for the shell.
+   * @param file The filename to convert
+   * @return The unix pathname
+   * @throws IOException on windows, there can be problems with the subprocess
+   */
+  public static String makeShellPath(File file) throws IOException {
+    return makeShellPath(file, false);
+  }
+
+  /**
+   * Convert a os-native filename to a path that works for the shell.
+   * @param file The filename to convert
+   * @param makeCanonicalPath 
+   *          Whether to make canonical path for the file passed
+   * @return The unix pathname
+   * @throws IOException on windows, there can be problems with the subprocess
+   */
+  public static String makeShellPath(File file, boolean makeCanonicalPath) 
+  throws IOException {
+    if (makeCanonicalPath) {
+      return makeShellPath(file.getCanonicalPath());
+    } else {
+      return makeShellPath(file.toString());
+    }
+  }
+
+  /**
+   * Takes an input dir and returns the du on that local directory. Very basic
+   * implementation.
+   * 
+   * @param dir
+   *          The input dir to get the disk space of this local dir
+   * @return The total disk space of the input local directory
+   */
+  public static long getDU(File dir) {
+    long size = 0;
+    if (!dir.exists())
+      return 0;
+    if (!dir.isDirectory()) {
+      return dir.length();
+    } else {
+      size = dir.length();
+      File[] allFiles = dir.listFiles();
+      for (int i = 0; i < allFiles.length; i++) {
+        size = size + getDU(allFiles[i]);
+      }
+      return size;
+    }
+  }
+    
+  /**
+   * Given a File input it will unzip the file in a the unzip directory
+   * passed as the second parameter
+   * @param inFile The zip file as input
+   * @param unzipDir The unzip directory where to unzip the zip file.
+   * @throws IOException
+   */
+  public static void unZip(File inFile, File unzipDir) throws IOException {
+    Enumeration<? extends ZipEntry> entries;
+    ZipFile zipFile = new ZipFile(inFile);
+
+    try {
+      entries = zipFile.entries();
+      while (entries.hasMoreElements()) {
+        ZipEntry entry = entries.nextElement();
+        if (!entry.isDirectory()) {
+          InputStream in = zipFile.getInputStream(entry);
+          try {
+            File file = new File(unzipDir, entry.getName());
+            if (!file.getParentFile().mkdirs()) {           
+              if (!file.getParentFile().isDirectory()) {
+                throw new IOException("Mkdirs failed to create " + 
+                                      file.getParentFile().toString());
+              }
+            }
+            OutputStream out = new FileOutputStream(file);
+            try {
+              byte[] buffer = new byte[8192];
+              int i;
+              while ((i = in.read(buffer)) != -1) {
+                out.write(buffer, 0, i);
+              }
+            } finally {
+              out.close();
+            }
+          } finally {
+            in.close();
+          }
+        }
+      }
+    } finally {
+      zipFile.close();
+    }
+  }
+
+  /**
+   * Given a Tar File as input it will untar the file in a the untar directory
+   * passed as the second parameter
+   * 
+   * This utility will untar ".tar" files and ".tar.gz","tgz" files.
+   *  
+   * @param inFile The tar file as input. 
+   * @param untarDir The untar directory where to untar the tar file.
+   * @throws IOException
+   */
+  public static void unTar(File inFile, File untarDir) throws IOException {
+    if (!untarDir.mkdirs()) {           
+      if (!untarDir.isDirectory()) {
+        throw new IOException("Mkdirs failed to create " + untarDir);
+      }
+    }
+
+    StringBuffer untarCommand = new StringBuffer();
+    boolean gzipped = inFile.toString().endsWith("gz");
+    if (gzipped) {
+      untarCommand.append(" gzip -dc '");
+      untarCommand.append(FileUtil.makeShellPath(inFile));
+      untarCommand.append("' | (");
+    } 
+    untarCommand.append("cd '");
+    untarCommand.append(FileUtil.makeShellPath(untarDir)); 
+    untarCommand.append("' ; ");
+    untarCommand.append("tar -xf ");
+    
+    if (gzipped) {
+      untarCommand.append(" -)");
+    } else {
+      untarCommand.append(FileUtil.makeShellPath(inFile));
+    }
+    String[] shellCmd = { "bash", "-c", untarCommand.toString() };
+    ShellCommandExecutor shexec = new ShellCommandExecutor(shellCmd);
+    shexec.execute();
+    int exitcode = shexec.getExitCode();
+    if (exitcode != 0) {
+      throw new IOException("Error untarring file " + inFile + 
+                  ". Tar process exited with exit code " + exitcode);
+    }
+  }
+
+  /**
+   * Class for creating hardlinks.
+   * Supports Unix, Cygwin, WindXP.
+   *  
+   */
+  public static class HardLink { 
+    enum OSType {
+      OS_TYPE_UNIX, 
+      OS_TYPE_WINXP,
+      OS_TYPE_SOLARIS,
+      OS_TYPE_MAC; 
+    }
+  
+    private static String[] hardLinkCommand;
+    private static String[] getLinkCountCommand;
+    private static OSType osType;
+    
+    static {
+      osType = getOSType();
+      switch(osType) {
+      case OS_TYPE_WINXP:
+        hardLinkCommand = new String[] {"fsutil","hardlink","create", null, null};
+        getLinkCountCommand = new String[] {"stat","-c%h"};
+        break;
+      case OS_TYPE_SOLARIS:
+        hardLinkCommand = new String[] {"ln", null, null};
+        getLinkCountCommand = new String[] {"ls","-l"};
+        break;
+      case OS_TYPE_MAC:
+        hardLinkCommand = new String[] {"ln", null, null};
+        getLinkCountCommand = new String[] {"stat","-f%l"};
+        break;
+      case OS_TYPE_UNIX:
+      default:
+        hardLinkCommand = new String[] {"ln", null, null};
+        getLinkCountCommand = new String[] {"stat","-c%h"};
+      }
+    }
+
+    static private OSType getOSType() {
+      String osName = System.getProperty("os.name");
+      if (osName.indexOf("Windows") >= 0 && 
+          (osName.indexOf("XP") >= 0 || osName.indexOf("2003") >= 0 || osName.indexOf("Vista") >= 0))
+        return OSType.OS_TYPE_WINXP;
+      else if (osName.indexOf("SunOS") >= 0)
+         return OSType.OS_TYPE_SOLARIS;
+      else if (osName.indexOf("Mac") >= 0)
+         return OSType.OS_TYPE_MAC;
+      else
+        return OSType.OS_TYPE_UNIX;
+    }
+    
+    /**
+     * Creates a hardlink 
+     */
+    public static void createHardLink(File target, 
+                                      File linkName) throws IOException {
+      int len = hardLinkCommand.length;
+      if (osType == OSType.OS_TYPE_WINXP) {
+       hardLinkCommand[len-1] = target.getCanonicalPath();
+       hardLinkCommand[len-2] = linkName.getCanonicalPath();
+      } else {
+       hardLinkCommand[len-2] = makeShellPath(target, true);
+       hardLinkCommand[len-1] = makeShellPath(linkName, true);
+      }
+      // execute shell command
+      Process process = Runtime.getRuntime().exec(hardLinkCommand);
+      try {
+        if (process.waitFor() != 0) {
+          String errMsg = new BufferedReader(new InputStreamReader(
+                                                                   process.getInputStream())).readLine();
+          if (errMsg == null)  errMsg = "";
+          String inpMsg = new BufferedReader(new InputStreamReader(
+                                                                   process.getErrorStream())).readLine();
+          if (inpMsg == null)  inpMsg = "";
+          throw new IOException(errMsg + inpMsg);
+        }
+      } catch (InterruptedException e) {
+        throw new IOException(StringUtils.stringifyException(e));
+      } finally {
+        process.destroy();
+      }
+    }
+
+    /**
+     * Retrieves the number of links to the specified file.
+     */
+    public static int getLinkCount(File fileName) throws IOException {
+      int len = getLinkCountCommand.length;
+      String[] cmd = new String[len + 1];
+      for (int i = 0; i < len; i++) {
+        cmd[i] = getLinkCountCommand[i];
+      }
+      cmd[len] = fileName.toString();
+      String inpMsg = "";
+      String errMsg = "";
+      int exitValue = -1;
+      BufferedReader in = null;
+      BufferedReader err = null;
+
+      // execute shell command
+      Process process = Runtime.getRuntime().exec(cmd);
+      try {
+        exitValue = process.waitFor();
+        in = new BufferedReader(new InputStreamReader(
+                                    process.getInputStream()));
+        inpMsg = in.readLine();
+        if (inpMsg == null)  inpMsg = "";
+        
+        err = new BufferedReader(new InputStreamReader(
+                                     process.getErrorStream()));
+        errMsg = err.readLine();
+        if (errMsg == null)  errMsg = "";
+        if (exitValue != 0) {
+          throw new IOException(inpMsg + errMsg);
+        }
+        if (getOSType() == OSType.OS_TYPE_SOLARIS) {
+          String[] result = inpMsg.split("\\s+");
+          return Integer.parseInt(result[1]);
+        } else {
+          return Integer.parseInt(inpMsg);
+        }
+      } catch (NumberFormatException e) {
+        throw new IOException(StringUtils.stringifyException(e) + 
+                              inpMsg + errMsg +
+                              " on file:" + fileName);
+      } catch (InterruptedException e) {
+        throw new IOException(StringUtils.stringifyException(e) + 
+                              inpMsg + errMsg +
+                              " on file:" + fileName);
+      } finally {
+        process.destroy();
+        if (in != null) in.close();
+        if (err != null) err.close();
+      }
+    }
+  }
+
+  /**
+   * Create a soft link between a src and destination
+   * only on a local disk. HDFS does not support this
+   * @param target the target for symlink 
+   * @param linkname the symlink
+   * @return value returned by the command
+   */
+  public static int symLink(String target, String linkname) throws IOException{
+    String cmd = "ln -s " + target + " " + linkname;
+    Process p = Runtime.getRuntime().exec(cmd, null);
+    int returnVal = -1;
+    try{
+      returnVal = p.waitFor();
+    } catch(InterruptedException e){
+      //do nothing as of yet
+    }
+    return returnVal;
+  }
+  
+  /**
+   * Change the permissions on a filename.
+   * @param filename the name of the file to change
+   * @param perm the permission string
+   * @return the exit code from the command
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public static int chmod(String filename, String perm
+                          ) throws IOException, InterruptedException {
+    return chmod(filename, perm, false);
+  }
+
+  /**
+   * Change the permissions on a file / directory, recursively, if
+   * needed.
+   * @param filename name of the file whose permissions are to change
+   * @param perm permission string
+   * @param recursive true, if permissions should be changed recursively
+   * @return the exit code from the command.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public static int chmod(String filename, String perm, boolean recursive)
+                            throws IOException, InterruptedException {
+    StringBuffer cmdBuf = new StringBuffer();
+    cmdBuf.append("chmod ");
+    if (recursive) {
+      cmdBuf.append("-R ");
+    }
+    cmdBuf.append(perm).append(" ");
+    cmdBuf.append(filename);
+    String[] shellCmd = {"bash", "-c" ,cmdBuf.toString()};
+    ShellCommandExecutor shExec = new ShellCommandExecutor(shellCmd);
+    try {
+      shExec.execute();
+    }catch(Exception e) {
+      if(Log.isDebugEnabled()) {
+        Log.debug("Error while changing permission : " + filename 
+            +" Exception: " + StringUtils.stringifyException(e));
+      }
+    }
+    return shExec.getExitCode();
+  }
+  
+  /**
+   * Create a tmp file for a base file.
+   * @param basefile the base file of the tmp
+   * @param prefix file name prefix of tmp
+   * @param isDeleteOnExit if true, the tmp will be deleted when the VM exits
+   * @return a newly created tmp file
+   * @exception IOException If a tmp file cannot created
+   * @see java.io.File#createTempFile(String, String, File)
+   * @see java.io.File#deleteOnExit()
+   */
+  public static final File createLocalTempFile(final File basefile,
+                                               final String prefix,
+                                               final boolean isDeleteOnExit)
+    throws IOException {
+    File tmp = File.createTempFile(prefix + basefile.getName(),
+                                   "", basefile.getParentFile());
+    if (isDeleteOnExit) {
+      tmp.deleteOnExit();
+    }
+    return tmp;
+  }
+
+  /**
+   * Move the src file to the name specified by target.
+   * @param src the source file
+   * @param target the target file
+   * @exception IOException If this operation fails
+   */
+  public static void replaceFile(File src, File target) throws IOException {
+    /* renameTo() has two limitations on Windows platform.
+     * src.renameTo(target) fails if
+     * 1) If target already exists OR
+     * 2) If target is already open for reading/writing.
+     */
+    if (!src.renameTo(target)) {
+      int retries = 5;
+      while (target.exists() && !target.delete() && retries-- >= 0) {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          throw new IOException("replaceFile interrupted.");
+        }
+      }
+      if (!src.renameTo(target)) {
+        throw new IOException("Unable to rename " + src +
+                              " to " + target);
+      }
+    }
+  }
+}

+ 278 - 0
src/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -0,0 +1,278 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.*;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/****************************************************************
+ * A <code>FilterFileSystem</code> contains
+ * some other file system, which it uses as
+ * its  basic file system, possibly transforming
+ * the data along the way or providing  additional
+ * functionality. The class <code>FilterFileSystem</code>
+ * itself simply overrides all  methods of
+ * <code>FileSystem</code> with versions that
+ * pass all requests to the contained  file
+ * system. Subclasses of <code>FilterFileSystem</code>
+ * may further override some of  these methods
+ * and may also provide additional methods
+ * and fields.
+ *
+ *****************************************************************/
+public class FilterFileSystem extends FileSystem {
+  
+  protected FileSystem fs;
+  
+  /*
+   * so that extending classes can define it
+   */
+  public FilterFileSystem() {
+  }
+  
+  public FilterFileSystem(FileSystem fs) {
+    this.fs = fs;
+    this.statistics = fs.statistics;
+  }
+
+  /** Called after a new FileSystem instance is constructed.
+   * @param name a uri whose authority section names the host, port, etc.
+   *   for this FileSystem
+   * @param conf the configuration
+   */
+  public void initialize(URI name, Configuration conf) throws IOException {
+    fs.initialize(name, conf);
+  }
+
+  /** Returns a URI whose scheme and authority identify this FileSystem.*/
+  public URI getUri() {
+    return fs.getUri();
+  }
+
+  /** Make sure that a path specifies a FileSystem. */
+  public Path makeQualified(Path path) {
+    return fs.makeQualified(path);
+  }
+  
+  ///////////////////////////////////////////////////////////////
+  // FileSystem
+  ///////////////////////////////////////////////////////////////
+
+  /** Check that a Path belongs to this FileSystem. */
+  protected void checkPath(Path path) {
+    fs.checkPath(path);
+  }
+
+  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+    long len) throws IOException {
+      return fs.getFileBlockLocations(file, start, len);
+  }
+  
+  /**
+   * Opens an FSDataInputStream at the indicated Path.
+   * @param f the file name to open
+   * @param bufferSize the size of the buffer to be used.
+   */
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    return fs.open(f, bufferSize);
+  }
+
+  /** {@inheritDoc} */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    return fs.append(f, bufferSize, progress);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return fs.create(f, permission,
+        overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  /**
+   * Set replication for an existing file.
+   * 
+   * @param src file name
+   * @param replication new replication
+   * @throws IOException
+   * @return true if successful;
+   *         false if file does not exist or is a directory
+   */
+  public boolean setReplication(Path src, short replication) throws IOException {
+    return fs.setReplication(src, replication);
+  }
+  
+  /**
+   * Renames Path src to Path dst.  Can take place on local fs
+   * or remote DFS.
+   */
+  public boolean rename(Path src, Path dst) throws IOException {
+    return fs.rename(src, dst);
+  }
+  
+  /** Delete a file */
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    return fs.delete(f, recursive);
+  }
+  
+  /** List files in a directory. */
+  public FileStatus[] listStatus(Path f) throws IOException {
+    return fs.listStatus(f);
+  }
+  
+  public Path getHomeDirectory() {
+    return fs.getHomeDirectory();
+  }
+
+
+  /**
+   * Set the current working directory for the given file system. All relative
+   * paths will be resolved relative to it.
+   * 
+   * @param newDir
+   */
+  public void setWorkingDirectory(Path newDir) {
+    fs.setWorkingDirectory(newDir);
+  }
+  
+  /**
+   * Get the current working directory for the given file system
+   * 
+   * @return the directory pathname
+   */
+  public Path getWorkingDirectory() {
+    return fs.getWorkingDirectory();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public FsStatus getStatus(Path p) throws IOException {
+    return fs.getStatus(p);
+  }
+  
+  /** {@inheritDoc} */
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    return fs.mkdirs(f, permission);
+  }
+
+  /**
+   * The src file is on the local disk.  Add it to FS at
+   * the given dst name.
+   * delSrc indicates if the source should be removed
+   */
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    fs.copyFromLocalFile(delSrc, src, dst);
+  }
+  
+  /**
+   * The src file is under FS, and the dst is on the local disk.
+   * Copy it from FS control to the local dst name.
+   * delSrc indicates if the src will be removed or not.
+   */   
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    fs.copyToLocalFile(delSrc, src, dst);
+  }
+  
+  /**
+   * Returns a local File that the user can write output to.  The caller
+   * provides both the eventual FS target name and the local working
+   * file.  If the FS is local, we write directly into the target.  If
+   * the FS is remote, we write into the tmp local area.
+   */
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    return fs.startLocalOutput(fsOutputFile, tmpLocalFile);
+  }
+
+  /**
+   * Called when we're all done writing to the target.  A local FS will
+   * do nothing, because we've written to exactly the right place.  A remote
+   * FS will copy the contents of tmpLocalFile to the correct target at
+   * fsOutputFile.
+   */
+  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    fs.completeLocalOutput(fsOutputFile, tmpLocalFile);
+  }
+
+  /** Return the number of bytes that large input files should be optimally
+   * be split into to minimize i/o time. */
+  public long getDefaultBlockSize() {
+    return fs.getDefaultBlockSize();
+  }
+  
+  /**
+   * Get the default replication.
+   */
+  public short getDefaultReplication() {
+    return fs.getDefaultReplication();
+  }
+
+  /**
+   * Get file status.
+   */
+  public FileStatus getFileStatus(Path f) throws IOException {
+    return fs.getFileStatus(f);
+  }
+
+  /** {@inheritDoc} */
+  public FileChecksum getFileChecksum(Path f) throws IOException {
+    return fs.getFileChecksum(f);
+  }
+  
+  /** {@inheritDoc} */
+  public void setVerifyChecksum(boolean verifyChecksum) {
+    fs.setVerifyChecksum(verifyChecksum);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return fs.getConf();
+  }
+  
+  @Override
+  public void close() throws IOException {
+    super.close();
+    fs.close();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+    fs.setOwner(p, username, groupname);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+    fs.setPermission(p, permission);
+  }
+}

+ 1925 - 0
src/java/org/apache/hadoop/fs/FsShell.java

@@ -0,0 +1,1925 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.fs.shell.Count;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.StringUtils;
+
+/** Provide command line access to a FileSystem. */
+public class FsShell extends Configured implements Tool {
+
+  protected FileSystem fs;
+  private Trash trash;
+  public static final SimpleDateFormat dateForm = 
+    new SimpleDateFormat("yyyy-MM-dd HH:mm");
+  protected static final SimpleDateFormat modifFmt =
+    new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+  static final int BORDER = 2;
+  static {
+    modifFmt.setTimeZone(TimeZone.getTimeZone("UTC"));
+  }
+  static final String SETREP_SHORT_USAGE="-setrep [-R] [-w] <rep> <path/file>";
+  static final String GET_SHORT_USAGE = "-get [-ignoreCrc] [-crc] <src> <localdst>";
+  static final String COPYTOLOCAL_SHORT_USAGE = GET_SHORT_USAGE.replace(
+      "-get", "-copyToLocal");
+  static final String TAIL_USAGE="-tail [-f] <file>";
+
+  /**
+   */
+  public FsShell() {
+    this(null);
+  }
+
+  public FsShell(Configuration conf) {
+    super(conf);
+    fs = null;
+    trash = null;
+  }
+  
+  protected void init() throws IOException {
+    getConf().setQuietMode(true);
+    if (this.fs == null) {
+     this.fs = FileSystem.get(getConf());
+    }
+    if (this.trash == null) {
+      this.trash = new Trash(getConf());
+    }
+  }
+
+  
+  /**
+   * Copies from stdin to the indicated file.
+   */
+  private void copyFromStdin(Path dst, FileSystem dstFs) throws IOException {
+    if (dstFs.isDirectory(dst)) {
+      throw new IOException("When source is stdin, destination must be a file.");
+    }
+    if (dstFs.exists(dst)) {
+      throw new IOException("Target " + dst.toString() + " already exists.");
+    }
+    FSDataOutputStream out = dstFs.create(dst); 
+    try {
+      IOUtils.copyBytes(System.in, out, getConf(), false);
+    } 
+    finally {
+      out.close();
+    }
+  }
+
+  /** 
+   * Print from src to stdout.
+   */
+  private void printToStdout(InputStream in) throws IOException {
+    try {
+      IOUtils.copyBytes(in, System.out, getConf(), false);
+    } finally {
+      in.close();
+    }
+  }
+
+  
+  /**
+   * Add local files to the indicated FileSystem name. src is kept.
+   */
+  void copyFromLocal(Path[] srcs, String dstf) throws IOException {
+    Path dstPath = new Path(dstf);
+    FileSystem dstFs = dstPath.getFileSystem(getConf());
+    if (srcs.length == 1 && srcs[0].toString().equals("-"))
+      copyFromStdin(dstPath, dstFs);
+    else
+      dstFs.copyFromLocalFile(false, false, srcs, dstPath);
+  }
+  
+  /**
+   * Add local files to the indicated FileSystem name. src is removed.
+   */
+  void moveFromLocal(Path[] srcs, String dstf) throws IOException {
+    Path dstPath = new Path(dstf);
+    FileSystem dstFs = dstPath.getFileSystem(getConf());
+    dstFs.moveFromLocalFile(srcs, dstPath);
+  }
+
+  /**
+   * Add a local file to the indicated FileSystem name. src is removed.
+   */
+  void moveFromLocal(Path src, String dstf) throws IOException {
+    moveFromLocal((new Path[]{src}), dstf);
+  }
+
+  /**
+   * Obtain the indicated files that match the file pattern <i>srcf</i>
+   * and copy them to the local name. srcf is kept.
+   * When copying multiple files, the destination must be a directory. 
+   * Otherwise, IOException is thrown.
+   * @param argv: arguments
+   * @param pos: Ignore everything before argv[pos]  
+   * @exception: IOException  
+   * @see org.apache.hadoop.fs.FileSystem.globStatus 
+   */
+  void copyToLocal(String[]argv, int pos) throws IOException {
+    CommandFormat cf = new CommandFormat("copyToLocal", 2,2,"crc","ignoreCrc");
+    
+    String srcstr = null;
+    String dststr = null;
+    try {
+      List<String> parameters = cf.parse(argv, pos);
+      srcstr = parameters.get(0);
+      dststr = parameters.get(1);
+    }
+    catch(IllegalArgumentException iae) {
+      System.err.println("Usage: java FsShell " + GET_SHORT_USAGE);
+      throw iae;
+    }
+    boolean copyCrc = cf.getOpt("crc");
+    final boolean verifyChecksum = !cf.getOpt("ignoreCrc");
+
+    if (dststr.equals("-")) {
+      if (copyCrc) {
+        System.err.println("-crc option is not valid when destination is stdout.");
+      }
+      cat(srcstr, verifyChecksum);
+    } else {
+      File dst = new File(dststr);      
+      Path srcpath = new Path(srcstr);
+      FileSystem srcFS = getSrcFileSystem(srcpath, verifyChecksum);
+      if (copyCrc && !(srcFS instanceof ChecksumFileSystem)) {
+        System.err.println("-crc option is not valid when source file system " +
+            "does not have crc files. Automatically turn the option off.");
+        copyCrc = false;
+      }
+      FileStatus[] srcs = srcFS.globStatus(srcpath);
+      boolean dstIsDir = dst.isDirectory(); 
+      if (srcs.length > 1 && !dstIsDir) {
+        throw new IOException("When copying multiple files, "
+                              + "destination should be a directory.");
+      }
+      for (FileStatus status : srcs) {
+        Path p = status.getPath();
+        File f = dstIsDir? new File(dst, p.getName()): dst;
+        copyToLocal(srcFS, status, f, copyCrc);
+      }
+    }
+  }
+
+  /**
+   * Return the {@link FileSystem} specified by src and the conf.
+   * It the {@link FileSystem} supports checksum, set verifyChecksum.
+   */
+  private FileSystem getSrcFileSystem(Path src, boolean verifyChecksum
+      ) throws IOException { 
+    FileSystem srcFs = src.getFileSystem(getConf());
+    srcFs.setVerifyChecksum(verifyChecksum);
+    return srcFs;
+  }
+
+  /**
+   * The prefix for the tmp file used in copyToLocal.
+   * It must be at least three characters long, required by
+   * {@link java.io.File#createTempFile(String, String, File)}.
+   */
+  static final String COPYTOLOCAL_PREFIX = "_copyToLocal_";
+
+  /**
+   * Copy a source file from a given file system to local destination.
+   * @param srcFS source file system
+   * @param src source path
+   * @param dst destination
+   * @param copyCrc copy CRC files?
+   * @exception IOException If some IO failed
+   */
+  private void copyToLocal(final FileSystem srcFS, final FileStatus srcStatus,
+                           final File dst, final boolean copyCrc)
+    throws IOException {
+    /* Keep the structure similar to ChecksumFileSystem.copyToLocal(). 
+     * Ideal these two should just invoke FileUtil.copy() and not repeat
+     * recursion here. Of course, copy() should support two more options :
+     * copyCrc and useTmpFile (may be useTmpFile need not be an option).
+     */
+    
+    Path src = srcStatus.getPath();
+    if (!srcStatus.isDir()) {
+      if (dst.exists()) {
+        // match the error message in FileUtil.checkDest():
+        throw new IOException("Target " + dst + " already exists");
+      }
+      
+      // use absolute name so that tmp file is always created under dest dir
+      File tmp = FileUtil.createLocalTempFile(dst.getAbsoluteFile(),
+                                              COPYTOLOCAL_PREFIX, true);
+      if (!FileUtil.copy(srcFS, src, tmp, false, srcFS.getConf())) {
+        throw new IOException("Failed to copy " + src + " to " + dst); 
+      }
+      
+      if (!tmp.renameTo(dst)) {
+        throw new IOException("Failed to rename tmp file " + tmp + 
+                              " to local destination \"" + dst + "\".");
+      }
+
+      if (copyCrc) {
+        if (!(srcFS instanceof ChecksumFileSystem)) {
+          throw new IOException("Source file system does not have crc files");
+        }
+        
+        ChecksumFileSystem csfs = (ChecksumFileSystem) srcFS;
+        File dstcs = FileSystem.getLocal(srcFS.getConf())
+          .pathToFile(csfs.getChecksumFile(new Path(dst.getCanonicalPath())));
+        FileSystem fs = csfs.getRawFileSystem();
+        FileStatus status = csfs.getFileStatus(csfs.getChecksumFile(src));
+        copyToLocal(fs, status, dstcs, false);
+      } 
+    } else {
+      // once FileUtil.copy() supports tmp file, we don't need to mkdirs().
+      if (!dst.mkdirs()) {
+        throw new IOException("Failed to create local destination \"" +
+                              dst + "\".");
+      }
+      for(FileStatus status : srcFS.listStatus(src)) {
+        copyToLocal(srcFS, status,
+                    new File(dst, status.getPath().getName()), copyCrc);
+      }
+    }
+  }
+
+  /**
+   * Get all the files in the directories that match the source file 
+   * pattern and merge and sort them to only one file on local fs 
+   * srcf is kept.
+   * @param srcf: a file pattern specifying source files
+   * @param dstf: a destination local file/directory 
+   * @exception: IOException  
+   * @see org.apache.hadoop.fs.FileSystem.globStatus 
+   */
+  void copyMergeToLocal(String srcf, Path dst) throws IOException {
+    copyMergeToLocal(srcf, dst, false);
+  }    
+    
+
+  /**
+   * Get all the files in the directories that match the source file pattern
+   * and merge and sort them to only one file on local fs 
+   * srcf is kept.
+   * 
+   * Also adds a string between the files (useful for adding \n
+   * to a text file)
+   * @param srcf: a file pattern specifying source files
+   * @param dstf: a destination local file/directory
+   * @param endline: if an end of line character is added to a text file 
+   * @exception: IOException  
+   * @see org.apache.hadoop.fs.FileSystem.globStatus 
+   */
+  void copyMergeToLocal(String srcf, Path dst, boolean endline) throws IOException {
+    Path srcPath = new Path(srcf);
+    FileSystem srcFs = srcPath.getFileSystem(getConf());
+    Path [] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), 
+                                       srcPath);
+    for(int i=0; i<srcs.length; i++) {
+      if (endline) {
+        FileUtil.copyMerge(srcFs, srcs[i], 
+                           FileSystem.getLocal(getConf()), dst, false, getConf(), "\n");
+      } else {
+        FileUtil.copyMerge(srcFs, srcs[i], 
+                           FileSystem.getLocal(getConf()), dst, false, getConf(), null);
+      }
+    }
+  }      
+
+  /**
+   * Obtain the indicated file and copy to the local name.
+   * srcf is removed.
+   */
+  void moveToLocal(String srcf, Path dst) throws IOException {
+    System.err.println("Option '-moveToLocal' is not implemented yet.");
+  }
+
+  /**
+   * Fetch all files that match the file pattern <i>srcf</i> and display
+   * their content on stdout. 
+   * @param srcf: a file pattern specifying source files
+   * @exception: IOException
+   * @see org.apache.hadoop.fs.FileSystem.globStatus 
+   */
+  void cat(String src, boolean verifyChecksum) throws IOException {
+    //cat behavior in Linux
+    //  [~/1207]$ ls ?.txt
+    //  x.txt  z.txt
+    //  [~/1207]$ cat x.txt y.txt z.txt
+    //  xxx
+    //  cat: y.txt: No such file or directory
+    //  zzz
+
+    Path srcPattern = new Path(src);
+    new DelayedExceptionThrowing() {
+      @Override
+      void process(Path p, FileSystem srcFs) throws IOException {
+        if (srcFs.getFileStatus(p).isDir()) {
+          throw new IOException("Source must be a file.");
+        }
+        printToStdout(srcFs.open(p));
+      }
+    }.globAndProcess(srcPattern, getSrcFileSystem(srcPattern, verifyChecksum));
+  }
+
+  private class TextRecordInputStream extends InputStream {
+    SequenceFile.Reader r;
+    WritableComparable key;
+    Writable val;
+
+    DataInputBuffer inbuf;
+    DataOutputBuffer outbuf;
+
+    public TextRecordInputStream(FileStatus f) throws IOException {
+      r = new SequenceFile.Reader(fs, f.getPath(), getConf());
+      key = ReflectionUtils.newInstance(r.getKeyClass().asSubclass(WritableComparable.class),
+                                        getConf());
+      val = ReflectionUtils.newInstance(r.getValueClass().asSubclass(Writable.class),
+                                        getConf());
+      inbuf = new DataInputBuffer();
+      outbuf = new DataOutputBuffer();
+    }
+
+    public int read() throws IOException {
+      int ret;
+      if (null == inbuf || -1 == (ret = inbuf.read())) {
+        if (!r.next(key, val)) {
+          return -1;
+        }
+        byte[] tmp = key.toString().getBytes();
+        outbuf.write(tmp, 0, tmp.length);
+        outbuf.write('\t');
+        tmp = val.toString().getBytes();
+        outbuf.write(tmp, 0, tmp.length);
+        outbuf.write('\n');
+        inbuf.reset(outbuf.getData(), outbuf.getLength());
+        outbuf.reset();
+        ret = inbuf.read();
+      }
+      return ret;
+    }
+  }
+
+  private InputStream forMagic(Path p, FileSystem srcFs) throws IOException {
+    FSDataInputStream i = srcFs.open(p);
+    switch(i.readShort()) {
+      case 0x1f8b: // RFC 1952
+        i.seek(0);
+        return new GZIPInputStream(i);
+      case 0x5345: // 'S' 'E'
+        if (i.readByte() == 'Q') {
+          i.close();
+          return new TextRecordInputStream(srcFs.getFileStatus(p));
+        }
+        break;
+    }
+    i.seek(0);
+    return i;
+  }
+
+  void text(String srcf) throws IOException {
+    Path srcPattern = new Path(srcf);
+    new DelayedExceptionThrowing() {
+      @Override
+      void process(Path p, FileSystem srcFs) throws IOException {
+        if (srcFs.isDirectory(p)) {
+          throw new IOException("Source must be a file.");
+        }
+        printToStdout(forMagic(p, srcFs));
+      }
+    }.globAndProcess(srcPattern, srcPattern.getFileSystem(getConf()));
+  }
+
+  /**
+   * Parse the incoming command string
+   * @param cmd
+   * @param pos ignore anything before this pos in cmd
+   * @throws IOException 
+   */
+  private void setReplication(String[] cmd, int pos) throws IOException {
+    CommandFormat c = new CommandFormat("setrep", 2, 2, "R", "w");
+    String dst = null;
+    short rep = 0;
+
+    try {
+      List<String> parameters = c.parse(cmd, pos);
+      rep = Short.parseShort(parameters.get(0));
+      dst = parameters.get(1);
+    } catch (NumberFormatException nfe) {
+      System.err.println("Illegal replication, a positive integer expected");
+      throw nfe;
+    }
+    catch(IllegalArgumentException iae) {
+      System.err.println("Usage: java FsShell " + SETREP_SHORT_USAGE);
+      throw iae;
+    }
+
+    if (rep < 1) {
+      System.err.println("Cannot set replication to: " + rep);
+      throw new IllegalArgumentException("replication must be >= 1");
+    }
+
+    List<Path> waitList = c.getOpt("w")? new ArrayList<Path>(): null;
+    setReplication(rep, dst, c.getOpt("R"), waitList);
+
+    if (waitList != null) {
+      waitForReplication(waitList, rep);
+    }
+  }
+    
+  /**
+   * Wait for all files in waitList to have replication number equal to rep.
+   * @param waitList The files are waited for.
+   * @param rep The new replication number.
+   * @throws IOException IOException
+   */
+  void waitForReplication(List<Path> waitList, int rep) throws IOException {
+    for(Path f : waitList) {
+      System.out.print("Waiting for " + f + " ...");
+      System.out.flush();
+
+      boolean printWarning = false;
+      FileStatus status = fs.getFileStatus(f);
+      long len = status.getLen();
+
+      for(boolean done = false; !done; ) {
+        BlockLocation[] locations = fs.getFileBlockLocations(status, 0, len);
+        int i = 0;
+        for(; i < locations.length && 
+          locations[i].getHosts().length == rep; i++)
+          if (!printWarning && locations[i].getHosts().length > rep) {
+            System.out.println("\nWARNING: the waiting time may be long for "
+                + "DECREASING the number of replication.");
+            printWarning = true;
+          }
+        done = i == locations.length;
+
+        if (!done) {
+          System.out.print(".");
+          System.out.flush();
+          try {Thread.sleep(10000);} catch (InterruptedException e) {}
+        }
+      }
+
+      System.out.println(" done");
+    }
+  }
+
+  /**
+   * Set the replication for files that match file pattern <i>srcf</i>
+   * if it's a directory and recursive is true,
+   * set replication for all the subdirs and those files too.
+   * @param newRep new replication factor
+   * @param srcf a file pattern specifying source files
+   * @param recursive if need to set replication factor for files in subdirs
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  void setReplication(short newRep, String srcf, boolean recursive,
+                      List<Path> waitingList)
+    throws IOException {
+    Path srcPath = new Path(srcf);
+    FileSystem srcFs = srcPath.getFileSystem(getConf());
+    Path[] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath),
+                                      srcPath);
+    for(int i=0; i<srcs.length; i++) {
+      setReplication(newRep, srcFs, srcs[i], recursive, waitingList);
+    }
+  }
+
+  private void setReplication(short newRep, FileSystem srcFs, 
+                              Path src, boolean recursive,
+                              List<Path> waitingList)
+    throws IOException {
+    if (!srcFs.getFileStatus(src).isDir()) {
+      setFileReplication(src, srcFs, newRep, waitingList);
+      return;
+    }
+    FileStatus items[] = srcFs.listStatus(src);
+    if (items == null) {
+      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);
+        }
+      }
+    }
+  }
+    
+  /**
+   * Actually set the replication for this file
+   * If it fails either throw IOException or print an error msg
+   * @param file: a file/directory
+   * @param newRep: new replication factor
+   * @throws IOException
+   */
+  private void setFileReplication(Path file, FileSystem srcFs, short newRep, List<Path> waitList)
+    throws IOException {
+    if (srcFs.setReplication(file, newRep)) {
+      if (waitList != null) {
+        waitList.add(file);
+      }
+      System.out.println("Replication " + newRep + " set: " + file);
+    } else {
+      System.err.println("Could not set replication for: " + file);
+    }
+  }
+    
+    
+  /**
+   * Get a listing of all files in that match the file pattern <i>srcf</i>.
+   * @param srcf a file pattern specifying source files
+   * @param recursive if need to list files in subdirs
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  private int ls(String srcf, boolean recursive) throws IOException {
+    Path srcPath = new Path(srcf);
+    FileSystem srcFs = srcPath.getFileSystem(this.getConf());
+    FileStatus[] srcs = srcFs.globStatus(srcPath);
+    if (srcs==null || srcs.length==0) {
+      throw new FileNotFoundException("Cannot access " + srcf + 
+          ": No such file or directory.");
+    }
+ 
+    boolean printHeader = (srcs.length == 1) ? true: false;
+    int numOfErrors = 0;
+    for(int i=0; i<srcs.length; i++) {
+      numOfErrors += ls(srcs[i], srcFs, recursive, printHeader);
+    }
+    return numOfErrors == 0 ? 0 : -1;
+  }
+
+  /* list all files under the directory <i>src</i>
+   * ideally we should provide "-l" option, that lists like "ls -l".
+   */
+  private int ls(FileStatus src, FileSystem srcFs, boolean recursive,
+      boolean printHeader) throws IOException {
+    final String cmd = recursive? "lsr": "ls";
+    final FileStatus[] items = shellListStatus(cmd, srcFs, src);
+    if (items == null) {
+      return 1;
+    } else {
+      int numOfErrors = 0;
+      if (!recursive && printHeader) {
+        if (items.length != 0) {
+          System.out.println("Found " + items.length + " items");
+        }
+      }
+      
+      int maxReplication = 3, maxLen = 10, maxOwner = 0,maxGroup = 0;
+
+      for(int i = 0; i < items.length; i++) {
+        FileStatus stat = items[i];
+        int replication = String.valueOf(stat.getReplication()).length();
+        int len = String.valueOf(stat.getLen()).length();
+        int owner = String.valueOf(stat.getOwner()).length();
+        int group = String.valueOf(stat.getGroup()).length();
+        
+        if (replication > maxReplication) maxReplication = replication;
+        if (len > maxLen) maxLen = len;
+        if (owner > maxOwner)  maxOwner = owner;
+        if (group > maxGroup)  maxGroup = group;
+      }
+      
+      for (int i = 0; i < items.length; i++) {
+        FileStatus stat = items[i];
+        Path cur = stat.getPath();
+        String mdate = dateForm.format(new Date(stat.getModificationTime()));
+        
+        System.out.print((stat.isDir() ? "d" : "-") + 
+          stat.getPermission() + " ");
+        System.out.printf("%"+ maxReplication + 
+          "s ", (!stat.isDir() ? stat.getReplication() : "-"));
+        if (maxOwner > 0)
+          System.out.printf("%-"+ maxOwner + "s ", stat.getOwner());
+        if (maxGroup > 0)
+          System.out.printf("%-"+ maxGroup + "s ", stat.getGroup());
+        System.out.printf("%"+ maxLen + "d ", stat.getLen());
+        System.out.print(mdate + " ");
+        System.out.println(cur.toUri().getPath());
+        if (recursive && stat.isDir()) {
+          numOfErrors += ls(stat,srcFs, recursive, printHeader);
+        }
+      }
+      return numOfErrors;
+    }
+  }
+
+   /**
+   * Show the size of a partition in the filesystem that contains
+   * the specified <i>path</i>.
+   * @param path a path specifying the source partition. null means /.
+   * @throws IOException  
+   */
+  void df(String path) throws IOException {
+    if (path == null) path = "/";
+    final Path srcPath = new Path(path);
+    final FileSystem srcFs = srcPath.getFileSystem(getConf());
+    if (! srcFs.exists(srcPath)) {
+      throw new FileNotFoundException("Cannot access "+srcPath.toString());
+    }
+    final FsStatus stats = srcFs.getStatus(srcPath);
+    final int PercentUsed = (int)(100.0f *  (float)stats.getUsed() / (float)stats.getCapacity());
+    System.out.println("Filesystem\t\tSize\tUsed\tAvail\tUse%");
+    System.out.printf("%s\t\t%d\t%d\t%d\t%d%%\n",
+      path, 
+      stats.getCapacity(), stats.getUsed(), stats.getRemaining(),
+      PercentUsed);
+  }
+
+  /**
+   * Show the size of all files that match the file pattern <i>src</i>
+   * @param src a file pattern specifying source files
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  void du(String src) throws IOException {
+    Path srcPath = new Path(src);
+    FileSystem srcFs = srcPath.getFileSystem(getConf());
+    Path[] pathItems = FileUtil.stat2Paths(srcFs.globStatus(srcPath), 
+                                           srcPath);
+    FileStatus items[] = srcFs.listStatus(pathItems);
+    if ((items == null) || ((items.length == 0) && 
+        (!srcFs.exists(srcPath)))){
+      throw new FileNotFoundException("Cannot access " + src
+            + ": No such file or directory.");
+    } else {
+      System.out.println("Found " + items.length + " items");
+      int maxLength = 10;
+      
+      long length[] = new long[items.length];
+      for (int i = 0; i < items.length; i++) {
+        length[i] = items[i].isDir() ?
+          srcFs.getContentSummary(items[i].getPath()).getLength() :
+          items[i].getLen();
+        int len = String.valueOf(length[i]).length();
+        if (len > maxLength) maxLength = len;
+      }
+      for(int i = 0; i < items.length; i++) {
+        System.out.printf("%-"+ (maxLength + BORDER) +"d", length[i]);
+        System.out.println(items[i].getPath());
+      }
+    }
+  }
+    
+  /**
+   * Show the summary disk usage of each dir/file 
+   * that matches the file pattern <i>src</i>
+   * @param src a file pattern specifying source files
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  void dus(String src) throws IOException {
+    Path srcPath = new Path(src);
+    FileSystem srcFs = srcPath.getFileSystem(getConf());
+    FileStatus status[] = srcFs.globStatus(new Path(src));
+    if (status==null || status.length==0) {
+      throw new FileNotFoundException("Cannot access " + src + 
+          ": No such file or directory.");
+    }
+    for(int i=0; i<status.length; i++) {
+      long totalSize = srcFs.getContentSummary(status[i].getPath()).getLength();
+      String pathStr = status[i].getPath().toString();
+      System.out.println(("".equals(pathStr)?".":pathStr) + "\t" + totalSize);
+    }
+  }
+
+  /**
+   * Create the given dir
+   */
+  void mkdir(String src) throws IOException {
+    Path f = new Path(src);
+    FileSystem srcFs = f.getFileSystem(getConf());
+    FileStatus fstatus = null;
+    try {
+      fstatus = srcFs.getFileStatus(f);
+      if (fstatus.isDir()) {
+        throw new IOException("cannot create directory " 
+            + src + ": File exists");
+      }
+      else {
+        throw new IOException(src + " exists but " +
+            "is not a directory");
+      }
+    } catch(FileNotFoundException e) {
+        if (!srcFs.mkdirs(f)) {
+          throw new IOException("failed to create " + src);
+        }
+    }
+  }
+
+  /**
+   * (Re)create zero-length file at the specified path.
+   * This will be replaced by a more UNIX-like touch when files may be
+   * modified.
+   */
+  void touchz(String src) throws IOException {
+    Path f = new Path(src);
+    FileSystem srcFs = f.getFileSystem(getConf());
+    FileStatus st;
+    if (srcFs.exists(f)) {
+      st = srcFs.getFileStatus(f);
+      if (st.isDir()) {
+        // TODO: handle this
+        throw new IOException(src + " is a directory");
+      } else if (st.getLen() != 0)
+        throw new IOException(src + " must be a zero-length file");
+    }
+    FSDataOutputStream out = srcFs.create(f);
+    out.close();
+  }
+
+  /**
+   * Check file types.
+   */
+  int test(String argv[], int i) throws IOException {
+    if (!argv[i].startsWith("-") || argv[i].length() > 2)
+      throw new IOException("Not a flag: " + argv[i]);
+    char flag = argv[i].toCharArray()[1];
+    Path f = new Path(argv[++i]);
+    FileSystem srcFs = f.getFileSystem(getConf());
+    switch(flag) {
+      case 'e':
+        return srcFs.exists(f) ? 0 : 1;
+      case 'z':
+        return srcFs.getFileStatus(f).getLen() == 0 ? 0 : 1;
+      case 'd':
+        return srcFs.getFileStatus(f).isDir() ? 0 : 1;
+      default:
+        throw new IOException("Unknown flag: " + flag);
+    }
+  }
+
+  /**
+   * Print statistics about path in specified format.
+   * Format sequences:
+   *   %b: Size of file in blocks
+   *   %n: Filename
+   *   %o: Block size
+   *   %r: replication
+   *   %y: UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;
+   *   %Y: Milliseconds since January 1, 1970 UTC
+   */
+  void stat(char[] fmt, String src) throws IOException {
+    Path srcPath = new Path(src);
+    FileSystem srcFs = srcPath.getFileSystem(getConf());
+    FileStatus glob[] = srcFs.globStatus(srcPath);
+    if (null == glob)
+      throw new IOException("cannot stat `" + src + "': No such file or directory");
+    for (FileStatus f : glob) {
+      StringBuilder buf = new StringBuilder();
+      for (int i = 0; i < fmt.length; ++i) {
+        if (fmt[i] != '%') {
+          buf.append(fmt[i]);
+        } else {
+          if (i + 1 == fmt.length) break;
+          switch(fmt[++i]) {
+            case 'b':
+              buf.append(f.getLen());
+              break;
+            case 'F':
+              buf.append(f.isDir() ? "directory" : "regular file");
+              break;
+            case 'n':
+              buf.append(f.getPath().getName());
+              break;
+            case 'o':
+              buf.append(f.getBlockSize());
+              break;
+            case 'r':
+              buf.append(f.getReplication());
+              break;
+            case 'y':
+              buf.append(modifFmt.format(new Date(f.getModificationTime())));
+              break;
+            case 'Y':
+              buf.append(f.getModificationTime());
+              break;
+            default:
+              buf.append(fmt[i]);
+              break;
+          }
+        }
+      }
+      System.out.println(buf.toString());
+    }
+  }
+
+  /**
+   * Move files that match the file pattern <i>srcf</i>
+   * to a destination file.
+   * When moving mutiple files, the destination must be a directory. 
+   * Otherwise, IOException is thrown.
+   * @param srcf a file pattern specifying source files
+   * @param dstf a destination local file/directory 
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  void rename(String srcf, String dstf) throws IOException {
+    Path srcPath = new Path(srcf);
+    Path dstPath = new Path(dstf);
+    FileSystem fs = srcPath.getFileSystem(getConf());
+    URI srcURI = fs.getUri();
+    URI dstURI = dstPath.getFileSystem(getConf()).getUri();
+    if (srcURI.compareTo(dstURI) != 0) {
+      throw new IOException("src and destination filesystems do not match.");
+    }
+    Path[] srcs = FileUtil.stat2Paths(fs.globStatus(srcPath), srcPath);
+    Path dst = new Path(dstf);
+    if (srcs.length > 1 && !fs.isDirectory(dst)) {
+      throw new IOException("When moving multiple files, " 
+                            + "destination should be a directory.");
+    }
+    for(int i=0; i<srcs.length; i++) {
+      if (!fs.rename(srcs[i], dst)) {
+        FileStatus srcFstatus = null;
+        FileStatus dstFstatus = null;
+        try {
+          srcFstatus = fs.getFileStatus(srcs[i]);
+        } catch(FileNotFoundException e) {
+          throw new FileNotFoundException(srcs[i] + 
+          ": No such file or directory");
+        }
+        try {
+          dstFstatus = fs.getFileStatus(dst);
+        } catch(IOException e) {
+        }
+        if((srcFstatus!= null) && (dstFstatus!= null)) {
+          if (srcFstatus.isDir()  && !dstFstatus.isDir()) {
+            throw new IOException("cannot overwrite non directory "
+                + dst + " with directory " + srcs[i]);
+          }
+        }
+        throw new IOException("Failed to rename " + srcs[i] + " to " + dst);
+      }
+    }
+  }
+
+  /**
+   * Move/rename file(s) to a destination file. Multiple source
+   * files can be specified. The destination is the last element of
+   * the argvp[] array.
+   * If multiple source files are specified, then the destination 
+   * must be a directory. Otherwise, IOException is thrown.
+   * @exception: IOException  
+   */
+  private int rename(String argv[], Configuration conf) throws IOException {
+    int i = 0;
+    int exitCode = 0;
+    String cmd = argv[i++];  
+    String dest = argv[argv.length-1];
+    //
+    // If the user has specified multiple source files, then
+    // the destination has to be a directory
+    //
+    if (argv.length > 3) {
+      Path dst = new Path(dest);
+      FileSystem dstFs = dst.getFileSystem(getConf());
+      if (!dstFs.isDirectory(dst)) {
+        throw new IOException("When moving multiple files, " 
+                              + "destination " + dest + " should be a directory.");
+      }
+    }
+    //
+    // for each source file, issue the rename
+    //
+    for (; i < argv.length - 1; i++) {
+      try {
+        //
+        // issue the rename to the fs
+        //
+        rename(argv[i], dest);
+      } catch (RemoteException e) {
+        //
+        // This is a error returned by hadoop server. Print
+        // out the first line of the error mesage.
+        //
+        exitCode = -1;
+        try {
+          String[] content;
+          content = e.getLocalizedMessage().split("\n");
+          System.err.println(cmd.substring(1) + ": " + content[0]);
+        } catch (Exception ex) {
+          System.err.println(cmd.substring(1) + ": " +
+                             ex.getLocalizedMessage());
+        }
+      } catch (IOException e) {
+        //
+        // IO exception encountered locally.
+        //
+        exitCode = -1;
+        System.err.println(cmd.substring(1) + ": " +
+                           e.getLocalizedMessage());
+      }
+    }
+    return exitCode;
+  }
+
+  /**
+   * Copy files that match the file pattern <i>srcf</i>
+   * to a destination file.
+   * When copying mutiple files, the destination must be a directory. 
+   * Otherwise, IOException is thrown.
+   * @param srcf a file pattern specifying source files
+   * @param dstf a destination local file/directory 
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  void copy(String srcf, String dstf, Configuration conf) throws IOException {
+    Path srcPath = new Path(srcf);
+    FileSystem srcFs = srcPath.getFileSystem(getConf());
+    Path dstPath = new Path(dstf);
+    FileSystem dstFs = dstPath.getFileSystem(getConf());
+    Path [] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), srcPath);
+    if (srcs.length > 1 && !dstFs.isDirectory(dstPath)) {
+      throw new IOException("When copying multiple files, " 
+                            + "destination should be a directory.");
+    }
+    for(int i=0; i<srcs.length; i++) {
+      FileUtil.copy(srcFs, srcs[i], dstFs, dstPath, false, conf);
+    }
+  }
+
+  /**
+   * Copy file(s) to a destination file. Multiple source
+   * files can be specified. The destination is the last element of
+   * the argvp[] array.
+   * If multiple source files are specified, then the destination 
+   * must be a directory. Otherwise, IOException is thrown.
+   * @exception: IOException  
+   */
+  private int copy(String argv[], Configuration conf) throws IOException {
+    int i = 0;
+    int exitCode = 0;
+    String cmd = argv[i++];  
+    String dest = argv[argv.length-1];
+    //
+    // If the user has specified multiple source files, then
+    // the destination has to be a directory
+    //
+    if (argv.length > 3) {
+      Path dst = new Path(dest);
+      if (!fs.isDirectory(dst)) {
+        throw new IOException("When copying multiple files, " 
+                              + "destination " + dest + " should be a directory.");
+      }
+    }
+    //
+    // for each source file, issue the copy
+    //
+    for (; i < argv.length - 1; i++) {
+      try {
+        //
+        // issue the copy to the fs
+        //
+        copy(argv[i], dest, conf);
+      } catch (RemoteException e) {
+        //
+        // This is a error returned by hadoop server. Print
+        // out the first line of the error mesage.
+        //
+        exitCode = -1;
+        try {
+          String[] content;
+          content = e.getLocalizedMessage().split("\n");
+          System.err.println(cmd.substring(1) + ": " +
+                             content[0]);
+        } catch (Exception ex) {
+          System.err.println(cmd.substring(1) + ": " +
+                             ex.getLocalizedMessage());
+        }
+      } catch (IOException e) {
+        //
+        // IO exception encountered locally.
+        //
+        exitCode = -1;
+        System.err.println(cmd.substring(1) + ": " +
+                           e.getLocalizedMessage());
+      }
+    }
+    return exitCode;
+  }
+
+  /**
+   * Delete all files that match the file pattern <i>srcf</i>.
+   * @param srcf a file pattern specifying source files
+   * @param recursive if need to delete subdirs
+   * @throws IOException  
+   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
+   */
+  void delete(String srcf, final boolean recursive) throws IOException {
+    //rm behavior in Linux
+    //  [~/1207]$ ls ?.txt
+    //  x.txt  z.txt
+    //  [~/1207]$ rm x.txt y.txt z.txt 
+    //  rm: cannot remove `y.txt': No such file or directory
+
+    Path srcPattern = new Path(srcf);
+    new DelayedExceptionThrowing() {
+      @Override
+      void process(Path p, FileSystem srcFs) throws IOException {
+        delete(p, srcFs, recursive);
+      }
+    }.globAndProcess(srcPattern, srcPattern.getFileSystem(getConf()));
+  }
+    
+  /* delete a file */
+  private void delete(Path src, FileSystem srcFs, boolean recursive) throws IOException {
+    if (srcFs.isDirectory(src) && !recursive) {
+      throw new IOException("Cannot remove directory \"" + src +
+                            "\", use -rmr instead");
+    }
+    Trash trashTmp = new Trash(srcFs, getConf());
+    if (trashTmp.moveToTrash(src)) {
+      System.out.println("Moved to trash: " + src);
+      return;
+    }
+    if (srcFs.delete(src, true)) {
+      System.out.println("Deleted " + src);
+    } else {
+      if (!srcFs.exists(src)) {
+        throw new FileNotFoundException("cannot remove "
+            + src + ": No such file or directory.");
+        }
+      throw new IOException("Delete failed " + src);
+    }
+  }
+
+  private void expunge() throws IOException {
+    trash.expunge();
+    trash.checkpoint();
+  }
+
+  /**
+   * Returns the Trash object associated with this shell.
+   */
+  public Path getCurrentTrashDir() {
+    return trash.getCurrentTrashDir();
+  }
+
+  /**
+   * Parse the incoming command string
+   * @param cmd
+   * @param pos ignore anything before this pos in cmd
+   * @throws IOException 
+   */
+  private void tail(String[] cmd, int pos) throws IOException {
+    CommandFormat c = new CommandFormat("tail", 1, 1, "f");
+    String src = null;
+    Path path = null;
+
+    try {
+      List<String> parameters = c.parse(cmd, pos);
+      src = parameters.get(0);
+    } catch(IllegalArgumentException iae) {
+      System.err.println("Usage: java FsShell " + TAIL_USAGE);
+      throw iae;
+    }
+    boolean foption = c.getOpt("f") ? true: false;
+    path = new Path(src);
+    FileSystem srcFs = path.getFileSystem(getConf());
+    FileStatus fileStatus = srcFs.getFileStatus(path);
+    if (fileStatus.isDir()) {
+      throw new IOException("Source must be a file.");
+    }
+
+    long fileSize = fileStatus.getLen();
+    long offset = (fileSize > 1024) ? fileSize - 1024: 0;
+
+    while (true) {
+      FSDataInputStream in = srcFs.open(path);
+      in.seek(offset);
+      IOUtils.copyBytes(in, System.out, 1024, false);
+      offset = in.getPos();
+      in.close();
+      if (!foption) {
+        break;
+      }
+      fileSize = srcFs.getFileStatus(path).getLen();
+      offset = (fileSize > offset) ? offset: fileSize;
+      try {
+        Thread.sleep(5000);
+      } catch (InterruptedException e) {
+        break;
+      }
+    }
+  }
+
+  /**
+   * This class runs a command on a given FileStatus. This can be used for
+   * running various commands like chmod, chown etc.
+   */
+  static abstract class CmdHandler {
+    
+    protected int errorCode = 0;
+    protected boolean okToContinue = true;
+    protected String cmdName;
+    
+    int getErrorCode() { return errorCode; }
+    boolean okToContinue() { return okToContinue; }
+    String getName() { return cmdName; }
+    
+    protected CmdHandler(String cmdName, FileSystem fs) {
+      this.cmdName = cmdName;
+    }
+    
+    public abstract void run(FileStatus file, FileSystem fs) throws IOException;
+  }
+  
+  /** helper returns listStatus() */
+  private static FileStatus[] shellListStatus(String cmd, 
+                                                   FileSystem srcFs,
+                                                   FileStatus src) {
+    if (!src.isDir()) {
+      FileStatus[] files = { src };
+      return files;
+    }
+    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 (IOException e) {
+      System.err.println(cmd + 
+                         ": could not get get listing for '" + path + "' : " +
+                         e.getMessage().split("\n")[0]);
+    }
+    return null;
+  }
+  
+  
+  /**
+   * Runs the command on a given file with the command handler. 
+   * If recursive is set, command is run recursively.
+   */                                       
+  private static int runCmdHandler(CmdHandler handler, FileStatus stat, 
+                                   FileSystem srcFs, 
+                                   boolean recursive) throws IOException {
+    int errors = 0;
+    handler.run(stat, srcFs);
+    if (recursive && stat.isDir() && handler.okToContinue()) {
+      FileStatus[] files = shellListStatus(handler.getName(), srcFs, stat);
+      if (files == null) {
+        return 1;
+      }
+      for(FileStatus file : files ) {
+        errors += runCmdHandler(handler, file, srcFs, recursive);
+      }
+    }
+    return errors;
+  }
+
+  ///top level runCmdHandler
+  int runCmdHandler(CmdHandler handler, String[] args,
+                                   int startIndex, boolean recursive) 
+                                   throws IOException {
+    int errors = 0;
+    
+    for (int i=startIndex; i<args.length; i++) {
+      Path srcPath = new Path(args[i]);
+      FileSystem srcFs = srcPath.getFileSystem(getConf());
+      Path[] paths = FileUtil.stat2Paths(srcFs.globStatus(srcPath), srcPath);
+      for(Path path : paths) {
+        try {
+          FileStatus file = srcFs.getFileStatus(path);
+          if (file == null) {
+            System.err.println(handler.getName() + 
+                               ": could not get status for '" + path + "'");
+            errors++;
+          } else {
+            errors += runCmdHandler(handler, file, srcFs, recursive);
+          }
+        } catch (IOException e) {
+          String msg = (e.getMessage() != null ? e.getLocalizedMessage() :
+            (e.getCause().getMessage() != null ? 
+                e.getCause().getLocalizedMessage() : "null"));
+          System.err.println(handler.getName() + ": could not get status for '"
+                                        + path + "': " + msg.split("\n")[0]);        
+        }
+      }
+    }
+    
+    return (errors > 0 || handler.getErrorCode() != 0) ? 1 : 0;
+  }
+  
+  /**
+   * Return an abbreviated English-language desc of the byte length
+   * @deprecated Consider using {@link org.apache.hadoop.util.StringUtils#byteDesc} instead.
+   */
+  @Deprecated
+  public static String byteDesc(long len) {
+    return StringUtils.byteDesc(len);
+  }
+
+  /**
+   * @deprecated Consider using {@link org.apache.hadoop.util.StringUtils#limitDecimalTo2} instead.
+   */
+  @Deprecated
+  public static synchronized String limitDecimalTo2(double d) {
+    return StringUtils.limitDecimalTo2(d);
+  }
+
+  private void printHelp(String cmd) {
+    String summary = "hadoop fs is the command to execute fs commands. " +
+      "The full syntax is: \n\n" +
+      "hadoop fs [-fs <local | file system URI>] [-conf <configuration file>]\n\t" +
+      "[-D <property=value>] [-ls <path>] [-lsr <path>] [-df [<path>]] [-du <path>]\n\t" + 
+      "[-dus <path>] [-mv <src> <dst>] [-cp <src> <dst>] [-rm <src>]\n\t" + 
+      "[-rmr <src>] [-put <localsrc> ... <dst>] [-copyFromLocal <localsrc> ... <dst>]\n\t" +
+      "[-moveFromLocal <localsrc> ... <dst>] [" + 
+      GET_SHORT_USAGE + "\n\t" +
+      "[-getmerge <src> <localdst> [addnl]] [-cat <src>]\n\t" +
+      "[" + COPYTOLOCAL_SHORT_USAGE + "] [-moveToLocal <src> <localdst>]\n\t" +
+      "[-mkdir <path>] [-report] [" + SETREP_SHORT_USAGE + "]\n\t" +
+      "[-touchz <path>] [-test -[ezd] <path>] [-stat [format] <path>]\n\t" +
+      "[-tail [-f] <path>] [-text <path>]\n\t" +
+      "[" + FsShellPermissions.CHMOD_USAGE + "]\n\t" +
+      "[" + FsShellPermissions.CHOWN_USAGE + "]\n\t" +
+      "[" + FsShellPermissions.CHGRP_USAGE + "]\n\t" +      
+      "[" + Count.USAGE + "]\n\t" +      
+      "[-help [cmd]]\n";
+
+    String conf ="-conf <configuration file>:  Specify an application configuration file.";
+ 
+    String D = "-D <property=value>:  Use value for given property.";
+  
+    String fs = "-fs [local | <file system URI>]: \tSpecify the file system to use.\n" + 
+      "\t\tIf not specified, the current configuration is used, \n" +
+      "\t\ttaken from the following, in increasing precedence: \n" + 
+      "\t\t\tcore-default.xml inside the hadoop jar file \n" +
+      "\t\t\tcore-site.xml in $HADOOP_CONF_DIR \n" +
+      "\t\t'local' means use the local file system as your DFS. \n" +
+      "\t\t<file system URI> specifies a particular file system to \n" +
+      "\t\tcontact. This argument is optional but if used must appear\n" +
+      "\t\tappear first on the command line.  Exactly one additional\n" +
+      "\t\targument must be specified. \n";
+
+        
+    String ls = "-ls <path>: \tList the contents that match the specified file pattern. If\n" + 
+      "\t\tpath is not specified, the contents of /user/<currentUser>\n" +
+      "\t\twill be listed. Directory entries are of the form \n" +
+      "\t\t\tdirName (full path) <dir> \n" +
+      "\t\tand file entries are of the form \n" + 
+      "\t\t\tfileName(full path) <r n> size \n" +
+      "\t\twhere n is the number of replicas specified for the file \n" + 
+      "\t\tand size is the size of the file, in bytes.\n";
+
+    String lsr = "-lsr <path>: \tRecursively list the contents that match the specified\n" +
+      "\t\tfile pattern.  Behaves very similarly to hadoop fs -ls,\n" + 
+      "\t\texcept that the data is shown for all the entries in the\n" +
+      "\t\tsubtree.\n";
+
+    String df = "-df [<path>]: \tShows the capacity, free and used space of the filesystem.\n"+
+      "\t\tIf the filesystem has multiple partitions, and no path to a particular partition\n"+
+      "\t\tis specified, then the status of the root partitions will be shown.\n";
+
+    String du = "-du <path>: \tShow the amount of space, in bytes, used by the files that \n" +
+      "\t\tmatch the specified file pattern.  Equivalent to the unix\n" + 
+      "\t\tcommand \"du -sb <path>/*\" in case of a directory, \n" +
+      "\t\tand to \"du -b <path>\" in case of a file.\n" +
+      "\t\tThe output is in the form \n" + 
+      "\t\t\tname(full path) size (in bytes)\n"; 
+
+    String dus = "-dus <path>: \tShow the amount of space, in bytes, used by the files that \n" +
+      "\t\tmatch the specified file pattern.  Equivalent to the unix\n" + 
+      "\t\tcommand \"du -sb\"  The output is in the form \n" + 
+      "\t\t\tname(full path) size (in bytes)\n"; 
+    
+    String mv = "-mv <src> <dst>:   Move files that match the specified file pattern <src>\n" +
+      "\t\tto a destination <dst>.  When moving multiple files, the \n" +
+      "\t\tdestination must be a directory. \n";
+
+    String cp = "-cp <src> <dst>:   Copy files that match the file pattern <src> to a \n" +
+      "\t\tdestination.  When copying multiple files, the destination\n" +
+      "\t\tmust be a directory. \n";
+
+    String rm = "-rm <src>: \tDelete all files that match the specified file pattern.\n" +
+      "\t\tEquivlent to the Unix command \"rm <src>\"\n";
+
+    String rmr = "-rmr <src>: \tRemove all directories which match the specified file \n" +
+      "\t\tpattern. Equivlent to the Unix command \"rm -rf <src>\"\n";
+
+    String put = "-put <localsrc> ... <dst>: \tCopy files " + 
+    "from the local file system \n\t\tinto fs. \n";
+
+    String copyFromLocal = "-copyFromLocal <localsrc> ... <dst>:" +
+    " Identical to the -put command.\n";
+
+    String moveFromLocal = "-moveFromLocal <localsrc> ... <dst>:" +
+    " Same as -put, except that the source is\n\t\tdeleted after it's copied.\n"; 
+
+    String get = GET_SHORT_USAGE
+      + ":  Copy files that match the file pattern <src> \n" +
+      "\t\tto the local name.  <src> is kept.  When copying mutiple, \n" +
+      "\t\tfiles, the destination must be a directory. \n";
+
+    String getmerge = "-getmerge <src> <localdst>:  Get all the files in the directories that \n" +
+      "\t\tmatch the source file pattern and merge and sort them to only\n" +
+      "\t\tone file on local fs. <src> is kept.\n";
+
+    String cat = "-cat <src>: \tFetch all files that match the file pattern <src> \n" +
+      "\t\tand display their content on stdout.\n";
+
+    
+    String text = "-text <src>: \tTakes a source file and outputs the file in text format.\n" +
+      "\t\tThe allowed formats are zip and TextRecordInputStream.\n";
+         
+    
+    String copyToLocal = COPYTOLOCAL_SHORT_USAGE
+                         + ":  Identical to the -get command.\n";
+
+    String moveToLocal = "-moveToLocal <src> <localdst>:  Not implemented yet \n";
+        
+    String mkdir = "-mkdir <path>: \tCreate a directory in specified location. \n";
+
+    String setrep = SETREP_SHORT_USAGE
+      + ":  Set the replication level of a file. \n"
+      + "\t\tThe -R flag requests a recursive change of replication level \n"
+      + "\t\tfor an entire tree.\n";
+
+    String touchz = "-touchz <path>: Write a timestamp in yyyy-MM-dd HH:mm:ss format\n" +
+      "\t\tin a file at <path>. An error is returned if the file exists with non-zero length\n";
+
+    String test = "-test -[ezd] <path>: If file { exists, has zero length, is a directory\n" +
+      "\t\tthen return 0, else return 1.\n";
+
+    String stat = "-stat [format] <path>: Print statistics about the file/directory at <path>\n" +
+      "\t\tin the specified format. Format accepts filesize in blocks (%b), filename (%n),\n" +
+      "\t\tblock size (%o), replication (%r), modification date (%y, %Y)\n";
+
+    String tail = TAIL_USAGE
+      + ":  Show the last 1KB of the file. \n"
+      + "\t\tThe -f option shows apended data as the file grows. \n";
+
+    String chmod = FsShellPermissions.CHMOD_USAGE + "\n" +
+      "\t\tChanges permissions of a file.\n" +
+      "\t\tThis works similar to shell's chmod with a few exceptions.\n\n" +
+      "\t-R\tmodifies the files recursively. This is the only option\n" +
+      "\t\tcurrently supported.\n\n" +
+      "\tMODE\tMode is same as mode used for chmod shell command.\n" +
+      "\t\tOnly letters recognized are 'rwxXt'. E.g. +t,a+r,g-w,+rwx,o=r\n\n" +
+      "\tOCTALMODE Mode specifed in 3 or 4 digits. If 4 digits, the first may\n" +
+      "\tbe 1 or 0 to turn the sticky bit on or off, respectively.  Unlike " +
+      "\tshell command, it is not possible to specify only part of the mode\n" +
+      "\t\tE.g. 754 is same as u=rwx,g=rx,o=r\n\n" +
+      "\t\tIf none of 'augo' is specified, 'a' is assumed and unlike\n" +
+      "\t\tshell command, no umask is applied.\n";
+    
+    String chown = FsShellPermissions.CHOWN_USAGE + "\n" +
+      "\t\tChanges owner and group of a file.\n" +
+      "\t\tThis is similar to shell's chown with a few exceptions.\n\n" +
+      "\t-R\tmodifies the files recursively. This is the only option\n" +
+      "\t\tcurrently supported.\n\n" +
+      "\t\tIf only owner or group is specified then only owner or\n" +
+      "\t\tgroup is modified.\n\n" +
+      "\t\tThe owner and group names may only cosists of digits, alphabet,\n"+
+      "\t\tand any of '-_.@/' i.e. [-_.@/a-zA-Z0-9]. The names are case\n" +
+      "\t\tsensitive.\n\n" +
+      "\t\tWARNING: Avoid using '.' to separate user name and group though\n" +
+      "\t\tLinux allows it. If user names have dots in them and you are\n" +
+      "\t\tusing local file system, you might see surprising results since\n" +
+      "\t\tshell command 'chown' is used for local files.\n";
+    
+    String chgrp = FsShellPermissions.CHGRP_USAGE + "\n" +
+      "\t\tThis is equivalent to -chown ... :GROUP ...\n";
+    
+    String help = "-help [cmd]: \tDisplays help for given command or all commands if none\n" +
+      "\t\tis specified.\n";
+
+    if ("fs".equals(cmd)) {
+      System.out.println(fs);
+    } else if ("conf".equals(cmd)) {
+      System.out.println(conf);
+    } else if ("D".equals(cmd)) {
+      System.out.println(D);
+    } else if ("ls".equals(cmd)) {
+      System.out.println(ls);
+    } else if ("lsr".equals(cmd)) {
+      System.out.println(lsr);
+    } else if ("df".equals(cmd)) {
+      System.out.println(df);
+    } else if ("du".equals(cmd)) {
+      System.out.println(du);
+    } else if ("dus".equals(cmd)) {
+      System.out.println(dus);
+    } else if ("rm".equals(cmd)) {
+      System.out.println(rm);
+    } else if ("rmr".equals(cmd)) {
+      System.out.println(rmr);
+    } else if ("mkdir".equals(cmd)) {
+      System.out.println(mkdir);
+    } else if ("mv".equals(cmd)) {
+      System.out.println(mv);
+    } else if ("cp".equals(cmd)) {
+      System.out.println(cp);
+    } else if ("put".equals(cmd)) {
+      System.out.println(put);
+    } else if ("copyFromLocal".equals(cmd)) {
+      System.out.println(copyFromLocal);
+    } else if ("moveFromLocal".equals(cmd)) {
+      System.out.println(moveFromLocal);
+    } else if ("get".equals(cmd)) {
+      System.out.println(get);
+    } else if ("getmerge".equals(cmd)) {
+      System.out.println(getmerge);
+    } else if ("copyToLocal".equals(cmd)) {
+      System.out.println(copyToLocal);
+    } else if ("moveToLocal".equals(cmd)) {
+      System.out.println(moveToLocal);
+    } else if ("cat".equals(cmd)) {
+      System.out.println(cat);
+    } else if ("get".equals(cmd)) {
+      System.out.println(get);
+    } else if ("setrep".equals(cmd)) {
+      System.out.println(setrep);
+    } else if ("touchz".equals(cmd)) {
+      System.out.println(touchz);
+    } else if ("test".equals(cmd)) {
+      System.out.println(test);
+    } else if ("text".equals(cmd)) {
+      System.out.println(text);
+    } else if ("stat".equals(cmd)) {
+      System.out.println(stat);
+    } else if ("tail".equals(cmd)) {
+      System.out.println(tail);
+    } else if ("chmod".equals(cmd)) {
+      System.out.println(chmod);
+    } else if ("chown".equals(cmd)) {
+      System.out.println(chown);
+    } else if ("chgrp".equals(cmd)) {
+      System.out.println(chgrp);
+    } else if (Count.matches(cmd)) {
+      System.out.println(Count.DESCRIPTION);
+    } else if ("help".equals(cmd)) {
+      System.out.println(help);
+    } else {
+      System.out.println(summary);
+      System.out.println(fs);
+      System.out.println(ls);
+      System.out.println(lsr);
+      System.out.println(df);
+      System.out.println(du);
+      System.out.println(dus);
+      System.out.println(mv);
+      System.out.println(cp);
+      System.out.println(rm);
+      System.out.println(rmr);
+      System.out.println(put);
+      System.out.println(copyFromLocal);
+      System.out.println(moveFromLocal);
+      System.out.println(get);
+      System.out.println(getmerge);
+      System.out.println(cat);
+      System.out.println(copyToLocal);
+      System.out.println(moveToLocal);
+      System.out.println(mkdir);
+      System.out.println(setrep);
+      System.out.println(tail);
+      System.out.println(touchz);
+      System.out.println(test);
+      System.out.println(text);
+      System.out.println(stat);
+      System.out.println(chmod);
+      System.out.println(chown);      
+      System.out.println(chgrp);
+      System.out.println(Count.DESCRIPTION);
+      System.out.println(help);
+    }        
+
+                           
+  }
+
+  /**
+   * Apply operation specified by 'cmd' on all parameters
+   * starting from argv[startindex].
+   */
+  private int doall(String cmd, String argv[], int startindex) {
+    int exitCode = 0;
+    int i = startindex;
+    //
+    // for each source file, issue the command
+    //
+    for (; i < argv.length; i++) {
+      try {
+        //
+        // issue the command to the fs
+        //
+        if ("-cat".equals(cmd)) {
+          cat(argv[i], true);
+        } else if ("-mkdir".equals(cmd)) {
+          mkdir(argv[i]);
+        } else if ("-rm".equals(cmd)) {
+          delete(argv[i], false);
+        } else if ("-rmr".equals(cmd)) {
+          delete(argv[i], true);
+        } else if ("-df".equals(cmd)) {
+          df(argv[i]);
+        } else if ("-du".equals(cmd)) {
+          du(argv[i]);
+        } else if ("-dus".equals(cmd)) {
+          dus(argv[i]);
+        } else if (Count.matches(cmd)) {
+          new Count(argv, i, getConf()).runAll();
+        } else if ("-ls".equals(cmd)) {
+          exitCode = ls(argv[i], false);
+        } else if ("-lsr".equals(cmd)) {
+          exitCode = ls(argv[i], true);
+        } else if ("-touchz".equals(cmd)) {
+          touchz(argv[i]);
+        } else if ("-text".equals(cmd)) {
+          text(argv[i]);
+        }
+      } catch (RemoteException e) {
+        //
+        // This is a error returned by hadoop server. Print
+        // out the first line of the error message.
+        //
+        exitCode = -1;
+        try {
+          String[] content;
+          content = e.getLocalizedMessage().split("\n");
+          System.err.println(cmd.substring(1) + ": " +
+                             content[0]);
+        } catch (Exception ex) {
+          System.err.println(cmd.substring(1) + ": " +
+                             ex.getLocalizedMessage());
+        }
+      } catch (IOException e) {
+        //
+        // IO exception encountered locally.
+        //
+        exitCode = -1;
+        String content = e.getLocalizedMessage();
+        if (content != null) {
+          content = content.split("\n")[0];
+        }
+        System.err.println(cmd.substring(1) + ": " +
+                          content);
+      }
+    }
+    return exitCode;
+  }
+
+  /**
+   * Displays format of commands.
+   * 
+   */
+  private static void printUsage(String cmd) {
+    String prefix = "Usage: java " + FsShell.class.getSimpleName();
+    if ("-fs".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [-fs <local | file system URI>]");
+    } else if ("-conf".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [-conf <configuration file>]");
+    } else if ("-D".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [-D <[property=value>]");
+    } else if ("-ls".equals(cmd) || "-lsr".equals(cmd) ||
+               "-du".equals(cmd) || "-dus".equals(cmd) ||
+               "-rm".equals(cmd) || "-rmr".equals(cmd) ||
+               "-touchz".equals(cmd) || "-mkdir".equals(cmd) ||
+               "-text".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [" + cmd + " <path>]");
+    } else if ("-df".equals(cmd) ) {
+      System.err.println("Usage: java FsShell" +
+                         " [" + cmd + " [<path>]]");
+    } else if (Count.matches(cmd)) {
+      System.err.println(prefix + " [" + Count.USAGE + "]");
+    } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [" + cmd + " <src> <dst>]");
+    } else if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd) ||
+               "-moveFromLocal".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [" + cmd + " <localsrc> ... <dst>]");
+    } else if ("-get".equals(cmd)) {
+      System.err.println("Usage: java FsShell [" + GET_SHORT_USAGE + "]"); 
+    } else if ("-copyToLocal".equals(cmd)) {
+      System.err.println("Usage: java FsShell [" + COPYTOLOCAL_SHORT_USAGE+ "]"); 
+    } else if ("-moveToLocal".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [" + cmd + " [-crc] <src> <localdst>]");
+    } else if ("-cat".equals(cmd)) {
+      System.err.println("Usage: java FsShell" + 
+                         " [" + cmd + " <src>]");
+    } else if ("-setrep".equals(cmd)) {
+      System.err.println("Usage: java FsShell [" + SETREP_SHORT_USAGE + "]");
+    } else if ("-test".equals(cmd)) {
+      System.err.println("Usage: java FsShell" +
+                         " [-test -[ezd] <path>]");
+    } else if ("-stat".equals(cmd)) {
+      System.err.println("Usage: java FsShell" +
+                         " [-stat [format] <path>]");
+    } else if ("-tail".equals(cmd)) {
+      System.err.println("Usage: java FsShell [" + TAIL_USAGE + "]");
+    } else {
+      System.err.println("Usage: java FsShell");
+      System.err.println("           [-ls <path>]");
+      System.err.println("           [-lsr <path>]");
+      System.err.println("           [-df [<path>]]");
+      System.err.println("           [-du <path>]");
+      System.err.println("           [-dus <path>]");
+      System.err.println("           [" + Count.USAGE + "]");
+      System.err.println("           [-mv <src> <dst>]");
+      System.err.println("           [-cp <src> <dst>]");
+      System.err.println("           [-rm <path>]");
+      System.err.println("           [-rmr <path>]");
+      System.err.println("           [-expunge]");
+      System.err.println("           [-put <localsrc> ... <dst>]");
+      System.err.println("           [-copyFromLocal <localsrc> ... <dst>]");
+      System.err.println("           [-moveFromLocal <localsrc> ... <dst>]");
+      System.err.println("           [" + GET_SHORT_USAGE + "]");
+      System.err.println("           [-getmerge <src> <localdst> [addnl]]");
+      System.err.println("           [-cat <src>]");
+      System.err.println("           [-text <src>]");
+      System.err.println("           [" + COPYTOLOCAL_SHORT_USAGE + "]");
+      System.err.println("           [-moveToLocal [-crc] <src> <localdst>]");
+      System.err.println("           [-mkdir <path>]");
+      System.err.println("           [" + SETREP_SHORT_USAGE + "]");
+      System.err.println("           [-touchz <path>]");
+      System.err.println("           [-test -[ezd] <path>]");
+      System.err.println("           [-stat [format] <path>]");
+      System.err.println("           [" + TAIL_USAGE + "]");
+      System.err.println("           [" + FsShellPermissions.CHMOD_USAGE + "]");      
+      System.err.println("           [" + FsShellPermissions.CHOWN_USAGE + "]");
+      System.err.println("           [" + FsShellPermissions.CHGRP_USAGE + "]");
+      System.err.println("           [-help [cmd]]");
+      System.err.println();
+      ToolRunner.printGenericCommandUsage(System.err);
+    }
+  }
+
+  /**
+   * run
+   */
+  public int run(String argv[]) throws Exception {
+
+    if (argv.length < 1) {
+      printUsage(""); 
+      return -1;
+    }
+
+    int exitCode = -1;
+    int i = 0;
+    String cmd = argv[i++];
+    //
+    // verify that we have enough command line parameters
+    //
+    if ("-put".equals(cmd) || "-test".equals(cmd) ||
+        "-copyFromLocal".equals(cmd) || "-moveFromLocal".equals(cmd)) {
+      if (argv.length < 3) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    } else if ("-get".equals(cmd) || 
+               "-copyToLocal".equals(cmd) || "-moveToLocal".equals(cmd)) {
+      if (argv.length < 3) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
+      if (argv.length < 3) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    } else if ("-rm".equals(cmd) || "-rmr".equals(cmd) ||
+               "-cat".equals(cmd) || "-mkdir".equals(cmd) ||
+               "-touchz".equals(cmd) || "-stat".equals(cmd) ||
+               "-text".equals(cmd)) {
+      if (argv.length < 2) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    }
+    // initialize FsShell
+    try {
+      init();
+    } catch (RPC.VersionMismatch v) { 
+      System.err.println("Version Mismatch between client and server" +
+                         "... command aborted.");
+      return exitCode;
+    } catch (IOException e) {
+      System.err.println("Bad connection to FS. command aborted.");
+      return exitCode;
+    }
+
+    exitCode = 0;
+    try {
+      if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd)) {
+        Path[] srcs = new Path[argv.length-2];
+        for (int j=0 ; i < argv.length-1 ;) 
+          srcs[j++] = new Path(argv[i++]);
+        copyFromLocal(srcs, argv[i++]);
+      } else if ("-moveFromLocal".equals(cmd)) {
+        Path[] srcs = new Path[argv.length-2];
+        for (int j=0 ; i < argv.length-1 ;) 
+          srcs[j++] = new Path(argv[i++]);
+        moveFromLocal(srcs, argv[i++]);
+      } else if ("-get".equals(cmd) || "-copyToLocal".equals(cmd)) {
+        copyToLocal(argv, i);
+      } else if ("-getmerge".equals(cmd)) {
+        if (argv.length>i+2)
+          copyMergeToLocal(argv[i++], new Path(argv[i++]), Boolean.parseBoolean(argv[i++]));
+        else
+          copyMergeToLocal(argv[i++], new Path(argv[i++]));
+      } else if ("-cat".equals(cmd)) {
+        exitCode = doall(cmd, argv, i);
+      } else if ("-text".equals(cmd)) {
+        exitCode = doall(cmd, argv, i);
+      } else if ("-moveToLocal".equals(cmd)) {
+        moveToLocal(argv[i++], new Path(argv[i++]));
+      } else if ("-setrep".equals(cmd)) {
+        setReplication(argv, i);           
+      } else if ("-chmod".equals(cmd) || 
+                 "-chown".equals(cmd) ||
+                 "-chgrp".equals(cmd)) {
+        FsShellPermissions.changePermissions(fs, cmd, argv, i, this);
+      } else if ("-ls".equals(cmd)) {
+        if (i < argv.length) {
+          exitCode = doall(cmd, argv, i);
+        } else {
+          exitCode = ls(Path.CUR_DIR, false);
+        } 
+      } else if ("-lsr".equals(cmd)) {
+        if (i < argv.length) {
+          exitCode = doall(cmd, argv, i);
+        } else {
+          exitCode = ls(Path.CUR_DIR, true);
+        } 
+      } else if ("-mv".equals(cmd)) {
+        exitCode = rename(argv, getConf());
+      } else if ("-cp".equals(cmd)) {
+        exitCode = copy(argv, getConf());
+      } else if ("-rm".equals(cmd)) {
+        exitCode = doall(cmd, argv, i);
+      } else if ("-rmr".equals(cmd)) {
+        exitCode = doall(cmd, argv, i);
+      } else if ("-expunge".equals(cmd)) {
+        expunge();
+      } else if ("-df".equals(cmd)) {
+        if (argv.length-1 > 0) {
+          exitCode = doall(cmd, argv, i);
+        } else {
+          df(null);
+        }
+      } else if ("-du".equals(cmd)) {
+        if (i < argv.length) {
+          exitCode = doall(cmd, argv, i);
+        } else {
+          du(".");
+        }
+      } else if ("-dus".equals(cmd)) {
+        if (i < argv.length) {
+          exitCode = doall(cmd, argv, i);
+        } else {
+          dus(".");
+        }         
+      } else if (Count.matches(cmd)) {
+        exitCode = new Count(argv, i, getConf()).runAll();
+      } else if ("-mkdir".equals(cmd)) {
+        exitCode = doall(cmd, argv, i);
+      } else if ("-touchz".equals(cmd)) {
+        exitCode = doall(cmd, argv, i);
+      } else if ("-test".equals(cmd)) {
+        exitCode = test(argv, i);
+      } else if ("-stat".equals(cmd)) {
+        if (i + 1 < argv.length) {
+          stat(argv[i++].toCharArray(), argv[i++]);
+        } else {
+          stat("%y".toCharArray(), argv[i]);
+        }
+      } else if ("-help".equals(cmd)) {
+        if (i < argv.length) {
+          printHelp(argv[i]);
+        } else {
+          printHelp("");
+        }
+      } else if ("-tail".equals(cmd)) {
+        tail(argv, i);           
+      } else {
+        exitCode = -1;
+        System.err.println(cmd.substring(1) + ": Unknown command");
+        printUsage("");
+      }
+    } catch (IllegalArgumentException arge) {
+      exitCode = -1;
+      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
+      printUsage(cmd);
+    } catch (RemoteException e) {
+      //
+      // This is a error returned by hadoop server. Print
+      // out the first line of the error mesage, ignore the stack trace.
+      exitCode = -1;
+      try {
+        String[] content;
+        content = e.getLocalizedMessage().split("\n");
+        System.err.println(cmd.substring(1) + ": " + 
+                           content[0]);
+      } catch (Exception ex) {
+        System.err.println(cmd.substring(1) + ": " + 
+                           ex.getLocalizedMessage());  
+      }
+    } catch (IOException e) {
+      //
+      // IO exception encountered locally.
+      // 
+      exitCode = -1;
+      System.err.println(cmd.substring(1) + ": " + 
+                         e.getLocalizedMessage());  
+    } catch (Exception re) {
+      exitCode = -1;
+      System.err.println(cmd.substring(1) + ": " + re.getLocalizedMessage());  
+    } finally {
+    }
+    return exitCode;
+  }
+
+  public void close() throws IOException {
+    if (fs != null) {
+      fs.close();
+      fs = null;
+    }
+  }
+
+  /**
+   * main() has some simple utility methods
+   */
+  public static void main(String argv[]) throws Exception {
+    FsShell shell = new FsShell();
+    int res;
+    try {
+      res = ToolRunner.run(shell, argv);
+    } finally {
+      shell.close();
+    }
+    System.exit(res);
+  }
+
+  /**
+   * Accumulate exceptions if there is any.  Throw them at last.
+   */
+  private abstract class DelayedExceptionThrowing {
+    abstract void process(Path p, FileSystem srcFs) throws IOException;
+
+    final void globAndProcess(Path srcPattern, FileSystem srcFs
+        ) throws IOException {
+      List<IOException> exceptions = new ArrayList<IOException>();
+      for(Path p : FileUtil.stat2Paths(srcFs.globStatus(srcPattern), 
+                                       srcPattern))
+        try { process(p, srcFs); } 
+        catch(IOException ioe) { exceptions.add(ioe); }
+    
+      if (!exceptions.isEmpty())
+        if (exceptions.size() == 1)
+          throw exceptions.get(0);
+        else 
+          throw new IOException("Multiple IOExceptions: " + exceptions);
+    }
+  }
+}

+ 315 - 0
src/java/org/apache/hadoop/fs/FsShellPermissions.java

@@ -0,0 +1,315 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.FsShell.CmdHandler;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+
+/**
+ * This class is the home for file permissions related commands.
+ * Moved to this separate class since FsShell is getting too large.
+ */
+class FsShellPermissions {
+  
+  /*========== chmod ==========*/
+
+  /*
+   * The pattern is almost as flexible as mode allowed by chmod shell command.
+   * The main restriction is that we recognize only rwxXt. To reduce errors we
+   * also enforce octal mode specifications of either 3 digits without a sticky
+   * bit setting or four digits with a sticky bit setting.
+   */
+  private static Pattern chmodNormalPattern =
+   Pattern.compile("\\G\\s*([ugoa]*)([+=-]+)([rwxXt]+)([,\\s]*)\\s*");
+  private static Pattern chmodOctalPattern =
+            Pattern.compile("^\\s*[+]?([01]?)([0-7]{3})\\s*$");
+
+  static String CHMOD_USAGE =
+                            "-chmod [-R] <MODE[,MODE]... | OCTALMODE> PATH...";
+
+  private static class ChmodHandler extends CmdHandler {
+
+    private short userMode;
+    private short groupMode;
+    private short othersMode;
+    private short stickyMode;
+    private char userType = '+';
+    private char groupType = '+';
+    private char othersType = '+';
+    private char stickyBitType = '+';
+
+    private void applyNormalPattern(String modeStr, Matcher matcher)
+                                    throws IOException {
+      // Are there multiple permissions stored in one chmod?
+      boolean commaSeperated = false;
+
+      for(int i=0; i < 1 || matcher.end() < modeStr.length(); i++) {
+        if (i>0 && (!commaSeperated || !matcher.find())) {
+          patternError(modeStr);
+        }
+
+        /* groups : 1 : [ugoa]*
+         *          2 : [+-=]
+         *          3 : [rwxXt]+
+         *          4 : [,\s]*
+         */
+
+        String str = matcher.group(2);
+        char type = str.charAt(str.length() - 1);
+
+        boolean user, group, others, stickyBit;
+        user = group = others = stickyBit = false;
+
+        for(char c : matcher.group(1).toCharArray()) {
+          switch (c) {
+          case 'u' : user = true; break;
+          case 'g' : group = true; break;
+          case 'o' : others = true; break;
+          case 'a' : break;
+          default  : throw new RuntimeException("Unexpected");          
+          }
+        }
+
+        if (!(user || group || others)) { // same as specifying 'a'
+          user = group = others = true;
+        }
+
+        short mode = 0;
+
+        for(char c : matcher.group(3).toCharArray()) {
+          switch (c) {
+          case 'r' : mode |= 4; break;
+          case 'w' : mode |= 2; break;
+          case 'x' : mode |= 1; break;
+          case 'X' : mode |= 8; break;
+          case 't' : stickyBit = true; break;
+          default  : throw new RuntimeException("Unexpected");
+          }
+        }
+
+        if ( user ) {
+          userMode = mode;
+          userType = type;
+        }
+
+        if ( group ) {
+          groupMode = mode;
+          groupType = type;
+        }
+
+        if ( others ) {
+          othersMode = mode;
+          othersType = type;
+          
+          stickyMode = (short) (stickyBit ? 1 : 0);
+          stickyBitType = type;
+        }
+
+        commaSeperated = matcher.group(4).contains(",");
+      }
+    }
+
+    private void applyOctalPattern(String modeStr, Matcher matcher) {
+      userType = groupType = othersType = '=';
+
+      // Check if sticky bit is specified
+      String sb = matcher.group(1);
+      if(!sb.isEmpty()) {
+        stickyMode = Short.valueOf(sb.substring(0, 1));
+        stickyBitType = '=';
+      }
+
+      String str = matcher.group(2);
+      userMode = Short.valueOf(str.substring(0, 1));
+      groupMode = Short.valueOf(str.substring(1, 2));
+      othersMode = Short.valueOf(str.substring(2, 3));      
+    }
+
+    private void patternError(String mode) throws IOException {
+      throw new IOException("chmod : mode '" + mode + 
+                            "' does not match the expected pattern.");      
+    }
+
+    ChmodHandler(FileSystem fs, String modeStr) throws IOException {
+      super("chmod", fs);
+      Matcher matcher = null;
+
+      if ((matcher = chmodNormalPattern.matcher(modeStr)).find()) {
+        applyNormalPattern(modeStr, matcher);
+      } else if ((matcher = chmodOctalPattern.matcher(modeStr)).matches()) {
+        applyOctalPattern(modeStr, matcher);
+      } else {
+        patternError(modeStr);
+      }
+    }
+
+    private int applyChmod(char type, int mode, int existing, boolean exeOk) {
+      boolean capX = false;
+
+      if ((mode&8) != 0) { // convert X to x;
+        capX = true;
+        mode &= ~8;
+        mode |= 1;
+      }
+
+      switch (type) {
+      case '+' : mode = mode | existing; break;
+      case '-' : mode = (~mode) & existing; break;
+      case '=' : break;
+      default  : throw new RuntimeException("Unexpected");      
+      }
+
+      // if X is specified add 'x' only if exeOk or x was already set.
+      if (capX && !exeOk && (mode&1) != 0 && (existing&1) == 0) {
+        mode &= ~1; // remove x
+      }
+
+      return mode;
+    }
+
+    @Override
+    public void run(FileStatus file, FileSystem srcFs) throws IOException {
+      FsPermission perms = file.getPermission();
+      int existing = perms.toShort();
+      boolean exeOk = file.isDir() || (existing & 0111) != 0;
+      int newperms = ( applyChmod(stickyBitType, stickyMode,
+                             (existing>>>9), false) << 9 |
+                       applyChmod(userType, userMode,
+                             (existing>>>6)&7, exeOk) << 6 |
+                       applyChmod(groupType, groupMode,
+                             (existing>>>3)&7, exeOk) << 3 |
+                       applyChmod(othersType, othersMode, existing&7, exeOk));
+
+      if (existing != newperms) {
+        try {
+          srcFs.setPermission(file.getPath(), 
+                                new FsPermission((short)newperms));
+        } catch (IOException e) {
+          System.err.println(getName() + ": changing permissions of '" + 
+                             file.getPath() + "':" + e.getMessage());
+        }
+      }
+    }
+  }
+
+  /*========== chown ==========*/
+  
+  static private String allowedChars = "[-_./@a-zA-Z0-9]";
+  ///allows only "allowedChars" above in names for owner and group
+  static private Pattern chownPattern = 
+         Pattern.compile("^\\s*(" + allowedChars + "+)?" +
+                          "([:](" + allowedChars + "*))?\\s*$");
+  static private Pattern chgrpPattern = 
+         Pattern.compile("^\\s*(" + allowedChars + "+)\\s*$");
+  
+  static String CHOWN_USAGE = "-chown [-R] [OWNER][:[GROUP]] PATH...";
+  static String CHGRP_USAGE = "-chgrp [-R] GROUP PATH...";  
+
+  private static class ChownHandler extends CmdHandler {
+    protected String owner = null;
+    protected String group = null;
+
+    protected ChownHandler(String cmd, FileSystem fs) { //for chgrp
+      super(cmd, fs);
+    }
+
+    ChownHandler(FileSystem fs, String ownerStr) throws IOException {
+      super("chown", fs);
+      Matcher matcher = chownPattern.matcher(ownerStr);
+      if (!matcher.matches()) {
+        throw new IOException("'" + ownerStr + "' does not match " +
+                              "expected pattern for [owner][:group].");
+      }
+      owner = matcher.group(1);
+      group = matcher.group(3);
+      if (group != null && group.length() == 0) {
+        group = null;
+      }
+      if (owner == null && group == null) {
+        throw new IOException("'" + ownerStr + "' does not specify " +
+                              " onwer or group.");
+      }
+    }
+
+    @Override
+    public void run(FileStatus file, FileSystem srcFs) throws IOException {
+      //Should we do case insensitive match?  
+      String newOwner = (owner == null || owner.equals(file.getOwner())) ?
+                        null : owner;
+      String newGroup = (group == null || group.equals(file.getGroup())) ?
+                        null : group;
+
+      if (newOwner != null || newGroup != null) {
+        try {
+          srcFs.setOwner(file.getPath(), newOwner, newGroup);
+        } catch (IOException e) {
+          System.err.println(getName() + ": changing ownership of '" + 
+                             file.getPath() + "':" + e.getMessage());
+
+        }
+      }
+    }
+  }
+
+  /*========== chgrp ==========*/    
+  
+  private static class ChgrpHandler extends ChownHandler {
+    ChgrpHandler(FileSystem fs, String groupStr) throws IOException {
+      super("chgrp", fs);
+
+      Matcher matcher = chgrpPattern.matcher(groupStr);
+      if (!matcher.matches()) {
+        throw new IOException("'" + groupStr + "' does not match " +
+        "expected pattern for group");
+      }
+      group = matcher.group(1);
+    }
+  }
+
+  static void changePermissions(FileSystem fs, String cmd, 
+                                String argv[], int startIndex, FsShell shell)
+                                throws IOException {
+    CmdHandler handler = null;
+    boolean recursive = false;
+
+    // handle common arguments, currently only "-R" 
+    for (; startIndex < argv.length && argv[startIndex].equals("-R"); 
+    startIndex++) {
+      recursive = true;
+    }
+
+    if ( startIndex >= argv.length ) {
+      throw new IOException("Not enough arguments for the command");
+    }
+
+    if (cmd.equals("-chmod")) {
+      handler = new ChmodHandler(fs, argv[startIndex++]);
+    } else if (cmd.equals("-chown")) {
+      handler = new ChownHandler(fs, argv[startIndex++]);
+    } else if (cmd.equals("-chgrp")) {
+      handler = new ChgrpHandler(fs, argv[startIndex++]);
+    }
+
+    shell.runCmdHandler(handler, argv, startIndex, recursive);
+  } 
+}

+ 70 - 0
src/java/org/apache/hadoop/fs/FsStatus.java

@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+/** This class is used to represent the capacity, free and used space on a
+  * {@link FileSystem}.
+  */
+public class FsStatus implements Writable {
+  private long capacity;
+  private long used;
+  private long remaining;
+
+  /** Construct a FsStatus object, using the specified statistics */
+  public FsStatus(long capacity, long used, long remaining) {
+    this.capacity = capacity;
+    this.used = used;
+    this.remaining = remaining;
+  }
+
+  /** Return the capacity in bytes of the file system */
+  public long getCapacity() {
+    return capacity;
+  }
+
+  /** Return the number of bytes used on the file system */
+  public long getUsed() {
+    return used;
+  }
+
+  /** Return the number of remaining bytes on the file system */
+  public long getRemaining() {
+    return remaining;
+  }
+
+  //////////////////////////////////////////////////
+  // Writable
+  //////////////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(capacity);
+    out.writeLong(used);
+    out.writeLong(remaining);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    capacity = in.readLong();
+    used = in.readLong();
+    remaining = in.readLong();
+  }
+}

+ 61 - 0
src/java/org/apache/hadoop/fs/FsUrlConnection.java

@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLConnection;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Representation of a URL connection to open InputStreams.
+ */
+class FsUrlConnection extends URLConnection {
+
+  private Configuration conf;
+
+  private InputStream is;
+
+  FsUrlConnection(Configuration conf, URL url) {
+    super(url);
+    this.conf = conf;
+  }
+
+  @Override
+  public void connect() throws IOException {
+    try {
+      FileSystem fs = FileSystem.get(url.toURI(), conf);
+      is = fs.open(new Path(url.getPath()));
+    } catch (URISyntaxException e) {
+      throw new IOException(e.toString());
+    }
+  }
+
+  /* @inheritDoc */
+  @Override
+  public InputStream getInputStream() throws IOException {
+    if (is == null) {
+      connect();
+    }
+    return is;
+  }
+
+}

+ 47 - 0
src/java/org/apache/hadoop/fs/FsUrlStreamHandler.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLStreamHandler;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * URLStream handler relying on FileSystem and on a given Configuration to
+ * handle URL protocols.
+ */
+class FsUrlStreamHandler extends URLStreamHandler {
+
+  private Configuration conf;
+
+  FsUrlStreamHandler(Configuration conf) {
+    this.conf = conf;
+  }
+
+  FsUrlStreamHandler() {
+    this.conf = new Configuration();
+  }
+
+  @Override
+  protected FsUrlConnection openConnection(URL url) throws IOException {
+    return new FsUrlConnection(conf, url);
+  }
+
+}

+ 78 - 0
src/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.net.URLStreamHandlerFactory;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Factory for URL stream handlers.
+ * 
+ * There is only one handler whose job is to create UrlConnections. A
+ * FsUrlConnection relies on FileSystem to choose the appropriate FS
+ * implementation.
+ * 
+ * Before returning our handler, we make sure that FileSystem knows an
+ * implementation for the requested scheme/protocol.
+ */
+public class FsUrlStreamHandlerFactory implements
+    URLStreamHandlerFactory {
+
+  // The configuration holds supported FS implementation class names.
+  private Configuration conf;
+
+  // This map stores whether a protocol is know or not by FileSystem
+  private Map<String, Boolean> protocols = new HashMap<String, Boolean>();
+
+  // The URL Stream handler
+  private java.net.URLStreamHandler handler;
+
+  public FsUrlStreamHandlerFactory() {
+    this.conf = new Configuration();
+    // force the resolution of the configuration files
+    // this is required if we want the factory to be able to handle
+    // file:// URLs
+    this.conf.getClass("fs.file.impl", null);
+    this.handler = new FsUrlStreamHandler(this.conf);
+  }
+
+  public FsUrlStreamHandlerFactory(Configuration conf) {
+    this.conf = new Configuration(conf);
+    // force the resolution of the configuration files
+    this.conf.getClass("fs.file.impl", null);
+    this.handler = new FsUrlStreamHandler(this.conf);
+  }
+
+  public java.net.URLStreamHandler createURLStreamHandler(String protocol) {
+    if (!protocols.containsKey(protocol)) {
+      boolean known =
+          (conf.getClass("fs." + protocol + ".impl", null) != null);
+      protocols.put(protocol, known);
+    }
+    if (protocols.get(protocol)) {
+      return handler;
+    } else {
+      // FileSystem does not know the protocol, let the VM handle this
+      return null;
+    }
+  }
+
+}

+ 166 - 0
src/java/org/apache/hadoop/fs/GlobExpander.java

@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+class GlobExpander {
+  
+  static class StringWithOffset {
+    String string;
+    int offset;
+    public StringWithOffset(String string, int offset) {
+      super();
+      this.string = string;
+      this.offset = offset;
+    }
+  }
+  
+  /**
+   * Expand globs in the given <code>filePattern</code> into a collection of 
+   * file patterns so that in the expanded set no file pattern has a
+   * slash character ("/") in a curly bracket pair.
+   * @param filePattern
+   * @return expanded file patterns
+   * @throws IOException 
+   */
+  public static List<String> expand(String filePattern) throws IOException {
+    List<String> fullyExpanded = new ArrayList<String>();
+    List<StringWithOffset> toExpand = new ArrayList<StringWithOffset>();
+    toExpand.add(new StringWithOffset(filePattern, 0));
+    while (!toExpand.isEmpty()) {
+      StringWithOffset path = toExpand.remove(0);
+      List<StringWithOffset> expanded = expandLeftmost(path);
+      if (expanded == null) {
+        fullyExpanded.add(path.string);
+      } else {
+        toExpand.addAll(0, expanded);
+      }
+    }
+    return fullyExpanded;
+  }
+  
+  /**
+   * Expand the leftmost outer curly bracket pair containing a
+   * slash character ("/") in <code>filePattern</code>.
+   * @param filePattern
+   * @return expanded file patterns
+   * @throws IOException 
+   */
+  private static List<StringWithOffset> expandLeftmost(StringWithOffset
+      filePatternWithOffset) throws IOException {
+    
+    String filePattern = filePatternWithOffset.string;
+    int leftmost = leftmostOuterCurlyContainingSlash(filePattern,
+        filePatternWithOffset.offset);
+    if (leftmost == -1) {
+      return null;
+    }
+    int curlyOpen = 0;
+    StringBuilder prefix = new StringBuilder(filePattern.substring(0, leftmost));
+    StringBuilder suffix = new StringBuilder();
+    List<String> alts = new ArrayList<String>();
+    StringBuilder alt = new StringBuilder();
+    StringBuilder cur = prefix;
+    for (int i = leftmost; i < filePattern.length(); i++) {
+      char c = filePattern.charAt(i);
+      if (cur == suffix) {
+        cur.append(c);
+      } else if (c == '\\') {
+        i++;
+        if (i >= filePattern.length()) {
+          throw new IOException("Illegal file pattern: "
+              + "An escaped character does not present for glob "
+              + filePattern + " at " + i);
+        }
+        c = filePattern.charAt(i);
+        cur.append(c);
+      } else if (c == '{') {
+        if (curlyOpen++ == 0) {
+          alt.setLength(0);
+          cur = alt;
+        } else {
+          cur.append(c);
+        }
+
+      } else if (c == '}' && curlyOpen > 0) {
+        if (--curlyOpen == 0) {
+          alts.add(alt.toString());
+          alt.setLength(0);
+          cur = suffix;
+        } else {
+          cur.append(c);
+        }
+      } else if (c == ',') {
+        if (curlyOpen == 1) {
+          alts.add(alt.toString());
+          alt.setLength(0);
+        } else {
+          cur.append(c);
+        }
+      } else {
+        cur.append(c);
+      }
+    }
+    List<StringWithOffset> exp = new ArrayList<StringWithOffset>();
+    for (String string : alts) {
+      exp.add(new StringWithOffset(prefix + string + suffix, prefix.length()));
+    }
+    return exp;
+  }
+  
+  /**
+   * Finds the index of the leftmost opening curly bracket containing a
+   * slash character ("/") in <code>filePattern</code>.
+   * @param filePattern
+   * @return the index of the leftmost opening curly bracket containing a
+   * slash character ("/"), or -1 if there is no such bracket
+   * @throws IOException 
+   */
+  private static int leftmostOuterCurlyContainingSlash(String filePattern,
+      int offset) throws IOException {
+    int curlyOpen = 0;
+    int leftmost = -1;
+    boolean seenSlash = false;
+    for (int i = offset; i < filePattern.length(); i++) {
+      char c = filePattern.charAt(i);
+      if (c == '\\') {
+        i++;
+        if (i >= filePattern.length()) {
+          throw new IOException("Illegal file pattern: "
+              + "An escaped character does not present for glob "
+              + filePattern + " at " + i);
+        }
+      } else if (c == '{') {
+        if (curlyOpen++ == 0) {
+          leftmost = i;
+        }
+      } else if (c == '}' && curlyOpen > 0) {
+        if (--curlyOpen == 0 && leftmost != -1 && seenSlash) {
+          return leftmost;
+        }
+      } else if (c == '/' && curlyOpen > 0) {
+        seenSlash = true;
+      }
+    }
+    return -1;
+  }
+
+}

+ 892 - 0
src/java/org/apache/hadoop/fs/HarFileSystem.java

@@ -0,0 +1,892 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * This is an implementation of the Hadoop Archive 
+ * Filesystem. This archive Filesystem has index files
+ * of the form _index* and has contents of the form
+ * part-*. The index files store the indexes of the 
+ * real files. The index files are of the form _masterindex
+ * and _index. The master index is a level of indirection 
+ * in to the index file to make the look ups faster. the index
+ * file is sorted with hash code of the paths that it contains 
+ * and the master index contains pointers to the positions in 
+ * index for ranges of hashcodes.
+ */
+
+public class HarFileSystem extends FilterFileSystem {
+  public static final int VERSION = 1;
+  // uri representation of this Har filesystem
+  private URI uri;
+  // the version of this har filesystem
+  private int version;
+  // underlying uri 
+  private URI underLyingURI;
+  // the top level path of the archive
+  // in the underlying file system
+  private Path archivePath;
+  // the masterIndex of the archive
+  private Path masterIndex;
+  // the index file 
+  private Path archiveIndex;
+  // the har auth
+  private String harAuth;
+  
+  /**
+   * public construction of harfilesystem
+   *
+   */
+  public HarFileSystem() {
+  }
+  
+  /**
+   * Constructor to create a HarFileSystem with an
+   * underlying filesystem.
+   * @param fs
+   */
+  public HarFileSystem(FileSystem fs) {
+    super(fs);
+  }
+  
+  /**
+   * Initialize a Har filesystem per har archive. The 
+   * archive home directory is the top level directory
+   * in the filesystem that contains the HAR archive.
+   * Be careful with this method, you do not want to go 
+   * on creating new Filesystem instances per call to 
+   * path.getFileSystem().
+   * the uri of Har is 
+   * har://underlyingfsscheme-host:port/archivepath.
+   * or 
+   * har:///archivepath. This assumes the underlying filesystem
+   * to be used in case not specified.
+   */
+  public void initialize(URI name, Configuration conf) throws IOException {
+    //decode the name
+    underLyingURI = decodeHarURI(name, conf);
+    //  we got the right har Path- now check if this is 
+    //truly a har filesystem
+    Path harPath = archivePath(new Path(name.toString()));
+    if (harPath == null) { 
+      throw new IOException("Invalid path for the Har Filesystem. " + 
+                           name.toString());
+    }
+    if (fs == null) {
+      fs = FileSystem.get(underLyingURI, conf);
+    }
+    this.uri = harPath.toUri();
+    this.archivePath = new Path(this.uri.getPath());
+    this.harAuth = getHarAuth(this.underLyingURI);
+    //check for the underlying fs containing
+    // the index file
+    this.masterIndex = new Path(archivePath, "_masterindex");
+    this.archiveIndex = new Path(archivePath, "_index");
+    if (!fs.exists(masterIndex) || !fs.exists(archiveIndex)) {
+      throw new IOException("Invalid path for the Har Filesystem. " +
+          "No index file in " + harPath);
+    }
+    try{ 
+      this.version = getHarVersion();
+    } catch(IOException io) {
+      throw new IOException("Unable to " +
+          "read the version of the Har file system: " + this.archivePath);
+    }
+    if (this.version != HarFileSystem.VERSION) {
+      throw new IOException("Invalid version " + 
+          this.version + " expected " + HarFileSystem.VERSION);
+    }
+  }
+  
+  // get the version of the filesystem from the masterindex file
+  // the version is currently not useful since its the first version 
+  // of archives
+  public int getHarVersion() throws IOException { 
+    FSDataInputStream masterIn = fs.open(masterIndex);
+    LineReader lmaster = new LineReader(masterIn, getConf());
+    Text line = new Text();
+    lmaster.readLine(line);
+    try {
+      masterIn.close();
+    } catch(IOException e){
+      //disregard it.
+      // its a read.
+    }
+    String versionLine = line.toString();
+    String[] arr = versionLine.split(" ");
+    int version = Integer.parseInt(arr[0]);
+    return version;
+  }
+  
+  /*
+   * find the parent path that is the 
+   * archive path in the path. The last
+   * path segment that ends with .har is 
+   * the path that will be returned.
+   */
+  private Path archivePath(Path p) {
+    Path retPath = null;
+    Path tmp = p;
+    for (int i=0; i< p.depth(); i++) {
+      if (tmp.toString().endsWith(".har")) {
+        retPath = tmp;
+        break;
+      }
+      tmp = tmp.getParent();
+    }
+    return retPath;
+  }
+
+  /**
+   * decode the raw URI to get the underlying URI
+   * @param rawURI raw Har URI
+   * @return filtered URI of the underlying fileSystem
+   */
+  private URI decodeHarURI(URI rawURI, Configuration conf) throws IOException {
+    String tmpAuth = rawURI.getAuthority();
+    //we are using the default file
+    //system in the config 
+    //so create a underlying uri and 
+    //return it
+    if (tmpAuth == null) {
+      //create a path 
+      return FileSystem.getDefaultUri(conf);
+    }
+    String host = rawURI.getHost();
+    String[] str = host.split("-", 2);
+    if (str[0] == null) {
+      throw new IOException("URI: " + rawURI + " is an invalid Har URI.");
+    }
+    String underLyingScheme = str[0];
+    String underLyingHost = (str.length > 1)? str[1]:null;
+    int underLyingPort = rawURI.getPort();
+    String auth = (underLyingHost == null && underLyingPort == -1)?
+                  null:(underLyingHost+":"+underLyingPort);
+    URI tmp = null;
+    if (rawURI.getQuery() != null) {
+      // query component not allowed
+      throw new IOException("query component in Path not supported  " + rawURI);
+    }
+    try {
+      tmp = new URI(underLyingScheme, auth, rawURI.getPath(), 
+            rawURI.getQuery(), rawURI.getFragment());
+    } catch (URISyntaxException e) {
+        // do nothing should not happen
+    }
+    return tmp;
+  }
+  
+  /**
+   * return the top level archive.
+   */
+  public Path getWorkingDirectory() {
+    return new Path(uri.toString());
+  }
+  
+  /**
+   * Create a har specific auth 
+   * har-underlyingfs:port
+   * @param underLyingURI the uri of underlying
+   * filesystem
+   * @return har specific auth
+   */
+  private String getHarAuth(URI underLyingUri) {
+    String auth = underLyingUri.getScheme() + "-";
+    if (underLyingUri.getHost() != null) {
+      auth += underLyingUri.getHost() + ":";
+      if (underLyingUri.getPort() != -1) {
+        auth +=  underLyingUri.getPort();
+      }
+    }
+    else {
+      auth += ":";
+    }
+    return auth;
+  }
+  
+  /**
+   * Returns the uri of this filesystem.
+   * The uri is of the form 
+   * har://underlyingfsschema-host:port/pathintheunderlyingfs
+   */
+  @Override
+  public URI getUri() {
+    return this.uri;
+  }
+  
+  /**
+   * this method returns the path 
+   * inside the har filesystem.
+   * this is relative path inside 
+   * the har filesystem.
+   * @param path the fully qualified path in the har filesystem.
+   * @return relative path in the filesystem.
+   */
+  private Path getPathInHar(Path path) {
+    Path harPath = new Path(path.toUri().getPath());
+    if (archivePath.compareTo(harPath) == 0)
+      return new Path(Path.SEPARATOR);
+    Path tmp = new Path(harPath.getName());
+    Path parent = harPath.getParent();
+    while (!(parent.compareTo(archivePath) == 0)) {
+      if (parent.toString().equals(Path.SEPARATOR)) {
+        tmp = null;
+        break;
+      }
+      tmp = new Path(parent.getName(), tmp);
+      parent = parent.getParent();
+    }
+    if (tmp != null) 
+      tmp = new Path(Path.SEPARATOR, tmp);
+    return tmp;
+  }
+  
+  //the relative path of p. basically 
+  // getting rid of /. Parsing and doing 
+  // string manipulation is not good - so
+  // just use the path api to do it.
+  private Path makeRelative(String initial, Path p) {
+    Path root = new Path(Path.SEPARATOR);
+    if (root.compareTo(p) == 0)
+      return new Path(initial);
+    Path retPath = new Path(p.getName());
+    Path parent = p.getParent();
+    for (int i=0; i < p.depth()-1; i++) {
+      retPath = new Path(parent.getName(), retPath);
+      parent = parent.getParent();
+    }
+    return new Path(initial, retPath.toString());
+  }
+  
+  /* this makes a path qualified in the har filesystem
+   * (non-Javadoc)
+   * @see org.apache.hadoop.fs.FilterFileSystem#makeQualified(
+   * org.apache.hadoop.fs.Path)
+   */
+  @Override
+  public Path makeQualified(Path path) {
+    // make sure that we just get the 
+    // path component 
+    Path fsPath = path;
+    if (!path.isAbsolute()) {
+      fsPath = new Path(archivePath, path);
+    }
+
+    URI tmpURI = fsPath.toUri();
+    fsPath = new Path(tmpURI.getPath());
+    //change this to Har uri 
+    URI tmp = null;
+    try {
+      tmp = new URI(uri.getScheme(), harAuth, fsPath.toString(),
+                    tmpURI.getQuery(), tmpURI.getFragment());
+    } catch(URISyntaxException ue) {
+      LOG.error("Error in URI ", ue);
+    }
+    if (tmp != null) {
+      return new Path(tmp.toString());
+    }
+    return null;
+  }
+  
+  /**
+   * get block locations from the underlying fs
+   * @param file the input filestatus to get block locations
+   * @param start the start in the file
+   * @param len the length in the file
+   * @return block locations for this segment of file
+   * @throws IOException
+   */
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+      long len) throws IOException {
+    // need to look up the file in the underlying fs
+    // look up the index 
+    
+    // make sure this is a prt of this har filesystem
+    Path p = makeQualified(file.getPath());
+    Path harPath = getPathInHar(p);
+    String line = fileStatusInIndex(harPath);
+    if (line == null)  {
+      throw new FileNotFoundException("File " + file.getPath() + " not found");
+    }
+    HarStatus harStatus = new HarStatus(line);
+    if (harStatus.isDir()) {
+      return new BlockLocation[0];
+    }
+    FileStatus fsFile = fs.getFileStatus(new Path(archivePath,
+        harStatus.getPartName()));
+    BlockLocation[] rawBlocks = fs.getFileBlockLocations(fsFile, 
+        harStatus.getStartIndex() + start, len);
+    return fakeBlockLocations(rawBlocks, harStatus.getStartIndex());
+  }
+  
+  /**
+   * fake the rawblocks since map reduce uses the block offsets to 
+   * fo some computations regarding the blocks
+   * @param rawBlocks the raw blocks returned by the filesystem
+   * @return faked blocks with changed offsets.
+   */
+  private BlockLocation[] fakeBlockLocations(BlockLocation[] rawBlocks, 
+		  long startIndex) {
+	for (BlockLocation block : rawBlocks) {
+		long rawOffset = block.getOffset();
+		block.setOffset(rawOffset - startIndex);
+	}
+	return rawBlocks;
+  }
+  
+  /**
+   * the hash of the path p inside iniside
+   * the filesystem
+   * @param p the path in the harfilesystem
+   * @return the hash code of the path.
+   */
+  public static int getHarHash(Path p) {
+    return (p.toString().hashCode() & 0x7fffffff);
+  }
+  
+  static class Store {
+    public Store() {
+      begin = end = startHash = endHash = 0;
+    }
+    public Store(long begin, long end, int startHash, int endHash) {
+      this.begin = begin;
+      this.end = end;
+      this.startHash = startHash;
+      this.endHash = endHash;
+    }
+    public long begin;
+    public long end;
+    public int startHash;
+    public int endHash;
+  }
+  
+  // make sure that this harPath is relative to the har filesystem
+  // this only works for relative paths. This returns the line matching
+  // the file in the index. Returns a null if there is not matching 
+  // filename in the index file.
+  private String fileStatusInIndex(Path harPath) throws IOException {
+    // read the index file 
+    int hashCode = getHarHash(harPath);
+    // get the master index to find the pos 
+    // in the index file
+    FSDataInputStream in = fs.open(masterIndex);
+    FileStatus masterStat = fs.getFileStatus(masterIndex);
+    LineReader lin = new LineReader(in, getConf());
+    Text line = new Text();
+    long read = lin.readLine(line);
+   //ignore the first line. this is the header of the index files
+    String[] readStr = null;
+    List<Store> stores = new ArrayList<Store>();
+    while(read < masterStat.getLen()) {
+      int b = lin.readLine(line);
+      read += b;
+      readStr = line.toString().split(" ");
+      int startHash = Integer.parseInt(readStr[0]);
+      int endHash  = Integer.parseInt(readStr[1]);
+      if (startHash <= hashCode && hashCode <= endHash) {
+        stores.add(new Store(Long.parseLong(readStr[2]), 
+            Long.parseLong(readStr[3]), startHash,
+            endHash));
+      }
+      line.clear();
+    }
+    try {
+      lin.close();
+    } catch(IOException io){
+      // do nothing just a read.
+    }
+    FSDataInputStream aIn = fs.open(archiveIndex);
+    LineReader aLin = new LineReader(aIn, getConf());
+    String retStr = null;
+    // now start reading the real index file
+     read = 0;
+    for (Store s: stores) {
+      aIn.seek(s.begin);
+      while (read + s.begin < s.end) {
+        int tmp = aLin.readLine(line);
+        read += tmp;
+        String lineFeed = line.toString();
+        String[] parsed = lineFeed.split(" ");
+        if (harPath.compareTo(new Path(parsed[0])) == 0) {
+          // bingo!
+          retStr = lineFeed;
+          break;
+        }
+        line.clear();
+      }
+      if (retStr != null)
+        break;
+    }
+    try {
+      aIn.close();
+    } catch(IOException io) {
+      //do nothing
+    }
+    return retStr;
+  }
+  
+  // a single line parser for hadoop archives status 
+  // stored in a single line in the index files 
+  // the format is of the form 
+  // filename "dir"/"file" partFileName startIndex length 
+  // <space seperated children>
+  private static class HarStatus {
+    boolean isDir;
+    String name;
+    List<String> children;
+    String partName;
+    long startIndex;
+    long length;
+    public HarStatus(String harString) {
+      String[] splits = harString.split(" ");
+      this.name = splits[0];
+      this.isDir = "dir".equals(splits[1]) ? true: false;
+      // this is equal to "none" if its a directory
+      this.partName = splits[2];
+      this.startIndex = Long.parseLong(splits[3]);
+      this.length = Long.parseLong(splits[4]);
+      if (isDir) {
+        children = new ArrayList<String>();
+        for (int i = 5; i < splits.length; i++) {
+          children.add(splits[i]);
+        }
+      }
+    }
+    public boolean isDir() {
+      return isDir;
+    }
+    
+    public String getName() {
+      return name;
+    }
+    
+    public List<String> getChildren() {
+      return children;
+    }
+    public String getFileName() {
+      return name;
+    }
+    public String getPartName() {
+      return partName;
+    }
+    public long getStartIndex() {
+      return startIndex;
+    }
+    public long getLength() {
+      return length;
+    }
+  }
+  
+  /**
+   * return the filestatus of files in har archive.
+   * The permission returned are that of the archive
+   * index files. The permissions are not persisted 
+   * while creating a hadoop archive.
+   * @param f the path in har filesystem
+   * @return filestatus.
+   * @throws IOException
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    FileStatus archiveStatus = fs.getFileStatus(archiveIndex);
+    // get the fs DataInputStream for the underlying file
+    // look up the index.
+    Path p = makeQualified(f);
+    Path harPath = getPathInHar(p);
+    if (harPath == null) {
+      throw new IOException("Invalid file name: " + f + " in " + uri);
+    }
+    String readStr = fileStatusInIndex(harPath);
+    if (readStr == null) {
+      throw new FileNotFoundException("File: " +  f + " does not exist in " + uri);
+    }
+    HarStatus hstatus = null;
+    hstatus = new HarStatus(readStr);
+    return new FileStatus(hstatus.isDir()?0:hstatus.getLength(), hstatus.isDir(),
+        (int)archiveStatus.getReplication(), archiveStatus.getBlockSize(),
+        archiveStatus.getModificationTime(), archiveStatus.getAccessTime(),
+        new FsPermission(
+        archiveStatus.getPermission()), archiveStatus.getOwner(), 
+        archiveStatus.getGroup(), 
+            makeRelative(this.uri.toString(), new Path(hstatus.name)));
+  }
+
+  /**
+   * Returns a har input stream which fakes end of 
+   * file. It reads the index files to get the part 
+   * file name and the size and start of the file.
+   */
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    // get the fs DataInputStream for the underlying file
+    // look up the index.
+    Path p = makeQualified(f);
+    Path harPath = getPathInHar(p);
+    if (harPath == null) {
+      throw new IOException("Invalid file name: " + f + " in " + uri);
+    }
+    String readStr = fileStatusInIndex(harPath);
+    if (readStr == null) {
+      throw new FileNotFoundException(f + ": not found in " + archivePath);
+    }
+    HarStatus hstatus = new HarStatus(readStr); 
+    // we got it.. woo hooo!!! 
+    if (hstatus.isDir()) {
+      throw new FileNotFoundException(f + " : not a file in " +
+                archivePath);
+    }
+    return new HarFSDataInputStream(fs, new Path(archivePath, 
+        hstatus.getPartName()),
+        hstatus.getStartIndex(), hstatus.getLength(), bufferSize);
+  }
+ 
+  /*
+   * create throws an exception in Har filesystem.
+   * The archive once created cannot be changed.
+   */
+  public FSDataOutputStream create(Path f, int bufferSize) 
+                                    throws IOException {
+    throw new IOException("Har: Create not allowed");
+  }
+  
+  public FSDataOutputStream create(Path f,
+      FsPermission permission,
+      boolean overwrite,
+      int bufferSize,
+      short replication,
+      long blockSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Har: create not allowed.");
+  }
+  
+  @Override
+  public void close() throws IOException {
+    if (fs != null) {
+      try {
+        fs.close();
+      } catch(IOException ie) {
+        //this might already be closed
+        // ignore
+      }
+    }
+  }
+  
+  /**
+   * Not implemented.
+   */
+  @Override
+  public boolean setReplication(Path src, short replication) throws IOException{
+    throw new IOException("Har: setreplication not allowed");
+  }
+  
+  /**
+   * Not implemented.
+   */
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException { 
+    throw new IOException("Har: delete not allowed");
+  }
+  
+  /**
+   * liststatus returns the children of a directory 
+   * after looking up the index files.
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    //need to see if the file is an index in file
+    //get the filestatus of the archive directory
+    // we will create fake filestatuses to return
+    // to the client
+    List<FileStatus> statuses = new ArrayList<FileStatus>();
+    FileStatus archiveStatus = fs.getFileStatus(archiveIndex);
+    Path tmpPath = makeQualified(f);
+    Path harPath = getPathInHar(tmpPath);
+    String readStr = fileStatusInIndex(harPath);
+    if (readStr == null) {
+      throw new FileNotFoundException("File " + f + " not found in " + archivePath);
+    }
+    HarStatus hstatus = new HarStatus(readStr);
+    if (!hstatus.isDir()) 
+        statuses.add(new FileStatus(hstatus.getLength(), 
+            hstatus.isDir(),
+            archiveStatus.getReplication(), archiveStatus.getBlockSize(),
+            archiveStatus.getModificationTime(), archiveStatus.getAccessTime(),
+            new FsPermission(archiveStatus.getPermission()),
+            archiveStatus.getOwner(), archiveStatus.getGroup(), 
+            makeRelative(this.uri.toString(), new Path(hstatus.name))));
+    else 
+      for (String child: hstatus.children) {
+        FileStatus tmp = getFileStatus(new Path(tmpPath, child));
+        statuses.add(tmp);
+      }
+    return statuses.toArray(new FileStatus[statuses.size()]);
+  }
+  
+  /**
+   * return the top level archive path.
+   */
+  public Path getHomeDirectory() {
+    return new Path(uri.toString());
+  }
+  
+  public void setWorkingDirectory(Path newDir) {
+    //does nothing.
+  }
+  
+  /**
+   * not implemented.
+   */
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    throw new IOException("Har: mkdirs not allowed");
+  }
+  
+  /**
+   * not implemented.
+   */
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws 
+        IOException {
+    throw new IOException("Har: copyfromlocalfile not allowed");
+  }
+  
+  /**
+   * copies the file in the har filesystem to a local file.
+   */
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst) 
+    throws IOException {
+    FileUtil.copy(this, src, getLocal(getConf()), dst, false, getConf());
+  }
+  
+  /**
+   * not implemented.
+   */
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) 
+    throws IOException {
+    throw new IOException("Har: startLocalOutput not allowed");
+  }
+  
+  /**
+   * not implemented.
+   */
+  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) 
+    throws IOException {
+    throw new IOException("Har: completeLocalOutput not allowed");
+  }
+  
+  /**
+   * not implemented.
+   */
+  public void setOwner(Path p, String username, String groupname)
+    throws IOException {
+    throw new IOException("Har: setowner not allowed");
+  }
+
+  /**
+   * Not implemented.
+   */
+  public void setPermission(Path p, FsPermission permisssion) 
+    throws IOException {
+    throw new IOException("Har: setPermission not allowed");
+  }
+  
+  /**
+   * Hadoop archives input stream. This input stream fakes EOF 
+   * since archive files are part of bigger part files.
+   */
+  private static class HarFSDataInputStream extends FSDataInputStream {
+    /**
+     * Create an input stream that fakes all the reads/positions/seeking.
+     */
+    private static class HarFsInputStream extends FSInputStream {
+      private long position, start, end;
+      //The underlying data input stream that the
+      // underlying filesystem will return.
+      private FSDataInputStream underLyingStream;
+      //one byte buffer
+      private byte[] oneBytebuff = new byte[1];
+      HarFsInputStream(FileSystem fs, Path path, long start,
+          long length, int bufferSize) throws IOException {
+        underLyingStream = fs.open(path, bufferSize);
+        underLyingStream.seek(start);
+        // the start of this file in the part file
+        this.start = start;
+        // the position pointer in the part file
+        this.position = start;
+        // the end pointer in the part file
+        this.end = start + length;
+      }
+      
+      public synchronized int available() throws IOException {
+        long remaining = end - underLyingStream.getPos();
+        if (remaining > (long)Integer.MAX_VALUE) {
+          return Integer.MAX_VALUE;
+        }
+        return (int) remaining;
+      }
+      
+      public synchronized  void close() throws IOException {
+        underLyingStream.close();
+        super.close();
+      }
+      
+      //not implemented
+      @Override
+      public void mark(int readLimit) {
+        // do nothing 
+      }
+      
+      /**
+       * reset is not implemented
+       */
+      public void reset() throws IOException {
+        throw new IOException("reset not implemented.");
+      }
+      
+      public synchronized int read() throws IOException {
+        int ret = read(oneBytebuff, 0, 1);
+        return (ret <= 0) ? -1: (oneBytebuff[0] & 0xff);
+      }
+      
+      public synchronized int read(byte[] b) throws IOException {
+        int ret = read(b, 0, b.length);
+        if (ret != -1) {
+          position += ret;
+        }
+        return ret;
+      }
+      
+      /**
+       * 
+       */
+      public synchronized int read(byte[] b, int offset, int len) 
+        throws IOException {
+        int newlen = len;
+        int ret = -1;
+        if (position + len > end) {
+          newlen = (int) (end - position);
+        }
+        // end case
+        if (newlen == 0) 
+          return ret;
+        ret = underLyingStream.read(b, offset, newlen);
+        position += ret;
+        return ret;
+      }
+      
+      public synchronized long skip(long n) throws IOException {
+        long tmpN = n;
+        if (tmpN > 0) {
+          if (position + tmpN > end) {
+            tmpN = end - position;
+          }
+          underLyingStream.seek(tmpN + position);
+          position += tmpN;
+          return tmpN;
+        }
+        return (tmpN < 0)? -1 : 0;
+      }
+      
+      public synchronized long getPos() throws IOException {
+        return (position - start);
+      }
+      
+      public synchronized void seek(long pos) throws IOException {
+        if (pos < 0 || (start + pos > end)) {
+          throw new IOException("Failed to seek: EOF");
+        }
+        position = start + pos;
+        underLyingStream.seek(position);
+      }
+
+      public boolean seekToNewSource(long targetPos) throws IOException {
+        //do not need to implement this
+        // hdfs in itself does seektonewsource 
+        // while reading.
+        return false;
+      }
+      
+      /**
+       * implementing position readable. 
+       */
+      public int read(long pos, byte[] b, int offset, int length) 
+      throws IOException {
+        int nlength = length;
+        if (start + nlength + pos > end) {
+          nlength = (int) (end - (start + pos));
+        }
+        return underLyingStream.read(pos + start , b, offset, nlength);
+      }
+      
+      /**
+       * position readable again.
+       */
+      public void readFully(long pos, byte[] b, int offset, int length) 
+      throws IOException {
+        if (start + length + pos > end) {
+          throw new IOException("Not enough bytes to read.");
+        }
+        underLyingStream.readFully(pos + start, b, offset, length);
+      }
+      
+      public void readFully(long pos, byte[] b) throws IOException {
+          readFully(pos, b, 0, b.length);
+      }
+      
+    }
+  
+    /**
+     * constructors for har input stream.
+     * @param fs the underlying filesystem
+     * @param p The path in the underlying filesystem
+     * @param start the start position in the part file
+     * @param length the length of valid data in the part file
+     * @param bufsize the buffer size
+     * @throws IOException
+     */
+    public HarFSDataInputStream(FileSystem fs, Path  p, long start, 
+        long length, int bufsize) throws IOException {
+        super(new HarFsInputStream(fs, p, start, length, bufsize));
+    }
+
+    /**
+     * constructor for har input stream.
+     * @param fs the underlying filesystem
+     * @param p the path in the underlying file system
+     * @param start the start position in the part file
+     * @param length the length of valid data in the part file.
+     * @throws IOException
+     */
+    public HarFSDataInputStream(FileSystem fs, Path  p, long start, long length)
+      throws IOException {
+        super(new HarFsInputStream(fs, p, start, length, 0));
+    }
+  }
+}

+ 0 - 0
src/java/org/apache/hadoop/fs/LengthFileChecksum.java


+ 418 - 0
src/java/org/apache/hadoop/fs/LocalDirAllocator.java

@@ -0,0 +1,418 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.*;
+import java.util.*;
+
+import org.apache.commons.logging.*;
+
+import org.apache.hadoop.util.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.conf.Configuration; 
+
+/** An implementation of a round-robin scheme for disk allocation for creating
+ * files. The way it works is that it is kept track what disk was last
+ * allocated for a file write. For the current request, the next disk from
+ * the set of disks would be allocated if the free space on the disk is 
+ * sufficient enough to accomodate the file that is being considered for
+ * creation. If the space requirements cannot be met, the next disk in order
+ * would be tried and so on till a disk is found with sufficient capacity.
+ * Once a disk with sufficient space is identified, a check is done to make
+ * sure that the disk is writable. Also, there is an API provided that doesn't
+ * take the space requirements into consideration but just checks whether the
+ * disk under consideration is writable (this should be used for cases where
+ * the file size is not known apriori). An API is provided to read a path that
+ * was created earlier. That API works by doing a scan of all the disks for the
+ * input pathname.
+ * This implementation also provides the functionality of having multiple 
+ * allocators per JVM (one for each unique functionality or context, like 
+ * mapred, dfs-client, etc.). It ensures that there is only one instance of
+ * an allocator per context per JVM.
+ * Note:
+ * 1. The contexts referred above are actually the configuration items defined
+ * in the Configuration class like "mapred.local.dir" (for which we want to 
+ * control the dir allocations). The context-strings are exactly those 
+ * configuration items.
+ * 2. This implementation does not take into consideration cases where
+ * a disk becomes read-only or goes out of space while a file is being written
+ * to (disks are shared between multiple processes, and so the latter situation
+ * is probable).
+ * 3. In the class implementation, "Disk" is referred to as "Dir", which
+ * actually points to the configured directory on the Disk which will be the
+ * parent for all file write/read allocations.
+ */
+public class LocalDirAllocator {
+  
+  //A Map from the config item names like "mapred.local.dir", 
+  //"dfs.client.buffer.dir" to the instance of the AllocatorPerContext. This
+  //is a static object to make sure there exists exactly one instance per JVM
+  private static Map <String, AllocatorPerContext> contexts = 
+                 new TreeMap<String, AllocatorPerContext>();
+  private String contextCfgItemName;
+
+  /**Create an allocator object
+   * @param contextCfgItemName
+   */
+  public LocalDirAllocator(String contextCfgItemName) {
+    this.contextCfgItemName = contextCfgItemName;
+  }
+  
+  /** This method must be used to obtain the dir allocation context for a 
+   * particular value of the context name. The context name must be an item
+   * defined in the Configuration object for which we want to control the 
+   * dir allocations (e.g., <code>mapred.local.dir</code>). The method will
+   * create a context for that name if it doesn't already exist.
+   */
+  private AllocatorPerContext obtainContext(String contextCfgItemName) {
+    synchronized (contexts) {
+      AllocatorPerContext l = contexts.get(contextCfgItemName);
+      if (l == null) {
+        contexts.put(contextCfgItemName, 
+                    (l = new AllocatorPerContext(contextCfgItemName)));
+      }
+      return l;
+    }
+  }
+  
+  /** Get a path from the local FS. This method should be used if the size of 
+   *  the file is not known apriori. We go round-robin over the set of disks
+   *  (via the configured dirs) and return the first complete path where
+   *  we could create the parent directory of the passed path. 
+   *  @param pathStr the requested path (this will be created on the first 
+   *  available disk)
+   *  @param conf the Configuration object
+   *  @return the complete path to the file on a local disk
+   *  @throws IOException
+   */
+  public Path getLocalPathForWrite(String pathStr, 
+      Configuration conf) throws IOException {
+    return getLocalPathForWrite(pathStr, -1, conf);
+  }
+  
+  /** Get a path from the local FS. Pass size as -1 if not known apriori. We
+   *  round-robin over the set of disks (via the configured dirs) and return
+   *  the first complete path which has enough space 
+   *  @param pathStr the requested path (this will be created on the first 
+   *  available disk)
+   *  @param size the size of the file that is going to be written
+   *  @param conf the Configuration object
+   *  @return the complete path to the file on a local disk
+   *  @throws IOException
+   */
+  public Path getLocalPathForWrite(String pathStr, long size, 
+      Configuration conf) throws IOException {
+    AllocatorPerContext context = obtainContext(contextCfgItemName);
+    return context.getLocalPathForWrite(pathStr, size, conf);
+  }
+  
+  /** Get a path from the local FS for reading. We search through all the
+   *  configured dirs for the file's existence and return the complete
+   *  path to the file when we find one 
+   *  @param pathStr the requested file (this will be searched)
+   *  @param conf the Configuration object
+   *  @return the complete path to the file on a local disk
+   *  @throws IOException
+   */
+  public Path getLocalPathToRead(String pathStr, 
+      Configuration conf) throws IOException {
+    AllocatorPerContext context = obtainContext(contextCfgItemName);
+    return context.getLocalPathToRead(pathStr, conf);
+  }
+
+  /** Creates a temporary file in the local FS. Pass size as -1 if not known 
+   *  apriori. We round-robin over the set of disks (via the configured dirs) 
+   *  and select the first complete path which has enough space. A file is
+   *  created on this directory. The file is guaranteed to go away when the
+   *  JVM exits.
+   *  @param pathStr prefix for the temporary file
+   *  @param size the size of the file that is going to be written
+   *  @param conf the Configuration object
+   *  @return a unique temporary file
+   *  @throws IOException
+   */
+  public File createTmpFileForWrite(String pathStr, long size, 
+      Configuration conf) throws IOException {
+    AllocatorPerContext context = obtainContext(contextCfgItemName);
+    return context.createTmpFileForWrite(pathStr, size, conf);
+  }
+  
+  /** Method to check whether a context is valid
+   * @param contextCfgItemName
+   * @return true/false
+   */
+  public static boolean isContextValid(String contextCfgItemName) {
+    synchronized (contexts) {
+      return contexts.containsKey(contextCfgItemName);
+    }
+  }
+    
+  /** We search through all the configured dirs for the file's existence
+   *  and return true when we find  
+   *  @param pathStr the requested file (this will be searched)
+   *  @param conf the Configuration object
+   *  @return true if files exist. false otherwise
+   *  @throws IOException
+   */
+  public boolean ifExists(String pathStr,Configuration conf) {
+    AllocatorPerContext context = obtainContext(contextCfgItemName);
+    return context.ifExists(pathStr, conf);
+  }
+
+  /**
+   * Get the current directory index for the given configuration item.
+   * @return the current directory index for the given configuration item.
+   */
+  int getCurrentDirectoryIndex() {
+    AllocatorPerContext context = obtainContext(contextCfgItemName);
+    return context.getCurrentDirectoryIndex();
+  }
+  
+  private static class AllocatorPerContext {
+
+    private final Log LOG =
+      LogFactory.getLog(AllocatorPerContext.class);
+
+    private int dirNumLastAccessed;
+    private Random dirIndexRandomizer = new Random();
+    private FileSystem localFS;
+    private DF[] dirDF;
+    private String contextCfgItemName;
+    private String[] localDirs;
+    private String savedLocalDirs = "";
+
+    public AllocatorPerContext(String contextCfgItemName) {
+      this.contextCfgItemName = contextCfgItemName;
+    }
+
+    /** This method gets called everytime before any read/write to make sure
+     * that any change to localDirs is reflected immediately.
+     */
+    private void confChanged(Configuration conf) throws IOException {
+      String newLocalDirs = conf.get(contextCfgItemName);
+      if (!newLocalDirs.equals(savedLocalDirs)) {
+        localDirs = conf.getStrings(contextCfgItemName);
+        localFS = FileSystem.getLocal(conf);
+        int numDirs = localDirs.length;
+        ArrayList<String> dirs = new ArrayList<String>(numDirs);
+        ArrayList<DF> dfList = new ArrayList<DF>(numDirs);
+        for (int i = 0; i < numDirs; i++) {
+          try {
+            // filter problematic directories
+            Path tmpDir = new Path(localDirs[i]);
+            if(localFS.mkdirs(tmpDir)|| localFS.exists(tmpDir)) {
+              try {
+                DiskChecker.checkDir(new File(localDirs[i]));
+                dirs.add(localDirs[i]);
+                dfList.add(new DF(new File(localDirs[i]), 30000));
+              } catch (DiskErrorException de) {
+                LOG.warn( localDirs[i] + "is not writable\n" +
+                    StringUtils.stringifyException(de));
+              }
+            } else {
+              LOG.warn( "Failed to create " + localDirs[i]);
+            }
+          } catch (IOException ie) { 
+            LOG.warn( "Failed to create " + localDirs[i] + ": " +
+                ie.getMessage() + "\n" + StringUtils.stringifyException(ie));
+          } //ignore
+        }
+        localDirs = dirs.toArray(new String[dirs.size()]);
+        dirDF = dfList.toArray(new DF[dirs.size()]);
+        savedLocalDirs = newLocalDirs;
+        
+        // randomize the first disk picked in the round-robin selection 
+        dirNumLastAccessed = dirIndexRandomizer.nextInt(dirs.size());
+      }
+    }
+
+    private Path createPath(String path) throws IOException {
+      Path file = new Path(new Path(localDirs[dirNumLastAccessed]),
+                                    path);
+      //check whether we are able to create a directory here. If the disk
+      //happens to be RDONLY we will fail
+      try {
+        DiskChecker.checkDir(new File(file.getParent().toUri().getPath()));
+        return file;
+      } catch (DiskErrorException d) {
+        LOG.warn(StringUtils.stringifyException(d));
+        return null;
+      }
+    }
+
+    /**
+     * Get the current directory index.
+     * @return the current directory index.
+     */
+    int getCurrentDirectoryIndex() {
+      return dirNumLastAccessed;
+    }
+    
+    /** Get a path from the local FS. This method should be used if the size of 
+     *  the file is not known a priori. 
+     *  
+     *  It will use roulette selection, picking directories
+     *  with probability proportional to their available space. 
+     */
+    public synchronized Path getLocalPathForWrite(String path, 
+        Configuration conf) throws IOException {
+      return getLocalPathForWrite(path, -1, conf);
+    }
+
+    /** Get a path from the local FS. If size is known, we go
+     *  round-robin over the set of disks (via the configured dirs) and return
+     *  the first complete path which has enough space.
+     *  
+     *  If size is not known, use roulette selection -- pick directories
+     *  with probability proportional to their available space.
+     */
+    public synchronized Path getLocalPathForWrite(String pathStr, long size, 
+        Configuration conf) throws IOException {
+      confChanged(conf);
+      int numDirs = localDirs.length;
+      int numDirsSearched = 0;
+      //remove the leading slash from the path (to make sure that the uri
+      //resolution results in a valid path on the dir being checked)
+      if (pathStr.startsWith("/")) {
+        pathStr = pathStr.substring(1);
+      }
+      Path returnPath = null;
+      
+      if(size == -1) {  //do roulette selection: pick dir with probability 
+                    //proportional to available size
+        long[] availableOnDisk = new long[dirDF.length];
+        long totalAvailable = 0;
+        
+            //build the "roulette wheel"
+        for(int i =0; i < dirDF.length; ++i) {
+          availableOnDisk[i] = dirDF[i].getAvailable();
+          totalAvailable += availableOnDisk[i];
+        }
+
+        // Keep rolling the wheel till we get a valid path
+        Random r = new java.util.Random();
+        while (numDirsSearched < numDirs && returnPath == null) {
+          long randomPosition = Math.abs(r.nextLong()) % totalAvailable;
+          int dir = 0;
+          while (randomPosition > availableOnDisk[dir]) {
+            randomPosition -= availableOnDisk[dir];
+            dir++;
+          }
+          dirNumLastAccessed = dir;
+          returnPath = createPath(pathStr);
+          if (returnPath == null) {
+            totalAvailable -= availableOnDisk[dir];
+            availableOnDisk[dir] = 0; // skip this disk
+            numDirsSearched++;
+          }
+        }
+      } else {
+        while (numDirsSearched < numDirs && returnPath == null) {
+          long capacity = dirDF[dirNumLastAccessed].getAvailable();
+          if (capacity > size) {
+            returnPath = createPath(pathStr);
+          }
+          dirNumLastAccessed++;
+          dirNumLastAccessed = dirNumLastAccessed % numDirs; 
+          numDirsSearched++;
+        } 
+      }
+      if (returnPath != null) {
+        return returnPath;
+      }
+      
+      //no path found
+      throw new DiskErrorException("Could not find any valid local " +
+          "directory for " + pathStr);
+    }
+
+    /** Creates a file on the local FS. Pass size as -1 if not known apriori. We
+     *  round-robin over the set of disks (via the configured dirs) and return
+     *  a file on the first path which has enough space. The file is guaranteed
+     *  to go away when the JVM exits.
+     */
+    public File createTmpFileForWrite(String pathStr, long size, 
+        Configuration conf) throws IOException {
+
+      // find an appropriate directory
+      Path path = getLocalPathForWrite(pathStr, size, conf);
+      File dir = new File(path.getParent().toUri().getPath());
+      String prefix = path.getName();
+
+      // create a temp file on this directory
+      File result = File.createTempFile(prefix, null, dir);
+      result.deleteOnExit();
+      return result;
+    }
+
+    /** Get a path from the local FS for reading. We search through all the
+     *  configured dirs for the file's existence and return the complete
+     *  path to the file when we find one 
+     */
+    public synchronized Path getLocalPathToRead(String pathStr, 
+        Configuration conf) throws IOException {
+      confChanged(conf);
+      int numDirs = localDirs.length;
+      int numDirsSearched = 0;
+      //remove the leading slash from the path (to make sure that the uri
+      //resolution results in a valid path on the dir being checked)
+      if (pathStr.startsWith("/")) {
+        pathStr = pathStr.substring(1);
+      }
+      while (numDirsSearched < numDirs) {
+        Path file = new Path(localDirs[numDirsSearched], pathStr);
+        if (localFS.exists(file)) {
+          return file;
+        }
+        numDirsSearched++;
+      }
+
+      //no path found
+      throw new DiskErrorException ("Could not find " + pathStr +" in any of" +
+      " the configured local directories");
+    }
+
+    /** We search through all the configured dirs for the file's existence
+     *  and return true when we find one 
+     */
+    public synchronized boolean ifExists(String pathStr,Configuration conf) {
+      try {
+        int numDirs = localDirs.length;
+        int numDirsSearched = 0;
+        //remove the leading slash from the path (to make sure that the uri
+        //resolution results in a valid path on the dir being checked)
+        if (pathStr.startsWith("/")) {
+          pathStr = pathStr.substring(1);
+        }
+        while (numDirsSearched < numDirs) {
+          Path file = new Path(localDirs[numDirsSearched], pathStr);
+          if (localFS.exists(file)) {
+            return true;
+          }
+          numDirsSearched++;
+        }
+      } catch (IOException e) {
+        // IGNORE and try again
+      }
+      return false;
+    }
+  }
+}

+ 115 - 0
src/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.*;
+import java.net.URI;
+import java.util.*;
+
+/****************************************************************
+ * Implement the FileSystem API for the checksumed local filesystem.
+ *
+ *****************************************************************/
+public class LocalFileSystem extends ChecksumFileSystem {
+  static final URI NAME = URI.create("file:///");
+  static private Random rand = new Random();
+  FileSystem rfs;
+  
+  public LocalFileSystem() {
+    this(new RawLocalFileSystem());
+  }
+  
+  public FileSystem getRaw() {
+    return rfs;
+  }
+    
+  public LocalFileSystem(FileSystem rawLocalFileSystem) {
+    super(rawLocalFileSystem);
+    rfs = rawLocalFileSystem;
+  }
+    
+  /** Convert a path to a File. */
+  public File pathToFile(Path path) {
+    return ((RawLocalFileSystem)fs).pathToFile(path);
+  }
+
+  @Override
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    FileUtil.copy(this, src, this, dst, delSrc, getConf());
+  }
+
+  @Override
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst)
+    throws IOException {
+    FileUtil.copy(this, src, this, dst, delSrc, getConf());
+  }
+
+  /**
+   * Moves files to a bad file directory on the same device, so that their
+   * storage will not be reused.
+   */
+  public boolean reportChecksumFailure(Path p, FSDataInputStream in,
+                                       long inPos,
+                                       FSDataInputStream sums, long sumsPos) {
+    try {
+      // canonicalize f
+      File f = ((RawLocalFileSystem)fs).pathToFile(p).getCanonicalFile();
+      
+      // find highest writable parent dir of f on the same device
+      String device = new DF(f, getConf()).getMount();
+      File parent = f.getParentFile();
+      File dir = null;
+      while (parent!=null && parent.canWrite() && parent.toString().startsWith(device)) {
+        dir = parent;
+        parent = parent.getParentFile();
+      }
+
+      if (dir==null) {
+        throw new IOException(
+                              "not able to find the highest writable parent dir");
+      }
+        
+      // move the file there
+      File badDir = new File(dir, "bad_files");
+      if (!badDir.mkdirs()) {
+        if (!badDir.isDirectory()) {
+          throw new IOException("Mkdirs failed to create " + badDir.toString());
+        }
+      }
+      String suffix = "." + rand.nextInt();
+      File badFile = new File(badDir, f.getName()+suffix);
+      LOG.warn("Moving bad file " + f + " to " + badFile);
+      in.close();                               // close it first
+      boolean b = f.renameTo(badFile);                      // rename it
+      if (!b) {
+        LOG.warn("Ignoring failure of renameTo");
+      }
+      // move checksum file too
+      File checkFile = ((RawLocalFileSystem)fs).pathToFile(getChecksumFile(p));
+      b = checkFile.renameTo(new File(badDir, checkFile.getName()+suffix));
+      if (!b) {
+          LOG.warn("Ignoring failure of renameTo");
+        }
+    } catch (IOException e) {
+      LOG.warn("Error moving bad file " + p + ": " + e);
+    }
+    return false;
+  }
+}

+ 113 - 0
src/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java

@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.WritableUtils;
+import org.xml.sax.Attributes;
+import org.xml.sax.SAXException;
+import org.znerd.xmlenc.XMLOutputter;
+
+/** MD5 of MD5 of CRC32. */
+public class MD5MD5CRC32FileChecksum extends FileChecksum {
+  public static final int LENGTH = MD5Hash.MD5_LEN
+      + (Integer.SIZE + Long.SIZE)/Byte.SIZE;
+
+  private int bytesPerCRC;
+  private long crcPerBlock;
+  private MD5Hash md5;
+
+  /** Same as this(0, 0, null) */
+  public MD5MD5CRC32FileChecksum() {
+    this(0, 0, null);
+  }
+
+  /** Create a MD5FileChecksum */
+  public MD5MD5CRC32FileChecksum(int bytesPerCRC, long crcPerBlock, MD5Hash md5) {
+    this.bytesPerCRC = bytesPerCRC;
+    this.crcPerBlock = crcPerBlock;
+    this.md5 = md5;
+  }
+  
+  /** {@inheritDoc} */ 
+  public String getAlgorithmName() {
+    return "MD5-of-" + crcPerBlock + "MD5-of-" + bytesPerCRC + "CRC32";
+  }
+
+  /** {@inheritDoc} */ 
+  public int getLength() {return LENGTH;}
+
+  /** {@inheritDoc} */ 
+  public byte[] getBytes() {
+    return WritableUtils.toByteArray(this);
+  }
+
+  /** {@inheritDoc} */ 
+  public void readFields(DataInput in) throws IOException {
+    bytesPerCRC = in.readInt();
+    crcPerBlock = in.readLong();
+    md5 = MD5Hash.read(in);
+  }
+
+  /** {@inheritDoc} */ 
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(bytesPerCRC);
+    out.writeLong(crcPerBlock);
+    md5.write(out);    
+  }
+
+  /** Write that object to xml output. */
+  public static void write(XMLOutputter xml, MD5MD5CRC32FileChecksum that
+      ) throws IOException {
+    xml.startTag(MD5MD5CRC32FileChecksum.class.getName());
+    if (that != null) {
+      xml.attribute("bytesPerCRC", "" + that.bytesPerCRC);
+      xml.attribute("crcPerBlock", "" + that.crcPerBlock);
+      xml.attribute("md5", "" + that.md5);
+    }
+    xml.endTag();
+  }
+
+  /** Return the object represented in the attributes. */
+  public static MD5MD5CRC32FileChecksum valueOf(Attributes attrs
+      ) throws SAXException {
+    final String bytesPerCRC = attrs.getValue("bytesPerCRC");
+    final String crcPerBlock = attrs.getValue("crcPerBlock");
+    final String md5 = attrs.getValue("md5");
+    if (bytesPerCRC == null || crcPerBlock == null || md5 == null) {
+      return null;
+    }
+
+    try {
+      return new MD5MD5CRC32FileChecksum(Integer.valueOf(bytesPerCRC),
+          Integer.valueOf(crcPerBlock), new MD5Hash(md5));
+    } catch(Exception e) {
+      throw new SAXException("Invalid attributes: bytesPerCRC=" + bytesPerCRC
+          + ", crcPerBlock=" + crcPerBlock + ", md5=" + md5, e);
+    }
+  }
+
+  /** {@inheritDoc} */ 
+  public String toString() {
+    return getAlgorithmName() + ":" + md5;
+  }
+}

+ 298 - 0
src/java/org/apache/hadoop/fs/Path.java

@@ -0,0 +1,298 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.net.*;
+import java.io.*;
+
+import org.apache.hadoop.conf.Configuration;
+
+/** Names a file or directory in a {@link FileSystem}.
+ * Path strings use slash as the directory separator.  A path string is
+ * absolute if it begins with a slash.
+ */
+public class Path implements Comparable {
+
+  /** The directory separator, a slash. */
+  public static final String SEPARATOR = "/";
+  public static final char SEPARATOR_CHAR = '/';
+  
+  public static final String CUR_DIR = ".";
+  
+  static final boolean WINDOWS
+    = System.getProperty("os.name").startsWith("Windows");
+
+  private URI uri;                                // a hierarchical uri
+
+  /** Resolve a child path against a parent path. */
+  public Path(String parent, String child) {
+    this(new Path(parent), new Path(child));
+  }
+
+  /** Resolve a child path against a parent path. */
+  public Path(Path parent, String child) {
+    this(parent, new Path(child));
+  }
+
+  /** Resolve a child path against a parent path. */
+  public Path(String parent, Path child) {
+    this(new Path(parent), child);
+  }
+
+  /** Resolve a child path against a parent path. */
+  public Path(Path parent, Path child) {
+    // Add a slash to parent's path so resolution is compatible with URI's
+    URI parentUri = parent.uri;
+    String parentPath = parentUri.getPath();
+    if (!(parentPath.equals("/") || parentPath.equals("")))
+      try {
+        parentUri = new URI(parentUri.getScheme(), parentUri.getAuthority(),
+                            parentUri.getPath()+"/", null, null);
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException(e);
+      }
+    URI resolved = parentUri.resolve(child.uri);
+    initialize(resolved.getScheme(), resolved.getAuthority(),
+               normalizePath(resolved.getPath()));
+  }
+
+  private void checkPathArg( String path ) {
+    // disallow construction of a Path from an empty string
+    if ( path == null ) {
+      throw new IllegalArgumentException(
+          "Can not create a Path from a null string");
+    }
+    if( path.length() == 0 ) {
+       throw new IllegalArgumentException(
+           "Can not create a Path from an empty string");
+    }   
+  }
+  
+  /** Construct a path from a String.  Path strings are URIs, but with
+   * unescaped elements and some additional normalization. */
+  public Path(String pathString) {
+    checkPathArg( pathString );
+    
+    // We can't use 'new URI(String)' directly, since it assumes things are
+    // escaped, which we don't require of Paths. 
+    
+    // add a slash in front of paths with Windows drive letters
+    if (hasWindowsDrive(pathString, false))
+      pathString = "/"+pathString;
+
+    // parse uri components
+    String scheme = null;
+    String authority = null;
+
+    int start = 0;
+
+    // parse uri scheme, if any
+    int colon = pathString.indexOf(':');
+    int slash = pathString.indexOf('/');
+    if ((colon != -1) &&
+        ((slash == -1) || (colon < slash))) {     // has a scheme
+      scheme = pathString.substring(0, colon);
+      start = colon+1;
+    }
+
+    // parse uri authority, if any
+    if (pathString.startsWith("//", start) &&
+        (pathString.length()-start > 2)) {       // has authority
+      int nextSlash = pathString.indexOf('/', start+2);
+      int authEnd = nextSlash > 0 ? nextSlash : pathString.length();
+      authority = pathString.substring(start+2, authEnd);
+      start = authEnd;
+    }
+
+    // uri path is the rest of the string -- query & fragment not supported
+    String path = pathString.substring(start, pathString.length());
+
+    initialize(scheme, authority, path);
+  }
+
+  /** Construct a Path from components. */
+  public Path(String scheme, String authority, String path) {
+    checkPathArg( path );
+    initialize(scheme, authority, path);
+  }
+
+  private void initialize(String scheme, String authority, String path) {
+    try {
+      this.uri = new URI(scheme, authority, normalizePath(path), null, null)
+        .normalize();
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  private String normalizePath(String path) {
+    // remove double slashes & backslashes
+    path = path.replace("//", "/");
+    path = path.replace("\\", "/");
+    
+    // trim trailing slash from non-root path (ignoring windows drive)
+    int minLength = hasWindowsDrive(path, true) ? 4 : 1;
+    if (path.length() > minLength && path.endsWith("/")) {
+      path = path.substring(0, path.length()-1);
+    }
+    
+    return path;
+  }
+
+  private boolean hasWindowsDrive(String path, boolean slashed) {
+    if (!WINDOWS) return false;
+    int start = slashed ? 1 : 0;
+    return
+      path.length() >= start+2 &&
+      (slashed ? path.charAt(0) == '/' : true) &&
+      path.charAt(start+1) == ':' &&
+      ((path.charAt(start) >= 'A' && path.charAt(start) <= 'Z') ||
+       (path.charAt(start) >= 'a' && path.charAt(start) <= 'z'));
+  }
+
+
+  /** Convert this to a URI. */
+  public URI toUri() { return uri; }
+
+  /** Return the FileSystem that owns this Path. */
+  public FileSystem getFileSystem(Configuration conf) throws IOException {
+    return FileSystem.get(this.toUri(), conf);
+  }
+
+  /** True if the directory of this path is absolute. */
+  public boolean isAbsolute() {
+    int start = hasWindowsDrive(uri.getPath(), true) ? 3 : 0;
+    return uri.getPath().startsWith(SEPARATOR, start);
+  }
+
+  /** Returns the final component of this path.*/
+  public String getName() {
+    String path = uri.getPath();
+    int slash = path.lastIndexOf(SEPARATOR);
+    return path.substring(slash+1);
+  }
+
+  /** Returns the parent of a path or null if at root. */
+  public Path getParent() {
+    String path = uri.getPath();
+    int lastSlash = path.lastIndexOf('/');
+    int start = hasWindowsDrive(path, true) ? 3 : 0;
+    if ((path.length() == start) ||               // empty path
+        (lastSlash == start && path.length() == start+1)) { // at root
+      return null;
+    }
+    String parent;
+    if (lastSlash==-1) {
+      parent = CUR_DIR;
+    } else {
+      int end = hasWindowsDrive(path, true) ? 3 : 0;
+      parent = path.substring(0, lastSlash==end?end+1:lastSlash);
+    }
+    return new Path(uri.getScheme(), uri.getAuthority(), parent);
+  }
+
+  /** Adds a suffix to the final name in the path.*/
+  public Path suffix(String suffix) {
+    return new Path(getParent(), getName()+suffix);
+  }
+
+  public String toString() {
+    // we can't use uri.toString(), which escapes everything, because we want
+    // illegal characters unescaped in the string, for glob processing, etc.
+    StringBuffer buffer = new StringBuffer();
+    if (uri.getScheme() != null) {
+      buffer.append(uri.getScheme());
+      buffer.append(":");
+    }
+    if (uri.getAuthority() != null) {
+      buffer.append("//");
+      buffer.append(uri.getAuthority());
+    }
+    if (uri.getPath() != null) {
+      String path = uri.getPath();
+      if (path.indexOf('/')==0 &&
+          hasWindowsDrive(path, true) &&          // has windows drive
+          uri.getScheme() == null &&              // but no scheme
+          uri.getAuthority() == null)             // or authority
+        path = path.substring(1);                 // remove slash before drive
+      buffer.append(path);
+    }
+    return buffer.toString();
+  }
+
+  public boolean equals(Object o) {
+    if (!(o instanceof Path)) {
+      return false;
+    }
+    Path that = (Path)o;
+    return this.uri.equals(that.uri);
+  }
+
+  public int hashCode() {
+    return uri.hashCode();
+  }
+
+  public int compareTo(Object o) {
+    Path that = (Path)o;
+    return this.uri.compareTo(that.uri);
+  }
+  
+  /** Return the number of elements in this path. */
+  public int depth() {
+    String path = uri.getPath();
+    int depth = 0;
+    int slash = path.length()==1 && path.charAt(0)=='/' ? -1 : 0;
+    while (slash != -1) {
+      depth++;
+      slash = path.indexOf(SEPARATOR, slash+1);
+    }
+    return depth;
+  }
+
+  /** Returns a qualified path object. */
+  public Path makeQualified(FileSystem fs) {
+    Path path = this;
+    if (!isAbsolute()) {
+      path = new Path(fs.getWorkingDirectory(), this);
+    }
+
+    URI pathUri = path.toUri();
+    URI fsUri = fs.getUri();
+      
+    String scheme = pathUri.getScheme();
+    String authority = pathUri.getAuthority();
+
+    if (scheme != null &&
+        (authority != null || fsUri.getAuthority() == null))
+      return path;
+
+    if (scheme == null) {
+      scheme = fsUri.getScheme();
+    }
+
+    if (authority == null) {
+      authority = fsUri.getAuthority();
+      if (authority == null) {
+        authority = "";
+      }
+    }
+
+    return new Path(scheme+":"+"//"+authority + pathUri.getPath());
+  }
+}

+ 32 - 0
src/java/org/apache/hadoop/fs/PathFilter.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+public interface PathFilter {
+  /**
+   * Tests whether or not the specified abstract pathname should be
+   * included in a pathname list.
+   *
+   * @param  path  The abstract pathname to be tested
+   * @return  <code>true</code> if and only if <code>pathname</code>
+   *          should be included
+   */
+  boolean accept(Path path);
+}
+
+

+ 47 - 0
src/java/org/apache/hadoop/fs/PositionedReadable.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+
+/** Stream that permits positional reading. */
+public interface PositionedReadable {
+  /**
+   * Read upto the specified number of bytes, from a given
+   * position within a file, and return the number of bytes read. This does not
+   * change the current offset of a file, and is thread-safe.
+   */
+  public int read(long position, byte[] buffer, int offset, int length)
+    throws IOException;
+  
+  /**
+   * Read the specified number of bytes, from a given
+   * position within a file. This does not
+   * change the current offset of a file, and is thread-safe.
+   */
+  public void readFully(long position, byte[] buffer, int offset, int length)
+    throws IOException;
+  
+  /**
+   * Read number of bytes equalt to the length of the buffer, from a given
+   * position within a file. This does not
+   * change the current offset of a file, and is thread-safe.
+   */
+  public void readFully(long position, byte[] buffer) throws IOException;
+}

+ 496 - 0
src/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -0,0 +1,496 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
+
+/****************************************************************
+ * Implement the FileSystem API for the raw local filesystem.
+ *
+ *****************************************************************/
+public class RawLocalFileSystem extends FileSystem {
+  static final URI NAME = URI.create("file:///");
+  private Path workingDir;
+  
+  public RawLocalFileSystem() {
+    workingDir = new Path(System.getProperty("user.dir")).makeQualified(this);
+  }
+  
+  /** Convert a path to a File. */
+  public File pathToFile(Path path) {
+    checkPath(path);
+    if (!path.isAbsolute()) {
+      path = new Path(getWorkingDirectory(), path);
+    }
+    return new File(path.toUri().getPath());
+  }
+
+  public URI getUri() { return NAME; }
+  
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    setConf(conf);
+  }
+  
+  class TrackingFileInputStream extends FileInputStream {
+    public TrackingFileInputStream(File f) throws IOException {
+      super(f);
+    }
+    
+    public int read() throws IOException {
+      int result = super.read();
+      if (result != -1) {
+        statistics.incrementBytesRead(1);
+      }
+      return result;
+    }
+    
+    public int read(byte[] data) throws IOException {
+      int result = super.read(data);
+      if (result != -1) {
+        statistics.incrementBytesRead(result);
+      }
+      return result;
+    }
+    
+    public int read(byte[] data, int offset, int length) throws IOException {
+      int result = super.read(data, offset, length);
+      if (result != -1) {
+        statistics.incrementBytesRead(result);
+      }
+      return result;
+    }
+  }
+
+  /*******************************************************
+   * For open()'s FSInputStream
+   *******************************************************/
+  class LocalFSFileInputStream extends FSInputStream {
+    FileInputStream fis;
+    private long position;
+
+    public LocalFSFileInputStream(Path f) throws IOException {
+      this.fis = new TrackingFileInputStream(pathToFile(f));
+    }
+    
+    public void seek(long pos) throws IOException {
+      fis.getChannel().position(pos);
+      this.position = pos;
+    }
+    
+    public long getPos() throws IOException {
+      return this.position;
+    }
+    
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+    
+    /*
+     * Just forward to the fis
+     */
+    public int available() throws IOException { return fis.available(); }
+    public void close() throws IOException { fis.close(); }
+    public boolean markSupport() { return false; }
+    
+    public int read() throws IOException {
+      try {
+        int value = fis.read();
+        if (value >= 0) {
+          this.position++;
+        }
+        return value;
+      } catch (IOException e) {                 // unexpected exception
+        throw new FSError(e);                   // assume native fs error
+      }
+    }
+    
+    public int read(byte[] b, int off, int len) throws IOException {
+      try {
+        int value = fis.read(b, off, len);
+        if (value > 0) {
+          this.position += value;
+        }
+        return value;
+      } catch (IOException e) {                 // unexpected exception
+        throw new FSError(e);                   // assume native fs error
+      }
+    }
+    
+    public int read(long position, byte[] b, int off, int len)
+      throws IOException {
+      ByteBuffer bb = ByteBuffer.wrap(b, off, len);
+      try {
+        return fis.getChannel().read(bb, position);
+      } catch (IOException e) {
+        throw new FSError(e);
+      }
+    }
+    
+    public long skip(long n) throws IOException {
+      long value = fis.skip(n);
+      if (value > 0) {
+        this.position += value;
+      }
+      return value;
+    }
+  }
+  
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    if (!exists(f)) {
+      throw new FileNotFoundException(f.toString());
+    }
+    return new FSDataInputStream(new BufferedFSInputStream(
+        new LocalFSFileInputStream(f), bufferSize));
+  }
+  
+  /*********************************************************
+   * For create()'s FSOutputStream.
+   *********************************************************/
+  class LocalFSFileOutputStream extends OutputStream implements Syncable {
+    FileOutputStream fos;
+    
+    private LocalFSFileOutputStream(Path f, boolean append) throws IOException {
+      this.fos = new FileOutputStream(pathToFile(f), append);
+    }
+    
+    /*
+     * Just forward to the fos
+     */
+    public void close() throws IOException { fos.close(); }
+    public void flush() throws IOException { fos.flush(); }
+    public void write(byte[] b, int off, int len) throws IOException {
+      try {
+        fos.write(b, off, len);
+      } catch (IOException e) {                // unexpected exception
+        throw new FSError(e);                  // assume native fs error
+      }
+    }
+    
+    public void write(int b) throws IOException {
+      try {
+        fos.write(b);
+      } catch (IOException e) {              // unexpected exception
+        throw new FSError(e);                // assume native fs error
+      }
+    }
+
+    /** {@inheritDoc} */
+    public void sync() throws IOException {
+      fos.getFD().sync();      
+    }
+  }
+  
+  /** {@inheritDoc} */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    if (!exists(f)) {
+      throw new FileNotFoundException("File " + f + " not found.");
+    }
+    if (getFileStatus(f).isDir()) {
+      throw new IOException("Cannot append to a diretory (=" + f + " ).");
+    }
+    return new FSDataOutputStream(new BufferedOutputStream(
+        new LocalFSFileOutputStream(f, true), bufferSize), statistics);
+  }
+
+  /** {@inheritDoc} */
+  public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
+                                   short replication, long blockSize, Progressable progress)
+    throws IOException {
+    if (exists(f) && !overwrite) {
+      throw new IOException("File already exists:"+f);
+    }
+    Path parent = f.getParent();
+    if (parent != null && !mkdirs(parent)) {
+      throw new IOException("Mkdirs failed to create " + parent.toString());
+    }
+    return new FSDataOutputStream(new BufferedOutputStream(
+        new LocalFSFileOutputStream(f, false), bufferSize), statistics);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    FSDataOutputStream out = create(f,
+        overwrite, bufferSize, replication, blockSize, progress);
+    setPermission(f, permission);
+    return out;
+  }
+  
+  public boolean rename(Path src, Path dst) throws IOException {
+    if (pathToFile(src).renameTo(pathToFile(dst))) {
+      return true;
+    }
+    return FileUtil.copy(this, src, this, dst, true, getConf());
+  }
+  
+  public boolean delete(Path p, boolean recursive) throws IOException {
+    File f = pathToFile(p);
+    if (f.isFile()) {
+      return f.delete();
+    } else if ((!recursive) && f.isDirectory() && 
+        (f.listFiles().length != 0)) {
+      throw new IOException("Directory " + f.toString() + " is not empty");
+    }
+    return FileUtil.fullyDelete(f);
+  }
+ 
+  public FileStatus[] listStatus(Path f) throws IOException {
+    File localf = pathToFile(f);
+    FileStatus[] results;
+
+    if (!localf.exists()) {
+      return null;
+    }
+    if (localf.isFile()) {
+      return new FileStatus[] {
+          new RawLocalFileStatus(localf, getDefaultBlockSize(), this) };
+    }
+
+    String[] names = localf.list();
+    if (names == null) {
+      return null;
+    }
+    results = new FileStatus[names.length];
+    for (int i = 0; i < names.length; i++) {
+      results[i] = getFileStatus(new Path(f, names[i]));
+    }
+    return results;
+  }
+
+  /**
+   * Creates the specified directory hierarchy. Does not
+   * treat existence as an error.
+   */
+  public boolean mkdirs(Path f) throws IOException {
+    Path parent = f.getParent();
+    File p2f = pathToFile(f);
+    return (parent == null || mkdirs(parent)) &&
+      (p2f.mkdir() || p2f.isDirectory());
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    boolean b = mkdirs(f);
+    setPermission(f, permission);
+    return b;
+  }
+  
+  @Override
+  public Path getHomeDirectory() {
+    return new Path(System.getProperty("user.home")).makeQualified(this);
+  }
+
+  /**
+   * Set the working directory to the given directory.
+   */
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    workingDir = newDir;
+  }
+  
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public FsStatus getStatus(Path p) throws IOException {
+    File partition = pathToFile(p == null ? new Path("/") : p);
+    //File provides getUsableSpace() and getFreeSpace()
+    //File provides no API to obtain used space, assume used = total - free
+    return new FsStatus(partition.getTotalSpace(), 
+      partition.getTotalSpace() - partition.getFreeSpace(),
+      partition.getFreeSpace());
+  }
+  
+  // In the case of the local filesystem, we can just rename the file.
+  public void moveFromLocalFile(Path src, Path dst) throws IOException {
+    rename(src, dst);
+  }
+  
+  // We can write output directly to the final location
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+    throws IOException {
+    return fsOutputFile;
+  }
+  
+  // It's in the right place - nothing to do.
+  public void completeLocalOutput(Path fsWorkingFile, Path tmpLocalFile)
+    throws IOException {
+  }
+  
+  public void close() throws IOException {
+    super.close();
+  }
+  
+  public String toString() {
+    return "LocalFS";
+  }
+  
+  public FileStatus getFileStatus(Path f) throws IOException {
+    File path = pathToFile(f);
+    if (path.exists()) {
+      return new RawLocalFileStatus(pathToFile(f), getDefaultBlockSize(), this);
+    } else {
+      throw new FileNotFoundException( "File " + f + " does not exist.");
+    }
+  }
+
+  static class RawLocalFileStatus extends FileStatus {
+    /* We can add extra fields here. It breaks at least CopyFiles.FilePair().
+     * We recognize if the information is already loaded by check if
+     * onwer.equals("").
+     */
+    private boolean isPermissionLoaded() {
+      return !super.getOwner().equals(""); 
+    }
+    
+    RawLocalFileStatus(File f, long defaultBlockSize, FileSystem fs) {
+      super(f.length(), f.isDirectory(), 1, defaultBlockSize,
+            f.lastModified(), new Path(f.getPath()).makeQualified(fs));
+    }
+    
+    @Override
+    public FsPermission getPermission() {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      return super.getPermission();
+    }
+
+    @Override
+    public String getOwner() {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      return super.getOwner();
+    }
+
+    @Override
+    public String getGroup() {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      return super.getGroup();
+    }
+
+    /// loads permissions, owner, and group from `ls -ld`
+    private void loadPermissionInfo() {
+      IOException e = null;
+      try {
+        StringTokenizer t = new StringTokenizer(
+            execCommand(new File(getPath().toUri()), 
+                        Shell.getGET_PERMISSION_COMMAND()));
+        //expected format
+        //-rw-------    1 username groupname ...
+        String permission = t.nextToken();
+        if (permission.length() > 10) { //files with ACLs might have a '+'
+          permission = permission.substring(0, 10);
+        }
+        setPermission(FsPermission.valueOf(permission));
+        t.nextToken();
+        setOwner(t.nextToken());
+        setGroup(t.nextToken());
+      } catch (Shell.ExitCodeException ioe) {
+        if (ioe.getExitCode() != 1) {
+          e = ioe;
+        } else {
+          setPermission(null);
+          setOwner(null);
+          setGroup(null);
+        }
+      } catch (IOException ioe) {
+        e = ioe;
+      } finally {
+        if (e != null) {
+          throw new RuntimeException("Error while running command to get " +
+                                     "file permissions : " + 
+                                     StringUtils.stringifyException(e));
+        }
+      }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      super.write(out);
+    }
+  }
+
+  /**
+   * Use the command chown to set owner.
+   */
+  @Override
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+    if (username == null && groupname == null) {
+      throw new IOException("username == null && groupname == null");
+    }
+
+    if (username == null) {
+      execCommand(pathToFile(p), Shell.SET_GROUP_COMMAND, groupname); 
+    } else {
+      //OWNER[:[GROUP]]
+      String s = username + (groupname == null? "": ":" + groupname);
+      execCommand(pathToFile(p), Shell.SET_OWNER_COMMAND, s);
+    }
+  }
+
+  /**
+   * Use the command chmod to set permission.
+   */
+  @Override
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+    execCommand(pathToFile(p), Shell.SET_PERMISSION_COMMAND,
+        String.format("%05o", permission.toShort()));
+  }
+
+  private static String execCommand(File f, String... cmd) throws IOException {
+    String[] args = new String[cmd.length + 1];
+    System.arraycopy(cmd, 0, args, 0, cmd.length);
+    args[cmd.length] = f.getCanonicalPath();
+    String output = Shell.execCommand(args);
+    return output;
+  }
+}

+ 41 - 0
src/java/org/apache/hadoop/fs/Seekable.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.*;
+
+/** Stream that permits seeking. */
+public interface Seekable {
+  /**
+   * Seek to the given offset from the start of the file.
+   * The next read() will be from that location.  Can't
+   * seek past the end of the file.
+   */
+  void seek(long pos) throws IOException;
+  
+  /**
+   * Return the current offset from the start of the file
+   */
+  long getPos() throws IOException;
+
+  /**
+   * Seeks a different copy of the data.  Returns true if 
+   * found a new source, false otherwise.
+   */
+  boolean seekToNewSource(long targetPos) throws IOException;
+}

+ 30 - 0
src/java/org/apache/hadoop/fs/Syncable.java

@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+/** This interface declare the sync() operation. */
+public interface Syncable {
+  /**
+   * Synchronize all buffer with the underlying devices.
+   * @throws IOException
+   */
+  public void sync() throws IOException;
+}

+ 291 - 0
src/java/org/apache/hadoop/fs/Trash.java

@@ -0,0 +1,291 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.text.*;
+import java.io.*;
+import java.util.Date;
+
+import org.apache.commons.logging.*;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.util.StringUtils;
+
+/** Provides a <i>trash</i> feature.  Files are moved to a user's trash
+ * directory, a subdirectory of their home directory named ".Trash".  Files are
+ * initially moved to a <i>current</i> sub-directory of the trash directory.
+ * Within that sub-directory their original path is preserved.  Periodically
+ * one may checkpoint the current trash and remove older checkpoints.  (This
+ * design permits trash management without enumeration of the full trash
+ * content, without date support in the filesystem, and without clock
+ * synchronization.)
+ */
+public class Trash extends Configured {
+  private static final Log LOG =
+    LogFactory.getLog(Trash.class);
+
+  private static final Path CURRENT = new Path("Current");
+  private static final Path TRASH = new Path(".Trash/");
+  private static final Path HOMES = new Path("/user/");
+
+  private static final FsPermission PERMISSION =
+    new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE);
+
+  private static final DateFormat CHECKPOINT = new SimpleDateFormat("yyMMddHHmm");
+  private static final int MSECS_PER_MINUTE = 60*1000;
+
+  private final FileSystem fs;
+  private final Path trash;
+  private final Path current;
+  private final long interval;
+
+  /** Construct a trash can accessor.
+   * @param conf a Configuration
+   */
+  public Trash(Configuration conf) throws IOException {
+    this(FileSystem.get(conf), conf);
+  }
+
+  /**
+   * Construct a trash can accessor for the FileSystem provided.
+   */
+  public Trash(FileSystem fs, Configuration conf) throws IOException {
+    super(conf);
+    this.fs = fs;
+    this.trash = new Path(fs.getHomeDirectory(), TRASH);
+    this.current = new Path(trash, CURRENT);
+    this.interval = conf.getLong("fs.trash.interval", 60) * MSECS_PER_MINUTE;
+  }
+
+  private Trash(Path home, Configuration conf) throws IOException {
+    super(conf);
+    this.fs = home.getFileSystem(conf);
+    this.trash = new Path(home, TRASH);
+    this.current = new Path(trash, CURRENT);
+    this.interval = conf.getLong("fs.trash.interval", 60) * MSECS_PER_MINUTE;
+  }
+  
+  private Path makeTrashRelativePath(Path basePath, Path rmFilePath) {
+    return new Path(basePath + rmFilePath.toUri().getPath());
+  }
+
+  /** Move a file or directory to the current trash directory.
+   * @return false if the item is already in the trash or trash is disabled
+   */ 
+  public boolean moveToTrash(Path path) throws IOException {
+    if (interval == 0)
+      return false;
+
+    if (!path.isAbsolute())                       // make path absolute
+      path = new Path(fs.getWorkingDirectory(), path);
+
+    if (!fs.exists(path))                         // check that path exists
+      throw new FileNotFoundException(path.toString());
+
+    String qpath = path.makeQualified(fs).toString();
+
+    if (qpath.startsWith(trash.toString())) {
+      return false;                               // already in trash
+    }
+
+    if (trash.getParent().toString().startsWith(qpath)) {
+      throw new IOException("Cannot move \"" + path +
+                            "\" to the trash, as it contains the trash");
+    }
+
+    Path trashPath = makeTrashRelativePath(current, path);
+    Path baseTrashPath = makeTrashRelativePath(current, path.getParent());
+    
+    IOException cause = null;
+
+    // try twice, in case checkpoint between the mkdirs() & rename()
+    for (int i = 0; i < 2; i++) {
+      try {
+        if (!fs.mkdirs(baseTrashPath, PERMISSION)) {      // create current
+          LOG.warn("Can't create trash directory: "+baseTrashPath);
+          return false;
+        }
+      } catch (IOException e) {
+        LOG.warn("Can't create trash directory: "+baseTrashPath);
+        return false;
+      }
+      try {
+        //
+        // if the target path in Trash already exists, then append with 
+        // a number. Start from 1.
+        //
+        String orig = trashPath.toString();
+        for (int j = 1; fs.exists(trashPath); j++) {
+          trashPath = new Path(orig + "." + j);
+        }
+        if (fs.rename(path, trashPath))           // move to current trash
+          return true;
+      } catch (IOException e) {
+        cause = e;
+      }
+    }
+    throw (IOException)
+      new IOException("Failed to move to trash: "+path).initCause(cause);
+  }
+
+  /** Create a trash checkpoint. */
+  public void checkpoint() throws IOException {
+    if (!fs.exists(current))                      // no trash, no checkpoint
+      return;
+
+    Path checkpoint;
+    synchronized (CHECKPOINT) {
+      checkpoint = new Path(trash, CHECKPOINT.format(new Date()));
+    }
+
+    if (fs.rename(current, checkpoint)) {
+      LOG.info("Created trash checkpoint: "+checkpoint.toUri().getPath());
+    } else {
+      throw new IOException("Failed to checkpoint trash: "+checkpoint);
+    }
+  }
+
+  /** Delete old checkpoints. */
+  public void expunge() throws IOException {
+    FileStatus[] dirs = fs.listStatus(trash);            // scan trash sub-directories
+    if( dirs == null){
+      return;
+    }
+    long now = System.currentTimeMillis();
+    for (int i = 0; i < dirs.length; i++) {
+      Path path = dirs[i].getPath();
+      String dir = path.toUri().getPath();
+      String name = path.getName();
+      if (name.equals(CURRENT.getName()))         // skip current
+        continue;
+
+      long time;
+      try {
+        synchronized (CHECKPOINT) {
+          time = CHECKPOINT.parse(name).getTime();
+        }
+      } catch (ParseException e) {
+        LOG.warn("Unexpected item in trash: "+dir+". Ignoring.");
+        continue;
+      }
+
+      if ((now - interval) > time) {
+        if (fs.delete(path, true)) {
+          LOG.info("Deleted trash checkpoint: "+dir);
+        } else {
+          LOG.warn("Couldn't delete checkpoint: "+dir+" Ignoring.");
+        }
+      }
+    }
+  }
+
+  //
+  // get the current working directory
+  //
+  Path getCurrentTrashDir() {
+    return current;
+  }
+
+  /** Return a {@link Runnable} that periodically empties the trash of all
+   * users, intended to be run by the superuser.  Only one checkpoint is kept
+   * at a time.
+   */
+  public Runnable getEmptier() throws IOException {
+    return new Emptier(getConf());
+  }
+
+  private class Emptier implements Runnable {
+
+    private Configuration conf;
+    private long interval;
+
+    Emptier(Configuration conf) throws IOException {
+      this.conf = conf;
+      this.interval = conf.getLong("fs.trash.interval", 0) * MSECS_PER_MINUTE;
+    }
+
+    public void run() {
+      if (interval == 0)
+        return;                                   // trash disabled
+
+      long now = System.currentTimeMillis();
+      long end;
+      while (true) {
+        end = ceiling(now, interval);
+        try {                                     // sleep for interval
+          Thread.sleep(end - now);
+        } catch (InterruptedException e) {
+          break;                                  // exit on interrupt
+        }
+          
+        try {
+          now = System.currentTimeMillis();
+          if (now >= end) {
+
+            FileStatus[] homes = null;
+            try {
+              homes = fs.listStatus(HOMES);         // list all home dirs
+            } catch (IOException e) {
+              LOG.warn("Trash can't list homes: "+e+" Sleeping.");
+              continue;
+            }
+
+            if (homes == null)
+              continue;
+
+            for (FileStatus home : homes) {         // dump each trash
+              if (!home.isDir())
+                continue;
+              try {
+                Trash trash = new Trash(home.getPath(), conf);
+                trash.expunge();
+                trash.checkpoint();
+              } catch (IOException e) {
+                LOG.warn("Trash caught: "+e+". Skipping "+home.getPath()+".");
+              } 
+            }
+          }
+        } catch (Exception e) {
+          LOG.warn("RuntimeException during Trash.Emptier.run() " + 
+                   StringUtils.stringifyException(e));
+        }
+      }
+      try {
+        fs.close();
+      } catch(IOException e) {
+        LOG.warn("Trash cannot close FileSystem. " +
+            StringUtils.stringifyException(e));
+      }
+    }
+
+    private long ceiling(long time, long interval) {
+      return floor(time, interval) + interval;
+    }
+    private long floor(long time, long interval) {
+      return (time / interval) * interval;
+    }
+
+  }
+
+  /** Run an emptier.*/
+  public static void main(String[] args) throws Exception {
+    new Trash(new Configuration()).getEmptier().run();
+  }
+
+}

+ 38 - 0
src/java/org/apache/hadoop/fs/ftp/FTPException.java

@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.ftp;
+
+/**
+ * A class to wrap a {@link Throwable} into a Runtime Exception.
+ */
+public class FTPException extends RuntimeException {
+
+  private static final long serialVersionUID = 1L;
+
+  public FTPException(String message) {
+    super(message);
+  }
+
+  public FTPException(Throwable t) {
+    super(t);
+  }
+
+  public FTPException(String message, Throwable t) {
+    super(message, t);
+  }
+}

+ 576 - 0
src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java

@@ -0,0 +1,576 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.ftp;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.net.ftp.FTP;
+import org.apache.commons.net.ftp.FTPClient;
+import org.apache.commons.net.ftp.FTPFile;
+import org.apache.commons.net.ftp.FTPReply;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * <p>
+ * A {@link FileSystem} backed by an FTP client provided by <a
+ * href="http://commons.apache.org/net/">Apache Commons Net</a>.
+ * </p>
+ */
+public class FTPFileSystem extends FileSystem {
+
+  public static final Log LOG = LogFactory
+      .getLog(FTPFileSystem.class);
+
+  public static final int DEFAULT_BUFFER_SIZE = 1024 * 1024;
+
+  public static final int DEFAULT_BLOCK_SIZE = 4 * 1024;
+
+  private URI uri;
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException { // get
+    super.initialize(uri, conf);
+    // get host information from uri (overrides info in conf)
+    String host = uri.getHost();
+    host = (host == null) ? conf.get("fs.ftp.host", null) : host;
+    if (host == null) {
+      throw new IOException("Invalid host specified");
+    }
+    conf.set("fs.ftp.host", host);
+
+    // get port information from uri, (overrides info in conf)
+    int port = uri.getPort();
+    port = (port == -1) ? FTP.DEFAULT_PORT : port;
+    conf.setInt("fs.ftp.host.port", port);
+
+    // get user/password information from URI (overrides info in conf)
+    String userAndPassword = uri.getUserInfo();
+    if (userAndPassword == null) {
+      userAndPassword = (conf.get("fs.ftp.user." + host, null) + ":" + conf
+          .get("fs.ftp.password." + host, null));
+      if (userAndPassword == null) {
+        throw new IOException("Invalid user/passsword specified");
+      }
+    }
+    String[] userPasswdInfo = userAndPassword.split(":");
+    conf.set("fs.ftp.user." + host, userPasswdInfo[0]);
+    if (userPasswdInfo.length > 1) {
+      conf.set("fs.ftp.password." + host, userPasswdInfo[1]);
+    } else {
+      conf.set("fs.ftp.password." + host, null);
+    }
+    setConf(conf);
+    this.uri = uri;
+  }
+
+  /**
+   * Connect to the FTP server using configuration parameters *
+   * 
+   * @return An FTPClient instance
+   * @throws IOException
+   */
+  private FTPClient connect() throws IOException {
+    FTPClient client = null;
+    Configuration conf = getConf();
+    String host = conf.get("fs.ftp.host");
+    int port = conf.getInt("fs.ftp.host.port", FTP.DEFAULT_PORT);
+    String user = conf.get("fs.ftp.user." + host);
+    String password = conf.get("fs.ftp.password." + host);
+    client = new FTPClient();
+    client.connect(host, port);
+    int reply = client.getReplyCode();
+    if (!FTPReply.isPositiveCompletion(reply)) {
+      throw new IOException("Server - " + host
+          + " refused connection on port - " + port);
+    } else if (client.login(user, password)) {
+      client.setFileTransferMode(FTP.BLOCK_TRANSFER_MODE);
+      client.setFileType(FTP.BINARY_FILE_TYPE);
+      client.setBufferSize(DEFAULT_BUFFER_SIZE);
+    } else {
+      throw new IOException("Login failed on server - " + host + ", port - "
+          + port);
+    }
+
+    return client;
+  }
+
+  /**
+   * Logout and disconnect the given FTPClient. *
+   * 
+   * @param client
+   * @throws IOException
+   */
+  private void disconnect(FTPClient client) throws IOException {
+    if (client != null) {
+      if (!client.isConnected()) {
+        throw new FTPException("Client not connected");
+      }
+      boolean logoutSuccess = client.logout();
+      client.disconnect();
+      if (!logoutSuccess) {
+        LOG.warn("Logout failed while disconnecting, error code - "
+            + client.getReplyCode());
+      }
+    }
+  }
+
+  /**
+   * Resolve against given working directory. *
+   * 
+   * @param workDir
+   * @param path
+   * @return
+   */
+  private Path makeAbsolute(Path workDir, Path path) {
+    if (path.isAbsolute()) {
+      return path;
+    }
+    return new Path(workDir, path);
+  }
+
+  @Override
+  public FSDataInputStream open(Path file, int bufferSize) throws IOException {
+    FTPClient client = connect();
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absolute = makeAbsolute(workDir, file);
+    FileStatus fileStat = getFileStatus(client, absolute);
+    if (fileStat.isDir()) {
+      disconnect(client);
+      throw new IOException("Path " + file + " is a directory.");
+    }
+    client.allocate(bufferSize);
+    Path parent = absolute.getParent();
+    // Change to parent directory on the
+    // server. Only then can we read the
+    // file
+    // on the server by opening up an InputStream. As a side effect the working
+    // directory on the server is changed to the parent directory of the file.
+    // The FTP client connection is closed when close() is called on the
+    // FSDataInputStream.
+    client.changeWorkingDirectory(parent.toUri().getPath());
+    InputStream is = client.retrieveFileStream(file.getName());
+    FSDataInputStream fis = new FSDataInputStream(new FTPInputStream(is,
+        client, statistics));
+    if (!FTPReply.isPositivePreliminary(client.getReplyCode())) {
+      // The ftpClient is an inconsistent state. Must close the stream
+      // which in turn will logout and disconnect from FTP server
+      fis.close();
+      throw new IOException("Unable to open file: " + file + ", Aborting");
+    }
+    return fis;
+  }
+
+  /**
+   * A stream obtained via this call must be closed before using other APIs of
+   * this class or else the invocation will block.
+   */
+  @Override
+  public FSDataOutputStream create(Path file, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    final FTPClient client = connect();
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absolute = makeAbsolute(workDir, file);
+    if (exists(client, file)) {
+      if (overwrite) {
+        delete(client, file);
+      } else {
+        disconnect(client);
+        throw new IOException("File already exists: " + file);
+      }
+    }
+    Path parent = absolute.getParent();
+    if (parent == null || !mkdirs(client, parent, FsPermission.getDefault())) {
+      parent = (parent == null) ? new Path("/") : parent;
+      disconnect(client);
+      throw new IOException("create(): Mkdirs failed to create: " + parent);
+    }
+    client.allocate(bufferSize);
+    // Change to parent directory on the server. Only then can we write to the
+    // file on the server by opening up an OutputStream. As a side effect the
+    // working directory on the server is changed to the parent directory of the
+    // file. The FTP client connection is closed when close() is called on the
+    // FSDataOutputStream.
+    client.changeWorkingDirectory(parent.toUri().getPath());
+    FSDataOutputStream fos = new FSDataOutputStream(client.storeFileStream(file
+        .getName()), statistics) {
+      @Override
+      public void close() throws IOException {
+        super.close();
+        if (!client.isConnected()) {
+          throw new FTPException("Client not connected");
+        }
+        boolean cmdCompleted = client.completePendingCommand();
+        disconnect(client);
+        if (!cmdCompleted) {
+          throw new FTPException("Could not complete transfer, Reply Code - "
+              + client.getReplyCode());
+        }
+      }
+    };
+    if (!FTPReply.isPositivePreliminary(client.getReplyCode())) {
+      // The ftpClient is an inconsistent state. Must close the stream
+      // which in turn will logout and disconnect from FTP server
+      fos.close();
+      throw new IOException("Unable to create file: " + file + ", Aborting");
+    }
+    return fos;
+  }
+
+  /** This optional operation is not yet supported. */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Not supported");
+  }
+  
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private boolean exists(FTPClient client, Path file) {
+    try {
+      return getFileStatus(client, file) != null;
+    } catch (FileNotFoundException fnfe) {
+      return false;
+    } catch (IOException ioe) {
+      throw new FTPException("Failed to get file status", ioe);
+    }
+  }
+
+  @Override
+  public boolean delete(Path file, boolean recursive) throws IOException {
+    FTPClient client = connect();
+    try {
+      boolean success = delete(client, file, recursive);
+      return success;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  /** @deprecated Use delete(Path, boolean) instead */
+  @Deprecated
+  private boolean delete(FTPClient client, Path file) throws IOException {
+    return delete(client, file, false);
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private boolean delete(FTPClient client, Path file, boolean recursive)
+      throws IOException {
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absolute = makeAbsolute(workDir, file);
+    String pathName = absolute.toUri().getPath();
+    FileStatus fileStat = getFileStatus(client, absolute);
+    if (!fileStat.isDir()) {
+      return client.deleteFile(pathName);
+    }
+    FileStatus[] dirEntries = listStatus(client, absolute);
+    if (dirEntries != null && dirEntries.length > 0 && !(recursive)) {
+      throw new IOException("Directory: " + file + " is not empty.");
+    }
+    if (dirEntries != null) {
+      for (int i = 0; i < dirEntries.length; i++) {
+        delete(client, new Path(absolute, dirEntries[i].getPath()), recursive);
+      }
+    }
+    return client.removeDirectory(pathName);
+  }
+
+  private FsAction getFsAction(int accessGroup, FTPFile ftpFile) {
+    FsAction action = FsAction.NONE;
+    if (ftpFile.hasPermission(accessGroup, FTPFile.READ_PERMISSION)) {
+      action.or(FsAction.READ);
+    }
+    if (ftpFile.hasPermission(accessGroup, FTPFile.WRITE_PERMISSION)) {
+      action.or(FsAction.WRITE);
+    }
+    if (ftpFile.hasPermission(accessGroup, FTPFile.EXECUTE_PERMISSION)) {
+      action.or(FsAction.EXECUTE);
+    }
+    return action;
+  }
+
+  private FsPermission getPermissions(FTPFile ftpFile) {
+    FsAction user, group, others;
+    user = getFsAction(FTPFile.USER_ACCESS, ftpFile);
+    group = getFsAction(FTPFile.GROUP_ACCESS, ftpFile);
+    others = getFsAction(FTPFile.WORLD_ACCESS, ftpFile);
+    return new FsPermission(user, group, others);
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path file) throws IOException {
+    FTPClient client = connect();
+    try {
+      FileStatus[] stats = listStatus(client, file);
+      return stats;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private FileStatus[] listStatus(FTPClient client, Path file)
+      throws IOException {
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absolute = makeAbsolute(workDir, file);
+    FileStatus fileStat = getFileStatus(client, absolute);
+    if (!fileStat.isDir()) {
+      return new FileStatus[] { fileStat };
+    }
+    FTPFile[] ftpFiles = client.listFiles(absolute.toUri().getPath());
+    FileStatus[] fileStats = new FileStatus[ftpFiles.length];
+    for (int i = 0; i < ftpFiles.length; i++) {
+      fileStats[i] = getFileStatus(ftpFiles[i], absolute);
+    }
+    return fileStats;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path file) throws IOException {
+    FTPClient client = connect();
+    try {
+      FileStatus status = getFileStatus(client, file);
+      return status;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private FileStatus getFileStatus(FTPClient client, Path file)
+      throws IOException {
+    FileStatus fileStat = null;
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absolute = makeAbsolute(workDir, file);
+    Path parentPath = absolute.getParent();
+    if (parentPath == null) { // root dir
+      long length = -1; // Length of root dir on server not known
+      boolean isDir = true;
+      int blockReplication = 1;
+      long blockSize = DEFAULT_BLOCK_SIZE; // Block Size not known.
+      long modTime = -1; // Modification time of root dir not known.
+      Path root = new Path("/");
+      return new FileStatus(length, isDir, blockReplication, blockSize,
+          modTime, root.makeQualified(this));
+    }
+    String pathName = parentPath.toUri().getPath();
+    FTPFile[] ftpFiles = client.listFiles(pathName);
+    if (ftpFiles != null) {
+      for (FTPFile ftpFile : ftpFiles) {
+        if (ftpFile.getName().equals(file.getName())) { // file found in dir
+          fileStat = getFileStatus(ftpFile, parentPath);
+          break;
+        }
+      }
+      if (fileStat == null) {
+        throw new FileNotFoundException("File " + file + " does not exist.");
+      }
+    } else {
+      throw new FileNotFoundException("File " + file + " does not exist.");
+    }
+    return fileStat;
+  }
+
+  /**
+   * Convert the file information in FTPFile to a {@link FileStatus} object. *
+   * 
+   * @param ftpFile
+   * @param parentPath
+   * @return FileStatus
+   */
+  private FileStatus getFileStatus(FTPFile ftpFile, Path parentPath) {
+    long length = ftpFile.getSize();
+    boolean isDir = ftpFile.isDirectory();
+    int blockReplication = 1;
+    // Using default block size since there is no way in FTP client to know of
+    // block sizes on server. The assumption could be less than ideal.
+    long blockSize = DEFAULT_BLOCK_SIZE;
+    long modTime = ftpFile.getTimestamp().getTimeInMillis();
+    long accessTime = 0;
+    FsPermission permission = getPermissions(ftpFile);
+    String user = ftpFile.getUser();
+    String group = ftpFile.getGroup();
+    Path filePath = new Path(parentPath, ftpFile.getName());
+    return new FileStatus(length, isDir, blockReplication, blockSize, modTime,
+        accessTime, permission, user, group, filePath.makeQualified(this));
+  }
+
+  @Override
+  public boolean mkdirs(Path file, FsPermission permission) throws IOException {
+    FTPClient client = connect();
+    try {
+      boolean success = mkdirs(client, file, permission);
+      return success;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private boolean mkdirs(FTPClient client, Path file, FsPermission permission)
+      throws IOException {
+    boolean created = true;
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absolute = makeAbsolute(workDir, file);
+    String pathName = absolute.getName();
+    if (!exists(client, absolute)) {
+      Path parent = absolute.getParent();
+      created = (parent == null || mkdirs(client, parent, FsPermission
+          .getDefault()));
+      if (created) {
+        String parentDir = parent.toUri().getPath();
+        client.changeWorkingDirectory(parentDir);
+        created = created & client.makeDirectory(pathName);
+      }
+    } else if (isFile(client, absolute)) {
+      throw new IOException(String.format(
+          "Can't make directory for path %s since it is a file.", absolute));
+    }
+    return created;
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private boolean isFile(FTPClient client, Path file) {
+    try {
+      return !getFileStatus(client, file).isDir();
+    } catch (FileNotFoundException e) {
+      return false; // file does not exist
+    } catch (IOException ioe) {
+      throw new FTPException("File check failed", ioe);
+    }
+  }
+
+  /*
+   * Assuming that parent of both source and destination is the same. Is the
+   * assumption correct or it is suppose to work like 'move' ?
+   */
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    FTPClient client = connect();
+    try {
+      boolean success = rename(client, src, dst);
+      return success;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   * 
+   * @param client
+   * @param src
+   * @param dst
+   * @return
+   * @throws IOException
+   */
+  private boolean rename(FTPClient client, Path src, Path dst)
+      throws IOException {
+    Path workDir = new Path(client.printWorkingDirectory());
+    Path absoluteSrc = makeAbsolute(workDir, src);
+    Path absoluteDst = makeAbsolute(workDir, dst);
+    if (!exists(client, absoluteSrc)) {
+      throw new IOException("Source path " + src + " does not exist");
+    }
+    if (exists(client, absoluteDst)) {
+      throw new IOException("Destination path " + dst
+          + " already exist, cannot rename!");
+    }
+    String parentSrc = absoluteSrc.getParent().toUri().toString();
+    String parentDst = absoluteDst.getParent().toUri().toString();
+    String from = src.getName();
+    String to = dst.getName();
+    if (!parentSrc.equals(parentDst)) {
+      throw new IOException("Cannot rename parent(source): " + parentSrc
+          + ", parent(destination):  " + parentDst);
+    }
+    client.changeWorkingDirectory(parentSrc);
+    boolean renamed = client.rename(from, to);
+    return renamed;
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    // Return home directory always since we do not maintain state.
+    return getHomeDirectory();
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    FTPClient client = null;
+    try {
+      client = connect();
+      Path homeDir = new Path(client.printWorkingDirectory());
+      return homeDir;
+    } catch (IOException ioe) {
+      throw new FTPException("Failed to get home directory", ioe);
+    } finally {
+      try {
+        disconnect(client);
+      } catch (IOException ioe) {
+        throw new FTPException("Failed to disconnect", ioe);
+      }
+    }
+  }
+
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    // we do not maintain the working directory state
+  }
+}

+ 126 - 0
src/java/org/apache/hadoop/fs/ftp/FTPInputStream.java

@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.ftp;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.commons.net.ftp.FTPClient;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem;
+
+public class FTPInputStream extends FSInputStream {
+
+  InputStream wrappedStream;
+  FTPClient client;
+  FileSystem.Statistics stats;
+  boolean closed;
+  long pos;
+
+  public FTPInputStream(InputStream stream, FTPClient client,
+      FileSystem.Statistics stats) {
+    if (stream == null) {
+      throw new IllegalArgumentException("Null InputStream");
+    }
+    if (client == null || !client.isConnected()) {
+      throw new IllegalArgumentException("FTP client null or not connected");
+    }
+    this.wrappedStream = stream;
+    this.client = client;
+    this.stats = stats;
+    this.pos = 0;
+    this.closed = false;
+  }
+
+  public long getPos() throws IOException {
+    return pos;
+  }
+
+  // We don't support seek.
+  public void seek(long pos) throws IOException {
+    throw new IOException("Seek not supported");
+  }
+
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    throw new IOException("Seek not supported");
+  }
+
+  public synchronized int read() throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+
+    int byteRead = wrappedStream.read();
+    if (byteRead >= 0) {
+      pos++;
+    }
+    if (stats != null & byteRead >= 0) {
+      stats.incrementBytesRead(1);
+    }
+    return byteRead;
+  }
+
+  public synchronized int read(byte buf[], int off, int len) throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+
+    int result = wrappedStream.read(buf, off, len);
+    if (result > 0) {
+      pos += result;
+    }
+    if (stats != null & result > 0) {
+      stats.incrementBytesRead(result);
+    }
+
+    return result;
+  }
+
+  public synchronized void close() throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+    super.close();
+    closed = true;
+    if (!client.isConnected()) {
+      throw new FTPException("Client not connected");
+    }
+
+    boolean cmdCompleted = client.completePendingCommand();
+    client.logout();
+    client.disconnect();
+    if (!cmdCompleted) {
+      throw new FTPException("Could not complete transfer, Reply Code - "
+          + client.getReplyCode());
+    }
+  }
+
+  // Not supported.
+
+  public boolean markSupported() {
+    return false;
+  }
+
+  public void mark(int readLimit) {
+    // Do nothing
+  }
+
+  public void reset() throws IOException {
+    throw new IOException("Mark not supported");
+  }
+}

+ 60 - 0
src/java/org/apache/hadoop/fs/kfs/IFSImpl.java

@@ -0,0 +1,60 @@
+/**
+ *
+ * Licensed under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ * implied. See the License for the specific language governing
+ * permissions and limitations under the License.
+ *
+ * @author: Sriram Rao (Kosmix Corp.)
+ * 
+ * We need to provide the ability to the code in fs/kfs without really
+ * having a KFS deployment.  In particular, the glue code that wraps
+ * around calls to KfsAccess object.  This is accomplished by defining a
+ * filesystem implementation interface:  
+ *   -- for testing purposes, a dummy implementation of this interface
+ * will suffice; as long as the dummy implementation is close enough
+ * to doing what KFS does, we are good.
+ *   -- for deployment purposes with KFS, this interface is
+ * implemented by the KfsImpl object.
+ */
+
+package org.apache.hadoop.fs.kfs;
+
+import java.io.*;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Progressable;
+
+interface IFSImpl {
+    public boolean exists(String path) throws IOException;
+    public boolean isDirectory(String path) throws IOException;
+    public boolean isFile(String path) throws IOException;
+    public String[] readdir(String path) throws IOException;
+    public FileStatus[] readdirplus(Path path) throws IOException;
+
+    public int mkdirs(String path) throws IOException;
+    public int rename(String source, String dest) throws IOException;
+
+    public int rmdir(String path) throws IOException; 
+    public int remove(String path) throws IOException;
+    public long filesize(String path) throws IOException;
+    public short getReplication(String path) throws IOException;
+    public short setReplication(String path, short replication) throws IOException;
+    public String[][] getDataLocation(String path, long start, long len) throws IOException;
+
+    public long getModificationTime(String path) throws IOException;
+    public FSDataOutputStream create(String path, short replication, int bufferSize, Progressable progress) throws IOException;
+    public FSDataInputStream open(String path, int bufferSize) throws IOException;
+    public FSDataOutputStream append(String path, int bufferSize, Progressable progress) throws IOException;
+    
+};

+ 151 - 0
src/java/org/apache/hadoop/fs/kfs/KFSImpl.java

@@ -0,0 +1,151 @@
+/**
+ *
+ * Licensed under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ * implied. See the License for the specific language governing
+ * permissions and limitations under the License.
+ *
+ * @author: Sriram Rao (Kosmix Corp.)
+ * 
+ * Provide the implementation of KFS which turn into calls to KfsAccess.
+ */
+
+package org.apache.hadoop.fs.kfs;
+
+import java.io.*;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.kosmix.kosmosfs.access.KfsAccess;
+import org.kosmix.kosmosfs.access.KfsFileAttr;
+import org.apache.hadoop.util.Progressable;
+
+class KFSImpl implements IFSImpl {
+    private KfsAccess kfsAccess = null;
+    private FileSystem.Statistics statistics;
+
+    @Deprecated
+    public KFSImpl(String metaServerHost, int metaServerPort
+                   ) throws IOException {
+      this(metaServerHost, metaServerPort, null);
+    }
+
+    public KFSImpl(String metaServerHost, int metaServerPort, 
+                   FileSystem.Statistics stats) throws IOException {
+        kfsAccess = new KfsAccess(metaServerHost, metaServerPort);
+        statistics = stats;
+    }
+
+    public boolean exists(String path) throws IOException {
+        return kfsAccess.kfs_exists(path);
+    }
+
+    public boolean isDirectory(String path) throws IOException {
+        return kfsAccess.kfs_isDirectory(path);
+    }
+
+    public boolean isFile(String path) throws IOException {
+        return kfsAccess.kfs_isFile(path);
+    }
+
+    public String[] readdir(String path) throws IOException {
+        return kfsAccess.kfs_readdir(path);
+    }
+
+    public FileStatus[] readdirplus(Path path) throws IOException {
+        String srep = path.toUri().getPath();
+        KfsFileAttr[] fattr = kfsAccess.kfs_readdirplus(srep);
+        if (fattr == null)
+            return null;
+        int numEntries = 0;
+        for (int i = 0; i < fattr.length; i++) {
+            if ((fattr[i].filename.compareTo(".") == 0) || (fattr[i].filename.compareTo("..") == 0))
+                continue;
+            numEntries++;
+        }
+        FileStatus[] fstatus = new FileStatus[numEntries];
+        int j = 0;
+        for (int i = 0; i < fattr.length; i++) {
+            if ((fattr[i].filename.compareTo(".") == 0) || (fattr[i].filename.compareTo("..") == 0))
+                continue;
+            Path fn = new Path(path, fattr[i].filename);
+
+            if (fattr[i].isDirectory)
+                fstatus[j] = new FileStatus(0, true, 1, 0, fattr[i].modificationTime, fn);
+            else
+                fstatus[j] = new FileStatus(fattr[i].filesize, fattr[i].isDirectory,
+                                            fattr[i].replication,
+                                            (long)
+                                            (1 << 26),
+                                            fattr[i].modificationTime,
+                                            fn);
+
+            j++;
+        }
+        return fstatus;
+    }
+
+
+    public int mkdirs(String path) throws IOException {
+        return kfsAccess.kfs_mkdirs(path);
+    }
+
+    public int rename(String source, String dest) throws IOException {
+        return kfsAccess.kfs_rename(source, dest);
+    }
+
+    public int rmdir(String path) throws IOException {
+        return kfsAccess.kfs_rmdir(path);
+    }
+
+    public int remove(String path) throws IOException {
+        return kfsAccess.kfs_remove(path);
+    }
+
+    public long filesize(String path) throws IOException {
+        return kfsAccess.kfs_filesize(path);
+    }
+
+    public short getReplication(String path) throws IOException {
+        return kfsAccess.kfs_getReplication(path);
+    }
+
+    public short setReplication(String path, short replication) throws IOException {
+        return kfsAccess.kfs_setReplication(path, replication);
+    }
+
+    public String[][] getDataLocation(String path, long start, long len) throws IOException {
+        return kfsAccess.kfs_getDataLocation(path, start, len);
+    }
+
+    public long getModificationTime(String path) throws IOException {
+        return kfsAccess.kfs_getModificationTime(path);
+    }
+
+    public FSDataInputStream open(String path, int bufferSize) throws IOException {
+        return new FSDataInputStream(new KFSInputStream(kfsAccess, path, 
+                                                        statistics));
+    }
+
+    public FSDataOutputStream create(String path, short replication, int bufferSize, Progressable progress) throws IOException {
+        return new FSDataOutputStream(new KFSOutputStream(kfsAccess, path, replication, false, progress), 
+                                      statistics);
+    }
+
+    public FSDataOutputStream append(String path, int bufferSize, Progressable progress) throws IOException {
+        // when opening for append, # of replicas is ignored
+        return new FSDataOutputStream(new KFSOutputStream(kfsAccess, path, (short) 1, true, progress), 
+                                      statistics);
+    }
+}

+ 130 - 0
src/java/org/apache/hadoop/fs/kfs/KFSInputStream.java

@@ -0,0 +1,130 @@
+/**
+ *
+ * Licensed under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ * implied. See the License for the specific language governing
+ * permissions and limitations under the License.
+ *
+ * @author: Sriram Rao (Kosmix Corp.)
+ * 
+ * Implements the Hadoop FSInputStream interfaces to allow applications to read
+ * files in Kosmos File System (KFS).
+ */
+
+package org.apache.hadoop.fs.kfs;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSInputStream;
+
+import org.kosmix.kosmosfs.access.KfsAccess;
+import org.kosmix.kosmosfs.access.KfsInputChannel;
+
+class KFSInputStream extends FSInputStream {
+
+    private KfsInputChannel kfsChannel;
+    private FileSystem.Statistics statistics;
+    private long fsize;
+
+    @Deprecated
+    public KFSInputStream(KfsAccess kfsAccess, String path) {
+      this(kfsAccess, path, null);
+    }
+
+    public KFSInputStream(KfsAccess kfsAccess, String path,
+                            FileSystem.Statistics stats) {
+        this.statistics = stats;
+        this.kfsChannel = kfsAccess.kfs_open(path);
+        if (this.kfsChannel != null)
+            this.fsize = kfsAccess.kfs_filesize(path);
+        else
+            this.fsize = 0;
+    }
+
+    public long getPos() throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        return kfsChannel.tell();
+    }
+
+    public synchronized int available() throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        return (int) (this.fsize - getPos());
+    }
+
+    public synchronized void seek(long targetPos) throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        kfsChannel.seek(targetPos);
+    }
+
+    public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+        return false;
+    }
+
+    public synchronized int read() throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        byte b[] = new byte[1];
+        int res = read(b, 0, 1);
+        if (res == 1) {
+          if (statistics != null) {
+            statistics.incrementBytesRead(1);
+          }
+          return b[0] & 0xff;
+        }
+        return -1;
+    }
+
+    public synchronized int read(byte b[], int off, int len) throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+	int res;
+
+	res = kfsChannel.read(ByteBuffer.wrap(b, off, len));
+	// Use -1 to signify EOF
+	if (res == 0)
+	    return -1;
+	if (statistics != null) {
+	  statistics.incrementBytesRead(res);
+	}
+	return res;
+    }
+
+    public synchronized void close() throws IOException {
+        if (kfsChannel == null) {
+            return;
+        }
+
+        kfsChannel.close();
+        kfsChannel = null;
+    }
+
+    public boolean markSupported() {
+        return false;
+    }
+
+    public void mark(int readLimit) {
+        // Do nothing
+    }
+
+    public void reset() throws IOException {
+        throw new IOException("Mark not supported");
+    }
+
+}

+ 97 - 0
src/java/org/apache/hadoop/fs/kfs/KFSOutputStream.java

@@ -0,0 +1,97 @@
+/**
+ *
+ * Licensed under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ * implied. See the License for the specific language governing
+ * permissions and limitations under the License.
+ *
+ * @author: Sriram Rao (Kosmix Corp.)
+ * 
+ * Implements the Hadoop FSOutputStream interfaces to allow applications to write to
+ * files in Kosmos File System (KFS).
+ */
+
+package org.apache.hadoop.fs.kfs;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.util.Progressable;
+
+import org.kosmix.kosmosfs.access.KfsAccess;
+import org.kosmix.kosmosfs.access.KfsOutputChannel;
+
+class KFSOutputStream extends OutputStream {
+
+    private String path;
+    private KfsOutputChannel kfsChannel;
+    private Progressable progressReporter;
+
+    public KFSOutputStream(KfsAccess kfsAccess, String path, short replication,
+                           boolean append, Progressable prog) {
+        this.path = path;
+
+        if ((append) && (kfsAccess.kfs_isFile(path)))
+                this.kfsChannel = kfsAccess.kfs_append(path);
+        else
+                this.kfsChannel = kfsAccess.kfs_create(path, replication);
+        this.progressReporter = prog;
+    }
+
+    public long getPos() throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        return kfsChannel.tell();
+    }
+
+    public void write(int v) throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        byte[] b = new byte[1];
+
+        b[0] = (byte) v;
+        write(b, 0, 1);
+    }
+
+    public void write(byte b[], int off, int len) throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+
+        // touch the progress before going into KFS since the call can block
+        progressReporter.progress();
+        kfsChannel.write(ByteBuffer.wrap(b, off, len));
+    }
+
+    public void flush() throws IOException {
+        if (kfsChannel == null) {
+            throw new IOException("File closed");
+        }
+        // touch the progress before going into KFS since the call can block
+        progressReporter.progress();
+        kfsChannel.sync();
+    }
+
+    public synchronized void close() throws IOException {
+        if (kfsChannel == null) {
+            return;
+        }
+        flush();
+        kfsChannel.close();
+        kfsChannel = null;
+    }
+}

+ 340 - 0
src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java

@@ -0,0 +1,340 @@
+/**
+ *
+ * Licensed under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ * implied. See the License for the specific language governing
+ * permissions and limitations under the License.
+ *
+ * @author: Sriram Rao (Kosmix Corp.)
+ * 
+ * Implements the Hadoop FS interfaces to allow applications to store
+ *files in Kosmos File System (KFS).
+ */
+
+package org.apache.hadoop.fs.kfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * A FileSystem backed by KFS.
+ *
+ */
+
+public class KosmosFileSystem extends FileSystem {
+
+    private FileSystem localFs;
+    private IFSImpl kfsImpl = null;
+    private URI uri;
+    private Path workingDir = new Path("/");
+
+    public KosmosFileSystem() {
+
+    }
+
+    KosmosFileSystem(IFSImpl fsimpl) {
+        this.kfsImpl = fsimpl;
+    }
+
+    @Override
+    public URI getUri() {
+	return uri;
+    }
+
+    @Override
+    public void initialize(URI uri, Configuration conf) throws IOException {
+      super.initialize(uri, conf);
+      try {
+        if (kfsImpl == null) {
+          if (uri.getHost() == null) {
+            kfsImpl = new KFSImpl(conf.get("fs.kfs.metaServerHost", ""),
+                                  conf.getInt("fs.kfs.metaServerPort", -1),
+                                  statistics);
+          } else {
+            kfsImpl = new KFSImpl(uri.getHost(), uri.getPort(), statistics);
+          }
+        }
+
+        this.localFs = FileSystem.getLocal(conf);
+        this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+        this.workingDir = new Path("/user", System.getProperty("user.name")
+                                   ).makeQualified(this);
+        setConf(conf);
+
+      } catch (Exception e) {
+        e.printStackTrace();
+        System.out.println("Unable to initialize KFS");
+        System.exit(-1);
+      }
+    }
+
+    @Override
+    public Path getWorkingDirectory() {
+	return workingDir;
+    }
+
+    @Override
+    public void setWorkingDirectory(Path dir) {
+	workingDir = makeAbsolute(dir);
+    }
+
+    private Path makeAbsolute(Path path) {
+	if (path.isAbsolute()) {
+	    return path;
+	}
+	return new Path(workingDir, path);
+    }
+
+    @Override
+    public boolean mkdirs(Path path, FsPermission permission
+        ) throws IOException {
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+
+	int res;
+
+	// System.out.println("Calling mkdirs on: " + srep);
+
+	res = kfsImpl.mkdirs(srep);
+	
+	return res == 0;
+    }
+
+    @Override
+    @Deprecated
+    public boolean isDirectory(Path path) throws IOException {
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+
+	// System.out.println("Calling isdir on: " + srep);
+
+        return kfsImpl.isDirectory(srep);
+    }
+
+    @Override
+    @Deprecated
+    public boolean isFile(Path path) throws IOException {
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+        return kfsImpl.isFile(srep);
+    }
+
+    @Override
+    public FileStatus[] listStatus(Path path) throws IOException {
+        Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+
+        if (kfsImpl.isFile(srep))
+                return new FileStatus[] { getFileStatus(path) } ;
+
+        return kfsImpl.readdirplus(absolute);
+    }
+
+    @Override
+    public FileStatus getFileStatus(Path path) throws IOException {
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+        if (!kfsImpl.exists(srep)) {
+          throw new FileNotFoundException("File " + path + " does not exist.");
+        }
+        if (kfsImpl.isDirectory(srep)) {
+            // System.out.println("Status of path: " + path + " is dir");
+            return new FileStatus(0, true, 1, 0, kfsImpl.getModificationTime(srep), 
+                                  path.makeQualified(this));
+        } else {
+            // System.out.println("Status of path: " + path + " is file");
+            return new FileStatus(kfsImpl.filesize(srep), false, 
+                                  kfsImpl.getReplication(srep),
+                                  getDefaultBlockSize(),
+                                  kfsImpl.getModificationTime(srep),
+                                  path.makeQualified(this));
+        }
+    }
+    
+    @Override
+    public FSDataOutputStream append(Path f, int bufferSize,
+        Progressable progress) throws IOException {
+        Path parent = f.getParent();
+        if (parent != null && !mkdirs(parent)) {
+            throw new IOException("Mkdirs failed to create " + parent);
+        }
+
+        Path absolute = makeAbsolute(f);
+        String srep = absolute.toUri().getPath();
+
+        return kfsImpl.append(srep, bufferSize, progress);
+    }
+
+    @Override
+    public FSDataOutputStream create(Path file, FsPermission permission,
+                                     boolean overwrite, int bufferSize,
+				     short replication, long blockSize, Progressable progress)
+	throws IOException {
+
+        if (exists(file)) {
+            if (overwrite) {
+                delete(file, true);
+            } else {
+                throw new IOException("File already exists: " + file);
+            }
+        }
+
+	Path parent = file.getParent();
+	if (parent != null && !mkdirs(parent)) {
+	    throw new IOException("Mkdirs failed to create " + parent);
+	}
+
+        Path absolute = makeAbsolute(file);
+        String srep = absolute.toUri().getPath();
+
+        return kfsImpl.create(srep, replication, bufferSize, progress);
+    }
+
+    @Override
+    public FSDataInputStream open(Path path, int bufferSize) throws IOException {
+        if (!exists(path))
+            throw new IOException("File does not exist: " + path);
+
+        Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+
+        return kfsImpl.open(srep, bufferSize);
+    }
+
+    @Override
+    public boolean rename(Path src, Path dst) throws IOException {
+	Path absoluteS = makeAbsolute(src);
+        String srepS = absoluteS.toUri().getPath();
+	Path absoluteD = makeAbsolute(dst);
+        String srepD = absoluteD.toUri().getPath();
+
+        // System.out.println("Calling rename on: " + srepS + " -> " + srepD);
+
+        return kfsImpl.rename(srepS, srepD) == 0;
+    }
+
+    // recursively delete the directory and its contents
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+      Path absolute = makeAbsolute(path);
+      String srep = absolute.toUri().getPath();
+      if (kfsImpl.isFile(srep))
+        return kfsImpl.remove(srep) == 0;
+
+      FileStatus[] dirEntries = listStatus(absolute);
+      if ((!recursive) && (dirEntries != null) && 
+            (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);
+        }
+      }
+      return kfsImpl.rmdir(srep) == 0;
+    }
+    
+    @Override
+    public short getDefaultReplication() {
+	return 3;
+    }
+
+    @Override
+    public boolean setReplication(Path path, short replication)
+	throws IOException {
+
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+
+        int res = kfsImpl.setReplication(srep, replication);
+        return res >= 0;
+    }
+
+    // 64MB is the KFS block size
+
+    @Override
+    public long getDefaultBlockSize() {
+	return 1 << 26;
+    }
+
+    @Deprecated            
+    public void lock(Path path, boolean shared) throws IOException {
+
+    }
+
+    @Deprecated            
+    public void release(Path path) throws IOException {
+
+    }
+
+    /**
+     * Return null if the file doesn't exist; otherwise, get the
+     * locations of the various chunks of the file file from KFS.
+     */
+    @Override
+    public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+        long len) throws IOException {
+
+      if (file == null) {
+        return null;
+      }
+      String srep = makeAbsolute(file.getPath()).toUri().getPath();
+      String[][] hints = kfsImpl.getDataLocation(srep, start, len);
+      if (hints == null) {
+        return null;
+      }
+      BlockLocation[] result = new BlockLocation[hints.length];
+      long blockSize = getDefaultBlockSize();
+      long length = len;
+      long blockStart = start;
+      for(int i=0; i < result.length; ++i) {
+        result[i] = new BlockLocation(null, hints[i], blockStart, 
+                                      length < blockSize ? length : blockSize);
+        blockStart += blockSize;
+        length -= blockSize;
+      }
+      return result;
+    }
+
+    @Override
+    public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
+	FileUtil.copy(localFs, src, this, dst, delSrc, getConf());
+    }
+
+    @Override
+    public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
+	FileUtil.copy(this, src, localFs, dst, delSrc, getConf());
+    }
+
+    @Override
+    public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+	throws IOException {
+	return tmpLocalFile;
+    }
+
+    @Override
+    public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+	throws IOException {
+	moveFromLocalFile(tmpLocalFile, fsOutputFile);
+    }
+}

+ 98 - 0
src/java/org/apache/hadoop/fs/kfs/package.html

@@ -0,0 +1,98 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<head></head>
+<body>
+<h1>A client for the Kosmos filesystem (KFS)</h1>
+
+<h3>Introduction</h3>
+
+This pages describes how to use Kosmos Filesystem 
+(<a href="http://kosmosfs.sourceforge.net"> KFS </a>) as a backing
+store with Hadoop.   This page assumes that you have downloaded the
+KFS software and installed necessary binaries as outlined in the KFS
+documentation.
+
+<h3>Steps</h3>
+
+        <ul>
+          <li>In the Hadoop conf directory edit core-site.xml,
+          add the following:
+            <pre>
+&lt;property&gt;
+  &lt;name&gt;fs.kfs.impl&lt;/name&gt;
+  &lt;value&gt;org.apache.hadoop.fs.kfs.KosmosFileSystem&lt;/value&gt;
+  &lt;description&gt;The FileSystem for kfs: uris.&lt;/description&gt;
+&lt;/property&gt;
+            </pre>
+
+          <li>In the Hadoop conf directory edit core-site.xml,
+          adding the following (with appropriate values for
+          &lt;server&gt; and &lt;port&gt;):
+            <pre>
+&lt;property&gt;
+  &lt;name&gt;fs.default.name&lt;/name&gt;
+  &lt;value&gt;kfs://&lt;server:port&gt;&lt;/value&gt; 
+&lt;/property&gt;
+
+&lt;property&gt;
+  &lt;name&gt;fs.kfs.metaServerHost&lt;/name&gt;
+  &lt;value&gt;&lt;server&gt;&lt;/value&gt;
+  &lt;description&gt;The location of the KFS meta server.&lt;/description&gt;
+&lt;/property&gt;
+
+&lt;property&gt;
+  &lt;name&gt;fs.kfs.metaServerPort&lt;/name&gt;
+  &lt;value&gt;&lt;port&gt;&lt;/value&gt;
+  &lt;description&gt;The location of the meta server's port.&lt;/description&gt;
+&lt;/property&gt;
+
+</pre>
+          </li>
+
+          <li>Copy KFS's <i> kfs-0.1.jar </i> to Hadoop's lib directory.  This step
+          enables Hadoop's to load the KFS specific modules.  Note
+          that, kfs-0.1.jar was built when you compiled KFS source
+          code.  This jar file contains code that calls KFS's client
+          library code via JNI; the native code is in KFS's <i>
+          libkfsClient.so </i> library.
+          </li>
+
+          <li> When the Hadoop map/reduce trackers start up, those
+processes (on local as well as remote nodes) will now need to load
+KFS's <i> libkfsClient.so </i> library.  To simplify this process, it is advisable to
+store libkfsClient.so in an NFS accessible directory (similar to where
+Hadoop binaries/scripts are stored); then, modify Hadoop's
+conf/hadoop-env.sh adding the following line and providing suitable
+value for &lt;path&gt;:
+<pre>
+export LD_LIBRARY_PATH=&lt;path&gt;
+</pre>
+
+
+          <li>Start only the map/reduce trackers
+          <br />
+          example: execute Hadoop's bin/start-mapred.sh</li>
+        </ul>
+<br/>
+
+If the map/reduce job trackers start up, all file-I/O is done to KFS.
+
+</body>
+</html>

+ 23 - 0
src/java/org/apache/hadoop/fs/package.html

@@ -0,0 +1,23 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+An abstract file system API.
+</body>
+</html>

+ 61 - 0
src/java/org/apache/hadoop/fs/permission/AccessControlException.java

@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+import java.io.IOException;
+
+/**
+ * An exception class for access control related issues.
+ * @deprecated Use {@link org.apache.hadoop.security.AccessControlException} 
+ *             instead.
+ */
+@Deprecated
+public class AccessControlException extends IOException {
+  //Required by {@link java.io.Serializable}.
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * Default constructor is needed for unwrapping from 
+   * {@link org.apache.hadoop.ipc.RemoteException}.
+   */
+  public AccessControlException() {
+    super("Permission denied.");
+  }
+
+  /**
+   * Constructs an {@link AccessControlException}
+   * with the specified detail message.
+   * @param s the detail message.
+   */
+  public AccessControlException(String s) {
+    super(s);
+  }
+  
+  /**
+   * Constructs a new exception with the specified cause and a detail
+   * message of <tt>(cause==null ? null : cause.toString())</tt> (which
+   * typically contains the class and detail message of <tt>cause</tt>).
+   * @param  cause the cause (which is saved for later retrieval by the
+   *         {@link #getCause()} method).  (A <tt>null</tt> value is
+   *         permitted, and indicates that the cause is nonexistent or
+   *         unknown.)
+   */
+  public AccessControlException(Throwable cause) {
+    super(cause);
+  }
+}

+ 67 - 0
src/java/org/apache/hadoop/fs/permission/FsAction.java

@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+/**
+ * File system actions, e.g. read, write, etc.
+ */
+public enum FsAction {
+  // POSIX style
+  NONE("---"),
+  EXECUTE("--x"),
+  WRITE("-w-"),
+  WRITE_EXECUTE("-wx"),
+  READ("r--"),
+  READ_EXECUTE("r-x"),
+  READ_WRITE("rw-"),
+  ALL("rwx");
+
+  /** Retain reference to value array. */
+  private final static FsAction[] vals = values();
+
+  /** Symbolic representation */
+  public final String SYMBOL;
+
+  private FsAction(String s) {
+    SYMBOL = s;
+  }
+
+  /**
+   * Return true if this action implies that action.
+   * @param that
+   */
+  public boolean implies(FsAction that) {
+    if (that != null) {
+      return (ordinal() & that.ordinal()) == that.ordinal();
+    }
+    return false;
+  }
+
+  /** AND operation. */
+  public FsAction and(FsAction that) {
+    return vals[ordinal() & that.ordinal()];
+  }
+  /** OR operation. */
+  public FsAction or(FsAction that) {
+    return vals[ordinal() | that.ordinal()];
+  }
+  /** NOT operation. */
+  public FsAction not() {
+    return vals[7 - ordinal()];
+  }
+}

+ 232 - 0
src/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.*;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * A class for file/directory permissions.
+ */
+public class FsPermission implements Writable {
+  static final WritableFactory FACTORY = new WritableFactory() {
+    public Writable newInstance() { return new FsPermission(); }
+  };
+  static {                                      // register a ctor
+    WritableFactories.setFactory(FsPermission.class, FACTORY);
+  }
+
+  /** Create an immutable {@link FsPermission} object. */
+  public static FsPermission createImmutable(short permission) {
+    return new FsPermission(permission) {
+      public FsPermission applyUMask(FsPermission umask) {
+        throw new UnsupportedOperationException();
+      }
+      public void readFields(DataInput in) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  //POSIX permission style
+  private FsAction useraction = null;
+  private FsAction groupaction = null;
+  private FsAction otheraction = null;
+  private boolean stickyBit = false;
+
+  private FsPermission() {}
+
+  /**
+   * Construct by the given {@link FsAction}.
+   * @param u user action
+   * @param g group action
+   * @param o other action
+   */
+  public FsPermission(FsAction u, FsAction g, FsAction o) {
+    this(u, g, o, false);
+  }
+
+  public FsPermission(FsAction u, FsAction g, FsAction o, boolean sb) {
+    set(u, g, o, sb);
+  }
+
+  /**
+   * Construct by the given mode.
+   * @param mode
+   * @see #toShort()
+   */
+  public FsPermission(short mode) { fromShort(mode); }
+
+  /**
+   * Copy constructor
+   * 
+   * @param other other permission
+   */
+  public FsPermission(FsPermission other) {
+    this.useraction = other.useraction;
+    this.groupaction = other.groupaction;
+    this.otheraction = other.otheraction;
+  }
+  
+  /** Return user {@link FsAction}. */
+  public FsAction getUserAction() {return useraction;}
+
+  /** Return group {@link FsAction}. */
+  public FsAction getGroupAction() {return groupaction;}
+
+  /** Return other {@link FsAction}. */
+  public FsAction getOtherAction() {return otheraction;}
+
+  private void set(FsAction u, FsAction g, FsAction o, boolean sb) {
+    useraction = u;
+    groupaction = g;
+    otheraction = o;
+    stickyBit = sb;
+  }
+
+  public void fromShort(short n) {
+    FsAction[] v = FsAction.values();
+
+    set(v[(n >>> 6) & 7], v[(n >>> 3) & 7], v[n & 7], (((n >>> 9) & 1) == 1) );
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    out.writeShort(toShort());
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    fromShort(in.readShort());
+  }
+
+  /**
+   * Create and initialize a {@link FsPermission} from {@link DataInput}.
+   */
+  public static FsPermission read(DataInput in) throws IOException {
+    FsPermission p = new FsPermission();
+    p.readFields(in);
+    return p;
+  }
+
+  /**
+   * Encode the object to a short.
+   */
+  public short toShort() {
+    int s =  (stickyBit ? 1 << 9 : 0)     |
+             (useraction.ordinal() << 6)  |
+             (groupaction.ordinal() << 3) |
+             otheraction.ordinal();
+
+    return (short)s;
+  }
+
+  /** {@inheritDoc} */
+  public boolean equals(Object obj) {
+    if (obj instanceof FsPermission) {
+      FsPermission that = (FsPermission)obj;
+      return this.useraction == that.useraction
+          && this.groupaction == that.groupaction
+          && this.otheraction == that.otheraction
+          && this.stickyBit == that.stickyBit;
+    }
+    return false;
+  }
+
+  /** {@inheritDoc} */
+  public int hashCode() {return toShort();}
+
+  /** {@inheritDoc} */
+  public String toString() {
+    String str = useraction.SYMBOL + groupaction.SYMBOL + otheraction.SYMBOL;
+    if(stickyBit) {
+      StringBuilder str2 = new StringBuilder(str);
+      str2.replace(str2.length() - 1, str2.length(),
+           otheraction.implies(FsAction.EXECUTE) ? "t" : "T");
+      str = str2.toString();
+    }
+
+    return str;
+  }
+
+  /** Apply a umask to this permission and return a new one */
+  public FsPermission applyUMask(FsPermission umask) {
+    return new FsPermission(useraction.and(umask.useraction.not()),
+        groupaction.and(umask.groupaction.not()),
+        otheraction.and(umask.otheraction.not()));
+  }
+
+  /** umask property label */
+  public static final String UMASK_LABEL = "dfs.umask";
+  public static final int DEFAULT_UMASK = 0022;
+
+  /** Get the user file creation mask (umask) */
+  public static FsPermission getUMask(Configuration conf) {
+    int umask = DEFAULT_UMASK;
+    if (conf != null) {
+      umask = conf.getInt(UMASK_LABEL, DEFAULT_UMASK);
+    }
+    return new FsPermission((short)umask);
+  }
+
+  public boolean getStickyBit() {
+    return stickyBit;
+  }
+
+  /** Set the user file creation mask (umask) */
+  public static void setUMask(Configuration conf, FsPermission umask) {
+    conf.setInt(UMASK_LABEL, umask.toShort());
+  }
+
+  /** Get the default permission. */
+  public static FsPermission getDefault() {
+    return new FsPermission((short)00777);
+  }
+
+  /**
+   * Create a FsPermission from a Unix symbolic permission string
+   * @param unixSymbolicPermission e.g. "-rw-rw-rw-"
+   */
+  public static FsPermission valueOf(String unixSymbolicPermission) {
+    if (unixSymbolicPermission == null) {
+      return null;
+    }
+    else if (unixSymbolicPermission.length() != 10) {
+      throw new IllegalArgumentException("length != 10(unixSymbolicPermission="
+          + unixSymbolicPermission + ")");
+    }
+
+    int n = 0;
+    for(int i = 1; i < unixSymbolicPermission.length(); i++) {
+      n = n << 1;
+      char c = unixSymbolicPermission.charAt(i);
+      n += (c == '-' || c == 'T' || c == 'S') ? 0: 1;
+    }
+
+    // Add sticky bit value if set
+    if(unixSymbolicPermission.charAt(9) == 't' ||
+        unixSymbolicPermission.charAt(9) == 'T')
+      n += 01000;
+
+    return new FsPermission((short)n);
+  }
+}

+ 118 - 0
src/java/org/apache/hadoop/fs/permission/PermissionStatus.java

@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+import org.apache.hadoop.io.*;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Store permission related information.
+ */
+public class PermissionStatus implements Writable {
+  static final WritableFactory FACTORY = new WritableFactory() {
+    public Writable newInstance() { return new PermissionStatus(); }
+  };
+  static {                                      // register a ctor
+    WritableFactories.setFactory(PermissionStatus.class, FACTORY);
+  }
+
+  /** Create an immutable {@link PermissionStatus} object. */
+  public static PermissionStatus createImmutable(
+      String user, String group, FsPermission permission) {
+    return new PermissionStatus(user, group, permission) {
+      public PermissionStatus applyUMask(FsPermission umask) {
+        throw new UnsupportedOperationException();
+      }
+      public void readFields(DataInput in) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  private String username;
+  private String groupname;
+  private FsPermission permission;
+
+  private PermissionStatus() {}
+
+  /** Constructor */
+  public PermissionStatus(String user, String group, FsPermission permission) {
+    username = user;
+    groupname = group;
+    this.permission = permission;
+  }
+
+  /** Return user name */
+  public String getUserName() {return username;}
+
+  /** Return group name */
+  public String getGroupName() {return groupname;}
+
+  /** Return permission */
+  public FsPermission getPermission() {return permission;}
+
+  /**
+   * Apply umask.
+   * @see FsPermission#applyUMask(FsPermission)
+   */
+  public PermissionStatus applyUMask(FsPermission umask) {
+    permission = permission.applyUMask(umask);
+    return this;
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    username = Text.readString(in);
+    groupname = Text.readString(in);
+    permission = FsPermission.read(in);
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    write(out, username, groupname, permission);
+  }
+
+  /**
+   * Create and initialize a {@link PermissionStatus} from {@link DataInput}.
+   */
+  public static PermissionStatus read(DataInput in) throws IOException {
+    PermissionStatus p = new PermissionStatus();
+    p.readFields(in);
+    return p;
+  }
+
+  /**
+   * Serialize a {@link PermissionStatus} from its base components.
+   */
+  public static void write(DataOutput out,
+                           String username, 
+                           String groupname,
+                           FsPermission permission) throws IOException {
+    Text.writeString(out, username);
+    Text.writeString(out, groupname);
+    permission.write(out);
+  }
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return username + ":" + groupname + ":" + permission;
+  }
+}

+ 47 - 0
src/java/org/apache/hadoop/fs/s3/Block.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+/**
+ * Holds metadata about a block of data being stored in a {@link FileSystemStore}.
+ */
+public class Block {
+  private long id;
+
+  private long length;
+
+  public Block(long id, long length) {
+    this.id = id;
+    this.length = length;
+  }
+
+  public long getId() {
+    return id;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  @Override
+  public String toString() {
+    return "Block[" + id + ", " + length + "]";
+  }
+
+}

+ 63 - 0
src/java/org/apache/hadoop/fs/s3/FileSystemStore.java

@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A facility for storing and retrieving {@link INode}s and {@link Block}s.
+ */
+public interface FileSystemStore {
+  
+  void initialize(URI uri, Configuration conf) throws IOException;
+  String getVersion() throws IOException;
+
+  void storeINode(Path path, INode inode) throws IOException;
+  void storeBlock(Block block, File file) throws IOException;
+  
+  boolean inodeExists(Path path) throws IOException;
+  boolean blockExists(long blockId) throws IOException;
+
+  INode retrieveINode(Path path) throws IOException;
+  File retrieveBlock(Block block, long byteRangeStart) throws IOException;
+
+  void deleteINode(Path path) throws IOException;
+  void deleteBlock(Block block) throws IOException;
+
+  Set<Path> listSubPaths(Path path) throws IOException;
+  Set<Path> listDeepSubPaths(Path path) throws IOException;
+
+  /**
+   * Delete everything. Used for testing.
+   * @throws IOException
+   */
+  void purge() throws IOException;
+  
+  /**
+   * Diagnostic method to dump all INodes to the console.
+   * @throws IOException
+   */
+  void dump() throws IOException;
+}

+ 117 - 0
src/java/org/apache/hadoop/fs/s3/INode.java

@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Holds file metadata including type (regular file, or directory),
+ * and the list of blocks that are pointers to the data.
+ */
+public class INode {
+	
+  enum FileType {
+    DIRECTORY, FILE
+  }
+  
+  public static final FileType[] FILE_TYPES = {
+    FileType.DIRECTORY,
+    FileType.FILE
+  };
+
+  public static final INode DIRECTORY_INODE = new INode(FileType.DIRECTORY, null);
+  
+  private FileType fileType;
+  private Block[] blocks;
+
+  public INode(FileType fileType, Block[] blocks) {
+    this.fileType = fileType;
+    if (isDirectory() && blocks != null) {
+      throw new IllegalArgumentException("A directory cannot contain blocks.");
+    }
+    this.blocks = blocks;
+  }
+
+  public Block[] getBlocks() {
+    return blocks;
+  }
+  
+  public FileType getFileType() {
+    return fileType;
+  }
+
+  public boolean isDirectory() {
+    return fileType == FileType.DIRECTORY;
+  }  
+
+  public boolean isFile() {
+    return fileType == FileType.FILE;
+  }
+  
+  public long getSerializedLength() {
+    return 1L + (blocks == null ? 0 : 4 + blocks.length * 16);
+  }
+  
+
+  public InputStream serialize() throws IOException {
+    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(bytes);
+    out.writeByte(fileType.ordinal());
+    if (isFile()) {
+      out.writeInt(blocks.length);
+      for (int i = 0; i < blocks.length; i++) {
+        out.writeLong(blocks[i].getId());
+        out.writeLong(blocks[i].getLength());
+      }
+    }
+    out.close();
+    return new ByteArrayInputStream(bytes.toByteArray());
+  }
+  
+  public static INode deserialize(InputStream in) throws IOException {
+    if (in == null) {
+      return null;
+    }
+    DataInputStream dataIn = new DataInputStream(in);
+    FileType fileType = INode.FILE_TYPES[dataIn.readByte()];
+    switch (fileType) {
+    case DIRECTORY:
+      in.close();
+      return INode.DIRECTORY_INODE;
+    case FILE:
+      int numBlocks = dataIn.readInt();
+      Block[] blocks = new Block[numBlocks];
+      for (int i = 0; i < numBlocks; i++) {
+        long id = dataIn.readLong();
+        long length = dataIn.readLong();
+        blocks[i] = new Block(id, length);
+      }
+      in.close();
+      return new INode(fileType, blocks);
+    default:
+      throw new IllegalArgumentException("Cannot deserialize inode.");
+    }    
+  }  
+  
+}

+ 390 - 0
src/java/org/apache/hadoop/fs/s3/Jets3tFileSystemStore.java

@@ -0,0 +1,390 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3.INode.FileType;
+import org.jets3t.service.S3Service;
+import org.jets3t.service.S3ServiceException;
+import org.jets3t.service.impl.rest.httpclient.RestS3Service;
+import org.jets3t.service.model.S3Bucket;
+import org.jets3t.service.model.S3Object;
+import org.jets3t.service.security.AWSCredentials;
+
+class Jets3tFileSystemStore implements FileSystemStore {
+  
+  private static final String FILE_SYSTEM_NAME = "fs";
+  private static final String FILE_SYSTEM_VALUE = "Hadoop";
+
+  private static final String FILE_SYSTEM_TYPE_NAME = "fs-type";
+  private static final String FILE_SYSTEM_TYPE_VALUE = "block";
+
+  private static final String FILE_SYSTEM_VERSION_NAME = "fs-version";
+  private static final String FILE_SYSTEM_VERSION_VALUE = "1";
+  
+  private static final Map<String, String> METADATA =
+    new HashMap<String, String>();
+  
+  static {
+    METADATA.put(FILE_SYSTEM_NAME, FILE_SYSTEM_VALUE);
+    METADATA.put(FILE_SYSTEM_TYPE_NAME, FILE_SYSTEM_TYPE_VALUE);
+    METADATA.put(FILE_SYSTEM_VERSION_NAME, FILE_SYSTEM_VERSION_VALUE);
+  }
+
+  private static final String PATH_DELIMITER = Path.SEPARATOR;
+  private static final String BLOCK_PREFIX = "block_";
+
+  private Configuration conf;
+  
+  private S3Service s3Service;
+
+  private S3Bucket bucket;
+  
+  private int bufferSize;
+  
+  private static final Log LOG = 
+    LogFactory.getLog(Jets3tFileSystemStore.class.getName());
+  
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    
+    this.conf = conf;
+    
+    S3Credentials s3Credentials = new S3Credentials();
+    s3Credentials.initialize(uri, conf);
+    try {
+      AWSCredentials awsCredentials =
+        new AWSCredentials(s3Credentials.getAccessKey(),
+            s3Credentials.getSecretAccessKey());
+      this.s3Service = new RestS3Service(awsCredentials);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+    bucket = new S3Bucket(uri.getHost());
+
+    this.bufferSize = conf.getInt("io.file.buffer.size", 4096);
+  }
+
+  public String getVersion() throws IOException {
+    return FILE_SYSTEM_VERSION_VALUE;
+  }
+
+  private void delete(String key) throws IOException {
+    try {
+      s3Service.deleteObject(bucket, key);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public void deleteINode(Path path) throws IOException {
+    delete(pathToKey(path));
+  }
+
+  public void deleteBlock(Block block) throws IOException {
+    delete(blockToKey(block));
+  }
+
+  public boolean inodeExists(Path path) throws IOException {
+    InputStream in = get(pathToKey(path), true);
+    if (in == null) {
+      return false;
+    }
+    in.close();
+    return true;
+  }
+  
+  public boolean blockExists(long blockId) throws IOException {
+    InputStream in = get(blockToKey(blockId), false);
+    if (in == null) {
+      return false;
+    }
+    in.close();
+    return true;
+  }
+
+  private InputStream get(String key, boolean checkMetadata)
+      throws IOException {
+    
+    try {
+      S3Object object = s3Service.getObject(bucket, key);
+      if (checkMetadata) {
+        checkMetadata(object);
+      }
+      return object.getDataInputStream();
+    } catch (S3ServiceException e) {
+      if ("NoSuchKey".equals(e.getS3ErrorCode())) {
+        return null;
+      }
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  private InputStream get(String key, long byteRangeStart) throws IOException {
+    try {
+      S3Object object = s3Service.getObject(bucket, key, null, null, null,
+                                            null, byteRangeStart, null);
+      return object.getDataInputStream();
+    } catch (S3ServiceException e) {
+      if ("NoSuchKey".equals(e.getS3ErrorCode())) {
+        return null;
+      }
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  private void checkMetadata(S3Object object) throws S3FileSystemException,
+      S3ServiceException {
+    
+    String name = (String) object.getMetadata(FILE_SYSTEM_NAME);
+    if (!FILE_SYSTEM_VALUE.equals(name)) {
+      throw new S3FileSystemException("Not a Hadoop S3 file.");
+    }
+    String type = (String) object.getMetadata(FILE_SYSTEM_TYPE_NAME);
+    if (!FILE_SYSTEM_TYPE_VALUE.equals(type)) {
+      throw new S3FileSystemException("Not a block file.");
+    }
+    String dataVersion = (String) object.getMetadata(FILE_SYSTEM_VERSION_NAME);
+    if (!FILE_SYSTEM_VERSION_VALUE.equals(dataVersion)) {
+      throw new VersionMismatchException(FILE_SYSTEM_VERSION_VALUE,
+          dataVersion);
+    }
+  }
+
+  public INode retrieveINode(Path path) throws IOException {
+    return INode.deserialize(get(pathToKey(path), true));
+  }
+
+  public File retrieveBlock(Block block, long byteRangeStart)
+    throws IOException {
+    File fileBlock = null;
+    InputStream in = null;
+    OutputStream out = null;
+    try {
+      fileBlock = newBackupFile();
+      in = get(blockToKey(block), byteRangeStart);
+      out = new BufferedOutputStream(new FileOutputStream(fileBlock));
+      byte[] buf = new byte[bufferSize];
+      int numRead;
+      while ((numRead = in.read(buf)) >= 0) {
+        out.write(buf, 0, numRead);
+      }
+      return fileBlock;
+    } catch (IOException e) {
+      // close output stream to file then delete file
+      closeQuietly(out);
+      out = null; // to prevent a second close
+      if (fileBlock != null) {
+        boolean b = fileBlock.delete();
+        if (!b) {
+          LOG.warn("Ignoring failed delete");
+        }
+      }
+      throw e;
+    } finally {
+      closeQuietly(out);
+      closeQuietly(in);
+    }
+  }
+  
+  private File newBackupFile() throws IOException {
+    File dir = new File(conf.get("fs.s3.buffer.dir"));
+    if (!dir.exists() && !dir.mkdirs()) {
+      throw new IOException("Cannot create S3 buffer directory: " + dir);
+    }
+    File result = File.createTempFile("input-", ".tmp", dir);
+    result.deleteOnExit();
+    return result;
+  }
+
+  public Set<Path> listSubPaths(Path path) throws IOException {
+    try {
+      String prefix = pathToKey(path);
+      if (!prefix.endsWith(PATH_DELIMITER)) {
+        prefix += PATH_DELIMITER;
+      }
+      S3Object[] objects = s3Service.listObjects(bucket, prefix, PATH_DELIMITER);
+      Set<Path> prefixes = new TreeSet<Path>();
+      for (int i = 0; i < objects.length; i++) {
+        prefixes.add(keyToPath(objects[i].getKey()));
+      }
+      prefixes.remove(path);
+      return prefixes;
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+  
+  public Set<Path> listDeepSubPaths(Path path) throws IOException {
+    try {
+      String prefix = pathToKey(path);
+      if (!prefix.endsWith(PATH_DELIMITER)) {
+        prefix += PATH_DELIMITER;
+      }
+      S3Object[] objects = s3Service.listObjects(bucket, prefix, null);
+      Set<Path> prefixes = new TreeSet<Path>();
+      for (int i = 0; i < objects.length; i++) {
+        prefixes.add(keyToPath(objects[i].getKey()));
+      }
+      prefixes.remove(path);
+      return prefixes;
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }    
+  }
+
+  private void put(String key, InputStream in, long length, boolean storeMetadata)
+      throws IOException {
+    
+    try {
+      S3Object object = new S3Object(key);
+      object.setDataInputStream(in);
+      object.setContentType("binary/octet-stream");
+      object.setContentLength(length);
+      if (storeMetadata) {
+        object.addAllMetadata(METADATA);
+      }
+      s3Service.putObject(bucket, object);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public void storeINode(Path path, INode inode) throws IOException {
+    put(pathToKey(path), inode.serialize(), inode.getSerializedLength(), true);
+  }
+
+  public void storeBlock(Block block, File file) throws IOException {
+    BufferedInputStream in = null;
+    try {
+      in = new BufferedInputStream(new FileInputStream(file));
+      put(blockToKey(block), in, block.getLength(), false);
+    } finally {
+      closeQuietly(in);
+    }    
+  }
+
+  private void closeQuietly(Closeable closeable) {
+    if (closeable != null) {
+      try {
+        closeable.close();
+      } catch (IOException e) {
+        // ignore
+      }
+    }
+  }
+
+  private String pathToKey(Path path) {
+    if (!path.isAbsolute()) {
+      throw new IllegalArgumentException("Path must be absolute: " + path);
+    }
+    return path.toUri().getPath();
+  }
+
+  private Path keyToPath(String key) {
+    return new Path(key);
+  }
+  
+  private String blockToKey(long blockId) {
+    return BLOCK_PREFIX + blockId;
+  }
+
+  private String blockToKey(Block block) {
+    return blockToKey(block.getId());
+  }
+
+  public void purge() throws IOException {
+    try {
+      S3Object[] objects = s3Service.listObjects(bucket);
+      for (int i = 0; i < objects.length; i++) {
+        s3Service.deleteObject(bucket, objects[i].getKey());
+      }
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public void dump() throws IOException {
+    StringBuilder sb = new StringBuilder("S3 Filesystem, ");
+    sb.append(bucket.getName()).append("\n");
+    try {
+      S3Object[] objects = s3Service.listObjects(bucket, PATH_DELIMITER, null);
+      for (int i = 0; i < objects.length; i++) {
+        Path path = keyToPath(objects[i].getKey());
+        sb.append(path).append("\n");
+        INode m = retrieveINode(path);
+        sb.append("\t").append(m.getFileType()).append("\n");
+        if (m.getFileType() == FileType.DIRECTORY) {
+          continue;
+        }
+        for (int j = 0; j < m.getBlocks().length; j++) {
+          sb.append("\t").append(m.getBlocks()[j]).append("\n");
+        }
+      }
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+    System.out.println(sb);
+  }
+
+}

+ 280 - 0
src/java/org/apache/hadoop/fs/s3/MigrationTool.java

@@ -0,0 +1,280 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.s3;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.jets3t.service.S3Service;
+import org.jets3t.service.S3ServiceException;
+import org.jets3t.service.impl.rest.httpclient.RestS3Service;
+import org.jets3t.service.model.S3Bucket;
+import org.jets3t.service.model.S3Object;
+import org.jets3t.service.security.AWSCredentials;
+
+/**
+ * <p>
+ * This class is a tool for migrating data from an older to a newer version
+ * of an S3 filesystem.
+ * </p>
+ * <p>
+ * All files in the filesystem are migrated by re-writing the block metadata
+ * - no datafiles are touched.
+ * </p>
+ */
+public class MigrationTool extends Configured implements Tool {
+  
+  private S3Service s3Service;
+  private S3Bucket bucket;
+  
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new MigrationTool(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
+    
+    if (args.length == 0) {
+      System.err.println("Usage: MigrationTool <S3 file system URI>");
+      System.err.println("\t<S3 file system URI>\tfilesystem to migrate");
+      ToolRunner.printGenericCommandUsage(System.err);
+      return -1;
+    }
+    
+    URI uri = URI.create(args[0]);
+    
+    initialize(uri);
+    
+    FileSystemStore newStore = new Jets3tFileSystemStore();
+    newStore.initialize(uri, getConf());
+    
+    if (get("%2F") != null) { 
+      System.err.println("Current version number is [unversioned].");
+      System.err.println("Target version number is " +
+          newStore.getVersion() + ".");
+      Store oldStore = new UnversionedStore();
+      migrate(oldStore, newStore);
+      return 0;
+    } else {
+      S3Object root = get("/");
+      if (root != null) {
+        String version = (String) root.getMetadata("fs-version");
+        if (version == null) {
+          System.err.println("Can't detect version - exiting.");
+        } else {
+          String newVersion = newStore.getVersion();
+          System.err.println("Current version number is " + version + ".");
+          System.err.println("Target version number is " + newVersion + ".");
+          if (version.equals(newStore.getVersion())) {
+            System.err.println("No migration required.");
+            return 0;
+          }
+          // use version number to create Store
+          //Store oldStore = ... 
+          //migrate(oldStore, newStore);
+          System.err.println("Not currently implemented.");
+          return 0;
+        }
+      }
+      System.err.println("Can't detect version - exiting.");
+      return 0;
+    }
+    
+  }
+  
+  public void initialize(URI uri) throws IOException {
+    
+    
+    
+    try {
+      String accessKey = null;
+      String secretAccessKey = null;
+      String userInfo = uri.getUserInfo();
+      if (userInfo != null) {
+        int index = userInfo.indexOf(':');
+        if (index != -1) {
+          accessKey = userInfo.substring(0, index);
+          secretAccessKey = userInfo.substring(index + 1);
+        } else {
+          accessKey = userInfo;
+        }
+      }
+      if (accessKey == null) {
+        accessKey = getConf().get("fs.s3.awsAccessKeyId");
+      }
+      if (secretAccessKey == null) {
+        secretAccessKey = getConf().get("fs.s3.awsSecretAccessKey");
+      }
+      if (accessKey == null && secretAccessKey == null) {
+        throw new IllegalArgumentException("AWS " +
+                                           "Access Key ID and Secret Access Key " +
+                                           "must be specified as the username " +
+                                           "or password (respectively) of a s3 URL, " +
+                                           "or by setting the " +
+                                           "fs.s3.awsAccessKeyId or " +                         
+                                           "fs.s3.awsSecretAccessKey properties (respectively).");
+      } else if (accessKey == null) {
+        throw new IllegalArgumentException("AWS " +
+                                           "Access Key ID must be specified " +
+                                           "as the username of a s3 URL, or by setting the " +
+                                           "fs.s3.awsAccessKeyId property.");
+      } else if (secretAccessKey == null) {
+        throw new IllegalArgumentException("AWS " +
+                                           "Secret Access Key must be specified " +
+                                           "as the password of a s3 URL, or by setting the " +
+                                           "fs.s3.awsSecretAccessKey property.");         
+      }
+      AWSCredentials awsCredentials =
+        new AWSCredentials(accessKey, secretAccessKey);
+      this.s3Service = new RestS3Service(awsCredentials);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+    bucket = new S3Bucket(uri.getHost());
+  }
+  
+  private void migrate(Store oldStore, FileSystemStore newStore)
+      throws IOException {
+    for (Path path : oldStore.listAllPaths()) {
+      INode inode = oldStore.retrieveINode(path);
+      oldStore.deleteINode(path);
+      newStore.storeINode(path, inode);
+    }
+  }
+  
+  private S3Object get(String key) {
+    try {
+      return s3Service.getObject(bucket, key);
+    } catch (S3ServiceException e) {
+      if ("NoSuchKey".equals(e.getS3ErrorCode())) {
+        return null;
+      }
+    }
+    return null;
+  }
+  
+  interface Store {
+
+    Set<Path> listAllPaths() throws IOException;
+    INode retrieveINode(Path path) throws IOException;
+    void deleteINode(Path path) throws IOException;
+    
+  }
+  
+  class UnversionedStore implements Store {
+
+    public Set<Path> listAllPaths() throws IOException {
+      try {
+        String prefix = urlEncode(Path.SEPARATOR);
+        S3Object[] objects = s3Service.listObjects(bucket, prefix, null);
+        Set<Path> prefixes = new TreeSet<Path>();
+        for (int i = 0; i < objects.length; i++) {
+          prefixes.add(keyToPath(objects[i].getKey()));
+        }
+        return prefixes;
+      } catch (S3ServiceException e) {
+        if (e.getCause() instanceof IOException) {
+          throw (IOException) e.getCause();
+        }
+        throw new S3Exception(e);
+      }   
+    }
+
+    public void deleteINode(Path path) throws IOException {
+      delete(pathToKey(path));
+    }
+    
+    private void delete(String key) throws IOException {
+      try {
+        s3Service.deleteObject(bucket, key);
+      } catch (S3ServiceException e) {
+        if (e.getCause() instanceof IOException) {
+          throw (IOException) e.getCause();
+        }
+        throw new S3Exception(e);
+      }
+    }
+    
+    public INode retrieveINode(Path path) throws IOException {
+      return INode.deserialize(get(pathToKey(path)));
+    }
+
+    private InputStream get(String key) throws IOException {
+      try {
+        S3Object object = s3Service.getObject(bucket, key);
+        return object.getDataInputStream();
+      } catch (S3ServiceException e) {
+        if ("NoSuchKey".equals(e.getS3ErrorCode())) {
+          return null;
+        }
+        if (e.getCause() instanceof IOException) {
+          throw (IOException) e.getCause();
+        }
+        throw new S3Exception(e);
+      }
+    }
+    
+    private String pathToKey(Path path) {
+      if (!path.isAbsolute()) {
+        throw new IllegalArgumentException("Path must be absolute: " + path);
+      }
+      return urlEncode(path.toUri().getPath());
+    }
+    
+    private Path keyToPath(String key) {
+      return new Path(urlDecode(key));
+    }
+
+    private String urlEncode(String s) {
+      try {
+        return URLEncoder.encode(s, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        // Should never happen since every implementation of the Java Platform
+        // is required to support UTF-8.
+        // See http://java.sun.com/j2se/1.5.0/docs/api/java/nio/charset/Charset.html
+        throw new IllegalStateException(e);
+      }
+    }
+    
+    private String urlDecode(String s) {
+      try {
+        return URLDecoder.decode(s, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        // Should never happen since every implementation of the Java Platform
+        // is required to support UTF-8.
+        // See http://java.sun.com/j2se/1.5.0/docs/api/java/nio/charset/Charset.html
+        throw new IllegalStateException(e);
+      }
+    }
+    
+  }
+  
+}

+ 99 - 0
src/java/org/apache/hadoop/fs/s3/S3Credentials.java

@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * <p>
+ * Extracts AWS credentials from the filesystem URI or configuration.
+ * </p>
+ */
+public class S3Credentials {
+  
+  private String accessKey;
+  private String secretAccessKey; 
+
+  /**
+   * @throws IllegalArgumentException if credentials for S3 cannot be
+   * determined.
+   */
+  public void initialize(URI uri, Configuration conf) {
+    if (uri.getHost() == null) {
+      throw new IllegalArgumentException("Invalid hostname in URI " + uri);
+    }
+    
+    String userInfo = uri.getUserInfo();
+    if (userInfo != null) {
+      int index = userInfo.indexOf(':');
+      if (index != -1) {
+        accessKey = userInfo.substring(0, index);
+        secretAccessKey = userInfo.substring(index + 1);
+      } else {
+        accessKey = userInfo;
+      }
+    }
+    
+    String scheme = uri.getScheme();
+    String accessKeyProperty = String.format("fs.%s.awsAccessKeyId", scheme);
+    String secretAccessKeyProperty =
+      String.format("fs.%s.awsSecretAccessKey", scheme);
+    if (accessKey == null) {
+      accessKey = conf.get(accessKeyProperty);
+    }
+    if (secretAccessKey == null) {
+      secretAccessKey = conf.get(secretAccessKeyProperty);
+    }
+    if (accessKey == null && secretAccessKey == null) {
+      throw new IllegalArgumentException("AWS " +
+                                         "Access Key ID and Secret Access " +
+                                         "Key must be specified as the " +
+                                         "username or password " +
+                                         "(respectively) of a " + scheme +
+                                         " URL, or by setting the " +
+                                         accessKeyProperty + " or " +
+                                         secretAccessKeyProperty +
+                                         " properties (respectively).");
+    } else if (accessKey == null) {
+      throw new IllegalArgumentException("AWS " +
+                                         "Access Key ID must be specified " +
+                                         "as the username of a " + scheme +
+                                         " URL, or by setting the " +
+                                         accessKeyProperty + " property.");
+    } else if (secretAccessKey == null) {
+      throw new IllegalArgumentException("AWS " +
+                                         "Secret Access Key must be " +
+                                         "specified as the password of a " +
+                                         scheme + " URL, or by setting the " +
+                                         secretAccessKeyProperty +
+                                         " property.");       
+    }
+
+  }
+  
+  public String getAccessKey() {
+    return accessKey;
+  }
+  
+  public String getSecretAccessKey() {
+    return secretAccessKey;
+  }
+}

+ 34 - 0
src/java/org/apache/hadoop/fs/s3/S3Exception.java

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.IOException;
+
+/**
+ * Thrown if there is a problem communicating with Amazon S3.
+ */
+public class S3Exception extends IOException {
+
+  private static final long serialVersionUID = 1L;
+
+  public S3Exception(Throwable t) {
+    super(t);
+  }
+
+}

+ 361 - 0
src/java/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3native.NativeS3FileSystem;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * <p>
+ * A block-based {@link FileSystem} backed by
+ * <a href="http://aws.amazon.com/s3">Amazon S3</a>.
+ * </p>
+ * @see NativeS3FileSystem
+ */
+public class S3FileSystem extends FileSystem {
+
+  private URI uri;
+
+  private FileSystemStore store;
+
+  private Path workingDir;
+
+  public S3FileSystem() {
+    // set store in initialize()
+  }
+  
+  public S3FileSystem(FileSystemStore store) {
+    this.store = store;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    if (store == null) {
+      store = createDefaultStore(conf);
+    }
+    store.initialize(uri, conf);
+    setConf(conf);
+    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());    
+    this.workingDir =
+      new Path("/user", System.getProperty("user.name")).makeQualified(this);
+  }  
+
+  private static FileSystemStore createDefaultStore(Configuration conf) {
+    FileSystemStore store = new Jets3tFileSystemStore();
+    
+    RetryPolicy basePolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
+                                                                               conf.getInt("fs.s3.maxRetries", 4),
+                                                                               conf.getLong("fs.s3.sleepTimeSeconds", 10), TimeUnit.SECONDS);
+    Map<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
+      new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(IOException.class, basePolicy);
+    exceptionToPolicyMap.put(S3Exception.class, basePolicy);
+    
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+                                                              RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String,RetryPolicy> methodNameToPolicyMap = new HashMap<String,RetryPolicy>();
+    methodNameToPolicyMap.put("storeBlock", methodPolicy);
+    methodNameToPolicyMap.put("retrieveBlock", methodPolicy);
+    
+    return (FileSystemStore) RetryProxy.create(FileSystemStore.class,
+                                               store, methodNameToPolicyMap);
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public void setWorkingDirectory(Path dir) {
+    workingDir = makeAbsolute(dir);
+  }
+
+  private Path makeAbsolute(Path path) {
+    if (path.isAbsolute()) {
+      return path;
+    }
+    return new Path(workingDir, path);
+  }
+
+  /**
+   * @param permission Currently ignored.
+   */
+  @Override
+  public boolean mkdirs(Path path, FsPermission permission) throws IOException {
+    Path absolutePath = makeAbsolute(path);
+    List<Path> paths = new ArrayList<Path>();
+    do {
+      paths.add(0, absolutePath);
+      absolutePath = absolutePath.getParent();
+    } while (absolutePath != null);
+    
+    boolean result = true;
+    for (Path p : paths) {
+      result &= mkdir(p);
+    }
+    return result;
+  }
+  
+  private boolean mkdir(Path path) throws IOException {
+    Path absolutePath = makeAbsolute(path);
+    INode inode = store.retrieveINode(absolutePath);
+    if (inode == null) {
+      store.storeINode(absolutePath, INode.DIRECTORY_INODE);
+    } else if (inode.isFile()) {
+      throw new IOException(String.format(
+          "Can't make directory for path %s since it is a file.",
+          absolutePath));
+    }
+    return true;
+  }
+
+  @Override
+  public boolean isFile(Path path) throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(path));
+    if (inode == null) {
+      return false;
+    }
+    return inode.isFile();
+  }
+
+  private INode checkFile(Path path) throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(path));
+    if (inode == null) {
+      throw new IOException("No such file.");
+    }
+    if (inode.isDirectory()) {
+      throw new IOException("Path " + path + " is a directory.");
+    }
+    return inode;
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    Path absolutePath = makeAbsolute(f);
+    INode inode = store.retrieveINode(absolutePath);
+    if (inode == null) {
+      return null;
+    }
+    if (inode.isFile()) {
+      return new FileStatus[] {
+        new S3FileStatus(f.makeQualified(this), inode)
+      };
+    }
+    ArrayList<FileStatus> ret = new ArrayList<FileStatus>();
+    for (Path p : store.listSubPaths(absolutePath)) {
+      ret.add(getFileStatus(p.makeQualified(this)));
+    }
+    return ret.toArray(new FileStatus[0]);
+  }
+
+  /** This optional operation is not yet supported. */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Not supported");
+  }
+
+  /**
+   * @param permission Currently ignored.
+   */
+  @Override
+  public FSDataOutputStream create(Path file, FsPermission permission,
+      boolean overwrite, int bufferSize,
+      short replication, long blockSize, Progressable progress)
+    throws IOException {
+
+    INode inode = store.retrieveINode(makeAbsolute(file));
+    if (inode != null) {
+      if (overwrite) {
+        delete(file, true);
+      } else {
+        throw new IOException("File already exists: " + file);
+      }
+    } else {
+      Path parent = file.getParent();
+      if (parent != null) {
+        if (!mkdirs(parent)) {
+          throw new IOException("Mkdirs failed to create " + parent.toString());
+        }
+      }      
+    }
+    return new FSDataOutputStream
+        (new S3OutputStream(getConf(), store, makeAbsolute(file),
+                            blockSize, progress, bufferSize),
+         statistics);
+  }
+
+  @Override
+  public FSDataInputStream open(Path path, int bufferSize) throws IOException {
+    INode inode = checkFile(path);
+    return new FSDataInputStream(new S3InputStream(getConf(), store, inode,
+                                                   statistics));
+  }
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    Path absoluteSrc = makeAbsolute(src);
+    INode srcINode = store.retrieveINode(absoluteSrc);
+    if (srcINode == null) {
+      // src path doesn't exist
+      return false; 
+    }
+    Path absoluteDst = makeAbsolute(dst);
+    INode dstINode = store.retrieveINode(absoluteDst);
+    if (dstINode != null && dstINode.isDirectory()) {
+      absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
+      dstINode = store.retrieveINode(absoluteDst);
+    }
+    if (dstINode != null) {
+      // dst path already exists - can't overwrite
+      return false;
+    }
+    Path dstParent = absoluteDst.getParent();
+    if (dstParent != null) {
+      INode dstParentINode = store.retrieveINode(dstParent);
+      if (dstParentINode == null || dstParentINode.isFile()) {
+        // dst parent doesn't exist or is a file
+        return false;
+      }
+    }
+    return renameRecursive(absoluteSrc, absoluteDst);
+  }
+  
+  private boolean renameRecursive(Path src, Path dst) throws IOException {
+    INode srcINode = store.retrieveINode(src);
+    store.storeINode(dst, srcINode);
+    store.deleteINode(src);
+    if (srcINode.isDirectory()) {
+      for (Path oldSrc : store.listDeepSubPaths(src)) {
+        INode inode = store.retrieveINode(oldSrc);
+        if (inode == null) {
+          return false;
+        }
+        String oldSrcPath = oldSrc.toUri().getPath();
+        String srcPath = src.toUri().getPath();
+        String dstPath = dst.toUri().getPath();
+        Path newDst = new Path(oldSrcPath.replaceFirst(srcPath, dstPath));
+        store.storeINode(newDst, inode);
+        store.deleteINode(oldSrc);
+      }
+    }
+    return true;
+  }
+
+  public boolean delete(Path path, boolean recursive) throws IOException {
+   Path absolutePath = makeAbsolute(path);
+   INode inode = store.retrieveINode(absolutePath);
+   if (inode == null) {
+     return false;
+   }
+   if (inode.isFile()) {
+     store.deleteINode(absolutePath);
+     for (Block block: inode.getBlocks()) {
+       store.deleteBlock(block);
+     }
+   } else {
+     FileStatus[] contents = listStatus(absolutePath);
+     if (contents == null) {
+       return false;
+     }
+     if ((contents.length !=0) && (!recursive)) {
+       throw new IOException("Directory " + path.toString() 
+           + " is not empty.");
+     }
+     for (FileStatus p:contents) {
+       if (!delete(p.getPath(), recursive)) {
+         return false;
+       }
+     }
+     store.deleteINode(absolutePath);
+   }
+   return true;
+  }
+  
+  /**
+   * FileStatus for S3 file systems. 
+   */
+  @Override
+  public FileStatus getFileStatus(Path f)  throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(f));
+    if (inode == null) {
+      throw new FileNotFoundException(f + ": No such file or directory.");
+    }
+    return new S3FileStatus(f.makeQualified(this), inode);
+  }
+
+  // diagnostic methods
+
+  void dump() throws IOException {
+    store.dump();
+  }
+
+  void purge() throws IOException {
+    store.purge();
+  }
+
+  private static class S3FileStatus extends FileStatus {
+
+    S3FileStatus(Path f, INode inode) throws IOException {
+      super(findLength(inode), inode.isDirectory(), 1,
+            findBlocksize(inode), 0, f);
+    }
+
+    private static long findLength(INode inode) {
+      if (!inode.isDirectory()) {
+        long length = 0L;
+        for (Block block : inode.getBlocks()) {
+          length += block.getLength();
+        }
+        return length;
+      }
+      return 0;
+    }
+
+    private static long findBlocksize(INode inode) {
+      final Block[] ret = inode.getBlocks();
+      return ret == null ? 0L : ret[0].getLength();
+    }
+  }
+}

+ 31 - 0
src/java/org/apache/hadoop/fs/s3/S3FileSystemException.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.s3;
+
+import java.io.IOException;
+
+/**
+ * Thrown when there is a fatal exception while using {@link S3FileSystem}.
+ */
+public class S3FileSystemException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public S3FileSystemException(String message) {
+    super(message);
+  }
+}

+ 211 - 0
src/java/org/apache/hadoop/fs/s3/S3InputStream.java

@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem;
+
+class S3InputStream extends FSInputStream {
+
+  private FileSystemStore store;
+
+  private Block[] blocks;
+
+  private boolean closed;
+
+  private long fileLength;
+
+  private long pos = 0;
+
+  private File blockFile;
+  
+  private DataInputStream blockStream;
+
+  private long blockEnd = -1;
+  
+  private FileSystem.Statistics stats;
+  
+  private static final Log LOG = 
+    LogFactory.getLog(S3InputStream.class.getName());
+
+
+  @Deprecated
+  public S3InputStream(Configuration conf, FileSystemStore store,
+                       INode inode) {
+    this(conf, store, inode, null);
+  }
+
+  public S3InputStream(Configuration conf, FileSystemStore store,
+                       INode inode, FileSystem.Statistics stats) {
+    
+    this.store = store;
+    this.stats = stats;
+    this.blocks = inode.getBlocks();
+    for (Block block : blocks) {
+      this.fileLength += block.getLength();
+    }
+  }
+
+  @Override
+  public synchronized long getPos() throws IOException {
+    return pos;
+  }
+
+  @Override
+  public synchronized int available() throws IOException {
+    return (int) (fileLength - pos);
+  }
+
+  @Override
+  public synchronized void seek(long targetPos) throws IOException {
+    if (targetPos > fileLength) {
+      throw new IOException("Cannot seek after EOF");
+    }
+    pos = targetPos;
+    blockEnd = -1;
+  }
+
+  @Override
+  public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+    return false;
+  }
+
+  @Override
+  public synchronized int read() throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+    int result = -1;
+    if (pos < fileLength) {
+      if (pos > blockEnd) {
+        blockSeekTo(pos);
+      }
+      result = blockStream.read();
+      if (result >= 0) {
+        pos++;
+      }
+    }
+    if (stats != null & result >= 0) {
+      stats.incrementBytesRead(1);
+    }
+    return result;
+  }
+
+  @Override
+  public synchronized int read(byte buf[], int off, int len) throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+    if (pos < fileLength) {
+      if (pos > blockEnd) {
+        blockSeekTo(pos);
+      }
+      int realLen = Math.min(len, (int) (blockEnd - pos + 1));
+      int result = blockStream.read(buf, off, realLen);
+      if (result >= 0) {
+        pos += result;
+      }
+      if (stats != null && result > 0) {
+        stats.incrementBytesRead(result);
+      }
+      return result;
+    }
+    return -1;
+  }
+
+  private synchronized void blockSeekTo(long target) throws IOException {
+    //
+    // Compute desired block
+    //
+    int targetBlock = -1;
+    long targetBlockStart = 0;
+    long targetBlockEnd = 0;
+    for (int i = 0; i < blocks.length; i++) {
+      long blockLength = blocks[i].getLength();
+      targetBlockEnd = targetBlockStart + blockLength - 1;
+
+      if (target >= targetBlockStart && target <= targetBlockEnd) {
+        targetBlock = i;
+        break;
+      } else {
+        targetBlockStart = targetBlockEnd + 1;
+      }
+    }
+    if (targetBlock < 0) {
+      throw new IOException(
+                            "Impossible situation: could not find target position " + target);
+    }
+    long offsetIntoBlock = target - targetBlockStart;
+
+    // read block blocks[targetBlock] from position offsetIntoBlock
+
+    this.blockFile = store.retrieveBlock(blocks[targetBlock], offsetIntoBlock);
+
+    this.pos = target;
+    this.blockEnd = targetBlockEnd;
+    this.blockStream = new DataInputStream(new FileInputStream(blockFile));
+
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    if (blockStream != null) {
+      blockStream.close();
+      blockStream = null;
+    }
+    if (blockFile != null) {
+      boolean b = blockFile.delete();
+      if (!b) {
+        LOG.warn("Ignoring failed delete");
+      }
+    }
+    super.close();
+    closed = true;
+  }
+
+  /**
+   * We don't support marks.
+   */
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+
+  @Override
+  public void mark(int readLimit) {
+    // Do nothing
+  }
+
+  @Override
+  public void reset() throws IOException {
+    throw new IOException("Mark not supported");
+  }
+
+}

+ 231 - 0
src/java/org/apache/hadoop/fs/s3/S3OutputStream.java

@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3.INode.FileType;
+import org.apache.hadoop.util.Progressable;
+
+class S3OutputStream extends OutputStream {
+
+  private Configuration conf;
+  
+  private int bufferSize;
+
+  private FileSystemStore store;
+
+  private Path path;
+
+  private long blockSize;
+
+  private File backupFile;
+
+  private OutputStream backupStream;
+
+  private Random r = new Random();
+
+  private boolean closed;
+
+  private int pos = 0;
+
+  private long filePos = 0;
+
+  private int bytesWrittenToBlock = 0;
+
+  private byte[] outBuf;
+
+  private List<Block> blocks = new ArrayList<Block>();
+
+  private Block nextBlock;
+  
+  private static final Log LOG = 
+    LogFactory.getLog(S3OutputStream.class.getName());
+
+
+  public S3OutputStream(Configuration conf, FileSystemStore store,
+                        Path path, long blockSize, Progressable progress,
+                        int buffersize) throws IOException {
+    
+    this.conf = conf;
+    this.store = store;
+    this.path = path;
+    this.blockSize = blockSize;
+    this.backupFile = newBackupFile();
+    this.backupStream = new FileOutputStream(backupFile);
+    this.bufferSize = buffersize;
+    this.outBuf = new byte[bufferSize];
+
+  }
+
+  private File newBackupFile() throws IOException {
+    File dir = new File(conf.get("fs.s3.buffer.dir"));
+    if (!dir.exists() && !dir.mkdirs()) {
+      throw new IOException("Cannot create S3 buffer directory: " + dir);
+    }
+    File result = File.createTempFile("output-", ".tmp", dir);
+    result.deleteOnExit();
+    return result;
+  }
+
+  public long getPos() throws IOException {
+    return filePos;
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+
+    if ((bytesWrittenToBlock + pos == blockSize) || (pos >= bufferSize)) {
+      flush();
+    }
+    outBuf[pos++] = (byte) b;
+    filePos++;
+  }
+
+  @Override
+  public synchronized void write(byte b[], int off, int len) throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+    while (len > 0) {
+      int remaining = bufferSize - pos;
+      int toWrite = Math.min(remaining, len);
+      System.arraycopy(b, off, outBuf, pos, toWrite);
+      pos += toWrite;
+      off += toWrite;
+      len -= toWrite;
+      filePos += toWrite;
+
+      if ((bytesWrittenToBlock + pos >= blockSize) || (pos == bufferSize)) {
+        flush();
+      }
+    }
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+
+    if (bytesWrittenToBlock + pos >= blockSize) {
+      flushData((int) blockSize - bytesWrittenToBlock);
+    }
+    if (bytesWrittenToBlock == blockSize) {
+      endBlock();
+    }
+    flushData(pos);
+  }
+
+  private synchronized void flushData(int maxPos) throws IOException {
+    int workingPos = Math.min(pos, maxPos);
+
+    if (workingPos > 0) {
+      //
+      // To the local block backup, write just the bytes
+      //
+      backupStream.write(outBuf, 0, workingPos);
+
+      //
+      // Track position
+      //
+      bytesWrittenToBlock += workingPos;
+      System.arraycopy(outBuf, workingPos, outBuf, 0, pos - workingPos);
+      pos -= workingPos;
+    }
+  }
+
+  private synchronized void endBlock() throws IOException {
+    //
+    // Done with local copy
+    //
+    backupStream.close();
+
+    //
+    // Send it to S3
+    //
+    // TODO: Use passed in Progressable to report progress.
+    nextBlockOutputStream();
+    store.storeBlock(nextBlock, backupFile);
+    internalClose();
+
+    //
+    // Delete local backup, start new one
+    //
+    boolean b = backupFile.delete();
+    if (!b) {
+      LOG.warn("Ignoring failed delete");
+    }
+    backupFile = newBackupFile();
+    backupStream = new FileOutputStream(backupFile);
+    bytesWrittenToBlock = 0;
+  }
+
+  private synchronized void nextBlockOutputStream() throws IOException {
+    long blockId = r.nextLong();
+    while (store.blockExists(blockId)) {
+      blockId = r.nextLong();
+    }
+    nextBlock = new Block(blockId, bytesWrittenToBlock);
+    blocks.add(nextBlock);
+    bytesWrittenToBlock = 0;
+  }
+
+  private synchronized void internalClose() throws IOException {
+    INode inode = new INode(FileType.FILE, blocks.toArray(new Block[blocks
+                                                                    .size()]));
+    store.storeINode(path, inode);
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+
+    flush();
+    if (filePos == 0 || bytesWrittenToBlock != 0) {
+      endBlock();
+    }
+
+    backupStream.close();
+    boolean b = backupFile.delete();
+    if (!b) {
+      LOG.warn("Ignoring failed delete");
+    }
+
+    super.close();
+
+    closed = true;
+  }
+
+}

+ 32 - 0
src/java/org/apache/hadoop/fs/s3/VersionMismatchException.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.s3;
+
+/**
+ * Thrown when Hadoop cannot read the version of the data stored
+ * in {@link S3FileSystem}.
+ */
+public class VersionMismatchException extends S3FileSystemException {
+  private static final long serialVersionUID = 1L;
+
+  public VersionMismatchException(String clientVersion, String dataVersion) {
+    super("Version mismatch: client expects version " + clientVersion +
+        ", but data has version " +
+        (dataVersion == null ? "[unversioned]" : dataVersion));
+  }
+}

+ 55 - 0
src/java/org/apache/hadoop/fs/s3/package.html

@@ -0,0 +1,55 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+
+<p>A distributed, block-based implementation of {@link
+org.apache.hadoop.fs.FileSystem} that uses <a href="http://aws.amazon.com/s3">Amazon S3</a>
+as a backing store.</p>
+
+<p>
+Files are stored in S3 as blocks (represented by 
+{@link org.apache.hadoop.fs.s3.Block}), which have an ID and a length.
+Block metadata is stored in S3 as a small record (represented by 
+{@link org.apache.hadoop.fs.s3.INode}) using the URL-encoded
+path string as a key. Inodes record the file type (regular file or directory) and the list of blocks.
+This design makes it easy to seek to any given position in a file by reading the inode data to compute
+which block to access, then using S3's support for 
+<a href="http://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.35.2">HTTP Range</a> headers
+to start streaming from the correct position.
+Renames are also efficient since only the inode is moved (by a DELETE followed by a PUT since 
+S3 does not support renames).
+</p>
+<p>
+For a single file <i>/dir1/file1</i> which takes two blocks of storage, the file structure in S3
+would be something like this:
+</p>
+<pre>
+/
+/dir1
+/dir1/file1
+block-6415776850131549260
+block-3026438247347758425
+</pre>
+<p>
+Inodes start with a leading <code>/</code>, while blocks are prefixed with <code>block-</code>.
+</p>
+
+</body>
+</html>

+ 54 - 0
src/java/org/apache/hadoop/fs/s3native/FileMetadata.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+/**
+ * <p>
+ * Holds basic metadata for a file stored in a {@link NativeFileSystemStore}.
+ * </p>
+ */
+class FileMetadata {
+  private final String key;
+  private final long length;
+  private final long lastModified;
+  
+  public FileMetadata(String key, long length, long lastModified) {
+    this.key = key;
+    this.length = length;
+    this.lastModified = lastModified;
+  }
+  
+  public String getKey() {
+    return key;
+  }
+  
+  public long getLength() {
+    return length;
+  }
+
+  public long getLastModified() {
+    return lastModified;
+  }
+  
+  @Override
+  public String toString() {
+    return "FileMetadata[" + key + ", " + length + ", " + lastModified + "]";
+  }
+  
+}

+ 255 - 0
src/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java

@@ -0,0 +1,255 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+import static org.apache.hadoop.fs.s3native.NativeS3FileSystem.PATH_DELIMITER;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3.S3Credentials;
+import org.apache.hadoop.fs.s3.S3Exception;
+import org.jets3t.service.S3ObjectsChunk;
+import org.jets3t.service.S3Service;
+import org.jets3t.service.S3ServiceException;
+import org.jets3t.service.impl.rest.httpclient.RestS3Service;
+import org.jets3t.service.model.S3Bucket;
+import org.jets3t.service.model.S3Object;
+import org.jets3t.service.security.AWSCredentials;
+
+class Jets3tNativeFileSystemStore implements NativeFileSystemStore {
+  
+  private S3Service s3Service;
+  private S3Bucket bucket;
+  
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    S3Credentials s3Credentials = new S3Credentials();
+    s3Credentials.initialize(uri, conf);
+    try {
+      AWSCredentials awsCredentials =
+        new AWSCredentials(s3Credentials.getAccessKey(),
+            s3Credentials.getSecretAccessKey());
+      this.s3Service = new RestS3Service(awsCredentials);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+    bucket = new S3Bucket(uri.getHost());
+  }
+  
+  public void storeFile(String key, File file, byte[] md5Hash)
+    throws IOException {
+    
+    BufferedInputStream in = null;
+    try {
+      in = new BufferedInputStream(new FileInputStream(file));
+      S3Object object = new S3Object(key);
+      object.setDataInputStream(in);
+      object.setContentType("binary/octet-stream");
+      object.setContentLength(file.length());
+      if (md5Hash != null) {
+        object.setMd5Hash(md5Hash);
+      }
+      s3Service.putObject(bucket, object);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    } finally {
+      if (in != null) {
+        try {
+          in.close();
+        } catch (IOException e) {
+          // ignore
+        }
+      }
+    }
+  }
+
+  public void storeEmptyFile(String key) throws IOException {
+    try {
+      S3Object object = new S3Object(key);
+      object.setDataInputStream(new ByteArrayInputStream(new byte[0]));
+      object.setContentType("binary/octet-stream");
+      object.setContentLength(0);
+      s3Service.putObject(bucket, object);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+  
+  public FileMetadata retrieveMetadata(String key) throws IOException {
+    try {
+      S3Object object = s3Service.getObjectDetails(bucket, key);
+      return new FileMetadata(key, object.getContentLength(),
+          object.getLastModifiedDate().getTime());
+    } catch (S3ServiceException e) {
+      // Following is brittle. Is there a better way?
+      if (e.getMessage().contains("ResponseCode=404")) {
+        return null;
+      }
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+  
+  public InputStream retrieve(String key) throws IOException {
+    try {
+      S3Object object = s3Service.getObject(bucket, key);
+      return object.getDataInputStream();
+    } catch (S3ServiceException e) {
+      if ("NoSuchKey".equals(e.getS3ErrorCode())) {
+        return null;
+      }
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+  
+  public InputStream retrieve(String key, long byteRangeStart)
+    throws IOException {
+    try {
+      S3Object object = s3Service.getObject(bucket, key, null, null, null,
+                                            null, byteRangeStart, null);
+      return object.getDataInputStream();
+    } catch (S3ServiceException e) {
+      if ("NoSuchKey".equals(e.getS3ErrorCode())) {
+        return null;
+      }
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public PartialListing list(String prefix, int maxListingLength)
+    throws IOException {
+    return list(prefix, maxListingLength, null);
+  }
+  
+  public PartialListing list(String prefix, int maxListingLength,
+      String priorLastKey) throws IOException {
+
+    return list(prefix, PATH_DELIMITER, maxListingLength, priorLastKey);
+  }
+
+  public PartialListing listAll(String prefix, int maxListingLength,
+      String priorLastKey) throws IOException {
+
+    return list(prefix, null, maxListingLength, priorLastKey);
+  }
+
+  private PartialListing list(String prefix, String delimiter,
+      int maxListingLength, String priorLastKey) throws IOException {
+    try {
+      if (prefix.length() > 0 && !prefix.endsWith(PATH_DELIMITER)) {
+        prefix += PATH_DELIMITER;
+      }
+      S3ObjectsChunk chunk = s3Service.listObjectsChunked(bucket.getName(),
+          prefix, delimiter, maxListingLength, priorLastKey);
+      
+      FileMetadata[] fileMetadata =
+        new FileMetadata[chunk.getObjects().length];
+      for (int i = 0; i < fileMetadata.length; i++) {
+        S3Object object = chunk.getObjects()[i];
+        fileMetadata[i] = new FileMetadata(object.getKey(),
+            object.getContentLength(), object.getLastModifiedDate().getTime());
+      }
+      return new PartialListing(chunk.getPriorLastKey(), fileMetadata,
+          chunk.getCommonPrefixes());
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public void delete(String key) throws IOException {
+    try {
+      s3Service.deleteObject(bucket, key);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+  
+  public void rename(String srcKey, String dstKey) throws IOException {
+    try {
+      s3Service.moveObject(bucket.getName(), srcKey, bucket.getName(),
+          new S3Object(dstKey), false);
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public void purge(String prefix) throws IOException {
+    try {
+      S3Object[] objects = s3Service.listObjects(bucket, prefix, null);
+      for (int i = 0; i < objects.length; i++) {
+        s3Service.deleteObject(bucket, objects[i].getKey());
+      }
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+  }
+
+  public void dump() throws IOException {
+    StringBuilder sb = new StringBuilder("S3 Native Filesystem, ");
+    sb.append(bucket.getName()).append("\n");
+    try {
+      S3Object[] objects = s3Service.listObjects(bucket);
+      for (int i = 0; i < objects.length; i++) {
+        sb.append(objects[i].getKey()).append("\n");
+      }
+    } catch (S3ServiceException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      }
+      throw new S3Exception(e);
+    }
+    System.out.println(sb);
+  }
+  
+}

+ 65 - 0
src/java/org/apache/hadoop/fs/s3native/NativeFileSystemStore.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * <p>
+ * An abstraction for a key-based {@link File} store.
+ * </p>
+ */
+interface NativeFileSystemStore {
+  
+  void initialize(URI uri, Configuration conf) throws IOException;
+  
+  void storeFile(String key, File file, byte[] md5Hash) throws IOException;
+  void storeEmptyFile(String key) throws IOException;
+  
+  FileMetadata retrieveMetadata(String key) throws IOException;
+  InputStream retrieve(String key) throws IOException;
+  InputStream retrieve(String key, long byteRangeStart) throws IOException;
+  
+  PartialListing list(String prefix, int maxListingLength) throws IOException;
+  PartialListing list(String prefix, int maxListingLength, String priorLastKey)
+    throws IOException;
+  PartialListing listAll(String prefix, int maxListingLength,
+      String priorLastKey) throws IOException;
+  
+  void delete(String key) throws IOException;
+
+  void rename(String srcKey, String dstKey) throws IOException;
+  
+  /**
+   * Delete all keys with the given prefix. Used for testing.
+   * @throws IOException
+   */
+  void purge(String prefix) throws IOException;
+  
+  /**
+   * Diagnostic method to dump state to the console.
+   * @throws IOException
+   */
+  void dump() throws IOException;
+}

+ 578 - 0
src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java

@@ -0,0 +1,578 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.security.DigestOutputStream;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BufferedFSInputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3.S3Exception;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * <p>
+ * A {@link FileSystem} for reading and writing files stored on
+ * <a href="http://aws.amazon.com/s3">Amazon S3</a>.
+ * Unlike {@link org.apache.hadoop.fs.s3.S3FileSystem} this implementation
+ * stores files on S3 in their
+ * native form so they can be read by other S3 tools.
+ * </p>
+ * @see org.apache.hadoop.fs.s3.S3FileSystem
+ */
+public class NativeS3FileSystem extends FileSystem {
+  
+  public static final Log LOG = 
+    LogFactory.getLog(NativeS3FileSystem.class);
+  
+  private static final String FOLDER_SUFFIX = "_$folder$";
+  private static final long MAX_S3_FILE_SIZE = 5 * 1024 * 1024 * 1024L;
+  static final String PATH_DELIMITER = Path.SEPARATOR;
+  private static final int S3_MAX_LISTING_LENGTH = 1000;
+  
+  private class NativeS3FsInputStream extends FSInputStream {
+    
+    private InputStream in;
+    private final String key;
+    private long pos = 0;
+    
+    public NativeS3FsInputStream(InputStream in, String key) {
+      this.in = in;
+      this.key = key;
+    }
+    
+    public synchronized int read() throws IOException {
+      int result = in.read();
+      if (result != -1) {
+        pos++;
+      }
+      if (statistics != null && result != -1) {
+        statistics.incrementBytesRead(1);
+      }
+      return result;
+    }
+    public synchronized int read(byte[] b, int off, int len)
+      throws IOException {
+      
+      int result = in.read(b, off, len);
+      if (result > 0) {
+        pos += result;
+      }
+      if (statistics != null && result > 0) {
+        statistics.incrementBytesRead(result);
+      }
+      return result;
+    }
+
+    public void close() throws IOException {
+      in.close();
+    }
+
+    public synchronized void seek(long pos) throws IOException {
+      in.close();
+      in = store.retrieve(key, pos);
+      this.pos = pos;
+    }
+    public synchronized long getPos() throws IOException {
+      return pos;
+    }
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+  }
+  
+  private class NativeS3FsOutputStream extends OutputStream {
+    
+    private Configuration conf;
+    private String key;
+    private File backupFile;
+    private OutputStream backupStream;
+    private MessageDigest digest;
+    private boolean closed;
+    
+    public NativeS3FsOutputStream(Configuration conf,
+        NativeFileSystemStore store, String key, Progressable progress,
+        int bufferSize) throws IOException {
+      this.conf = conf;
+      this.key = key;
+      this.backupFile = newBackupFile();
+      try {
+        this.digest = MessageDigest.getInstance("MD5");
+        this.backupStream = new BufferedOutputStream(new DigestOutputStream(
+            new FileOutputStream(backupFile), this.digest));
+      } catch (NoSuchAlgorithmException e) {
+        LOG.warn("Cannot load MD5 digest algorithm," +
+            "skipping message integrity check.", e);
+        this.backupStream = new BufferedOutputStream(
+            new FileOutputStream(backupFile));
+      }
+    }
+
+    private File newBackupFile() throws IOException {
+      File dir = new File(conf.get("fs.s3.buffer.dir"));
+      if (!dir.mkdirs() && !dir.exists()) {
+        throw new IOException("Cannot create S3 buffer directory: " + dir);
+      }
+      File result = File.createTempFile("output-", ".tmp", dir);
+      result.deleteOnExit();
+      return result;
+    }
+    
+    @Override
+    public void flush() throws IOException {
+      backupStream.flush();
+    }
+    
+    @Override
+    public synchronized void close() throws IOException {
+      if (closed) {
+        return;
+      }
+
+      backupStream.close();
+      
+      try {
+        byte[] md5Hash = digest == null ? null : digest.digest();
+        store.storeFile(key, backupFile, md5Hash);
+      } finally {
+        if (!backupFile.delete()) {
+          LOG.warn("Could not delete temporary s3n file: " + backupFile);
+        }
+        super.close();
+        closed = true;
+      } 
+
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      backupStream.write(b);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      backupStream.write(b, off, len);
+    }
+    
+    
+  }
+  
+  private URI uri;
+  private NativeFileSystemStore store;
+  private Path workingDir;
+  
+  public NativeS3FileSystem() {
+    // set store in initialize()
+  }
+  
+  public NativeS3FileSystem(NativeFileSystemStore store) {
+    this.store = store;
+  }
+  
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    if (store == null) {
+      store = createDefaultStore(conf);
+    }
+    store.initialize(uri, conf);
+    setConf(conf);
+    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.workingDir =
+      new Path("/user", System.getProperty("user.name")).makeQualified(this);
+  }
+  
+  private static NativeFileSystemStore createDefaultStore(Configuration conf) {
+    NativeFileSystemStore store = new Jets3tNativeFileSystemStore();
+    
+    RetryPolicy basePolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
+        conf.getInt("fs.s3.maxRetries", 4),
+        conf.getLong("fs.s3.sleepTimeSeconds", 10), TimeUnit.SECONDS);
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
+      new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(IOException.class, basePolicy);
+    exceptionToPolicyMap.put(S3Exception.class, basePolicy);
+    
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String, RetryPolicy> methodNameToPolicyMap =
+      new HashMap<String, RetryPolicy>();
+    methodNameToPolicyMap.put("storeFile", methodPolicy);
+    
+    return (NativeFileSystemStore)
+      RetryProxy.create(NativeFileSystemStore.class, store,
+          methodNameToPolicyMap);
+  }
+  
+  private static String pathToKey(Path path) {
+    if (!path.isAbsolute()) {
+      throw new IllegalArgumentException("Path must be absolute: " + path);
+    }
+    return path.toUri().getPath().substring(1); // remove initial slash
+  }
+  
+  private static Path keyToPath(String key) {
+    return new Path("/" + key);
+  }
+  
+  private Path makeAbsolute(Path path) {
+    if (path.isAbsolute()) {
+      return path;
+    }
+    return new Path(workingDir, path);
+  }
+
+  /** This optional operation is not yet supported. */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Not supported");
+  }
+  
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+
+    if (exists(f) && !overwrite) {
+      throw new IOException("File already exists:"+f);
+    }
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    return new FSDataOutputStream(new NativeS3FsOutputStream(getConf(), store,
+        key, progress, bufferSize), statistics);
+  }
+  
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    FileStatus status;
+    try {
+      status = getFileStatus(f);
+    } catch (FileNotFoundException e) {
+      return false;
+    }
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    if (status.isDir()) {
+      FileStatus[] contents = listStatus(f);
+      if (!recursive && contents.length > 0) {
+        throw new IOException("Directory " + f.toString() + " is not empty.");
+      }
+      for (FileStatus p : contents) {
+        if (!delete(p.getPath(), recursive)) {
+          return false;
+        }
+      }
+      store.delete(key + FOLDER_SUFFIX);
+    } else {
+      store.delete(key);
+    }
+    return true;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    
+    if (key.length() == 0) { // root always exists
+      return newDirectory(absolutePath);
+    }
+    
+    FileMetadata meta = store.retrieveMetadata(key);
+    if (meta != null) {
+      return newFile(meta, absolutePath);
+    }
+    if (store.retrieveMetadata(key + FOLDER_SUFFIX) != null) {
+      return newDirectory(absolutePath);
+    }
+    
+    PartialListing listing = store.list(key, 1);
+    if (listing.getFiles().length > 0 ||
+        listing.getCommonPrefixes().length > 0) {
+      return newDirectory(absolutePath);
+    }
+    
+    throw new FileNotFoundException(absolutePath +
+        ": No such file or directory.");
+    
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  /**
+   * <p>
+   * If <code>f</code> is a file, this method will make a single call to S3.
+   * If <code>f</code> is a directory, this method will make a maximum of
+   * (<i>n</i> / 1000) + 2 calls to S3, where <i>n</i> is the total number of
+   * files and directories contained directly in <code>f</code>.
+   * </p>
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    
+    if (key.length() > 0) {
+      FileMetadata meta = store.retrieveMetadata(key);
+      if (meta != null) {
+        return new FileStatus[] { newFile(meta, absolutePath) };
+      }
+    }
+    
+    URI pathUri = absolutePath.toUri();
+    Set<FileStatus> status = new TreeSet<FileStatus>();
+    String priorLastKey = null;
+    do {
+      PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, 
+          priorLastKey);
+      for (FileMetadata fileMetadata : listing.getFiles()) {
+        Path subpath = keyToPath(fileMetadata.getKey());
+        String relativePath = pathUri.relativize(subpath.toUri()).getPath();
+        if (relativePath.endsWith(FOLDER_SUFFIX)) {
+          status.add(newDirectory(new Path(absolutePath,
+              relativePath.substring(0,
+                  relativePath.indexOf(FOLDER_SUFFIX)))));
+        } else {
+          status.add(newFile(fileMetadata, subpath));
+        }
+      }
+      for (String commonPrefix : listing.getCommonPrefixes()) {
+        Path subpath = keyToPath(commonPrefix);
+        String relativePath = pathUri.relativize(subpath.toUri()).getPath();
+        status.add(newDirectory(new Path(absolutePath, relativePath)));
+      }
+      priorLastKey = listing.getPriorLastKey();
+    } while (priorLastKey != null);
+    
+    if (status.isEmpty() &&
+        store.retrieveMetadata(key + FOLDER_SUFFIX) == null) {
+      return null;
+    }
+    
+    return status.toArray(new FileStatus[0]);
+  }
+  
+  private FileStatus newFile(FileMetadata meta, Path path) {
+    return new FileStatus(meta.getLength(), false, 1, MAX_S3_FILE_SIZE,
+        meta.getLastModified(), path.makeQualified(this));
+  }
+  
+  private FileStatus newDirectory(Path path) {
+    return new FileStatus(0, true, 1, MAX_S3_FILE_SIZE, 0,
+        path.makeQualified(this));
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    Path absolutePath = makeAbsolute(f);
+    List<Path> paths = new ArrayList<Path>();
+    do {
+      paths.add(0, absolutePath);
+      absolutePath = absolutePath.getParent();
+    } while (absolutePath != null);
+    
+    boolean result = true;
+    for (Path path : paths) {
+      result &= mkdir(path);
+    }
+    return result;
+  }
+  
+  private boolean mkdir(Path f) throws IOException {
+    try {
+      FileStatus fileStatus = getFileStatus(f);
+      if (!fileStatus.isDir()) {
+        throw new IOException(String.format(
+            "Can't make directory for path %s since it is a file.", f));
+
+      }
+    } catch (FileNotFoundException e) {
+      String key = pathToKey(f) + FOLDER_SUFFIX;
+      store.storeEmptyFile(key);    
+    }
+    return true;
+  }
+
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    if (!exists(f)) {
+      throw new FileNotFoundException(f.toString());
+    }
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    return new FSDataInputStream(new BufferedFSInputStream(
+        new NativeS3FsInputStream(store.retrieve(key), key), bufferSize));
+  }
+  
+  // rename() and delete() use this method to ensure that the parent directory
+  // of the source does not vanish.
+  private void createParent(Path path) throws IOException {
+      Path parent = path.getParent();
+      if (parent != null) {
+          String key = pathToKey(makeAbsolute(parent));
+          if (key.length() > 0) {
+              store.storeEmptyFile(key + FOLDER_SUFFIX);
+          }
+      }
+  }
+  
+  private boolean existsAndIsFile(Path f) throws IOException {
+    
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    
+    if (key.length() == 0) {
+        return false;
+    }
+    
+    FileMetadata meta = store.retrieveMetadata(key);
+    if (meta != null) {
+        // S3 object with given key exists, so this is a file
+        return true;
+    }
+    
+    if (store.retrieveMetadata(key + FOLDER_SUFFIX) != null) {
+        // Signifies empty directory
+        return false;
+    }
+    
+    PartialListing listing = store.list(key, 1, null);
+    if (listing.getFiles().length > 0 ||
+        listing.getCommonPrefixes().length > 0) {
+        // Non-empty directory
+        return false;
+    }
+    
+    throw new FileNotFoundException(absolutePath +
+        ": No such file or directory");
+}
+
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+
+    String srcKey = pathToKey(makeAbsolute(src));
+
+    if (srcKey.length() == 0) {
+      // Cannot rename root of file system
+      return false;
+    }
+
+    // Figure out the final destination
+    String dstKey;
+    try {
+      boolean dstIsFile = existsAndIsFile(dst);
+      if (dstIsFile) {
+        // Attempting to overwrite a file using rename()
+        return false;
+      } else {
+        // Move to within the existent directory
+        dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName())));
+      }
+    } catch (FileNotFoundException e) {
+      // dst doesn't exist, so we can proceed
+      dstKey = pathToKey(makeAbsolute(dst));
+      try {
+        if (!getFileStatus(dst.getParent()).isDir()) {
+          return false; // parent dst is a file
+        }
+      } catch (FileNotFoundException ex) {
+        return false; // parent dst does not exist
+      }
+    }
+
+    try {
+      boolean srcIsFile = existsAndIsFile(src);
+      if (srcIsFile) {
+        store.rename(srcKey, dstKey);
+      } else {
+        // Move the folder object
+        store.delete(srcKey + FOLDER_SUFFIX);
+        store.storeEmptyFile(dstKey + FOLDER_SUFFIX);
+
+        // Move everything inside the folder
+        String priorLastKey = null;
+        do {
+          PartialListing listing = store.listAll(srcKey, S3_MAX_LISTING_LENGTH,
+              priorLastKey);
+          for (FileMetadata file : listing.getFiles()) {
+            store.rename(file.getKey(), dstKey
+                + file.getKey().substring(srcKey.length()));
+          }
+          priorLastKey = listing.getPriorLastKey();
+        } while (priorLastKey != null);
+      }
+
+      createParent(src);
+      return true;
+
+    } catch (FileNotFoundException e) {
+      // Source file does not exist;
+      return false;
+    }
+  }
+
+
+  /**
+   * Set the working directory to the given directory.
+   */
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    workingDir = newDir;
+  }
+  
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+}

+ 59 - 0
src/java/org/apache/hadoop/fs/s3native/PartialListing.java

@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3native;
+
+/**
+ * <p>
+ * Holds information on a directory listing for a
+ * {@link NativeFileSystemStore}.
+ * This includes the {@link FileMetadata files} and directories
+ * (their names) contained in a directory.
+ * </p>
+ * <p>
+ * This listing may be returned in chunks, so a <code>priorLastKey</code>
+ * is provided so that the next chunk may be requested.
+ * </p>
+ * @see NativeFileSystemStore#list(String, int, String)
+ */
+class PartialListing {
+  
+  private final String priorLastKey;
+  private final FileMetadata[] files;
+  private final String[] commonPrefixes;
+  
+  public PartialListing(String priorLastKey, FileMetadata[] files,
+      String[] commonPrefixes) {
+    this.priorLastKey = priorLastKey;
+    this.files = files;
+    this.commonPrefixes = commonPrefixes;
+  }
+
+  public FileMetadata[] getFiles() {
+    return files;
+  }
+
+  public String[] getCommonPrefixes() {
+    return commonPrefixes;
+  }
+
+  public String getPriorLastKey() {
+    return priorLastKey;
+  }
+  
+}

+ 32 - 0
src/java/org/apache/hadoop/fs/s3native/package.html

@@ -0,0 +1,32 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+
+<p>
+A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem} for reading and writing files on
+<a href="http://aws.amazon.com/s3">Amazon S3</a>.
+Unlike {@link org.apache.hadoop.fs.s3.S3FileSystem}, which is block-based,
+this implementation stores
+files on S3 in their native form for interoperability with other S3 tools.
+</p>
+
+</body>
+</html>

+ 86 - 0
src/java/org/apache/hadoop/fs/shell/Command.java

@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.shell;
+
+import java.io.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ipc.RemoteException;
+
+/**
+ * An abstract class for the execution of a file system command
+ */
+abstract public class Command extends Configured {
+  protected String[] args;
+  
+  /** Constructor */
+  protected Command(Configuration conf) {
+    super(conf);
+  }
+  
+  /** Return the command's name excluding the leading character - */
+  abstract public String getCommandName();
+  
+  /** 
+   * Execute the command on the input path
+   * 
+   * @param path the input path
+   * @throws IOException if any error occurs
+   */
+  abstract protected void run(Path path) throws IOException;
+  
+  /** 
+   * For each source path, execute the command
+   * 
+   * @return 0 if it runs successfully; -1 if it fails
+   */
+  public int runAll() {
+    int exitCode = 0;
+    for (String src : args) {
+      try {
+        Path srcPath = new Path(src);
+        FileSystem fs = srcPath.getFileSystem(getConf());
+        FileStatus[] statuses = fs.globStatus(srcPath);
+        if (statuses == null) {
+          System.err.println("Can not find listing for " + src);
+          exitCode = -1;
+        } else {
+          for(FileStatus s : statuses) {
+            run(s.getPath());
+          }
+        }
+      } catch (RemoteException re) {
+        exitCode = -1;
+        String content = re.getLocalizedMessage();
+        int eol = content.indexOf('\n');
+        if (eol>=0) {
+          content = content.substring(0, eol);
+        }
+        System.err.println(getCommandName() + ": " + content);
+      } catch (IOException e) {
+        exitCode = -1;
+        System.err.println(getCommandName() + ": " + e.getLocalizedMessage());
+      }
+    }
+    return exitCode;
+  }
+}

+ 75 - 0
src/java/org/apache/hadoop/fs/shell/CommandFormat.java

@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.shell;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Parse the args of a command and check the format of args.
+ */
+public class CommandFormat {
+  final String name;
+  final int minPar, maxPar;
+  final Map<String, Boolean> options = new HashMap<String, Boolean>();
+
+  /** constructor */
+  public CommandFormat(String n, int min, int max, String ... possibleOpt) {
+    name = n;
+    minPar = min;
+    maxPar = max;
+    for(String opt : possibleOpt)
+      options.put(opt, Boolean.FALSE);
+  }
+
+  /** Parse parameters starting from the given position
+   * 
+   * @param args an array of input arguments
+   * @param pos the position at which starts to parse
+   * @return a list of parameters
+   */
+  public List<String> parse(String[] args, int pos) {
+    List<String> parameters = new ArrayList<String>();
+    for(; pos < args.length; pos++) {
+      if (args[pos].charAt(0) == '-' && args[pos].length() > 1) {
+        String opt = args[pos].substring(1);
+        if (options.containsKey(opt))
+          options.put(opt, Boolean.TRUE);
+        else
+          throw new IllegalArgumentException("Illegal option " + args[pos]);
+      }
+      else
+        parameters.add(args[pos]);
+    }
+    int psize = parameters.size();
+    if (psize < minPar || psize > maxPar)
+      throw new IllegalArgumentException("Illegal number of arguments");
+    return parameters;
+  }
+  
+  /** Return if the option is set or not
+   * 
+   * @param option String representation of an option
+   * @return true is the option is set; false otherwise
+   */
+  public boolean getOpt(String option) {
+    return options.get(option);
+  }
+}

+ 28 - 0
src/java/org/apache/hadoop/fs/shell/CommandUtils.java

@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.shell;
+
+final class CommandUtils {
+  static String formatDescription(String usage, String... desciptions) {
+    StringBuilder b = new StringBuilder(usage + ": " + desciptions[0]);
+    for(int i = 1; i < desciptions.length; i++) {
+      b.append("\n\t\t" + desciptions[i]);
+    }
+    return b.toString();
+  }
+}

+ 77 - 0
src/java/org/apache/hadoop/fs/shell/Count.java

@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.shell;
+
+import java.io.*;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Count the number of directories, files, bytes, quota, and remaining quota.
+ */
+public class Count extends Command {
+  public static final String NAME = "count";
+  public static final String USAGE = "-" + NAME + "[-q] <path>";
+  public static final String DESCRIPTION = CommandUtils.formatDescription(USAGE, 
+      "Count the number of directories, files and bytes under the paths",
+      "that match the specified file pattern.  The output columns are:",
+      "DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME or",
+      "QUOTA REMAINING_QUATA SPACE_QUOTA REMAINING_SPACE_QUOTA ",
+      "      DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME");
+  
+  private boolean qOption;
+
+  /** Constructor
+   * 
+   * @param cmd the count command
+   * @param pos the starting index of the arguments 
+   */
+  public Count(String[] cmd, int pos, Configuration conf) {
+    super(conf);
+    CommandFormat c = new CommandFormat(NAME, 1, Integer.MAX_VALUE, "q");
+    List<String> parameters = c.parse(cmd, pos);
+    this.args = parameters.toArray(new String[parameters.size()]);
+    if (this.args.length == 0) { // default path is the current working directory
+      this.args = new String[] {"."};
+    }
+    this.qOption = c.getOpt("q") ? true: false;
+  }
+  
+  /** Check if a command is the count command
+   * 
+   * @param cmd A string representation of a command starting with "-"
+   * @return true if this is a count command; false otherwise
+   */
+  public static boolean matches(String cmd) {
+    return ("-" + NAME).equals(cmd); 
+  }
+
+  @Override
+  public String getCommandName() {
+    return NAME;
+  }
+
+  @Override
+  protected void run(Path path) throws IOException {
+    FileSystem fs = path.getFileSystem(getConf());
+    System.out.println(fs.getContentSummary(path).toString(qOption) + path);
+  }
+}

+ 40 - 0
src/java/org/apache/hadoop/http/FilterContainer.java

@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+import java.util.Map;
+
+/**
+ * A container class for javax.servlet.Filter. 
+ */
+public interface FilterContainer {
+  /**
+   * Add a filter to the container.
+   * @param name Filter name
+   * @param classname Filter class name
+   * @param parameters a map from parameter names to initial values
+   */
+  void addFilter(String name, String classname, Map<String, String> parameters);
+  /**
+   * Add a global filter to the container.
+   * @param name filter name
+   * @param classname filter class name
+   * @param parameters a map from parameter names to initial values
+   */
+  void addGlobalFilter(String name, String classname, Map<String, String> parameters);
+}

+ 29 - 0
src/java/org/apache/hadoop/http/FilterInitializer.java

@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+/**
+ * Initialize a javax.servlet.Filter. 
+ */
+public abstract class FilterInitializer {
+  /**
+   * Initialize a Filter to a FilterContainer.
+   * @param container The filter container
+   */
+  abstract void initFilter(FilterContainer container);
+}

+ 519 - 0
src/java/org/apache/hadoop/http/HttpServer.java

@@ -0,0 +1,519 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.nio.channels.ServerSocketChannel;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.log.LogLevel;
+import org.apache.hadoop.metrics.MetricsServlet;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Handler;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.nio.SelectChannelConnector;
+import org.mortbay.jetty.security.SslSocketConnector;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.DefaultServlet;
+import org.mortbay.jetty.servlet.FilterHolder;
+import org.mortbay.jetty.servlet.FilterMapping;
+import org.mortbay.jetty.servlet.ServletHandler;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.mortbay.thread.QueuedThreadPool;
+import org.mortbay.util.MultiException;
+
+/**
+ * Create a Jetty embedded server to answer http requests. The primary goal
+ * is to serve up status information for the server.
+ * There are three contexts:
+ *   "/logs/" -> points to the log directory
+ *   "/static/" -> points to common static files (src/webapps/static)
+ *   "/" -> the jsp server code from (src/webapps/<name>)
+ */
+public class HttpServer implements FilterContainer {
+  public static final Log LOG = LogFactory.getLog(HttpServer.class);
+
+  static final String FILTER_INITIALIZER_PROPERTY
+      = "hadoop.http.filter.initializers";
+
+  protected final Server webServer;
+  protected final Connector listener;
+  protected final WebAppContext webAppContext;
+  protected final boolean findPort;
+  protected final Map<Context, Boolean> defaultContexts =
+      new HashMap<Context, Boolean>();
+  protected final List<String> filterNames = new ArrayList<String>();
+  private static final int MAX_RETRIES = 10;
+
+  /** Same as this(name, bindAddress, port, findPort, null); */
+  public HttpServer(String name, String bindAddress, int port, boolean findPort
+      ) throws IOException {
+    this(name, bindAddress, port, findPort, new Configuration());
+  }
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/<name>.
+   * @param name The name of the server
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and 
+   *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
+   */
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf) throws IOException {
+    webServer = new Server();
+    this.findPort = findPort;
+
+    listener = createBaseListener(conf);
+    listener.setHost(bindAddress);
+    listener.setPort(port);
+    webServer.addConnector(listener);
+
+    webServer.setThreadPool(new QueuedThreadPool());
+
+    final String appDir = getWebAppsPath();
+    ContextHandlerCollection contexts = new ContextHandlerCollection();
+    webServer.setHandler(contexts);
+
+    webAppContext = new WebAppContext();
+    webAppContext.setContextPath("/");
+    webAppContext.setWar(appDir + "/" + name);
+    webServer.addHandler(webAppContext);
+
+    addDefaultApps(contexts, appDir);
+
+    final FilterInitializer[] initializers = getFilterInitializers(conf); 
+    if (initializers != null) {
+      for(FilterInitializer c : initializers) {
+        c.initFilter(this);
+      }
+    }
+    addDefaultServlets();
+  }
+
+  /**
+   * Create a required listener for the Jetty instance listening on the port
+   * provided. This wrapper and all subclasses must create at least one
+   * listener.
+   */
+  protected Connector createBaseListener(Configuration conf)
+      throws IOException {
+    SelectChannelConnector ret = new SelectChannelConnector();
+    ret.setLowResourceMaxIdleTime(10000);
+    ret.setAcceptQueueSize(128);
+    ret.setResolveNames(false);
+    ret.setUseDirectBuffers(false);
+    return ret;
+  }
+
+  /** Get an array of FilterConfiguration specified in the conf */
+  private static FilterInitializer[] getFilterInitializers(Configuration conf) {
+    if (conf == null) {
+      return null;
+    }
+
+    Class<?>[] classes = conf.getClasses(FILTER_INITIALIZER_PROPERTY);
+    if (classes == null) {
+      return null;
+    }
+
+    FilterInitializer[] initializers = new FilterInitializer[classes.length];
+    for(int i = 0; i < classes.length; i++) {
+      initializers[i] = (FilterInitializer)ReflectionUtils.newInstance(
+          classes[i], conf);
+    }
+    return initializers;
+  }
+
+  /**
+   * Add default apps.
+   * @param appDir The application directory
+   * @throws IOException
+   */
+  protected void addDefaultApps(ContextHandlerCollection parent,
+      final String appDir) throws IOException {
+    // set up the context for "/logs/" if "hadoop.log.dir" property is defined. 
+    String logDir = System.getProperty("hadoop.log.dir");
+    if (logDir != null) {
+      Context logContext = new Context(parent, "/logs");
+      logContext.setResourceBase(logDir);
+      logContext.addServlet(DefaultServlet.class, "/");
+      defaultContexts.put(logContext, true);
+    }
+    // set up the context for "/static/*"
+    Context staticContext = new Context(parent, "/static");
+    staticContext.setResourceBase(appDir + "/static");
+    staticContext.addServlet(DefaultServlet.class, "/*");
+    defaultContexts.put(staticContext, true);
+  }
+  
+  /**
+   * Add default servlets.
+   */
+  protected void addDefaultServlets() {
+    // set up default servlets
+    addServlet("stacks", "/stacks", StackServlet.class);
+    addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
+    addServlet("metrics", "/metrics", MetricsServlet.class);
+  }
+
+  public void addContext(Context ctxt, boolean isFiltered)
+      throws IOException {
+    webServer.addHandler(ctxt);
+    defaultContexts.put(ctxt, isFiltered);
+  }
+
+  /**
+   * Add a context 
+   * @param pathSpec The path spec for the context
+   * @param dir The directory containing the context
+   * @param isFiltered if true, the servlet is added to the filter path mapping 
+   * @throws IOException
+   */
+  protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
+    if (0 == webServer.getHandlers().length) {
+      throw new RuntimeException("Couldn't find handler");
+    }
+    WebAppContext webAppCtx = new WebAppContext();
+    webAppCtx.setContextPath(pathSpec);
+    webAppCtx.setWar(dir);
+    addContext(webAppCtx, true);
+  }
+
+  /**
+   * Set a value in the webapp context. These values are available to the jsp
+   * pages as "application.getAttribute(name)".
+   * @param name The name of the attribute
+   * @param value The value of the attribute
+   */
+  public void setAttribute(String name, Object value) {
+    webAppContext.setAttribute(name, value);
+  }
+
+  /**
+   * Add a servlet in the server.
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   */
+  public void addServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    addInternalServlet(name, pathSpec, clazz);
+    addFilterPathMapping(pathSpec, webAppContext);
+  }
+
+  /**
+   * Add an internal servlet in the server.
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   * @deprecated this is a temporary method
+   */
+  @Deprecated
+  public void addInternalServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    ServletHolder holder = new ServletHolder(clazz);
+    if (name != null) {
+      holder.setName(name);
+    }
+    webAppContext.addServlet(holder, pathSpec);
+  }
+
+  /** {@inheritDoc} */
+  public void addFilter(String name, String classname,
+      Map<String, String> parameters) {
+
+    final String[] USER_FACING_URLS = { "*.html", "*.jsp" };
+    defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS);
+    final String[] ALL_URLS = { "/*" };
+    for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
+      if (e.getValue()) {
+        Context ctx = e.getKey();
+        defineFilter(ctx, name, classname, parameters, ALL_URLS);
+        LOG.info("Added filter " + name + " (class=" + classname
+            + ") to context " + ctx.getDisplayName());
+      }
+    }
+    filterNames.add(name);
+  }
+
+  /** {@inheritDoc} */
+  public void addGlobalFilter(String name, String classname,
+      Map<String, String> parameters) {
+    final String[] ALL_URLS = { "/*" };
+    defineFilter(webAppContext, name, classname, parameters, ALL_URLS);
+    for (Context ctx : defaultContexts.keySet()) {
+      defineFilter(ctx, name, classname, parameters, ALL_URLS);
+    }
+    LOG.info("Added global filter" + name + " (class=" + classname + ")");
+  }
+
+  /**
+   * Define a filter for a context and set up default url mappings.
+   */
+  protected void defineFilter(Context ctx, String name,
+      String classname, Map<String,String> parameters, String[] urls) {
+
+    FilterHolder holder = new FilterHolder();
+    holder.setName(name);
+    holder.setClassName(classname);
+    holder.setInitParameters(parameters);
+    FilterMapping fmap = new FilterMapping();
+    fmap.setPathSpecs(urls);
+    fmap.setDispatches(Handler.ALL);
+    fmap.setFilterName(name);
+    ServletHandler handler = ctx.getServletHandler();
+    handler.addFilter(holder, fmap);
+  }
+
+  /**
+   * Add the path spec to the filter path mapping.
+   * @param pathSpec The path spec
+   * @param webAppCtx The WebApplicationContext to add to
+   */
+  protected void addFilterPathMapping(String pathSpec,
+      Context webAppCtx) {
+    ServletHandler handler = webAppCtx.getServletHandler();
+    for(String name : filterNames) {
+      FilterMapping fmap = new FilterMapping();
+      fmap.setPathSpec(pathSpec);
+      fmap.setFilterName(name);
+      fmap.setDispatches(Handler.ALL);
+      handler.addFilterMapping(fmap);
+    }
+  }
+  
+  /**
+   * Get the value in the webapp context.
+   * @param name The name of the attribute
+   * @return The value of the attribute
+   */
+  public Object getAttribute(String name) {
+    return webAppContext.getAttribute(name);
+  }
+
+  /**
+   * Get the pathname to the webapps files.
+   * @return the pathname as a URL
+   * @throws IOException if 'webapps' directory cannot be found on CLASSPATH.
+   */
+  protected String getWebAppsPath() throws IOException {
+    URL url = getClass().getClassLoader().getResource("webapps");
+    if (url == null) 
+      throw new IOException("webapps not found in CLASSPATH"); 
+    return url.toString();
+  }
+
+  /**
+   * Get the port that the server is on
+   * @return the port
+   */
+  public int getPort() {
+    return webServer.getConnectors()[0].getLocalPort();
+  }
+
+  /**
+   * Set the min, max number of worker threads (simultaneous connections).
+   */
+  public void setThreads(int min, int max) {
+    QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool() ;
+    pool.setMinThreads(min);
+    pool.setMaxThreads(max);
+  }
+
+  /**
+   * Configure an ssl listener on the server.
+   * @param addr address to listen on
+   * @param keystore location of the keystore
+   * @param storPass password for the keystore
+   * @param keyPass password for the key
+   * @deprecated Use {@link #addSslListener(InetSocketAddress, Configuration, boolean)}
+   */
+  @Deprecated
+  public void addSslListener(InetSocketAddress addr, String keystore,
+      String storPass, String keyPass) throws IOException {
+    if (webServer.isStarted()) {
+      throw new IOException("Failed to add ssl listener");
+    }
+    SslSocketConnector sslListener = new SslSocketConnector();
+    sslListener.setHost(addr.getHostName());
+    sslListener.setPort(addr.getPort());
+    sslListener.setKeystore(keystore);
+    sslListener.setPassword(storPass);
+    sslListener.setKeyPassword(keyPass);
+    webServer.addConnector(sslListener);
+  }
+
+  /**
+   * Configure an ssl listener on the server.
+   * @param addr address to listen on
+   * @param sslConf conf to retrieve ssl options
+   * @param needClientAuth whether client authentication is required
+   */
+  public void addSslListener(InetSocketAddress addr, Configuration sslConf,
+      boolean needClientAuth) throws IOException {
+    if (webServer.isStarted()) {
+      throw new IOException("Failed to add ssl listener");
+    }
+    if (needClientAuth) {
+      // setting up SSL truststore for authenticating clients
+      System.setProperty("javax.net.ssl.trustStore", sslConf.get(
+          "ssl.server.truststore.location", ""));
+      System.setProperty("javax.net.ssl.trustStorePassword", sslConf.get(
+          "ssl.server.truststore.password", ""));
+      System.setProperty("javax.net.ssl.trustStoreType", sslConf.get(
+          "ssl.server.truststore.type", "jks"));
+    }
+    SslSocketConnector sslListener = new SslSocketConnector();
+    sslListener.setHost(addr.getHostName());
+    sslListener.setPort(addr.getPort());
+    sslListener.setKeystore(sslConf.get("ssl.server.keystore.location"));
+    sslListener.setPassword(sslConf.get("ssl.server.keystore.password", ""));
+    sslListener.setKeyPassword(sslConf.get("ssl.server.keystore.keypassword", ""));
+    sslListener.setKeystoreType(sslConf.get("ssl.server.keystore.type", "jks"));
+    sslListener.setNeedClientAuth(needClientAuth);
+    webServer.addConnector(sslListener);
+  }
+
+  /**
+   * Start the server. Does not wait for the server to start.
+   */
+  public void start() throws IOException {
+    try {
+      int port = 0;
+      int oriPort = listener.getPort(); // The original requested port
+      while (true) {
+        try {
+          port = webServer.getConnectors()[0].getLocalPort();
+          LOG.info("Port returned by webServer.getConnectors()[0]." +
+          		"getLocalPort() before open() is "+ port + 
+          		". Opening the listener on " + oriPort);
+          listener.open();
+          port = listener.getLocalPort();
+          LOG.info("listener.getLocalPort() returned " + listener.getLocalPort() + 
+                " webServer.getConnectors()[0].getLocalPort() returned " +
+                webServer.getConnectors()[0].getLocalPort());
+          //Workaround to handle the problem reported in HADOOP-4744
+          if (port < 0) {
+            Thread.sleep(100);
+            int numRetries = 1;
+            while (port < 0) {
+              LOG.warn("listener.getLocalPort returned " + port);
+              if (numRetries++ > MAX_RETRIES) {
+                throw new Exception(" listener.getLocalPort is returning " +
+                		"less than 0 even after " +numRetries+" resets");
+              }
+              for (int i = 0; i < 2; i++) {
+                LOG.info("Retrying listener.getLocalPort()");
+                port = listener.getLocalPort();
+                if (port > 0) {
+                  break;
+                }
+                Thread.sleep(200);
+              }
+              if (port > 0) {
+                break;
+              }
+              LOG.info("Bouncing the listener");
+              listener.close();
+              Thread.sleep(1000);
+              listener.setPort(oriPort == 0 ? 0 : (oriPort += 1));
+              listener.open();
+              Thread.sleep(100);
+              port = listener.getLocalPort();
+            }
+          } //Workaround end
+          LOG.info("Jetty bound to port " + port);
+          webServer.start();
+          break;
+        } catch (IOException ex) {
+          // if this is a bind exception,
+          // then try the next port number.
+          if (ex instanceof BindException) {
+            if (!findPort) {
+              throw (BindException) ex;
+            }
+          } else {
+            LOG.info("HttpServer.start() threw a non Bind IOException"); 
+            throw ex;
+          }
+        } catch (MultiException ex) {
+          LOG.info("HttpServer.start() threw a MultiException"); 
+          throw ex;
+        }
+        listener.setPort((oriPort += 1));
+      }
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException("Problem starting http server", e);
+    }
+  }
+
+  /**
+   * stop the server
+   */
+  public void stop() throws Exception {
+    listener.close();
+    webServer.stop();
+  }
+
+  public void join() throws InterruptedException {
+    webServer.join();
+  }
+
+  /**
+   * A very simple servlet to serve up a text representation of the current
+   * stack traces. It both returns the stacks to the caller and logs them.
+   * Currently the stack traces are done sequentially rather than exactly the
+   * same data.
+   */
+  public static class StackServlet extends HttpServlet {
+    private static final long serialVersionUID = -6284183679759467039L;
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+      
+      PrintWriter out = new PrintWriter(response.getOutputStream());
+      ReflectionUtils.printThreadInfo(out, "");
+      out.close();
+      ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1);      
+    }
+  }
+}

+ 207 - 0
src/java/org/apache/hadoop/io/AbstractMapWritable.java

@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Abstract base class for MapWritable and SortedMapWritable
+ * 
+ * Unlike org.apache.nutch.crawl.MapWritable, this class allows creation of
+ * MapWritable&lt;Writable, MapWritable&gt; so the CLASS_TO_ID and ID_TO_CLASS
+ * maps travel with the class instead of being static.
+ * 
+ * Class ids range from 1 to 127 so there can be at most 127 distinct classes
+ * in any specific map instance.
+ */
+public abstract class AbstractMapWritable implements Writable, Configurable {
+  private AtomicReference<Configuration> conf;
+  
+  /* Class to id mappings */
+  private Map<Class, Byte> classToIdMap = new ConcurrentHashMap<Class, Byte>();
+  
+  /* Id to Class mappings */
+  private Map<Byte, Class> idToClassMap = new ConcurrentHashMap<Byte, Class>();
+  
+  /* The number of new classes (those not established by the constructor) */
+  private volatile byte newClasses = 0;
+  
+  /** @return the number of known classes */
+  byte getNewClasses() {
+    return newClasses;
+  }
+
+  /**
+   * Used to add "predefined" classes and by Writable to copy "new" classes.
+   */
+  private synchronized void addToMap(Class clazz, byte id) {
+    if (classToIdMap.containsKey(clazz)) {
+      byte b = classToIdMap.get(clazz);
+      if (b != id) {
+        throw new IllegalArgumentException ("Class " + clazz.getName() +
+          " already registered but maps to " + b + " and not " + id);
+      }
+    }
+    if (idToClassMap.containsKey(id)) {
+      Class c = idToClassMap.get(id);
+      if (!c.equals(clazz)) {
+        throw new IllegalArgumentException("Id " + id + " exists but maps to " +
+            c.getName() + " and not " + clazz.getName());
+      }
+    }
+    classToIdMap.put(clazz, id);
+    idToClassMap.put(id, clazz);
+  }
+  
+  /** Add a Class to the maps if it is not already present. */ 
+  protected synchronized void addToMap(Class clazz) {
+    if (classToIdMap.containsKey(clazz)) {
+      return;
+    }
+    if (newClasses + 1 > Byte.MAX_VALUE) {
+      throw new IndexOutOfBoundsException("adding an additional class would" +
+      " exceed the maximum number allowed");
+    }
+    byte id = ++newClasses;
+    addToMap(clazz, id);
+  }
+
+  /** @return the Class class for the specified id */
+  protected Class getClass(byte id) {
+    return idToClassMap.get(id);
+  }
+
+  /** @return the id for the specified Class */
+  protected byte getId(Class clazz) {
+    return classToIdMap.containsKey(clazz) ? classToIdMap.get(clazz) : -1;
+  }
+
+  /** Used by child copy constructors. */
+  protected synchronized void copy(Writable other) {
+    if (other != null) {
+      try {
+        DataOutputBuffer out = new DataOutputBuffer();
+        other.write(out);
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), out.getLength());
+        readFields(in);
+        
+      } catch (IOException e) {
+        throw new IllegalArgumentException("map cannot be copied: " +
+            e.getMessage());
+      }
+      
+    } else {
+      throw new IllegalArgumentException("source map cannot be null");
+    }
+  }
+  
+  /** constructor. */
+  protected AbstractMapWritable() {
+    this.conf = new AtomicReference<Configuration>();
+
+    addToMap(ArrayWritable.class,
+        Byte.valueOf(Integer.valueOf(-127).byteValue())); 
+    addToMap(BooleanWritable.class,
+        Byte.valueOf(Integer.valueOf(-126).byteValue()));
+    addToMap(BytesWritable.class,
+        Byte.valueOf(Integer.valueOf(-125).byteValue()));
+    addToMap(FloatWritable.class,
+        Byte.valueOf(Integer.valueOf(-124).byteValue()));
+    addToMap(IntWritable.class,
+        Byte.valueOf(Integer.valueOf(-123).byteValue()));
+    addToMap(LongWritable.class,
+        Byte.valueOf(Integer.valueOf(-122).byteValue()));
+    addToMap(MapWritable.class,
+        Byte.valueOf(Integer.valueOf(-121).byteValue()));
+    addToMap(MD5Hash.class,
+        Byte.valueOf(Integer.valueOf(-120).byteValue()));
+    addToMap(NullWritable.class,
+        Byte.valueOf(Integer.valueOf(-119).byteValue()));
+    addToMap(ObjectWritable.class,
+        Byte.valueOf(Integer.valueOf(-118).byteValue()));
+    addToMap(SortedMapWritable.class,
+        Byte.valueOf(Integer.valueOf(-117).byteValue()));
+    addToMap(Text.class,
+        Byte.valueOf(Integer.valueOf(-116).byteValue()));
+    addToMap(TwoDArrayWritable.class,
+        Byte.valueOf(Integer.valueOf(-115).byteValue()));
+    
+    // UTF8 is deprecated so we don't support it
+
+    addToMap(VIntWritable.class,
+        Byte.valueOf(Integer.valueOf(-114).byteValue()));
+    addToMap(VLongWritable.class,
+        Byte.valueOf(Integer.valueOf(-113).byteValue()));
+
+  }
+
+  /** @return the conf */
+  public Configuration getConf() {
+    return conf.get();
+  }
+
+  /** @param conf the conf to set */
+  public void setConf(Configuration conf) {
+    this.conf.set(conf);
+  }
+  
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    
+    // First write out the size of the class table and any classes that are
+    // "unknown" classes
+    
+    out.writeByte(newClasses);
+
+    for (byte i = 1; i <= newClasses; i++) {
+      out.writeByte(i);
+      out.writeUTF(getClass(i).getName());
+    }
+  }
+  
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    
+    // Get the number of "unknown" classes
+    
+    newClasses = in.readByte();
+    
+    // Then read in the class names and add them to our tables
+    
+    for (int i = 0; i < newClasses; i++) {
+      byte id = in.readByte();
+      String className = in.readUTF();
+      try {
+        addToMap(Class.forName(className), id);
+        
+      } catch (ClassNotFoundException e) {
+        throw new IOException("can't find class: " + className + " because "+
+            e.getMessage());
+      }
+    }
+  }    
+}

+ 94 - 0
src/java/org/apache/hadoop/io/ArrayFile.java

@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.*;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+
+
+/** A dense file-based mapping from integers to values. */
+public class ArrayFile extends MapFile {
+
+  protected ArrayFile() {}                            // no public ctor
+
+  /** Write a new array file. */
+  public static class Writer extends MapFile.Writer {
+    private LongWritable count = new LongWritable(0);
+
+    /** Create the named file for values of the named class. */
+    public Writer(Configuration conf, FileSystem fs,
+                  String file, Class<? extends Writable> valClass)
+      throws IOException {
+      super(conf, fs, file, LongWritable.class, valClass);
+    }
+
+    /** Create the named file for values of the named class. */
+    public Writer(Configuration conf, FileSystem fs,
+                  String file, Class<? extends Writable> valClass,
+                  CompressionType compress, Progressable progress)
+      throws IOException {
+      super(conf, fs, file, LongWritable.class, valClass, compress, progress);
+    }
+
+    /** Append a value to the file. */
+    public synchronized void append(Writable value) throws IOException {
+      super.append(count, value);                 // add to map
+      count.set(count.get()+1);                   // increment count
+    }
+  }
+
+  /** Provide access to an existing array file. */
+  public static class Reader extends MapFile.Reader {
+    private LongWritable key = new LongWritable();
+
+    /** Construct an array reader for the named file.*/
+    public Reader(FileSystem fs, String file, Configuration conf) throws IOException {
+      super(fs, file, conf);
+    }
+
+    /** Positions the reader before its <code>n</code>th value. */
+    public synchronized void seek(long n) throws IOException {
+      key.set(n);
+      seek(key);
+    }
+
+    /** Read and return the next value in the file. */
+    public synchronized Writable next(Writable value) throws IOException {
+      return next(key, value) ? value : null;
+    }
+
+    /** Returns the key associated with the most recent call to {@link
+     * #seek(long)}, {@link #next(Writable)}, or {@link
+     * #get(long,Writable)}. */
+    public synchronized long key() throws IOException {
+      return key.get();
+    }
+
+    /** Return the <code>n</code>th value in the file. */
+    public synchronized Writable get(long n, Writable value)
+      throws IOException {
+      key.set(n);
+      return get(key, value);
+    }
+  }
+
+}

+ 103 - 0
src/java/org/apache/hadoop/io/ArrayWritable.java

@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.*;
+import java.lang.reflect.Array;
+
+/** 
+ * A Writable for arrays containing instances of a class. The elements of this
+ * writable must all be instances of the same class. If this writable will be
+ * the input for a Reducer, you will need to create a subclass that sets the
+ * value to be of the proper type.
+ *
+ * For example:
+ * <code>
+ * public class IntArrayWritable extends ArrayWritable {
+ *   public IntArrayWritable() { 
+ *     super(IntWritable.class); 
+ *   }	
+ * }
+ * </code>
+ */
+public class ArrayWritable implements Writable {
+  private Class<? extends Writable> valueClass;
+  private Writable[] values;
+
+  public ArrayWritable(Class<? extends Writable> valueClass) {
+    if (valueClass == null) { 
+      throw new IllegalArgumentException("null valueClass"); 
+    }    
+    this.valueClass = valueClass;
+  }
+
+  public ArrayWritable(Class<? extends Writable> valueClass, Writable[] values) {
+    this(valueClass);
+    this.values = values;
+  }
+
+  public ArrayWritable(String[] strings) {
+    this(UTF8.class, new Writable[strings.length]);
+    for (int i = 0; i < strings.length; i++) {
+      values[i] = new UTF8(strings[i]);
+    }
+  }
+
+  public Class getValueClass() {
+    return valueClass;
+  }
+
+  public String[] toStrings() {
+    String[] strings = new String[values.length];
+    for (int i = 0; i < values.length; i++) {
+      strings[i] = values[i].toString();
+    }
+    return strings;
+  }
+
+  public Object toArray() {
+    Object result = Array.newInstance(valueClass, values.length);
+    for (int i = 0; i < values.length; i++) {
+      Array.set(result, i, values[i]);
+    }
+    return result;
+  }
+
+  public void set(Writable[] values) { this.values = values; }
+
+  public Writable[] get() { return values; }
+
+  public void readFields(DataInput in) throws IOException {
+    values = new Writable[in.readInt()];          // construct values
+    for (int i = 0; i < values.length; i++) {
+      Writable value = WritableFactories.newInstance(valueClass);
+      value.readFields(in);                       // read a value
+      values[i] = value;                          // store it in values
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(values.length);                 // write values
+    for (int i = 0; i < values.length; i++) {
+      values[i].write(out);
+    }
+  }
+
+}
+

+ 76 - 0
src/java/org/apache/hadoop/io/BinaryComparable.java

@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+/**
+ * Interface supported by {@link org.apache.hadoop.io.WritableComparable}
+ * types supporting ordering/permutation by a representative set of bytes.
+ */
+public abstract class BinaryComparable implements Comparable<BinaryComparable> {
+
+  /**
+   * Return n st bytes 0..n-1 from {#getBytes()} are valid.
+   */
+  public abstract int getLength();
+
+  /**
+   * Return representative byte array for this instance.
+   */
+  public abstract byte[] getBytes();
+
+  /**
+   * Compare bytes from {#getBytes()}.
+   * @see org.apache.hadoop.io.WritableComparator#compareBytes(byte[],int,int,byte[],int,int)
+   */
+  public int compareTo(BinaryComparable other) {
+    if (this == other)
+      return 0;
+    return WritableComparator.compareBytes(getBytes(), 0, getLength(),
+             other.getBytes(), 0, other.getLength());
+  }
+
+  /**
+   * Compare bytes from {#getBytes()} to those provided.
+   */
+  public int compareTo(byte[] other, int off, int len) {
+    return WritableComparator.compareBytes(getBytes(), 0, getLength(),
+             other, off, len);
+  }
+
+  /**
+   * Return true if bytes from {#getBytes()} match.
+   */
+  public boolean equals(Object other) {
+    if (!(other instanceof BinaryComparable))
+      return false;
+    BinaryComparable that = (BinaryComparable)other;
+    if (this.getLength() != that.getLength())
+      return false;
+    return this.compareTo(that) == 0;
+  }
+
+  /**
+   * Return a hash of the bytes returned from {#getBytes()}.
+   * @see org.apache.hadoop.io.WritableComparator#hashBytes(byte[],int)
+   */
+  public int hashCode() {
+    return WritableComparator.hashBytes(getBytes(), getLength());
+  }
+
+}

+ 259 - 0
src/java/org/apache/hadoop/io/BloomMapFile.java

@@ -0,0 +1,259 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.bloom.DynamicBloomFilter;
+import org.apache.hadoop.util.bloom.Filter;
+import org.apache.hadoop.util.bloom.Key;
+import org.apache.hadoop.util.hash.Hash;
+
+/**
+ * This class extends {@link MapFile} and provides very much the same
+ * functionality. However, it uses dynamic Bloom filters to provide
+ * quick membership test for keys, and it offers a fast version of 
+ * {@link Reader#get(WritableComparable, Writable)} operation, especially in
+ * case of sparsely populated MapFile-s.
+ */
+public class BloomMapFile {
+  private static final Log LOG = LogFactory.getLog(BloomMapFile.class);
+  public static final String BLOOM_FILE_NAME = "bloom";
+  public static final int HASH_COUNT = 5;
+  
+  public static void delete(FileSystem fs, String name) throws IOException {
+    Path dir = new Path(name);
+    Path data = new Path(dir, MapFile.DATA_FILE_NAME);
+    Path index = new Path(dir, MapFile.INDEX_FILE_NAME);
+    Path bloom = new Path(dir, BLOOM_FILE_NAME);
+
+    fs.delete(data, true);
+    fs.delete(index, true);
+    fs.delete(bloom, true);
+    fs.delete(dir, true);
+  }
+  
+  public static class Writer extends MapFile.Writer {
+    private DynamicBloomFilter bloomFilter;
+    private int numKeys;
+    private int vectorSize;
+    private Key bloomKey = new Key();
+    private DataOutputBuffer buf = new DataOutputBuffer();
+    private FileSystem fs;
+    private Path dir;
+    
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        Class<? extends WritableComparable> keyClass,
+        Class<? extends Writable> valClass, CompressionType compress,
+        CompressionCodec codec, Progressable progress) throws IOException {
+      super(conf, fs, dirName, keyClass, valClass, compress, codec, progress);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        Class<? extends WritableComparable> keyClass,
+        Class valClass, CompressionType compress,
+        Progressable progress) throws IOException {
+      super(conf, fs, dirName, keyClass, valClass, compress, progress);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        Class<? extends WritableComparable> keyClass,
+        Class valClass, CompressionType compress)
+        throws IOException {
+      super(conf, fs, dirName, keyClass, valClass, compress);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        WritableComparator comparator, Class valClass,
+        CompressionType compress, CompressionCodec codec, Progressable progress)
+        throws IOException {
+      super(conf, fs, dirName, comparator, valClass, compress, codec, progress);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        WritableComparator comparator, Class valClass,
+        CompressionType compress, Progressable progress) throws IOException {
+      super(conf, fs, dirName, comparator, valClass, compress, progress);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        WritableComparator comparator, Class valClass, CompressionType compress)
+        throws IOException {
+      super(conf, fs, dirName, comparator, valClass, compress);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        WritableComparator comparator, Class valClass) throws IOException {
+      super(conf, fs, dirName, comparator, valClass);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+        Class<? extends WritableComparable> keyClass,
+        Class valClass) throws IOException {
+      super(conf, fs, dirName, keyClass, valClass);
+      this.fs = fs;
+      this.dir = new Path(dirName);
+      initBloomFilter(conf);
+    }
+
+    private synchronized void initBloomFilter(Configuration conf) {
+      numKeys = conf.getInt("io.mapfile.bloom.size", 1024 * 1024);
+      // vector size should be <code>-kn / (ln(1 - c^(1/k)))</code> bits for
+      // single key, where <code> is the number of hash functions,
+      // <code>n</code> is the number of keys and <code>c</code> is the desired
+      // max. error rate.
+      // Our desired error rate is by default 0.005, i.e. 0.5%
+      float errorRate = conf.getFloat("io.mapfile.bloom.error.rate", 0.005f);
+      vectorSize = (int)Math.ceil((double)(-HASH_COUNT * numKeys) /
+          Math.log(1.0 - Math.pow(errorRate, 1.0/HASH_COUNT)));
+      bloomFilter = new DynamicBloomFilter(vectorSize, HASH_COUNT,
+          Hash.getHashType(conf), numKeys);
+    }
+
+    @Override
+    public synchronized void append(WritableComparable key, Writable val)
+        throws IOException {
+      super.append(key, val);
+      buf.reset();
+      key.write(buf);
+      bloomKey.set(buf.getData(), 1.0);
+      bloomFilter.add(bloomKey);
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+      super.close();
+      DataOutputStream out = fs.create(new Path(dir, BLOOM_FILE_NAME), true);
+      bloomFilter.write(out);
+      out.flush();
+      out.close();
+    }
+
+  }
+  
+  public static class Reader extends MapFile.Reader {
+    private DynamicBloomFilter bloomFilter;
+    private DataOutputBuffer buf = new DataOutputBuffer();
+    private Key bloomKey = new Key();
+
+    public Reader(FileSystem fs, String dirName, Configuration conf)
+        throws IOException {
+      super(fs, dirName, conf);
+      initBloomFilter(fs, dirName, conf);
+    }
+
+    public Reader(FileSystem fs, String dirName, WritableComparator comparator,
+        Configuration conf, boolean open) throws IOException {
+      super(fs, dirName, comparator, conf, open);
+      initBloomFilter(fs, dirName, conf);
+    }
+
+    public Reader(FileSystem fs, String dirName, WritableComparator comparator,
+        Configuration conf) throws IOException {
+      super(fs, dirName, comparator, conf);
+      initBloomFilter(fs, dirName, conf);
+    }
+    
+    private void initBloomFilter(FileSystem fs, String dirName,
+        Configuration conf) {
+      try {
+        DataInputStream in = fs.open(new Path(dirName, BLOOM_FILE_NAME));
+        bloomFilter = new DynamicBloomFilter();
+        bloomFilter.readFields(in);
+        in.close();
+      } catch (IOException ioe) {
+        LOG.warn("Can't open BloomFilter: " + ioe + " - fallback to MapFile.");
+        bloomFilter = null;
+      }
+    }
+    
+    /**
+     * Checks if this MapFile has the indicated key. The membership test is
+     * performed using a Bloom filter, so the result has always non-zero
+     * probability of false positives.
+     * @param key key to check
+     * @return  false iff key doesn't exist, true if key probably exists.
+     * @throws IOException
+     */
+    public boolean probablyHasKey(WritableComparable key) throws IOException {
+      if (bloomFilter == null) {
+        return true;
+      }
+      buf.reset();
+      key.write(buf);
+      bloomKey.set(buf.getData(), 1.0);
+      return bloomFilter.membershipTest(bloomKey);
+    }
+    
+    /**
+     * Fast version of the
+     * {@link MapFile.Reader#get(WritableComparable, Writable)} method. First
+     * it checks the Bloom filter for the existence of the key, and only if
+     * present it performs the real get operation. This yields significant
+     * performance improvements for get operations on sparsely populated files.
+     */
+    @Override
+    public synchronized Writable get(WritableComparable key, Writable val)
+        throws IOException {
+      if (!probablyHasKey(key)) {
+        return null;
+      }
+      return super.get(key, val);
+    }
+    
+    /**
+     * Retrieve the Bloom filter used by this instance of the Reader.
+     * @return a Bloom filter (see {@link Filter})
+     */
+    public Filter getBloomFilter() {
+      return bloomFilter;
+    }
+  }
+}

+ 111 - 0
src/java/org/apache/hadoop/io/BooleanWritable.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.*;
+
+/** 
+ * A WritableComparable for booleans. 
+ */
+public class BooleanWritable implements WritableComparable {
+  private boolean value;
+
+  /** 
+   */
+  public BooleanWritable() {};
+
+  /** 
+   */
+  public BooleanWritable(boolean value) {
+    set(value);
+  }
+
+  /** 
+   * Set the value of the BooleanWritable
+   */    
+  public void set(boolean value) {
+    this.value = value;
+  }
+
+  /**
+   * Returns the value of the BooleanWritable
+   */
+  public boolean get() {
+    return value;
+  }
+
+  /**
+   */
+  public void readFields(DataInput in) throws IOException {
+    value = in.readBoolean();
+  }
+
+  /**
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(value);
+  }
+
+  /**
+   */
+  public boolean equals(Object o) {
+    if (!(o instanceof BooleanWritable)) {
+      return false;
+    }
+    BooleanWritable other = (BooleanWritable) o;
+    return this.value == other.value;
+  }
+
+  public int hashCode() {
+    return value ? 0 : 1;
+  }
+
+
+
+  /**
+   */
+  public int compareTo(Object o) {
+    boolean a = this.value;
+    boolean b = ((BooleanWritable) o).value;
+    return ((a == b) ? 0 : (a == false) ? -1 : 1);
+  }
+  
+  public String toString() {
+    return Boolean.toString(get());
+  }
+
+  /** 
+   * A Comparator optimized for BooleanWritable. 
+   */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(BooleanWritable.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      return compareBytes(b1, s1, l1, b2, s2, l2);
+    }
+  }
+
+
+  static {
+    WritableComparator.define(BooleanWritable.class, new Comparator());
+  }
+}

+ 87 - 0
src/java/org/apache/hadoop/io/ByteWritable.java

@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.*;
+
+/** A WritableComparable for a single byte. */
+public class ByteWritable implements WritableComparable {
+  private byte value;
+
+  public ByteWritable() {}
+
+  public ByteWritable(byte value) { set(value); }
+
+  /** Set the value of this ByteWritable. */
+  public void set(byte value) { this.value = value; }
+
+  /** Return the value of this ByteWritable. */
+  public byte get() { return value; }
+
+  public void readFields(DataInput in) throws IOException {
+    value = in.readByte();
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeByte(value);
+  }
+
+  /** Returns true iff <code>o</code> is a ByteWritable with the same value. */
+  public boolean equals(Object o) {
+    if (!(o instanceof ByteWritable)) {
+      return false;
+    }
+    ByteWritable other = (ByteWritable)o;
+    return this.value == other.value;
+  }
+
+  public int hashCode() {
+    return (int)value;
+  }
+
+  /** Compares two ByteWritables. */
+  public int compareTo(Object o) {
+    int thisValue = this.value;
+    int thatValue = ((ByteWritable)o).value;
+    return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
+  }
+
+  public String toString() {
+    return Byte.toString(value);
+  }
+
+  /** A Comparator optimized for ByteWritable. */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(ByteWritable.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      byte thisValue = b1[s1];
+      byte thatValue = b2[s2];
+      return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
+    }
+  }
+
+  static {                                        // register this comparator
+    WritableComparator.define(ByteWritable.class, new Comparator());
+  }
+}
+

+ 216 - 0
src/java/org/apache/hadoop/io/BytesWritable.java

@@ -0,0 +1,216 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/** 
+ * A byte sequence that is usable as a key or value.
+ * It is resizable and distinguishes between the size of the seqeunce and
+ * the current capacity. The hash function is the front of the md5 of the 
+ * buffer. The sort order is the same as memcmp.
+ */
+public class BytesWritable extends BinaryComparable
+    implements WritableComparable<BinaryComparable> {
+  private static final Log LOG = LogFactory.getLog(BytesWritable.class);
+  private static final int LENGTH_BYTES = 4;
+  private static final byte[] EMPTY_BYTES = {};
+
+  private int size;
+  private byte[] bytes;
+  
+  /**
+   * Create a zero-size sequence.
+   */
+  public BytesWritable() {this(EMPTY_BYTES);}
+  
+  /**
+   * Create a BytesWritable using the byte array as the initial value.
+   * @param bytes This array becomes the backing storage for the object.
+   */
+  public BytesWritable(byte[] bytes) {
+    this.bytes = bytes;
+    this.size = bytes.length;
+  }
+  
+  /**
+   * Get the data from the BytesWritable.
+   * @return The data is only valid between 0 and getLength() - 1.
+   */
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /**
+   * Get the data from the BytesWritable.
+   * @deprecated Use {@link #getBytes()} instead.
+   */
+  @Deprecated
+  public byte[] get() {
+    return getBytes();
+  }
+
+  /**
+   * Get the current size of the buffer.
+   */
+  public int getLength() {
+    return size;
+  }
+
+  /**
+   * Get the current size of the buffer.
+   * @deprecated Use {@link #getLength()} instead.
+   */
+  @Deprecated
+  public int getSize() {
+    return getLength();
+  }
+  
+  /**
+   * Change the size of the buffer. The values in the old range are preserved
+   * and any new values are undefined. The capacity is changed if it is 
+   * necessary.
+   * @param size The new number of bytes
+   */
+  public void setSize(int size) {
+    if (size > getCapacity()) {
+      setCapacity(size * 3 / 2);
+    }
+    this.size = size;
+  }
+  
+  /**
+   * Get the capacity, which is the maximum size that could handled without
+   * resizing the backing storage.
+   * @return The number of bytes
+   */
+  public int getCapacity() {
+    return bytes.length;
+  }
+  
+  /**
+   * Change the capacity of the backing storage.
+   * The data is preserved.
+   * @param new_cap The new capacity in bytes.
+   */
+  public void setCapacity(int new_cap) {
+    if (new_cap != getCapacity()) {
+      byte[] new_data = new byte[new_cap];
+      if (new_cap < size) {
+        size = new_cap;
+      }
+      if (size != 0) {
+        System.arraycopy(bytes, 0, new_data, 0, size);
+      }
+      bytes = new_data;
+    }
+  }
+
+  /**
+   * Set the BytesWritable to the contents of the given newData.
+   * @param newData the value to set this BytesWritable to.
+   */
+  public void set(BytesWritable newData) {
+    set(newData.bytes, 0, newData.size);
+  }
+
+  /**
+   * Set the value to a copy of the given byte range
+   * @param newData the new values to copy in
+   * @param offset the offset in newData to start at
+   * @param length the number of bytes to copy
+   */
+  public void set(byte[] newData, int offset, int length) {
+    setSize(0);
+    setSize(length);
+    System.arraycopy(newData, offset, bytes, 0, size);
+  }
+
+  // inherit javadoc
+  public void readFields(DataInput in) throws IOException {
+    setSize(0); // clear the old data
+    setSize(in.readInt());
+    in.readFully(bytes, 0, size);
+  }
+  
+  // inherit javadoc
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(size);
+    out.write(bytes, 0, size);
+  }
+  
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  /**
+   * Are the two byte sequences equal?
+   */
+  public boolean equals(Object right_obj) {
+    if (right_obj instanceof BytesWritable)
+      return super.equals(right_obj);
+    return false;
+  }
+
+  /**
+   * Generate the stream of bytes as hex pairs separated by ' '.
+   */
+  public String toString() { 
+    StringBuffer sb = new StringBuffer(3*size);
+    for (int idx = 0; idx < size; idx++) {
+      // if not the first, put a blank separator in
+      if (idx != 0) {
+        sb.append(' ');
+      }
+      String num = Integer.toHexString(0xff & bytes[idx]);
+      // if it is only one digit, add a leading 0.
+      if (num.length() < 2) {
+        sb.append('0');
+      }
+      sb.append(num);
+    }
+    return sb.toString();
+  }
+
+  /** A Comparator optimized for BytesWritable. */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(BytesWritable.class);
+    }
+    
+    /**
+     * Compare the buffers in serialized form.
+     */
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      return compareBytes(b1, s1+LENGTH_BYTES, l1-LENGTH_BYTES, 
+                          b2, s2+LENGTH_BYTES, l2-LENGTH_BYTES);
+    }
+  }
+  
+  static {                                        // register this comparator
+    WritableComparator.define(BytesWritable.class, new Comparator());
+  }
+  
+}

+ 24 - 0
src/java/org/apache/hadoop/io/Closeable.java

@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+/** @deprecated use java.io.Closeable */
+@Deprecated
+public interface Closeable extends java.io.Closeable{
+}

+ 86 - 0
src/java/org/apache/hadoop/io/CompressedWritable.java

@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.DataInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+/** A base-class for Writables which store themselves compressed and lazily
+ * inflate on field access.  This is useful for large objects whose fields are
+ * not be altered during a map or reduce operation: leaving the field data
+ * compressed makes copying the instance from one file to another much
+ * faster. */
+public abstract class CompressedWritable implements Writable {
+  // if non-null, the compressed field data of this instance.
+  private byte[] compressed;
+
+  public CompressedWritable() {}
+
+  public final void readFields(DataInput in) throws IOException {
+    compressed = new byte[in.readInt()];
+    in.readFully(compressed, 0, compressed.length);
+  }
+
+  /** Must be called by all methods which access fields to ensure that the data
+   * has been uncompressed. */
+  protected void ensureInflated() {
+    if (compressed != null) {
+      try {
+        ByteArrayInputStream deflated = new ByteArrayInputStream(compressed);
+        DataInput inflater =
+          new DataInputStream(new InflaterInputStream(deflated));
+        readFieldsCompressed(inflater);
+        compressed = null;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  /** Subclasses implement this instead of {@link #readFields(DataInput)}. */
+  protected abstract void readFieldsCompressed(DataInput in)
+    throws IOException;
+
+  public final void write(DataOutput out) throws IOException {
+    if (compressed == null) {
+      ByteArrayOutputStream deflated = new ByteArrayOutputStream();
+      Deflater deflater = new Deflater(Deflater.BEST_SPEED);
+      DataOutputStream dout =
+        new DataOutputStream(new DeflaterOutputStream(deflated, deflater));
+      writeCompressed(dout);
+      dout.close();
+      deflater.end();
+      compressed = deflated.toByteArray();
+    }
+    out.writeInt(compressed.length);
+    out.write(compressed);
+  }
+
+  /** Subclasses implement this instead of {@link #write(DataOutput)}. */
+  protected abstract void writeCompressed(DataOutput out) throws IOException;
+
+}

+ 91 - 0
src/java/org/apache/hadoop/io/DataInputBuffer.java

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.*;
+
+/** A reusable {@link DataInput} implementation that reads from an in-memory
+ * buffer.
+ *
+ * <p>This saves memory over creating a new DataInputStream and
+ * ByteArrayInputStream each time data is read.
+ *
+ * <p>Typical usage is something like the following:<pre>
+ *
+ * DataInputBuffer buffer = new DataInputBuffer();
+ * while (... loop condition ...) {
+ *   byte[] data = ... get data ...;
+ *   int dataLength = ... get data length ...;
+ *   buffer.reset(data, dataLength);
+ *   ... read buffer using DataInput methods ...
+ * }
+ * </pre>
+ *  
+ */
+public class DataInputBuffer extends DataInputStream {
+  private static class Buffer extends ByteArrayInputStream {
+    public Buffer() {
+      super(new byte[] {});
+    }
+
+    public void reset(byte[] input, int start, int length) {
+      this.buf = input;
+      this.count = start+length;
+      this.mark = start;
+      this.pos = start;
+    }
+
+    public byte[] getData() { return buf; }
+    public int getPosition() { return pos; }
+    public int getLength() { return count; }
+  }
+
+  private Buffer buffer;
+  
+  /** Constructs a new empty buffer. */
+  public DataInputBuffer() {
+    this(new Buffer());
+  }
+
+  private DataInputBuffer(Buffer buffer) {
+    super(buffer);
+    this.buffer = buffer;
+  }
+
+  /** Resets the data that the buffer reads. */
+  public void reset(byte[] input, int length) {
+    buffer.reset(input, 0, length);
+  }
+
+  /** Resets the data that the buffer reads. */
+  public void reset(byte[] input, int start, int length) {
+    buffer.reset(input, start, length);
+  }
+  
+  public byte[] getData() {
+    return buffer.getData();
+  }
+
+  /** Returns the current position in the input. */
+  public int getPosition() { return buffer.getPosition(); }
+
+  /** Returns the length of the input. */
+  public int getLength() { return buffer.getLength(); }
+
+}

+ 108 - 0
src/java/org/apache/hadoop/io/DataOutputBuffer.java

@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.*;
+
+/** A reusable {@link DataOutput} implementation that writes to an in-memory
+ * buffer.
+ *
+ * <p>This saves memory over creating a new DataOutputStream and
+ * ByteArrayOutputStream each time data is written.
+ *
+ * <p>Typical usage is something like the following:<pre>
+ *
+ * DataOutputBuffer buffer = new DataOutputBuffer();
+ * while (... loop condition ...) {
+ *   buffer.reset();
+ *   ... write buffer using DataOutput methods ...
+ *   byte[] data = buffer.getData();
+ *   int dataLength = buffer.getLength();
+ *   ... write data to its ultimate destination ...
+ * }
+ * </pre>
+ *  
+ */
+public class DataOutputBuffer extends DataOutputStream {
+
+  private static class Buffer extends ByteArrayOutputStream {
+    public byte[] getData() { return buf; }
+    public int getLength() { return count; }
+
+    public Buffer() {
+      super();
+    }
+    
+    public Buffer(int size) {
+      super(size);
+    }
+    
+    public void write(DataInput in, int len) throws IOException {
+      int newcount = count + len;
+      if (newcount > buf.length) {
+        byte newbuf[] = new byte[Math.max(buf.length << 1, newcount)];
+        System.arraycopy(buf, 0, newbuf, 0, count);
+        buf = newbuf;
+      }
+      in.readFully(buf, count, len);
+      count = newcount;
+    }
+  }
+
+  private Buffer buffer;
+  
+  /** Constructs a new empty buffer. */
+  public DataOutputBuffer() {
+    this(new Buffer());
+  }
+  
+  public DataOutputBuffer(int size) {
+    this(new Buffer(size));
+  }
+  
+  private DataOutputBuffer(Buffer buffer) {
+    super(buffer);
+    this.buffer = buffer;
+  }
+
+  /** Returns the current contents of the buffer.
+   *  Data is only valid to {@link #getLength()}.
+   */
+  public byte[] getData() { return buffer.getData(); }
+
+  /** Returns the length of the valid data currently in the buffer. */
+  public int getLength() { return buffer.getLength(); }
+
+  /** Resets the buffer to empty. */
+  public DataOutputBuffer reset() {
+    this.written = 0;
+    buffer.reset();
+    return this;
+  }
+
+  /** Writes bytes from a DataInput directly into the buffer. */
+  public void write(DataInput in, int length) throws IOException {
+    buffer.write(in, length);
+  }
+
+  /** Write to a file stream */
+  public void writeTo(OutputStream out) throws IOException {
+    buffer.writeTo(out);
+  }
+}

+ 199 - 0
src/java/org/apache/hadoop/io/DefaultStringifier.java

@@ -0,0 +1,199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.IOException;
+import java.nio.charset.UnsupportedCharsetException;
+import java.util.ArrayList;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.Serialization;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.util.GenericsUtil;
+
+/**
+ * DefaultStringifier is the default implementation of the {@link Stringifier}
+ * interface which stringifies the objects using base64 encoding of the
+ * serialized version of the objects. The {@link Serializer} and
+ * {@link Deserializer} are obtained from the {@link SerializationFactory}.
+ * <br>
+ * DefaultStringifier offers convenience methods to store/load objects to/from
+ * the configuration.
+ * 
+ * @param <T> the class of the objects to stringify
+ */
+public class DefaultStringifier<T> implements Stringifier<T> {
+
+  private static final String SEPARATOR = ",";
+
+  private Serializer<T> serializer;
+
+  private Deserializer<T> deserializer;
+
+  private DataInputBuffer inBuf;
+
+  private DataOutputBuffer outBuf;
+
+  public DefaultStringifier(Configuration conf, Class<T> c) {
+
+    SerializationFactory factory = new SerializationFactory(conf);
+    this.serializer = factory.getSerializer(c);
+    this.deserializer = factory.getDeserializer(c);
+    this.inBuf = new DataInputBuffer();
+    this.outBuf = new DataOutputBuffer();
+    try {
+      serializer.open(outBuf);
+      deserializer.open(inBuf);
+    } catch (IOException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  public T fromString(String str) throws IOException {
+    try {
+      byte[] bytes = Base64.decodeBase64(str.getBytes("UTF-8"));
+      inBuf.reset(bytes, bytes.length);
+      T restored = deserializer.deserialize(null);
+      return restored;
+    } catch (UnsupportedCharsetException ex) {
+      throw new IOException(ex.toString());
+    }
+  }
+
+  public String toString(T obj) throws IOException {
+    outBuf.reset();
+    serializer.serialize(obj);
+    byte[] buf = new byte[outBuf.getLength()];
+    System.arraycopy(outBuf.getData(), 0, buf, 0, buf.length);
+    return new String(Base64.encodeBase64(buf));
+  }
+
+  public void close() throws IOException {
+    inBuf.close();
+    outBuf.close();
+    deserializer.close();
+    serializer.close();
+  }
+
+  /**
+   * Stores the item in the configuration with the given keyName.
+   * 
+   * @param <K>  the class of the item
+   * @param conf the configuration to store
+   * @param item the object to be stored
+   * @param keyName the name of the key to use
+   * @throws IOException : forwards Exceptions from the underlying 
+   * {@link Serialization} classes. 
+   */
+  public static <K> void store(Configuration conf, K item, String keyName)
+  throws IOException {
+
+    DefaultStringifier<K> stringifier = new DefaultStringifier<K>(conf,
+        GenericsUtil.getClass(item));
+    conf.set(keyName, stringifier.toString(item));
+    stringifier.close();
+  }
+
+  /**
+   * Restores the object from the configuration.
+   * 
+   * @param <K> the class of the item
+   * @param conf the configuration to use
+   * @param keyName the name of the key to use
+   * @param itemClass the class of the item
+   * @return restored object
+   * @throws IOException : forwards Exceptions from the underlying 
+   * {@link Serialization} classes.
+   */
+  public static <K> K load(Configuration conf, String keyName,
+      Class<K> itemClass) throws IOException {
+    DefaultStringifier<K> stringifier = new DefaultStringifier<K>(conf,
+        itemClass);
+    try {
+      String itemStr = conf.get(keyName);
+      return stringifier.fromString(itemStr);
+    } finally {
+      stringifier.close();
+    }
+  }
+
+  /**
+   * Stores the array of items in the configuration with the given keyName.
+   * 
+   * @param <K> the class of the item
+   * @param conf the configuration to use 
+   * @param items the objects to be stored
+   * @param keyName the name of the key to use
+   * @throws IndexOutOfBoundsException if the items array is empty
+   * @throws IOException : forwards Exceptions from the underlying 
+   * {@link Serialization} classes.         
+   */
+  public static <K> void storeArray(Configuration conf, K[] items,
+      String keyName) throws IOException {
+
+    DefaultStringifier<K> stringifier = new DefaultStringifier<K>(conf, 
+        GenericsUtil.getClass(items[0]));
+    try {
+      StringBuilder builder = new StringBuilder();
+      for (K item : items) {
+        builder.append(stringifier.toString(item)).append(SEPARATOR);
+      }
+      conf.set(keyName, builder.toString());
+    }
+    finally {
+      stringifier.close();
+    }
+  }
+
+  /**
+   * Restores the array of objects from the configuration.
+   * 
+   * @param <K> the class of the item
+   * @param conf the configuration to use
+   * @param keyName the name of the key to use
+   * @param itemClass the class of the item
+   * @return restored object
+   * @throws IOException : forwards Exceptions from the underlying 
+   * {@link Serialization} classes.
+   */
+  public static <K> K[] loadArray(Configuration conf, String keyName,
+      Class<K> itemClass) throws IOException {
+    DefaultStringifier<K> stringifier = new DefaultStringifier<K>(conf,
+        itemClass);
+    try {
+      String itemStr = conf.get(keyName);
+      ArrayList<K> list = new ArrayList<K>();
+      String[] parts = itemStr.split(SEPARATOR);
+
+      for (String part : parts) {
+        if (!part.equals(""))
+          list.add(stringifier.fromString(part));
+      }
+
+      return GenericsUtil.toArray(itemClass, list);
+    }
+    finally {
+      stringifier.close();
+    }
+  }
+
+}

+ 60 - 0
src/java/org/apache/hadoop/io/DeprecatedUTF8.java

@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Wrapper for {@link UTF8}.
+ * This class should be used only when it is absolutely necessary
+ * to use {@link UTF8}. The only difference is that using this class
+ * does not require "@SuppressWarning" annotation to avoid javac warning. 
+ * Instead the deprecation is implied in the class name.
+ */
+@SuppressWarnings("deprecation")
+public class DeprecatedUTF8 extends UTF8 {
+  
+  public DeprecatedUTF8() {
+    super();
+  }
+
+  /** Construct from a given string. */
+  public DeprecatedUTF8(String string) {
+    super(string);
+  }
+
+  /** Construct from a given string. */
+  public DeprecatedUTF8(DeprecatedUTF8 utf8) {
+    super(utf8);
+  }
+  
+  /* The following two are the mostly commonly used methods.
+   * wrapping them so that editors do not complain about the deprecation.
+   */
+  
+  public static String readString(DataInput in) throws IOException {
+    return UTF8.readString(in);
+  }
+  
+  public static int writeString(DataOutput out, String s) throws IOException {
+    return UTF8.writeString(out, s);
+  }
+}

+ 95 - 0
src/java/org/apache/hadoop/io/DoubleWritable.java

@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Writable for Double values.
+ */
+public class DoubleWritable implements WritableComparable {
+
+  private double value = 0.0;
+  
+  public DoubleWritable() {
+    
+  }
+  
+  public DoubleWritable(double value) {
+    set(value);
+  }
+  
+  public void readFields(DataInput in) throws IOException {
+    value = in.readDouble();
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeDouble(value);
+  }
+  
+  public void set(double value) { this.value = value; }
+  
+  public double get() { return value; }
+
+  /**
+   * Returns true iff <code>o</code> is a DoubleWritable with the same value.
+   */
+  public boolean equals(Object o) {
+    if (!(o instanceof DoubleWritable)) {
+      return false;
+    }
+    DoubleWritable other = (DoubleWritable)o;
+    return this.value == other.value;
+  }
+  
+  public int hashCode() {
+    return (int)Double.doubleToLongBits(value);
+  }
+  
+  public int compareTo(Object o) {
+    DoubleWritable other = (DoubleWritable)o;
+    return (value < other.value ? -1 : (value == other.value ? 0 : 1));
+  }
+  
+  public String toString() {
+    return Double.toString(value);
+  }
+
+  /** A Comparator optimized for DoubleWritable. */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(DoubleWritable.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      double thisValue = readDouble(b1, s1);
+      double thatValue = readDouble(b2, s2);
+      return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
+    }
+  }
+
+  static {                                        // register this comparator
+    WritableComparator.define(DoubleWritable.class, new Comparator());
+  }
+
+}
+

+ 202 - 0
src/java/org/apache/hadoop/io/EnumSetWritable.java

@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/** A Writable wrapper for EnumSet. */
+public class EnumSetWritable<E extends Enum<E>> implements Writable,
+    Configurable {
+
+  private EnumSet<E> value;
+
+  private Class<E> elementType;
+
+  private Configuration conf;
+
+  EnumSetWritable() {
+  }
+
+  /**
+   * Construct a new EnumSetWritable. If the <tt>value</tt> argument is null or
+   * its size is zero, the <tt>elementType</tt> argument must not be null. If
+   * the argument <tt>value</tt>'s size is bigger than zero, the argument
+   * <tt>elementType</tt> is not be used.
+   * 
+   * @param value
+   * @param elementType
+   */
+  public EnumSetWritable(EnumSet<E> value, Class<E> elementType) {
+    set(value, elementType);
+  }
+
+  /**
+   * Construct a new EnumSetWritable. Argument <tt>value</tt> should not be null
+   * or empty.
+   * 
+   * @param value
+   */
+  public EnumSetWritable(EnumSet<E> value) {
+    this(value, null);
+  }
+
+  /**
+   * reset the EnumSetWritable with specified
+   * <tt>value</value> and <tt>elementType</tt>. If the <tt>value</tt> argument
+   * is null or its size is zero, the <tt>elementType</tt> argument must not be
+   * null. If the argument <tt>value</tt>'s size is bigger than zero, the
+   * argument <tt>elementType</tt> is not be used.
+   * 
+   * @param value
+   * @param elementType
+   */
+  public void set(EnumSet<E> value, Class<E> elementType) {
+    if ((value == null || value.size() == 0)
+        && (this.elementType == null && elementType == null)) {
+      throw new IllegalArgumentException(
+          "The EnumSet argument is null, or is an empty set but with no elementType provided.");
+    }
+    this.value = value;
+    if (value != null && value.size() > 0) {
+      Iterator<E> iterator = value.iterator();
+      this.elementType = iterator.next().getDeclaringClass();
+    } else if (elementType != null) {
+      this.elementType = elementType;
+    }
+  }
+
+  /** Return the value of this EnumSetWritable. */
+  public EnumSet<E> get() {
+    return value;
+  }
+
+  /** {@inheritDoc} */
+  @SuppressWarnings("unchecked")
+  public void readFields(DataInput in) throws IOException {
+    int length = in.readInt();
+    if (length == -1)
+      this.value = null;
+    else if (length == 0) {
+      this.elementType = (Class<E>) ObjectWritable.loadClass(conf,
+          WritableUtils.readString(in));
+      this.value = EnumSet.noneOf(this.elementType);
+    } else {
+      E first = (E) ObjectWritable.readObject(in, conf);
+      this.value = (EnumSet<E>) EnumSet.of(first);
+      for (int i = 1; i < length; i++)
+        this.value.add((E) ObjectWritable.readObject(in, conf));
+    }
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    if (this.value == null) {
+      out.writeInt(-1);
+      WritableUtils.writeString(out, this.elementType.getName());
+    } else {
+      Object[] array = this.value.toArray();
+      int length = array.length;
+      out.writeInt(length);
+      if (length == 0) {
+        if (this.elementType == null)
+          throw new UnsupportedOperationException(
+              "Unable to serialize empty EnumSet with no element type provided.");
+        WritableUtils.writeString(out, this.elementType.getName());
+      }
+      for (int i = 0; i < length; i++) {
+        ObjectWritable.writeObject(out, array[i], array[i].getClass(), conf);
+      }
+    }
+  }
+
+  /**
+   * Returns true if <code>o</code> is an EnumSetWritable with the same value,
+   * or both are null.
+   */
+  public boolean equals(Object o) {
+    if (o == null) {
+      throw new IllegalArgumentException("null argument passed in equal().");
+    }
+
+    if (!(o instanceof EnumSetWritable))
+      return false;
+
+    EnumSetWritable<?> other = (EnumSetWritable<?>) o;
+
+    if (this == o || (this.value == other.value))
+      return true;
+    if (this.value == null) // other.value must not be null if we reach here
+      return false;
+
+    return this.value.equals(other.value);
+  }
+
+  /**
+   * Returns the class of all the elements of the underlying EnumSetWriable. It
+   * may return null.
+   * 
+   * @return the element class
+   */
+  public Class<E> getElementType() {
+    return elementType;
+  }
+
+  /** {@inheritDoc} */
+  public int hashCode() {
+    if (value == null)
+      return 0;
+    return (int) value.hashCode();
+  }
+
+  /** {@inheritDoc} */
+  public String toString() {
+    if (value == null)
+      return "(null)";
+    return value.toString();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  static {
+    WritableFactories.setFactory(EnumSetWritable.class, new WritableFactory() {
+      @SuppressWarnings("unchecked")
+      @Override
+      public Writable newInstance() {
+        return new EnumSetWritable();
+      }
+    });
+  }
+}

Vissa filer visades inte eftersom för många filer har ändrats