Prechádzať zdrojové kódy

Merge trunk into QJM branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3077@1383030 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 12 rokov pred
rodič
commit
1cea56d6ef
100 zmenil súbory, kde vykonal 8017 pridanie a 601 odobranie
  1. 0 4
      hadoop-client/pom.xml
  2. 34 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 14 0
      hadoop-common-project/hadoop-common/pom.xml
  4. 8 0
      hadoop-common-project/hadoop-common/src/CMakeLists.txt
  5. 8 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  6. 20 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java
  7. 3 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
  8. 19 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
  9. 41 18
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c
  10. 35 5
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.h
  11. 77 0
      hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c
  12. 30 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  13. 12 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
  14. 4 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
  15. 0 4
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  16. 32 2
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  17. 2 1
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  18. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
  19. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/dev-support/findbugsExcludeFile.xml
  20. 84 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
  22. 45 28
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  23. 21 26
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/CurrentInprogress.java
  24. 49 39
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
  25. 19 6
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/MaxTxId.java
  26. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto
  27. 16 4
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperConfiguration.java
  28. 68 25
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java
  29. 95 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt
  30. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake
  31. 237 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/exception.c
  32. 178 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/exception.h
  33. 101 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/expect.h
  34. 352 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.c
  35. 74 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.h
  36. 254 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.c
  37. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.h
  38. 616 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_jni.c
  39. 388 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c
  40. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.h
  41. 1113 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c
  42. 609 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/jni_helper.c
  43. 122 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/jni_helper.h
  44. 180 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_multi_write.c
  45. 504 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_ops.c
  46. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_read.c
  47. 225 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_threaded.c
  48. 118 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_write.c
  49. 110 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_read_bm.c
  50. 694 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/webhdfs.h
  51. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
  52. 20 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  53. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  54. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  55. 26 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  56. 18 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  57. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  58. 251 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java
  59. 93 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  60. 0 54
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java
  61. 80 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMultipleNNDataBlockScanner.java
  62. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
  63. 73 60
      hadoop-mapreduce-project/CHANGES.txt
  64. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  65. 7 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  66. 0 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
  67. 15 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java
  68. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  69. 12 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
  70. 14 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  71. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  72. 4 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
  73. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  74. 6 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRClientClusterFactory.java
  75. 5 0
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
  76. 4 0
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
  77. 1 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
  78. 14 2
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
  79. 18 0
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
  80. 27 7
      hadoop-yarn-project/CHANGES.txt
  81. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
  82. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  83. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java
  84. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
  85. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
  86. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  87. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
  88. 32 63
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
  89. 8 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
  90. 145 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
  91. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  92. 54 57
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
  93. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
  94. 0 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregator.java
  95. 5 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
  96. 33 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
  97. 31 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
  98. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
  99. 15 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
  100. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

+ 0 - 4
hadoop-client/pom.xml

@@ -39,10 +39,6 @@
       <artifactId>hadoop-common</artifactId>
       <scope>compile</scope>
       <exclusions>
-        <exclusion>
-          <groupId>commons-cli</groupId>
-          <artifactId>commons-cli</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>commons-httpclient</groupId>
           <artifactId>commons-httpclient</artifactId>

+ 34 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -1,6 +1,6 @@
 Hadoop Change Log
 
-Trunk (unreleased changes)
+Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
 
@@ -206,7 +206,19 @@ Trunk (unreleased changes)
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
 
-Branch-2 ( Unreleased changes )
+Release 2.0.3-alpha - Unreleased 
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
 
@@ -328,6 +340,9 @@ Branch-2 ( Unreleased changes )
     HADOOP-8748. Refactor DFSClient retry utility methods to a new class
     in org.apache.hadoop.io.retry.  (Arun C Murthy via szetszwo)
 
+    HADOOP-8754. Deprecate all the RPC.getServer() variants.  (Brandon Li
+    via szetszwo)
+
   BUG FIXES
 
     HADOOP-8372. NetUtils.normalizeHostName() incorrectly handles hostname
@@ -465,6 +480,23 @@ Branch-2 ( Unreleased changes )
 
     HADOOP-8764. CMake: HADOOP-8737 broke ARM build. (Trevor Robinson via eli)
 
+    HADOOP-8770. NN should not RPC to self to find trash defaults. (eli)
+
+    HADOOP-8648. libhadoop: native CRC32 validation crashes when
+    io.bytes.per.checksum=1. (Colin Patrick McCabe via eli)
+
+    HADOOP-8766. FileContextMainOperationsBaseTest should randomize the root
+    dir. (Colin Patrick McCabe via atm)
+
+    HADOOP-8749. HADOOP-8031 changed the way in which relative xincludes are handled in 
+    Configuration. (ahmed via tucu)
+
+    HADOOP-8431. Running distcp wo args throws IllegalArgumentException.
+    (Sandy Ryza via eli)
+
+    HADOOP-8775. MR2 distcp permits non-positive value to -bandwidth option
+    which causes job never to complete. (Sandy Ryza via atm)
+
   BREAKDOWN OF HDFS-3042 SUBTASKS
 
     HADOOP-8220. ZKFailoverController doesn't handle failure to become active

+ 14 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -535,6 +535,20 @@
                   </target>
                 </configuration>
               </execution>
+              <execution>
+                <id>native_tests</id>
+                <phase>test</phase>
+                <goals><goal>run</goal></goals>
+                <configuration>
+                  <target>
+                    <exec executable="sh" failonerror="true" dir="${project.build.directory}/native">
+                      <arg value="-c"/>
+                      <arg value="[ x$SKIPTESTS = xtrue ] || ${project.build.directory}/native/test_bulk_crc32"/>
+                      <env key="SKIPTESTS" value="${skipTests}"/>
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
             </executions>
           </plugin>
         </plugins>

+ 8 - 0
hadoop-common-project/hadoop-common/src/CMakeLists.txt

@@ -60,6 +60,7 @@ find_package(ZLIB REQUIRED)
 set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
 set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
 set(D main/native/src/org/apache/hadoop)
+set(T main/native/src/test/org/apache/hadoop)
 
 GET_FILENAME_COMPONENT(HADOOP_ZLIB_LIBRARY ${ZLIB_LIBRARIES} NAME)
 
@@ -98,9 +99,16 @@ include_directories(
     ${JNI_INCLUDE_DIRS}
     ${ZLIB_INCLUDE_DIRS}
     ${SNAPPY_INCLUDE_DIR}
+    ${D}/util
 )
 CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
+add_executable(test_bulk_crc32
+    ${D}/util/bulk_crc32.c
+    ${T}/util/test_bulk_crc32.c
+)
+set_property(SOURCE main.cpp PROPERTY INCLUDE_DIRECTORIES "\"-Werror\" \"-Wall\"")
+
 add_dual_library(hadoop
     ${D}/io/compress/lz4/Lz4Compressor.c
     ${D}/io/compress/lz4/Lz4Decompressor.c

+ 8 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -1871,11 +1871,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (url == null) {
       return null;
     }
-    return parse(builder, url.openStream());
+    return parse(builder, url.openStream(), url.toString());
   }
 
-  private Document parse(DocumentBuilder builder, InputStream is)
-      throws IOException, SAXException {
+  private Document parse(DocumentBuilder builder, InputStream is,
+      String systemId) throws IOException, SAXException {
     if (!quietmode) {
       LOG.info("parsing input stream " + is);
     }
@@ -1883,7 +1883,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       return null;
     }
     try {
-      return builder.parse(is);
+      return (systemId == null) ? builder.parse(is) : builder.parse(is,
+          systemId);
     } finally {
       is.close();
     }
@@ -1951,10 +1952,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
           if (!quiet) {
             LOG.info("parsing File " + file);
           }
-          doc = parse(builder, new BufferedInputStream(new FileInputStream(file)));
+          doc = parse(builder, new BufferedInputStream(
+              new FileInputStream(file)), ((Path)resource).toString());
         }
       } else if (resource instanceof InputStream) {
-        doc = parse(builder, (InputStream) resource);
+        doc = parse(builder, (InputStream) resource, null);
         returnCachedProperties = true;
       } else if (resource instanceof Properties) {
         overlay(properties, (Properties)resource);

+ 20 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java

@@ -68,8 +68,26 @@ public class Trash extends Configured {
   public static boolean moveToAppropriateTrash(FileSystem fs, Path p,
       Configuration conf) throws IOException {
     Path fullyResolvedPath = fs.resolvePath(p);
-    Trash trash = new Trash(FileSystem.get(fullyResolvedPath.toUri(), conf), conf);
-    boolean success =  trash.moveToTrash(fullyResolvedPath);
+    FileSystem fullyResolvedFs =
+        FileSystem.get(fullyResolvedPath.toUri(), conf);
+    // If the trash interval is configured server side then clobber this
+    // configuration so that we always respect the server configuration.
+    try {
+      long trashInterval = fullyResolvedFs.getServerDefaults(
+          fullyResolvedPath).getTrashInterval();
+      if (0 != trashInterval) {
+        Configuration confCopy = new Configuration(conf);
+        confCopy.setLong(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY,
+            trashInterval);
+        conf = confCopy;
+      }
+    } catch (Exception e) {
+      // If we can not determine that trash is enabled server side then
+      // bail rather than potentially deleting a file when trash is enabled.
+      throw new IOException("Failed to get server trash configuration", e);
+    }
+    Trash trash = new Trash(fullyResolvedFs, conf);
+    boolean success = trash.moveToTrash(fullyResolvedPath);
     if (success) {
       System.out.println("Moved: '" + p + "' to trash at: " +
           trash.getCurrentTrashDir() );

+ 3 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java

@@ -79,24 +79,9 @@ public class TrashPolicyDefault extends TrashPolicy {
     this.trash = new Path(home, TRASH);
     this.homesParent = home.getParent();
     this.current = new Path(trash, CURRENT);
-    long trashInterval = 0;
-    try {
-      trashInterval = fs.getServerDefaults(home).getTrashInterval();
-    } catch (IOException ioe) {
-      LOG.warn("Unable to get server defaults", ioe);
-    }
-    // If the trash interval is not configured or is disabled on the
-    // server side then check the config which may be client side.
-    if (0 == trashInterval) {
-      this.deletionInterval = (long)(conf.getFloat(
-          FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT)
-          * MSECS_PER_MINUTE);
-    } else {
-      this.deletionInterval = trashInterval * MSECS_PER_MINUTE;
-    }
-    // For the checkpoint interval use the given config instead of
-    // checking the server as it's OK if a client starts an emptier
-    // with a different interval than the server.
+    this.deletionInterval = (long)(conf.getFloat(
+        FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT)
+        * MSECS_PER_MINUTE);
     this.emptierInterval = (long)(conf.getFloat(
         FS_TRASH_CHECKPOINT_INTERVAL_KEY, FS_TRASH_CHECKPOINT_INTERVAL_DEFAULT)
         * MSECS_PER_MINUTE);

+ 19 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java

@@ -629,7 +629,7 @@ public class RPC {
 
   /** Construct a server for a protocol implementation instance listening on a
    * port and address.
-   * @deprecated protocol interface should be passed.
+   * @deprecated Please use {@link Builder} to build the {@link Server}
    */
   @Deprecated
   public static Server getServer(final Object instance, final String bindAddress, final int port, Configuration conf) 
@@ -639,7 +639,7 @@ public class RPC {
 
   /** Construct a server for a protocol implementation instance listening on a
    * port and address.
-   * @deprecated protocol interface should be passed.
+   * @deprecated Please use {@link Builder} to build the {@link Server}
    */
   @Deprecated
   public static Server getServer(final Object instance, final String bindAddress, final int port,
@@ -651,7 +651,10 @@ public class RPC {
                      null);
   }
 
-  /** Construct a server for a protocol implementation instance. */
+  /** Construct a server for a protocol implementation instance.
+   *  @deprecated Please use {@link Builder} to build the {@link Server}
+   */
+  @Deprecated
   public static Server getServer(Class<?> protocol,
                                  Object instance, String bindAddress,
                                  int port, Configuration conf) 
@@ -661,7 +664,7 @@ public class RPC {
   }
 
   /** Construct a server for a protocol implementation instance.
-   * @deprecated secretManager should be passed.
+   * @deprecated Please use {@link Builder} to build the {@link Server}
    */
   @Deprecated
   public static Server getServer(Class<?> protocol,
@@ -674,7 +677,10 @@ public class RPC {
                  conf, null, null);
   }
   
-  /** Construct a server for a protocol implementation instance. */
+  /** Construct a server for a protocol implementation instance. 
+   *  @deprecated Please use {@link Builder} to build the {@link Server}
+   */
+  @Deprecated
   public static Server getServer(Class<?> protocol,
                                  Object instance, String bindAddress, int port,
                                  int numHandlers,
@@ -685,6 +691,10 @@ public class RPC {
         conf, secretManager, null);
   }
   
+  /**
+   *  @deprecated Please use {@link Builder} to build the {@link Server}
+   */
+  @Deprecated
   public static Server getServer(Class<?> protocol,
       Object instance, String bindAddress, int port,
       int numHandlers,
@@ -697,8 +707,10 @@ public class RPC {
                  verbose, conf, secretManager, portRangeConfig);
   }
 
-  /** Construct a server for a protocol implementation instance. */
-
+  /** Construct a server for a protocol implementation instance.
+   *  @deprecated Please use {@link Builder} to build the {@link Server}
+   */
+  @Deprecated
   public static <PROTO extends VersionedProtocol, IMPL extends PROTO> 
         Server getServer(Class<PROTO> protocol,
                                  IMPL instance, String bindAddress, int port,

+ 41 - 18
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.c

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

+ 35 - 5
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32.h

@@ -19,6 +19,7 @@
 #define BULK_CRC32_H_INCLUDED
 
 #include <stdint.h>
+#include <unistd.h> /* for size_t */
 
 // Constants for different CRC algorithms
 #define CRC32C_POLYNOMIAL 1
@@ -42,16 +43,45 @@ typedef struct crc32_error {
  * of bytes_per_checksum bytes. The checksums are each 32 bits
  * and are stored in sequential indexes of the 'sums' array.
  *
- *  checksum_type - one of the CRC32 constants defined above
- *  error_info - if non-NULL, will be filled in if an error
- *               is detected
+ * @param data                  The data to checksum
+ * @param dataLen               Length of the data buffer
+ * @param sums                  (out param) buffer to write checksums into.
+ *                              It must contain at least dataLen * 4 bytes.
+ * @param checksum_type         One of the CRC32 algorithm constants defined 
+ *                              above
+ * @param bytes_per_checksum    How many bytes of data to process per checksum.
+ * @param error_info            If non-NULL, will be filled in if an error
+ *                              is detected
  *
- * Returns: 0 for success, non-zero for an error, result codes
- *          for which are defined above
+ * @return                      0 for success, non-zero for an error, result codes
+ *                              for which are defined above
  */
 extern int bulk_verify_crc(const uint8_t *data, size_t data_len,
     const uint32_t *sums, int checksum_type,
     int bytes_per_checksum,
     crc32_error_t *error_info);
 
+/**
+ * Calculate checksums for some data.
+ *
+ * The checksums are each 32 bits and are stored in sequential indexes of the
+ * 'sums' array.
+ *
+ * This function is not (yet) optimized.  It is provided for testing purposes
+ * only.
+ *
+ * @param data                  The data to checksum
+ * @param dataLen               Length of the data buffer
+ * @param sums                  (out param) buffer to write checksums into.
+ *                              It must contain at least dataLen * 4 bytes.
+ * @param checksum_type         One of the CRC32 algorithm constants defined 
+ *                              above
+ * @param bytesPerChecksum      How many bytes of data to process per checksum.
+ *
+ * @return                      0 for success, non-zero for an error
+ */
+int bulk_calculate_crc(const uint8_t *data, size_t data_len,
+                    uint32_t *sums, int checksum_type,
+                    int bytes_per_checksum);
+
 #endif

+ 77 - 0
hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/util/test_bulk_crc32.c

@@ -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.
+ */
+
+#include "bulk_crc32.h"
+
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+#define EXPECT_ZERO(x) \
+    do { \
+        int __my_ret__ = x; \
+        if (__my_ret__) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+              "code %d: got nonzero from %s\n", __LINE__, __my_ret__, #x); \
+            return __my_ret__; \
+        } \
+    } while (0);
+
+static int testBulkVerifyCrc(int dataLen, int crcType, int bytesPerChecksum)
+{
+  int i;
+  uint8_t *data;
+  uint32_t *sums;
+  crc32_error_t errorData;
+
+  data = malloc(dataLen);
+  for (i = 0; i < dataLen; i++) {
+    data[i] = (i % 16) + 1;
+  }
+  sums = calloc(sizeof(uint32_t),
+                (dataLen + bytesPerChecksum - 1) / bytesPerChecksum);
+
+  EXPECT_ZERO(bulk_calculate_crc(data, dataLen, sums, crcType,
+                                 bytesPerChecksum));
+  EXPECT_ZERO(bulk_verify_crc(data, dataLen, sums, crcType,
+                            bytesPerChecksum, &errorData));
+  free(data);
+  free(sums);
+  return 0;
+}
+
+int main(int argc, char **argv)
+{
+  /* Test running bulk_calculate_crc with some different algorithms and
+   * bytePerChecksum values. */
+  EXPECT_ZERO(testBulkVerifyCrc(4096, CRC32C_POLYNOMIAL, 512));
+  EXPECT_ZERO(testBulkVerifyCrc(4096, CRC32_ZLIB_POLYNOMIAL, 512));
+  EXPECT_ZERO(testBulkVerifyCrc(256, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(256, CRC32_ZLIB_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(1, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(1, CRC32_ZLIB_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(2, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 2));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 2));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 4));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 4));
+
+  fprintf(stderr, "%s: SUCCESS.\n", argv[0]);
+  return EXIT_SUCCESS;
+}

+ 30 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -358,6 +358,36 @@ public class TestConfiguration extends TestCase {
     tearDown();
   }
 
+  public void testRelativeIncludes() throws Exception {
+    tearDown();
+    String relConfig = new File("./tmp/test-config.xml").getAbsolutePath();
+    String relConfig2 = new File("./tmp/test-config2.xml").getAbsolutePath();
+
+    new File(new File(relConfig).getParent()).mkdirs();
+    out = new BufferedWriter(new FileWriter(relConfig2));
+    startConfig();
+    appendProperty("a", "b");
+    endConfig();
+
+    out = new BufferedWriter(new FileWriter(relConfig));
+    startConfig();
+    // Add the relative path instead of the absolute one.
+    addInclude(new File(relConfig2).getName());
+    appendProperty("c", "d");
+    endConfig();
+
+    // verify that the includes file contains all properties
+    Path fileResource = new Path(relConfig);
+    conf.addResource(fileResource);
+    assertEquals(conf.get("a"), "b");
+    assertEquals(conf.get("c"), "d");
+
+    // Cleanup
+    new File(relConfig).delete();
+    new File(relConfig2).delete();
+    new File(new File(relConfig).getParent()).delete();
+  }
+
   BufferedWriter out;
 	
   public void testIntegerRanges() {

+ 12 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs;
 
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
@@ -61,8 +62,15 @@ public abstract class FileContextMainOperationsBaseTest  {
   private static String TEST_DIR_AXX = "test/hadoop/axx";
   private static int numBlocks = 2;
   
-  static  final String LOCAL_FS_ROOT_URI = "file:///tmp/test";
-  
+  public static final Path LOCAL_FS_ROOT_PATH;
+      
+  static {
+    File testBuildData = new File(System.getProperty("test.build.data",
+                                    "build/test/data"));
+    Path localFsRootPath = new Path(testBuildData.getAbsolutePath(), 
+                                    "root-uri");
+    LOCAL_FS_ROOT_PATH = localFsRootPath.makeQualified(LocalFileSystem.NAME, null);
+  }
   
   protected static FileContext fc;
   
@@ -95,7 +103,7 @@ public abstract class FileContextMainOperationsBaseTest  {
   @After
   public void tearDown() throws Exception {
     fc.delete(new Path(getAbsoluteTestRootPath(fc), new Path("test")), true);
-    fc.delete(new Path(LOCAL_FS_ROOT_URI), true);
+    fc.delete(LOCAL_FS_ROOT_PATH, true);
   }
   
   
@@ -176,7 +184,7 @@ public abstract class FileContextMainOperationsBaseTest  {
     
     // Try a URI
 
-    absoluteDir = new Path(LOCAL_FS_ROOT_URI + "/existingDir");
+    absoluteDir = new Path(LOCAL_FS_ROOT_PATH, "existingDir");
     fc.mkdir(absoluteDir, FileContext.DEFAULT_PERM, true);
     fc.setWorkingDirectory(absoluteDir);
     Assert.assertEquals(absoluteDir, fc.getWorkingDirectory());

+ 4 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java

@@ -99,7 +99,6 @@ public class TestTrash extends TestCase {
   }
 
   /**
-   * 
    * Test trash for the shell's delete command for the default file system
    * specified in the paramter conf
    * @param conf 
@@ -429,8 +428,10 @@ public class TestTrash extends TestCase {
       String output = byteStream.toString();
       System.setOut(stdout);
       System.setErr(stderr);
-      assertTrue("skipTrash wasn't suggested as remedy to failed rm command",
-        output.indexOf(("Consider using -skipTrash option")) != -1 );
+      assertTrue("skipTrash wasn't suggested as remedy to failed rm command" +
+          " or we deleted / even though we could not get server defaults",
+          output.indexOf("Consider using -skipTrash option") != -1 ||
+          output.indexOf("Failed to determine server trash configuration") != -1);
     }
 
   }

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml

@@ -103,10 +103,6 @@
           <groupId>javax.xml.stream</groupId>
           <artifactId>stax-api</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>commons-cli</groupId>
-          <artifactId>commons-cli</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>commons-httpclient</groupId>
           <artifactId>commons-httpclient</artifactId>

+ 32 - 2
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -1,6 +1,6 @@
 Hadoop HDFS Change Log
 
-Trunk (unreleased changes)
+Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
 
@@ -215,7 +215,22 @@ Trunk (unreleased changes)
 
     HDFS-3678. Edit log files are never being purged from 2NN. (atm)
 
-Branch-2 ( Unreleased changes )
+Release 2.0.3-alpha - Unreleased 
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+    HDFS-2656. Add libwebhdfs, a pure C client based on WebHDFS.
+    (Jaimin D Jetly and Jing Zhao via szetszwo)
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
 
@@ -723,6 +738,18 @@ Branch-2 ( Unreleased changes )
 
     HDFS-3469. start-dfs.sh will start zkfc, but stop-dfs.sh will not stop zkfc similarly.
     (Vinay via umamahesh)
+
+    HDFS-1490. TransferFSImage should timeout (Dmytro Molkov and Vinay via todd)
+
+    HDFS-3828. Block Scanner rescans blocks too frequently.
+    (Andy Isaacson via eli)
+
+    HDFS-3809. Make BKJM use protobufs for all serialization with ZK.(Ivan Kelly via umamahesh)
+
+    HDFS-3895. hadoop-client must include commons-cli (tucu)
+
+    HDFS-2757. Cannot read a local block that's being written to when
+    using the local read short circuit. (Jean-Daniel Cryans via eli)
     
   BREAKDOWN OF HDFS-3042 SUBTASKS
 
@@ -1623,6 +1650,9 @@ Release 0.23.3 - UNRELEASED
 
     HDFS-3852. TestHftpDelegationToken is broken after HADOOP-8225 (daryn)
 
+    HDFS-3890. filecontext mkdirs doesn't apply umask as expected
+    (Tom Graves via daryn)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -35,6 +35,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <kdc.resource.dir>../../hadoop-common-project/hadoop-common/src/test/resources/kdc</kdc.resource.dir>
     <is.hadoop.component>true</is.hadoop.component>
     <require.fuse>false</require.fuse>
+    <require.libwebhdfs>false</require.libwebhdfs>
   </properties>
 
   <dependencies>
@@ -495,7 +496,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                     <mkdir dir="${project.build.directory}/native"/>
                     <exec executable="cmake" dir="${project.build.directory}/native" 
                         failonerror="true">
-                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model} -DREQUIRE_FUSE=${require.fuse}"/>
+                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model} -DREQUIRE_LIBWEBHDFS=${require.libwebhdfs} -DREQUIRE_FUSE=${require.fuse}"/>
                     </exec>
                     <exec executable="make" dir="${project.build.directory}/native" failonerror="true">
                       <arg line="VERBOSE=1"/>

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -147,4 +147,7 @@ target_link_libraries(test_libhdfs_threaded
     pthread
 )
 
+IF(REQUIRE_LIBWEBHDFS)
+    add_subdirectory(contrib/libwebhdfs)
+ENDIF(REQUIRE_LIBWEBHDFS)
 add_subdirectory(main/native/fuse-dfs)

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/dev-support/findbugsExcludeFile.xml

@@ -0,0 +1,5 @@
+<FindBugsFilter>
+     <Match>
+       <Class name="~org.apache.hadoop.contrib.bkjournal.BKJournalProtos.*" />
+     </Match>
+</FindBugsFilter>

+ 84 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml

@@ -89,6 +89,90 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>test</scope>
     </dependency>
   </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>${project.build.directory}/generated-sources/java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <configuration>
+          <skipTests>false</skipTests>
+        </configuration>
+        <executions>
+          <execution>
+            <id>compile-proto</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <echo file="target/compile-proto.sh">
+                    PROTO_DIR=src/main/proto
+                    INCLUDE_DIR=../../main/proto
+                    JAVA_DIR=target/generated-sources/java
+                    which cygpath 2&gt; /dev/null
+                    if [ $? = 1 ]; then
+                      IS_WIN=false
+                    else
+                      IS_WIN=true
+                      WIN_PROTO_DIR=`cygpath --windows $PROTO_DIR`
+                      WIN_JAVA_DIR=`cygpath --windows $JAVA_DIR`
+                      WIN_INCLUDE_DIR=`cygpath --windows $INCLUDE_DIR`
+                    fi
+                    mkdir -p $JAVA_DIR 2&gt; /dev/null
+                    for PROTO_FILE in `ls $PROTO_DIR/*.proto 2&gt; /dev/null`
+                    do
+                        if [ "$IS_WIN" = "true" ]; then
+                          protoc -I$WIN_PROTO_DIR -I$WIN_INCLUDE_DIR --java_out=$WIN_JAVA_DIR $PROTO_FILE
+                        else
+                          protoc -I$PROTO_DIR -I$INCLUDE_DIR --java_out=$JAVA_DIR $PROTO_FILE
+                        fi
+                    done
+                </echo>
+                <exec executable="sh" dir="${basedir}" failonerror="true">
+                  <arg line="target/compile-proto.sh"/>
+                </exec>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>dev-support/findbugsExcludeFile.xml</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
   <profiles>
     <profile>
       <id>dist</id>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java

@@ -70,7 +70,7 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
     this.lh = lh;
     this.firstTxId = metadata.getFirstTxId();
     this.lastTxId = metadata.getLastTxId();
-    this.logVersion = metadata.getVersion();
+    this.logVersion = metadata.getDataLayoutVersion();
     this.inProgress = metadata.isInProgress();
 
     if (firstBookKeeperEntry < 0

+ 45 - 28
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -50,6 +50,11 @@ import java.io.IOException;
 
 import java.net.URI;
 
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.VersionProto;
+import com.google.protobuf.TextFormat;
+import static com.google.common.base.Charsets.UTF_8;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import com.google.common.annotations.VisibleForTesting;
@@ -143,36 +148,17 @@ public class BookKeeperJournalManager implements JournalManager {
   private final int quorumSize;
   private final String digestpw;
   private final CountDownLatch zkConnectLatch;
-
+  private final NamespaceInfo nsInfo;
   private LedgerHandle currentLedger = null;
 
-  private int bytesToInt(byte[] b) {
-    assert b.length >= 4;
-    return b[0] << 24 | b[1] << 16 | b[2] << 8 | b[3];
-  }
-
-  private byte[] intToBytes(int i) {
-    return new byte[] {
-      (byte)(i >> 24),
-      (byte)(i >> 16),
-      (byte)(i >> 8),
-      (byte)(i) };
-  }
-
-  BookKeeperJournalManager(Configuration conf, URI uri) throws IOException {
-    this(conf, uri, null);
-    // TODO(ivank): update BookKeeperJournalManager to do something
-    // with the NamespaceInfo. This constructor has been added
-    // for compatibility with the old tests, and may be removed
-    // when the tests are updated.
-  }
-
   /**
    * Construct a Bookkeeper journal manager.
    */
   public BookKeeperJournalManager(Configuration conf, URI uri,
       NamespaceInfo nsInfo) throws IOException {
     this.conf = conf;
+    this.nsInfo = nsInfo;
+
     String zkConnect = uri.getAuthority().replace(";", ",");
     String zkPath = uri.getPath();
     ensembleSize = conf.getInt(BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
@@ -202,10 +188,32 @@ public class BookKeeperJournalManager implements JournalManager {
       Stat versionStat = zkc.exists(versionPath, false);
       if (versionStat != null) {
         byte[] d = zkc.getData(versionPath, false, versionStat);
+        VersionProto.Builder builder = VersionProto.newBuilder();
+        TextFormat.merge(new String(d, UTF_8), builder);
+        if (!builder.isInitialized()) {
+          throw new IOException("Invalid/Incomplete data in znode");
+        }
+        VersionProto vp = builder.build();
+
         // There's only one version at the moment
-        assert bytesToInt(d) == BKJM_LAYOUT_VERSION;
-      } else {
-        zkc.create(versionPath, intToBytes(BKJM_LAYOUT_VERSION),
+        assert vp.getLayoutVersion() == BKJM_LAYOUT_VERSION;
+
+        NamespaceInfo readns = PBHelper.convert(vp.getNamespaceInfo());
+
+        if (nsInfo.getNamespaceID() != readns.getNamespaceID() ||
+            !nsInfo.clusterID.equals(readns.getClusterID()) ||
+            !nsInfo.getBlockPoolID().equals(readns.getBlockPoolID())) {
+          String err = String.format("Environment mismatch. Running process %s"
+                                     +", stored in ZK %s", nsInfo, readns);
+          LOG.error(err);
+          throw new IOException(err);
+        }
+      } else if (nsInfo.getNamespaceID() > 0) {
+        VersionProto.Builder builder = VersionProto.newBuilder();
+        builder.setNamespaceInfo(PBHelper.convert(nsInfo))
+          .setLayoutVersion(BKJM_LAYOUT_VERSION);
+        byte[] data = TextFormat.printToString(builder.build()).getBytes(UTF_8);
+        zkc.create(versionPath, data,
                    Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
       }
 
@@ -214,11 +222,11 @@ public class BookKeeperJournalManager implements JournalManager {
             Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
       }
       prepareBookKeeperEnv();
-      bkc = new BookKeeper(new ClientConfiguration(),
-                           zkc);
+      bkc = new BookKeeper(new ClientConfiguration(), zkc);
     } catch (KeeperException e) {
       throw new IOException("Error initializing zk", e);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       throw new IOException("Interrupted while initializing bk journal manager",
                             ie);
     }
@@ -322,13 +330,14 @@ public class BookKeeperJournalManager implements JournalManager {
     } catch (KeeperException ke) {
       throw new IOException("Error in zookeeper while creating ledger", ke);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       throw new IOException("Interrupted creating ledger", ie);
     }
 
     try {
       String znodePath = inprogressZNode(txId);
       EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
-          HdfsConstants.LAYOUT_VERSION,  currentLedger.getId(), txId);
+          HdfsConstants.LAYOUT_VERSION, currentLedger.getId(), txId);
       /* Write the ledger metadata out to the inprogress ledger znode
        * This can fail if for some reason our write lock has
        * expired (@see WriteLock) and another process has managed to
@@ -356,6 +365,7 @@ public class BookKeeperJournalManager implements JournalManager {
       //log & ignore, an IOException will be thrown soon
       LOG.error("Error closing ledger", bke);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       LOG.warn("Interrupted while closing ledger", ie);
     }
   }
@@ -425,6 +435,7 @@ public class BookKeeperJournalManager implements JournalManager {
     } catch (KeeperException e) {
       throw new IOException("Error finalising ledger", e);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       throw new IOException("Error finalising ledger", ie);
     } 
   }
@@ -454,6 +465,7 @@ public class BookKeeperJournalManager implements JournalManager {
         } catch (BKException e) {
           throw new IOException("Could not open ledger for " + fromTxId, e);
         } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
           throw new IOException("Interrupted opening ledger for "
                                          + fromTxId, ie);
         }
@@ -567,6 +579,7 @@ public class BookKeeperJournalManager implements JournalManager {
       } catch (KeeperException ke) {
         throw new IOException("Couldn't get list of inprogress segments", ke);
       } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
         throw new IOException("Interrupted getting list of inprogress segments",
                               ie);
       }
@@ -583,6 +596,7 @@ public class BookKeeperJournalManager implements JournalManager {
           zkc.delete(l.getZkPath(), stat.getVersion());
           bkc.deleteLedger(l.getLedgerId());
         } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
           LOG.error("Interrupted while purging " + l, ie);
         } catch (BKException bke) {
           LOG.error("Couldn't delete ledger from bookkeeper", bke);
@@ -601,6 +615,7 @@ public class BookKeeperJournalManager implements JournalManager {
     } catch (BKException bke) {
       throw new IOException("Couldn't close bookkeeper client", bke);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       throw new IOException("Interrupted while closing journal manager", ie);
     }
   }
@@ -635,6 +650,7 @@ public class BookKeeperJournalManager implements JournalManager {
     } catch (BKException bke) {
       throw new IOException("Exception opening ledger for " + l, bke);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       throw new IOException("Interrupted opening ledger for " + l, ie);
     }
 
@@ -692,6 +708,7 @@ public class BookKeeperJournalManager implements JournalManager {
     } catch (KeeperException e) {
       throw new IOException("Exception reading ledger list from zk", e);
     } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
       throw new IOException("Interrupted getting list of ledgers from zk", ie);
     }
 

+ 21 - 26
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/CurrentInprogress.java

@@ -29,6 +29,10 @@ import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 
+import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.CurrentInprogressProto;
+import com.google.protobuf.TextFormat;
+import static com.google.common.base.Charsets.UTF_8;
+
 /**
  * Distributed write permission lock, using ZooKeeper. Read the version number
  * and return the current inprogress node path available in CurrentInprogress
@@ -42,29 +46,28 @@ import org.apache.zookeeper.data.Stat;
  */
 
 class CurrentInprogress {
-  private static final String CONTENT_DELIMITER = ",";
-
   static final Log LOG = LogFactory.getLog(CurrentInprogress.class);
 
   private final ZooKeeper zkc;
   private final String currentInprogressNode;
   private volatile int versionNumberForPermission = -1;
-  private static final int CURRENT_INPROGRESS_LAYOUT_VERSION = -1; 
   private final String hostName = InetAddress.getLocalHost().toString();
 
   CurrentInprogress(ZooKeeper zkc, String lockpath) throws IOException {
     this.currentInprogressNode = lockpath;
     this.zkc = zkc;
     try {
-      Stat isCurrentInprogressNodeExists = zkc.exists(lockpath, false);
+      Stat isCurrentInprogressNodeExists = zkc.exists(currentInprogressNode,
+                                                      false);
       if (isCurrentInprogressNodeExists == null) {
         try {
-          zkc.create(lockpath, null, Ids.OPEN_ACL_UNSAFE,
-                  CreateMode.PERSISTENT);
+          zkc.create(currentInprogressNode, null, Ids.OPEN_ACL_UNSAFE,
+                     CreateMode.PERSISTENT);
         } catch (NodeExistsException e) {
           // Node might created by other process at the same time. Ignore it.
           if (LOG.isDebugEnabled()) {
-            LOG.debug(lockpath + " already created by other process.", e);
+            LOG.debug(currentInprogressNode + " already created by other process.",
+                      e);
           }
         }
       }
@@ -83,10 +86,13 @@ class CurrentInprogress {
    * @throws IOException
    */
   void update(String path) throws IOException {
-    String content = CURRENT_INPROGRESS_LAYOUT_VERSION
-        + CONTENT_DELIMITER + hostName + CONTENT_DELIMITER + path;
+    CurrentInprogressProto.Builder builder = CurrentInprogressProto.newBuilder();
+    builder.setPath(path).setHostname(hostName);
+
+    String content = TextFormat.printToString(builder.build());
+
     try {
-      zkc.setData(this.currentInprogressNode, content.getBytes(),
+      zkc.setData(this.currentInprogressNode, content.getBytes(UTF_8),
           this.versionNumberForPermission);
     } catch (KeeperException e) {
       throw new IOException("Exception when setting the data "
@@ -123,23 +129,12 @@ class CurrentInprogress {
     }
     this.versionNumberForPermission = stat.getVersion();
     if (data != null) {
-      String stringData = new String(data);
-      LOG.info("Read data[layout version number,hostname,inprogressNode path]"
-          + "= [" + stringData + "] from CurrentInprogress");
-      String[] contents = stringData.split(CONTENT_DELIMITER);
-      assert contents.length == 3 : "As per the current data format, "
-          + "CurrentInprogress node data should contain 3 fields. "
-          + "i.e layout version number,hostname,inprogressNode path";
-      String layoutVersion = contents[0];
-      if (Long.valueOf(layoutVersion) > CURRENT_INPROGRESS_LAYOUT_VERSION) {
-        throw new IOException(
-            "Supported layout version of CurrentInprogress node is : "
-                + CURRENT_INPROGRESS_LAYOUT_VERSION
-                + " . Layout version of CurrentInprogress node in ZK is : "
-                + layoutVersion);
+      CurrentInprogressProto.Builder builder = CurrentInprogressProto.newBuilder();
+      TextFormat.merge(new String(data, UTF_8), builder);
+      if (!builder.isInitialized()) {
+        throw new IOException("Invalid/Incomplete data in znode");
       }
-      String inprogressNodePath = contents[2];
-      return inprogressNodePath;
+      return builder.build().getPath();
     } else {
       LOG.info("No data available in CurrentInprogress");
     }

+ 49 - 39
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java

@@ -29,6 +29,10 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.EditLogLedgerProto;
+import com.google.protobuf.TextFormat;
+import static com.google.common.base.Charsets.UTF_8;
+
 /**
  * Utility class for storing the metadata associated 
  * with a single edit log segment, stored in a single ledger
@@ -37,8 +41,8 @@ public class EditLogLedgerMetadata {
   static final Log LOG = LogFactory.getLog(EditLogLedgerMetadata.class);
 
   private String zkPath;
+  private final int dataLayoutVersion;
   private final long ledgerId;
-  private final int version;
   private final long firstTxId;
   private long lastTxId;
   private boolean inprogress;
@@ -57,21 +61,22 @@ public class EditLogLedgerMetadata {
     }
   };
 
-  EditLogLedgerMetadata(String zkPath, int version, 
+  EditLogLedgerMetadata(String zkPath, int dataLayoutVersion,
                         long ledgerId, long firstTxId) {
     this.zkPath = zkPath;
+    this.dataLayoutVersion = dataLayoutVersion;
     this.ledgerId = ledgerId;
-    this.version = version;
     this.firstTxId = firstTxId;
     this.lastTxId = HdfsConstants.INVALID_TXID;
     this.inprogress = true;
   }
   
-  EditLogLedgerMetadata(String zkPath, int version, long ledgerId, 
-                        long firstTxId, long lastTxId) {
+  EditLogLedgerMetadata(String zkPath, int dataLayoutVersion,
+                        long ledgerId, long firstTxId,
+                        long lastTxId) {
     this.zkPath = zkPath;
+    this.dataLayoutVersion = dataLayoutVersion;
     this.ledgerId = ledgerId;
-    this.version = version;
     this.firstTxId = firstTxId;
     this.lastTxId = lastTxId;
     this.inprogress = false;
@@ -93,14 +98,14 @@ public class EditLogLedgerMetadata {
     return ledgerId;
   }
   
-  int getVersion() {
-    return version;
-  }
-
   boolean isInProgress() {
     return this.inprogress;
   }
 
+  int getDataLayoutVersion() {
+    return this.dataLayoutVersion;
+  }
+
   void finalizeLedger(long newLastTxId) {
     assert this.lastTxId == HdfsConstants.INVALID_TXID;
     this.lastTxId = newLastTxId;
@@ -111,22 +116,27 @@ public class EditLogLedgerMetadata {
       throws IOException, KeeperException.NoNodeException  {
     try {
       byte[] data = zkc.getData(path, false, null);
-      String[] parts = new String(data).split(";");
-      if (parts.length == 3) {
-        int version = Integer.valueOf(parts[0]);
-        long ledgerId = Long.valueOf(parts[1]);
-        long txId = Long.valueOf(parts[2]);
-        return new EditLogLedgerMetadata(path, version, ledgerId, txId);
-      } else if (parts.length == 4) {
-        int version = Integer.valueOf(parts[0]);
-        long ledgerId = Long.valueOf(parts[1]);
-        long firstTxId = Long.valueOf(parts[2]);
-        long lastTxId = Long.valueOf(parts[3]);
-        return new EditLogLedgerMetadata(path, version, ledgerId,
-                                         firstTxId, lastTxId);
+
+      EditLogLedgerProto.Builder builder = EditLogLedgerProto.newBuilder();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Reading " + path + " data: " + new String(data, UTF_8));
+      }
+      TextFormat.merge(new String(data, UTF_8), builder);
+      if (!builder.isInitialized()) {
+        throw new IOException("Invalid/Incomplete data in znode");
+      }
+      EditLogLedgerProto ledger = builder.build();
+
+      int dataLayoutVersion = ledger.getDataLayoutVersion();
+      long ledgerId = ledger.getLedgerId();
+      long firstTxId = ledger.getFirstTxId();
+      if (ledger.hasLastTxId()) {
+        long lastTxId = ledger.getLastTxId();
+        return new EditLogLedgerMetadata(path, dataLayoutVersion,
+                                         ledgerId, firstTxId, lastTxId);
       } else {
-        throw new IOException("Invalid ledger entry, "
-                              + new String(data));
+        return new EditLogLedgerMetadata(path, dataLayoutVersion,
+                                         ledgerId, firstTxId);
       }
     } catch(KeeperException.NoNodeException nne) {
       throw nne;
@@ -140,17 +150,17 @@ public class EditLogLedgerMetadata {
   void write(ZooKeeper zkc, String path)
       throws IOException, KeeperException.NodeExistsException {
     this.zkPath = path;
-    String finalisedData;
-    if (inprogress) {
-      finalisedData = String.format("%d;%d;%d",
-          version, ledgerId, firstTxId);
-    } else {
-      finalisedData = String.format("%d;%d;%d;%d",
-          version, ledgerId, firstTxId, lastTxId);
+
+    EditLogLedgerProto.Builder builder = EditLogLedgerProto.newBuilder();
+    builder.setDataLayoutVersion(dataLayoutVersion)
+      .setLedgerId(ledgerId).setFirstTxId(firstTxId);
+
+    if (!inprogress) {
+      builder.setLastTxId(lastTxId);
     }
     try {
-      zkc.create(path, finalisedData.getBytes(), Ids.OPEN_ACL_UNSAFE,
-          CreateMode.PERSISTENT);
+      zkc.create(path, TextFormat.printToString(builder.build()).getBytes(UTF_8),
+                 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
     } catch (KeeperException.NodeExistsException nee) {
       throw nee;
     } catch (KeeperException e) {
@@ -183,9 +193,9 @@ public class EditLogLedgerMetadata {
     }
     EditLogLedgerMetadata ol = (EditLogLedgerMetadata)o;
     return ledgerId == ol.ledgerId
+      && dataLayoutVersion == ol.dataLayoutVersion
       && firstTxId == ol.firstTxId
-      && lastTxId == ol.lastTxId
-      && version == ol.version;
+      && lastTxId == ol.lastTxId;
   }
 
   public int hashCode() {
@@ -193,15 +203,15 @@ public class EditLogLedgerMetadata {
     hash = hash * 31 + (int) ledgerId;
     hash = hash * 31 + (int) firstTxId;
     hash = hash * 31 + (int) lastTxId;
-    hash = hash * 31 + (int) version;
+    hash = hash * 31 + (int) dataLayoutVersion;
     return hash;
   }
     
   public String toString() {
     return "[LedgerId:"+ledgerId +
       ", firstTxId:" + firstTxId +
-      ", lastTxId:" + lastTxId + 
-      ", version:" + version + "]";
+      ", lastTxId:" + lastTxId +
+      ", dataLayoutVersion:" + dataLayoutVersion + "]";
   }
 
 }

+ 19 - 6
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/MaxTxId.java

@@ -27,6 +27,10 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 
+import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.MaxTxIdProto;
+import com.google.protobuf.TextFormat;
+import static com.google.common.base.Charsets.UTF_8;
+
 /**
  * Utility class for storing and reading
  * the max seen txid in zookeeper
@@ -55,14 +59,16 @@ class MaxTxId {
   }
 
   synchronized void reset(long maxTxId) throws IOException {
-    String txidStr = Long.toString(maxTxId);
     try {
+      MaxTxIdProto.Builder builder = MaxTxIdProto.newBuilder().setTxId(maxTxId);
+
+      byte[] data = TextFormat.printToString(builder.build()).getBytes(UTF_8);
       if (currentStat != null) {
-        currentStat = zkc.setData(path, txidStr.getBytes("UTF-8"), currentStat
+        currentStat = zkc.setData(path, data, currentStat
             .getVersion());
       } else {
-        zkc.create(path, txidStr.getBytes("UTF-8"), Ids.OPEN_ACL_UNSAFE,
-            CreateMode.PERSISTENT);
+        zkc.create(path, data, Ids.OPEN_ACL_UNSAFE,
+                   CreateMode.PERSISTENT);
       }
     } catch (KeeperException e) {
       throw new IOException("Error writing max tx id", e);
@@ -77,9 +83,16 @@ class MaxTxId {
       if (currentStat == null) {
         return 0;
       } else {
+
         byte[] bytes = zkc.getData(path, false, currentStat);
-        String txidString = new String(bytes, "UTF-8");
-        return Long.valueOf(txidString);
+
+        MaxTxIdProto.Builder builder = MaxTxIdProto.newBuilder();
+        TextFormat.merge(new String(bytes, UTF_8), builder);
+        if (!builder.isInitialized()) {
+          throw new IOException("Invalid/Incomplete data in znode");
+        }
+
+        return builder.build().getTxId();
       }
     } catch (KeeperException e) {
       throw new IOException("Error reading the max tx id from zk", e);

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto

@@ -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.
+ */
+
+// This file contains protocol buffers that are used by bkjournal
+// mostly for storing data in zookeeper
+
+option java_package = "org.apache.hadoop.contrib.bkjournal";
+option java_outer_classname = "BKJournalProtos";
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+message VersionProto {
+  required int32 layoutVersion = 1;
+  optional NamespaceInfoProto namespaceInfo = 2;
+}
+
+message EditLogLedgerProto {
+  required int32 dataLayoutVersion = 1;
+  required int64 ledgerId = 2;
+  required int64 firstTxId = 3;
+  optional int64 lastTxId = 4;
+}
+
+message MaxTxIdProto {
+  required int64 txId = 1;
+}
+
+message CurrentInprogressProto {
+  required string path = 1;
+  optional string hostname = 2;
+}

+ 16 - 4
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperConfiguration.java

@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.Random;
 
 import org.apache.bookkeeper.util.LocalBookKeeper;
 import org.apache.commons.logging.Log;
@@ -42,6 +43,8 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
 public class TestBookKeeperConfiguration {
   private static final Log LOG = LogFactory
       .getLog(TestBookKeeperConfiguration.class);
@@ -73,6 +76,11 @@ public class TestBookKeeperConfiguration {
     return zkc;
   }
 
+  private NamespaceInfo newNSInfo() {
+    Random r = new Random();
+    return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
+  }
+
   @BeforeClass
   public static void setupZooKeeper() throws Exception {
     // create a ZooKeeper server(dataDir, dataLogDir, port)
@@ -137,8 +145,10 @@ public class TestBookKeeperConfiguration {
         bkAvailablePath);
     Assert.assertNull(bkAvailablePath + " already exists", zkc.exists(
         bkAvailablePath, false));
-    bkjm = new BookKeeperJournalManager(conf, URI.create("bookkeeper://"
-        + HOSTPORT + "/hdfsjournal-WithBKPath"));
+    NamespaceInfo nsi = newNSInfo();
+    bkjm = new BookKeeperJournalManager(conf,
+        URI.create("bookkeeper://" + HOSTPORT + "/hdfsjournal-WithBKPath"),
+        nsi);
     Assert.assertNotNull("Bookie available path : " + bkAvailablePath
         + " doesn't exists", zkc.exists(bkAvailablePath, false));
   }
@@ -152,8 +162,10 @@ public class TestBookKeeperConfiguration {
     Configuration conf = new Configuration();
     Assert.assertNull(BK_ROOT_PATH + " already exists", zkc.exists(
         BK_ROOT_PATH, false));
-    new BookKeeperJournalManager(conf, URI.create("bookkeeper://" + HOSTPORT
-        + "/hdfsjournal-DefaultBKPath"));
+    NamespaceInfo nsi = newNSInfo();
+    bkjm = new BookKeeperJournalManager(conf,
+        URI.create("bookkeeper://" + HOSTPORT + "/hdfsjournal-DefaultBKPath"),
+        nsi);
     Assert.assertNotNull("Bookie available path : " + BK_ROOT_PATH
         + " doesn't exists", zkc.exists(BK_ROOT_PATH, false));
   }

+ 68 - 25
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java

@@ -29,6 +29,7 @@ import org.mockito.Mockito;
 import java.io.IOException;
 import java.net.URI;
 import java.util.List;
+import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 
@@ -37,6 +38,7 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.zookeeper.CreateMode;
@@ -78,10 +80,17 @@ public class TestBookKeeperJournalManager {
     zkc.close();
   }
 
+  private NamespaceInfo newNSInfo() {
+    Random r = new Random();
+    return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
+  }
+
   @Test
   public void testSimpleWrite() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-simplewrite"));
+        BKJMUtil.createJournalURI("/hdfsjournal-simplewrite"), nsi);
+
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1 ; i <= 100; i++) {
       FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
@@ -99,8 +108,10 @@ public class TestBookKeeperJournalManager {
 
   @Test
   public void testNumberOfTransactions() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
+
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-txncount"));
+        BKJMUtil.createJournalURI("/hdfsjournal-txncount"), nsi);
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1 ; i <= 100; i++) {
       FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
@@ -116,8 +127,10 @@ public class TestBookKeeperJournalManager {
 
   @Test 
   public void testNumberOfTransactionsWithGaps() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-gaps"));
+        BKJMUtil.createJournalURI("/hdfsjournal-gaps"), nsi);
+
     long txid = 1;
     for (long i = 0; i < 3; i++) {
       long start = txid;
@@ -151,8 +164,10 @@ public class TestBookKeeperJournalManager {
 
   @Test
   public void testNumberOfTransactionsWithInprogressAtEnd() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-inprogressAtEnd"));
+        BKJMUtil.createJournalURI("/hdfsjournal-inprogressAtEnd"), nsi);
+
     long txid = 1;
     for (long i = 0; i < 3; i++) {
       long start = txid;
@@ -190,8 +205,10 @@ public class TestBookKeeperJournalManager {
    */
   @Test
   public void testWriteRestartFrom1() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-restartFrom1"));
+        BKJMUtil.createJournalURI("/hdfsjournal-restartFrom1"), nsi);
+
     long txid = 1;
     long start = txid;
     EditLogOutputStream out = bkjm.startLogSegment(txid);
@@ -245,11 +262,15 @@ public class TestBookKeeperJournalManager {
   @Test
   public void testTwoWriters() throws Exception {
     long start = 1;
+    NamespaceInfo nsi = newNSInfo();
+
     BookKeeperJournalManager bkjm1 = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"));
+        BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"), nsi);
+
     BookKeeperJournalManager bkjm2 = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"));
-    
+        BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"), nsi);
+
+
     EditLogOutputStream out1 = bkjm1.startLogSegment(start);
     try {
       bkjm2.startLogSegment(start);
@@ -263,8 +284,11 @@ public class TestBookKeeperJournalManager {
 
   @Test
   public void testSimpleRead() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-simpleread"));
+        BKJMUtil.createJournalURI("/hdfsjournal-simpleread"),
+        nsi);
+
     final long numTransactions = 10000;
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1 ; i <= numTransactions; i++) {
@@ -287,8 +311,11 @@ public class TestBookKeeperJournalManager {
 
   @Test
   public void testSimpleRecovery() throws Exception {
+    NamespaceInfo nsi = newNSInfo();
     BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-simplerecovery"));
+        BKJMUtil.createJournalURI("/hdfsjournal-simplerecovery"),
+        nsi);
+
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1 ; i <= 100; i++) {
       FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
@@ -334,8 +361,10 @@ public class TestBookKeeperJournalManager {
       conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
                   ensembleSize);
       long txid = 1;
+      NamespaceInfo nsi = newNSInfo();
       BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-          BKJMUtil.createJournalURI("/hdfsjournal-allbookiefailure"));
+          BKJMUtil.createJournalURI("/hdfsjournal-allbookiefailure"),
+          nsi);
       EditLogOutputStream out = bkjm.startLogSegment(txid);
 
       for (long i = 1 ; i <= 3; i++) {
@@ -416,8 +445,12 @@ public class TestBookKeeperJournalManager {
       conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
                   ensembleSize);
       long txid = 1;
+
+      NamespaceInfo nsi = newNSInfo();
       BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-          BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"));
+          BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"),
+          nsi);
+
       EditLogOutputStream out = bkjm.startLogSegment(txid);
       for (long i = 1 ; i <= 3; i++) {
         FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
@@ -464,7 +497,9 @@ public class TestBookKeeperJournalManager {
   @Test
   public void testEmptyInprogressNode() throws Exception {
     URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogress");
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
+    NamespaceInfo nsi = newNSInfo();
+    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
+                                                                 nsi);
 
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1; i <= 100; i++) {
@@ -481,7 +516,7 @@ public class TestBookKeeperJournalManager {
     String inprogressZNode = bkjm.inprogressZNode(101);
     zkc.setData(inprogressZNode, new byte[0], -1);
 
-    bkjm = new BookKeeperJournalManager(conf, uri);
+    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
     try {
       bkjm.recoverUnfinalizedSegments();
       fail("Should have failed. There should be no way of creating"
@@ -489,7 +524,7 @@ public class TestBookKeeperJournalManager {
     } catch (IOException e) {
       // correct behaviour
       assertTrue("Exception different than expected", e.getMessage().contains(
-          "Invalid ledger entry,"));
+          "Invalid/Incomplete data in znode"));
     } finally {
       bkjm.close();
     }
@@ -503,7 +538,9 @@ public class TestBookKeeperJournalManager {
   @Test
   public void testCorruptInprogressNode() throws Exception {
     URI uri = BKJMUtil.createJournalURI("/hdfsjournal-corruptInprogress");
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
+    NamespaceInfo nsi = newNSInfo();
+    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
+                                                                 nsi);
 
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1; i <= 100; i++) {
@@ -521,7 +558,7 @@ public class TestBookKeeperJournalManager {
     String inprogressZNode = bkjm.inprogressZNode(101);
     zkc.setData(inprogressZNode, "WholeLottaJunk".getBytes(), -1);
 
-    bkjm = new BookKeeperJournalManager(conf, uri);
+    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
     try {
       bkjm.recoverUnfinalizedSegments();
       fail("Should have failed. There should be no way of creating"
@@ -529,8 +566,7 @@ public class TestBookKeeperJournalManager {
     } catch (IOException e) {
       // correct behaviour
       assertTrue("Exception different than expected", e.getMessage().contains(
-          "Invalid ledger entry,"));
-
+          "has no field named"));
     } finally {
       bkjm.close();
     }
@@ -544,7 +580,9 @@ public class TestBookKeeperJournalManager {
   @Test
   public void testEmptyInprogressLedger() throws Exception {
     URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogressLedger");
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
+    NamespaceInfo nsi = newNSInfo();
+    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
+                                                                 nsi);
 
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1; i <= 100; i++) {
@@ -559,7 +597,7 @@ public class TestBookKeeperJournalManager {
     out.close();
     bkjm.close();
 
-    bkjm = new BookKeeperJournalManager(conf, uri);
+    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
     bkjm.recoverUnfinalizedSegments();
     out = bkjm.startLogSegment(101);
     for (long i = 1; i <= 100; i++) {
@@ -581,7 +619,9 @@ public class TestBookKeeperJournalManager {
   public void testRefinalizeAlreadyFinalizedInprogress() throws Exception {
     URI uri = BKJMUtil
         .createJournalURI("/hdfsjournal-refinalizeInprogressLedger");
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
+    NamespaceInfo nsi = newNSInfo();
+    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
+                                                                 nsi);
 
     EditLogOutputStream out = bkjm.startLogSegment(1);
     for (long i = 1; i <= 100; i++) {
@@ -601,7 +641,7 @@ public class TestBookKeeperJournalManager {
     byte[] inprogressData = zkc.getData(inprogressZNode, false, null);
 
     // finalize
-    bkjm = new BookKeeperJournalManager(conf, uri);
+    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
     bkjm.recoverUnfinalizedSegments();
     bkjm.close();
 
@@ -613,7 +653,7 @@ public class TestBookKeeperJournalManager {
         CreateMode.PERSISTENT);
 
     // should work fine
-    bkjm = new BookKeeperJournalManager(conf, uri);
+    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
     bkjm.recoverUnfinalizedSegments();
     bkjm.close();
   }
@@ -626,7 +666,10 @@ public class TestBookKeeperJournalManager {
   @Test
   public void testEditLogFileNotExistsWhenReadingMetadata() throws Exception {
     URI uri = BKJMUtil.createJournalURI("/hdfsjournal-editlogfile");
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri);
+    NamespaceInfo nsi = newNSInfo();
+    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
+                                                                 nsi);
+
     try {
       // start new inprogress log segment with txid=1
       // and write transactions till txid=50

+ 95 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt

@@ -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.
+#
+
+find_package(CURL)
+if (CURL_FOUND)
+    include_directories(${CURL_INCLUDE_DIRS})
+else (CURL_FOUND)
+    MESSAGE(STATUS "Failed to find CURL library.")
+endif (CURL_FOUND)
+
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH}
+"${CMAKE_SOURCE_DIR}/contrib/libwebhdfs/resources/")
+MESSAGE("CMAKE_MODULE_PATH IS: " ${CMAKE_MODULE_PATH})
+
+find_package(Jansson)
+include_directories(${JANSSON_INCLUDE_DIR})
+
+add_dual_library(webhdfs
+    src/exception.c
+    src/hdfs_web.c
+    src/hdfs_jni.c
+    src/jni_helper.c
+    src/hdfs_http_client.c
+    src/hdfs_http_query.c
+    src/hdfs_json_parser.c
+)
+target_link_dual_libraries(webhdfs
+    ${JAVA_JVM_LIBRARY}
+    ${CURL_LIBRARY}
+    ${JANSSON_LIBRARY}
+    pthread
+)
+dual_output_directory(webhdfs target)
+set(LIBWEBHDFS_VERSION "0.0.0")
+set_target_properties(webhdfs PROPERTIES
+    SOVERSION ${LIBWEBHDFS_VERSION})
+
+add_executable(test_libwebhdfs_ops
+    src/test_libwebhdfs_ops.c
+)
+target_link_libraries(test_libwebhdfs_ops
+    webhdfs
+    ${CURL_LIBRARY}
+    ${JAVA_JVM_LIBRARY}
+    ${JANSSON_LIBRARY}
+    pthread
+)
+
+add_executable(test_libwebhdfs_read
+    src/test_libwebhdfs_read.c
+)
+target_link_libraries(test_libwebhdfs_read
+    webhdfs
+    ${CURL_LIBRARY}
+    ${JAVA_JVM_LIBRARY}
+    ${JANSSON_LIBRARY}
+    pthread
+)
+
+add_executable(test_libwebhdfs_write
+    src/test_libwebhdfs_write.c
+)
+target_link_libraries(test_libwebhdfs_write
+    webhdfs
+    ${CURL_LIBRARY}
+    ${JAVA_JVM_LIBRARY}
+    ${JANSSON_LIBRARY}
+    pthread
+)
+
+add_executable(test_libwebhdfs_threaded
+    src/test_libwebhdfs_threaded.c
+)
+target_link_libraries(test_libwebhdfs_threaded
+    webhdfs
+    ${CURL_LIBRARY}
+    ${JAVA_JVM_LIBRARY}
+    ${JANSSON_LIBRARY}
+    pthread
+)

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake

@@ -0,0 +1,42 @@
+# 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.
+
+
+# - Try to find Jansson
+# Once done this will define
+#  JANSSON_FOUND - System has Jansson
+#  JANSSON_INCLUDE_DIRS - The Jansson include directories
+#  JANSSON_LIBRARIES - The libraries needed to use Jansson
+#  JANSSON_DEFINITIONS - Compiler switches required for using Jansson
+
+find_path(JANSSON_INCLUDE_DIR jansson.h
+          /usr/incluce
+          /usr/local/include )
+
+find_library(JANSSON_LIBRARY NAMES jansson
+             PATHS /usr/lib /usr/local/lib )
+
+set(JANSSON_LIBRARIES ${JANSSON_LIBRARY} )
+set(JANSSON_INCLUDE_DIRS ${JANSSON_INCLUDE_DIR} )
+
+include(FindPackageHandleStandardArgs)
+# handle the QUIETLY and REQUIRED arguments and set JANSSON_FOUND to TRUE
+# if all listed variables are TRUE
+find_package_handle_standard_args(Jansson  DEFAULT_MSG
+                                  JANSSON_LIBRARY JANSSON_INCLUDE_DIR)
+
+mark_as_advanced(JANSSON_INCLUDE_DIR JANSSON_LIBRARY )

+ 237 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/exception.c

@@ -0,0 +1,237 @@
+/**
+ * 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.
+ */
+
+#include "exception.h"
+#include "webhdfs.h"
+#include "jni_helper.h"
+
+#include <inttypes.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define EXCEPTION_INFO_LEN (sizeof(gExceptionInfo)/sizeof(gExceptionInfo[0]))
+
+struct ExceptionInfo {
+    const char * const name;
+    int noPrintFlag;
+    int excErrno;
+};
+
+static const struct ExceptionInfo gExceptionInfo[] = {
+    {
+        .name = "java/io/FileNotFoundException",
+        .noPrintFlag = NOPRINT_EXC_FILE_NOT_FOUND,
+        .excErrno = ENOENT,
+    },
+    {
+        .name = "org/apache/hadoop/security/AccessControlException",
+        .noPrintFlag = NOPRINT_EXC_ACCESS_CONTROL,
+        .excErrno = EACCES,
+    },
+    {
+        .name = "org/apache/hadoop/fs/UnresolvedLinkException",
+        .noPrintFlag = NOPRINT_EXC_UNRESOLVED_LINK,
+        .excErrno = ENOLINK,
+    },
+    {
+        .name = "org/apache/hadoop/fs/ParentNotDirectoryException",
+        .noPrintFlag = NOPRINT_EXC_PARENT_NOT_DIRECTORY,
+        .excErrno = ENOTDIR,
+    },
+    {
+        .name = "java/lang/IllegalArgumentException",
+        .noPrintFlag = NOPRINT_EXC_ILLEGAL_ARGUMENT,
+        .excErrno = EINVAL,
+    },
+    {
+        .name = "java/lang/OutOfMemoryError",
+        .noPrintFlag = 0,
+        .excErrno = ENOMEM,
+    },
+    
+};
+
+int printExceptionWebV(hdfs_exception_msg *exc, int noPrintFlags, const char *fmt, va_list ap)
+{
+    int i, noPrint, excErrno;
+    if (!exc) {
+        fprintf(stderr, "printExceptionWebV: the hdfs_exception_msg is NULL\n");
+        return EINTERNAL;
+    }
+    
+    for (i = 0; i < EXCEPTION_INFO_LEN; i++) {
+        if (strstr(gExceptionInfo[i].name, exc->exception)) {
+            break;
+        }
+    }
+    if (i < EXCEPTION_INFO_LEN) {
+        noPrint = (gExceptionInfo[i].noPrintFlag & noPrintFlags);
+        excErrno = gExceptionInfo[i].excErrno;
+    } else {
+        noPrint = 0;
+        excErrno = EINTERNAL;
+    }
+    
+    if (!noPrint) {
+        vfprintf(stderr, fmt, ap);
+        fprintf(stderr, " error:\n");
+        fprintf(stderr, "Exception: %s\nJavaClassName: %s\nMessage: %s\n", exc->exception, exc->javaClassName, exc->message);
+    }
+    
+    free(exc);
+    return excErrno;
+}
+
+int printExceptionWeb(hdfs_exception_msg *exc, int noPrintFlags, const char *fmt, ...)
+{
+    va_list ap;
+    int ret;
+    
+    va_start(ap, fmt);
+    ret = printExceptionWebV(exc, noPrintFlags, fmt, ap);
+    va_end(ap);
+    return ret;
+}
+
+int printExceptionAndFreeV(JNIEnv *env, jthrowable exc, int noPrintFlags,
+        const char *fmt, va_list ap)
+{
+    int i, noPrint, excErrno;
+    char *className = NULL;
+    jstring jStr = NULL;
+    jvalue jVal;
+    jthrowable jthr;
+
+    jthr = classNameOfObject(exc, env, &className);
+    if (jthr) {
+        fprintf(stderr, "PrintExceptionAndFree: error determining class name "
+            "of exception.\n");
+        className = strdup("(unknown)");
+        destroyLocalReference(env, jthr);
+    }
+    for (i = 0; i < EXCEPTION_INFO_LEN; i++) {
+        if (!strcmp(gExceptionInfo[i].name, className)) {
+            break;
+        }
+    }
+    if (i < EXCEPTION_INFO_LEN) {
+        noPrint = (gExceptionInfo[i].noPrintFlag & noPrintFlags);
+        excErrno = gExceptionInfo[i].excErrno;
+    } else {
+        noPrint = 0;
+        excErrno = EINTERNAL;
+    }
+    if (!noPrint) {
+        vfprintf(stderr, fmt, ap);
+        fprintf(stderr, " error:\n");
+
+        // We don't want to  use ExceptionDescribe here, because that requires a
+        // pending exception.  Instead, use ExceptionUtils.
+        jthr = invokeMethod(env, &jVal, STATIC, NULL, 
+            "org/apache/commons/lang/exception/ExceptionUtils",
+            "getStackTrace", "(Ljava/lang/Throwable;)Ljava/lang/String;", exc);
+        if (jthr) {
+            fprintf(stderr, "(unable to get stack trace for %s exception: "
+                    "ExceptionUtils::getStackTrace error.)\n", className);
+            destroyLocalReference(env, jthr);
+        } else {
+            jStr = jVal.l;
+            const char *stackTrace = (*env)->GetStringUTFChars(env, jStr, NULL);
+            if (!stackTrace) {
+                fprintf(stderr, "(unable to get stack trace for %s exception: "
+                        "GetStringUTFChars error.)\n", className);
+            } else {
+                fprintf(stderr, "%s", stackTrace);
+                (*env)->ReleaseStringUTFChars(env, jStr, stackTrace);
+            }
+        }
+    }
+    destroyLocalReference(env, jStr);
+    destroyLocalReference(env, exc);
+    free(className);
+    return excErrno;
+}
+
+int printExceptionAndFree(JNIEnv *env, jthrowable exc, int noPrintFlags,
+        const char *fmt, ...)
+{
+    va_list ap;
+    int ret;
+
+    va_start(ap, fmt);
+    ret = printExceptionAndFreeV(env, exc, noPrintFlags, fmt, ap);
+    va_end(ap);
+    return ret;
+}
+
+int printPendingExceptionAndFree(JNIEnv *env, int noPrintFlags,
+        const char *fmt, ...)
+{
+    va_list ap;
+    int ret;
+    jthrowable exc;
+
+    exc = (*env)->ExceptionOccurred(env);
+    if (!exc) {
+        va_start(ap, fmt);
+        vfprintf(stderr, fmt, ap);
+        va_end(ap);
+        fprintf(stderr, " error: (no exception)");
+        ret = 0;
+    } else {
+        (*env)->ExceptionClear(env);
+        va_start(ap, fmt);
+        ret = printExceptionAndFreeV(env, exc, noPrintFlags, fmt, ap);
+        va_end(ap);
+    }
+    return ret;
+}
+
+jthrowable getPendingExceptionAndClear(JNIEnv *env)
+{
+    jthrowable jthr = (*env)->ExceptionOccurred(env);
+    if (!jthr)
+        return NULL;
+    (*env)->ExceptionClear(env);
+    return jthr;
+}
+
+jthrowable newRuntimeError(JNIEnv *env, const char *fmt, ...)
+{
+    char buf[512];
+    jobject out, exc;
+    jstring jstr;
+    va_list ap;
+
+    va_start(ap, fmt);
+    vsnprintf(buf, sizeof(buf), fmt, ap);
+    va_end(ap);
+    jstr = (*env)->NewStringUTF(env, buf);
+    if (!jstr) {
+        // We got an out of memory exception rather than a RuntimeException.
+        // Too bad...
+        return getPendingExceptionAndClear(env);
+    }
+    exc = constructNewObjectOfClass(env, &out, "RuntimeException",
+        "(java/lang/String;)V", jstr);
+    (*env)->DeleteLocalRef(env, jstr);
+    // Again, we'll either get an out of memory exception or the
+    // RuntimeException we wanted.
+    return (exc) ? exc : out;
+}

+ 178 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/exception.h

@@ -0,0 +1,178 @@
+/**
+ * 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.
+ */
+
+#ifndef LIBHDFS_EXCEPTION_H
+#define LIBHDFS_EXCEPTION_H
+
+/**
+ * Exception handling routines for libhdfs.
+ *
+ * The convention we follow here is to clear pending exceptions as soon as they
+ * are raised.  Never assume that the caller of your function will clean up
+ * after you-- do it yourself.  Unhandled exceptions can lead to memory leaks
+ * and other undefined behavior.
+ *
+ * If you encounter an exception, return a local reference to it.  The caller is
+ * responsible for freeing the local reference, by calling a function like
+ * PrintExceptionAndFree.  (You can also free exceptions directly by calling
+ * DeleteLocalRef.  However, that would not produce an error message, so it's
+ * usually not what you want.)
+ */
+
+#include <jni.h>
+#include <stdio.h>
+
+#include <stdlib.h>
+#include <stdarg.h>
+#include <search.h>
+#include <pthread.h>
+#include <errno.h>
+
+/**
+ * Exception noprint flags
+ *
+ * Theses flags determine which exceptions should NOT be printed to stderr by
+ * the exception printing routines.  For example, if you expect to see
+ * FileNotFound, you might use NOPRINT_EXC_FILE_NOT_FOUND, to avoid filling the
+ * logs with messages about routine events.
+ *
+ * On the other hand, if you don't expect any failures, you might pass
+ * PRINT_EXC_ALL.
+ *
+ * You can OR these flags together to avoid printing multiple classes of
+ * exceptions.
+ */
+#define PRINT_EXC_ALL                           0x00
+#define NOPRINT_EXC_FILE_NOT_FOUND              0x01
+#define NOPRINT_EXC_ACCESS_CONTROL              0x02
+#define NOPRINT_EXC_UNRESOLVED_LINK             0x04
+#define NOPRINT_EXC_PARENT_NOT_DIRECTORY        0x08
+#define NOPRINT_EXC_ILLEGAL_ARGUMENT            0x10
+
+/**
+ * Exception information after calling webhdfs operations
+ */
+typedef struct {
+    const char *exception;
+    const char *javaClassName;
+    const char *message;
+} hdfs_exception_msg;
+
+/**
+ * Print out exception information got after calling webhdfs operations
+ *
+ * @param exc             The exception information to print and free
+ * @param noPrintFlags    Flags which determine which exceptions we should NOT
+ *                        print.
+ * @param fmt             Printf-style format list
+ * @param ap              Printf-style varargs
+ *
+ * @return                The POSIX error number associated with the exception
+ *                        object.
+ */
+int printExceptionWebV(hdfs_exception_msg *exc, int noPrintFlags, const char *fmt, va_list ap);
+
+/**
+ * Print out exception information got after calling webhdfs operations
+ *
+ * @param exc             The exception information to print and free
+ * @param noPrintFlags    Flags which determine which exceptions we should NOT
+ *                        print.
+ * @param fmt             Printf-style format list
+ * @param ...             Printf-style varargs
+ *
+ * @return                The POSIX error number associated with the exception
+ *                        object.
+ */
+int printExceptionWeb(hdfs_exception_msg *exc, int noPrintFlags,
+                      const char *fmt, ...) __attribute__((format(printf, 3, 4)));
+
+/**
+ * Print out information about an exception and free it.
+ *
+ * @param env             The JNI environment
+ * @param exc             The exception to print and free
+ * @param noPrintFlags    Flags which determine which exceptions we should NOT
+ *                        print.
+ * @param fmt             Printf-style format list
+ * @param ap              Printf-style varargs
+ *
+ * @return                The POSIX error number associated with the exception
+ *                        object.
+ */
+int printExceptionAndFreeV(JNIEnv *env, jthrowable exc, int noPrintFlags,
+        const char *fmt, va_list ap);
+
+/**
+ * Print out information about an exception and free it.
+ *
+ * @param env             The JNI environment
+ * @param exc             The exception to print and free
+ * @param noPrintFlags    Flags which determine which exceptions we should NOT
+ *                        print.
+ * @param fmt             Printf-style format list
+ * @param ...             Printf-style varargs
+ *
+ * @return                The POSIX error number associated with the exception
+ *                        object.
+ */
+int printExceptionAndFree(JNIEnv *env, jthrowable exc, int noPrintFlags,
+        const char *fmt, ...) __attribute__((format(printf, 4, 5)));  
+
+/**
+ * Print out information about the pending exception and free it.
+ *
+ * @param env             The JNI environment
+ * @param noPrintFlags    Flags which determine which exceptions we should NOT
+ *                        print.
+ * @param fmt             Printf-style format list
+ * @param ...             Printf-style varargs
+ *
+ * @return                The POSIX error number associated with the exception
+ *                        object.
+ */
+int printPendingExceptionAndFree(JNIEnv *env, int noPrintFlags,
+        const char *fmt, ...) __attribute__((format(printf, 3, 4)));  
+
+/**
+ * Get a local reference to the pending exception and clear it.
+ *
+ * Once it is cleared, the exception will no longer be pending.  The caller will
+ * have to decide what to do with the exception object.
+ *
+ * @param env             The JNI environment
+ *
+ * @return                The exception, or NULL if there was no exception
+ */
+jthrowable getPendingExceptionAndClear(JNIEnv *env);
+
+/**
+ * Create a new runtime error.
+ *
+ * This creates (but does not throw) a new RuntimeError.
+ *
+ * @param env             The JNI environment
+ * @param fmt             Printf-style format list
+ * @param ...             Printf-style varargs
+ *
+ * @return                A local reference to a RuntimeError
+ */
+jthrowable newRuntimeError(JNIEnv *env, const char *fmt, ...)
+        __attribute__((format(printf, 2, 3)));
+
+#endif

+ 101 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/expect.h

@@ -0,0 +1,101 @@
+/**
+ * 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.
+ */
+
+#ifndef LIBHDFS_NATIVE_TESTS_EXPECT_H
+#define LIBHDFS_NATIVE_TESTS_EXPECT_H
+
+#include <stdio.h>
+
+#define EXPECT_ZERO(x) \
+    do { \
+        int __my_ret__ = x; \
+        if (__my_ret__) { \
+            int __my_errno__ = errno; \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+		    "code %d (errno: %d): got nonzero from %s\n", \
+		    __LINE__, __my_ret__, __my_errno__, #x); \
+            return __my_ret__; \
+        } \
+    } while (0);
+
+#define EXPECT_NULL(x) \
+    do { \
+        void* __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ != NULL) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d (errno: %d): " \
+		    "got non-NULL value %p from %s\n", \
+		    __LINE__, __my_errno__, __my_ret__, #x); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_NONNULL(x) \
+    do { \
+        void* __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ == NULL) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d (errno: %d): " \
+		    "got NULL from %s\n", __LINE__, __my_errno__, #x); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_NEGATIVE_ONE_WITH_ERRNO(x, e) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ != -1) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+                "code %d (errno: %d): expected -1 from %s\n", __LINE__, \
+                __my_ret__, __my_errno__, #x); \
+            return -1; \
+        } \
+        if (__my_errno__ != e) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+                "code %d (errno: %d): expected errno = %d from %s\n", \
+                __LINE__, __my_ret__, __my_errno__, e, #x); \
+            return -1; \
+	} \
+    } while (0);
+
+#define EXPECT_NONZERO(x) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+		    "code %d (errno: %d): got zero from %s\n", __LINE__, \
+                __my_ret__, __my_errno__, #x); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_NONNEGATIVE(x) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ < 0) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+                "code %d (errno: %d): got negative return from %s\n", \
+		    __LINE__, __my_ret__, __my_errno__, #x); \
+            return __my_ret__; \
+        } \
+    } while (0);
+
+#endif

+ 352 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.c

@@ -0,0 +1,352 @@
+/**
+ * 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.
+ */
+#include <stdlib.h>
+#include <string.h>
+#include <curl/curl.h>
+#include <pthread.h>
+#include "hdfs_http_client.h"
+
+static pthread_mutex_t curlInitMutex = PTHREAD_MUTEX_INITIALIZER;
+static volatile int curlGlobalInited = 0;
+
+ResponseBuffer initResponseBuffer() {
+    ResponseBuffer info = (ResponseBuffer) calloc(1, sizeof(ResponseBufferInternal));
+    if (!info) {
+        fprintf(stderr, "Cannot allocate memory for responseInfo\n");
+        return NULL;
+    }
+    info->remaining = 0;
+    info->offset = 0;
+    info->content = NULL;
+    return info;
+}
+
+void freeResponseBuffer(ResponseBuffer buffer) {
+    if (buffer) {
+        if (buffer->content) {
+            free(buffer->content);
+        }
+        free(buffer);
+        buffer = NULL;
+    }
+}
+
+void freeResponse(Response resp)  {
+    if(resp) {
+        freeResponseBuffer(resp->body);
+        freeResponseBuffer(resp->header);
+        free(resp);
+        resp = NULL;
+    }
+}
+
+/* Callback for allocating local buffer and reading data to local buffer */
+static size_t writefunc(void *ptr, size_t size, size_t nmemb, ResponseBuffer rbuffer) {
+    if (size * nmemb < 1) {
+        return 0;
+    }
+    if (!rbuffer) {
+        fprintf(stderr, "In writefunc, ResponseBuffer is NULL.\n");
+        return -1;
+    }
+    
+    if (rbuffer->remaining < size * nmemb) {
+        rbuffer->content = realloc(rbuffer->content, rbuffer->offset + size * nmemb + 1);
+        if (rbuffer->content == NULL) {
+            return -1;
+        }
+        rbuffer->remaining = size * nmemb;
+    }
+    memcpy(rbuffer->content + rbuffer->offset, ptr, size * nmemb);
+    rbuffer->offset += size * nmemb;
+    (rbuffer->content)[rbuffer->offset] = '\0';
+    rbuffer->remaining -= size * nmemb;
+    return size * nmemb;
+}
+
+/**
+ * Callback for reading data to buffer provided by user, 
+ * thus no need to reallocate buffer.
+ */
+static size_t writefunc_withbuffer(void *ptr, size_t size, size_t nmemb, ResponseBuffer rbuffer) {
+    if (size * nmemb < 1) {
+        return 0;
+    }
+    if (!rbuffer || !rbuffer->content) {
+        fprintf(stderr, "In writefunc_withbuffer, the buffer provided by user is NULL.\n");
+        return 0;
+    }
+    
+    size_t toCopy = rbuffer->remaining < (size * nmemb) ? rbuffer->remaining : (size * nmemb);
+    memcpy(rbuffer->content + rbuffer->offset, ptr, toCopy);
+    rbuffer->offset += toCopy;
+    rbuffer->remaining -= toCopy;
+    return toCopy;
+}
+
+//callback for writing data to remote peer
+static size_t readfunc(void *ptr, size_t size, size_t nmemb, void *stream) {
+    if (size * nmemb < 1) {
+        fprintf(stderr, "In readfunc callback: size * nmemb == %ld\n", size * nmemb);
+        return 0;
+    }
+    webhdfsBuffer *wbuffer = (webhdfsBuffer *) stream;
+    
+    pthread_mutex_lock(&wbuffer->writeMutex);
+    while (wbuffer->remaining == 0) {
+        /*
+         * the current remainning bytes to write is 0,
+         * check whether need to finish the transfer
+         * if yes, return 0; else, wait
+         */
+        if (wbuffer->closeFlag) {
+            //we can close the transfer now
+            fprintf(stderr, "CloseFlag is set, ready to close the transfer\n");
+            pthread_mutex_unlock(&wbuffer->writeMutex);
+            return 0;
+        } else {
+            // len == 0 indicates that user's buffer has been transferred
+            pthread_cond_signal(&wbuffer->transfer_finish);
+            pthread_cond_wait(&wbuffer->newwrite_or_close, &wbuffer->writeMutex);
+        }
+    }
+    
+    if(wbuffer->remaining > 0 && !wbuffer->closeFlag) {
+        size_t copySize = wbuffer->remaining < size * nmemb ? wbuffer->remaining : size * nmemb;
+        memcpy(ptr, wbuffer->wbuffer + wbuffer->offset, copySize);
+        wbuffer->offset += copySize;
+        wbuffer->remaining -= copySize;
+        pthread_mutex_unlock(&wbuffer->writeMutex);
+        return copySize;
+    } else {
+        fprintf(stderr, "Webhdfs buffer is %ld, it should be a positive value!\n", wbuffer->remaining);
+        pthread_mutex_unlock(&wbuffer->writeMutex);
+        return 0;
+    }
+}
+
+static void initCurlGlobal() {
+    if (!curlGlobalInited) {
+        pthread_mutex_lock(&curlInitMutex);
+        if (!curlGlobalInited) {
+            curl_global_init(CURL_GLOBAL_ALL);
+            curlGlobalInited = 1;
+        }
+        pthread_mutex_unlock(&curlInitMutex);
+    }
+}
+
+static Response launchCmd(char *url, enum HttpHeader method, enum Redirect followloc) {
+    CURL *curl;
+    CURLcode res;
+    Response resp;
+    
+    resp = (Response) calloc(1, sizeof(*resp));
+    if (!resp) {
+        return NULL;
+    }
+    resp->body = initResponseBuffer();
+    resp->header = initResponseBuffer();
+    initCurlGlobal();
+    curl = curl_easy_init();                     /* get a curl handle */
+    if(curl) {
+        curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc);
+        curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
+        curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
+        curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
+        curl_easy_setopt(curl, CURLOPT_URL, url);       /* specify target URL */
+        switch(method) {
+            case GET:
+                break;
+            case PUT:
+                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"PUT");
+                break;
+            case POST:
+                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"POST");
+                break;
+            case DELETE:
+                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"DELETE");
+                break;
+            default:
+                fprintf(stderr, "\nHTTP method not defined\n");
+                exit(EXIT_FAILURE);
+        }
+        if(followloc == YES) {
+            curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, 1);
+        }
+        
+        res = curl_easy_perform(curl);                 /* Now run the curl handler */
+        if(res != CURLE_OK) {
+            fprintf(stderr, "preform the URL %s failed\n", url);
+            return NULL;
+        }
+        curl_easy_cleanup(curl);
+    }
+    return resp;
+}
+
+static Response launchRead_internal(char *url, enum HttpHeader method, enum Redirect followloc, Response resp) {
+    if (!resp || !resp->body || !resp->body->content) {
+        fprintf(stderr, "The user provided buffer should not be NULL!\n");
+        return NULL;
+    }
+    
+    CURL *curl;
+    CURLcode res;
+    initCurlGlobal();
+    curl = curl_easy_init();                     /* get a curl handle */
+    if(curl) {
+        curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc_withbuffer);
+        curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
+        curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
+        curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
+        curl_easy_setopt(curl, CURLOPT_URL, url);       /* specify target URL */
+        if(followloc == YES) {
+            curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, 1);
+        }
+        
+        res = curl_easy_perform(curl);                 /* Now run the curl handler */
+        if(res != CURLE_OK && res != CURLE_PARTIAL_FILE) {
+            fprintf(stderr, "preform the URL %s failed\n", url);
+            return NULL;
+        }
+        curl_easy_cleanup(curl);
+    }
+    return resp;
+
+}
+
+static Response launchWrite(const char *url, enum HttpHeader method, webhdfsBuffer *uploadBuffer) {
+    if (!uploadBuffer) {
+        fprintf(stderr, "upload buffer is NULL!\n");
+        errno = EINVAL;
+        return NULL;
+    }
+    initCurlGlobal();
+    CURLcode res;
+    Response response = (Response) calloc(1, sizeof(*response));
+    if (!response) {
+        fprintf(stderr, "failed to allocate memory for response\n");
+        return NULL;
+    }
+    response->body = initResponseBuffer();
+    response->header = initResponseBuffer();
+    
+    //connect to the datanode in order to create the lease in the namenode
+    CURL *curl = curl_easy_init();
+    if (!curl) {
+        fprintf(stderr, "Failed to initialize the curl handle.\n");
+        return NULL;
+    }
+    curl_easy_setopt(curl, CURLOPT_URL, url);
+    
+    if(curl) {
+        curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc);
+        curl_easy_setopt(curl, CURLOPT_WRITEDATA, response->body);
+        curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
+        curl_easy_setopt(curl, CURLOPT_WRITEHEADER, response->header);
+        curl_easy_setopt(curl, CURLOPT_READFUNCTION, readfunc);
+        curl_easy_setopt(curl, CURLOPT_READDATA, uploadBuffer);
+        curl_easy_setopt(curl, CURLOPT_UPLOAD, 1L);
+        curl_easy_setopt(curl, CURLOPT_VERBOSE, 1);
+        
+        struct curl_slist *chunk = NULL;
+        chunk = curl_slist_append(chunk, "Transfer-Encoding: chunked");
+        res = curl_easy_setopt(curl, CURLOPT_HTTPHEADER, chunk);
+        chunk = curl_slist_append(chunk, "Expect:");
+        res = curl_easy_setopt(curl, CURLOPT_HTTPHEADER, chunk);
+        
+        switch(method) {
+            case GET:
+                break;
+            case PUT:
+                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"PUT");
+                break;
+            case POST:
+                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"POST");
+                break;
+            case DELETE:
+                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"DELETE");
+                break;
+            default:
+                fprintf(stderr, "\nHTTP method not defined\n");
+                exit(EXIT_FAILURE);
+        }
+        res = curl_easy_perform(curl);
+        curl_slist_free_all(chunk);
+        curl_easy_cleanup(curl);
+    }
+    
+    return response;
+}
+
+Response launchMKDIR(char *url) {
+    return launchCmd(url, PUT, NO);
+}
+
+Response launchRENAME(char *url) {
+    return launchCmd(url, PUT, NO);
+}
+
+Response launchGFS(char *url) {
+    return launchCmd(url, GET, NO);
+}
+
+Response launchLS(char *url) {
+    return launchCmd(url, GET, NO);
+}
+
+Response launchCHMOD(char *url) {
+    return launchCmd(url, PUT, NO);
+}
+
+Response launchCHOWN(char *url) {
+    return launchCmd(url, PUT, NO);
+}
+
+Response launchDELETE(char *url) {
+    return launchCmd(url, DELETE, NO);
+}
+
+Response launchOPEN(char *url, Response resp) {
+    return launchRead_internal(url, GET, YES, resp);
+}
+
+Response launchUTIMES(char *url) {
+    return launchCmd(url, PUT, NO);
+}
+
+Response launchNnWRITE(char *url) {
+    return launchCmd(url, PUT, NO);
+}
+
+Response launchNnAPPEND(char *url) {
+    return launchCmd(url, POST, NO);
+}
+
+Response launchDnWRITE(const char *url, webhdfsBuffer *buffer) {
+    return launchWrite(url, PUT, buffer);
+}
+
+Response launchDnAPPEND(const char *url, webhdfsBuffer *buffer) {
+    return launchWrite(url, POST, buffer);
+}
+
+Response launchSETREPLICATION(char *url) {
+    return launchCmd(url, PUT, NO);
+}

+ 74 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.h

@@ -0,0 +1,74 @@
+/**
+ * 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.
+ */
+
+
+
+#ifndef _HDFS_HTTP_CLIENT_H_
+#define _HDFS_HTTP_CLIENT_H_
+
+#include "webhdfs.h"
+#include <curl/curl.h>
+
+enum HttpHeader {
+    GET,
+    PUT,
+    POST,
+    DELETE
+};
+
+enum Redirect {
+    YES,
+    NO
+};
+
+typedef struct {
+    char *content;
+    size_t remaining;
+    size_t offset;
+} ResponseBufferInternal;
+typedef ResponseBufferInternal *ResponseBuffer;
+
+/**
+ * The response got through webhdfs
+ */
+typedef struct {
+    ResponseBuffer body;
+    ResponseBuffer header;
+}* Response;
+
+ResponseBuffer initResponseBuffer();
+void freeResponseBuffer(ResponseBuffer buffer);
+void freeResponse(Response resp);
+
+Response launchMKDIR(char *url);
+Response launchRENAME(char *url);
+Response launchCHMOD(char *url);
+Response launchGFS(char *url);
+Response launchLS(char *url);
+Response launchDELETE(char *url);
+Response launchCHOWN(char *url);
+Response launchOPEN(char *url, Response resp);
+Response launchUTIMES(char *url);
+Response launchNnWRITE(char *url);
+
+Response launchDnWRITE(const char *url, webhdfsBuffer *buffer);
+Response launchNnAPPEND(char *url);
+Response launchSETREPLICATION(char *url);
+Response launchDnAPPEND(const char *url, webhdfsBuffer *buffer);
+
+#endif //_HDFS_HTTP_CLIENT_H_

+ 254 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.c

@@ -0,0 +1,254 @@
+/**
+ * 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.
+ */
+#include "hdfs_http_query.h"
+#include <math.h>
+#include <stdlib.h>
+#include <string.h>
+#include <stdio.h>
+#include <errno.h>
+
+#define NUM_OF_PERMISSION_BITS 4
+#define NUM_OF_PORT_BITS 6
+#define NUM_OF_REPLICATION_BITS 6
+
+static char *prepareQUERY(const char *host, int nnPort, const char *srcpath, const char *OP, const char *user) {
+    size_t length;
+    char *url;
+    const char *const protocol = "http://";
+    const char *const prefix = "/webhdfs/v1";
+    char *temp;
+    char *port;
+    port= (char*) malloc(NUM_OF_PORT_BITS);
+    if (!port) {
+        return NULL;
+    }
+    sprintf(port,"%d",nnPort);
+    if (user != NULL) {
+        length = strlen(protocol) + strlen(host) + strlen(":") + strlen(port) + strlen(prefix) + strlen(srcpath) + strlen ("?op=") + strlen(OP) + strlen("&user.name=") + strlen(user);
+    } else {
+        length = strlen(protocol) + strlen(host) + strlen(":") + strlen(port) + strlen(prefix) + strlen(srcpath) +  strlen ("?op=") + strlen(OP);
+    }
+    
+    temp = (char*) malloc(length + 1);
+    if (!temp) {
+        return NULL;
+    }
+    strcpy(temp,protocol);
+    temp = strcat(temp,host);
+    temp = strcat(temp,":");
+    temp = strcat(temp,port);
+    temp = strcat(temp,prefix);
+    temp = strcat(temp,srcpath);
+    temp = strcat(temp,"?op=");
+    temp = strcat(temp,OP);
+    if (user) {
+        temp = strcat(temp,"&user.name=");
+        temp = strcat(temp,user);
+    }
+    url = temp;
+    return url;
+}
+
+
+static int decToOctal(int decNo) {
+    int octNo=0;
+    int expo =0;
+    while (decNo != 0)  {
+        octNo = ((decNo % 8) * pow(10,expo)) + octNo;
+        decNo = decNo / 8;
+        expo++;
+    }
+    return octNo;
+}
+
+
+char *prepareMKDIR(const char *host, int nnPort, const char *dirsubpath, const char *user) {
+    return prepareQUERY(host, nnPort, dirsubpath, "MKDIRS", user);
+}
+
+
+char *prepareMKDIRwithMode(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user) {
+    char *url;
+    char *permission;
+    permission = (char*) malloc(NUM_OF_PERMISSION_BITS);
+    if (!permission) {
+        return NULL;
+    }
+    mode = decToOctal(mode);
+    sprintf(permission,"%d",mode);
+    url = prepareMKDIR(host, nnPort, dirsubpath, user);
+    url = realloc(url,(strlen(url) + strlen("&permission=") + strlen(permission) + 1));
+    if (!url) {
+        return NULL;
+    }
+    url = strcat(url,"&permission=");
+    url = strcat(url,permission);
+    return url;
+}
+
+
+char *prepareRENAME(const char *host, int nnPort, const char *srcpath, const char *destpath, const char *user) {
+    char *url;
+    url = prepareQUERY(host, nnPort, srcpath, "RENAME", user);
+    url = realloc(url,(strlen(url) + strlen("&destination=") + strlen(destpath) + 1));
+    if (!url) {
+        return NULL;
+    }
+    url = strcat(url,"&destination=");
+    url = strcat(url,destpath);
+    return url;
+}
+
+char *prepareGFS(const char *host, int nnPort, const char *dirsubpath, const char *user) {
+    return (prepareQUERY(host, nnPort, dirsubpath, "GETFILESTATUS", user));
+}
+
+char *prepareLS(const char *host, int nnPort, const char *dirsubpath, const char *user) {
+    return (prepareQUERY(host, nnPort, dirsubpath, "LISTSTATUS", user));
+}
+
+char *prepareCHMOD(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user) {
+    char *url;
+    char *permission;
+    permission = (char*) malloc(NUM_OF_PERMISSION_BITS);
+    if (!permission) {
+        return NULL;
+    }
+    mode &= 0x3FFF;
+    mode = decToOctal(mode);
+    sprintf(permission,"%d",mode);
+    url = prepareQUERY(host, nnPort, dirsubpath, "SETPERMISSION", user);
+    url = realloc(url,(strlen(url) + strlen("&permission=") + strlen(permission) + 1));
+    if (!url) {
+        return NULL;
+    }
+    url = strcat(url,"&permission=");
+    url = strcat(url,permission);
+    return url;
+}
+
+char *prepareDELETE(const char *host, int nnPort, const char *dirsubpath, int recursive, const char *user) {
+    char *url = (prepareQUERY(host, nnPort, dirsubpath, "DELETE", user));
+    char *recursiveFlag = (char *)malloc(6);
+    if (!recursive) {
+        strcpy(recursiveFlag, "false");
+    } else {
+        strcpy(recursiveFlag, "true");
+    }
+    url = (char *) realloc(url, strlen(url) + strlen("&recursive=") + strlen(recursiveFlag) + 1);
+    if (!url) {
+        return NULL;
+    }
+    
+    strcat(url, "&recursive=");
+    strcat(url, recursiveFlag);
+    return url;
+}
+
+char *prepareCHOWN(const char *host, int nnPort, const char *dirsubpath, const char *owner, const char *group, const char *user) {
+    char *url;
+    url = prepareQUERY(host, nnPort, dirsubpath, "SETOWNER", user);
+    if (!url) {
+        return NULL;
+    }
+    if(owner != NULL) {
+        url = realloc(url,(strlen(url) + strlen("&owner=") + strlen(owner) + 1));
+        url = strcat(url,"&owner=");
+        url = strcat(url,owner);
+    }
+    if (group != NULL) {
+        url = realloc(url,(strlen(url) + strlen("&group=") + strlen(group) + 1));
+        url = strcat(url,"&group=");
+        url = strcat(url,group);
+    }
+    return url;
+}
+
+char *prepareOPEN(const char *host, int nnPort, const char *dirsubpath, const char *user, size_t offset, size_t length) {
+    char *base_url = prepareQUERY(host, nnPort, dirsubpath, "OPEN", user);
+    char *url = (char *) malloc(strlen(base_url) + strlen("&offset=") + 15 + strlen("&length=") + 15);
+    if (!url) {
+        return NULL;
+    }
+    sprintf(url, "%s&offset=%ld&length=%ld", base_url, offset, length);
+    return url;
+}
+
+char *prepareUTIMES(const char *host, int nnPort, const char *dirsubpath, long unsigned mTime, long unsigned aTime, const char *user) {
+    char *url;
+    char *modTime;
+    char *acsTime;
+    modTime = (char*) malloc(12);
+    acsTime = (char*) malloc(12);
+    url = prepareQUERY(host, nnPort, dirsubpath, "SETTIMES", user);
+    sprintf(modTime,"%lu",mTime);
+    sprintf(acsTime,"%lu",aTime);
+    url = realloc(url,(strlen(url) + strlen("&modificationtime=") + strlen(modTime) + strlen("&accesstime=") + strlen(acsTime) + 1));
+    if (!url) {
+        return NULL;
+    }
+    url = strcat(url, "&modificationtime=");
+    url = strcat(url, modTime);
+    url = strcat(url,"&accesstime=");
+    url = strcat(url, acsTime);
+    return url;
+}
+
+char *prepareNnWRITE(const char *host, int nnPort, const char *dirsubpath, const char *user, int16_t replication, size_t blockSize) {
+    char *url;
+    url = prepareQUERY(host, nnPort, dirsubpath, "CREATE", user);
+    url = realloc(url, (strlen(url) + strlen("&overwrite=true") + 1));
+    if (!url) {
+        return NULL;
+    }
+    url = strcat(url, "&overwrite=true");
+    if (replication > 0) {
+        url = realloc(url, (strlen(url) + strlen("&replication=") + 6));
+        if (!url) {
+            return NULL;
+        }
+        sprintf(url, "%s&replication=%d", url, replication);
+    }
+    if (blockSize > 0) {
+        url = realloc(url, (strlen(url) + strlen("&blocksize=") + 16));
+        if (!url) {
+            return NULL;
+        }
+        sprintf(url, "%s&blocksize=%ld", url, blockSize);
+    }
+    return url;
+}
+
+char *prepareNnAPPEND(const char *host, int nnPort, const char *dirsubpath, const char *user) {
+    return (prepareQUERY(host, nnPort, dirsubpath, "APPEND", user));
+}
+
+char *prepareSETREPLICATION(const char *host, int nnPort, const char *path, int16_t replication, const char *user)
+{
+    char *url = prepareQUERY(host, nnPort, path, "SETREPLICATION", user);
+    char *replicationNum = (char *) malloc(NUM_OF_REPLICATION_BITS);
+    sprintf(replicationNum, "%u", replication);
+    url = realloc(url, strlen(url) + strlen("&replication=") + strlen(replicationNum)+ 1);
+    if (!url) {
+        return NULL;
+    }
+    
+    url = strcat(url, "&replication=");
+    url = strcat(url, replicationNum);
+    return url;
+}

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.h

@@ -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.
+ */
+
+
+#ifndef _HDFS_HTTP_QUERY_H_
+#define _HDFS_HTTP_QUERY_H_
+
+#include <stdint.h>
+#include <stdio.h>
+
+char *prepareMKDIR(const char *host, int nnPort, const char *dirsubpath, const char *user);
+char *prepareMKDIRwithMode(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user);
+char *prepareRENAME(const char *host, int nnPort, const char *srcpath, const char *destpath, const char *user);
+char *prepareCHMOD(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user);
+char *prepareGFS(const char *host, int nnPort, const char *dirsubpath, const char *user);
+char *prepareLS(const char *host, int nnPort, const char *dirsubpath, const char *user);
+char *prepareDELETE(const char *host, int nnPort, const char *dirsubpath, int recursive, const char *user);
+char *prepareCHOWN(const char *host, int nnPort, const char *dirsubpath, const char *owner, const char *group, const char *user);
+char *prepareOPEN(const char *host, int nnPort, const char *dirsubpath, const char *user, size_t offset, size_t length);
+char *prepareUTIMES(const char *host, int nnPort, const char *dirsubpath, long unsigned mTime, long unsigned aTime, const char *user);
+char *prepareNnWRITE(const char *host, int nnPort, const char *dirsubpath, const char *user, int16_t replication, size_t blockSize);
+char *prepareNnAPPEND(const char *host, int nnPort, const char *dirsubpath, const char *user);
+char *prepareSETREPLICATION(const char *host, int nnPort, const char *path, int16_t replication, const char *user);
+
+
+#endif  //_HDFS_HTTP_QUERY_H_

+ 616 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_jni.c

@@ -0,0 +1,616 @@
+/**
+ * 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.
+ */
+
+#include <stdio.h>
+#include <string.h>
+#include "webhdfs.h"
+#include "jni_helper.h"
+#include "exception.h"
+
+/* Some frequently used Java paths */
+#define HADOOP_CONF     "org/apache/hadoop/conf/Configuration"
+#define HADOOP_PATH     "org/apache/hadoop/fs/Path"
+#define HADOOP_LOCALFS  "org/apache/hadoop/fs/LocalFileSystem"
+#define HADOOP_FS       "org/apache/hadoop/fs/FileSystem"
+#define HADOOP_FSSTATUS "org/apache/hadoop/fs/FsStatus"
+#define HADOOP_BLK_LOC  "org/apache/hadoop/fs/BlockLocation"
+#define HADOOP_DFS      "org/apache/hadoop/hdfs/DistributedFileSystem"
+#define HADOOP_ISTRM    "org/apache/hadoop/fs/FSDataInputStream"
+#define HADOOP_OSTRM    "org/apache/hadoop/fs/FSDataOutputStream"
+#define HADOOP_STAT     "org/apache/hadoop/fs/FileStatus"
+#define HADOOP_FSPERM   "org/apache/hadoop/fs/permission/FsPermission"
+#define JAVA_NET_ISA    "java/net/InetSocketAddress"
+#define JAVA_NET_URI    "java/net/URI"
+#define JAVA_STRING     "java/lang/String"
+
+#define JAVA_VOID       "V"
+
+/* Macros for constructing method signatures */
+#define JPARAM(X)           "L" X ";"
+#define JARRPARAM(X)        "[L" X ";"
+#define JMETHOD1(X, R)      "(" X ")" R
+#define JMETHOD2(X, Y, R)   "(" X Y ")" R
+#define JMETHOD3(X, Y, Z, R)   "(" X Y Z")" R
+
+#define KERBEROS_TICKET_CACHE_PATH "hadoop.security.kerberos.ticket.cache.path"
+
+/**
+ * Helper function to create a org.apache.hadoop.fs.Path object.
+ * @param env: The JNIEnv pointer.
+ * @param path: The file-path for which to construct org.apache.hadoop.fs.Path
+ * object.
+ * @return Returns a jobject on success and NULL on error.
+ */
+static jthrowable constructNewObjectOfPath(JNIEnv *env, const char *path,
+                                           jobject *out)
+{
+    jthrowable jthr;
+    jstring jPathString;
+    jobject jPath;
+    
+    //Construct a java.lang.String object
+    jthr = newJavaStr(env, path, &jPathString);
+    if (jthr)
+        return jthr;
+    //Construct the org.apache.hadoop.fs.Path object
+    jthr = constructNewObjectOfClass(env, &jPath, "org/apache/hadoop/fs/Path",
+                                     "(Ljava/lang/String;)V", jPathString);
+    destroyLocalReference(env, jPathString);
+    if (jthr)
+        return jthr;
+    *out = jPath;
+    return NULL;
+}
+
+/**
+ * Set a configuration value.
+ *
+ * @param env               The JNI environment
+ * @param jConfiguration    The configuration object to modify
+ * @param key               The key to modify
+ * @param value             The value to set the key to
+ *
+ * @return                  NULL on success; exception otherwise
+ */
+static jthrowable hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
+                                   const char *key, const char *value)
+{
+    jthrowable jthr;
+    jstring jkey = NULL, jvalue = NULL;
+    
+    jthr = newJavaStr(env, key, &jkey);
+    if (jthr)
+        goto done;
+    jthr = newJavaStr(env, value, &jvalue);
+    if (jthr)
+        goto done;
+    jthr = invokeMethod(env, NULL, INSTANCE, jConfiguration,
+                        HADOOP_CONF, "set", JMETHOD2(JPARAM(JAVA_STRING),
+                                                     JPARAM(JAVA_STRING), JAVA_VOID),
+                        jkey, jvalue);
+    if (jthr)
+        goto done;
+done:
+    destroyLocalReference(env, jkey);
+    destroyLocalReference(env, jvalue);
+    return jthr;
+}
+
+static jthrowable hadoopConfGetStr(JNIEnv *env, jobject jConfiguration,
+                                   const char *key, char **val)
+{
+    jthrowable jthr;
+    jvalue jVal;
+    jstring jkey = NULL, jRet = NULL;
+    
+    jthr = newJavaStr(env, key, &jkey);
+    if (jthr)
+        goto done;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
+                        HADOOP_CONF, "get", JMETHOD1(JPARAM(JAVA_STRING),
+                                                     JPARAM(JAVA_STRING)), jkey);
+    if (jthr)
+        goto done;
+    jRet = jVal.l;
+    jthr = newCStr(env, jRet, val);
+done:
+    destroyLocalReference(env, jkey);
+    destroyLocalReference(env, jRet);
+    return jthr;
+}
+
+int hdfsConfGetStr(const char *key, char **val)
+{
+    JNIEnv *env;
+    int ret;
+    jthrowable jthr;
+    jobject jConfiguration = NULL;
+    
+    env = getJNIEnv();
+    if (env == NULL) {
+        ret = EINTERNAL;
+        goto done;
+    }
+    jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsConfGetStr(%s): new Configuration", key);
+        goto done;
+    }
+    jthr = hadoopConfGetStr(env, jConfiguration, key, val);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsConfGetStr(%s): hadoopConfGetStr", key);
+        goto done;
+    }
+    ret = 0;
+done:
+    destroyLocalReference(env, jConfiguration);
+    if (ret)
+        errno = ret;
+    return ret;
+}
+
+void hdfsConfStrFree(char *val)
+{
+    free(val);
+}
+
+static jthrowable hadoopConfGetInt(JNIEnv *env, jobject jConfiguration,
+                                   const char *key, int32_t *val)
+{
+    jthrowable jthr = NULL;
+    jvalue jVal;
+    jstring jkey = NULL;
+    
+    jthr = newJavaStr(env, key, &jkey);
+    if (jthr)
+        return jthr;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
+                        HADOOP_CONF, "getInt", JMETHOD2(JPARAM(JAVA_STRING), "I", "I"),
+                        jkey, (jint)(*val));
+    destroyLocalReference(env, jkey);
+    if (jthr)
+        return jthr;
+    *val = jVal.i;
+    return NULL;
+}
+
+int hdfsConfGetInt(const char *key, int32_t *val)
+{
+    JNIEnv *env;
+    int ret;
+    jobject jConfiguration = NULL;
+    jthrowable jthr;
+    
+    env = getJNIEnv();
+    if (env == NULL) {
+        ret = EINTERNAL;
+        goto done;
+    }
+    jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsConfGetInt(%s): new Configuration", key);
+        goto done;
+    }
+    jthr = hadoopConfGetInt(env, jConfiguration, key, val);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsConfGetInt(%s): hadoopConfGetInt", key);
+        goto done;
+    }
+    ret = 0;
+done:
+    destroyLocalReference(env, jConfiguration);
+    if (ret)
+        errno = ret;
+    return ret;
+}
+
+/**
+ * Calculate the effective URI to use, given a builder configuration.
+ *
+ * If there is not already a URI scheme, we prepend 'hdfs://'.
+ *
+ * If there is not already a port specified, and a port was given to the
+ * builder, we suffix that port.  If there is a port specified but also one in
+ * the URI, that is an error.
+ *
+ * @param bld       The hdfs builder object
+ * @param uri       (out param) dynamically allocated string representing the
+ *                  effective URI
+ *
+ * @return          0 on success; error code otherwise
+ */
+static int calcEffectiveURI(struct hdfsBuilder *bld, char ** uri)
+{
+    const char *scheme;
+    char suffix[64];
+    const char *lastColon;
+    char *u;
+    size_t uriLen;
+    
+    if (!bld->nn_jni)
+        return EINVAL;
+    scheme = (strstr(bld->nn_jni, "://")) ? "" : "hdfs://";
+    if (bld->port == 0) {
+        suffix[0] = '\0';
+    } else {
+        lastColon = rindex(bld->nn_jni, ':');
+        if (lastColon && (strspn(lastColon + 1, "0123456789") ==
+                          strlen(lastColon + 1))) {
+            fprintf(stderr, "port %d was given, but URI '%s' already "
+                    "contains a port!\n", bld->port, bld->nn_jni);
+            return EINVAL;
+        }
+        snprintf(suffix, sizeof(suffix), ":%d", bld->port);
+    }
+    
+    uriLen = strlen(scheme) + strlen(bld->nn_jni) + strlen(suffix);
+    u = malloc((uriLen + 1) * (sizeof(char)));
+    if (!u) {
+        fprintf(stderr, "calcEffectiveURI: out of memory");
+        return ENOMEM;
+    }
+    snprintf(u, uriLen + 1, "%s%s%s", scheme, bld->nn_jni, suffix);
+    *uri = u;
+    return 0;
+}
+
+static const char *maybeNull(const char *str)
+{
+    return str ? str : "(NULL)";
+}
+
+const char *hdfsBuilderToStr(const struct hdfsBuilder *bld,
+                                    char *buf, size_t bufLen)
+{
+    snprintf(buf, bufLen, "forceNewInstance=%d, nn=%s, port=%d, "
+             "kerbTicketCachePath=%s, userName=%s, workingDir=%s\n",
+             bld->forceNewInstance, maybeNull(bld->nn), bld->port,
+             maybeNull(bld->kerbTicketCachePath),
+             maybeNull(bld->userName), maybeNull(bld->workingDir));
+    return buf;
+}
+
+/*
+ * The JNI version of builderConnect, return the reflection of FileSystem
+ */
+jobject hdfsBuilderConnect_JNI(JNIEnv *env, struct hdfsBuilder *bld)
+{
+    jobject jConfiguration = NULL, jFS = NULL, jURI = NULL, jCachePath = NULL;
+    jstring jURIString = NULL, jUserString = NULL;
+    jvalue  jVal;
+    jthrowable jthr = NULL;
+    char *cURI = 0, buf[512];
+    int ret;
+    jobject jRet = NULL;
+    
+    //  jConfiguration = new Configuration();
+    jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsBuilderConnect_JNI(%s)", hdfsBuilderToStr(bld, buf, sizeof(buf)));
+        goto done;
+    }
+    
+    //Check what type of FileSystem the caller wants...
+    if (bld->nn_jni == NULL) {
+        // Get a local filesystem.
+        // Also handle the scenario where nn of hdfsBuilder is set to localhost.
+        if (bld->forceNewInstance) {
+            // fs = FileSytem#newInstanceLocal(conf);
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
+                                "newInstanceLocal", JMETHOD1(JPARAM(HADOOP_CONF),
+                                                             JPARAM(HADOOP_LOCALFS)), jConfiguration);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jFS = jVal.l;
+        } else {
+            // fs = FileSytem#getLocal(conf);
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS, "getLocal",
+                                JMETHOD1(JPARAM(HADOOP_CONF),
+                                         JPARAM(HADOOP_LOCALFS)),
+                                jConfiguration);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jFS = jVal.l;
+        }
+    } else {
+        if (!strcmp(bld->nn_jni, "default")) {
+            // jURI = FileSystem.getDefaultUri(conf)
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
+                                "getDefaultUri",
+                                "(Lorg/apache/hadoop/conf/Configuration;)Ljava/net/URI;",
+                                jConfiguration);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jURI = jVal.l;
+        } else {
+            // fs = FileSystem#get(URI, conf, ugi);
+            ret = calcEffectiveURI(bld, &cURI);
+            if (ret)
+                goto done;
+            jthr = newJavaStr(env, cURI, &jURIString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, JAVA_NET_URI,
+                                "create", "(Ljava/lang/String;)Ljava/net/URI;",
+                                jURIString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jURI = jVal.l;
+        }
+        
+        if (bld->kerbTicketCachePath) {
+            jthr = hadoopConfSetStr(env, jConfiguration,
+                                    KERBEROS_TICKET_CACHE_PATH, bld->kerbTicketCachePath);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+        }
+        jthr = newJavaStr(env, bld->userName, &jUserString);
+        if (jthr) {
+            ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                        "hdfsBuilderConnect_JNI(%s)",
+                                        hdfsBuilderToStr(bld, buf, sizeof(buf)));
+            goto done;
+        }
+        if (bld->forceNewInstance) {
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
+                                "newInstance", JMETHOD3(JPARAM(JAVA_NET_URI),
+                                                        JPARAM(HADOOP_CONF), JPARAM(JAVA_STRING),
+                                                        JPARAM(HADOOP_FS)),
+                                jURI, jConfiguration, jUserString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jFS = jVal.l;
+        } else {
+            jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS, "get",
+                                JMETHOD3(JPARAM(JAVA_NET_URI), JPARAM(HADOOP_CONF),
+                                         JPARAM(JAVA_STRING), JPARAM(HADOOP_FS)),
+                                jURI, jConfiguration, jUserString);
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect_JNI(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            jFS = jVal.l;
+        }
+    }
+    jRet = (*env)->NewGlobalRef(env, jFS);
+    if (!jRet) {
+        ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
+                                           "hdfsBuilderConnect_JNI(%s)",
+                                           hdfsBuilderToStr(bld, buf, sizeof(buf)));
+        goto done;
+    }
+    ret = 0;
+    
+done:
+    // Release unnecessary local references
+    destroyLocalReference(env, jConfiguration);
+    destroyLocalReference(env, jFS);
+    destroyLocalReference(env, jURI);
+    destroyLocalReference(env, jCachePath);
+    destroyLocalReference(env, jURIString);
+    destroyLocalReference(env, jUserString);
+    free(cURI);
+    
+    if (ret) {
+        errno = ret;
+        return NULL;
+    }
+    return jRet;
+}
+
+int hdfsDisconnect_JNI(jobject jFS)
+{
+    // JAVA EQUIVALENT:
+    //  fs.close()
+    
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+    int ret;
+    
+    if (env == NULL) {
+        errno = EINTERNAL;
+        return -1;
+    }
+    
+    //Sanity check
+    if (jFS == NULL) {
+        errno = EBADF;
+        return -1;
+    }
+    
+    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
+                                   "close", "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsDisconnect: FileSystem#close");
+    } else {
+        ret = 0;
+    }
+    (*env)->DeleteGlobalRef(env, jFS);
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
+    return 0;
+}
+
+static int hdfsCopyImpl(hdfsFS srcFS, const char* src, hdfsFS dstFS,
+                        const char* dst, jboolean deleteSource)
+{
+    //JAVA EQUIVALENT
+    //  FileUtil#copy(srcFS, srcPath, dstFS, dstPath,
+    //                 deleteSource = false, conf)
+    
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+    if (env == NULL) {
+        errno = EINTERNAL;
+        return -1;
+    }
+    
+    //In libwebhdfs, the hdfsFS derived from hdfsBuilderConnect series functions
+    //is actually a hdfsBuilder instance containing address information of NameNode.
+    //Thus here we need to use JNI to get the real java FileSystem objects.
+    jobject jSrcFS = hdfsBuilderConnect_JNI(env, (struct hdfsBuilder *) srcFS);
+    jobject jDstFS = hdfsBuilderConnect_JNI(env, (struct hdfsBuilder *) dstFS);
+    
+    //Parameters
+    jobject jConfiguration = NULL, jSrcPath = NULL, jDstPath = NULL;
+    jthrowable jthr;
+    jvalue jVal;
+    int ret;
+    
+    jthr = constructNewObjectOfPath(env, src, &jSrcPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsCopyImpl(src=%s): constructNewObjectOfPath", src);
+        goto done;
+    }
+    jthr = constructNewObjectOfPath(env, dst, &jDstPath);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsCopyImpl(dst=%s): constructNewObjectOfPath", dst);
+        goto done;
+    }
+    
+    //Create the org.apache.hadoop.conf.Configuration object
+    jthr = constructNewObjectOfClass(env, &jConfiguration,
+                                     HADOOP_CONF, "()V");
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsCopyImpl: Configuration constructor");
+        goto done;
+    }
+    
+    //FileUtil#copy
+    jthr = invokeMethod(env, &jVal, STATIC,
+                        NULL, "org/apache/hadoop/fs/FileUtil", "copy",
+                        "(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
+                        "Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
+                        "ZLorg/apache/hadoop/conf/Configuration;)Z",
+                        jSrcFS, jSrcPath, jDstFS, jDstPath, deleteSource,
+                        jConfiguration);
+    if (jthr) {
+        ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                    "hdfsCopyImpl(src=%s, dst=%s, deleteSource=%d): "
+                                    "FileUtil#copy", src, dst, deleteSource);
+        goto done;
+    }
+    if (!jVal.z) {
+        ret = EIO;
+        goto done;
+    }
+    ret = 0;
+    
+done:
+    destroyLocalReference(env, jConfiguration);
+    destroyLocalReference(env, jSrcPath);
+    destroyLocalReference(env, jDstPath);
+    //Disconnect src/dst FileSystem
+    hdfsDisconnect_JNI(jSrcFS);
+    hdfsDisconnect_JNI(jDstFS);
+    
+    if (ret) {
+        errno = ret;
+        return -1;
+    }
+    return 0;
+}
+
+int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
+{
+    return hdfsCopyImpl(srcFS, src, dstFS, dst, 0);
+}
+
+int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
+{
+    return hdfsCopyImpl(srcFS, src, dstFS, dst, 1);
+}
+
+tOffset hdfsGetDefaultBlockSize(hdfsFS fs)
+{
+    // JAVA EQUIVALENT:
+    //  fs.getDefaultBlockSize();
+    
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+    if (env == NULL) {
+        errno = EINTERNAL;
+        return -1;
+    }
+    
+    //In libwebhdfs, the hdfsFS derived from hdfsConnect functions
+    //is actually a hdfsBuilder instance containing address information of NameNode.
+    //Thus here we need to use JNI to get the real java FileSystem objects.
+    jobject jFS = hdfsBuilderConnect_JNI(env, (struct hdfsBuilder *) fs);
+    
+    //FileSystem#getDefaultBlockSize()
+    jvalue jVal;
+    jthrowable jthr;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                        "getDefaultBlockSize", "()J");
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                      "hdfsGetDefaultBlockSize: FileSystem#getDefaultBlockSize");
+        //Disconnect
+        hdfsDisconnect_JNI(jFS);
+        return -1;
+    }
+    
+    //Disconnect
+    hdfsDisconnect_JNI(jFS);
+    return jVal.j;
+}
+
+
+

+ 388 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c

@@ -0,0 +1,388 @@
+/**
+ * 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.
+ */
+#include <stdlib.h>
+#include <string.h>
+#include <ctype.h>
+#include <jansson.h>
+#include "hdfs_json_parser.h"
+#include "exception.h"
+
+hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat, int *numEntries, const char *operation); //Forward Declaration
+
+static hdfsFileInfo *json_parse_array(json_t *jobj, char *key, hdfsFileInfo *fileStat, int *numEntries, const char *operation) {
+    int arraylen = json_array_size(jobj);                      //Getting the length of the array
+    *numEntries = arraylen;
+    if (!key) {
+        return NULL;
+    }
+    if(arraylen > 0) {
+        fileStat = (hdfsFileInfo *)realloc(fileStat,sizeof(hdfsFileInfo)*arraylen);
+    }
+    json_t *jvalue;
+    int i;
+    for (i=0; i< arraylen; i++) {
+        jvalue = json_array_get(jobj, i);            //Getting the array element at position i
+        if (json_is_array(jvalue)) {                 // array within an array - program should never come here for now
+            json_parse_array(jvalue, NULL, &fileStat[i], numEntries, operation);
+        }
+        else if (json_is_object(jvalue)) {           // program will definitely come over here
+            parseJsonGFS(jvalue, &fileStat[i], numEntries, operation);
+        }
+        else {
+            return NULL;                               // program will never come over here for now
+        }
+    }
+    *numEntries = arraylen;
+    return fileStat;
+}
+
+int parseBoolean(char *response) {
+    json_t *root;
+    json_error_t error;
+    size_t flags = 0;
+    int result = 0;
+    const char *key;
+    json_t *value;
+    root = json_loads(response, flags, &error);
+    void *iter = json_object_iter(root);
+    while(iter)  {
+        key = json_object_iter_key(iter);
+        value = json_object_iter_value(iter);
+        switch (json_typeof(value))  {
+            case JSON_TRUE:
+                result = 1;
+                break;
+            default:
+                result = 0;
+                break;
+        }
+        iter = json_object_iter_next(root, iter);
+    }
+    return result;
+}
+
+int parseMKDIR(char *response) {
+    return (parseBoolean(response));
+}
+
+int parseRENAME(char *response) {
+    return (parseBoolean(response));
+}
+
+int parseDELETE(char *response) {
+    return (parseBoolean(response));
+}
+
+hdfs_exception_msg *parseJsonException(json_t *jobj) {
+    const char *key;
+    json_t *value;
+    hdfs_exception_msg *exception = NULL;
+    
+    exception = (hdfs_exception_msg *) calloc(1, sizeof(hdfs_exception_msg));
+    if (!exception) {
+        return NULL;
+    }
+    
+    void *iter = json_object_iter(jobj);
+    while (iter) {
+        key = json_object_iter_key(iter);
+        value = json_object_iter_value(iter);
+        
+        if (!strcmp(key, "exception")) {
+            exception->exception = json_string_value(value);
+        } else if (!strcmp(key, "javaClassName")) {
+            exception->javaClassName = json_string_value(value);
+        } else if (!strcmp(key, "message")) {
+            exception->message = json_string_value(value);
+        }
+        
+        iter = json_object_iter_next(jobj, iter);
+    }
+    
+    return exception;
+}
+
+hdfs_exception_msg *parseException(const char *content) {
+    if (!content) {
+        return NULL;
+    }
+    
+    json_error_t error;
+    size_t flags = 0;
+    const char *key;
+    json_t *value;
+    json_t *jobj = json_loads(content, flags, &error);
+    
+    if (!jobj) {
+        fprintf(stderr, "JSon parsing failed\n");
+        return NULL;
+    }
+    void *iter = json_object_iter(jobj);
+    while(iter)  {
+        key = json_object_iter_key(iter);
+        value = json_object_iter_value(iter);
+        
+        if (!strcmp(key, "RemoteException") && json_typeof(value) == JSON_OBJECT) {
+            return parseJsonException(value);
+        }
+        iter = json_object_iter_next(jobj, iter);
+    }
+    return NULL;
+}
+
+hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat, int *numEntries, const char *operation) {
+    const char *tempstr;
+    const char *key;
+    json_t *value;
+    void *iter = json_object_iter(jobj);
+    while(iter)  {
+        key = json_object_iter_key(iter);
+        value = json_object_iter_value(iter);
+        
+        switch (json_typeof(value)) {
+            case JSON_INTEGER:
+                if(!strcmp(key,"accessTime")) {
+                    fileStat->mLastAccess = (time_t)(json_integer_value(value)/1000);
+                } else if (!strcmp(key,"blockSize")) {
+                    fileStat->mBlockSize = (tOffset)json_integer_value(value);
+                } else if (!strcmp(key,"length")) {
+                    fileStat->mSize = (tOffset)json_integer_value(value);
+                } else if(!strcmp(key,"modificationTime")) {
+                    fileStat->mLastMod = (time_t)(json_integer_value(value)/1000);
+                } else if (!strcmp(key,"replication")) {
+                    fileStat->mReplication = (short)json_integer_value(value);
+                }
+                break;
+                
+            case JSON_STRING:
+                if(!strcmp(key,"group")) {
+                    fileStat->mGroup=(char *)json_string_value(value);
+                } else if (!strcmp(key,"owner")) {
+                    fileStat->mOwner=(char *)json_string_value(value);
+                } else if (!strcmp(key,"pathSuffix")) {
+                    fileStat->mName=(char *)json_string_value(value);
+                } else if (!strcmp(key,"permission")) {
+                    tempstr=(char *)json_string_value(value);
+                    fileStat->mPermissions = (short)strtol(tempstr,(char **)NULL,8);
+                } else if (!strcmp(key,"type")) {
+                    char *cvalue = (char *)json_string_value(value);
+                    if (!strcmp(cvalue, "DIRECTORY")) {
+                        fileStat->mKind = kObjectKindDirectory;
+                    } else {
+                        fileStat->mKind = kObjectKindFile;
+                    }
+                }
+                break;
+                
+            case JSON_OBJECT:
+                if(!strcmp(key,"FileStatus")) {
+                    parseJsonGFS(value, fileStat, numEntries, operation);
+                } else if (!strcmp(key,"FileStatuses")) {
+                    fileStat = parseJsonGFS(value, &fileStat[0], numEntries, operation);
+                } else if (!strcmp(key,"RemoteException")) {
+                    //Besides returning NULL, we also need to print the exception information
+                    hdfs_exception_msg *exception = parseJsonException(value);
+                    if (exception) {
+                        errno = printExceptionWeb(exception, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation);
+                    }
+                    
+                    if(fileStat != NULL) {
+                        free(fileStat);
+                        fileStat = NULL;
+                    }
+                }
+                break;
+                
+            case JSON_ARRAY:
+                if (!strcmp(key,"FileStatus")) {
+                    fileStat = json_parse_array(value,(char *) key,fileStat,numEntries, operation);
+                }
+                break;
+                
+            default:
+                if(fileStat != NULL) {
+                    free(fileStat);
+                    fileStat = NULL;
+                }
+        }
+        iter = json_object_iter_next(jobj, iter);
+    }
+    return fileStat;
+}
+
+
+int checkHeader(char *header, const char *content, const char *operation) {
+    char *result = NULL;
+    char delims[] = ":";
+    char *responseCode= "200 OK";
+    if(header == '\0' || strncmp(header, "HTTP/", strlen("HTTP/"))) {
+        return 0;
+    }
+    if(!(strstr(header, responseCode)) || !(header = strstr(header, "Content-Length"))) {
+        hdfs_exception_msg *exc = parseException(content);
+        if (exc) {
+            errno = printExceptionWeb(exc, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation);
+        }
+        return 0;
+    }
+    result = strtok(header, delims);
+    result = strtok(NULL,delims);
+    while (isspace(*result)) {
+        result++;
+    }
+    if(strcmp(result,"0")) {                 //Content-Length should be equal to 0
+        return 1;
+    } else {
+        return 0;
+    }
+}
+
+int parseOPEN(const char *header, const char *content) {
+    const char *responseCode1 = "307 TEMPORARY_REDIRECT";
+    const char *responseCode2 = "200 OK";
+    if(header == '\0' || strncmp(header,"HTTP/",strlen("HTTP/"))) {
+        return -1;
+    }
+    if(!(strstr(header,responseCode1) && strstr(header, responseCode2))) {
+        hdfs_exception_msg *exc = parseException(content);
+        if (exc) {
+            //if the exception is an IOException and it is because the offset is out of the range
+            //do not print out the exception
+            if (!strcasecmp(exc->exception, "IOException") && strstr(exc->message, "out of the range")) {
+                return 0;
+            }
+            errno = printExceptionWeb(exc, PRINT_EXC_ALL, "Calling WEBHDFS (OPEN)");
+        }
+        return -1;
+    }
+    
+    return 1;
+}
+
+int parseCHMOD(char *header, const char *content) {
+    return checkHeader(header, content, "CHMOD");
+}
+
+
+int parseCHOWN(char *header, const char *content) {
+    return checkHeader(header, content, "CHOWN");
+}
+
+int parseUTIMES(char *header, const char *content) {
+    return checkHeader(header, content, "UTIMES");
+}
+
+
+int checkIfRedirect(const char *const headerstr, const char *content, const char *operation) {
+    char *responseCode = "307 TEMPORARY_REDIRECT";
+    char * locTag = "Location";
+    char * tempHeader;
+    if(headerstr == '\0' || strncmp(headerstr,"HTTP/", 5)) {
+        return 0;
+    }
+    if(!(tempHeader = strstr(headerstr,responseCode))) {
+        //process possible exception information
+        hdfs_exception_msg *exc = parseException(content);
+        if (exc) {
+            errno = printExceptionWeb(exc, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation);
+        }
+        return 0;
+    }
+    if(!(strstr(tempHeader,locTag))) {
+        return 0;
+    }
+    return 1;
+}
+
+
+int parseNnWRITE(const char *header, const char *content) {
+    return checkIfRedirect(header, content, "Write(NameNode)");
+}
+
+
+int parseNnAPPEND(const char *header, const char *content) {
+    return checkIfRedirect(header, content, "Append(NameNode)");
+}
+
+char *parseDnLoc(char *content) {
+    char delims[] = "\r\n";
+    char *url = NULL;
+    char *DnLocation = NULL;
+    char *savepter;
+    DnLocation = strtok_r(content, delims, &savepter);
+    while (DnLocation && strncmp(DnLocation, "Location:", strlen("Location:"))) {
+        DnLocation = strtok_r(NULL, delims, &savepter);
+    }
+    if (!DnLocation) {
+        return NULL;
+    }
+    DnLocation = strstr(DnLocation, "http");
+    if (!DnLocation) {
+        return NULL;
+    }
+    url = malloc(strlen(DnLocation) + 1);
+    if (!url) {
+        return NULL;
+    }
+    strcpy(url, DnLocation);
+    return url;
+}
+
+int parseDnWRITE(const char *header, const char *content) {
+    char *responseCode = "201 Created";
+    fprintf(stderr, "\nheaderstr is: %s\n", header);
+    if(header == '\0' || strncmp(header,"HTTP/",strlen("HTTP/"))) {
+        return 0;
+    }
+    if(!(strstr(header,responseCode))) {
+        hdfs_exception_msg *exc = parseException(content);
+        if (exc) {
+            errno = printExceptionWeb(exc, PRINT_EXC_ALL, "Calling WEBHDFS (WRITE(DataNode))");
+        }
+        return 0;
+    }
+    return 1;
+}
+
+int parseDnAPPEND(const char *header, const char *content) {
+    char *responseCode = "200 OK";
+    if(header == '\0' || strncmp(header, "HTTP/", strlen("HTTP/"))) {
+        return 0;
+    }
+    if(!(strstr(header, responseCode))) {
+        hdfs_exception_msg *exc = parseException(content);
+        if (exc) {
+            errno = printExceptionWeb(exc, PRINT_EXC_ALL, "Calling WEBHDFS (APPEND(DataNode))");
+        }
+        return 0;
+    }
+    return 1;
+}
+
+hdfsFileInfo *parseGFS(char *str, hdfsFileInfo *fileStat, int *numEntries) {
+    json_error_t error;
+    size_t flags = 0;
+    json_t *jobj = json_loads(str, flags, &error);
+    fileStat = parseJsonGFS(jobj, fileStat, numEntries, "GETPATHSTATUS/LISTSTATUS");
+    return fileStat;
+}
+
+int parseSETREPLICATION(char *response) {
+    return (parseBoolean(response));
+}
+

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.h

@@ -0,0 +1,42 @@
+/**
+ * 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.
+ */
+#ifndef _HDFS_JSON_PARSER_H_
+#define _HDFS_JSON_PARSER_H_
+#include "webhdfs.h"
+
+int parseMKDIR(char *response);
+int parseRENAME(char *response);
+int parseDELETE (char *response);
+int parseSETREPLICATION(char *response);
+
+int parseOPEN(const char *header, const char *content);
+
+int parseNnWRITE(const char *header, const char *content);
+int parseDnWRITE(const char *header, const char *content);
+int parseNnAPPEND(const char *header, const char *content);
+int parseDnAPPEND(const char *header, const char *content);
+
+char* parseDnLoc(char *content);
+
+hdfsFileInfo *parseGFS(char *response, hdfsFileInfo *fileStat, int *numEntries);
+
+int parseCHOWN (char *header, const char *content);
+int parseCHMOD (char *header, const char *content);
+int parseUTIMES(char *header, const char *content);
+
+#endif //_FUSE_JSON_PARSER_H

+ 1113 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c

@@ -0,0 +1,1113 @@
+/**
+ * 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.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <jni.h>
+#include "webhdfs.h"
+#include "hdfs_http_client.h"
+#include "hdfs_http_query.h"
+#include "hdfs_json_parser.h"
+#include "jni_helper.h"
+#include "exception.h"
+
+#define HADOOP_HDFS_CONF        "org/apache/hadoop/hdfs/HdfsConfiguration"
+#define HADOOP_NAMENODE         "org/apache/hadoop/hdfs/server/namenode/NameNode"
+#define JAVA_INETSOCKETADDRESS  "java/net/InetSocketAddress"
+
+static void initFileinfo(hdfsFileInfo *fileInfo) {
+    if (fileInfo) {
+        fileInfo->mKind = kObjectKindFile;
+        fileInfo->mName = NULL;
+        fileInfo->mLastMod = 0;
+        fileInfo->mSize = 0;
+        fileInfo->mReplication = 0;
+        fileInfo->mBlockSize = 0;
+        fileInfo->mOwner = NULL;
+        fileInfo->mGroup = NULL;
+        fileInfo->mPermissions = 0;
+        fileInfo->mLastAccess = 0;
+    }
+}
+
+static webhdfsBuffer *initWebHdfsBuffer() {
+    webhdfsBuffer *buffer = (webhdfsBuffer *) calloc(1, sizeof(webhdfsBuffer));
+    if (!buffer) {
+        fprintf(stderr, "Fail to allocate memory for webhdfsBuffer.\n");
+        return NULL;
+    }
+    buffer->remaining = 0;
+    buffer->offset = 0;
+    buffer->wbuffer = NULL;
+    buffer->closeFlag = 0;
+    buffer->openFlag = 0;
+    pthread_mutex_init(&buffer->writeMutex, NULL);
+    pthread_cond_init(&buffer->newwrite_or_close, NULL);
+    pthread_cond_init(&buffer->transfer_finish, NULL);
+    return buffer;
+}
+
+static webhdfsBuffer *resetWebhdfsBuffer(webhdfsBuffer *wb, const char *buffer, size_t length) {
+    if (buffer && length > 0) {
+        pthread_mutex_lock(&wb->writeMutex);
+        wb->wbuffer = buffer;
+        wb->offset = 0;
+        wb->remaining = length;
+        pthread_cond_signal(&wb->newwrite_or_close);
+        while (wb->remaining != 0) {
+            pthread_cond_wait(&wb->transfer_finish, &wb->writeMutex);
+        }
+        pthread_mutex_unlock(&wb->writeMutex);
+    }
+    return wb;
+}
+
+static void freeWebhdfsBuffer(webhdfsBuffer *buffer) {
+    if (buffer) {
+        int des = pthread_cond_destroy(&buffer->newwrite_or_close);
+        if (des == EBUSY) {
+            fprintf(stderr, "The condition newwrite_or_close is still referenced!\n");
+        } else if (des == EINVAL) {
+            fprintf(stderr, "The condition newwrite_or_close is invalid!\n");
+        }
+        des = pthread_cond_destroy(&buffer->transfer_finish);
+        if (des == EBUSY) {
+            fprintf(stderr, "The condition transfer_finish is still referenced!\n");
+        } else if (des == EINVAL) {
+            fprintf(stderr, "The condition transfer_finish is invalid!\n");
+        }
+        if (des == EBUSY) {
+            fprintf(stderr, "The condition close_clean is still referenced!\n");
+        } else if (des == EINVAL) {
+            fprintf(stderr, "The condition close_clean is invalid!\n");
+        }
+        des = pthread_mutex_destroy(&buffer->writeMutex);
+        if (des == EBUSY) {
+            fprintf(stderr, "The mutex is still locked or referenced!\n");
+        }
+        free(buffer);
+        buffer = NULL;
+    }
+}
+
+static void freeWebFileHandle(struct webhdfsFileHandle * handle) {
+    if (handle) {
+        freeWebhdfsBuffer(handle->uploadBuffer);
+        if (handle->datanode) {
+            free(handle->datanode);
+        }
+        if (handle->absPath) {
+            free(handle->absPath);
+        }
+        free(handle);
+        handle = NULL;
+    }
+}
+
+struct hdfsBuilder *hdfsNewBuilder(void)
+{
+    struct hdfsBuilder *bld = calloc(1, sizeof(struct hdfsBuilder));
+    if (!bld) {
+        return NULL;
+    }
+    hdfsSetWorkingDirectory(bld, "/");
+    return bld;
+}
+
+void hdfsFreeBuilder(struct hdfsBuilder *bld)
+{
+    if (bld && bld->workingDir) {
+        free(bld->workingDir);
+    }
+    free(bld);
+}
+
+void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld)
+{
+    if (bld) {
+        bld->forceNewInstance = 1;
+    }
+}
+
+void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn)
+{
+    if (bld) {
+        bld->nn = nn;
+        bld->nn_jni = nn;
+    }
+}
+
+void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port)
+{
+    if (bld) {
+        bld->port = port;
+    }
+}
+
+void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName)
+{
+    if (bld) {
+        bld->userName = userName;
+    }
+}
+
+void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
+                                       const char *kerbTicketCachePath)
+{
+    if (bld) {
+        bld->kerbTicketCachePath = kerbTicketCachePath;
+    }
+}
+
+hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user)
+{
+    struct hdfsBuilder* bld = hdfsNewBuilder();
+    if (!bld) {
+        return NULL;
+    }
+    hdfsBuilderSetNameNode(bld, nn);
+    hdfsBuilderSetNameNodePort(bld, port);
+    hdfsBuilderSetUserName(bld, user);
+    return hdfsBuilderConnect(bld);
+}
+
+hdfsFS hdfsConnect(const char* nn, tPort port)
+{
+    return hdfsConnectAsUser(nn, port, NULL);
+}
+
+hdfsFS hdfsConnectNewInstance(const char* nn, tPort port)
+{
+    struct hdfsBuilder* bld = (struct hdfsBuilder *) hdfsConnect(nn, port);
+    if (!bld) {
+        return NULL;
+    }
+    hdfsBuilderSetForceNewInstance(bld);
+    return bld;
+}
+
+hdfsFS hdfsConnectAsUserNewInstance(const char* host, tPort port,
+                                    const char *user)
+{
+    struct hdfsBuilder *bld = hdfsNewBuilder();
+    if (!bld)
+        return NULL;
+    hdfsBuilderSetNameNode(bld, host);
+    hdfsBuilderSetNameNodePort(bld, port);
+    hdfsBuilderSetUserName(bld, user);
+    hdfsBuilderSetForceNewInstance(bld);
+    return hdfsBuilderConnect(bld);
+}
+
+const char *hdfsBuilderToStr(const struct hdfsBuilder *bld,
+                             char *buf, size_t bufLen);
+
+hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld)
+{
+    if (!bld) {
+        return NULL;
+    }
+    // if the hostname is null for the namenode, set it to localhost
+    //only handle bld->nn
+    if (bld->nn == NULL) {
+        bld->nn = "localhost";
+    } else {
+        /* check whether the hostname of the namenode (nn in hdfsBuilder) has already contained the port */
+        const char *lastColon = rindex(bld->nn, ':');
+        if (lastColon && (strspn(lastColon + 1, "0123456789") == strlen(lastColon + 1))) {
+            fprintf(stderr, "port %d was given, but URI '%s' already "
+                    "contains a port!\n", bld->port, bld->nn);
+            char *newAddr = (char *)malloc(strlen(bld->nn) - strlen(lastColon) + 1);
+            if (!newAddr) {
+                return NULL;
+            }
+            strncpy(newAddr, bld->nn, strlen(bld->nn) - strlen(lastColon));
+            newAddr[strlen(bld->nn) - strlen(lastColon)] = '\0';
+            free(bld->nn);
+            bld->nn = newAddr;
+        }
+    }
+    
+    /* if the namenode is "default" and/or the port of namenode is 0, get the default namenode/port by using JNI */
+    if (bld->port == 0 || !strcasecmp("default", bld->nn)) {
+        JNIEnv *env = 0;
+        jobject jHDFSConf = NULL, jAddress = NULL;
+        jvalue jVal;
+        jthrowable jthr = NULL;
+        int ret = 0;
+        char buf[512];
+        
+        //Get the JNIEnv* corresponding to current thread
+        env = getJNIEnv();
+        if (env == NULL) {
+            errno = EINTERNAL;
+            free(bld);
+            bld = NULL;
+            return NULL;
+        }
+        
+        //  jHDFSConf = new HDFSConfiguration();
+        jthr = constructNewObjectOfClass(env, &jHDFSConf, HADOOP_HDFS_CONF, "()V");
+        if (jthr) {
+            ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                        "hdfsBuilderConnect(%s)",
+                                        hdfsBuilderToStr(bld, buf, sizeof(buf)));
+            goto done;
+        }
+        
+        jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_NAMENODE, "getHttpAddress",
+                            "(Lorg/apache/hadoop/conf/Configuration;)Ljava/net/InetSocketAddress;",
+                            jHDFSConf);
+        if (jthr) {
+            ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                            "hdfsBuilderConnect(%s)", hdfsBuilderToStr(bld, buf, sizeof(buf)));
+            goto done; //free(bld), deleteReference for jHDFSConf
+        }
+        jAddress = jVal.l;
+        
+        if (bld->port == 0) {
+            jthr = invokeMethod(env, &jVal, INSTANCE, jAddress,
+                                JAVA_INETSOCKETADDRESS, "getPort", "()I");
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            bld->port = jVal.i;
+        }
+        
+        if (!strcasecmp("default", bld->nn)) {
+            jthr = invokeMethod(env, &jVal, INSTANCE, jAddress,
+                                JAVA_INETSOCKETADDRESS, "getHostName", "()Ljava/lang/String;");
+            if (jthr) {
+                ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+                                            "hdfsBuilderConnect(%s)",
+                                            hdfsBuilderToStr(bld, buf, sizeof(buf)));
+                goto done;
+            }
+            bld->nn = (const char*) ((*env)->GetStringUTFChars(env, jVal.l, NULL));
+        }
+        
+    done:
+        destroyLocalReference(env, jHDFSConf);
+        destroyLocalReference(env, jAddress);
+        if (ret) { //if there is error/exception, we free the builder and return NULL
+            free(bld);
+            bld = NULL;
+        }
+    }
+    
+    //for debug
+    fprintf(stderr, "namenode: %s:%d\n", bld->nn, bld->port);
+    return bld;
+}
+
+int hdfsDisconnect(hdfsFS fs)
+{
+    if (fs == NULL) {
+        errno = EBADF;
+        return -1;
+    } else {
+        free(fs);
+        fs = NULL;
+    }
+    return 0;
+}
+
+char *getAbsolutePath(hdfsFS fs, const char *path) {
+    if (fs == NULL || path == NULL) {
+        return NULL;
+    }
+    char *absPath = NULL;
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    
+    if ('/' != *path && bld->workingDir) {
+        absPath = (char *)malloc(strlen(bld->workingDir) + strlen(path) + 1);
+        if (!absPath) {
+            return NULL;
+        }
+        absPath = strcpy(absPath, bld->workingDir);
+        absPath = strcat(absPath, path);
+        return absPath;
+    } else {
+        absPath = (char *)malloc(strlen(path) + 1);
+        if (!absPath) {
+            return NULL;
+        }
+        absPath = strcpy(absPath, path);
+        return absPath;
+    }
+}
+
+int hdfsCreateDirectory(hdfsFS fs, const char* path)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return -1;
+    }
+    
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url = NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareMKDIR(bld->nn, bld->port, absPath, bld->userName))
+         && (resp = launchMKDIR(url))
+         && (parseMKDIR(resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(url);
+    free(absPath);
+    return ret;
+}
+
+int hdfsChmod(hdfsFS fs, const char* path, short mode)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return -1;
+    }
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url=NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareCHMOD(bld->nn, bld->port, absPath, (int)mode, bld->userName))
+         && (resp = launchCHMOD(url))
+         && (parseCHMOD(resp->header->content, resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    return ret;
+}
+
+int hdfsChown(hdfsFS fs, const char* path, const char *owner, const char *group)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return -1;
+    }
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url=NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareCHOWN(bld->nn, bld->port, absPath, owner, group, bld->userName))
+         && (resp = launchCHOWN(url))
+         && (parseCHOWN(resp->header->content, resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    return ret;
+}
+
+int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath)
+{
+    if (fs == NULL || oldPath == NULL || newPath == NULL) {
+        return -1;
+    }
+    
+    char *oldAbsPath = getAbsolutePath(fs, oldPath);
+    if (!oldAbsPath) {
+        return -1;
+    }
+    char *newAbsPath = getAbsolutePath(fs, newPath);
+    if (!newAbsPath) {
+        return -1;
+    }
+    
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url=NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareRENAME(bld->nn, bld->port, oldAbsPath, newAbsPath, bld->userName))
+         && (resp = launchRENAME(url))
+         && (parseRENAME(resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(oldAbsPath);
+    free(newAbsPath);
+    free(url);
+    return ret;
+}
+
+hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path)
+{
+    if (fs == NULL || path == NULL) {
+        return NULL;
+    }
+    
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return NULL;
+    }
+    
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url=NULL;
+    Response resp = NULL;
+    int numEntries = 0;
+    int ret = 0;
+    
+    hdfsFileInfo * fileInfo = (hdfsFileInfo *) calloc(1, sizeof(hdfsFileInfo));
+    if (!fileInfo) {
+        ret = -1;
+        goto done;
+    }
+    initFileinfo(fileInfo);
+    
+    if(!((url = prepareGFS(bld->nn, bld->port, absPath, bld->userName))
+         && (resp = launchGFS(url))
+         && (fileInfo = parseGFS(resp->body->content, fileInfo, &numEntries))))  {
+        ret = -1;
+        goto done;
+    }
+    
+done:
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    
+    if (ret == 0) {
+        return fileInfo;
+    } else {
+        free(fileInfo);
+        return NULL;
+    }
+}
+
+hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries)
+{
+    if (fs == NULL || path == NULL) {
+        return NULL;
+    }
+    
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return NULL;
+    }
+
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url = NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    hdfsFileInfo * fileInfo = (hdfsFileInfo *) calloc(1, sizeof(hdfsFileInfo));
+    if (!fileInfo) {
+        ret = -1;
+        goto done;
+    }
+    
+    if(!((url = prepareLS(bld->nn, bld->port, absPath, bld->userName))
+         && (resp = launchLS(url))
+         && (fileInfo = parseGFS(resp->body->content, fileInfo, numEntries))))  {
+        ret = -1;
+        goto done;
+    }
+    
+done:
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    
+    if (ret == 0) {
+        return fileInfo;
+    } else {
+        free(fileInfo);
+        return NULL;
+    }
+}
+
+int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return -1;
+    }
+    
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url = NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareSETREPLICATION(bld->nn, bld->port, absPath, replication, bld->userName))
+         && (resp = launchSETREPLICATION(url))
+         && (parseSETREPLICATION(resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    return ret;
+}
+
+void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
+{
+    //Free the mName, mOwner, and mGroup
+    int i;
+    for (i=0; i < numEntries; ++i) {
+        if (hdfsFileInfo[i].mName) {
+            free(hdfsFileInfo[i].mName);
+        }
+        if (hdfsFileInfo[i].mOwner) {
+            free(hdfsFileInfo[i].mOwner);
+        }
+        if (hdfsFileInfo[i].mGroup) {
+            free(hdfsFileInfo[i].mGroup);
+        }
+    }
+    
+    //Free entire block
+    free(hdfsFileInfo);
+    hdfsFileInfo = NULL;
+}
+
+int hdfsDelete(hdfsFS fs, const char* path, int recursive)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return -1;
+    }
+    
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url = NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareDELETE(bld->nn, bld->port, absPath, recursive, bld->userName))
+         && (resp = launchDELETE(url))
+         && (parseDELETE(resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    return ret;
+}
+
+int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    char *absPath = getAbsolutePath(fs, path);
+    if (!absPath) {
+        return -1;
+    }
+    
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    char *url = NULL;
+    Response resp = NULL;
+    int ret = 0;
+    
+    if(!((url = prepareUTIMES(bld->nn, bld->port, absPath, mtime, atime, bld->userName))
+         && (resp = launchUTIMES(url))
+         && (parseUTIMES(resp->header->content, resp->body->content)))) {
+        ret = -1;
+    }
+    
+    freeResponse(resp);
+    free(absPath);
+    free(url);
+    return ret;
+}
+
+int hdfsExists(hdfsFS fs, const char *path)
+{
+    hdfsFileInfo *fileInfo = hdfsGetPathInfo(fs, path);
+    if (fileInfo) {
+        hdfsFreeFileInfo(fileInfo, 1);
+        return 0;
+    } else {
+        return -1;
+    }
+}
+
+typedef struct {
+    char *url;
+    webhdfsBuffer *uploadBuffer;
+    int flags;
+    Response resp;
+} threadData;
+
+static void freeThreadData(threadData *data) {
+    if (data) {
+        if (data->url) {
+            free(data->url);
+        }
+        if (data->resp) {
+            freeResponse(data->resp);
+        }
+        //the uploadBuffer would be freed by freeWebFileHandle()
+        free(data);
+        data = NULL;
+    }
+}
+
+static void *writeThreadOperation(void *v) {
+    threadData *data = (threadData *) v;
+    if (data->flags & O_APPEND) {
+        data->resp = launchDnAPPEND(data->url, data->uploadBuffer);
+    } else {
+        data->resp = launchDnWRITE(data->url, data->uploadBuffer);
+    }
+    return data;
+}
+
+hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
+                      int bufferSize, short replication, tSize blockSize)
+{
+    /*
+     * the original version of libhdfs based on JNI store a fsinputstream/fsoutputstream in the hdfsFile
+     * in libwebhdfs that is based on webhdfs, we store (absolute_path, buffersize, replication, blocksize) in it
+     */
+    if (fs == NULL || path == NULL) {
+        return NULL;
+    }
+
+    int accmode = flags & O_ACCMODE;
+    if (accmode == O_RDWR) {
+        fprintf(stderr, "ERROR: cannot open an hdfs file in O_RDWR mode\n");
+        errno = ENOTSUP;
+        return NULL;
+    }
+    
+    if ((flags & O_CREAT) && (flags & O_EXCL)) {
+        fprintf(stderr, "WARN: hdfs does not truly support O_CREATE && O_EXCL\n");
+    }
+    
+    hdfsFile hdfsFileHandle = (hdfsFile) calloc(1, sizeof(struct hdfsFile_internal));
+    if (!hdfsFileHandle) {
+        return NULL;
+    }
+    int ret = 0;
+    hdfsFileHandle->flags = flags;
+    hdfsFileHandle->type = accmode == O_RDONLY ? INPUT : OUTPUT;
+    hdfsFileHandle->offset = 0;
+    struct webhdfsFileHandle *webhandle = (struct webhdfsFileHandle *) calloc(1, sizeof(struct webhdfsFileHandle));
+    if (!webhandle) {
+        ret = -1;
+        goto done;
+    }
+    webhandle->bufferSize = bufferSize;
+    webhandle->replication = replication;
+    webhandle->blockSize = blockSize;
+    webhandle->absPath = getAbsolutePath(fs, path);
+    if (!webhandle->absPath) {
+        ret = -1;
+        goto done;
+    }
+    hdfsFileHandle->file = webhandle;
+    
+    //for write/append, need to connect to the namenode
+    //and get the url of corresponding datanode
+    if (hdfsFileHandle->type == OUTPUT) {
+        webhandle->uploadBuffer = initWebHdfsBuffer();
+        if (!webhandle->uploadBuffer) {
+            ret = -1;
+            goto done;
+        }
+        struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+        char *url = NULL;
+        Response resp = NULL;
+        int append = flags & O_APPEND;
+        int create = append ? 0 : 1;
+        
+        //if create: send create request to NN
+        if (create) {
+            url = prepareNnWRITE(bld->nn, bld->port, webhandle->absPath, bld->userName, webhandle->replication, webhandle->blockSize);
+        } else if (append) {
+            url = prepareNnAPPEND(bld->nn, bld->port, webhandle->absPath, bld->userName);
+        }
+        if (!url) {
+            fprintf(stderr,
+                    "fail to create the url connecting to namenode for file creation/appending\n");
+            ret = -1;
+            goto done;
+        }
+
+        if (create) {
+            resp = launchNnWRITE(url);
+        } else if (append) {
+            resp = launchNnAPPEND(url);
+        }
+        if (!resp) {
+            fprintf(stderr,
+                    "fail to get the response from namenode for file creation/appending\n");
+            free(url);
+            ret = -1;
+            goto done;
+        }
+        
+        int parseRet = 0;
+        if (create) {
+            parseRet = parseNnWRITE(resp->header->content, resp->body->content);
+        } else if (append) {
+            parseRet = parseNnAPPEND(resp->header->content, resp->body->content);
+        }
+        if (!parseRet) {
+            fprintf(stderr,
+                    "fail to parse the response from namenode for file creation/appending\n");
+            free(url);
+            freeResponse(resp);
+            ret = -1;
+            goto done;
+        }
+            
+        free(url);
+        url = parseDnLoc(resp->header->content);
+        if (!url) {
+            fprintf(stderr,
+                    "fail to get the datanode url from namenode for file creation/appending\n");
+            freeResponse(resp);
+            ret = -1;
+            return NULL;
+        }
+        freeResponse(resp);
+        //store the datanode url in the file handle
+        webhandle->datanode = strdup(url);
+ 
+        //create a new thread for performing the http transferring
+        threadData *data = (threadData *) calloc(1, sizeof(threadData));
+        if (!data) {
+            ret = -1;
+            goto done;
+        }
+        data->url = strdup(url);
+        data->flags = flags;
+        data->uploadBuffer = webhandle->uploadBuffer;
+        free(url);
+        ret = pthread_create(&webhandle->connThread, NULL, writeThreadOperation, data);
+        if (ret) {
+            fprintf(stderr, "Failed to create the writing thread.\n");
+        } else {
+            webhandle->uploadBuffer->openFlag = 1;
+        }
+    }
+    
+done:
+    if (ret == 0) {
+        return hdfsFileHandle;
+    } else {
+        freeWebFileHandle(webhandle);
+        free(hdfsFileHandle);
+        return NULL;
+    }
+}
+
+tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer, tSize length)
+{
+    if (length == 0) {
+        return 0;
+    }
+    if (fs == NULL || file == NULL || file->type != OUTPUT || length < 0) {
+        return -1;
+    }
+    
+    struct webhdfsFileHandle *wfile = (struct webhdfsFileHandle *) file->file;
+    if (wfile->uploadBuffer && wfile->uploadBuffer->openFlag) {
+        resetWebhdfsBuffer(wfile->uploadBuffer, buffer, length);
+        return length;
+    } else {
+        fprintf(stderr, "Error: have not opened the file %s for writing yet.\n", wfile->absPath);
+        return -1;
+    }
+}
+
+int hdfsCloseFile(hdfsFS fs, hdfsFile file)
+{
+    int ret = 0;
+    fprintf(stderr, "to close file...\n");
+    if (file->type == OUTPUT) {
+        void *respv;
+        threadData *tdata;
+        struct webhdfsFileHandle *wfile = (struct webhdfsFileHandle *) file->file;
+        pthread_mutex_lock(&(wfile->uploadBuffer->writeMutex));
+        wfile->uploadBuffer->closeFlag = 1;
+        pthread_cond_signal(&wfile->uploadBuffer->newwrite_or_close);
+        pthread_mutex_unlock(&(wfile->uploadBuffer->writeMutex));
+        
+        //waiting for the writing thread to terminate
+        ret = pthread_join(wfile->connThread, &respv);
+        if (ret) {
+            fprintf(stderr, "Error (code %d) when pthread_join.\n", ret);
+        }
+        //parse the response
+        tdata = (threadData *) respv;
+        if (!tdata) {
+            fprintf(stderr, "Response from the writing thread is NULL.\n");
+            ret = -1;
+        }
+        if (file->flags & O_APPEND) {
+            parseDnAPPEND(tdata->resp->header->content, tdata->resp->body->content);
+        } else {
+            parseDnWRITE(tdata->resp->header->content, tdata->resp->body->content);
+        }
+        //free the threaddata
+        freeThreadData(tdata);
+    }
+    
+    fprintf(stderr, "To clean the webfilehandle...\n");
+    if (file) {
+        freeWebFileHandle(file->file);
+        free(file);
+        file = NULL;
+        fprintf(stderr, "Cleaned the webfilehandle...\n");
+    }
+    return ret;
+}
+
+int hdfsFileIsOpenForRead(hdfsFile file)
+{
+    return (file->type == INPUT);
+}
+
+int hdfsFileIsOpenForWrite(hdfsFile file)
+{
+    return (file->type == OUTPUT);
+}
+
+tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length)
+{
+    if (length == 0) {
+        return 0;
+    }
+    if (fs == NULL || file == NULL || file->type != INPUT || buffer == NULL || length < 0) {
+        errno = EINVAL;
+        return -1;
+    }
+    struct hdfsBuilder *bld = (struct hdfsBuilder *) fs;
+    struct webhdfsFileHandle *webFile = (struct webhdfsFileHandle *) file->file;
+    char *url = NULL;
+    Response resp = NULL;
+    int openResult = -1;
+    
+    resp = (Response) calloc(1, sizeof(*resp));
+    if (!resp) {
+        return -1;
+    }
+    resp->header = initResponseBuffer();
+    resp->body = initResponseBuffer();
+    resp->body->content = buffer;
+    resp->body->remaining = length;
+    
+    if (!((url = prepareOPEN(bld->nn, bld->port, webFile->absPath, bld->userName, file->offset, length))
+          && (resp = launchOPEN(url, resp))
+          && ((openResult = parseOPEN(resp->header->content, resp->body->content)) > 0))) {
+        free(url);
+        freeResponseBuffer(resp->header);
+        if (openResult == 0) {
+            return 0;
+        } else {
+            return -1;
+        }
+    }
+    
+    size_t readSize = resp->body->offset;
+    file->offset += readSize;
+    
+    freeResponseBuffer(resp->header);
+    free(resp->body);
+    free(resp);
+    free(url);
+    return readSize;
+}
+
+int hdfsAvailable(hdfsFS fs, hdfsFile file)
+{
+    if (!file || !fs) {
+        return -1;
+    }
+    struct webhdfsFileHandle *wf = (struct webhdfsFileHandle *) file->file;
+    if (!wf) {
+        return -1;
+    }
+    hdfsFileInfo *fileInfo = hdfsGetPathInfo(fs, wf->absPath);
+    if (fileInfo) {
+        int available = (int)(fileInfo->mSize - file->offset);
+        hdfsFreeFileInfo(fileInfo, 1);
+        return available;
+    } else {
+        return -1;
+    }
+}
+
+int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos)
+{
+    if (!fs || !file || desiredPos < 0) {
+        return -1;
+    }
+    struct webhdfsFileHandle *wf = (struct webhdfsFileHandle *) file->file;
+    if (!wf) {
+        return -1;
+    }
+    hdfsFileInfo *fileInfo = hdfsGetPathInfo(fs, wf->absPath);
+    int ret = 0;
+    if (fileInfo) {
+        if (fileInfo->mSize < desiredPos) {
+            errno = ENOTSUP;
+            fprintf(stderr,
+                    "hdfsSeek for %s failed since the desired position %lld is beyond the size of the file %lld\n",
+                    wf->absPath, desiredPos, fileInfo->mSize);
+            ret = -1;
+        } else {
+            file->offset = desiredPos;
+        }
+        hdfsFreeFileInfo(fileInfo, 1);
+        return ret;
+    } else {
+        return -1;
+    }
+}
+
+tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position, void* buffer, tSize length)
+{
+    if (!fs || !file || file->type != INPUT || position < 0 || !buffer || length < 0) {
+        return -1;
+    }
+    file->offset = position;
+    return hdfsRead(fs, file, buffer, length);
+}
+
+tOffset hdfsTell(hdfsFS fs, hdfsFile file)
+{
+    if (!file) {
+        return -1;
+    }
+    return file->offset;
+}
+
+char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize)
+{
+    if (fs == NULL || buffer == NULL ||  bufferSize <= 0) {
+        return NULL;
+    }
+    
+    struct hdfsBuilder * bld = (struct hdfsBuilder *) fs;
+    if (bld->workingDir) {
+        strncpy(buffer, bld->workingDir, bufferSize);
+    }
+    return buffer;
+}
+
+int hdfsSetWorkingDirectory(hdfsFS fs, const char* path)
+{
+    if (fs == NULL || path == NULL) {
+        return -1;
+    }
+    
+    struct hdfsBuilder * bld = (struct hdfsBuilder *) fs;
+    free(bld->workingDir);
+    bld->workingDir = (char *)malloc(strlen(path) + 1);
+    if (!(bld->workingDir)) {
+        return -1;
+    }
+    strcpy(bld->workingDir, path);
+    return 0;
+}
+
+void hdfsFreeHosts(char ***blockHosts)
+{
+    int i, j;
+    for (i=0; blockHosts[i]; i++) {
+        for (j=0; blockHosts[i][j]; j++) {
+            free(blockHosts[i][j]);
+        }
+        free(blockHosts[i]);
+    }
+    free(blockHosts);
+}
+
+/* not useful for libwebhdfs */
+int hdfsFileUsesDirectRead(hdfsFile file)
+{
+    /* return !!(file->flags & HDFS_FILE_SUPPORTS_DIRECT_READ); */
+    fprintf(stderr, "hdfsFileUsesDirectRead is no longer useful for libwebhdfs.\n");
+    return -1;
+}
+
+/* not useful for libwebhdfs */
+void hdfsFileDisableDirectRead(hdfsFile file)
+{
+    /* file->flags &= ~HDFS_FILE_SUPPORTS_DIRECT_READ; */
+    fprintf(stderr, "hdfsFileDisableDirectRead is no longer useful for libwebhdfs.\n");
+}
+
+/* not useful for libwebhdfs */
+int hdfsHFlush(hdfsFS fs, hdfsFile file)
+{
+    return 0;
+}
+
+/* not useful for libwebhdfs */
+int hdfsFlush(hdfsFS fs, hdfsFile file)
+{
+    return 0;
+}
+
+char*** hdfsGetHosts(hdfsFS fs, const char* path,
+                     tOffset start, tOffset length)
+{
+    fprintf(stderr, "hdfsGetHosts is not but will be supported by libwebhdfs yet.\n");
+    return NULL;
+}
+
+tOffset hdfsGetCapacity(hdfsFS fs)
+{
+    fprintf(stderr, "hdfsGetCapacity is not but will be supported by libwebhdfs.\n");
+    return -1;
+}
+
+tOffset hdfsGetUsed(hdfsFS fs)
+{
+    fprintf(stderr, "hdfsGetUsed is not but will be supported by libwebhdfs yet.\n");
+    return -1;
+}
+

+ 609 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/jni_helper.c

@@ -0,0 +1,609 @@
+/**
+ * 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.
+ */
+
+//#include "config.h"
+#include "exception.h"
+#include "jni_helper.h"
+
+#include <stdio.h> 
+#include <string.h> 
+
+static pthread_mutex_t hdfsHashMutex = PTHREAD_MUTEX_INITIALIZER;
+static pthread_mutex_t jvmMutex = PTHREAD_MUTEX_INITIALIZER;
+static volatile int hashTableInited = 0;
+
+#define LOCK_HASH_TABLE() pthread_mutex_lock(&hdfsHashMutex)
+#define UNLOCK_HASH_TABLE() pthread_mutex_unlock(&hdfsHashMutex)
+
+
+/** The Native return types that methods could return */
+#define VOID          'V'
+#define JOBJECT       'L'
+#define JARRAYOBJECT  '['
+#define JBOOLEAN      'Z'
+#define JBYTE         'B'
+#define JCHAR         'C'
+#define JSHORT        'S'
+#define JINT          'I'
+#define JLONG         'J'
+#define JFLOAT        'F'
+#define JDOUBLE       'D'
+
+
+/**
+ * MAX_HASH_TABLE_ELEM: The maximum no. of entries in the hashtable.
+ * It's set to 4096 to account for (classNames + No. of threads)
+ */
+#define MAX_HASH_TABLE_ELEM 4096
+
+/** Key that allows us to retrieve thread-local storage */
+static pthread_key_t gTlsKey;
+
+/** nonzero if we succeeded in initializing gTlsKey. Protected by the jvmMutex */
+static int gTlsKeyInitialized = 0;
+
+/** Pthreads thread-local storage for each library thread. */
+struct hdfsTls {
+    JNIEnv *env;
+};
+
+/**
+ * The function that is called whenever a thread with libhdfs thread local data
+ * is destroyed.
+ *
+ * @param v         The thread-local data
+ */
+static void hdfsThreadDestructor(void *v)
+{
+    struct hdfsTls *tls = v;
+    JavaVM *vm;
+    JNIEnv *env = tls->env;
+    jint ret;
+
+    ret = (*env)->GetJavaVM(env, &vm);
+    if (ret) {
+        fprintf(stderr, "hdfsThreadDestructor: GetJavaVM failed with "
+                "error %d\n", ret);
+        (*env)->ExceptionDescribe(env);
+    } else {
+        (*vm)->DetachCurrentThread(vm);
+    }
+    free(tls);
+}
+
+void destroyLocalReference(JNIEnv *env, jobject jObject)
+{
+  if (jObject)
+    (*env)->DeleteLocalRef(env, jObject);
+}
+
+static jthrowable validateMethodType(JNIEnv *env, MethType methType)
+{
+    if (methType != STATIC && methType != INSTANCE) {
+        return newRuntimeError(env, "validateMethodType(methType=%d): "
+            "illegal method type.\n", methType);
+    }
+    return NULL;
+}
+
+jthrowable newJavaStr(JNIEnv *env, const char *str, jstring *out)
+{
+    jstring jstr;
+
+    if (!str) {
+        /* Can't pass NULL to NewStringUTF: the result would be
+         * implementation-defined. */
+        *out = NULL;
+        return NULL;
+    }
+    jstr = (*env)->NewStringUTF(env, str);
+    if (!jstr) {
+        /* If NewStringUTF returns NULL, an exception has been thrown,
+         * which we need to handle.  Probaly an OOM. */
+        return getPendingExceptionAndClear(env);
+    }
+    *out = jstr;
+    return NULL;
+}
+
+jthrowable newCStr(JNIEnv *env, jstring jstr, char **out)
+{
+    const char *tmp;
+
+    if (!jstr) {
+        *out = NULL;
+        return NULL;
+    }
+    tmp = (*env)->GetStringUTFChars(env, jstr, NULL);
+    if (!tmp) {
+        return getPendingExceptionAndClear(env);
+    }
+    *out = strdup(tmp);
+    (*env)->ReleaseStringUTFChars(env, jstr, tmp);
+    return NULL;
+}
+
+static int hashTableInit(void)
+{
+    if (!hashTableInited) {
+        LOCK_HASH_TABLE();
+        if (!hashTableInited) {
+            if (hcreate(MAX_HASH_TABLE_ELEM) == 0) {
+                fprintf(stderr, "error creating hashtable, <%d>: %s\n",
+                        errno, strerror(errno));
+                return 0;
+            } 
+            hashTableInited = 1;
+        }
+        UNLOCK_HASH_TABLE();
+    }
+    return 1;
+}
+
+
+static int insertEntryIntoTable(const char *key, void *data)
+{
+    ENTRY e, *ep;
+    if (key == NULL || data == NULL) {
+        return 0;
+    }
+    if (! hashTableInit()) {
+      return -1;
+    }
+    e.data = data;
+    e.key = (char*)key;
+    LOCK_HASH_TABLE();
+    ep = hsearch(e, ENTER);
+    UNLOCK_HASH_TABLE();
+    if (ep == NULL) {
+        fprintf(stderr, "warn adding key (%s) to hash table, <%d>: %s\n",
+                key, errno, strerror(errno));
+    }  
+    return 0;
+}
+
+
+
+static void* searchEntryFromTable(const char *key)
+{
+    ENTRY e,*ep;
+    if (key == NULL) {
+        return NULL;
+    }
+    hashTableInit();
+    e.key = (char*)key;
+    LOCK_HASH_TABLE();
+    ep = hsearch(e, FIND);
+    UNLOCK_HASH_TABLE();
+    if (ep != NULL) {
+        return ep->data;
+    }
+    return NULL;
+}
+
+
+
+jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
+                 jobject instObj, const char *className,
+                 const char *methName, const char *methSignature, ...)
+{
+    va_list args;
+    jclass cls;
+    jmethodID mid;
+    jthrowable jthr;
+    const char *str; 
+    char returnType;
+    
+    jthr = validateMethodType(env, methType);
+    if (jthr)
+        return jthr;
+    jthr = globalClassReference(className, env, &cls);
+    if (jthr)
+        return jthr;
+    jthr = methodIdFromClass(className, methName, methSignature, 
+                            methType, env, &mid);
+    if (jthr)
+        return jthr;
+    str = methSignature;
+    while (*str != ')') str++;
+    str++;
+    returnType = *str;
+    va_start(args, methSignature);
+    if (returnType == JOBJECT || returnType == JARRAYOBJECT) {
+        jobject jobj = NULL;
+        if (methType == STATIC) {
+            jobj = (*env)->CallStaticObjectMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            jobj = (*env)->CallObjectMethodV(env, instObj, mid, args);
+        }
+        retval->l = jobj;
+    }
+    else if (returnType == VOID) {
+        if (methType == STATIC) {
+            (*env)->CallStaticVoidMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            (*env)->CallVoidMethodV(env, instObj, mid, args);
+        }
+    }
+    else if (returnType == JBOOLEAN) {
+        jboolean jbool = 0;
+        if (methType == STATIC) {
+            jbool = (*env)->CallStaticBooleanMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            jbool = (*env)->CallBooleanMethodV(env, instObj, mid, args);
+        }
+        retval->z = jbool;
+    }
+    else if (returnType == JSHORT) {
+        jshort js = 0;
+        if (methType == STATIC) {
+            js = (*env)->CallStaticShortMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            js = (*env)->CallShortMethodV(env, instObj, mid, args);
+        }
+        retval->s = js;
+    }
+    else if (returnType == JLONG) {
+        jlong jl = -1;
+        if (methType == STATIC) {
+            jl = (*env)->CallStaticLongMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            jl = (*env)->CallLongMethodV(env, instObj, mid, args);
+        }
+        retval->j = jl;
+    }
+    else if (returnType == JINT) {
+        jint ji = -1;
+        if (methType == STATIC) {
+            ji = (*env)->CallStaticIntMethodV(env, cls, mid, args);
+        }
+        else if (methType == INSTANCE) {
+            ji = (*env)->CallIntMethodV(env, instObj, mid, args);
+        }
+        retval->i = ji;
+    }
+    va_end(args);
+
+    jthr = (*env)->ExceptionOccurred(env);
+    if (jthr) {
+        (*env)->ExceptionClear(env);
+        return jthr;
+    }
+    return NULL;
+}
+
+jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out, const char *className, 
+                                  const char *ctorSignature, ...)
+{
+    va_list args;
+    jclass cls;
+    jmethodID mid; 
+    jobject jobj;
+    jthrowable jthr;
+
+    jthr = globalClassReference(className, env, &cls);
+    if (jthr)
+        return jthr;
+    jthr = methodIdFromClass(className, "<init>", ctorSignature, 
+                            INSTANCE, env, &mid);
+    if (jthr)
+        return jthr;
+    va_start(args, ctorSignature);
+    jobj = (*env)->NewObjectV(env, cls, mid, args);
+    va_end(args);
+    if (!jobj)
+        return getPendingExceptionAndClear(env);
+    *out = jobj;
+    return NULL;
+}
+
+
+jthrowable methodIdFromClass(const char *className, const char *methName, 
+                            const char *methSignature, MethType methType, 
+                            JNIEnv *env, jmethodID *out)
+{
+    jclass cls;
+    jthrowable jthr;
+
+    jthr = globalClassReference(className, env, &cls);
+    if (jthr)
+        return jthr;
+    jmethodID mid = 0;
+    jthr = validateMethodType(env, methType);
+    if (jthr)
+        return jthr;
+    if (methType == STATIC) {
+        mid = (*env)->GetStaticMethodID(env, cls, methName, methSignature);
+    }
+    else if (methType == INSTANCE) {
+        mid = (*env)->GetMethodID(env, cls, methName, methSignature);
+    }
+    if (mid == NULL) {
+        fprintf(stderr, "could not find method %s from class %s with "
+            "signature %s\n", methName, className, methSignature);
+        return getPendingExceptionAndClear(env);
+    }
+    *out = mid;
+    return NULL;
+}
+
+jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out)
+{
+    jclass clsLocalRef;
+    jclass cls = searchEntryFromTable(className);
+    if (cls) {
+        *out = cls;
+        return NULL;
+    }
+    clsLocalRef = (*env)->FindClass(env,className);
+    if (clsLocalRef == NULL) {
+        return getPendingExceptionAndClear(env);
+    }
+    cls = (*env)->NewGlobalRef(env, clsLocalRef);
+    if (cls == NULL) {
+        (*env)->DeleteLocalRef(env, clsLocalRef);
+        return getPendingExceptionAndClear(env);
+    }
+    (*env)->DeleteLocalRef(env, clsLocalRef);
+    insertEntryIntoTable(className, cls);
+    *out = cls;
+    return NULL;
+}
+
+jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name)
+{
+    jthrowable jthr;
+    jclass cls, clsClass = NULL;
+    jmethodID mid;
+    jstring str = NULL;
+    const char *cstr = NULL;
+    char *newstr;
+
+    cls = (*env)->GetObjectClass(env, jobj);
+    if (cls == NULL) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    clsClass = (*env)->FindClass(env, "java/lang/Class");
+    if (clsClass == NULL) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    mid = (*env)->GetMethodID(env, clsClass, "getName", "()Ljava/lang/String;");
+    if (mid == NULL) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    str = (*env)->CallObjectMethod(env, cls, mid);
+    if (str == NULL) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    cstr = (*env)->GetStringUTFChars(env, str, NULL);
+    if (!cstr) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    newstr = strdup(cstr);
+    if (newstr == NULL) {
+        jthr = newRuntimeError(env, "classNameOfObject: out of memory");
+        goto done;
+    }
+    *name = newstr;
+    jthr = NULL;
+
+done:
+    destroyLocalReference(env, cls);
+    destroyLocalReference(env, clsClass);
+    if (str) {
+        if (cstr)
+            (*env)->ReleaseStringUTFChars(env, str, cstr);
+        (*env)->DeleteLocalRef(env, str);
+    }
+    return jthr;
+}
+
+
+/**
+ * Get the global JNI environemnt.
+ *
+ * We only have to create the JVM once.  After that, we can use it in
+ * every thread.  You must be holding the jvmMutex when you call this
+ * function.
+ *
+ * @return          The JNIEnv on success; error code otherwise
+ */
+static JNIEnv* getGlobalJNIEnv(void)
+{
+    const jsize vmBufLength = 1;
+    JavaVM* vmBuf[vmBufLength]; 
+    JNIEnv *env;
+    jint rv = 0; 
+    jint noVMs = 0;
+    jthrowable jthr;
+
+    rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), vmBufLength, &noVMs);
+    if (rv != 0) {
+        fprintf(stderr, "JNI_GetCreatedJavaVMs failed with error: %d\n", rv);
+        return NULL;
+    }
+
+    if (noVMs == 0) {
+        //Get the environment variables for initializing the JVM
+        char *hadoopClassPath = getenv("CLASSPATH");
+        if (hadoopClassPath == NULL) {
+            fprintf(stderr, "Environment variable CLASSPATH not set!\n");
+            return NULL;
+        } 
+        char *hadoopClassPathVMArg = "-Djava.class.path=";
+        size_t optHadoopClassPathLen = strlen(hadoopClassPath) + 
+          strlen(hadoopClassPathVMArg) + 1;
+        char *optHadoopClassPath = malloc(sizeof(char)*optHadoopClassPathLen);
+        snprintf(optHadoopClassPath, optHadoopClassPathLen,
+                "%s%s", hadoopClassPathVMArg, hadoopClassPath);
+
+        // Determine the # of LIBHDFS_OPTS args
+        int noArgs = 1;
+        char *hadoopJvmArgs = getenv("LIBHDFS_OPTS");
+        char jvmArgDelims[] = " ";
+        char *str, *token, *savePtr;
+        if (hadoopJvmArgs != NULL)  {
+          hadoopJvmArgs = strdup(hadoopJvmArgs);
+          for (noArgs = 1, str = hadoopJvmArgs; ; noArgs++, str = NULL) {
+            token = strtok_r(str, jvmArgDelims, &savePtr);
+            if (NULL == token) {
+              break;
+            }
+          }
+          free(hadoopJvmArgs);
+        }
+
+        // Now that we know the # args, populate the options array
+        JavaVMOption options[noArgs];
+        options[0].optionString = optHadoopClassPath;
+        hadoopJvmArgs = getenv("LIBHDFS_OPTS");
+	if (hadoopJvmArgs != NULL)  {
+          hadoopJvmArgs = strdup(hadoopJvmArgs);
+          for (noArgs = 1, str = hadoopJvmArgs; ; noArgs++, str = NULL) {
+            token = strtok_r(str, jvmArgDelims, &savePtr);
+            if (NULL == token) {
+              break;
+            }
+            options[noArgs].optionString = token;
+          }
+        }
+
+        //Create the VM
+        JavaVMInitArgs vm_args;
+        JavaVM *vm;
+        vm_args.version = JNI_VERSION_1_2;
+        vm_args.options = options;
+        vm_args.nOptions = noArgs; 
+        vm_args.ignoreUnrecognized = 1;
+
+        rv = JNI_CreateJavaVM(&vm, (void*)&env, &vm_args);
+
+        if (hadoopJvmArgs != NULL)  {
+          free(hadoopJvmArgs);
+        }
+        free(optHadoopClassPath);
+
+        if (rv != 0) {
+            fprintf(stderr, "Call to JNI_CreateJavaVM failed "
+                    "with error: %d\n", rv);
+            return NULL;
+        }
+        jthr = invokeMethod(env, NULL, STATIC, NULL,
+                         "org/apache/hadoop/fs/FileSystem",
+                         "loadFileSystems", "()V");
+        if (jthr) {
+            printExceptionAndFree(env, jthr, PRINT_EXC_ALL, "loadFileSystems");
+        }
+    }
+    else {
+        //Attach this thread to the VM
+        JavaVM* vm = vmBuf[0];
+        rv = (*vm)->AttachCurrentThread(vm, (void*)&env, 0);
+        if (rv != 0) {
+            fprintf(stderr, "Call to AttachCurrentThread "
+                    "failed with error: %d\n", rv);
+            return NULL;
+        }
+    }
+
+    return env;
+}
+
+/**
+ * getJNIEnv: A helper function to get the JNIEnv* for the given thread.
+ * If no JVM exists, then one will be created. JVM command line arguments
+ * are obtained from the LIBHDFS_OPTS environment variable.
+ *
+ * Implementation note: we rely on POSIX thread-local storage (tls).
+ * This allows us to associate a destructor function with each thread, that
+ * will detach the thread from the Java VM when the thread terminates.  If we
+ * failt to do this, it will cause a memory leak.
+ *
+ * However, POSIX TLS is not the most efficient way to do things.  It requires a
+ * key to be initialized before it can be used.  Since we don't know if this key
+ * is initialized at the start of this function, we have to lock a mutex first
+ * and check.  Luckily, most operating systems support the more efficient
+ * __thread construct, which is initialized by the linker.
+ *
+ * @param: None.
+ * @return The JNIEnv* corresponding to the thread.
+ */
+JNIEnv* getJNIEnv(void)
+{
+    JNIEnv *env;
+    struct hdfsTls *tls;
+    int ret;
+
+#ifdef HAVE_BETTER_TLS
+    static __thread struct hdfsTls *quickTls = NULL;
+    if (quickTls)
+        return quickTls->env;
+#endif
+    pthread_mutex_lock(&jvmMutex);
+    if (!gTlsKeyInitialized) {
+        ret = pthread_key_create(&gTlsKey, hdfsThreadDestructor);
+        if (ret) {
+            pthread_mutex_unlock(&jvmMutex);
+            fprintf(stderr, "getJNIEnv: pthread_key_create failed with "
+                "error %d\n", ret);
+            return NULL;
+        }
+        gTlsKeyInitialized = 1;
+    }
+    tls = pthread_getspecific(gTlsKey);
+    if (tls) {
+        pthread_mutex_unlock(&jvmMutex);
+        return tls->env;
+    }
+
+    env = getGlobalJNIEnv();
+    pthread_mutex_unlock(&jvmMutex);
+    if (!env) {
+        fprintf(stderr, "getJNIEnv: getGlobalJNIEnv failed\n");
+        return NULL;
+    }
+    tls = calloc(1, sizeof(struct hdfsTls));
+    if (!tls) {
+        fprintf(stderr, "getJNIEnv: OOM allocating %zd bytes\n",
+                sizeof(struct hdfsTls));
+        return NULL;
+    }
+    tls->env = env;
+    ret = pthread_setspecific(gTlsKey, tls);
+    if (ret) {
+        fprintf(stderr, "getJNIEnv: pthread_setspecific failed with "
+            "error code %d\n", ret);
+        hdfsThreadDestructor(tls);
+        return NULL;
+    }
+#ifdef HAVE_BETTER_TLS
+    quickTls = tls;
+#endif
+    return env;
+}
+

+ 122 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/jni_helper.h

@@ -0,0 +1,122 @@
+/**
+ * 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.
+ */
+
+#ifndef LIBHDFS_JNI_HELPER_H
+#define LIBHDFS_JNI_HELPER_H
+
+#include <jni.h>
+#include <stdio.h>
+
+#include <stdlib.h>
+#include <stdarg.h>
+#include <search.h>
+#include <pthread.h>
+#include <errno.h>
+
+#define PATH_SEPARATOR ':'
+
+
+/** Denote the method we want to invoke as STATIC or INSTANCE */
+typedef enum {
+    STATIC,
+    INSTANCE
+} MethType;
+
+/**
+ * Create a new malloc'ed C string from a Java string.
+ *
+ * @param env       The JNI environment
+ * @param jstr      The Java string
+ * @param out       (out param) the malloc'ed C string
+ *
+ * @return          NULL on success; the exception otherwise
+ */
+jthrowable newCStr(JNIEnv *env, jstring jstr, char **out);
+
+/**
+ * Create a new Java string from a C string.
+ *
+ * @param env       The JNI environment
+ * @param str       The C string
+ * @param out       (out param) the java string
+ *
+ * @return          NULL on success; the exception otherwise
+ */
+jthrowable newJavaStr(JNIEnv *env, const char *str, jstring *out);
+
+/**
+ * Helper function to destroy a local reference of java.lang.Object
+ * @param env: The JNIEnv pointer. 
+ * @param jFile: The local reference of java.lang.Object object
+ * @return None.
+ */
+void destroyLocalReference(JNIEnv *env, jobject jObject);
+
+/** invokeMethod: Invoke a Static or Instance method.
+ * className: Name of the class where the method can be found
+ * methName: Name of the method
+ * methSignature: the signature of the method "(arg-types)ret-type"
+ * methType: The type of the method (STATIC or INSTANCE)
+ * instObj: Required if the methType is INSTANCE. The object to invoke
+   the method on.
+ * env: The JNIEnv pointer
+ * retval: The pointer to a union type which will contain the result of the
+   method invocation, e.g. if the method returns an Object, retval will be
+   set to that, if the method returns boolean, retval will be set to the
+   value (JNI_TRUE or JNI_FALSE), etc.
+ * exc: If the methods throws any exception, this will contain the reference
+ * Arguments (the method arguments) must be passed after methSignature
+ * RETURNS: -1 on error and 0 on success. If -1 is returned, exc will have 
+   a valid exception reference, and the result stored at retval is undefined.
+ */
+jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
+                 jobject instObj, const char *className, const char *methName, 
+                 const char *methSignature, ...);
+
+jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out, const char *className, 
+                                  const char *ctorSignature, ...);
+
+jthrowable methodIdFromClass(const char *className, const char *methName, 
+                            const char *methSignature, MethType methType, 
+                            JNIEnv *env, jmethodID *out);
+
+jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out);
+
+/** classNameOfObject: Get an object's class name.
+ * @param jobj: The object.
+ * @param env: The JNIEnv pointer.
+ * @param name: (out param) On success, will contain a string containing the
+ * class name. This string must be freed by the caller.
+ * @return NULL on success, or the exception
+ */
+jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name);
+
+/** getJNIEnv: A helper function to get the JNIEnv* for the given thread.
+ * If no JVM exists, then one will be created. JVM command line arguments
+ * are obtained from the LIBHDFS_OPTS environment variable.
+ * @param: None.
+ * @return The JNIEnv* corresponding to the thread.
+ * */
+JNIEnv* getJNIEnv(void);
+
+#endif /*LIBHDFS_JNI_HELPER_H*/
+
+/**
+ * vim: ts=4: sw=4: et:
+ */
+

+ 180 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_multi_write.c

@@ -0,0 +1,180 @@
+/**
+ * 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.
+ */
+
+#include "expect.h"
+#include "webhdfs.h"
+
+#include <errno.h>
+#include <semaphore.h>
+#include <pthread.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include "hdfs_http_client.h"
+#include "hdfs_http_query.h"
+#include "hdfs_json_parser.h"
+#include <unistd.h>
+#include <curl/curl.h>
+
+#define TLH_MAX_THREADS 100
+
+static sem_t *tlhSem;
+
+static const char *nn;
+static const char *user;
+static int port;
+
+static const char *fileName = "/tmp/tlhData";
+
+struct tlhThreadInfo {
+    /** Thread index */
+    int threadIdx;
+    /** 0 = thread was successful; error code otherwise */
+    int success;
+    /** pthread identifier */
+    pthread_t thread;
+};
+
+static int hdfsSingleNameNodeConnect(const char *nn, int port, const char *user, hdfsFS *fs)
+{
+    hdfsFS hdfs;
+    if (port < 0) {
+        fprintf(stderr, "hdfsSingleNameNodeConnect: nmdGetNameNodePort "
+                "returned error %d\n", port);
+        return port;
+    }
+    
+    hdfs = hdfsConnectAsUserNewInstance(nn, port, user);
+    if (!hdfs) {
+        return -errno;
+    }
+    *fs = hdfs;
+    return 0;
+}
+
+static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs)
+{
+    hdfsFile file;
+    int ret = 0;
+    char buffer[1024 * (ti->threadIdx + 1)];
+    memset(buffer, 'a', sizeof(buffer));
+
+    file = hdfsOpenFile(fs, "/tmp/thread_test.txt", O_WRONLY, 0, 0, 0);
+    sleep(1);
+    hdfsCloseFile(fs, file);
+    return ret;
+}
+
+static void *testHdfsOperations(void *v)
+{
+    struct tlhThreadInfo *ti = (struct tlhThreadInfo*)v;
+    hdfsFS fs = NULL;
+    int ret;
+    
+    fprintf(stderr, "testHdfsOperations(threadIdx=%d): starting\n",
+            ti->threadIdx);
+    ret = hdfsSingleNameNodeConnect(nn, port, user, &fs);
+    if (ret) {
+        fprintf(stderr, "testHdfsOperations(threadIdx=%d): "
+                "hdfsSingleNameNodeConnect failed with error %d.\n",
+                ti->threadIdx, ret);
+        ti->success = EIO;
+        return NULL;
+    }
+    ti->success = doTestHdfsOperations(ti, fs);
+    if (hdfsDisconnect(fs)) {
+        ret = errno;
+        fprintf(stderr, "hdfsDisconnect error %d\n", ret);
+        ti->success = ret;
+    }
+    return NULL;
+}
+
+static int checkFailures(struct tlhThreadInfo *ti, int tlhNumThreads)
+{
+    int i, threadsFailed = 0;
+    const char *sep = "";
+    
+    for (i = 0; i < tlhNumThreads; i++) {
+        if (ti[i].success != 0) {
+            threadsFailed = 1;
+        }
+    }
+    if (!threadsFailed) {
+        fprintf(stderr, "testLibHdfs: all threads succeeded.  SUCCESS.\n");
+        return EXIT_SUCCESS;
+    }
+    fprintf(stderr, "testLibHdfs: some threads failed: [");
+    for (i = 0; i < tlhNumThreads; i++) {
+        if (ti[i].success != 0) {
+            fprintf(stderr, "%s%d", sep, i);
+            sep = ", ";
+        }
+    }
+    fprintf(stderr, "].  FAILURE.\n");
+    return EXIT_FAILURE;
+}
+
+/**
+ * Test that we can write a file with libhdfs and then read it back
+ */
+int main(int argc, const char *args[])
+{
+    if (argc != 4) {
+        fprintf(stderr, "usage: test_libhdfs_threaded <namenode> <port> <username>");
+        return -1;
+    }
+    
+    nn = args[1];
+    port = atoi(args[2]);
+    user = args[3];
+    
+    int i, tlhNumThreads;
+    const char *tlhNumThreadsStr;
+    struct tlhThreadInfo ti[TLH_MAX_THREADS];
+    
+    tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
+    if (!tlhNumThreadsStr) {
+        tlhNumThreadsStr = "3";
+    }
+    tlhNumThreads = atoi(tlhNumThreadsStr);
+    if ((tlhNumThreads <= 0) || (tlhNumThreads > TLH_MAX_THREADS)) {
+        fprintf(stderr, "testLibHdfs: must have a number of threads "
+                "between 1 and %d inclusive, not %d\n",
+                TLH_MAX_THREADS, tlhNumThreads);
+        return EXIT_FAILURE;
+    }
+    memset(&ti[0], 0, sizeof(ti));
+    for (i = 0; i < tlhNumThreads; i++) {
+        ti[i].threadIdx = i;
+    }
+    
+    tlhSem = sem_open("sem", O_CREAT, 0644, tlhNumThreads);
+    
+    for (i = 0; i < tlhNumThreads; i++) {
+        fprintf(stderr, "\ncreating thread %d\n", i);
+        EXPECT_ZERO(pthread_create(&ti[i].thread, NULL,
+                                   testHdfsOperations, &ti[i]));
+    }
+    for (i = 0; i < tlhNumThreads; i++) {
+        EXPECT_ZERO(pthread_join(ti[i].thread, NULL));
+    }
+    
+    EXPECT_ZERO(sem_close(tlhSem));
+    return checkFailures(ti, tlhNumThreads);
+}

+ 504 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_ops.c

@@ -0,0 +1,504 @@
+/**
+ * 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.
+ */
+
+#include "webhdfs.h"
+
+#include <inttypes.h>
+#include <jni.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <time.h>
+#include <unistd.h>
+
+void permission_disp(short permissions, char *rtr) {
+    rtr[9] = '\0';
+    int i;
+    for(i=2;i>=0;i--)
+    {
+        short permissionsId = permissions >> (i * 3) & (short)7;
+        char* perm;
+        switch(permissionsId) {
+            case 7:
+                perm = "rwx"; break;
+            case 6:
+                perm = "rw-"; break;
+            case 5:
+                perm = "r-x"; break;
+            case 4:
+                perm = "r--"; break;
+            case 3:
+                perm = "-wx"; break;
+            case 2:
+                perm = "-w-"; break;
+            case 1:
+                perm = "--x"; break;
+            case 0:
+                perm = "---"; break;
+            default:
+                perm = "???";
+        }
+        strncpy(rtr, perm, 3);
+        rtr+=3;
+    }
+}
+
+int main(int argc, char **argv) {
+    if (argc != 2) {
+        fprintf(stderr, "usage: test_libwebhdfs_ops <username>\n");
+        return -1;
+    }
+    
+    char buffer[32];
+    tSize num_written_bytes;
+    
+    hdfsFS fs = hdfsConnectAsUserNewInstance("default", 50070, argv[1]);
+    if(!fs) {
+        fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
+        exit(-1);
+    }
+    
+    const char* writePath = "/tmp/testfile.txt";
+    const char* fileContents = "Hello, World!";
+    
+    {
+        //Write tests
+        
+        hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
+        if(!writeFile) {
+            fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+        num_written_bytes = hdfsWrite(fs, writeFile, (void*)fileContents, strlen(fileContents) + 1);
+        if (num_written_bytes != strlen(fileContents) + 1) {
+            fprintf(stderr, "Failed to write correct number of bytes - expected %d, got %d\n",
+                    (int)(strlen(fileContents) + 1), (int)num_written_bytes);
+            exit(-1);
+        }
+        fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
+        
+        tOffset currentPos = -1;
+        if ((currentPos = hdfsTell(fs, writeFile)) == -1) {
+            fprintf(stderr,
+                    "Failed to get current file position correctly! Got %lld!\n",
+                    currentPos);
+            exit(-1);
+        }
+        fprintf(stderr, "Current position: %lld\n", currentPos);
+        
+        if (hdfsFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'flush' %s\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Flushed %s successfully!\n", writePath);
+        
+        if (hdfsHFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'hflush' %s\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "HFlushed %s successfully!\n", writePath);
+        
+        hdfsCloseFile(fs, writeFile);
+    }
+    
+    {
+        //Read tests
+        sleep(1);
+        const char* readPath = "/tmp/testfile.txt";
+        int exists = hdfsExists(fs, readPath);
+        
+        if (exists) {
+            fprintf(stderr, "Failed to validate existence of %s\n", readPath);
+            exists = hdfsExists(fs, readPath);
+            if (exists) {
+                fprintf(stderr, "Still failed to validate existence of %s\n", readPath);
+                exit(-1);
+            }
+        }
+        
+        hdfsFile readFile = hdfsOpenFile(fs, readPath, O_RDONLY, 0, 0, 0);
+        if (!readFile) {
+            fprintf(stderr, "Failed to open %s for reading!\n", readPath);
+            exit(-1);
+        }
+        
+        if (!hdfsFileIsOpenForRead(readFile)) {
+            fprintf(stderr, "hdfsFileIsOpenForRead: we just opened a file "
+                    "with O_RDONLY, and it did not show up as 'open for "
+                    "read'\n");
+            exit(-1);
+        }
+        
+        fprintf(stderr, "hdfsAvailable: %d\n", hdfsAvailable(fs, readFile));
+        
+        tOffset seekPos = 1;
+        if(hdfsSeek(fs, readFile, seekPos)) {
+            fprintf(stderr, "Failed to seek %s for reading!\n", readPath);
+            exit(-1);
+        }
+        
+        tOffset currentPos = -1;
+        if((currentPos = hdfsTell(fs, readFile)) != seekPos) {
+            fprintf(stderr,
+                    "Failed to get current file position correctly! Got %lld!\n",
+                    currentPos);
+            exit(-1);
+        }
+        fprintf(stderr, "Current position: %lld\n", currentPos);
+        
+        if (!hdfsFileUsesDirectRead(readFile)) {
+            fprintf(stderr, "Direct read support incorrectly not detected "
+                    "for HDFS filesystem\n");
+            exit(-1);
+        }
+        
+        fprintf(stderr, "Direct read support detected for HDFS\n");
+        
+        // Test the direct read path
+        if(hdfsSeek(fs, readFile, 0)) {
+            fprintf(stderr, "Failed to seek %s for reading!\n", readPath);
+            exit(-1);
+        }
+        memset(buffer, 0, sizeof(buffer));
+        tSize num_read_bytes = hdfsRead(fs, readFile, (void*)buffer,
+                                        sizeof(buffer));
+        if (strncmp(fileContents, buffer, strlen(fileContents)) != 0) {
+            fprintf(stderr, "Failed to read (direct). Expected %s but got %s (%d bytes)\n",
+                    fileContents, buffer, num_read_bytes);
+            exit(-1);
+        }
+        fprintf(stderr, "Read following %d bytes:\n%s\n",
+                num_read_bytes, buffer);
+        if (hdfsSeek(fs, readFile, 0L)) {
+            fprintf(stderr, "Failed to seek to file start!\n");
+            exit(-1);
+        }
+        
+        // Disable the direct read path so that we really go through the slow
+        // read path
+        hdfsFileDisableDirectRead(readFile);
+        
+        num_read_bytes = hdfsRead(fs, readFile, (void*)buffer,
+                                  sizeof(buffer));
+        fprintf(stderr, "Read following %d bytes:\n%s\n",
+                num_read_bytes, buffer);
+        
+        memset(buffer, 0, strlen(fileContents + 1));
+        
+        num_read_bytes = hdfsPread(fs, readFile, 0, (void*)buffer,
+                                   sizeof(buffer));
+        fprintf(stderr, "Read following %d bytes:\n%s\n",
+                num_read_bytes, buffer);
+        
+        hdfsCloseFile(fs, readFile);
+    }
+    
+    int totalResult = 0;
+    int result = 0;
+    {
+        //Generic file-system operations
+        
+        const char* srcPath = "/tmp/testfile.txt";
+        const char* dstPath = "/tmp/testfile2.txt";
+        const char* copyPath = "/tmp/testfile_copy.txt";
+        const char* movePath = "/tmp/testfile_move.txt";
+        
+        fprintf(stderr, "hdfsCopy: %s\n", ((result = hdfsCopy(fs, srcPath, fs, copyPath)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        fprintf(stderr, "hdfsMove: %s\n", ((result = hdfsMove(fs, copyPath, fs, movePath)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        fprintf(stderr, "hdfsGetDefaultBlockSize: %lld\n", hdfsGetDefaultBlockSize(fs));
+        
+        fprintf(stderr, "hdfsRename: %s\n", ((result = hdfsRename(fs, srcPath, dstPath)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        fprintf(stderr, "hdfsRename back: %s\n", ((result = hdfsRename(fs, dstPath, srcPath)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        const char* slashTmp = "/tmp";
+        const char* newDirectory = "/tmp/newdir";
+        fprintf(stderr, "hdfsCreateDirectory: %s\n", ((result = hdfsCreateDirectory(fs, newDirectory)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        fprintf(stderr, "hdfsSetReplication: %s\n", ((result = hdfsSetReplication(fs, srcPath, 1)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        char buffer[256];
+        const char *resp;
+        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer, sizeof(buffer))) ? buffer : "Failed!"));
+        totalResult += (resp ? 0 : 1);
+        fprintf(stderr, "hdfsSetWorkingDirectory: %s\n", ((result = hdfsSetWorkingDirectory(fs, slashTmp)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer, sizeof(buffer))) ? buffer : "Failed!"));
+        totalResult += (resp ? 0 : 1);
+        
+        hdfsFileInfo *fileInfo = NULL;
+        if((fileInfo = hdfsGetPathInfo(fs, slashTmp)) != NULL) {
+            fprintf(stderr, "hdfsGetPathInfo - SUCCESS!\n");
+            fprintf(stderr, "Name: %s, ", fileInfo->mName);
+            fprintf(stderr, "Type: %c, ", (char)(fileInfo->mKind));
+            fprintf(stderr, "Replication: %d, ", fileInfo->mReplication);
+            fprintf(stderr, "BlockSize: %lld, ", fileInfo->mBlockSize);
+            fprintf(stderr, "Size: %lld, ", fileInfo->mSize);
+            fprintf(stderr, "LastMod: %s", ctime(&fileInfo->mLastMod));
+            fprintf(stderr, "Owner: %s, ", fileInfo->mOwner);
+            fprintf(stderr, "Group: %s, ", fileInfo->mGroup);
+            char permissions[10];
+            permission_disp(fileInfo->mPermissions, permissions);
+            fprintf(stderr, "Permissions: %d (%s)\n", fileInfo->mPermissions, permissions);
+            hdfsFreeFileInfo(fileInfo, 1);
+        } else {
+            totalResult++;
+            fprintf(stderr, "waah! hdfsGetPathInfo for %s - FAILED!\n", slashTmp);
+        }
+        
+        hdfsFileInfo *fileList = 0;
+        int numEntries = 0;
+        if((fileList = hdfsListDirectory(fs, slashTmp, &numEntries)) != NULL) {
+            int i = 0;
+            for(i=0; i < numEntries; ++i) {
+                fprintf(stderr, "Name: %s, ", fileList[i].mName);
+                fprintf(stderr, "Type: %c, ", (char)fileList[i].mKind);
+                fprintf(stderr, "Replication: %d, ", fileList[i].mReplication);
+                fprintf(stderr, "BlockSize: %lld, ", fileList[i].mBlockSize);
+                fprintf(stderr, "Size: %lld, ", fileList[i].mSize);
+                fprintf(stderr, "LastMod: %s", ctime(&fileList[i].mLastMod));
+                fprintf(stderr, "Owner: %s, ", fileList[i].mOwner);
+                fprintf(stderr, "Group: %s, ", fileList[i].mGroup);
+                char permissions[10];
+                permission_disp(fileList[i].mPermissions, permissions);
+                fprintf(stderr, "Permissions: %d (%s)\n", fileList[i].mPermissions, permissions);
+            }
+            hdfsFreeFileInfo(fileList, numEntries);
+        } else {
+            if (errno) {
+                totalResult++;
+                fprintf(stderr, "waah! hdfsListDirectory - FAILED!\n");
+            } else {
+                fprintf(stderr, "Empty directory!\n");
+            }
+        }
+        
+        //        char*** hosts = hdfsGetHosts(fs, srcPath, 0, 1);
+        //        if(hosts) {
+        //            fprintf(stderr, "hdfsGetHosts - SUCCESS! ... \n");
+        //            int i=0;
+        //            while(hosts[i]) {
+        //                int j = 0;
+        //                while(hosts[i][j]) {
+        //                    fprintf(stderr,
+        //                            "\thosts[%d][%d] - %s\n", i, j, hosts[i][j]);
+        //                    ++j;
+        //                }
+        //                ++i;
+        //            }
+        //        } else {
+        //            totalResult++;
+        //            fprintf(stderr, "waah! hdfsGetHosts - FAILED!\n");
+        //        }
+        
+        char *newOwner = "root";
+        // setting tmp dir to 777 so later when connectAsUser nobody, we can write to it
+        short newPerm = 0666;
+        
+        // chown write
+        fprintf(stderr, "hdfsChown: %s\n", ((result = hdfsChown(fs, writePath, NULL, "users")) ? "Failed!" : "Success!"));
+        totalResult += result;
+        fprintf(stderr, "hdfsChown: %s\n", ((result = hdfsChown(fs, writePath, newOwner, NULL)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        // chmod write
+        fprintf(stderr, "hdfsChmod: %s\n", ((result = hdfsChmod(fs, writePath, newPerm)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        
+        
+        sleep(2);
+        tTime newMtime = time(NULL);
+        tTime newAtime = time(NULL);
+        
+        // utime write
+        fprintf(stderr, "hdfsUtime: %s\n", ((result = hdfsUtime(fs, writePath, newMtime, newAtime)) ? "Failed!" : "Success!"));
+        
+        totalResult += result;
+        
+        // chown/chmod/utime read
+        hdfsFileInfo *finfo = hdfsGetPathInfo(fs, writePath);
+        
+        fprintf(stderr, "hdfsChown read: %s\n", ((result = (strcmp(finfo->mOwner, newOwner) != 0)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        fprintf(stderr, "hdfsChmod read: %s\n", ((result = (finfo->mPermissions != newPerm)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        // will later use /tmp/ as a different user so enable it
+        fprintf(stderr, "hdfsChmod: %s\n", ((result = hdfsChmod(fs, "/tmp/", 0777)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        fprintf(stderr,"newMTime=%ld\n",newMtime);
+        fprintf(stderr,"curMTime=%ld\n",finfo->mLastMod);
+        
+        
+        fprintf(stderr, "hdfsUtime read (mtime): %s\n", ((result = (finfo->mLastMod != newMtime / 1000)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        hdfsFreeFileInfo(finfo, 1);
+        
+        // Clean up
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, newDirectory, 1)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, srcPath, 1)) ? "Failed!" : "Success!"));
+        totalResult += result;
+//        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, movePath, 1)) ? "Failed!" : "Success!"));
+//        totalResult += result;
+        fprintf(stderr, "hdfsExists: %s\n", ((result = hdfsExists(fs, newDirectory)) ? "Success!" : "Failed!"));
+        totalResult += (result ? 0 : 1);
+    }
+    
+    {
+        // TEST APPENDS
+        const char *writePath = "/tmp/appends";
+        
+        // CREATE
+        hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY, 0, 0, 0);
+        if(!writeFile) {
+            fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+        
+        const char* buffer = "Hello,";
+        tSize num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer));
+        fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
+        
+        if (hdfsFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'flush' %s\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Flushed %s successfully!\n", writePath);
+        
+        hdfsCloseFile(fs, writeFile);
+        
+        fprintf(stderr, "hdfsSetReplication: %s\n", ((result = hdfsSetReplication(fs, writePath, 1)) ? "Failed!" : "Success!"));
+        totalResult += result;
+        
+        // RE-OPEN
+        writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_APPEND, 0, 0, 0);
+        if(!writeFile) {
+            fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Opened %s for appending successfully...\n", writePath);
+        
+        buffer = " World";
+        num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer) + 1);
+        fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
+        
+        if (hdfsFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'flush' %s\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Flushed %s successfully!\n", writePath);
+        
+        hdfsCloseFile(fs, writeFile);
+
+        // CHECK size
+        hdfsFileInfo *finfo = hdfsGetPathInfo(fs, writePath);
+        fprintf(stderr, "fileinfo->mSize: == total %s\n", ((result = (finfo->mSize == strlen("Hello, World") + 1)) ? "Success!" : "Failed!"));
+        totalResult += (result ? 0 : 1);
+        
+        // READ and check data
+        hdfsFile readFile = hdfsOpenFile(fs, writePath, O_RDONLY, 0, 0, 0);
+        if (!readFile) {
+            fprintf(stderr, "Failed to open %s for reading!\n", writePath);
+            exit(-1);
+        }
+        
+        char rdbuffer[32];
+        tSize num_read_bytes = hdfsRead(fs, readFile, (void*)rdbuffer, sizeof(rdbuffer));
+        fprintf(stderr, "Read following %d bytes:\n%s\n",
+                num_read_bytes, rdbuffer);
+        
+        fprintf(stderr, "read == Hello, World %s\n", (result = (strcmp(rdbuffer, "Hello, World") == 0)) ? "Success!" : "Failed!");
+        
+        hdfsCloseFile(fs, readFile);
+        
+        // DONE test appends
+    }
+    
+    
+    totalResult += (hdfsDisconnect(fs) != 0);
+    
+    {
+        //
+        // Now test as connecting as a specific user
+        // This is only meant to test that we connected as that user, not to test
+        // the actual fs user capabilities. Thus just create a file and read
+        // the owner is correct.
+        
+        const char *tuser = "nobody";
+        const char* writePath = "/tmp/usertestfile.txt";
+        
+        fs = hdfsConnectAsUserNewInstance("default", 50070, tuser);
+        if(!fs) {
+            fprintf(stderr, "Oops! Failed to connect to hdfs as user %s!\n",tuser);
+            exit(-1);
+        }
+        
+        hdfsFile writeFile = hdfsOpenFile(fs, writePath, O_WRONLY|O_CREAT, 0, 0, 0);
+        if(!writeFile) {
+            fprintf(stderr, "Failed to open %s for writing!\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
+        
+        char* buffer = "Hello, World!";
+        tSize num_written_bytes = hdfsWrite(fs, writeFile, (void*)buffer, strlen(buffer)+1);
+        fprintf(stderr, "Wrote %d bytes\n", num_written_bytes);
+        
+        if (hdfsFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'flush' %s\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "Flushed %s successfully!\n", writePath);
+        
+        hdfsCloseFile(fs, writeFile);
+        
+        hdfsFileInfo *finfo = hdfsGetPathInfo(fs, writePath);
+        if (finfo) {
+            fprintf(stderr, "hdfs new file user is correct: %s\n", ((result = (strcmp(finfo->mOwner, tuser) != 0)) ? "Failed!" : "Success!"));
+        } else {
+            fprintf(stderr, "hdfsFileInfo returned by hdfsGetPathInfo is NULL\n");
+            result = -1;
+        }
+        totalResult += result;
+    }
+    
+    totalResult += (hdfsDisconnect(fs) != 0);
+    fprintf(stderr, "totalResult == %d\n", totalResult);
+    
+    if (totalResult != 0) {
+        return -1;
+    } else {
+        return 0;
+    }
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_read.c

@@ -0,0 +1,73 @@
+/**
+ * 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.
+ */
+
+#include "webhdfs.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+
+int main(int argc, char **argv) {
+    
+    if (argc != 4) {
+        fprintf(stderr, "Usage: hdfs_read <filename> <filesize> <buffersize>\n");
+        exit(-1);
+    }
+    
+    hdfsFS fs = hdfsConnect("0.0.0.0", 50070);
+    if (!fs) {
+        fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
+        exit(-1);
+    }
+    
+    const char* rfile = argv[1];
+    tSize fileTotalSize = strtoul(argv[2], NULL, 10);
+    tSize bufferSize = strtoul(argv[3], NULL, 10);
+    
+    hdfsFile readFile = hdfsOpenFile(fs, rfile, O_RDONLY, bufferSize, 0, 0);
+    if (!readFile) {
+        fprintf(stderr, "Failed to open %s for writing!\n", rfile);
+        exit(-2);
+    }
+    
+    // data to be written to the file
+    char* buffer = malloc(sizeof(char) * bufferSize);
+    if(buffer == NULL) {
+        return -2;
+    }
+    
+    // read from the file
+    tSize curSize = bufferSize;
+    tSize totalReadSize = 0;
+    for (; (curSize = hdfsRead(fs, readFile, (void*)buffer, bufferSize)) == bufferSize ;) {
+        totalReadSize += curSize;
+    }
+    totalReadSize += curSize;
+    
+    fprintf(stderr, "size of the file: %d; reading size: %d\n", fileTotalSize, totalReadSize);
+    
+    free(buffer);
+    hdfsCloseFile(fs, readFile);
+    hdfsDisconnect(fs);
+    
+    return 0;
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */
+

+ 225 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_threaded.c

@@ -0,0 +1,225 @@
+/**
+ * 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.
+ */
+
+#include "expect.h"
+#include "webhdfs.h"
+
+#include <errno.h>
+#include <semaphore.h>
+#include <pthread.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define TLH_MAX_THREADS 100
+
+static sem_t *tlhSem;
+
+static const char *nn;
+static const char *user;
+static int port;
+
+struct tlhThreadInfo {
+    /** Thread index */
+    int threadIdx;
+    /** 0 = thread was successful; error code otherwise */
+    int success;
+    /** pthread identifier */
+    pthread_t thread;
+};
+
+static int hdfsSingleNameNodeConnect(const char *nn, int port, const char *user, hdfsFS *fs)
+{
+    hdfsFS hdfs;
+    if (port < 0) {
+        fprintf(stderr, "hdfsSingleNameNodeConnect: nmdGetNameNodePort "
+                "returned error %d\n", port);
+        return port;
+    }
+    
+    hdfs = hdfsConnectAsUserNewInstance(nn, port, user);
+    if (!hdfs) {
+        return -errno;
+    }
+    *fs = hdfs;
+    return 0;
+}
+
+static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs)
+{
+    char prefix[256], tmp[256];
+    hdfsFile file;
+    int ret, expected;
+    
+    snprintf(prefix, sizeof(prefix), "/tlhData%04d", ti->threadIdx);
+    
+    if (hdfsExists(fs, prefix) == 0) {
+        EXPECT_ZERO(hdfsDelete(fs, prefix, 1));
+    }
+    EXPECT_ZERO(hdfsCreateDirectory(fs, prefix));
+    snprintf(tmp, sizeof(tmp), "%s/file", prefix);
+    
+    /*
+     * Although there should not be any file to open for reading,
+     * the right now implementation only construct a local
+     * information struct when opening file
+     */
+    EXPECT_NONNULL(hdfsOpenFile(fs, tmp, O_RDONLY, 0, 0, 0));
+    
+    file = hdfsOpenFile(fs, tmp, O_WRONLY, 0, 0, 0);
+    EXPECT_NONNULL(file);
+    
+    /* TODO: implement writeFully and use it here */
+    expected = strlen(prefix);
+    ret = hdfsWrite(fs, file, prefix, expected);
+    if (ret < 0) {
+        ret = errno;
+        fprintf(stderr, "hdfsWrite failed and set errno %d\n", ret);
+        return ret;
+    }
+    if (ret != expected) {
+        fprintf(stderr, "hdfsWrite was supposed to write %d bytes, but "
+                "it wrote %d\n", ret, expected);
+        return EIO;
+    }
+    EXPECT_ZERO(hdfsFlush(fs, file));
+    EXPECT_ZERO(hdfsCloseFile(fs, file));
+    
+    /* Let's re-open the file for reading */
+    file = hdfsOpenFile(fs, tmp, O_RDONLY, 0, 0, 0);
+    EXPECT_NONNULL(file);
+    
+    /* TODO: implement readFully and use it here */
+    ret = hdfsRead(fs, file, tmp, sizeof(tmp));
+    if (ret < 0) {
+        ret = errno;
+        fprintf(stderr, "hdfsRead failed and set errno %d\n", ret);
+        return ret;
+    }
+    if (ret != expected) {
+        fprintf(stderr, "hdfsRead was supposed to read %d bytes, but "
+                "it read %d\n", ret, expected);
+        return EIO;
+    }
+    EXPECT_ZERO(memcmp(prefix, tmp, expected));
+    EXPECT_ZERO(hdfsCloseFile(fs, file));
+    
+    // TODO: Non-recursive delete should fail?
+    //EXPECT_NONZERO(hdfsDelete(fs, prefix, 0));
+    
+    EXPECT_ZERO(hdfsDelete(fs, prefix, 1));
+    return 0;
+}
+
+static void *testHdfsOperations(void *v)
+{
+    struct tlhThreadInfo *ti = (struct tlhThreadInfo*)v;
+    hdfsFS fs = NULL;
+    int ret;
+    
+    fprintf(stderr, "testHdfsOperations(threadIdx=%d): starting\n",
+            ti->threadIdx);
+    ret = hdfsSingleNameNodeConnect(nn, port, user, &fs);
+    if (ret) {
+        fprintf(stderr, "testHdfsOperations(threadIdx=%d): "
+                "hdfsSingleNameNodeConnect failed with error %d.\n",
+                ti->threadIdx, ret);
+        ti->success = EIO;
+        return NULL;
+    }
+    ti->success = doTestHdfsOperations(ti, fs);
+    if (hdfsDisconnect(fs)) {
+        ret = errno;
+        fprintf(stderr, "hdfsDisconnect error %d\n", ret);
+        ti->success = ret;
+    }
+    return NULL;
+}
+
+static int checkFailures(struct tlhThreadInfo *ti, int tlhNumThreads)
+{
+    int i, threadsFailed = 0;
+    const char *sep = "";
+    
+    for (i = 0; i < tlhNumThreads; i++) {
+        if (ti[i].success != 0) {
+            threadsFailed = 1;
+        }
+    }
+    if (!threadsFailed) {
+        fprintf(stderr, "testLibHdfs: all threads succeeded.  SUCCESS.\n");
+        return EXIT_SUCCESS;
+    }
+    fprintf(stderr, "testLibHdfs: some threads failed: [");
+    for (i = 0; i < tlhNumThreads; i++) {
+        if (ti[i].success != 0) {
+            fprintf(stderr, "%s%d", sep, i);
+            sep = ", ";
+        }
+    }
+    fprintf(stderr, "].  FAILURE.\n");
+    return EXIT_FAILURE;
+}
+
+/**
+ * Test that we can write a file with libhdfs and then read it back
+ */
+int main(int argc, const char *args[])
+{
+    if (argc != 4) {
+        fprintf(stderr, "usage: test_libhdfs_threaded <namenode> <port> <username>");
+        return -1;
+    }
+    
+    nn = args[1];
+    port = atoi(args[2]);
+    user = args[3];
+    
+    int i, tlhNumThreads;
+    const char *tlhNumThreadsStr;
+    struct tlhThreadInfo ti[TLH_MAX_THREADS];
+    
+    tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
+    if (!tlhNumThreadsStr) {
+        tlhNumThreadsStr = "3";
+    }
+    tlhNumThreads = atoi(tlhNumThreadsStr);
+    if ((tlhNumThreads <= 0) || (tlhNumThreads > TLH_MAX_THREADS)) {
+        fprintf(stderr, "testLibHdfs: must have a number of threads "
+                "between 1 and %d inclusive, not %d\n",
+                TLH_MAX_THREADS, tlhNumThreads);
+        return EXIT_FAILURE;
+    }
+    memset(&ti[0], 0, sizeof(ti));
+    for (i = 0; i < tlhNumThreads; i++) {
+        ti[i].threadIdx = i;
+    }
+    
+//    tlhSem = sem_open("sem", O_CREAT, 0644, tlhNumThreads);
+    
+    for (i = 0; i < tlhNumThreads; i++) {
+        EXPECT_ZERO(pthread_create(&ti[i].thread, NULL,
+                                   testHdfsOperations, &ti[i]));
+    }
+    for (i = 0; i < tlhNumThreads; i++) {
+        EXPECT_ZERO(pthread_join(ti[i].thread, NULL));
+    }
+    
+//    EXPECT_ZERO(sem_close(tlhSem));
+    return checkFailures(ti, tlhNumThreads);
+}

+ 118 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_libwebhdfs_write.c

@@ -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.
+ */
+
+#include "webhdfs.h"
+
+#include <limits.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+int main(int argc, char **argv) {
+    
+    if (argc != 6) {
+        fprintf(stderr, "Usage: hdfs_write <filename> <filesize> <buffersize> <username> <append>\n");
+        exit(-1);
+    }
+    
+    hdfsFS fs = hdfsConnectAsUser("0.0.0.0", 50070, argv[4]);
+    if (!fs) {
+        fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
+        exit(-1);
+    }
+    
+    const char* writeFileName = argv[1];
+    off_t fileTotalSize = strtoul(argv[2], NULL, 10);
+    long long tmpBufferSize = strtoul(argv[3], NULL, 10);
+    
+    // sanity check
+    if(fileTotalSize == ULONG_MAX && errno == ERANGE) {
+        fprintf(stderr, "invalid file size %s - must be <= %lu\n", argv[2], ULONG_MAX);
+        exit(-3);
+    }
+    
+    // currently libhdfs writes are of tSize which is int32
+    if(tmpBufferSize > INT_MAX) {
+        fprintf(stderr, "invalid buffer size libhdfs API write chunks must be <= %d\n",INT_MAX);
+        exit(-3);
+    }
+    
+    tSize bufferSize = tmpBufferSize;
+    
+    hdfsFile writeFile = NULL;
+    int append = atoi(argv[5]);
+    if (!append) {
+        writeFile = hdfsOpenFile(fs, writeFileName, O_WRONLY, bufferSize, 2, 0);
+    } else {
+        writeFile = hdfsOpenFile(fs, writeFileName, O_WRONLY | O_APPEND, bufferSize, 2, 0);
+    }
+    if (!writeFile) {
+        fprintf(stderr, "Failed to open %s for writing!\n", writeFileName);
+        exit(-2);
+    }
+    
+    // data to be written to the file
+    char* buffer = malloc(sizeof(char) * bufferSize + 1);
+    if(buffer == NULL) {
+        fprintf(stderr, "Could not allocate buffer of size %d\n", bufferSize);
+        return -2;
+    }
+    int i = 0;
+    for (i=0; i < bufferSize; ++i) {
+        buffer[i] = 'a' + (i%26);
+    }
+    buffer[bufferSize] = '\0';
+
+    size_t totalWriteSize = 0;
+    for (; totalWriteSize < fileTotalSize; ) {
+        tSize toWrite = bufferSize < (fileTotalSize - totalWriteSize) ? bufferSize : (fileTotalSize - totalWriteSize);
+        size_t written = hdfsWrite(fs, writeFile, (void*)buffer, toWrite);
+        fprintf(stderr, "written size %ld, to write size %d\n", written, toWrite);
+        totalWriteSize += written;
+        //sleep(1);
+    }
+    
+    free(buffer);
+    hdfsCloseFile(fs, writeFile);
+    
+    fprintf(stderr, "file total size: %lld, total write size: %ld\n", fileTotalSize, totalWriteSize);
+    
+    hdfsFile readFile = hdfsOpenFile(fs, writeFileName, O_RDONLY, 0, 0, 0);
+    //sleep(1);
+    fprintf(stderr, "hdfsAvailable: %d\n", hdfsAvailable(fs, readFile));
+    
+    hdfsFile writeFile2 = hdfsOpenFile(fs, writeFileName, O_WRONLY | O_APPEND, 0, 2, 0);
+    fprintf(stderr, "Opened %s for writing successfully...\n", writeFileName);
+    const char *content = "Hello, World!";
+    size_t num_written_bytes = hdfsWrite(fs, writeFile2, content, strlen(content) + 1);
+    if (num_written_bytes != strlen(content) + 1) {
+        fprintf(stderr, "Failed to write correct number of bytes - expected %d, got %d\n",
+                                    (int)(strlen(content) + 1), (int)num_written_bytes);
+        exit(-1);
+    }
+    fprintf(stderr, "Wrote %zd bytes\n", num_written_bytes);
+    
+    hdfsDisconnect(fs);
+    
+    return 0;
+}
+
+/**
+ * vim: ts=4: sw=4: et:
+ */
+

+ 110 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/test_read_bm.c

@@ -0,0 +1,110 @@
+#include <time.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <sys/time.h>
+#include "webhdfs.h"
+
+#ifdef __MACH__
+#include <mach/clock.h>
+#include <mach/mach.h>
+#endif
+
+void current_utc_time(struct timespec *ts) {
+#ifdef __MACH__ // OS X does not have clock_gettime, use clock_get_time
+    clock_serv_t cclock;
+    mach_timespec_t mts;
+    host_get_clock_service(mach_host_self(), CALENDAR_CLOCK, &cclock);
+    clock_get_time(cclock, &mts);
+    mach_port_deallocate(mach_task_self(), cclock);
+    ts->tv_sec = mts.tv_sec;
+    ts->tv_nsec = mts.tv_nsec;
+#else
+    clock_gettime(CLOCK_REALTIME, ts);
+#endif
+    
+}
+
+long get_time() {
+    struct timespec tp;
+    current_utc_time(&tp);
+    return (long)((tp.tv_sec * 1000000000) + tp.tv_nsec);
+}
+
+#define SIZE 512*1024*1024
+#define READ_SIZE 512*1024*1024
+#define DISCARD_COUNT 5
+
+int main(int argc, char** argv) {
+    if (argc != 4) {
+        fprintf(stderr, "Usage: test_read_bm <namenode> <user_name> <iteration_number>\n");
+        exit(0);
+    }
+    
+    hdfsFS fs = hdfsConnectAsUser(argv[1], 50070, argv[2]);
+    
+    /* printf("File is null: %d\n", file == NULL ? 1 : 0); */
+    
+    char *buf = (char *) malloc(sizeof(unsigned char) * SIZE);
+    
+    printf("Read size: %d\n", READ_SIZE);
+    
+    int iterations = atoi(argv[3]);
+    
+    if (iterations <= DISCARD_COUNT) {
+        printf("Iterations should be at least %d\n", DISCARD_COUNT + 1);
+        exit(0);
+    }
+    
+    printf("Running %d iterations\n", iterations);
+    float time_total;
+    float max = 0.f;
+    float min = 999999999999999.f;
+    
+    printf("Start...\n");
+    int i;
+    for (i=0; i<iterations; ++i) {
+        long start = get_time();
+        hdfsFile file = hdfsOpenFile(fs, "/tmp/512_mb.txt", O_RDONLY, 0, 0, 0);
+        int n = 0;
+        
+        while (n < SIZE) {
+            int nread = hdfsRead(fs, file, buf + n, READ_SIZE);
+            if (nread <= 0) {
+                printf("EOF before finished, read %d bytes\n", n);
+                hdfsDisconnect(fs);
+                return 0;
+            }
+            n += nread;
+            printf("Read %d kilobytes\n", nread / 1024);
+        }
+        
+        long end = get_time();
+        printf("Read %d bytes, hoping for %d.\n", n, SIZE);
+        long elapsed = (end - start);
+        printf("Start: %lu, end: %lu\n", start, end);
+        float time = elapsed / (1000000000.0f);
+        printf ("Took %2.6fs\n", time);
+        printf("Throughput: %2.2fMB/s\n", SIZE * 1.0f / (1024 * 1024 * time));
+        if (i >= DISCARD_COUNT) {
+            time_total += time;
+            if (time < min) {
+                min = time;
+            }
+            if (time > max) {
+                max = time;
+            }
+        }
+    }
+    hdfsDisconnect(fs);
+    printf("------\n");
+    printf("Average time: %2.2fs\n", time_total / (iterations - DISCARD_COUNT));
+    printf("Max. time: %2.2f, min. time: %2.2f\n", max, min);
+    float maxt = SIZE * 1.f / (1024 * 1024 * max);
+    float mint = SIZE * 1.f / (1024 * 1024 * min);
+    printf("Average throughput: %2.2fMB/s\n", 1.f * SIZE * (iterations - DISCARD_COUNT) / (1024 * 1024 * time_total));
+    printf("Max. throughput: %2.2f, min. throughput: %2.2f\n", maxt, mint);
+    
+    //  printf("File contents: %d\n", buf[0]);
+    return 0;
+}
+

+ 694 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/webhdfs.h

@@ -0,0 +1,694 @@
+/**
+ * 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.
+ */
+
+#ifndef LIB_WEBHDFS_H
+#define LIB_WEBHDFS_H
+
+#include <errno.h> /* for EINTERNAL, etc. */
+#include <fcntl.h> /* for O_RDONLY, O_WRONLY */
+#include <stdint.h> /* for uint64_t, etc. */
+#include <time.h> /* for time_t */
+#include <pthread.h>
+
+#ifndef O_RDONLY
+#define O_RDONLY 1
+#endif
+
+#ifndef O_WRONLY
+#define O_WRONLY 2
+#endif
+
+#ifndef EINTERNAL
+#define EINTERNAL 255
+#endif
+
+/** All APIs set errno to meaningful values */
+
+#ifdef __cplusplus
+extern  "C" {
+#endif
+    /**
+     * Some utility decls used in libhdfs.
+     */
+    typedef int32_t   tSize; /// size of data for read/write io ops
+    typedef time_t    tTime; /// time type in seconds
+    typedef int64_t   tOffset;/// offset within the file
+    typedef uint16_t  tPort; /// port
+    
+    /**
+     * The information required for accessing webhdfs,
+     * including the network address of the namenode and the user name
+     */
+    struct hdfsBuilder {
+        int forceNewInstance;
+        const char *nn;
+        const char *nn_jni;
+        tPort port;
+        const char *kerbTicketCachePath;
+        const char *userName;
+        /*
+         * This is a new attribute compared to libhdfs.
+         * We maintain a local workingDir for constructing absolute path
+         */
+        char *workingDir;
+    };
+    
+    typedef enum tObjectKind {
+        kObjectKindFile = 'F',
+        kObjectKindDirectory = 'D',
+    } tObjectKind;
+    
+    /**
+     * For libhdfs based on JNI, this is used as
+     * the C reflection of org.apache.org.hadoop.FileSystem .
+     * In the current libwebhdfs based on webhdfs,
+     * this is actually hdfsBuilder which contains
+     * the network address of the namenode and the user name
+     */
+    struct hdfs_internal;
+    typedef struct hdfs_internal* hdfsFS;
+    
+    /**
+     * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
+     */
+    enum hdfsStreamType
+    {
+        UNINITIALIZED = 0,
+        INPUT = 1,
+        OUTPUT = 2,
+    };
+    
+    /**
+     * The 'file-handle' to a file in hdfs.
+     */
+    struct hdfsFile_internal {
+        void* file;
+        enum hdfsStreamType type;
+        int flags;
+        tOffset offset;
+    };
+    typedef struct hdfsFile_internal* hdfsFile;
+    
+    /**
+     * hdfsFileInfo - Information about a file/directory.
+     */
+    typedef struct  {
+        tObjectKind mKind;   /* file or directory */
+        char *mName;         /* the name of the file */
+        tTime mLastMod;      /* the last modification time for the file in seconds */
+        tOffset mSize;       /* the size of the file in bytes */
+        short mReplication;    /* the count of replicas */
+        tOffset mBlockSize;  /* the block size for the file */
+        char *mOwner;        /* the owner of the file */
+        char *mGroup;        /* the group associated with the file */
+        short mPermissions;  /* the permissions associated with the file */
+        tTime mLastAccess;    /* the last access time for the file in seconds */
+    } hdfsFileInfo;
+    
+    /**
+     * webhdfsBuffer - used for hold the data for read/write from/to http connection
+     */
+    typedef struct {
+        const char *wbuffer;      /* the user's buffer for uploading */
+        size_t remaining;         /* length of content */
+        size_t offset;            /* offset for reading */
+        int openFlag;             /* check whether the hdfsOpenFile has been called before */
+        int closeFlag;      /* whether to close the http connection for writing */
+        pthread_mutex_t writeMutex;    // used for syschronization between the curl thread and the hdfsWrite thread
+        pthread_cond_t newwrite_or_close;   // transferring thread waits for this condition
+                                       // when there is no more content for transferring in the buffer
+        pthread_cond_t transfer_finish; // condition used to indicate finishing transferring (one buffer)
+    } webhdfsBuffer;
+    
+    struct webhdfsFileHandle {
+        char *absPath;
+        int bufferSize;
+        short replication;
+        tSize blockSize;
+        char *datanode;
+        webhdfsBuffer *uploadBuffer;
+        pthread_t connThread;
+    };
+    
+    // Bit fields for hdfsFile_internal flags
+#define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0)
+    
+    /**
+     * Determine if a file is open for read.
+     *
+     * @param file     The HDFS file
+     * @return         1 if the file is open for read; 0 otherwise
+     */
+    int hdfsFileIsOpenForRead(hdfsFile file);
+    
+    /**
+     * Determine if a file is open for write.
+     *
+     * @param file     The HDFS file
+     * @return         1 if the file is open for write; 0 otherwise
+     */
+    int hdfsFileIsOpenForWrite(hdfsFile file);
+    
+    /**
+     * Disable the direct read optimization for a file in libhdfs.
+     * This is mainly provided for unit testing purposes.
+     * No longer useful in libwebhdfs since libwebhdfs is based on webhdfs.
+     *
+     * @param file     The HDFS file
+     */
+    void hdfsFileDisableDirectRead(hdfsFile file);
+    
+    /**
+     * hdfsConnectAsUser - Connect to a hdfs file system as a specific user
+     * Connect to the hdfs.
+     * @param nn   The NameNode.  See hdfsBuilderSetNameNode for details.
+     * @param port The port on which the server is listening.
+     * @param user the user name (this is hadoop domain user). Or NULL is equivelant to hhdfsConnect(host, port)
+     * @return Returns a handle to the filesystem or NULL on error.
+     * @deprecated Use hdfsBuilderConnect instead.
+     */
+    hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user);
+    
+    
+    /**
+     * hdfsConnect - Connect to a hdfs file system.
+     * Connect to the hdfs.
+     * @param nn   The NameNode.  See hdfsBuilderSetNameNode for details.
+     * @param port The port on which the server is listening.
+     * @return Returns a handle to the filesystem or NULL on error.
+     * @deprecated Use hdfsBuilderConnect instead.
+     */
+    hdfsFS hdfsConnect(const char* nn, tPort port);
+    
+    /**
+     * hdfsConnect - Connect to an hdfs file system.
+     *
+     * The effect with hdfsConnectAsUser in libwebhdfs.
+     *
+     * @param nn     The NameNode.  See hdfsBuilderSetNameNode for details.
+     * @param port   The port on which the server is listening.
+     * @param user   The user name to use when connecting
+     * @return       Returns a handle to the filesystem or NULL on error.
+     * @deprecated   Use hdfsBuilderConnect instead.
+     */
+    hdfsFS hdfsConnectAsUserNewInstance(const char* nn, tPort port, const char *user );
+    
+    /**
+     * hdfsConnect - Connect to an hdfs file system.
+     *
+     * The same effect with hdfsConnect in libwebhdfs.
+     *
+     * @param nn     The NameNode.  See hdfsBuilderSetNameNode for details.
+     * @param port   The port on which the server is listening.
+     * @return       Returns a handle to the filesystem or NULL on error.
+     * @deprecated   Use hdfsBuilderConnect instead.
+     */
+    hdfsFS hdfsConnectNewInstance(const char* nn, tPort port);
+    
+    /**
+     * Connect to HDFS using the parameters defined by the builder.
+     *
+     * Every successful call to hdfsBuilderConnect should be matched with a call
+     * to hdfsDisconnect, when the hdfsFS is no longer needed.
+     *
+     * @param bld    The HDFS builder
+     * @return       Returns a handle to the filesystem, or NULL on error.
+     */
+    hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld);
+    
+    /**
+     * Create an HDFS builder.
+     *
+     * @return The HDFS builder, or NULL on error.
+     */
+    struct hdfsBuilder *hdfsNewBuilder(void);
+    
+    /**
+     * In libhdfs: force the builder to always create a new instance of the FileSystem,
+     * rather than possibly finding one in the cache.
+     *
+     * @param bld The HDFS builder
+     * @deprecated No longer usefule in libwebhdfs.
+     */
+    void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld);
+    
+    /**
+     * Set the HDFS NameNode to connect to.
+     *
+     * @param bld  The HDFS builder
+     * @param nn   The NameNode to use.
+     *
+     *             If the string given is 'default', the default NameNode
+     *             configuration will be used (from the XML configuration files)
+     *
+     *             If NULL is given, a LocalFileSystem will be created.
+     *
+     *             If the string starts with a protocol type such as file:// or
+     *             hdfs://, this protocol type will be used.  If not, the
+     *             hdfs:// protocol type will be used.
+     *
+     *             You may specify a NameNode port in the usual way by
+     *             passing a string of the format hdfs://<hostname>:<port>.
+     *             Alternately, you may set the port with
+     *             hdfsBuilderSetNameNodePort.  However, you must not pass the
+     *             port in two different ways.
+     */
+    void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn);
+    
+    /**
+     * Set the port of the HDFS NameNode to connect to.
+     *
+     * @param bld The HDFS builder
+     * @param port The port.
+     */
+    void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port);
+    
+    /**
+     * Set the username to use when connecting to the HDFS cluster.
+     *
+     * @param bld The HDFS builder
+     * @param userName The user name.  The string will be shallow-copied.
+     */
+    void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName);
+    
+    /**
+     * Set the path to the Kerberos ticket cache to use when connecting to
+     * the HDFS cluster.
+     *
+     * @param bld The HDFS builder
+     * @param kerbTicketCachePath The Kerberos ticket cache path.  The string
+     *                            will be shallow-copied.
+     */
+    void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
+                                           const char *kerbTicketCachePath);
+    
+    /**
+     * Free an HDFS builder.
+     *
+     * @param bld The HDFS builder
+     */
+    void hdfsFreeBuilder(struct hdfsBuilder *bld);
+    
+    /**
+     * Get a configuration string.
+     *
+     * @param key      The key to find
+     * @param val      (out param) The value.  This will be set to NULL if the
+     *                 key isn't found.  You must free this string with
+     *                 hdfsConfStrFree.
+     *
+     * @return         0 on success; nonzero error code otherwise.
+     *                 Failure to find the key is not an error.
+     */
+    int hdfsConfGetStr(const char *key, char **val);
+    
+    /**
+     * Get a configuration integer.
+     *
+     * @param key      The key to find
+     * @param val      (out param) The value.  This will NOT be changed if the
+	 *                 key isn't found.
+     *
+     * @return         0 on success; nonzero error code otherwise.
+     *                 Failure to find the key is not an error.
+     */
+    int hdfsConfGetInt(const char *key, int32_t *val);
+    
+    /**
+     * Free a configuration string found with hdfsConfGetStr.
+     *
+     * @param val      A configuration string obtained from hdfsConfGetStr
+     */
+    void hdfsConfStrFree(char *val);
+    
+    /**
+     * hdfsDisconnect - Disconnect from the hdfs file system.
+     * Disconnect from hdfs.
+     *
+     * In libwebhdfs, we simply free the hdfsFS,
+     * so do not use it after hdfsCopy/hdfsMove/hdfsGetDefaultBlockSize which still use JNI for FileSystem connection.
+     *
+     * @param fs The configured filesystem handle.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsDisconnect(hdfsFS fs);
+    
+    
+    /**
+     * hdfsOpenFile - Open a hdfs file in given mode.
+     * In libwebhdfs we simply store corresponding information in a hdfsFile.
+     *
+     * @param fs The configured filesystem handle.
+     * @param path The full path to the file.
+     * @param flags - an | of bits/fcntl.h file flags - supported flags are O_RDONLY, O_WRONLY (meaning create or overwrite i.e., implies O_TRUNCAT),
+     * O_WRONLY|O_APPEND. Other flags are generally ignored other than (O_RDWR || (O_EXCL & O_CREAT)) which return NULL and set errno equal ENOTSUP.
+     * @param bufferSize Size of buffer for read/write - pass 0 if you want
+     * to use the default configured values.
+     * @param replication Block replication - pass 0 if you want to use
+     * the default configured values.
+     * @param blocksize Size of block - pass 0 if you want to use the
+     * default configured values.
+     * @return Returns the handle to the open file or NULL on error.
+     */
+    hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
+                          int bufferSize, short replication, tSize blocksize);
+    
+    
+    /**
+     * hdfsCloseFile - Close an open file.
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsCloseFile(hdfsFS fs, hdfsFile file);
+    
+    
+    /**
+     * hdfsExists - Checks if a given path exsits on the filesystem
+     * @param fs The configured filesystem handle.
+     * @param path The path to look for
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsExists(hdfsFS fs, const char *path);
+    
+    
+    /**
+     * hdfsSeek - Seek to given offset in file.
+     * This works only for files opened in read-only mode.
+     * In libwebhdfs we store the offset in the local hdfsFile handle, thus
+     * in this function we simply set the local offset.
+     *
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @param desiredPos Offset into the file to seek into.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos);
+    
+    
+    /**
+     * hdfsTell - Get the current offset in the file, in bytes.
+     * In libwebhdfs the current offset is stored in the local hdfsFile handle,
+     * thus this function simply sets the local offset.
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @return Current offset, -1 on error.
+     */
+    tOffset hdfsTell(hdfsFS fs, hdfsFile file);
+    
+    
+    /**
+     * hdfsRead - Read data from an open file.
+     * In libwebhdfs the reading starts from the current offset which is stored in the hdfsFile handle
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @param buffer The buffer to copy read bytes into.
+     * @param length The length of the buffer.
+     * @return      On success, a positive number indicating how many bytes
+     *              were read.
+     *              On end-of-file, 0.
+     *              On error, -1.  Errno will be set to the error code.
+     *              Just like the POSIX read function, hdfsRead will return -1
+     *              and set errno to EINTR if data is temporarily unavailable,
+     *              but we are not yet at the end of the file.
+     */
+    tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length);
+    
+    /**
+     * hdfsPread - Positional read of data from an open file.
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @param position Position from which to read
+     * @param buffer The buffer to copy read bytes into.
+     * @param length The length of the buffer.
+     * @return Returns the number of bytes actually read, possibly less than
+     * than length;-1 on error.
+     */
+    tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position,
+                    void* buffer, tSize length);
+    
+    
+    /**
+     * hdfsWrite - Write data into an open file.
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @param buffer The data.
+     * @param length The no. of bytes to write.
+     * @return Returns the number of bytes written, -1 on error.
+     */
+    tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer,
+                    tSize length);
+    
+    
+    /**
+     * hdfsWrite - Flush the data. No use for libwebhdfs.
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @return Returns 0 on success, -1 on error.
+     * @deprecated Not usefule in libwebhdfs.
+     */
+    int hdfsFlush(hdfsFS fs, hdfsFile file);
+    
+    
+    /**
+     * hdfsHFlush - Flush out the data in client's user buffer. After the
+     * return of this call, new readers will see the data.
+     * @param fs configured filesystem handle
+     * @param file file handle
+     * @return 0 on success, -1 on error and sets errno
+     * @deprecated Not usefule in libwebhdfs.
+     */
+    int hdfsHFlush(hdfsFS fs, hdfsFile file);
+    
+    
+    /**
+     * hdfsAvailable - Number of bytes that can be read from this
+     * input stream.
+     * @param fs The configured filesystem handle.
+     * @param file The file handle.
+     * @return Returns available bytes; -1 on error.
+     */
+    int hdfsAvailable(hdfsFS fs, hdfsFile file);
+    
+    
+    /**
+     * hdfsCopy - Copy file from one filesystem to another.
+     * @param srcFS The handle to source filesystem.
+     * @param src The path of source file.
+     * @param dstFS The handle to destination filesystem.
+     * @param dst The path of destination file.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
+    
+    
+    /**
+     * hdfsMove - Move file from one filesystem to another.
+     * @param srcFS The handle to source filesystem.
+     * @param src The path of source file.
+     * @param dstFS The handle to destination filesystem.
+     * @param dst The path of destination file.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
+    
+    
+    /**
+     * hdfsDelete - Delete file.
+     * @param fs The configured filesystem handle.
+     * @param path The path of the file.
+     * @param recursive if path is a directory and set to
+     * non-zero, the directory is deleted else throws an exception. In
+     * case of a file the recursive argument is irrelevant.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsDelete(hdfsFS fs, const char* path, int recursive);
+    
+    /**
+     * hdfsRename - Rename file.
+     * @param fs The configured filesystem handle.
+     * @param oldPath The path of the source file.
+     * @param newPath The path of the destination file.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath);
+    
+    
+    /**
+     * hdfsGetWorkingDirectory - Get the current working directory for
+     * the given filesystem. In libwebhdfs it is retrieved from local hdfsFS handle.
+     * @param fs The configured filesystem handle.
+     * @param buffer The user-buffer to copy path of cwd into.
+     * @param bufferSize The length of user-buffer.
+     * @return Returns buffer, NULL on error.
+     */
+    char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize);
+    
+    
+    /**
+     * hdfsSetWorkingDirectory - Set the working directory. All relative
+     * paths will be resolved relative to it. In libwebhdfs the local hdfsFS is modified.
+     * @param fs The configured filesystem handle.
+     * @param path The path of the new 'cwd'.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsSetWorkingDirectory(hdfsFS fs, const char* path);
+    
+    
+    /**
+     * hdfsCreateDirectory - Make the given file and all non-existent
+     * parents into directories.
+     * @param fs The configured filesystem handle.
+     * @param path The path of the directory.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsCreateDirectory(hdfsFS fs, const char* path);
+    
+    
+    /**
+     * hdfsSetReplication - Set the replication of the specified
+     * file to the supplied value
+     * @param fs The configured filesystem handle.
+     * @param path The path of the file.
+     * @return Returns 0 on success, -1 on error.
+     */
+    int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication);
+    
+    
+    /**
+     * hdfsListDirectory - Get list of files/directories for a given
+     * directory-path. hdfsFreeFileInfo should be called to deallocate memory.
+     * @param fs The configured filesystem handle.
+     * @param path The path of the directory.
+     * @param numEntries Set to the number of files/directories in path.
+     * @return Returns a dynamically-allocated array of hdfsFileInfo
+     * objects; NULL on error.
+     */
+    hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path,
+                                    int *numEntries);
+    
+    
+    /**
+     * hdfsGetPathInfo - Get information about a path as a (dynamically
+     * allocated) single hdfsFileInfo struct. hdfsFreeFileInfo should be
+     * called when the pointer is no longer needed.
+     * @param fs The configured filesystem handle.
+     * @param path The path of the file.
+     * @return Returns a dynamically-allocated hdfsFileInfo object;
+     * NULL on error.
+     */
+    hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path);
+    
+    
+    /**
+     * hdfsFreeFileInfo - Free up the hdfsFileInfo array (including fields)
+     * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
+     * objects.
+     * @param numEntries The size of the array.
+     */
+    void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries);
+    
+    
+    /**
+     * hdfsGetHosts - Get hostnames where a particular block (determined by
+     * pos & blocksize) of a file is stored. The last element in the array
+     * is NULL. Due to replication, a single block could be present on
+     * multiple hosts.
+     * @param fs The configured filesystem handle.
+     * @param path The path of the file.
+     * @param start The start of the block.
+     * @param length The length of the block.
+     * @return Returns a dynamically-allocated 2-d array of blocks-hosts;
+     * NULL on error.
+     *
+     * Not supported yet but will be supported by libwebhdfs based on webhdfs.
+     */
+    char*** hdfsGetHosts(hdfsFS fs, const char* path,
+                         tOffset start, tOffset length);
+    
+    
+    /**
+     * hdfsFreeHosts - Free up the structure returned by hdfsGetHosts
+     * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
+     * objects.
+     * @param numEntries The size of the array.
+     */
+    void hdfsFreeHosts(char ***blockHosts);
+    
+    
+    /**
+     * hdfsGetDefaultBlockSize - Get the optimum blocksize.
+     * @param fs The configured filesystem handle.
+     * @return Returns the blocksize; -1 on error.
+     */
+    tOffset hdfsGetDefaultBlockSize(hdfsFS fs);
+    
+    
+    /**
+     * hdfsGetCapacity - Return the raw capacity of the filesystem.
+     * @param fs The configured filesystem handle.
+     * @return Returns the raw-capacity; -1 on error.
+     *
+     * Not supported yet but will be supported by libwebhdfs based on webhdfs.
+     */
+    tOffset hdfsGetCapacity(hdfsFS fs);
+    
+    
+    /**
+     * hdfsGetUsed - Return the total raw size of all files in the filesystem.
+     * @param fs The configured filesystem handle.
+     * @return Returns the total-size; -1 on error.
+     *
+     * Not supported yet but will be supported by libwebhdfs based on webhdfs.
+     */
+    tOffset hdfsGetUsed(hdfsFS fs);
+    
+    /**
+     * hdfsChown
+     * @param fs The configured filesystem handle.
+     * @param path the path to the file or directory
+     * @param owner this is a string in Hadoop land. Set to null or "" if only setting group
+     * @param group  this is a string in Hadoop land. Set to null or "" if only setting user
+     * @return 0 on success else -1
+     */
+    int hdfsChown(hdfsFS fs, const char* path, const char *owner, const char *group);
+    
+    /**
+     * hdfsChmod
+     * @param fs The configured filesystem handle.
+     * @param path the path to the file or directory
+     * @param mode the bitmask to set it to
+     * @return 0 on success else -1
+     */
+    int hdfsChmod(hdfsFS fs, const char* path, short mode);
+    
+    /**
+     * hdfsUtime
+     * @param fs The configured filesystem handle.
+     * @param path the path to the file or directory
+     * @param mtime new modification time or -1 for no change
+     * @param atime new access time or -1 for no change
+     * @return 0 on success else -1
+     */
+    int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime);
+    
+#ifdef __cplusplus
+}
+#endif
+
+#endif /*LIB_WEBHDFS_H*/

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -321,7 +321,7 @@ public class Hdfs extends AbstractFileSystem {
   @Override
   public void mkdir(Path dir, FsPermission permission, boolean createParent)
     throws IOException, UnresolvedLinkException {
-    dfs.mkdirs(getUriPath(dir), permission, createParent);
+    dfs.primitiveMkdir(getUriPath(dir), permission, createParent);
   }
 
   @SuppressWarnings("deprecation")

+ 20 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1956,33 +1956,28 @@ public class DFSClient implements java.io.Closeable {
    */
   public boolean mkdirs(String src, FsPermission permission,
       boolean createParent) throws IOException {
-    checkOpen();
     if (permission == null) {
       permission = FsPermission.getDefault();
     }
     FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + masked);
-    }
-    try {
-      return namenode.mkdirs(src, masked, createParent);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     InvalidPathException.class,
-                                     FileAlreadyExistsException.class,
-                                     FileNotFoundException.class,
-                                     ParentNotDirectoryException.class,
-                                     SafeModeException.class,
-                                     NSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
-    }
+    return primitiveMkdir(src, masked, createParent);
   }
-  
+
   /**
    * Same {{@link #mkdirs(String, FsPermission, boolean)} except
    * that the permissions has already been masked against umask.
    */
   public boolean primitiveMkdir(String src, FsPermission absPermission)
+    throws IOException {
+    return primitiveMkdir(src, absPermission, true);
+  }
+
+  /**
+   * Same {{@link #mkdirs(String, FsPermission, boolean)} except
+   * that the permissions has already been masked against umask.
+   */
+  public boolean primitiveMkdir(String src, FsPermission absPermission, 
+    boolean createParent)
     throws IOException {
     checkOpen();
     if (absPermission == null) {
@@ -1994,15 +1989,20 @@ public class DFSClient implements java.io.Closeable {
       LOG.debug(src + ": masked=" + absPermission);
     }
     try {
-      return namenode.mkdirs(src, absPermission, true);
+      return namenode.mkdirs(src, absPermission, createParent);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
+                                     InvalidPathException.class,
+                                     FileAlreadyExistsException.class,
+                                     FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
+                                     SafeModeException.class,
                                      NSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class);
     }
   }
-
+  
   /**
    * Get {@link ContentSummary} rooted at the specified directory.
    * @param path The string representation of the path
@@ -2074,10 +2074,7 @@ public class DFSClient implements java.io.Closeable {
   }
   
   boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr) {
-    if (shortCircuitLocalReads && isLocalAddress(targetAddr)) {
-      return true;
-    }
-    return false;
+    return shortCircuitLocalReads && isLocalAddress(targetAddr);
   }
 
   void reportChecksumFailure(String file, ExtendedBlock blk, DatanodeInfo dn) {

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -329,6 +329,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
                                            "dfs.image.transfer.bandwidthPerSec";
   public static final long DFS_IMAGE_TRANSFER_RATE_DEFAULT = 0;  //no throttling
 
+  // Image transfer timeout
+  public static final String DFS_IMAGE_TRANSFER_TIMEOUT_KEY = "dfs.image.transfer.timeout";
+  public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 60 * 1000;
+
   //Keys with no defaults
   public static final String  DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";
   public static final String  DFS_DATANODE_FSDATASET_FACTORY_KEY = "dfs.datanode.fsdataset.factory";

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -243,6 +243,10 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
   }
 
+  private synchronized boolean blockUnderConstruction() {
+    return locatedBlocks.isUnderConstruction();
+  }
+
   /**
    * Returns the datanode from which the stream is currently reading.
    */
@@ -878,7 +882,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
                                        String clientName)
       throws IOException {
     
-    if (dfsClient.shouldTryShortCircuitRead(dnAddr)) {
+    // Can't local read a block under construction, see HDFS-2757
+    if (dfsClient.shouldTryShortCircuitRead(dnAddr) &&
+        !blockUnderConstruction()) {
       return DFSClient.getLocalBlockReader(dfsClient.conf, src, block,
           blockToken, chosenNode, dfsClient.hdfsTimeout, startOffset,
           dfsClient.connectToDnViaHostname());

+ 26 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -51,6 +51,8 @@ import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Time;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Scans the block files under a block pool and verifies that the
  * files are not corrupt.
@@ -255,6 +257,11 @@ class BlockPoolSliceScanner {
     }
   }
 
+  @VisibleForTesting
+  long getTotalScans() {
+    return totalScans;
+  }
+
   /** @return the last scan time for the block pool. */
   long getLastScanTime() {
     return lastScanTime.get();
@@ -563,7 +570,24 @@ class BlockPoolSliceScanner {
     currentPeriodStart = Time.now();
   }
   
+  private synchronized boolean workRemainingInCurrentPeriod() {
+    if (bytesLeft <= 0 && Time.now() < currentPeriodStart + scanPeriod) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping scan since bytesLeft=" + bytesLeft + ", Start=" +
+                  currentPeriodStart + ", period=" + scanPeriod + ", now=" +
+                  Time.now() + " " + blockPoolId);
+      }
+      return false;
+    } else {
+      return true;
+    }
+  }
+
   void scanBlockPoolSlice() {
+    if (!workRemainingInCurrentPeriod()) {
+      return;
+    }
+
     // Create a new processedBlocks structure
     processedBlocks = new HashMap<Long, Integer>();
     if (!assignInitialVerificationTimes()) {
@@ -608,14 +632,14 @@ class BlockPoolSliceScanner {
       LOG.warn("RuntimeException during BlockPoolScanner.scan()", e);
       throw e;
     } finally {
-      cleanUp();
+      rollVerificationLogs();
       if (LOG.isDebugEnabled()) {
         LOG.debug("Done scanning block pool: " + blockPoolId);
       }
     }
   }
   
-  private synchronized void cleanUp() {
+  private synchronized void rollVerificationLogs() {
     if (verificationLog != null) {
       try {
         verificationLog.logs.roll();

+ 18 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -34,6 +34,8 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * DataBlockScanner manages block scanning for all the block pools. For each
  * block pool a {@link BlockPoolSliceScanner} is created which runs in a separate
@@ -47,6 +49,8 @@ public class DataBlockScanner implements Runnable {
   private final FsDatasetSpi<? extends FsVolumeSpi> dataset;
   private final Configuration conf;
   
+  static final int SLEEP_PERIOD_MS = 5 * 1000;
+
   /**
    * Map to find the BlockPoolScanner for a given block pool id. This is updated
    * when a BPOfferService becomes alive or dies.
@@ -68,10 +72,10 @@ public class DataBlockScanner implements Runnable {
     String currentBpId = "";
     boolean firstRun = true;
     while (datanode.shouldRun && !Thread.interrupted()) {
-      //Sleep everytime except in the first interation.
+      //Sleep everytime except in the first iteration.
       if (!firstRun) {
         try {
-          Thread.sleep(5000);
+          Thread.sleep(SLEEP_PERIOD_MS);
         } catch (InterruptedException ex) {
           // Interrupt itself again to set the interrupt status
           blockScannerThread.interrupt();
@@ -103,7 +107,7 @@ public class DataBlockScanner implements Runnable {
     while ((getBlockPoolSetSize() < datanode.getAllBpOs().length)
         || (getBlockPoolSetSize() < 1)) {
       try {
-        Thread.sleep(5000);
+        Thread.sleep(SLEEP_PERIOD_MS);
       } catch (InterruptedException e) {
         blockScannerThread.interrupt();
         return;
@@ -249,7 +253,7 @@ public class DataBlockScanner implements Runnable {
     LOG.info("Removed bpid="+blockPoolId+" from blockPoolScannerMap");
   }
   
-  // This method is used for testing
+  @VisibleForTesting
   long getBlocksScannedInLastRun(String bpid) throws IOException {
     BlockPoolSliceScanner bpScanner = getBPScanner(bpid);
     if (bpScanner == null) {
@@ -259,6 +263,16 @@ public class DataBlockScanner implements Runnable {
     }
   }
 
+  @VisibleForTesting
+  long getTotalScans(String bpid) throws IOException {
+    BlockPoolSliceScanner bpScanner = getBPScanner(bpid);
+    if (bpScanner == null) {
+      throw new IOException("Block Pool: "+bpid+" is not running");
+    } else {
+      return bpScanner.getTotalScans();
+    }
+  }
+
   public void start() {
     blockScannerThread = new Thread(this);
     blockScannerThread.setDaemon(true);

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -32,9 +32,12 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -43,6 +46,7 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.MD5Hash;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 
 
@@ -54,6 +58,8 @@ public class TransferFsImage {
   
   public final static String CONTENT_LENGTH = "Content-Length";
   public final static String MD5_HEADER = "X-MD5-Digest";
+  @VisibleForTesting
+  static int timeout = 0;
 
   private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
   
@@ -222,6 +228,18 @@ public class TransferFsImage {
     HttpURLConnection connection = (HttpURLConnection)
       SecurityUtil.openSecureHttpConnection(url);
 
+    if (timeout <= 0) {
+      // Set the ping interval as timeout
+      Configuration conf = new HdfsConfiguration();
+      timeout = conf.getInt(DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_KEY,
+          DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT);
+    }
+
+    if (timeout > 0) {
+      connection.setConnectTimeout(timeout);
+      connection.setReadTimeout(timeout);
+    }
+
     if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
       throw new HttpGetFailedException(
           "Image transfer servlet at " + url +

+ 251 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java

@@ -0,0 +1,251 @@
+/**
+ * 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.URISyntaxException;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import static org.apache.hadoop.fs.FileContextTestHelper.*;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFcHdfsSetUMask {
+  
+  private static MiniDFSCluster cluster;
+  private static Path defaultWorkingDirectory;
+  private static FileContext fc;
+
+  // rwxrwx---
+  private static final FsPermission USER_GROUP_OPEN_PERMISSIONS = FsPermission
+      .createImmutable((short) 0770);
+
+  private static final FsPermission USER_GROUP_OPEN_FILE_PERMISSIONS = 
+      FsPermission.createImmutable((short) 0660);
+
+  private static final FsPermission USER_GROUP_OPEN_TEST_UMASK = FsPermission
+      .createImmutable((short) (0770 ^ 0777));
+
+  // ---------
+  private static final FsPermission BLANK_PERMISSIONS = FsPermission
+      .createImmutable((short) 0000);
+
+  // parent directory permissions when creating a directory with blank (000)
+  // permissions - it always add the -wx------ bits to the parent so that
+  // it can create the child
+  private static final FsPermission PARENT_PERMS_FOR_BLANK_PERMISSIONS = 
+      FsPermission.createImmutable((short) 0300);
+
+  private static final FsPermission BLANK_TEST_UMASK = FsPermission
+      .createImmutable((short) (0000 ^ 0777));
+  
+  // rwxrwxrwx
+  private static final FsPermission WIDE_OPEN_PERMISSIONS = FsPermission
+      .createImmutable((short) 0777);
+
+  private static final FsPermission WIDE_OPEN_FILE_PERMISSIONS = 
+      FsPermission.createImmutable((short) 0666);
+
+  private static final FsPermission WIDE_OPEN_TEST_UMASK = FsPermission
+      .createImmutable((short) (0777 ^ 0777));
+  
+  @BeforeClass
+  public static void clusterSetupAtBegining()
+        throws IOException, LoginException, URISyntaxException  {
+    Configuration conf = new HdfsConfiguration();
+    // set permissions very restrictive
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY,  "077");
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    fc = FileContext.getFileContext(cluster.getURI(0), conf);
+    defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
+        UserGroupInformation.getCurrentUser().getShortUserName()));
+    fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
+  }
+
+  @AfterClass
+  public static void ClusterShutdownAtEnd() throws Exception {
+    cluster.shutdown();   
+  }
+  
+  {
+    try {
+      ((Log4JLogger)FileSystem.LOG).getLogger().setLevel(Level.DEBUG);
+    }
+    catch(Exception e) {
+      System.out.println("Cannot change log level\n"
+          + StringUtils.stringifyException(e));
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    fc.setUMask(WIDE_OPEN_TEST_UMASK);
+    fc.mkdir(getTestRootPath(fc), FileContext.DEFAULT_PERM, true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fc.delete(getTestRootPath(fc), true);
+  }
+
+  @Test
+  public void testMkdirWithExistingDirClear() throws IOException {
+    testMkdirWithExistingDir(BLANK_TEST_UMASK, BLANK_PERMISSIONS);
+  }
+
+  @Test
+  public void testMkdirWithExistingDirOpen() throws IOException {
+    testMkdirWithExistingDir(WIDE_OPEN_TEST_UMASK, WIDE_OPEN_PERMISSIONS);
+  }
+
+  @Test
+  public void testMkdirWithExistingDirMiddle() throws IOException {
+    testMkdirWithExistingDir(USER_GROUP_OPEN_TEST_UMASK,
+        USER_GROUP_OPEN_PERMISSIONS);
+  }
+  
+  @Test
+  public void testMkdirRecursiveWithNonExistingDirClear() throws IOException {
+    // by default parent directories have -wx------ bits set
+    testMkdirRecursiveWithNonExistingDir(BLANK_TEST_UMASK, BLANK_PERMISSIONS, 
+        PARENT_PERMS_FOR_BLANK_PERMISSIONS);
+  }
+
+  @Test
+  public void testMkdirRecursiveWithNonExistingDirOpen() throws IOException {
+    testMkdirRecursiveWithNonExistingDir(WIDE_OPEN_TEST_UMASK, 
+        WIDE_OPEN_PERMISSIONS, WIDE_OPEN_PERMISSIONS);
+  }
+
+  @Test
+  public void testMkdirRecursiveWithNonExistingDirMiddle() throws IOException {
+    testMkdirRecursiveWithNonExistingDir(USER_GROUP_OPEN_TEST_UMASK,
+        USER_GROUP_OPEN_PERMISSIONS, USER_GROUP_OPEN_PERMISSIONS);
+  }
+
+
+  @Test
+  public void testCreateRecursiveWithExistingDirClear() throws IOException {
+    testCreateRecursiveWithExistingDir(BLANK_TEST_UMASK, BLANK_PERMISSIONS);
+  }
+
+  @Test
+  public void testCreateRecursiveWithExistingDirOpen() throws IOException {
+    testCreateRecursiveWithExistingDir(WIDE_OPEN_TEST_UMASK,
+        WIDE_OPEN_FILE_PERMISSIONS);
+  }
+
+  @Test
+  public void testCreateRecursiveWithExistingDirMiddle() throws IOException {
+    testCreateRecursiveWithExistingDir(USER_GROUP_OPEN_TEST_UMASK,
+        USER_GROUP_OPEN_FILE_PERMISSIONS);
+  }
+
+
+  @Test
+  public void testCreateRecursiveWithNonExistingDirClear() throws IOException {
+    // directory permission inherited from parent so this must match the @Before
+    // set of umask
+    testCreateRecursiveWithNonExistingDir(BLANK_TEST_UMASK,
+        WIDE_OPEN_PERMISSIONS, BLANK_PERMISSIONS);
+  }
+
+  @Test
+  public void testCreateRecursiveWithNonExistingDirOpen() throws IOException {
+    // directory permission inherited from parent so this must match the @Before
+    // set of umask
+    testCreateRecursiveWithNonExistingDir(WIDE_OPEN_TEST_UMASK,
+        WIDE_OPEN_PERMISSIONS, WIDE_OPEN_FILE_PERMISSIONS);
+  }
+
+  @Test
+  public void testCreateRecursiveWithNonExistingDirMiddle() throws IOException {
+    // directory permission inherited from parent so this must match the @Before
+    // set of umask
+    testCreateRecursiveWithNonExistingDir(USER_GROUP_OPEN_TEST_UMASK, 
+        WIDE_OPEN_PERMISSIONS, USER_GROUP_OPEN_FILE_PERMISSIONS);
+  }
+
+
+  public void testMkdirWithExistingDir(FsPermission umask, 
+      FsPermission expectedPerms) throws IOException {
+    Path f = getTestRootPath(fc, "aDir");
+    fc.setUMask(umask);
+    fc.mkdir(f, FileContext.DEFAULT_PERM, true);
+    Assert.assertTrue(isDir(fc, f));
+    Assert.assertEquals("permissions on directory are wrong",  
+        expectedPerms, fc.getFileStatus(f).getPermission());
+  }
+  
+  public void testMkdirRecursiveWithNonExistingDir(FsPermission umask,
+      FsPermission expectedPerms, FsPermission expectedParentPerms) 
+      throws IOException {
+    Path f = getTestRootPath(fc, "NonExistant2/aDir");
+    fc.setUMask(umask);
+    fc.mkdir(f, FileContext.DEFAULT_PERM, true);
+    Assert.assertTrue(isDir(fc, f));
+    Assert.assertEquals("permissions on directory are wrong",  
+        expectedPerms, fc.getFileStatus(f).getPermission());
+    Path fParent = getTestRootPath(fc, "NonExistant2");
+    Assert.assertEquals("permissions on parent directory are wrong",  
+        expectedParentPerms, fc.getFileStatus(fParent).getPermission());
+  }
+
+
+  public void testCreateRecursiveWithExistingDir(FsPermission umask,
+      FsPermission expectedPerms) throws IOException {
+    Path f = getTestRootPath(fc,"foo");
+    fc.setUMask(umask);
+    createFile(fc, f);
+    Assert.assertTrue(isFile(fc, f));
+    Assert.assertEquals("permissions on file are wrong",  
+        expectedPerms , fc.getFileStatus(f).getPermission());
+  }
+  
+  
+  public void testCreateRecursiveWithNonExistingDir(FsPermission umask,
+      FsPermission expectedDirPerms, FsPermission expectedFilePerms) 
+      throws IOException {
+    Path f = getTestRootPath(fc,"NonExisting/foo");
+    Path fParent = getTestRootPath(fc, "NonExisting");
+    Assert.assertFalse(exists(fc, fParent));
+    fc.setUMask(umask);
+    createFile(fc, f);
+    Assert.assertTrue(isFile(fc, f));
+    Assert.assertEquals("permissions on file are wrong",  
+        expectedFilePerms, fc.getFileStatus(f).getPermission());
+    Assert.assertEquals("permissions on parent directory are wrong",  
+        expectedDirPerms, fc.getFileStatus(fParent).getPermission());
+  }
+ 
+}

+ 93 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -57,6 +57,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+
 /**
  * This class tests commands from DFSShell.
  */
@@ -1480,4 +1482,95 @@ public class TestDFSShell {
 
   }
 
+  /**
+   * Delete a file optionally configuring trash on the server and client.
+   */
+  private void deleteFileUsingTrash(
+      boolean serverTrash, boolean clientTrash) throws Exception {
+    // Run a cluster, optionally with trash enabled on the server
+    Configuration serverConf = new HdfsConfiguration();
+    if (serverTrash) {
+      serverConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
+    }
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(serverConf)
+      .numDataNodes(1).format(true).build();
+    Configuration clientConf = new Configuration(serverConf);
+
+    // Create a client, optionally with trash enabled
+    if (clientTrash) {
+      clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
+    } else {
+      clientConf.setLong(FS_TRASH_INTERVAL_KEY, 0);
+    }
+
+    FsShell shell = new FsShell(clientConf);
+    FileSystem fs = null;
+
+    try {
+      // Create and delete a file
+      fs = cluster.getFileSystem();
+      writeFile(fs, new Path(TEST_ROOT_DIR, "foo"));
+      final String testFile = TEST_ROOT_DIR + "/foo";
+      final String trashFile = shell.getCurrentTrashDir() + "/" + testFile;
+      String[] argv = new String[] { "-rm", testFile };
+      int res = ToolRunner.run(shell, argv);
+      assertEquals("rm failed", 0, res);
+
+      if (serverTrash) {
+        // If the server config was set we should use it unconditionally
+        assertTrue("File not in trash", fs.exists(new Path(trashFile)));
+      } else if (clientTrash) {
+        // If the server config was not set but the client config was
+        // set then we should use it
+        assertTrue("File not in trashed", fs.exists(new Path(trashFile)));
+      } else {
+        // If neither was set then we should not have trashed the file
+        assertFalse("File was not removed", fs.exists(new Path(testFile)));
+        assertFalse("File was trashed", fs.exists(new Path(trashFile)));
+      }
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test that the server trash configuration is respected when
+   * the client configuration is not set.
+   */
+  @Test
+  public void testServerConfigRespected() throws Exception {
+    deleteFileUsingTrash(true, false);
+  }
+
+  /**
+   * Test that server trash configuration is respected even when the
+   * client configuration is set.
+   */
+  @Test
+  public void testServerConfigRespectedWithClient() throws Exception {
+    deleteFileUsingTrash(true, true);
+  }
+
+  /**
+   * Test that the client trash configuration is respected when
+   * the server configuration is not set.
+   */
+  @Test
+  public void testClientConfigRespected() throws Exception {
+    deleteFileUsingTrash(false, true);
+  }
+
+  /**
+   * Test that trash is disabled by default.
+   */
+  @Test
+  public void testNoTrashConfig() throws Exception {
+    deleteFileUsingTrash(false, false);
+  }
 }

+ 0 - 54
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java

@@ -23,11 +23,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.TestTrash;
-import org.apache.hadoop.fs.Trash;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -62,53 +57,4 @@ public class TestHDFSTrash {
     conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, fs.getUri().toString());
     TestTrash.trashNonDefaultFS(conf);
   }
-
-  /** Clients should always use trash if enabled server side */
-  @Test
-  public void testTrashEnabledServerSide() throws IOException {
-    Configuration serverConf = new HdfsConfiguration();
-    Configuration clientConf = new Configuration();
-
-    // Enable trash on the server and client
-    serverConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
-    clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
-
-    MiniDFSCluster cluster2 = null;
-    try {
-      cluster2 = new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
-      FileSystem fs = cluster2.getFileSystem();
-      assertTrue(new Trash(fs, clientConf).isEnabled());
-
-      // Disabling trash on the client is ignored
-      clientConf.setLong(FS_TRASH_INTERVAL_KEY, 0);
-      assertTrue(new Trash(fs, clientConf).isEnabled());
-    } finally {
-      if (cluster2 != null) cluster2.shutdown();
-    }
-  }
-
-  /** Clients should always use trash if enabled client side */
-  @Test
-  public void testTrashEnabledClientSide() throws IOException {
-    Configuration serverConf = new HdfsConfiguration();
-    Configuration clientConf = new Configuration();
-    
-    // Disable server side
-    serverConf.setLong(FS_TRASH_INTERVAL_KEY, 0);
-
-    MiniDFSCluster cluster2 = null;
-    try {
-      cluster2 = new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
-
-      // Client side is disabled by default
-      FileSystem fs = cluster2.getFileSystem();
-      assertFalse(new Trash(fs, clientConf).isEnabled());
-
-      // Enabling on the client works even though its disabled on the server
-      clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
-      assertTrue(new Trash(fs, clientConf).isEnabled());
-    } finally {
-      if (cluster2 != null) cluster2.shutdown();
-    }
-  }
 }

+ 80 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMultipleNNDataBlockScanner.java

@@ -20,8 +20,11 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
 
+import junit.framework.Assert;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,7 +34,13 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceScanner;
+import static org.apache.hadoop.hdfs.server.datanode.DataBlockScanner.SLEEP_PERIOD_MS;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.junit.Test;
+import org.junit.Ignore;
+import static org.junit.Assert.fail;
 
 
 public class TestMultipleNNDataBlockScanner {
@@ -166,4 +175,75 @@ public class TestMultipleNNDataBlockScanner {
       cluster.shutdown();
     }
   }
+  
+  @Test
+  public void test2NNBlockRescanInterval() throws IOException {
+    ((Log4JLogger)BlockPoolSliceScanner.LOG).getLogger().setLevel(Level.ALL);
+    Configuration conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
+        .build();
+
+    try {
+      FileSystem fs = cluster.getFileSystem(1);
+      Path file2 = new Path("/test/testBlockScanInterval");
+      DFSTestUtil.createFile(fs, file2, 30, (short) 1, 0);
+
+      fs = cluster.getFileSystem(0);
+      Path file1 = new Path("/test/testBlockScanInterval");
+      DFSTestUtil.createFile(fs, file1, 30, (short) 1, 0);
+      for (int i = 0; i < 8; i++) {
+        LOG.info("Verifying that the blockscanner scans exactly once");
+        waitAndScanBlocks(1, 1);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * HDFS-3828: DN rescans blocks too frequently
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testBlockRescanInterval() throws IOException {
+    ((Log4JLogger)BlockPoolSliceScanner.LOG).getLogger().setLevel(Level.ALL);
+    Configuration conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      Path file1 = new Path("/test/testBlockScanInterval");
+      DFSTestUtil.createFile(fs, file1, 30, (short) 1, 0);
+      for (int i = 0; i < 4; i++) {
+        LOG.info("Verifying that the blockscanner scans exactly once");
+        waitAndScanBlocks(1, 1);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  void waitAndScanBlocks(long scansLastRun, long scansTotal)
+      throws IOException {
+    // DataBlockScanner will run for every 5 seconds so we are checking for
+    // every 5 seconds
+    int n = 5;
+    String bpid = cluster.getNamesystem(0).getBlockPoolId();
+    DataNode dn = cluster.getDataNodes().get(0);
+    long blocksScanned, total;
+    do {
+      try {
+        Thread.sleep(SLEEP_PERIOD_MS);
+      } catch (InterruptedException e) {
+        fail("Interrupted: " + e);
+      }
+      blocksScanned = dn.blockScanner.getBlocksScannedInLastRun(bpid);
+      total = dn.blockScanner.getTotalScans(bpid);
+      LOG.info("bpid = " + bpid + " blocksScanned = " + blocksScanned + " total=" + total);
+    } while (n-- > 0 && (blocksScanned != scansLastRun || scansTotal != total));
+    Assert.assertEquals(scansTotal, total);
+    Assert.assertEquals(scansLastRun, blocksScanned);
+  }
 }

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java

@@ -17,17 +17,27 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.net.URL;
 import java.util.Collections;
 import java.util.List;
 
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
@@ -100,4 +110,45 @@ public class TestTransferFsImage {
       cluster.shutdown();      
     }
   }
+
+  /**
+   * Test to verify the read timeout
+   */
+  @Test(timeout = 5000)
+  public void testImageTransferTimeout() throws Exception {
+    HttpServer testServer = HttpServerFunctionalTest.createServer("hdfs");
+    try {
+      testServer.addServlet("GetImage", "/getimage", TestGetImageServlet.class);
+      testServer.start();
+      URL serverURL = HttpServerFunctionalTest.getServerURL(testServer);
+      TransferFsImage.timeout = 2000;
+      try {
+        TransferFsImage.getFileClient(serverURL.getAuthority(), "txid=1", null,
+            null, false);
+        fail("TransferImage Should fail with timeout");
+      } catch (SocketTimeoutException e) {
+        assertEquals("Read should timeout", "Read timed out", e.getMessage());
+      }
+    } finally {
+      if (testServer != null) {
+        testServer.stop();
+      }
+    }
+  }
+
+  public static class TestGetImageServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      synchronized (this) {
+        try {
+          wait(5000);
+        } catch (InterruptedException e) {
+          // Ignore
+        }
+      }
+    }
+  }
 }

+ 73 - 60
hadoop-mapreduce-project/CHANGES.txt

@@ -1,6 +1,6 @@
 Hadoop MapReduce Change Log
 
-Trunk (unreleased changes)
+Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
 
@@ -126,79 +126,29 @@ Trunk (unreleased changes)
 
     MAPREDUCE-3868. Make Raid Compile. (Weiyan Wang via schen)
 
-Branch-2 ( Unreleased changes )
+Release 2.0.3-alpha - Unreleased 
 
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
 
-    MAPREDUCE-987. Exposing MiniDFS and MiniMR clusters as a single process 
-    command-line. (ahmed via tucu)
-
-    MAPREDUCE-4417. add support for encrypted shuffle (tucu)
-
   IMPROVEMENTS
 
-    MAPREDUCE-4157. ResourceManager should not kill apps that are well behaved
-    (Jason Lowe via bobby)
-
-    MAPREDUCE-4511. Add IFile readahead (ahmed via tucu)
-
-    MAPREDUCE-4408. allow jobs to set a JAR that is in the distributed cached 
-    (rkanter via tucu)
-
-    MAPREDUCE-2786. Add compression option for TestDFSIO.
-    (Plamen Jeliazkov via shv)
+  OPTIMIZATIONS
 
   BUG FIXES
 
-    MAPREDUCE-4422. YARN_APPLICATION_CLASSPATH needs a documented default value in 
-    YarnConfiguration. (ahmed via tucu)
-
-    MAPREDUCE-4406. Users should be able to specify the MiniCluster ResourceManager 
-    and JobHistoryServer ports. (ahmed via tucu)
-
-    MAPREDUCE-4407. Add hadoop-yarn-server-tests-<version>-tests.jar to hadoop dist
-    package. (ahmed via tucu)
-
-    MAPREDUCE-4465. Update description of yarn.nodemanager.address property. 
-    (bowang via tucu)
-
-    MAPREDUCE-4342. Distributed Cache gives inconsistent result if cache files 
-    get deleted from tasktracker. (mayank_bansal via tucu)
-
-    MAPREDUCE-4498. Remove hsqldb jar from Hadoop runtime classpath. (rkanter via tucu)
-
-    MAPREDUCE-4494. TestFifoScheduler failing with Metrics source QueueMetrics,q0=default 
-    already exists!. (ahmed.radwan via tucu)
-
-    MAPREDUCE-4484. Incorrect IS_MINI_YARN_CLUSTER property name in YarnConfiguration. 
-    (ahmed.radwan via tucu)
-
-    MAPREDUCE-4562. Support for "FileSystemCounter" legacy counter group name
-    for compatibility reasons is creating incorrect counter name.
-    (Jarek Jarcec Cecho via tomwhite)
-
-    MAPREDUCE-4068. Jars in lib subdirectory of the submittable JAR are not added to the 
-    classpath (rkanter via tucu)
-
-    MAPREDUCE-4577. HDFS-3672 broke
-    TestCombineFileInputFormat.testMissingBlocks() test. (atm)
-
-    MAPREDUCE-4470. Fix TestCombineFileInputFormat.testForEmptyFile (ikatsov via tucu)
-
-    MAPREDUCE-4608. hadoop-mapreduce-client is missing some dependencies.
-    (tucu via tomwhite)
-
-    MAPREDUCE-4610. Support deprecated mapreduce.job.counters.limit property in
-    MR2. (tomwhite)
-
-Release 2.1.0-alpha - Unreleased 
+Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
 
+    MAPREDUCE-987. Exposing MiniDFS and MiniMR clusters as a single process 
+    command-line. (ahmed via tucu)
+
+    MAPREDUCE-4417. add support for encrypted shuffle (tucu)
+
     MAPREDUCE-4355. Add RunningJob.getJobStatus() (kkambatl via tucu)
 
     MAPREDUCE-3451. Port Fair Scheduler to MR2 (pwendell via tucu)
@@ -208,6 +158,17 @@ Release 2.1.0-alpha - Unreleased
 
   IMPROVEMENTS
 
+    MAPREDUCE-4157. ResourceManager should not kill apps that are well behaved
+    (Jason Lowe via bobby)
+
+    MAPREDUCE-4511. Add IFile readahead (ahmed via tucu)
+
+    MAPREDUCE-4408. allow jobs to set a JAR that is in the distributed cached 
+    (rkanter via tucu)
+
+    MAPREDUCE-2786. Add compression option for TestDFSIO.
+    (Plamen Jeliazkov via shv)
+
     MAPREDUCE-4440. Changed SchedulerApp and SchedulerNode to be a minimal
     interface to allow schedulers to maintain their own. (acmurthy) 
 
@@ -249,10 +210,56 @@ Release 2.1.0-alpha - Unreleased
     MAPREDUCE-4579. Split TestTaskAttempt into two so as to pass tests on
     jdk7. (Thomas Graves via vinodkv)
 
-  OPTIMIZATIONS
+    MAPREDUCE-4638. MR AM supplies MapReduce jars in classpath rather than
+    rely on YARN. (acmurthy) 
 
   BUG FIXES
 
+    MAPREDUCE-4422. YARN_APPLICATION_CLASSPATH needs a documented default value in 
+    YarnConfiguration. (ahmed via tucu)
+
+    MAPREDUCE-4406. Users should be able to specify the MiniCluster ResourceManager 
+    and JobHistoryServer ports. (ahmed via tucu)
+
+    MAPREDUCE-4407. Add hadoop-yarn-server-tests-<version>-tests.jar to hadoop dist
+    package. (ahmed via tucu)
+
+    MAPREDUCE-4465. Update description of yarn.nodemanager.address property. 
+    (bowang via tucu)
+
+    MAPREDUCE-4342. Distributed Cache gives inconsistent result if cache files 
+    get deleted from tasktracker. (mayank_bansal via tucu)
+
+    MAPREDUCE-4498. Remove hsqldb jar from Hadoop runtime classpath. (rkanter via tucu)
+
+    MAPREDUCE-4494. TestFifoScheduler failing with Metrics source QueueMetrics,q0=default 
+    already exists!. (ahmed.radwan via tucu)
+
+    MAPREDUCE-4484. Incorrect IS_MINI_YARN_CLUSTER property name in YarnConfiguration. 
+    (ahmed.radwan via tucu)
+
+    MAPREDUCE-4562. Support for "FileSystemCounter" legacy counter group name
+    for compatibility reasons is creating incorrect counter name.
+    (Jarek Jarcec Cecho via tomwhite)
+
+    MAPREDUCE-4068. Jars in lib subdirectory of the submittable JAR are not added to the 
+    classpath (rkanter via tucu)
+
+    MAPREDUCE-4577. HDFS-3672 broke
+    TestCombineFileInputFormat.testMissingBlocks() test. (atm)
+
+    MAPREDUCE-4470. Fix TestCombineFileInputFormat.testForEmptyFile (ikatsov via tucu)
+
+    MAPREDUCE-4608. hadoop-mapreduce-client is missing some dependencies.
+    (tucu via tomwhite)
+
+    MAPREDUCE-4610. Support deprecated mapreduce.job.counters.limit property in
+    MR2. (tomwhite)
+
+    MAPREDUCE-4629. Remove JobHistory.DEBUG_MODE (Karthik Kambatla via bobby)
+
+    MAPREDUCE-4642. MiniMRClientClusterFactory should not use job.setJar() (rkanter via tucu)
+
     MAPREDUCE-4148. MapReduce should not have a compile-time dependency on
     HDFS. (tomwhite)
 
@@ -880,6 +887,12 @@ Release 0.23.3 - UNRELEASED
     mapreduce.reduce.maxattempts defaults are set to 4 as well as
     mapreduce.job.maxtaskfailures.per.tracker. (Ravi Prakash via jeagles)
 
+    MAPREDUCE-4633. history server doesn't set permissions on all subdirs
+    (tgraves via bobby)
+
+    MAPREDUCE-4641. Exception in commitJob marks job as successful in job
+    history (Jason Lowe via bobby)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -759,7 +759,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
         job.getCommitter().commitJob(job.getJobContext());
       } catch (IOException e) {
         LOG.error("Could not do commit for Job", e);
-        job.logJobHistoryFinishedEvent();
+        job.addDiagnostic("Job commit failed: " + e.getMessage());
+        job.abortJob(org.apache.hadoop.mapreduce.JobStatus.State.FAILED);
         return job.finished(JobState.FAILED);
       }
       job.logJobHistoryFinishedEvent();
@@ -1199,7 +1200,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     }
   }
 
-  private void abortJob(
+  protected void abortJob(
       org.apache.hadoop.mapreduce.JobStatus.State finalState) {
     try {
       committer.abortJob(jobContext, finalState);
@@ -1501,7 +1502,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     }
   }
 
-  private void addDiagnostic(String diag) {
+  protected void addDiagnostic(String diag) {
     diagnostics.add(diag);
   }
   

+ 7 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -19,9 +19,11 @@
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
 import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
@@ -91,8 +93,6 @@ public class TestJobImpl {
     when(mockJob.getCommitter()).thenReturn(mockCommitter);
     when(mockJob.getEventHandler()).thenReturn(mockEventHandler);
     when(mockJob.getJobContext()).thenReturn(mockJobContext);
-    doNothing().when(mockJob).setFinishTime();
-    doNothing().when(mockJob).logJobHistoryFinishedEvent();
     when(mockJob.finished(JobState.KILLED)).thenReturn(JobState.KILLED);
     when(mockJob.finished(JobState.FAILED)).thenReturn(JobState.FAILED);
     when(mockJob.finished(JobState.SUCCEEDED)).thenReturn(JobState.SUCCEEDED);
@@ -103,11 +103,13 @@ public class TestJobImpl {
       // commitJob stubbed out, so this can't happen
     }
     doNothing().when(mockEventHandler).handle(any(JobHistoryEvent.class));
+    JobState jobState = JobImpl.checkJobCompleteSuccess(mockJob);
     Assert.assertNotNull("checkJobCompleteSuccess incorrectly returns null " +
-      "for successful job",
-      JobImpl.checkJobCompleteSuccess(mockJob));
+      "for successful job", jobState);
     Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
-        JobState.FAILED, JobImpl.checkJobCompleteSuccess(mockJob));
+        JobState.FAILED, jobState);
+    verify(mockJob).abortJob(
+        eq(org.apache.hadoop.mapreduce.JobStatus.State.FAILED));
   }
 
   @Test

+ 0 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java

@@ -61,11 +61,6 @@ public class JHAdminConfig {
     MR_HISTORY_PREFIX + "datestring.cache.size";
   public static final int DEFAULT_MR_HISTORY_DATESTRING_CACHE_SIZE = 200000;
   
-  //TODO REMOVE debug-mode
-  /** Equivalent to 0.20 mapreduce.jobhistory.debug.mode */
-  public static final String MR_HISTORY_DEBUG_MODE = 
-    MR_HISTORY_PREFIX + "debug-mode";
-  
   /** Path where history files should be stored for DONE jobs. **/
   public static final String MR_HISTORY_DONE_DIR =
     MR_HISTORY_PREFIX + "done-dir";

+ 15 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java

@@ -79,6 +79,13 @@ public class JobHistoryUtils {
 
   public static final FsPermission HISTORY_DONE_FILE_PERMISSION =
     FsPermission.createImmutable((short) 0770); // rwx------
+
+ /**
+   * Umask for the done dir and derivatives.
+   */
+  public static final FsPermission HISTORY_DONE_DIR_UMASK = FsPermission
+      .createImmutable((short) (0770 ^ 0777));
+
   
   /**
    * Permissions for the intermediate done directory.
@@ -336,20 +343,19 @@ public class JobHistoryUtils {
   /**
    * Gets the timestamp component based on millisecond time.
    * @param millisecondTime
-   * @param debugMode
    * @return the timestamp component based on millisecond time
    */
-  public static String timestampDirectoryComponent(long millisecondTime, boolean debugMode) {
+  public static String timestampDirectoryComponent(long millisecondTime) {
     Calendar timestamp = Calendar.getInstance();
     timestamp.setTimeInMillis(millisecondTime);
     String dateString = null;
-    dateString = String.format(
-        TIMESTAMP_DIR_FORMAT,
-        timestamp.get(Calendar.YEAR),
-        // months are 0-based in Calendar, but people will expect January
-        // to be month #1.
-        timestamp.get(debugMode ? Calendar.HOUR : Calendar.MONTH) + 1,
-        timestamp.get(debugMode ? Calendar.MINUTE : Calendar.DAY_OF_MONTH));
+    dateString = String
+        .format(TIMESTAMP_DIR_FORMAT,
+            timestamp.get(Calendar.YEAR),
+            // months are 0-based in Calendar, but people will expect January to
+            // be month #1.
+            timestamp.get(Calendar.MONTH) + 1,
+            timestamp.get(Calendar.DAY_OF_MONTH));
     dateString = dateString.intern();
     return dateString;
   }

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -179,6 +179,12 @@ public class MRApps extends Apps {
         Apps.addToEnvironment(environment, Environment.CLASSPATH.name(), c
             .trim());
       }
+      for (String c : conf.getStrings(
+          MRJobConfig.MAPREDUCE_APPLICATION_CLASSPATH,
+          MRJobConfig.DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH)) {
+        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(), c
+            .trim());
+      }
     } finally {
       if (classpathFileStream != null) {
         classpathFileStream.close();

+ 12 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java

@@ -140,11 +140,19 @@ public class TestMRApps {
     Map<String, String> environment = new HashMap<String, String>();
     MRApps.setClasspath(environment, job.getConfiguration());
     assertTrue(environment.get("CLASSPATH").startsWith("$PWD:"));
-    String confClasspath = job.getConfiguration().get(YarnConfiguration.YARN_APPLICATION_CLASSPATH);
-    if (confClasspath != null) {
-      confClasspath = confClasspath.replaceAll(",\\s*", ":").trim();
+    String yarnAppClasspath = 
+        job.getConfiguration().get(
+            YarnConfiguration.YARN_APPLICATION_CLASSPATH);
+    if (yarnAppClasspath != null) {
+      yarnAppClasspath = yarnAppClasspath.replaceAll(",\\s*", ":").trim();
     }
-    assertTrue(environment.get("CLASSPATH").contains(confClasspath));
+    assertTrue(environment.get("CLASSPATH").contains(yarnAppClasspath));
+    String mrAppClasspath = 
+        job.getConfiguration().get(MRJobConfig.MAPREDUCE_APPLICATION_CLASSPATH);
+    if (mrAppClasspath != null) {
+      mrAppClasspath = mrAppClasspath.replaceAll(",\\s*", ":").trim();
+    }
+    assertTrue(environment.get("CLASSPATH").contains(mrAppClasspath));
   }
 
  @Test public void testSetClasspathWithUserPrecendence() {

+ 14 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -587,4 +587,18 @@ public interface MRJobConfig {
   MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT =
       "security.job.client.protocol.acl";
 
+  /**
+   * CLASSPATH for all YARN MapReduce applications.
+   */
+  public static final String MAPREDUCE_APPLICATION_CLASSPATH = 
+      "mapreduce.application.classpath";
+
+  /**
+   * Default CLASSPATH for all YARN MapReduce applications.
+   */
+  public static final String[] DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH = {
+      "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*",
+      "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*",
+  };
+  
 }

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -1203,6 +1203,8 @@
 
 <!--  end of node health script variables -->
 
+<!-- MR YARN Application properties -->
+
 <property>
  <name>mapreduce.job.counters.limit</name>
   <value>120</value>
@@ -1312,6 +1314,13 @@
   <description>The amount of memory the MR AppMaster needs.</description>
 </property>
 
+<property>
+  <description>CLASSPATH for MR applications. A comma-separated list
+  of CLASSPATH entries</description>
+   <name>mapreduce.application.classpath</name>
+   <value>$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*,$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*</value>
+</property>
+
 <!-- jobhistory properties -->
 
 <property>

+ 4 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java

@@ -358,7 +358,6 @@ public class HistoryFileManager extends AbstractService {
 
   private Configuration conf;
 
-  private boolean debugMode;
   private String serialNumberFormat;
 
   private Path doneDirPrefixPath = null; // folder for completed jobs
@@ -379,8 +378,7 @@ public class HistoryFileManager extends AbstractService {
   public void init(Configuration conf) {
     this.conf = conf;
 
-    debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false);
-    int serialNumberLowDigits = debugMode ? 1 : 3;
+    int serialNumberLowDigits = 3;
     serialNumberFormat = ("%0"
         + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits)
         + "d");
@@ -392,6 +390,7 @@ public class HistoryFileManager extends AbstractService {
       doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
           new Path(doneDirPrefix));
       doneDirFc = FileContext.getFileContext(doneDirPrefixPath.toUri(), conf);
+      doneDirFc.setUMask(JobHistoryUtils.HISTORY_DONE_DIR_UMASK);
       mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(
           JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
     } catch (IOException e) {
@@ -779,8 +778,8 @@ public class HistoryFileManager extends AbstractService {
   }
 
   private Path canonicalHistoryLogPath(JobId id, long millisecondTime) {
-    String timestampComponent = JobHistoryUtils.timestampDirectoryComponent(
-        millisecondTime, debugMode);
+    String timestampComponent = JobHistoryUtils
+        .timestampDirectoryComponent(millisecondTime);
     return new Path(doneDirPrefixPath, JobHistoryUtils.historyLogSubdirectory(
         id, timestampComponent, serialNumberFormat));
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.ProtoUtils;
-import org.hadoop.yarn.client.YarnClientImpl;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
 
 public class ResourceMgrDelegate extends YarnClientImpl {
   private static final Log LOG = LogFactory.getLog(ResourceMgrDelegate.class);

+ 6 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRClientClusterFactory.java

@@ -58,8 +58,12 @@ public class MiniMRClientClusterFactory {
     Job job = Job.getInstance(conf);
 
     job.addFileToClassPath(appJar);
-    String callerJar = JarFinder.getJar(caller);
-    job.setJar(callerJar);
+
+    Path callerJar = new Path(JarFinder.getJar(caller));
+    Path remoteCallerJar = new Path(testRootDir, callerJar.getName());
+    fs.copyFromLocalFile(callerJar, remoteCallerJar);
+    fs.setPermission(remoteCallerJar, new FsPermission("744"));
+    job.addFileToClassPath(remoteCallerJar);
 
     MiniMRYarnCluster miniMRYarnCluster = new MiniMRYarnCluster(caller
         .getName(), noOfNMs);

+ 5 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java

@@ -98,6 +98,11 @@ public class DistCp extends Configured implements Tool {
    * @return On success, it returns 0. Else, -1.
    */
   public int run(String[] argv) {
+    if (argv.length < 1) {
+      OptionsParser.usage();
+      return DistCpConstants.INVALID_ARGUMENT;
+    }
+    
     try {
       inputOptions = (OptionsParser.parse(argv));
 

+ 4 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java

@@ -156,6 +156,10 @@ public class OptionsParser {
       try {
         Integer mapBandwidth = Integer.parseInt(
             getVal(command, DistCpOptionSwitch.BANDWIDTH.getSwitch()).trim());
+        if (mapBandwidth.intValue() <= 0) {
+          throw new IllegalArgumentException("Bandwidth specified is not positive: " +
+              mapBandwidth);
+        }
         option.setMapBandwidth(mapBandwidth);
       } catch (NumberFormatException e) {
         throw new IllegalArgumentException("Bandwidth specified is invalid: " +

+ 1 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java

@@ -255,7 +255,7 @@ public class CopyMapper extends Mapper<Text, FileStatus, Text, Text> {
 
     long bytesCopied;
     try {
-      bytesCopied = (Long)new RetriableFileCopyCommand(description)
+      bytesCopied = (Long)new RetriableFileCopyCommand(skipCrc, description)
                        .execute(sourceFileStatus, target, context, fileAttributes);
     } catch (Exception e) {
       context.setStatus("Copy Failure: " + sourceFileStatus.getPath());

+ 14 - 2
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java

@@ -41,7 +41,8 @@ public class RetriableFileCopyCommand extends RetriableCommand {
 
   private static Log LOG = LogFactory.getLog(RetriableFileCopyCommand.class);
   private static int BUFFER_SIZE = 8 * 1024;
-
+  private boolean skipCrc = false;
+  
   /**
    * Constructor, taking a description of the action.
    * @param description Verbose description of the copy operation.
@@ -49,6 +50,17 @@ public class RetriableFileCopyCommand extends RetriableCommand {
   public RetriableFileCopyCommand(String description) {
     super(description);
   }
+ 
+  /**
+   * Create a RetriableFileCopyCommand.
+   *
+   * @param skipCrc Whether to skip the crc check.
+   * @param description A verbose description of the copy operation.
+   */
+  public RetriableFileCopyCommand(boolean skipCrc, String description) {
+    this(description);
+    this.skipCrc = skipCrc;
+  }
 
   /**
    * Implementation of RetriableCommand::doExecute().
@@ -92,7 +104,7 @@ public class RetriableFileCopyCommand extends RetriableCommand {
 
       compareFileLengths(sourceFileStatus, tmpTargetPath, configuration, bytesRead);
       //At this point, src&dest lengths are same. if length==0, we skip checksum
-      if (bytesRead != 0) { 
+      if ((bytesRead != 0) && (!skipCrc)) {
         compareCheckSums(sourceFS, sourceFileStatus.getPath(), targetFS, tmpTargetPath);
       }
       promoteTmpToTarget(tmpTargetPath, target, targetFS);

+ 18 - 0
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java

@@ -110,6 +110,24 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/target/"});
     Assert.assertEquals(options.getMapBandwidth(), 11);
   }
+  
+  @Test(expected=IllegalArgumentException.class)
+  public void testParseNonPositiveBandwidth() {
+    OptionsParser.parse(new String[] {
+        "-bandwidth",
+        "-11",
+        "hdfs://localhost:8020/source/first",
+        "hdfs://localhost:8020/target/"});
+  }
+  
+  @Test(expected=IllegalArgumentException.class)
+  public void testParseZeroBandwidth() {
+    OptionsParser.parse(new String[] {
+        "-bandwidth",
+        "0",
+        "hdfs://localhost:8020/source/first",
+        "hdfs://localhost:8020/target/"});
+  }
 
   @Test
   public void testParseSkipCRC() {

+ 27 - 7
hadoop-yarn-project/CHANGES.txt

@@ -1,6 +1,6 @@
 Hadoop MapReduce Change Log
 
-Trunk (unreleased changes)
+Trunk - Unreleased 
 
   INCOMPATIBLE CHANGES
 
@@ -10,11 +10,14 @@ Trunk (unreleased changes)
 
     YARN-84. Use Builder to build RPC server. (Brandon Li via suresh)
 
+    YARN-57. Allow process-tree based resource calculation et al. to be
+    pluggable to support it on multiple platforms. (Radim Kolar via acmurthy)
+
   OPTIMAZATIONS
 
   BUG FIXES
 
-Branch-2 ( Unreleased changes )
+Release 2.0.3-alpha - Unreleased 
 
   INCOMPATIBLE CHANGES
 
@@ -22,14 +25,11 @@ Branch-2 ( Unreleased changes )
 
   IMPROVEMENTS
 
-  OPTIMAZATIONS
+  OPTIMIZATIONS
 
   BUG FIXES
 
-    MAPREDUCE-2374. "Text File Busy" errors launching MR tasks. (Andy Isaacson
-    via atm)
-
-Release 2.1.0-alpha - Unreleased 
+Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
 
@@ -44,6 +44,9 @@ Release 2.1.0-alpha - Unreleased
     YARN-10. Fix DistributedShell module to not have a dependency on 
     hadoop-mapreduce-client-core. (Hitesh Shah via vinodkv)
 
+    YARN-80. Add support for delaying rack-local containers in
+    CapacityScheduler. (acmurthy) 
+
   OPTIMAZATIONS
 
   BUG FIXES
@@ -62,6 +65,15 @@ Release 2.1.0-alpha - Unreleased
     YARN-42. Modify NM's non-aggregating logs' handler to stop properly so that
     NMs don't get NPEs on startup errors. (Devaraj K via vinodkv)
 
+    YARN-15. Updated default classpath for YARN applications to reflect split of
+    YARN into a sub-project. (Arun C Murthy via vinodkv)
+
+    YARN-75. Modified ResourceManager's RMContainer to handle a valid RELEASE
+    event at RUNNING state. (Siddharth Seth via vinodkv)
+
+    MAPREDUCE-2374. "Text File Busy" errors launching MR tasks. (Andy Isaacson
+    via atm)
+
 Release 0.23.4 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -70,6 +82,8 @@ Release 0.23.4 - UNRELEASED
 
   IMPROVEMENTS
 
+     Change package of YarnClient to org.apache.hadoop. (Bikas Saha via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -109,3 +123,9 @@ Release 0.23.3 - Unreleased
     thus causes all containers to be rejected. (vinodkv)
 
     YARN-66. aggregated logs permissions not set properly (tgraves via bobby)
+
+    YARN-68. NodeManager will refuse to shutdown indefinitely due to container
+    log aggregation (daryn via bobby)
+
+    YARN-87. NM ResourceLocalizationService does not set permissions of local 
+    cache directories (Jason Lowe via tgraves)

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java

@@ -152,9 +152,9 @@ public interface AllocateRequest {
   void clearAsks();
 
   /**
-   * Get the list of <code>ContainerId</code> of unused containers being 
+   * Get the list of <code>ContainerId</code> of containers being 
    * released by the <code>ApplicationMaster</code>.
-   * @return list of <code>ContainerId</code> of unused containers being 
+   * @return list of <code>ContainerId</code> of containers being 
    *         released by the <code>ApplicationMaster</code> 
    */
   @Public
@@ -170,9 +170,9 @@ public interface AllocateRequest {
   int getReleaseCount();
 
   /**
-   * Add the list of <code>ContainerId</code> of unused containers being 
+   * Add the list of <code>ContainerId</code> of containers being 
    * released by the <code>ApplicationMaster</code>
-   * @param releaseContainers list of <code>ContainerId</code> of unused 
+   * @param releaseContainers list of <code>ContainerId</code> of 
    *                          containers being released by the <
    *                          code>ApplicationMaster</code>
    */

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -61,11 +61,11 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
-import org.hadoop.yarn.client.YarnClientImpl;
 
 /**
  * Client for Distributed Shell application submission to YARN.

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/hadoop/yarn/client/YarnClient.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java

@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.hadoop.yarn.client;
+package org.apache.hadoop.yarn.client;
 
 import java.util.List;
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/hadoop/yarn/client/YarnClientImpl.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java

@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.hadoop.yarn.client;
+package org.apache.hadoop.yarn.client;
 
 import java.net.InetSocketAddress;
 import java.util.ArrayList;

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/hadoop/yarn/client/TestYarnClient.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java

@@ -16,9 +16,11 @@
 * limitations under the License.
 */
 
-package org.hadoop.yarn.client;
+package org.apache.hadoop.yarn.client;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.client.YarnClient;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.junit.Test;
 

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -416,10 +416,13 @@ public class YarnConfiguration extends Configuration {
   public final static String NM_CONTAINER_MON_INTERVAL_MS =
     NM_PREFIX + "container-monitor.interval-ms";
   public final static int DEFAULT_NM_CONTAINER_MON_INTERVAL_MS = 3000;
-  
+
   /** Class that calculates containers current resource utilization.*/
   public static final String NM_CONTAINER_MON_RESOURCE_CALCULATOR =
     NM_PREFIX + "container-monitor.resource-calculator.class";
+  /** Class that calculates process tree resource utilization.*/
+  public static final String NM_CONTAINER_MON_PROCESS_TREE =
+    NM_PREFIX + "container-monitor.process-tree.class";
 
   /**
    * Enable/Disable disks' health checker. Default is true.
@@ -571,8 +574,8 @@ public class YarnConfiguration extends Configuration {
       "$HADOOP_COMMON_HOME/share/hadoop/common/lib/*",
       "$HADOOP_HDFS_HOME/share/hadoop/hdfs/*",
       "$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*",
-      "$YARN_HOME/share/hadoop/mapreduce/*",
-      "$YARN_HOME/share/hadoop/mapreduce/lib/*"};
+      "$YARN_HOME/share/hadoop/yarn/*",
+      "$YARN_HOME/share/hadoop/yarn/lib/*"};
 
   /** Container temp directory */
   public static final String DEFAULT_CONTAINER_TEMP_DIR = "./tmp";

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java

@@ -72,7 +72,7 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   private static final Pattern CPU_TIME_FORMAT =
     Pattern.compile("^cpu[ \t]*([0-9]*)" +
     		            "[ \t]*([0-9]*)[ \t]*([0-9]*)[ \t].*");
-  
+
   private String procfsMemFile;
   private String procfsCpuFile;
   private String procfsStatFile;
@@ -91,11 +91,11 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   private float cpuUsage = UNAVAILABLE;
   private long sampleTime = UNAVAILABLE;
   private long lastSampleTime = UNAVAILABLE;
-  private ProcfsBasedProcessTree pTree = null;
+  private ResourceCalculatorProcessTree pTree = null;
 
   boolean readMemInfoFile = false;
   boolean readCpuInfoFile = false;
-  
+
   /**
    * Get current time
    * @return Unix time stamp in millisecond
@@ -103,7 +103,7 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   long getCurrentTime() {
     return System.currentTimeMillis();
   }
-  
+
   public LinuxResourceCalculatorPlugin() {
     procfsMemFile = PROCFS_MEMFILE;
     procfsCpuFile = PROCFS_CPUINFO;
@@ -112,7 +112,7 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
     String pid = System.getenv().get("JVM_PID");
     pTree = new ProcfsBasedProcessTree(pid);
   }
-  
+
   /**
    * Constructor which allows assigning the /proc/ directories. This will be
    * used only in unit tests

+ 32 - 63
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.util.StringUtils;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class ProcfsBasedProcessTree {
+public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
 
   static final Log LOG = LogFactory
       .getLog(ProcfsBasedProcessTree.class);
@@ -91,44 +91,36 @@ public class ProcfsBasedProcessTree {
   // to enable testing, using this variable which can be configured
   // to a test directory.
   private String procfsDir;
-  
+
   static private String deadPid = "-1";
   private String pid = deadPid;
   static private Pattern numberPattern = Pattern.compile("[1-9][0-9]*");
   private Long cpuTime = 0L;
-  private boolean setsidUsed = false;
 
   protected Map<String, ProcessInfo> processTree =
     new HashMap<String, ProcessInfo>();
 
   public ProcfsBasedProcessTree(String pid) {
-    this(pid, false);
-  }
-
-  public ProcfsBasedProcessTree(String pid, boolean setsidUsed) {
-    this(pid, setsidUsed, PROCFS);
+    this(pid, PROCFS);
   }
 
   /**
    * Build a new process tree rooted at the pid.
-   * 
+   *
    * This method is provided mainly for testing purposes, where
    * the root of the proc file system can be adjusted.
-   * 
+   *
    * @param pid root of the process tree
-   * @param setsidUsed true, if setsid was used for the root pid
-   * @param procfsDir the root of a proc file system - only used for testing. 
+   * @param procfsDir the root of a proc file system - only used for testing.
    */
-  public ProcfsBasedProcessTree(String pid, boolean setsidUsed,
-      String procfsDir) {
+  public ProcfsBasedProcessTree(String pid, String procfsDir) {
     this.pid = getValidPID(pid);
-    this.setsidUsed = setsidUsed;
     this.procfsDir = procfsDir;
   }
 
   /**
    * Checks if the ProcfsBasedProcessTree is available on this system.
-   * 
+   *
    * @return true if ProcfsBasedProcessTree is available. False otherwise.
    */
   public static boolean isAvailable() {
@@ -149,18 +141,19 @@ public class ProcfsBasedProcessTree {
   /**
    * Get the process-tree with latest state. If the root-process is not alive,
    * an empty tree will be returned.
-   * 
+   *
    * @return the process-tree with latest state.
    */
-  public ProcfsBasedProcessTree getProcessTree() {
+  @Override
+  public ResourceCalculatorProcessTree getProcessTree() {
     if (!pid.equals(deadPid)) {
       // Get the list of processes
       List<String> processList = getProcessList();
 
       Map<String, ProcessInfo> allProcessInfo = new HashMap<String, ProcessInfo>();
-      
+
       // cache the processTree to get the age for processes
-      Map<String, ProcessInfo> oldProcs = 
+      Map<String, ProcessInfo> oldProcs =
               new HashMap<String, ProcessInfo>(processTree);
       processTree.clear();
 
@@ -178,7 +171,7 @@ public class ProcfsBasedProcessTree {
       }
 
       if (me == null) {
-        return this; 
+        return this;
       }
 
       // Add each process to its parent.
@@ -210,7 +203,7 @@ public class ProcfsBasedProcessTree {
         if (procs.getValue() != null) {
           procs.getValue().updateJiffy(oldInfo);
           if (oldInfo != null) {
-            procs.getValue().updateAge(oldInfo);  
+            procs.getValue().updateAge(oldInfo);
           }
         }
       }
@@ -226,6 +219,7 @@ public class ProcfsBasedProcessTree {
   /** Verify that the given process id is same as its process group id.
    * @return true if the process id matches else return false.
    */
+  @Override
   public boolean checkPidPgrpidForMatch() {
     return checkPidPgrpidForMatch(pid, PROCFS);
   }
@@ -252,10 +246,11 @@ public class ProcfsBasedProcessTree {
 
   /**
    * Get a dump of the process-tree.
-   * 
+   *
    * @return a string concatenating the dump of information of all the processes
    *         in the process-tree
    */
+  @Override
   public String getProcessTreeDump() {
     StringBuilder ret = new StringBuilder();
     // The header.
@@ -273,38 +268,16 @@ public class ProcfsBasedProcessTree {
     return ret.toString();
   }
 
-  /**
-   * Get the cumulative virtual memory used by all the processes in the
-   * process-tree.
-   * 
-   * @return cumulative virtual memory used by the process-tree in bytes.
-   */
-  public long getCumulativeVmem() {
-    // include all processes.. all processes will be older than 0.
-    return getCumulativeVmem(0);
-  }
-
-  /**
-   * Get the cumulative resident set size (rss) memory used by all the processes
-   * in the process-tree.
-   *
-   * @return cumulative rss memory used by the process-tree in bytes. return 0
-   *         if it cannot be calculated
-   */
-  public long getCumulativeRssmem() {
-    // include all processes.. all processes will be older than 0.
-    return getCumulativeRssmem(0);
-  }
-
   /**
    * Get the cumulative virtual memory used by all the processes in the
    * process-tree that are older than the passed in age.
-   * 
+   *
    * @param olderThanAge processes above this age are included in the
    *                      memory addition
    * @return cumulative virtual memory used by the process-tree in bytes,
    *          for processes older than this age.
    */
+  @Override
   public long getCumulativeVmem(int olderThanAge) {
     long total = 0;
     for (ProcessInfo p : processTree.values()) {
@@ -314,7 +287,7 @@ public class ProcfsBasedProcessTree {
     }
     return total;
   }
-  
+
   /**
    * Get the cumulative resident set size (rss) memory used by all the processes
    * in the process-tree that are older than the passed in age.
@@ -325,6 +298,7 @@ public class ProcfsBasedProcessTree {
    *          for processes older than this age. return 0 if it cannot be
    *          calculated
    */
+  @Override
   public long getCumulativeRssmem(int olderThanAge) {
     if (PAGE_SIZE < 0) {
       return 0;
@@ -345,6 +319,7 @@ public class ProcfsBasedProcessTree {
    * @return cumulative CPU time in millisecond since the process-tree created
    *         return 0 if it cannot be calculated
    */
+  @Override
   public long getCumulativeCpuTime() {
     if (JIFFY_LENGTH_IN_MILLIS < 0) {
       return 0;
@@ -352,7 +327,7 @@ public class ProcfsBasedProcessTree {
     long incJiffies = 0;
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
-        incJiffies += p.dtime;
+        incJiffies += p.getDtime();
       }
     }
     cpuTime += incJiffies * JIFFY_LENGTH_IN_MILLIS;
@@ -391,14 +366,14 @@ public class ProcfsBasedProcessTree {
    * Construct the ProcessInfo using the process' PID and procfs rooted at the
    * specified directory and return the same. It is provided mainly to assist
    * testing purposes.
-   * 
+   *
    * Returns null on failing to read from procfs,
    *
    * @param pinfo ProcessInfo that needs to be updated
    * @param procfsDir root of the proc file system
    * @return updated ProcessInfo, null on errors.
    */
-  private static ProcessInfo constructProcessInfo(ProcessInfo pinfo, 
+  private static ProcessInfo constructProcessInfo(ProcessInfo pinfo,
                                                     String procfsDir) {
     ProcessInfo ret = null;
     // Read "procfsDir/<pid>/stat" file - typically /proc/<pid>/stat
@@ -454,6 +429,7 @@ public class ProcfsBasedProcessTree {
    * Returns a string printing PIDs of process present in the
    * ProcfsBasedProcessTree. Output format : [pid pid ..]
    */
+  @Override
   public String toString() {
     StringBuffer pTree = new StringBuffer("[ ");
     for (String p : processTree.keySet()) {
@@ -464,9 +440,9 @@ public class ProcfsBasedProcessTree {
   }
 
   /**
-   * 
+   *
    * Class containing information of a process.
-   * 
+   *
    */
   private static class ProcessInfo {
     private String pid; // process-id
@@ -480,7 +456,7 @@ public class ProcfsBasedProcessTree {
     private final BigInteger MAX_LONG = BigInteger.valueOf(Long.MAX_VALUE);
     private BigInteger stime = new BigInteger("0"); // # of jiffies in kernel mode
     // how many times has this process been seen alive
-    private int age; 
+    private int age;
 
     // # of jiffies used since last update:
     private Long dtime = 0L;
@@ -539,13 +515,6 @@ public class ProcfsBasedProcessTree {
     public int getAge() {
       return age;
     }
-    
-    public boolean isParent(ProcessInfo p) {
-      if (pid.equals(p.getPpid())) {
-        return true;
-      }
-      return false;
-    }
 
     public void updateProcessInfo(String name, String ppid, Integer pgrpId,
         Integer sessionId, Long utime, BigInteger stime, Long vmem, Long rssmem) {
@@ -558,7 +527,7 @@ public class ProcfsBasedProcessTree {
       this.vmem = vmem;
       this.rssmemPage = rssmem;
     }
-    
+
     public void updateJiffy(ProcessInfo oldInfo) {
       if (oldInfo == null) {
         BigInteger sum = this.stime.add(BigInteger.valueOf(this.utime));
@@ -578,7 +547,7 @@ public class ProcfsBasedProcessTree {
     public void updateAge(ProcessInfo oldInfo) {
       this.age = oldInfo.age + 1;
     }
-    
+
     public boolean addChild(ProcessInfo p) {
       return children.add(p);
     }

+ 8 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 
 /**
  * Plugin to calculate resource information on the system.
- * 
+ *
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -34,14 +34,14 @@ public abstract class ResourceCalculatorPlugin extends Configured {
 
   /**
    * Obtain the total size of the virtual memory present in the system.
-   * 
+   *
    * @return virtual memory size in bytes.
    */
   public abstract long getVirtualMemorySize();
 
   /**
    * Obtain the total size of the physical memory present in the system.
-   * 
+   *
    * @return physical memory size bytes.
    */
   public abstract long getPhysicalMemorySize();
@@ -133,13 +133,14 @@ public abstract class ResourceCalculatorPlugin extends Configured {
   }
 
   /**
-   * Get the ResourceCalculatorPlugin from the class name and configure it. If
+   * Create the ResourceCalculatorPlugin from the class name and configure it. If
    * class name is null, this method will try and return a memory calculator
    * plugin available for this system.
-   * 
-   * @param clazz class-name
+   *
+   * @param clazz ResourceCalculator plugin class-name
    * @param conf configure the plugin with this.
-   * @return ResourceCalculatorPlugin
+   * @return ResourceCalculatorPlugin or null if ResourceCalculatorPlugin is not
+   * 		 available for current system
    */
   public static ResourceCalculatorPlugin getResourceCalculatorPlugin(
       Class<? extends ResourceCalculatorPlugin> clazz, Configuration conf) {

+ 145 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java

@@ -0,0 +1,145 @@
+/**
+ * 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.yarn.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Interface class to obtain process resource usage
+ *
+ */
+public abstract class ResourceCalculatorProcessTree {
+  static final Log LOG = LogFactory
+      .getLog(ResourceCalculatorProcessTree.class);
+
+  /**
+   * Get the process-tree with latest state. If the root-process is not alive,
+   * an empty tree will be returned.
+   *
+   * Each call to this function should increment the age of the running
+   * processes that already exist in the process tree. Age is used other API's
+   * of the interface.
+   *
+   * @return the process-tree with latest state.
+   */
+  public abstract ResourceCalculatorProcessTree getProcessTree();
+
+  /**
+   * Get a dump of the process-tree.
+   *
+   * @return a string concatenating the dump of information of all the processes
+   *         in the process-tree
+   */
+  public abstract String getProcessTreeDump();
+
+  /**
+   * Get the cumulative virtual memory used by all the processes in the
+   * process-tree.
+   *
+   * @return cumulative virtual memory used by the process-tree in bytes.
+   */
+  public long getCumulativeVmem() {
+    return getCumulativeVmem(0);
+  }
+
+  /**
+   * Get the cumulative resident set size (rss) memory used by all the processes
+   * in the process-tree.
+   *
+   * @return cumulative rss memory used by the process-tree in bytes. return 0
+   *         if it cannot be calculated
+   */
+  public long getCumulativeRssmem() {
+    return getCumulativeRssmem(0);
+  }
+
+  /**
+   * Get the cumulative virtual memory used by all the processes in the
+   * process-tree that are older than the passed in age.
+   *
+   * @param olderThanAge processes above this age are included in the
+   *                      memory addition
+   * @return cumulative virtual memory used by the process-tree in bytes,
+   *          for processes older than this age.
+   */
+  public abstract long getCumulativeVmem(int olderThanAge);
+
+  /**
+   * Get the cumulative resident set size (rss) memory used by all the processes
+   * in the process-tree that are older than the passed in age.
+   *
+   * @param olderThanAge processes above this age are included in the
+   *                      memory addition
+   * @return cumulative rss memory used by the process-tree in bytes,
+   *          for processes older than this age. return 0 if it cannot be
+   *          calculated
+   */
+  public abstract long getCumulativeRssmem(int olderThanAge);
+
+  /**
+   * Get the CPU time in millisecond used by all the processes in the
+   * process-tree since the process-tree created
+   *
+   * @return cumulative CPU time in millisecond since the process-tree created
+   *         return 0 if it cannot be calculated
+   */
+  public abstract long getCumulativeCpuTime();
+
+  /** Verify that the tree process id is same as its process group id.
+   * @return true if the process id matches else return false.
+   */
+  public abstract boolean checkPidPgrpidForMatch();
+
+  /**
+   * Create the ResourceCalculatorProcessTree rooted to specified process 
+   * from the class name and configure it. If class name is null, this method
+   * will try and return a process tree plugin available for this system.
+   *
+   * @param pid process pid of the root of the process tree
+   * @param clazz class-name
+   * @param conf configure the plugin with this.
+   *
+   * @return ResourceCalculatorProcessTree or null if ResourceCalculatorPluginTree
+   *         is not available for this system.
+   */
+  public static ResourceCalculatorProcessTree getResourceCalculatorProcessTree(
+	  String pid, Class<? extends ResourceCalculatorProcessTree> clazz, Configuration conf) {
+
+    if (clazz != null) {
+      return ReflectionUtils.newInstance(clazz, conf);
+    }
+
+    // No class given, try a os specific class
+    try {
+      String osName = System.getProperty("os.name");
+      if (osName.startsWith("Linux")) {
+        return new ProcfsBasedProcessTree(pid);
+      }
+    } catch (SecurityException se) {
+      // Failed to get Operating System name.
+      return null;
+    }
+
+    // Not supported on this system.
+    return null;
+  }
+}

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -555,10 +555,13 @@
      <value/>
   </property>
 
+  <!-- Applications' Configuration-->
+  
   <property>
     <description>CLASSPATH for YARN applications. A comma-separated list
     of CLASSPATH entries</description>
      <name>yarn.application.classpath</name>
-     <value>$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$YARN_HOME/share/hadoop/yarn/*,$YARN_HOME/share/hadoop/yarn/lib/*,$YARN_HOME/share/hadoop/mapreduce/*,$YARN_HOME/share/hadoop/mapreduce/lib/*</value>
+     <value>$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$YARN_HOME/share/hadoop/yarn/*,$YARN_HOME/share/hadoop/yarn/lib/*</value>
   </property>
+
 </configuration>

+ 54 - 57
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java

@@ -161,7 +161,7 @@ public class TestProcfsBasedProcessTree {
     String pid = getRogueTaskPID();
     LOG.info("Root process pid: " + pid);
     ProcfsBasedProcessTree p = createProcessTree(pid);
-    p = p.getProcessTree(); // initialize
+    p.getProcessTree(); // initialize
     LOG.info("ProcessTree: " + p.toString());
 
     File leaf = new File(lowestDescendant);
@@ -174,7 +174,7 @@ public class TestProcfsBasedProcessTree {
       }
     }
 
-    p = p.getProcessTree(); // reconstruct
+    p.getProcessTree(); // reconstruct
     LOG.info("ProcessTree: " + p.toString());
 
     // Get the process-tree dump
@@ -213,7 +213,7 @@ public class TestProcfsBasedProcessTree {
     }
 
     // ProcessTree is gone now. Any further calls should be sane.
-    p = p.getProcessTree();
+    p.getProcessTree();
     Assert.assertFalse("ProcessTree must have been gone", isAlive(pid));
     Assert.assertTrue("Cumulative vmem for the gone-process is "
         + p.getCumulativeVmem() + " . It should be zero.", p
@@ -222,13 +222,11 @@ public class TestProcfsBasedProcessTree {
   }
 
   protected ProcfsBasedProcessTree createProcessTree(String pid) {
-    return new ProcfsBasedProcessTree(pid,
-                               isSetsidAvailable());
+    return new ProcfsBasedProcessTree(pid);
   }
 
-  protected ProcfsBasedProcessTree createProcessTree(String pid,
-      boolean setsidUsed, String procfsRootDir) {
-    return new ProcfsBasedProcessTree(pid, setsidUsed, procfsRootDir);
+  protected ProcfsBasedProcessTree createProcessTree(String pid, String procfsRootDir) {
+    return new ProcfsBasedProcessTree(pid, procfsRootDir);
   }
 
   protected void destroyProcessTree(String pid) throws IOException {
@@ -237,7 +235,7 @@ public class TestProcfsBasedProcessTree {
 
   /**
    * Get PID from a pid-file.
-   * 
+   *
    * @param pidFileName
    *          Name of the pid-file.
    * @return the PID string read from the pid-file. Returns null if the
@@ -279,11 +277,11 @@ public class TestProcfsBasedProcessTree {
     }
     return pid;
   }
-  
+
   public static class ProcessStatInfo {
-    // sample stat in a single line : 3910 (gpm) S 1 3910 3910 0 -1 4194624 
-    // 83 0 0 0 0 0 0 0 16 0 1 0 7852 2408448 88 4294967295 134512640 
-    // 134590050 3220521392 3220520036 10975138 0 0 4096 134234626 
+    // sample stat in a single line : 3910 (gpm) S 1 3910 3910 0 -1 4194624
+    // 83 0 0 0 0 0 0 0 16 0 1 0 7852 2408448 88 4294967295 134512640
+    // 134590050 3220521392 3220520036 10975138 0 0 4096 134234626
     // 4294967295 0 0 17 1 0 0
     String pid;
     String name;
@@ -294,7 +292,7 @@ public class TestProcfsBasedProcessTree {
     String rssmemPage = "0";
     String utime = "0";
     String stime = "0";
-    
+
     public ProcessStatInfo(String[] statEntries) {
       pid = statEntries[0];
       name = statEntries[1];
@@ -310,19 +308,19 @@ public class TestProcfsBasedProcessTree {
         stime = statEntries[8];
       }
     }
-    
+
     // construct a line that mimics the procfs stat file.
     // all unused numerical entries are set to 0.
     public String getStatLine() {
       return String.format("%s (%s) S %s %s %s 0 0 0" +
                       " 0 0 0 0 %s %s 0 0 0 0 0 0 0 %s %s 0 0" +
                       " 0 0 0 0 0 0 0 0" +
-                      " 0 0 0 0 0", 
+                      " 0 0 0 0 0",
                       pid, name, ppid, pgrpId, session,
                       utime, stime, vmem, rssmemPage);
     }
   }
-  
+
   /**
    * A basic test that creates a few process directories and writes
    * stat files. Verifies that the cpu time and memory is correctly
@@ -335,33 +333,33 @@ public class TestProcfsBasedProcessTree {
 
     // test processes
     String[] pids = { "100", "200", "300", "400" };
-    // create the fake procfs root directory. 
+    // create the fake procfs root directory.
     File procfsRootDir = new File(TEST_ROOT_DIR, "proc");
 
     try {
       setupProcfsRootDir(procfsRootDir);
       setupPidDirs(procfsRootDir, pids);
-      
+
       // create stat objects.
       // assuming processes 100, 200, 300 are in tree and 400 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
-      procInfos[0] = new ProcessStatInfo(new String[] 
+      procInfos[0] = new ProcessStatInfo(new String[]
           {"100", "proc1", "1", "100", "100", "100000", "100", "1000", "200"});
-      procInfos[1] = new ProcessStatInfo(new String[] 
+      procInfos[1] = new ProcessStatInfo(new String[]
           {"200", "proc2", "100", "100", "100", "200000", "200", "2000", "400"});
-      procInfos[2] = new ProcessStatInfo(new String[] 
+      procInfos[2] = new ProcessStatInfo(new String[]
           {"300", "proc3", "200", "100", "100", "300000", "300", "3000", "600"});
-      procInfos[3] = new ProcessStatInfo(new String[] 
+      procInfos[3] = new ProcessStatInfo(new String[]
           {"400", "proc4", "1", "400", "400", "400000", "400", "4000", "800"});
-      
+
       writeStatFiles(procfsRootDir, pids, procInfos);
-      
+
       // crank up the process tree class.
       ProcfsBasedProcessTree processTree =
-          createProcessTree("100", true, procfsRootDir.getAbsolutePath());
+          createProcessTree("100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.getProcessTree();
-      
+
       // verify cumulative memory
       Assert.assertEquals("Cumulative virtual memory does not match", 600000L,
                    processTree.getCumulativeVmem());
@@ -397,7 +395,7 @@ public class TestProcfsBasedProcessTree {
       FileUtil.fullyDelete(procfsRootDir);
     }
   }
-  
+
   /**
    * Tests that cumulative memory is computed only for
    * processes older than a given age.
@@ -408,33 +406,33 @@ public class TestProcfsBasedProcessTree {
   public void testMemForOlderProcesses() throws IOException {
     // initial list of processes
     String[] pids = { "100", "200", "300", "400" };
-    // create the fake procfs root directory. 
+    // create the fake procfs root directory.
     File procfsRootDir = new File(TEST_ROOT_DIR, "proc");
 
     try {
       setupProcfsRootDir(procfsRootDir);
       setupPidDirs(procfsRootDir, pids);
-      
+
       // create stat objects.
       // assuming 100, 200 and 400 are in tree, 300 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
-      procInfos[0] = new ProcessStatInfo(new String[] 
+      procInfos[0] = new ProcessStatInfo(new String[]
                         {"100", "proc1", "1", "100", "100", "100000", "100"});
-      procInfos[1] = new ProcessStatInfo(new String[] 
+      procInfos[1] = new ProcessStatInfo(new String[]
                         {"200", "proc2", "100", "100", "100", "200000", "200"});
-      procInfos[2] = new ProcessStatInfo(new String[] 
+      procInfos[2] = new ProcessStatInfo(new String[]
                         {"300", "proc3", "1", "300", "300", "300000", "300"});
-      procInfos[3] = new ProcessStatInfo(new String[] 
+      procInfos[3] = new ProcessStatInfo(new String[]
                         {"400", "proc4", "100", "100", "100", "400000", "400"});
-      
+
       writeStatFiles(procfsRootDir, pids, procInfos);
-      
+
       // crank up the process tree class.
       ProcfsBasedProcessTree processTree =
-          createProcessTree("100", true, procfsRootDir.getAbsolutePath());
+          createProcessTree("100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.getProcessTree();
-      
+
       // verify cumulative memory
       Assert.assertEquals("Cumulative memory does not match",
                    700000L, processTree.getCumulativeVmem());
@@ -442,12 +440,12 @@ public class TestProcfsBasedProcessTree {
       // write one more process as child of 100.
       String[] newPids = { "500" };
       setupPidDirs(procfsRootDir, newPids);
-      
+
       ProcessStatInfo[] newProcInfos = new ProcessStatInfo[1];
       newProcInfos[0] = new ProcessStatInfo(new String[]
                       {"500", "proc5", "100", "100", "100", "500000", "500"});
       writeStatFiles(procfsRootDir, newPids, newProcInfos);
-      
+
       // check memory includes the new process.
       processTree.getProcessTree();
       Assert.assertEquals("Cumulative vmem does not include new process",
@@ -456,7 +454,7 @@ public class TestProcfsBasedProcessTree {
                         1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
       Assert.assertEquals("Cumulative rssmem does not include new process",
                    cumuRssMem, processTree.getCumulativeRssmem());
-      
+
       // however processes older than 1 iteration will retain the older value
       Assert.assertEquals("Cumulative vmem shouldn't have included new process",
                    700000L, processTree.getCumulativeVmem(1));
@@ -468,7 +466,7 @@ public class TestProcfsBasedProcessTree {
       // one more process
       newPids = new String[]{ "600" };
       setupPidDirs(procfsRootDir, newPids);
-      
+
       newProcInfos = new ProcessStatInfo[1];
       newProcInfos[0] = new ProcessStatInfo(new String[]
                       {"600", "proc6", "100", "100", "100", "600000", "600"});
@@ -476,7 +474,7 @@ public class TestProcfsBasedProcessTree {
 
       // refresh process tree
       processTree.getProcessTree();
-      
+
       // processes older than 2 iterations should be same as before.
       Assert.assertEquals("Cumulative vmem shouldn't have included new processes",
                    700000L, processTree.getCumulativeVmem(2));
@@ -515,15 +513,14 @@ public class TestProcfsBasedProcessTree {
   public void testDestroyProcessTree() throws IOException {
     // test process
     String pid = "100";
-    // create the fake procfs root directory. 
+    // create the fake procfs root directory.
     File procfsRootDir = new File(TEST_ROOT_DIR, "proc");
 
     try {
       setupProcfsRootDir(procfsRootDir);
-      
+
       // crank up the process tree class.
-      ProcfsBasedProcessTree processTree =
-          createProcessTree(pid, true, procfsRootDir.getAbsolutePath());
+      createProcessTree(pid, procfsRootDir.getAbsolutePath());
 
       // Let us not create stat file for pid 100.
       Assert.assertTrue(ProcfsBasedProcessTree.checkPidPgrpidForMatch(
@@ -532,10 +529,10 @@ public class TestProcfsBasedProcessTree {
       FileUtil.fullyDelete(procfsRootDir);
     }
   }
-  
+
   /**
    * Test the correctness of process-tree dump.
-   * 
+   *
    * @throws IOException
    */
   @Test
@@ -578,7 +575,7 @@ public class TestProcfsBasedProcessTree {
       writeCmdLineFiles(procfsRootDir, pids, cmdLines);
 
       ProcfsBasedProcessTree processTree = createProcessTree(
-          "100", true, procfsRootDir.getAbsolutePath());
+          "100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.getProcessTree();
 
@@ -675,11 +672,11 @@ public class TestProcfsBasedProcessTree {
    * @param procfsRootDir root directory to create.
    * @throws IOException if could not delete the procfs root directory
    */
-  public static void setupProcfsRootDir(File procfsRootDir) 
-                                        throws IOException { 
+  public static void setupProcfsRootDir(File procfsRootDir)
+                                        throws IOException {
     // cleanup any existing process root dir.
     if (procfsRootDir.exists()) {
-      Assert.assertTrue(FileUtil.fullyDelete(procfsRootDir));  
+      Assert.assertTrue(FileUtil.fullyDelete(procfsRootDir));
     }
 
     // create afresh
@@ -692,7 +689,7 @@ public class TestProcfsBasedProcessTree {
    * @param pids the PID directories to create.
    * @throws IOException If PID dirs could not be created
    */
-  public static void setupPidDirs(File procfsRootDir, String[] pids) 
+  public static void setupPidDirs(File procfsRootDir, String[] pids)
                       throws IOException {
     for (String pid : pids) {
       File pidDir = new File(procfsRootDir, pid);
@@ -705,7 +702,7 @@ public class TestProcfsBasedProcessTree {
       }
     }
   }
-  
+
   /**
    * Write stat files under the specified pid directories with data
    * setup in the corresponding ProcessStatInfo objects
@@ -715,7 +712,7 @@ public class TestProcfsBasedProcessTree {
    *              written to the stat files.
    * @throws IOException if stat files could not be written
    */
-  public static void writeStatFiles(File procfsRootDir, String[] pids, 
+  public static void writeStatFiles(File procfsRootDir, String[] pids,
                               ProcessStatInfo[] procs) throws IOException {
     for (int i=0; i<pids.length; i++) {
       File statFile =
@@ -726,7 +723,7 @@ public class TestProcfsBasedProcessTree {
         FileWriter fw = new FileWriter(statFile);
         bw = new BufferedWriter(fw);
         bw.write(procs[i].getStatLine());
-        LOG.info("wrote stat file for " + pids[i] + 
+        LOG.info("wrote stat file for " + pids[i] +
                   " with contents: " + procs[i].getStatLine());
       } finally {
         // not handling exception - will throw an error and fail the test.

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -181,6 +181,7 @@ public class ResourceLocalizationService extends CompositeService
     try {
       // TODO queue deletions here, rather than NM init?
       FileContext lfs = getLocalFileContext(conf);
+      lfs.setUMask(new FsPermission((short)FsPermission.DEFAULT_UMASK));
       List<String> localDirs = dirsHandler.getLocalDirs();
       for (String localDir : localDirs) {
         // $local/usercache

+ 0 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregator.java

@@ -26,7 +26,4 @@ public interface AppLogAggregator extends Runnable {
       boolean wasContainerSuccessful);
 
   void finishLogAggregation();
-
-  void join();
-
 }

+ 5 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java

@@ -137,6 +137,9 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     try {
       doAppLogAggregation();
     } finally {
+      if (!this.appAggregationFinished.get()) {
+        LOG.warn("Aggregation did not complete for application " + appId);
+      }
       this.appAggregationFinished.set(true);
     }
   }
@@ -155,6 +158,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         }
       } catch (InterruptedException e) {
         LOG.warn("PendingContainers queue is interrupted");
+        this.appFinishing.set(true);
       }
     }
 
@@ -197,6 +201,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     this.dispatcher.getEventHandler().handle(
         new ApplicationEvent(this.appId,
             ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED));
+    this.appAggregationFinished.set(true);    
   }
 
   private Path getRemoteNodeTmpLogFileForApp() {
@@ -250,21 +255,4 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     LOG.info("Application just finished : " + this.applicationId);
     this.appFinishing.set(true);
   }
-
-  @Override
-  public void join() {
-    // Aggregation service is finishing
-    this.finishLogAggregation();
-
-    while (!this.appAggregationFinished.get()) {
-      LOG.info("Waiting for aggregation to complete for "
-          + this.applicationId);
-      try {
-        Thread.sleep(THREAD_SLEEP_TIME);
-      } catch (InterruptedException e) {
-        LOG.warn("Join interrupted. Some logs may not have been aggregated!!");
-        break;
-      }
-    }
-  }
 }

+ 33 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java

@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,8 +36,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -137,11 +136,33 @@ public class LogAggregationService extends AbstractService implements
   @Override
   public synchronized void stop() {
     LOG.info(this.getName() + " waiting for pending aggregation during exit");
-    for (AppLogAggregator appLogAggregator : this.appLogAggregators.values()) {
-      appLogAggregator.join();
-    }
+    stopAggregators();
     super.stop();
   }
+   
+  private void stopAggregators() {
+    threadPool.shutdown();
+    // politely ask to finish
+    for (AppLogAggregator aggregator : appLogAggregators.values()) {
+      aggregator.finishLogAggregation();
+    }
+    while (!threadPool.isTerminated()) { // wait for all threads to finish
+      for (ApplicationId appId : appLogAggregators.keySet()) {
+        LOG.info("Waiting for aggregation to complete for " + appId);
+      }
+      try {
+        if (!threadPool.awaitTermination(30, TimeUnit.SECONDS)) {
+          threadPool.shutdownNow(); // send interrupt to hurry them along
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Aggregation stop interrupted!");
+        break;
+      }
+    }
+    for (ApplicationId appId : appLogAggregators.keySet()) {
+      LOG.warn("Some logs may not have been aggregated for " + appId);
+    }
+  }
   
   private void verifyAndCreateRemoteLogDir(Configuration conf) {
     // Checking the existance of the TLD
@@ -293,10 +314,7 @@ public class LogAggregationService extends AbstractService implements
     final UserGroupInformation userUgi =
         UserGroupInformation.createRemoteUser(user);
     if (credentials != null) {
-      for (Token<? extends TokenIdentifier> token : credentials
-          .getAllTokens()) {
-        userUgi.addToken(token);
-      }
+      userUgi.addCredentials(credentials);
     }
 
     // New application
@@ -312,9 +330,13 @@ public class LogAggregationService extends AbstractService implements
     try {
       // Create the app dir
       createAppDir(user, appId, userUgi);
-    } catch (YarnException e) {
+    } catch (Exception e) {
+      appLogAggregators.remove(appId);
       closeFileSystems(userUgi);
-      throw e;
+      if (!(e instanceof YarnException)) {
+        e = new YarnException(e);
+      }
+      throw (YarnException)e;
     }
 
 

+ 31 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 
 import com.google.common.base.Preconditions;
@@ -60,6 +60,8 @@ public class ContainersMonitorImpl extends AbstractService implements
   private final Dispatcher eventDispatcher;
   private final Context context;
   private ResourceCalculatorPlugin resourceCalculatorPlugin;
+  private Configuration conf;
+  private Class<? extends ResourceCalculatorProcessTree> processTreeClass;
 
   private long maxVmemAllottedForContainers = DISABLED_MEMORY_LIMIT;
   private long maxPmemAllottedForContainers = DISABLED_MEMORY_LIMIT;
@@ -96,6 +98,11 @@ public class ContainersMonitorImpl extends AbstractService implements
         ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, conf);
     LOG.info(" Using ResourceCalculatorPlugin : "
         + this.resourceCalculatorPlugin);
+    processTreeClass = conf.getClass(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, null,
+            ResourceCalculatorProcessTree.class);
+    this.conf = conf;
+    LOG.info(" Using ResourceCalculatorProcessTree : "
+        + this.processTreeClass);
 
     long totalPhysicalMemoryOnNM = DISABLED_MEMORY_LIMIT;
     if (this.resourceCalculatorPlugin != null) {
@@ -140,7 +147,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   /**
    * Is the total physical memory check enabled?
-   * 
+   *
    * @return true if total physical memory check is enabled.
    */
   boolean isPhysicalMemoryCheckEnabled() {
@@ -149,7 +156,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   /**
    * Is the total virtual memory check enabled?
-   * 
+   *
    * @return true if total virtual memory check is enabled.
    */
   boolean isVirtualMemoryCheckEnabled() {
@@ -157,12 +164,16 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   private boolean isEnabled() {
-    if (!ProcfsBasedProcessTree.isAvailable()) {
-      LOG.info("ProcessTree implementation is missing on this system. "
-          + this.getClass().getName() + " is disabled.");
-      return false;
+    if (resourceCalculatorPlugin == null) {
+            LOG.info("ResourceCalculatorPlugin is unavailable on this system. "
+                + this.getClass().getName() + " is disabled.");
+            return false;
+    }
+    if (ResourceCalculatorProcessTree.getResourceCalculatorProcessTree("0", processTreeClass, conf) == null) {
+        LOG.info("ResourceCalculatorProcessTree is unavailable on this system. "
+                + this.getClass().getName() + " is disabled.");
+            return false;
     }
-
     if (!(isPhysicalMemoryCheckEnabled() || isVirtualMemoryCheckEnabled())) {
       LOG.info("Neither virutal-memory nor physical-memory monitoring is " +
           "needed. Not running the monitor-thread");
@@ -196,12 +207,12 @@ public class ContainersMonitorImpl extends AbstractService implements
   private static class ProcessTreeInfo {
     private ContainerId containerId;
     private String pid;
-    private ProcfsBasedProcessTree pTree;
+    private ResourceCalculatorProcessTree pTree;
     private long vmemLimit;
     private long pmemLimit;
 
     public ProcessTreeInfo(ContainerId containerId, String pid,
-        ProcfsBasedProcessTree pTree, long vmemLimit, long pmemLimit) {
+        ResourceCalculatorProcessTree pTree, long vmemLimit, long pmemLimit) {
       this.containerId = containerId;
       this.pid = pid;
       this.pTree = pTree;
@@ -221,11 +232,11 @@ public class ContainersMonitorImpl extends AbstractService implements
       this.pid = pid;
     }
 
-    public ProcfsBasedProcessTree getProcessTree() {
+    public ResourceCalculatorProcessTree getProcessTree() {
       return this.pTree;
     }
 
-    public void setProcessTree(ProcfsBasedProcessTree pTree) {
+    public void setProcessTree(ResourceCalculatorProcessTree pTree) {
       this.pTree = pTree;
     }
 
@@ -245,20 +256,20 @@ public class ContainersMonitorImpl extends AbstractService implements
   /**
    * Check whether a container's process tree's current memory usage is over
    * limit.
-   * 
+   *
    * When a java process exec's a program, it could momentarily account for
    * double the size of it's memory, because the JVM does a fork()+exec()
    * which at fork time creates a copy of the parent's memory. If the
    * monitoring thread detects the memory used by the container tree at the
    * same instance, it could assume it is over limit and kill the tree, for no
    * fault of the process itself.
-   * 
+   *
    * We counter this problem by employing a heuristic check: - if a process
    * tree exceeds the memory limit by more than twice, it is killed
    * immediately - if a process tree has processes older than the monitoring
    * interval exceeding the memory limit by even 1 time, it is killed. Else it
    * is given the benefit of doubt to lie around for one more iteration.
-   * 
+   *
    * @param containerId
    *          Container Id for the container tree
    * @param currentMemUsage
@@ -295,7 +306,7 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   // method provided just for easy testing purposes
-  boolean isProcessTreeOverLimit(ProcfsBasedProcessTree pTree,
+  boolean isProcessTreeOverLimit(ResourceCalculatorProcessTree pTree,
       String containerId, long limit) {
     long currentMemUsage = pTree.getCumulativeVmem();
     // as processes begin with an age 1, we want to see if there are processes
@@ -370,9 +381,8 @@ public class ContainersMonitorImpl extends AbstractService implements
                 LOG.debug("Tracking ProcessTree " + pId
                     + " for the first time");
 
-                ProcfsBasedProcessTree pt =
-                    new ProcfsBasedProcessTree(pId,
-                        ContainerExecutor.isSetsidAvailable);
+                ResourceCalculatorProcessTree pt =
+                    ResourceCalculatorProcessTree.getResourceCalculatorProcessTree(pId, processTreeClass, conf);
                 ptInfo.setPid(pId);
                 ptInfo.setProcessTree(pt);
               }
@@ -385,7 +395,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
             LOG.debug("Constructing ProcessTree for : PID = " + pId
                 + " ContainerId = " + containerId);
-            ProcfsBasedProcessTree pTree = ptInfo.getProcessTree();
+            ResourceCalculatorProcessTree pTree = ptInfo.getProcessTree();
             pTree = pTree.getProcessTree(); // get the updated process-tree
             ptInfo.setProcessTree(pTree); // update ptInfo with proces-tree of
                                           // updated state
@@ -471,7 +481,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     private String formatErrorMessage(String memTypeExceeded,
         long currentVmemUsage, long vmemLimit,
         long currentPmemUsage, long pmemLimit,
-        String pId, ContainerId containerId, ProcfsBasedProcessTree pTree) {
+        String pId, ContainerId containerId, ResourceCalculatorProcessTree pTree) {
       return
         String.format("Container [pid=%s,containerID=%s] is running beyond %s memory limits. ",
             pId, containerId, memTypeExceeded) +

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java

@@ -50,6 +50,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
@@ -116,6 +117,7 @@ public class TestResourceLocalizationService {
   @Test
   public void testLocalizationInit() throws Exception {
     final Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
     AsyncDispatcher dispatcher = new AsyncDispatcher();
     dispatcher.init(new Configuration());
 
@@ -151,14 +153,18 @@ public class TestResourceLocalizationService {
       // initialize ResourceLocalizationService
       locService.init(conf);
 
+      final FsPermission defaultPerm = new FsPermission((short)0755);
+
       // verify directory creation
       for (Path p : localDirs) {
         Path usercache = new Path(p, ContainerLocalizer.USERCACHE);
         verify(spylfs)
-          .mkdir(eq(usercache), isA(FsPermission.class), eq(true));
+          .mkdir(eq(usercache),
+              eq(defaultPerm), eq(true));
         Path publicCache = new Path(p, ContainerLocalizer.FILECACHE);
         verify(spylfs)
-          .mkdir(eq(publicCache), isA(FsPermission.class), eq(true));
+          .mkdir(eq(publicCache),
+              eq(defaultPerm), eq(true));
         Path nmPriv = new Path(p, ResourceLocalizationService.NM_PRIVATE_DIR);
         verify(spylfs).mkdir(eq(nmPriv),
             eq(ResourceLocalizationService.NM_PRIVATE_PERM), eq(true));

+ 15 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java

@@ -157,14 +157,18 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         application1));
 
     logAggregationService.stop();
+    assertEquals(0, logAggregationService.getNumAggregators());
     // ensure filesystems were closed
     verify(logAggregationService).closeFileSystems(
         any(UserGroupInformation.class));
     
+    delSrvc.stop();
+    
     String containerIdStr = ConverterUtils.toString(container11);
     File containerLogDir = new File(app1LogDir, containerIdStr);
     for (String fileType : new String[] { "stdout", "stderr", "syslog" }) {
-      Assert.assertFalse(new File(containerLogDir, fileType).exists());
+      File f = new File(containerLogDir, fileType);
+      Assert.assertFalse("check "+f, f.exists());
     }
 
     Assert.assertFalse(app1LogDir.exists());
@@ -222,6 +226,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         application1));
 
     logAggregationService.stop();
+    assertEquals(0, logAggregationService.getNumAggregators());
 
     Assert.assertFalse(new File(logAggregationService
         .getRemoteNodeLogFileForApp(application1, this.user).toUri().getPath())
@@ -356,6 +361,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         application1));
 
     logAggregationService.stop();
+    assertEquals(0, logAggregationService.getNumAggregators());
 
     verifyContainerLogs(logAggregationService, application1,
         new ContainerId[] { container11, container12 });
@@ -454,7 +460,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     
     ApplicationId appId = BuilderUtils.newApplicationId(
         System.currentTimeMillis(), (int)Math.random());
-    doThrow(new YarnException("KABOOM!"))
+    Exception e = new RuntimeException("KABOOM!");
+    doThrow(e)
       .when(logAggregationService).createAppDir(any(String.class),
           any(ApplicationId.class), any(UserGroupInformation.class));
     logAggregationService.handle(new LogHandlerAppStartedEvent(appId,
@@ -463,7 +470,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     
     dispatcher.await();
     ApplicationEvent expectedEvents[] = new ApplicationEvent[]{
-        new ApplicationFinishEvent(appId, "Application failed to init aggregation: KABOOM!")
+        new ApplicationFinishEvent(appId,
+            "Application failed to init aggregation: "+e)
     };
     checkEvents(appEventHandler, expectedEvents, false,
         "getType", "getApplicationID", "getDiagnostic");
@@ -479,6 +487,9 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     logAggregationService.handle(new LogHandlerAppFinishedEvent(
         BuilderUtils.newApplicationId(1, 5)));
     dispatcher.await();
+
+    logAggregationService.stop();
+    assertEquals(0, logAggregationService.getNumAggregators());
   }
 
   private void writeContainerLogs(File appLogDir, ContainerId containerId)
@@ -690,6 +701,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
             ContainerLogsRetentionPolicy.ALL_CONTAINERS, this.acls));
 
     logAggregationService.stop();
+    assertEquals(0, logAggregationService.getNumAggregators());
   }
 
   @Test

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

@@ -82,7 +82,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
 
   /**
    * Test to verify the check for whether a process tree is over limit or not.
-   * 
+   *
    * @throws IOException
    *           if there was a problem setting up the fake procfs directories or
    *           files.
@@ -132,7 +132,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
       // tree rooted at 100 is over limit immediately, as it is
       // twice over the mem limit.
       ProcfsBasedProcessTree pTree = new ProcfsBasedProcessTree(
-                                          "100", true,
+                                          "100",
                                           procfsRootDir.getAbsolutePath());
       pTree.getProcessTree();
       assertTrue("tree rooted at 100 should be over limit " +
@@ -140,7 +140,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
                   test.isProcessTreeOverLimit(pTree, "dummyId", limit));
 
       // the tree rooted at 200 is initially below limit.
-      pTree = new ProcfsBasedProcessTree("200", true,
+      pTree = new ProcfsBasedProcessTree("200",
                                           procfsRootDir.getAbsolutePath());
       pTree.getProcessTree();
       assertFalse("tree rooted at 200 shouldn't be over limit " +
@@ -154,7 +154,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
                   test.isProcessTreeOverLimit(pTree, "dummyId", limit));
 
       // the tree rooted at 600 is never over limit.
-      pTree = new ProcfsBasedProcessTree("600", true,
+      pTree = new ProcfsBasedProcessTree("600",
                                             procfsRootDir.getAbsolutePath());
       pTree.getProcessTree();
       assertFalse("tree rooted at 600 should never be over limit.",
@@ -198,7 +198,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
         recordFactory.newRecordInstance(ApplicationId.class);
     appId.setClusterTimestamp(0);
     appId.setId(0);
-    ApplicationAttemptId appAttemptId = 
+    ApplicationAttemptId appAttemptId =
         recordFactory.newRecordInstance(ApplicationAttemptId.class);
     appAttemptId.setApplicationId(appId);
     appAttemptId.setAttemptId(1);
@@ -220,7 +220,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     rsrc_alpha.setType(LocalResourceType.FILE);
     rsrc_alpha.setTimestamp(scriptFile.lastModified());
     String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources = 
+    Map<String, LocalResource> localResources =
         new HashMap<String, LocalResource>();
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);

Niektoré súbory nie sú zobrazené, pretože je v týchto rozdielových dátach zmenené mnoho súborov