You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by cl...@apache.org on 2014/08/12 19:02:12 UTC

svn commit: r1617532 [3/4] - in /hadoop/common/branches/fs-encryption/hadoop-hdfs-project: hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/ hadoop-hdfs-httpfs/src/main...

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java Tue Aug 12 17:02:07 2014
@@ -40,9 +40,8 @@ class FsVolumeList {
   private final VolumeChoosingPolicy<FsVolumeImpl> blockChooser;
   private volatile int numFailedVolumes;
 
-  FsVolumeList(List<FsVolumeImpl> volumes, int failedVols,
+  FsVolumeList(int failedVols,
       VolumeChoosingPolicy<FsVolumeImpl> blockChooser) {
-    this.volumes = Collections.unmodifiableList(volumes);
     this.blockChooser = blockChooser;
     this.numFailedVolumes = failedVols;
   }
@@ -101,12 +100,6 @@ class FsVolumeList {
     }
     return remaining;
   }
-    
-  void initializeReplicaMaps(ReplicaMap globalReplicaMap) throws IOException {
-    for (FsVolumeImpl v : volumes) {
-      v.getVolumeMap(globalReplicaMap);
-    }
-  }
   
   void getAllVolumesMap(final String bpid, final ReplicaMap volumeMap) throws IOException {
     long totalStartTime = Time.monotonicNow();
@@ -205,6 +198,19 @@ class FsVolumeList {
     return volumes.toString();
   }
 
+  /**
+   * Dynamically add new volumes to the existing volumes that this DN manages.
+   * @param newVolume the instance of new FsVolumeImpl.
+   */
+  synchronized void addVolume(FsVolumeImpl newVolume) {
+    // Make a copy of volumes to add new volumes.
+    final List<FsVolumeImpl> volumeList = volumes == null ?
+        new ArrayList<FsVolumeImpl>() :
+        new ArrayList<FsVolumeImpl>(volumes);
+    volumeList.add(newVolume);
+    volumes = Collections.unmodifiableList(volumeList);
+    FsDatasetImpl.LOG.info("Added new volume: " + newVolume.toString());
+  }
 
   void addBlockPool(final String bpid, final Configuration conf) throws IOException {
     long totalStartTime = Time.monotonicNow();

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Tue Aug 12 17:02:07 2014
@@ -786,8 +786,6 @@ public class FSDirectory implements Clos
     checkSnapshot(srcInode, null);
   }
 
-
-
   private class RenameOperation {
     private final INodesInPath srcIIP;
     private final INodesInPath dstIIP;
@@ -820,7 +818,7 @@ public class FSDirectory implements Clos
       // snapshot is taken on the dst tree, changes will be recorded in the latest
       // snapshot of the src tree.
       if (isSrcInSnapshot) {
-        srcChild = srcChild.recordModification(srcIIP.getLatestSnapshotId());
+        srcChild.recordModification(srcIIP.getLatestSnapshotId());
       }
 
       // check srcChild for reference
@@ -950,8 +948,7 @@ public class FSDirectory implements Clos
       updateCount(iip, 0, dsDelta, true);
     }
 
-    file = file.setFileReplication(replication, iip.getLatestSnapshotId(),
-        inodeMap);
+    file.setFileReplication(replication, iip.getLatestSnapshotId());
     
     final short newBR = file.getBlockReplication(); 
     // check newBR < oldBR case. 
@@ -1234,8 +1231,7 @@ public class FSDirectory implements Clos
 
     // record modification
     final int latestSnapshot = iip.getLatestSnapshotId();
-    targetNode = targetNode.recordModification(latestSnapshot);
-    iip.setLastINode(targetNode);
+    targetNode.recordModification(latestSnapshot);
 
     // Remove the node from the namespace
     long removed = removeLastINode(iip);
@@ -2161,7 +2157,7 @@ public class FSDirectory implements Clos
       }
 
       final int latest = iip.getLatestSnapshotId();
-      dirNode = dirNode.recordModification(latest);
+      dirNode.recordModification(latest);
       dirNode.setQuota(nsQuota, dsQuota);
       return dirNode;
     }

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Tue Aug 12 17:02:07 2014
@@ -2733,7 +2733,7 @@ public class FSNamesystem implements Nam
                                    boolean writeToEditLog,
                                    int latestSnapshot, boolean logRetryCache)
       throws IOException {
-    file = file.recordModification(latestSnapshot);
+    file.recordModification(latestSnapshot);
     final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine);
 
     leaseManager.addLease(cons.getFileUnderConstructionFeature()
@@ -4441,7 +4441,7 @@ public class FSNamesystem implements Nam
     Preconditions.checkArgument(uc != null);
     leaseManager.removeLease(uc.getClientName(), src);
     
-    pendingFile = pendingFile.recordModification(latestSnapshot);
+    pendingFile.recordModification(latestSnapshot);
 
     // The file is no longer pending.
     // Create permanent INode, update blocks. No need to replace the inode here
@@ -6341,7 +6341,6 @@ public class FSNamesystem implements Nam
       blockManager.shutdown();
     }
   }
-  
 
   @Override // FSNamesystemMBean
   public int getNumLiveDataNodes() {
@@ -6389,6 +6388,15 @@ public class FSNamesystem implements Nam
   }
 
   /**
+   * Storages are marked as "content stale" after NN restart or fails over and
+   * before NN receives the first Heartbeat followed by the first Blockreport.
+   */
+  @Override // FSNamesystemMBean
+  public int getNumStaleStorages() {
+    return getBlockManager().getDatanodeManager().getNumStaleStorages();
+  }
+
+  /**
    * Sets the current generation stamp for legacy blocks
    */
   void setGenerationStampV1(long stamp) {

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java Tue Aug 12 17:02:07 2014
@@ -97,9 +97,9 @@ public abstract class INode implements I
   /** Set user */
   final INode setUser(String user, int latestSnapshotId)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.setUser(user);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    setUser(user);
+    return this;
   }
   /**
    * @param snapshotId
@@ -122,9 +122,9 @@ public abstract class INode implements I
   /** Set group */
   final INode setGroup(String group, int latestSnapshotId)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.setGroup(group);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    setGroup(group);
+    return this;
   }
 
   /**
@@ -148,9 +148,9 @@ public abstract class INode implements I
   /** Set the {@link FsPermission} of this {@link INode} */
   INode setPermission(FsPermission permission, int latestSnapshotId) 
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.setPermission(permission);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    setPermission(permission);
+    return this;
   }
 
   abstract AclFeature getAclFeature(int snapshotId);
@@ -164,18 +164,18 @@ public abstract class INode implements I
 
   final INode addAclFeature(AclFeature aclFeature, int latestSnapshotId)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.addAclFeature(aclFeature);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    addAclFeature(aclFeature);
+    return this;
   }
 
   abstract void removeAclFeature();
 
   final INode removeAclFeature(int latestSnapshotId)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.removeAclFeature();
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    removeAclFeature();
+    return this;
   }
 
   /**
@@ -199,9 +199,9 @@ public abstract class INode implements I
   
   final INode addXAttrFeature(XAttrFeature xAttrFeature, int latestSnapshotId) 
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.addXAttrFeature(xAttrFeature);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    addXAttrFeature(xAttrFeature);
+    return this;
   }
   
   /**
@@ -211,9 +211,9 @@ public abstract class INode implements I
   
   final INode removeXAttrFeature(int lastestSnapshotId)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(lastestSnapshotId);
-    nodeToUpdate.removeXAttrFeature();
-    return nodeToUpdate;
+    recordModification(lastestSnapshotId);
+    removeXAttrFeature();
+    return this;
   }
   
   /**
@@ -298,11 +298,8 @@ public abstract class INode implements I
    * @param latestSnapshotId The id of the latest snapshot that has been taken.
    *                         Note that it is {@link Snapshot#CURRENT_STATE_ID} 
    *                         if no snapshots have been taken.
-   * @return The current inode, which usually is the same object of this inode.
-   *         However, in some cases, this inode may be replaced with a new inode
-   *         for maintaining snapshots. The current inode is then the new inode.
    */
-  abstract INode recordModification(final int latestSnapshotId)
+  abstract void recordModification(final int latestSnapshotId)
       throws QuotaExceededException;
 
   /** Check whether it's a reference. */
@@ -652,9 +649,9 @@ public abstract class INode implements I
   /** Set the last modification time of inode. */
   public final INode setModificationTime(long modificationTime,
       int latestSnapshotId) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.setModificationTime(modificationTime);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    setModificationTime(modificationTime);
+    return this;
   }
 
   /**
@@ -682,9 +679,9 @@ public abstract class INode implements I
    */
   public final INode setAccessTime(long accessTime, int latestSnapshotId)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.setAccessTime(accessTime);
-    return nodeToUpdate;
+    recordModification(latestSnapshotId);
+    setAccessTime(accessTime);
+    return this;
   }
 
 

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java Tue Aug 12 17:02:07 2014
@@ -318,7 +318,7 @@ public class INodeDirectory extends INod
   }
 
   @Override
-  public INodeDirectory recordModification(int latestSnapshotId) 
+  public void recordModification(int latestSnapshotId)
       throws QuotaExceededException {
     if (isInLatestSnapshot(latestSnapshotId)
         && !shouldRecordInSrcSnapshot(latestSnapshotId)) {
@@ -330,7 +330,6 @@ public class INodeDirectory extends INod
       // record self in the diff list if necessary
       sf.getDiffs().saveSelf2Snapshot(latestSnapshotId, this, null);
     }
-    return this;
   }
 
   /**

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java Tue Aug 12 17:02:07 2014
@@ -284,7 +284,7 @@ public class INodeFile extends INodeWith
   }
 
   @Override
-  public INodeFile recordModification(final int latestSnapshotId) 
+  public void recordModification(final int latestSnapshotId)
       throws QuotaExceededException {
     if (isInLatestSnapshot(latestSnapshotId)
         && !shouldRecordInSrcSnapshot(latestSnapshotId)) {
@@ -296,7 +296,6 @@ public class INodeFile extends INodeWith
       // record self in the diff list if necessary
       sf.getDiffs().saveSelf2Snapshot(latestSnapshotId, this, null);
     }
-    return this;
   }
   
   public FileDiffList getDiffs() {
@@ -344,11 +343,10 @@ public class INodeFile extends INodeWith
 
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
-      int latestSnapshotId, final INodeMap inodeMap)
-      throws QuotaExceededException {
-    final INodeFile nodeToUpdate = recordModification(latestSnapshotId);
-    nodeToUpdate.setFileReplication(replication);
-    return nodeToUpdate;
+      int latestSnapshotId) throws QuotaExceededException {
+    recordModification(latestSnapshotId);
+    setFileReplication(replication);
+    return this;
   }
 
   /** @return preferred block size (in bytes) of the file. */

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java Tue Aug 12 17:02:07 2014
@@ -93,9 +93,8 @@ public class INodeMap {
         "", "", new FsPermission((short) 0)), 0, 0) {
       
       @Override
-      INode recordModification(int latestSnapshotId)
+      void recordModification(int latestSnapshotId)
           throws QuotaExceededException {
-        return null;
       }
       
       @Override

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java Tue Aug 12 17:02:07 2014
@@ -287,11 +287,9 @@ public abstract class INodeReference ext
   }
 
   @Override
-  final INode recordModification(int latestSnapshotId)
+  final void recordModification(int latestSnapshotId)
       throws QuotaExceededException {
     referred.recordModification(latestSnapshotId);
-    // reference is never replaced 
-    return this;
   }
 
   @Override // used by WithCount

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java Tue Aug 12 17:02:07 2014
@@ -47,12 +47,11 @@ public class INodeSymlink extends INodeW
   }
 
   @Override
-  INode recordModification(int latestSnapshotId) throws QuotaExceededException {
+  void recordModification(int latestSnapshotId) throws QuotaExceededException {
     if (isInLatestSnapshot(latestSnapshotId)) {
       INodeDirectory parent = getParent();
       parent.saveChild2Snapshot(this, latestSnapshotId, new INodeSymlink(this));
     }
-    return this;
   }
 
   /** @return true unconditionally. */

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java Tue Aug 12 17:02:07 2014
@@ -151,4 +151,11 @@ public interface FSNamesystemMBean {
    * @return number of blocks pending deletion
    */
   long getPendingDeletionBlocks();
+
+  /**
+   * Number of content stale storages.
+   * @return number of content stale storages
+   */
+  public int getNumStaleStorages();
+
 }

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java Tue Aug 12 17:02:07 2014
@@ -22,6 +22,9 @@ import org.apache.hadoop.classification.
 
 /**
  * A BlockCommand is an instruction to a datanode to register with the namenode.
+ * This command can't be combined with other commands in the same response.
+ * This is because after the datanode processes RegisterCommand, it will skip
+ * the rest of the DatanodeCommands in the same HeartbeatResponse.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving

Propchange: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1613788-1617527

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt Tue Aug 12 17:02:07 2014
@@ -37,6 +37,10 @@ ELSE (${CMAKE_SYSTEM_NAME} MATCHES "Linu
 ENDIF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
 
 IF(FUSE_FOUND)
+    add_library(posix_util
+        ../util/posix_util.c
+    )
+
     add_executable(fuse_dfs
         fuse_dfs.c
         fuse_options.c 

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c Tue Aug 12 17:02:07 2014
@@ -19,8 +19,8 @@
 #include "exception.h"
 #include "hdfs.h"
 #include "jni_helper.h"
+#include "platform.h"
 
-#include <inttypes.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
@@ -35,54 +35,54 @@ struct ExceptionInfo {
 
 static const struct ExceptionInfo gExceptionInfo[] = {
     {
-        .name = "java.io.FileNotFoundException",
-        .noPrintFlag = NOPRINT_EXC_FILE_NOT_FOUND,
-        .excErrno = ENOENT,
+        "java.io.FileNotFoundException",
+        NOPRINT_EXC_FILE_NOT_FOUND,
+        ENOENT,
     },
     {
-        .name = "org.apache.hadoop.security.AccessControlException",
-        .noPrintFlag = NOPRINT_EXC_ACCESS_CONTROL,
-        .excErrno = EACCES,
+        "org.apache.hadoop.security.AccessControlException",
+        NOPRINT_EXC_ACCESS_CONTROL,
+        EACCES,
     },
     {
-        .name = "org.apache.hadoop.fs.UnresolvedLinkException",
-        .noPrintFlag = NOPRINT_EXC_UNRESOLVED_LINK,
-        .excErrno = ENOLINK,
+        "org.apache.hadoop.fs.UnresolvedLinkException",
+        NOPRINT_EXC_UNRESOLVED_LINK,
+        ENOLINK,
     },
     {
-        .name = "org.apache.hadoop.fs.ParentNotDirectoryException",
-        .noPrintFlag = NOPRINT_EXC_PARENT_NOT_DIRECTORY,
-        .excErrno = ENOTDIR,
+        "org.apache.hadoop.fs.ParentNotDirectoryException",
+        NOPRINT_EXC_PARENT_NOT_DIRECTORY,
+        ENOTDIR,
     },
     {
-        .name = "java.lang.IllegalArgumentException",
-        .noPrintFlag = NOPRINT_EXC_ILLEGAL_ARGUMENT,
-        .excErrno = EINVAL,
+        "java.lang.IllegalArgumentException",
+        NOPRINT_EXC_ILLEGAL_ARGUMENT,
+        EINVAL,
     },
     {
-        .name = "java.lang.OutOfMemoryError",
-        .noPrintFlag = 0,
-        .excErrno = ENOMEM,
+        "java.lang.OutOfMemoryError",
+        0,
+        ENOMEM,
     },
     {
-        .name = "org.apache.hadoop.hdfs.server.namenode.SafeModeException",
-        .noPrintFlag = 0,
-        .excErrno = EROFS,
+        "org.apache.hadoop.hdfs.server.namenode.SafeModeException",
+        0,
+        EROFS,
     },
     {
-        .name = "org.apache.hadoop.fs.FileAlreadyExistsException",
-        .noPrintFlag = 0,
-        .excErrno = EEXIST,
+        "org.apache.hadoop.fs.FileAlreadyExistsException",
+        0,
+        EEXIST,
     },
     {
-        .name = "org.apache.hadoop.hdfs.protocol.QuotaExceededException",
-        .noPrintFlag = 0,
-        .excErrno = EDQUOT,
+        "org.apache.hadoop.hdfs.protocol.QuotaExceededException",
+        0,
+        EDQUOT,
     },
     {
-        .name = "org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException",
-        .noPrintFlag = 0,
-        .excErrno = ESTALE,
+        "org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException",
+        0,
+        ESTALE,
     },
 };
 
@@ -113,6 +113,7 @@ int printExceptionAndFreeV(JNIEnv *env, 
     jstring jStr = NULL;
     jvalue jVal;
     jthrowable jthr;
+    const char *stackTrace;
 
     jthr = classNameOfObject(exc, env, &className);
     if (jthr) {
@@ -148,7 +149,7 @@ int printExceptionAndFreeV(JNIEnv *env, 
             destroyLocalReference(env, jthr);
         } else {
             jStr = jVal.l;
-            const char *stackTrace = (*env)->GetStringUTFChars(env, jStr, NULL);
+            stackTrace = (*env)->GetStringUTFChars(env, jStr, NULL);
             if (!stackTrace) {
                 fprintf(stderr, "(unable to get stack trace for %s exception: "
                         "GetStringUTFChars error.)\n", className);

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.h?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.h (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.h Tue Aug 12 17:02:07 2014
@@ -34,13 +34,14 @@
  * usually not what you want.)
  */
 
+#include "platform.h"
+
 #include <jni.h>
 #include <stdio.h>
 
 #include <stdlib.h>
 #include <stdarg.h>
 #include <search.h>
-#include <pthread.h>
 #include <errno.h>
 
 /**
@@ -109,7 +110,7 @@ int printExceptionAndFreeV(JNIEnv *env, 
  *                        object.
  */
 int printExceptionAndFree(JNIEnv *env, jthrowable exc, int noPrintFlags,
-        const char *fmt, ...) __attribute__((format(printf, 4, 5)));  
+        const char *fmt, ...) TYPE_CHECKED_PRINTF_FORMAT(4, 5);
 
 /**
  * Print out information about the pending exception and free it.
@@ -124,7 +125,7 @@ int printExceptionAndFree(JNIEnv *env, j
  *                        object.
  */
 int printPendingExceptionAndFree(JNIEnv *env, int noPrintFlags,
-        const char *fmt, ...) __attribute__((format(printf, 3, 4)));  
+        const char *fmt, ...) TYPE_CHECKED_PRINTF_FORMAT(3, 4);
 
 /**
  * Get a local reference to the pending exception and clear it.
@@ -150,6 +151,7 @@ jthrowable getPendingExceptionAndClear(J
  * @return                A local reference to a RuntimeError
  */
 jthrowable newRuntimeError(JNIEnv *env, const char *fmt, ...)
-        __attribute__((format(printf, 2, 3)));
+        TYPE_CHECKED_PRINTF_FORMAT(2, 3);
 
+#undef TYPE_CHECKED_PRINTF_FORMAT
 #endif

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.c?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.c (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.c Tue Aug 12 17:02:07 2014
@@ -49,18 +49,18 @@ int expectFileStats(hdfsFile file,
             stats->totalShortCircuitBytesRead,
             stats->totalZeroCopyBytesRead);
     if (expectedTotalBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
+        EXPECT_UINT64_EQ(expectedTotalBytesRead, stats->totalBytesRead);
     }
     if (expectedTotalLocalBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalLocalBytesRead,
+        EXPECT_UINT64_EQ(expectedTotalLocalBytesRead,
                       stats->totalLocalBytesRead);
     }
     if (expectedTotalShortCircuitBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalShortCircuitBytesRead,
+        EXPECT_UINT64_EQ(expectedTotalShortCircuitBytesRead,
                       stats->totalShortCircuitBytesRead);
     }
     if (expectedTotalZeroCopyBytesRead != UINT64_MAX) {
-        EXPECT_INT64_EQ(expectedTotalZeroCopyBytesRead,
+        EXPECT_UINT64_EQ(expectedTotalZeroCopyBytesRead,
                       stats->totalZeroCopyBytesRead);
     }
     hdfsFileFreeReadStatistics(stats);

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.h?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.h (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/expect.h Tue Aug 12 17:02:07 2014
@@ -126,6 +126,18 @@ struct hdfsFile_internal;
         } \
     } while (0);
 
+#define EXPECT_UINT64_EQ(x, y) \
+    do { \
+        uint64_t __my_ret__ = y; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ != (x)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "value %"PRIu64" (errno: %d): expected %"PRIu64"\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (x)); \
+            return -1; \
+        } \
+    } while (0);
+
 #define RETRY_ON_EINTR_GET_ERRNO(ret, expr) do { \
     ret = expr; \
     if (!ret) \

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c Tue Aug 12 17:02:07 2014
@@ -19,7 +19,9 @@
 #include "exception.h"
 #include "hdfs.h"
 #include "jni_helper.h"
+#include "platform.h"
 
+#include <fcntl.h>
 #include <inttypes.h>
 #include <stdio.h>
 #include <string.h>
@@ -63,9 +65,9 @@ static void hdfsFreeFileInfoEntry(hdfsFi
  */
 enum hdfsStreamType
 {
-    UNINITIALIZED = 0,
-    INPUT = 1,
-    OUTPUT = 2,
+    HDFS_STREAM_UNINITIALIZED = 0,
+    HDFS_STREAM_INPUT = 1,
+    HDFS_STREAM_OUTPUT = 2,
 };
 
 /**
@@ -79,7 +81,7 @@ struct hdfsFile_internal {
 
 int hdfsFileIsOpenForRead(hdfsFile file)
 {
-    return (file->type == INPUT);
+    return (file->type == HDFS_STREAM_INPUT);
 }
 
 int hdfsFileGetReadStatistics(hdfsFile file,
@@ -96,7 +98,7 @@ int hdfsFileGetReadStatistics(hdfsFile f
         errno = EINTERNAL;
         return -1;
     }
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         ret = EINVAL;
         goto done;
     }
@@ -180,7 +182,7 @@ void hdfsFileFreeReadStatistics(struct h
 
 int hdfsFileIsOpenForWrite(hdfsFile file)
 {
-    return (file->type == OUTPUT);
+    return (file->type == HDFS_STREAM_OUTPUT);
 }
 
 int hdfsFileUsesDirectRead(hdfsFile file)
@@ -441,7 +443,7 @@ void hdfsBuilderSetKerbTicketCachePath(s
     bld->kerbTicketCachePath = kerbTicketCachePath;
 }
 
-hdfsFS hdfsConnect(const char* host, tPort port)
+hdfsFS hdfsConnect(const char *host, tPort port)
 {
     struct hdfsBuilder *bld = hdfsNewBuilder();
     if (!bld)
@@ -452,7 +454,7 @@ hdfsFS hdfsConnect(const char* host, tPo
 }
 
 /** Always return a new FileSystem handle */
-hdfsFS hdfsConnectNewInstance(const char* host, tPort port)
+hdfsFS hdfsConnectNewInstance(const char *host, tPort port)
 {
     struct hdfsBuilder *bld = hdfsNewBuilder();
     if (!bld)
@@ -463,7 +465,7 @@ hdfsFS hdfsConnectNewInstance(const char
     return hdfsBuilderConnect(bld);
 }
 
-hdfsFS hdfsConnectAsUser(const char* host, tPort port, const char *user)
+hdfsFS hdfsConnectAsUser(const char *host, tPort port, const char *user)
 {
     struct hdfsBuilder *bld = hdfsNewBuilder();
     if (!bld)
@@ -475,7 +477,7 @@ hdfsFS hdfsConnectAsUser(const char* hos
 }
 
 /** Always return a new FileSystem handle */
-hdfsFS hdfsConnectAsUserNewInstance(const char* host, tPort port,
+hdfsFS hdfsConnectAsUserNewInstance(const char *host, tPort port,
         const char *user)
 {
     struct hdfsBuilder *bld = hdfsNewBuilder();
@@ -518,7 +520,7 @@ static int calcEffectiveURI(struct hdfsB
     if (bld->port == 0) {
         suffix[0] = '\0';
     } else {
-        lastColon = rindex(bld->nn, ':');
+        lastColon = strrchr(bld->nn, ':');
         if (lastColon && (strspn(lastColon + 1, "0123456789") ==
                           strlen(lastColon + 1))) {
             fprintf(stderr, "port %d was given, but URI '%s' already "
@@ -737,6 +739,8 @@ int hdfsDisconnect(hdfsFS fs)
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     int ret;
+    jobject jFS;
+    jthrowable jthr;
 
     if (env == NULL) {
       errno = EINTERNAL;
@@ -744,7 +748,7 @@ int hdfsDisconnect(hdfsFS fs)
     }
 
     //Parameters
-    jobject jFS = (jobject)fs;
+    jFS = (jobject)fs;
 
     //Sanity check
     if (fs == NULL) {
@@ -752,7 +756,7 @@ int hdfsDisconnect(hdfsFS fs)
         return -1;
     }
 
-    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
+    jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
                      "close", "()V");
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -792,7 +796,7 @@ static jthrowable getDefaultBlockSize(JN
     return NULL;
 }
 
-hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags, 
+hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, 
                       int bufferSize, short replication, tSize blockSize)
 {
     /*
@@ -801,15 +805,7 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
        FSData{Input|Output}Stream f{is|os} = fs.create(f);
        return f{is|os};
     */
-    /* Get the JNIEnv* corresponding to current thread */
-    JNIEnv* env = getJNIEnv();
     int accmode = flags & O_ACCMODE;
-
-    if (env == NULL) {
-      errno = EINTERNAL;
-      return NULL;
-    }
-
     jstring jStrBufferSize = NULL, jStrReplication = NULL;
     jobject jConfiguration = NULL, jPath = NULL, jFile = NULL;
     jobject jFS = (jobject)fs;
@@ -817,6 +813,20 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
     jvalue jVal;
     hdfsFile file = NULL;
     int ret;
+    jint jBufferSize = bufferSize;
+    jshort jReplication = replication;
+
+    /* The hadoop java api/signature */
+    const char *method = NULL;
+    const char *signature = NULL;
+
+    /* Get the JNIEnv* corresponding to current thread */
+    JNIEnv* env = getJNIEnv();
+    if (env == NULL) {
+      errno = EINTERNAL;
+      return NULL;
+    }
+
 
     if (accmode == O_RDONLY || accmode == O_WRONLY) {
 	/* yay */
@@ -834,10 +844,6 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
       fprintf(stderr, "WARN: hdfs does not truly support O_CREATE && O_EXCL\n");
     }
 
-    /* The hadoop java api/signature */
-    const char* method = NULL;
-    const char* signature = NULL;
-
     if (accmode == O_RDONLY) {
 	method = "open";
         signature = JMETHOD2(JPARAM(HADOOP_PATH), "I", JPARAM(HADOOP_ISTRM));
@@ -867,8 +873,6 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
     }
     jConfiguration = jVal.l;
 
-    jint jBufferSize = bufferSize;
-    jshort jReplication = replication;
     jStrBufferSize = (*env)->NewStringUTF(env, "io.file.buffer.size"); 
     if (!jStrBufferSize) {
         ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
@@ -905,7 +909,7 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
                     path);
                 goto done;
             }
-            jReplication = jVal.i;
+            jReplication = (jshort)jVal.i;
         }
     }
  
@@ -955,7 +959,8 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const c
             "hdfsOpenFile(%s): NewGlobalRef", path); 
         goto done;
     }
-    file->type = (((flags & O_WRONLY) == 0) ? INPUT : OUTPUT);
+    file->type = (((flags & O_WRONLY) == 0) ? HDFS_STREAM_INPUT :
+        HDFS_STREAM_OUTPUT);
     file->flags = 0;
 
     if ((flags & O_WRONLY) == 0) {
@@ -998,31 +1003,33 @@ int hdfsCloseFile(hdfsFS fs, hdfsFile fi
     // JAVA EQUIVALENT:
     //  file.close 
 
+    //The interface whose 'close' method to be called
+    const char *interface;
+    const char *interfaceShortName;
+
+    //Caught exception
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
-
     if (env == NULL) {
         errno = EINTERNAL;
         return -1;
     }
 
-    //Caught exception
-    jthrowable jthr;
-
     //Sanity check
-    if (!file || file->type == UNINITIALIZED) {
+    if (!file || file->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
 
-    //The interface whose 'close' method to be called
-    const char* interface = (file->type == INPUT) ? 
+    interface = (file->type == HDFS_STREAM_INPUT) ?
         HADOOP_ISTRM : HADOOP_OSTRM;
   
     jthr = invokeMethod(env, NULL, INSTANCE, file->file, interface,
                      "close", "()V");
     if (jthr) {
-        const char *interfaceShortName = (file->type == INPUT) ? 
+        interfaceShortName = (file->type == HDFS_STREAM_INPUT) ? 
             "FSDataInputStream" : "FSDataOutputStream";
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
                 "%s#close", interfaceShortName);
@@ -1044,15 +1051,15 @@ int hdfsCloseFile(hdfsFS fs, hdfsFile fi
 int hdfsExists(hdfsFS fs, const char *path)
 {
     JNIEnv *env = getJNIEnv();
-    if (env == NULL) {
-        errno = EINTERNAL;
-        return -1;
-    }
-
     jobject jPath;
     jvalue  jVal;
     jobject jFS = (jobject)fs;
     jthrowable jthr;
+
+    if (env == NULL) {
+        errno = EINTERNAL;
+        return -1;
+    }
     
     if (path == NULL) {
         errno = EINVAL;
@@ -1088,13 +1095,13 @@ static int readPrepare(JNIEnv* env, hdfs
     *jInputStream = (jobject)(f ? f->file : NULL);
 
     //Sanity check
-    if (!f || f->type == UNINITIALIZED) {
+    if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
       errno = EBADF;
       return -1;
     }
 
     //Error checking... make sure that this file is 'readable'
-    if (f->type != INPUT) {
+    if (f->type != HDFS_STREAM_INPUT) {
       fprintf(stderr, "Cannot read from a non-InputStream object!\n");
       errno = EINVAL;
       return -1;
@@ -1105,6 +1112,13 @@ static int readPrepare(JNIEnv* env, hdfs
 
 tSize hdfsRead(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
 {
+    jobject jInputStream;
+    jbyteArray jbRarray;
+    jint noReadBytes = length;
+    jvalue jVal;
+    jthrowable jthr;
+    JNIEnv* env;
+
     if (length == 0) {
         return 0;
     } else if (length < 0) {
@@ -1120,23 +1134,17 @@ tSize hdfsRead(hdfsFS fs, hdfsFile f, vo
     //  fis.read(bR);
 
     //Get the JNIEnv* corresponding to current thread
-    JNIEnv* env = getJNIEnv();
+    env = getJNIEnv();
     if (env == NULL) {
       errno = EINTERNAL;
       return -1;
     }
 
     //Parameters
-    jobject jInputStream;
     if (readPrepare(env, fs, f, &jInputStream) == -1) {
       return -1;
     }
 
-    jbyteArray jbRarray;
-    jint noReadBytes = length;
-    jvalue jVal;
-    jthrowable jthr;
-
     //Read the requisite bytes
     jbRarray = (*env)->NewByteArray(env, length);
     if (!jbRarray) {
@@ -1179,6 +1187,11 @@ tSize readDirect(hdfsFS fs, hdfsFile f, 
     //  ByteBuffer bbuffer = ByteBuffer.allocateDirect(length) // wraps C buffer
     //  fis.read(bbuffer);
 
+    jobject jInputStream;
+    jvalue jVal;
+    jthrowable jthr;
+    jobject bb;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1186,16 +1199,12 @@ tSize readDirect(hdfsFS fs, hdfsFile f, 
       return -1;
     }
 
-    jobject jInputStream;
     if (readPrepare(env, fs, f, &jInputStream) == -1) {
       return -1;
     }
 
-    jvalue jVal;
-    jthrowable jthr;
-
     //Read the requisite bytes
-    jobject bb = (*env)->NewDirectByteBuffer(env, buffer, length);
+    bb = (*env)->NewDirectByteBuffer(env, buffer, length);
     if (bb == NULL) {
         errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
             "readDirect: NewDirectByteBuffer");
@@ -1227,7 +1236,7 @@ tSize hdfsPread(hdfsFS fs, hdfsFile f, t
         errno = EINVAL;
         return -1;
     }
-    if (!f || f->type == UNINITIALIZED) {
+    if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
@@ -1239,7 +1248,7 @@ tSize hdfsPread(hdfsFS fs, hdfsFile f, t
     }
 
     //Error checking... make sure that this file is 'readable'
-    if (f->type != INPUT) {
+    if (f->type != HDFS_STREAM_INPUT) {
         fprintf(stderr, "Cannot read from a non-InputStream object!\n");
         errno = EINVAL;
         return -1;
@@ -1287,6 +1296,10 @@ tSize hdfsWrite(hdfsFS fs, hdfsFile f, c
     // byte b[] = str.getBytes();
     // fso.write(b);
 
+    jobject jOutputStream;
+    jbyteArray jbWarray;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1295,14 +1308,12 @@ tSize hdfsWrite(hdfsFS fs, hdfsFile f, c
     }
 
     //Sanity check
-    if (!f || f->type == UNINITIALIZED) {
+    if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
 
-    jobject jOutputStream = f->file;
-    jbyteArray jbWarray;
-    jthrowable jthr;
+    jOutputStream = f->file;
     
     if (length < 0) {
     	errno = EINVAL;
@@ -1310,7 +1321,7 @@ tSize hdfsWrite(hdfsFS fs, hdfsFile f, c
     }
 
     //Error checking... make sure that this file is 'writable'
-    if (f->type != OUTPUT) {
+    if (f->type != HDFS_STREAM_OUTPUT) {
         fprintf(stderr, "Cannot write into a non-OutputStream object!\n");
         errno = EINVAL;
         return -1;
@@ -1355,6 +1366,9 @@ int hdfsSeek(hdfsFS fs, hdfsFile f, tOff
     // JAVA EQUIVALENT
     //  fis.seek(pos);
 
+    jobject jInputStream;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1363,13 +1377,13 @@ int hdfsSeek(hdfsFS fs, hdfsFile f, tOff
     }
 
     //Sanity check
-    if (!f || f->type != INPUT) {
+    if (!f || f->type != HDFS_STREAM_INPUT) {
         errno = EBADF;
         return -1;
     }
 
-    jobject jInputStream = f->file;
-    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jInputStream,
+    jInputStream = f->file;
+    jthr = invokeMethod(env, NULL, INSTANCE, jInputStream,
             HADOOP_ISTRM, "seek", "(J)V", desiredPos);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1387,6 +1401,11 @@ tOffset hdfsTell(hdfsFS fs, hdfsFile f)
     // JAVA EQUIVALENT
     //  pos = f.getPos();
 
+    jobject jStream;
+    const char *interface;
+    jvalue jVal;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1395,22 +1414,21 @@ tOffset hdfsTell(hdfsFS fs, hdfsFile f)
     }
 
     //Sanity check
-    if (!f || f->type == UNINITIALIZED) {
+    if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
         errno = EBADF;
         return -1;
     }
 
     //Parameters
-    jobject jStream = f->file;
-    const char* interface = (f->type == INPUT) ?
+    jStream = f->file;
+    interface = (f->type == HDFS_STREAM_INPUT) ?
         HADOOP_ISTRM : HADOOP_OSTRM;
-    jvalue jVal;
-    jthrowable jthr = invokeMethod(env, &jVal, INSTANCE, jStream,
+    jthr = invokeMethod(env, &jVal, INSTANCE, jStream,
                      interface, "getPos", "()J");
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "hdfsTell: %s#getPos",
-            ((f->type == INPUT) ? "FSDataInputStream" :
+            ((f->type == HDFS_STREAM_INPUT) ? "FSDataInputStream" :
                                  "FSDataOutputStream"));
         return -1;
     }
@@ -1422,6 +1440,8 @@ int hdfsFlush(hdfsFS fs, hdfsFile f) 
     // JAVA EQUIVALENT
     //  fos.flush();
 
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1430,11 +1450,11 @@ int hdfsFlush(hdfsFS fs, hdfsFile f) 
     }
 
     //Sanity check
-    if (!f || f->type != OUTPUT) {
+    if (!f || f->type != HDFS_STREAM_OUTPUT) {
         errno = EBADF;
         return -1;
     }
-    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, f->file, 
+    jthr = invokeMethod(env, NULL, INSTANCE, f->file,
                      HADOOP_OSTRM, "flush", "()V");
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1446,6 +1466,9 @@ int hdfsFlush(hdfsFS fs, hdfsFile f) 
 
 int hdfsHFlush(hdfsFS fs, hdfsFile f)
 {
+    jobject jOutputStream;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1454,13 +1477,13 @@ int hdfsHFlush(hdfsFS fs, hdfsFile f)
     }
 
     //Sanity check
-    if (!f || f->type != OUTPUT) {
+    if (!f || f->type != HDFS_STREAM_OUTPUT) {
         errno = EBADF;
         return -1;
     }
 
-    jobject jOutputStream = f->file;
-    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
+    jOutputStream = f->file;
+    jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
                      HADOOP_OSTRM, "hflush", "()V");
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1472,6 +1495,9 @@ int hdfsHFlush(hdfsFS fs, hdfsFile f)
 
 int hdfsHSync(hdfsFS fs, hdfsFile f)
 {
+    jobject jOutputStream;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1480,13 +1506,13 @@ int hdfsHSync(hdfsFS fs, hdfsFile f)
     }
 
     //Sanity check
-    if (!f || f->type != OUTPUT) {
+    if (!f || f->type != HDFS_STREAM_OUTPUT) {
         errno = EBADF;
         return -1;
     }
 
-    jobject jOutputStream = f->file;
-    jthrowable jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
+    jOutputStream = f->file;
+    jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
                      HADOOP_OSTRM, "hsync", "()V");
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1501,6 +1527,10 @@ int hdfsAvailable(hdfsFS fs, hdfsFile f)
     // JAVA EQUIVALENT
     //  fis.available();
 
+    jobject jInputStream;
+    jvalue jVal;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1509,15 +1539,14 @@ int hdfsAvailable(hdfsFS fs, hdfsFile f)
     }
 
     //Sanity check
-    if (!f || f->type != INPUT) {
+    if (!f || f->type != HDFS_STREAM_INPUT) {
         errno = EBADF;
         return -1;
     }
 
     //Parameters
-    jobject jInputStream = f->file;
-    jvalue jVal;
-    jthrowable jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
+    jInputStream = f->file;
+    jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
                      HADOOP_ISTRM, "available", "()I");
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1527,20 +1556,13 @@ int hdfsAvailable(hdfsFS fs, hdfsFile f)
     return jVal.i;
 }
 
-static int hdfsCopyImpl(hdfsFS srcFS, const char* src, hdfsFS dstFS,
-        const char* dst, jboolean deleteSource)
+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;
-    }
-
     //Parameters
     jobject jSrcFS = (jobject)srcFS;
     jobject jDstFS = (jobject)dstFS;
@@ -1549,6 +1571,13 @@ static int hdfsCopyImpl(hdfsFS srcFS, co
     jvalue jVal;
     int ret;
 
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+    if (env == NULL) {
+      errno = EINTERNAL;
+      return -1;
+    }
+
     jthr = constructNewObjectOfPath(env, src, &jSrcPath);
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1603,22 +1632,28 @@ done:
     return 0;
 }
 
-int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
+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)
+int hdfsMove(hdfsFS srcFS, const char *src, hdfsFS dstFS, const char *dst)
 {
     return hdfsCopyImpl(srcFS, src, dstFS, dst, 1);
 }
 
-int hdfsDelete(hdfsFS fs, const char* path, int recursive)
+int hdfsDelete(hdfsFS fs, const char *path, int recursive)
 {
     // JAVA EQUIVALENT:
     //  Path p = new Path(path);
     //  bool retval = fs.delete(p, recursive);
 
+    jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jobject jPath;
+    jvalue jVal;
+    jboolean jRecursive;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1626,18 +1661,13 @@ int hdfsDelete(hdfsFS fs, const char* pa
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-    jthrowable jthr;
-    jobject jPath;
-    jvalue jVal;
-
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "hdfsDelete(path=%s): constructNewObjectOfPath", path);
         return -1;
     }
-    jboolean jRecursive = recursive ? JNI_TRUE : JNI_FALSE;
+    jRecursive = recursive ? JNI_TRUE : JNI_FALSE;
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "delete", "(Lorg/apache/hadoop/fs/Path;Z)Z",
                      jPath, jRecursive);
@@ -1657,13 +1687,19 @@ int hdfsDelete(hdfsFS fs, const char* pa
 
 
 
-int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath)
+int hdfsRename(hdfsFS fs, const char *oldPath, const char *newPath)
 {
     // JAVA EQUIVALENT:
     //  Path old = new Path(oldPath);
     //  Path new = new Path(newPath);
     //  fs.rename(old, new);
 
+    jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jobject jOldPath = NULL, jNewPath = NULL;
+    int ret = -1;
+    jvalue jVal;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1671,12 +1707,6 @@ int hdfsRename(hdfsFS fs, const char* ol
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-    jthrowable jthr;
-    jobject jOldPath = NULL, jNewPath = NULL;
-    int ret = -1;
-    jvalue jVal;
-
     jthr = constructNewObjectOfPath(env, oldPath, &jOldPath );
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1721,13 +1751,6 @@ char* hdfsGetWorkingDirectory(hdfsFS fs,
     //  Path p = fs.getWorkingDirectory(); 
     //  return p.toString()
 
-    //Get the JNIEnv* corresponding to current thread
-    JNIEnv* env = getJNIEnv();
-    if (env == NULL) {
-      errno = EINTERNAL;
-      return NULL;
-    }
-
     jobject jPath = NULL;
     jstring jPathString = NULL;
     jobject jFS = (jobject)fs;
@@ -1736,6 +1759,13 @@ char* hdfsGetWorkingDirectory(hdfsFS fs,
     int ret;
     const char *jPathChars = NULL;
 
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+    if (env == NULL) {
+      errno = EINTERNAL;
+      return NULL;
+    }
+
     //FileSystem#getWorkingDirectory()
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
                      HADOOP_FS, "getWorkingDirectory",
@@ -1794,11 +1824,15 @@ done:
 
 
 
-int hdfsSetWorkingDirectory(hdfsFS fs, const char* path)
+int hdfsSetWorkingDirectory(hdfsFS fs, const char *path)
 {
     // JAVA EQUIVALENT:
     //  fs.setWorkingDirectory(Path(path)); 
 
+    jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jobject jPath;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1806,10 +1840,6 @@ int hdfsSetWorkingDirectory(hdfsFS fs, c
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-    jthrowable jthr;
-    jobject jPath;
-
     //Create an object of org.apache.hadoop.fs.Path
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
@@ -1835,11 +1865,16 @@ int hdfsSetWorkingDirectory(hdfsFS fs, c
 
 
 
-int hdfsCreateDirectory(hdfsFS fs, const char* path)
+int hdfsCreateDirectory(hdfsFS fs, const char *path)
 {
     // JAVA EQUIVALENT:
     //  fs.mkdirs(new Path(path));
 
+    jobject jFS = (jobject)fs;
+    jobject jPath;
+    jthrowable jthr;
+    jvalue jVal;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1847,10 +1882,6 @@ int hdfsCreateDirectory(hdfsFS fs, const
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-    jobject jPath;
-    jthrowable jthr;
-
     //Create an object of org.apache.hadoop.fs.Path
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
@@ -1860,7 +1891,6 @@ int hdfsCreateDirectory(hdfsFS fs, const
     }
 
     //Create the directory
-    jvalue jVal;
     jVal.z = 0;
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "mkdirs", "(Lorg/apache/hadoop/fs/Path;)Z",
@@ -1886,11 +1916,16 @@ int hdfsCreateDirectory(hdfsFS fs, const
 }
 
 
-int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication)
+int hdfsSetReplication(hdfsFS fs, const char *path, int16_t replication)
 {
     // JAVA EQUIVALENT:
     //  fs.setReplication(new Path(path), replication);
 
+    jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jobject jPath;
+    jvalue jVal;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1898,11 +1933,7 @@ int hdfsSetReplication(hdfsFS fs, const 
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-    jthrowable jthr;
-
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath;
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -1911,7 +1942,6 @@ int hdfsSetReplication(hdfsFS fs, const 
     }
 
     //Create the directory
-    jvalue jVal;
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "setReplication", "(Lorg/apache/hadoop/fs/Path;S)Z",
                      jPath, replication);
@@ -1932,11 +1962,17 @@ int hdfsSetReplication(hdfsFS fs, const 
     return 0;
 }
 
-int hdfsChown(hdfsFS fs, const char* path, const char *owner, const char *group)
+int hdfsChown(hdfsFS fs, const char *path, const char *owner, const char *group)
 {
     // JAVA EQUIVALENT:
     //  fs.setOwner(path, owner, group)
 
+    jobject jFS = (jobject)fs;
+    jobject jPath = NULL;
+    jstring jOwner = NULL, jGroup = NULL;
+    jthrowable jthr;
+    int ret;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -1948,12 +1984,6 @@ int hdfsChown(hdfsFS fs, const char* pat
       return 0;
     }
 
-    jobject jFS = (jobject)fs;
-    jobject jPath = NULL;
-    jstring jOwner = NULL, jGroup = NULL;
-    jthrowable jthr;
-    int ret;
-
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -2001,12 +2031,17 @@ done:
     return 0;
 }
 
-int hdfsChmod(hdfsFS fs, const char* path, short mode)
+int hdfsChmod(hdfsFS fs, const char *path, short mode)
 {
     int ret;
     // JAVA EQUIVALENT:
     //  fs.setPermission(path, FsPermission)
 
+    jthrowable jthr;
+    jobject jPath = NULL, jPermObj = NULL;
+    jobject jFS = (jobject)fs;
+    jshort jmode = mode;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -2014,12 +2049,7 @@ int hdfsChmod(hdfsFS fs, const char* pat
       return -1;
     }
 
-    jthrowable jthr;
-    jobject jPath = NULL, jPermObj = NULL;
-    jobject jFS = (jobject)fs;
-
     // construct jPerm = FsPermission.createImmutable(short mode);
-    jshort jmode = mode;
     jthr = constructNewObjectOfClass(env, &jPermObj,
                 HADOOP_FSPERM,"(S)V",jmode);
     if (jthr) {
@@ -2061,11 +2091,16 @@ done:
     return 0;
 }
 
-int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime)
+int hdfsUtime(hdfsFS fs, const char *path, tTime mtime, tTime atime)
 {
     // JAVA EQUIVALENT:
     //  fs.setTimes(src, mtime, atime)
+
     jthrowable jthr;
+    jobject jFS = (jobject)fs;
+    jobject jPath;
+    static const tTime NO_CHANGE = -1;
+    jlong jmtime, jatime;
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
@@ -2074,10 +2109,7 @@ int hdfsUtime(hdfsFS fs, const char* pat
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath;
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
@@ -2085,9 +2117,8 @@ int hdfsUtime(hdfsFS fs, const char* pat
         return -1;
     }
 
-    const tTime NO_CHANGE = -1;
-    jlong jmtime = (mtime == NO_CHANGE) ? -1 : (mtime * (jlong)1000);
-    jlong jatime = (atime == NO_CHANGE) ? -1 : (atime * (jlong)1000);
+    jmtime = (mtime == NO_CHANGE) ? -1 : (mtime * (jlong)1000);
+    jatime = (atime == NO_CHANGE) ? -1 : (atime * (jlong)1000);
 
     jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
             "setTimes", JMETHOD3(JPARAM(HADOOP_PATH), "J", "J", JAVA_VOID),
@@ -2397,7 +2428,7 @@ struct hadoopRzBuffer* hadoopReadZero(hd
         errno = EINTERNAL;
         return NULL;
     }
-    if (file->type != INPUT) {
+    if (file->type != HDFS_STREAM_INPUT) {
         fputs("Cannot read from a non-InputStream object!\n", stderr);
         ret = EINVAL;
         goto done;
@@ -2495,10 +2526,12 @@ void hadoopRzBufferFree(hdfsFile file, s
 }
 
 char***
-hdfsGetHosts(hdfsFS fs, const char* path, tOffset start, tOffset length)
+hdfsGetHosts(hdfsFS fs, const char *path, tOffset start, tOffset length)
 {
     // JAVA EQUIVALENT:
     //  fs.getFileBlockLoctions(new Path(path), start, length);
+
+    jobject jFS = (jobject)fs;
     jthrowable jthr;
     jobject jPath = NULL;
     jobject jFileStatus = NULL;
@@ -2508,6 +2541,9 @@ hdfsGetHosts(hdfsFS fs, const char* path
     char*** blockHosts = NULL;
     int i, j, ret;
     jsize jNumFileBlocks = 0;
+    jobject jFileBlock;
+    jsize jNumBlockHosts;
+    const char *hostName;
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
@@ -2516,8 +2552,6 @@ hdfsGetHosts(hdfsFS fs, const char* path
       return NULL;
     }
 
-    jobject jFS = (jobject)fs;
-
     //Create an object of org.apache.hadoop.fs.Path
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
@@ -2567,7 +2601,7 @@ hdfsGetHosts(hdfsFS fs, const char* path
 
     //Now parse each block to get hostnames
     for (i = 0; i < jNumFileBlocks; ++i) {
-        jobject jFileBlock =
+        jFileBlock =
             (*env)->GetObjectArrayElement(env, jBlockLocations, i);
         if (!jFileBlock) {
             ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
@@ -2593,7 +2627,7 @@ hdfsGetHosts(hdfsFS fs, const char* path
             goto done;
         }
         //Figure out no of hosts in jFileBlockHosts, and allocate the memory
-        jsize jNumBlockHosts = (*env)->GetArrayLength(env, jFileBlockHosts);
+        jNumBlockHosts = (*env)->GetArrayLength(env, jFileBlockHosts);
         blockHosts[i] = calloc(jNumBlockHosts + 1, sizeof(char*));
         if (!blockHosts[i]) {
             ret = ENOMEM;
@@ -2601,7 +2635,6 @@ hdfsGetHosts(hdfsFS fs, const char* path
         }
 
         //Now parse each hostname
-        const char *hostName;
         for (j = 0; j < jNumBlockHosts; ++j) {
             jHost = (*env)->GetObjectArrayElement(env, jFileBlockHosts, j);
             if (!jHost) {
@@ -2669,6 +2702,10 @@ tOffset hdfsGetDefaultBlockSize(hdfsFS f
     // JAVA EQUIVALENT:
     //  fs.getDefaultBlockSize();
 
+    jobject jFS = (jobject)fs;
+    jvalue jVal;
+    jthrowable jthr;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -2676,11 +2713,7 @@ tOffset hdfsGetDefaultBlockSize(hdfsFS f
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-
     //FileSystem#getDefaultBlockSize()
-    jvalue jVal;
-    jthrowable jthr;
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "getDefaultBlockSize", "()J");
     if (jthr) {
@@ -2732,6 +2765,11 @@ tOffset hdfsGetCapacity(hdfsFS fs)
     //  FsStatus fss = fs.getStatus();
     //  return Fss.getCapacity();
 
+    jobject jFS = (jobject)fs;
+    jvalue  jVal;
+    jthrowable jthr;
+    jobject fss;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -2739,11 +2777,7 @@ tOffset hdfsGetCapacity(hdfsFS fs)
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-
     //FileSystem#getStatus
-    jvalue  jVal;
-    jthrowable jthr;
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
     if (jthr) {
@@ -2751,7 +2785,7 @@ tOffset hdfsGetCapacity(hdfsFS fs)
             "hdfsGetCapacity: FileSystem#getStatus");
         return -1;
     }
-    jobject fss = (jobject)jVal.l;
+    fss = (jobject)jVal.l;
     jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
                      "getCapacity", "()J");
     destroyLocalReference(env, fss);
@@ -2771,6 +2805,11 @@ tOffset hdfsGetUsed(hdfsFS fs)
     //  FsStatus fss = fs.getStatus();
     //  return Fss.getUsed();
 
+    jobject jFS = (jobject)fs;
+    jvalue  jVal;
+    jthrowable jthr;
+    jobject fss;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -2778,11 +2817,7 @@ tOffset hdfsGetUsed(hdfsFS fs)
       return -1;
     }
 
-    jobject jFS = (jobject)fs;
-
     //FileSystem#getStatus
-    jvalue  jVal;
-    jthrowable jthr;
     jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
                      "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
     if (jthr) {
@@ -2790,7 +2825,7 @@ tOffset hdfsGetUsed(hdfsFS fs)
             "hdfsGetUsed: FileSystem#getStatus");
         return -1;
     }
-    jobject fss = (jobject)jVal.l;
+    fss = (jobject)jVal.l;
     jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
                      "getUsed", "()J");
     destroyLocalReference(env, fss);
@@ -2814,6 +2849,9 @@ getFileInfoFromStat(JNIEnv *env, jobject
     jstring jUserName = NULL;
     jstring jGroupName = NULL;
     jobject jPermission = NULL;
+    const char *cPathName;
+    const char *cUserName;
+    const char *cGroupName;
 
     jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
                      HADOOP_STAT, "isDir", "()Z");
@@ -2869,7 +2907,7 @@ getFileInfoFromStat(JNIEnv *env, jobject
     if (jthr)
         goto done;
     jPathName = jVal.l;
-    const char *cPathName = 
+    cPathName =
         (const char*) ((*env)->GetStringUTFChars(env, jPathName, NULL));
     if (!cPathName) {
         jthr = getPendingExceptionAndClear(env);
@@ -2882,7 +2920,7 @@ getFileInfoFromStat(JNIEnv *env, jobject
     if (jthr)
         goto done;
     jUserName = jVal.l;
-    const char* cUserName = 
+    cUserName =
         (const char*) ((*env)->GetStringUTFChars(env, jUserName, NULL));
     if (!cUserName) {
         jthr = getPendingExceptionAndClear(env);
@@ -2891,7 +2929,6 @@ getFileInfoFromStat(JNIEnv *env, jobject
     fileInfo->mOwner = strdup(cUserName);
     (*env)->ReleaseStringUTFChars(env, jUserName, cUserName);
 
-    const char* cGroupName;
     jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
                     "getGroup", "()Ljava/lang/String;");
     if (jthr)
@@ -2978,13 +3015,15 @@ getFileInfo(JNIEnv *env, jobject jFS, jo
 
 
 
-hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries)
+hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char *path, int *numEntries)
 {
     // JAVA EQUIVALENT:
     //  Path p(path);
     //  Path []pathList = fs.listPaths(p)
     //  foreach path in pathList 
     //    getFileInfo(path)
+
+    jobject jFS = (jobject)fs;
     jthrowable jthr;
     jobject jPath = NULL;
     hdfsFileInfo *pathList = NULL; 
@@ -2992,6 +3031,8 @@ hdfsFileInfo* hdfsListDirectory(hdfsFS f
     jvalue jVal;
     jsize jPathListSize = 0;
     int ret;
+    jsize i;
+    jobject tmpStat;
 
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
@@ -3000,8 +3041,6 @@ hdfsFileInfo* hdfsListDirectory(hdfsFS f
       return NULL;
     }
 
-    jobject jFS = (jobject)fs;
-
     //Create an object of org.apache.hadoop.fs.Path
     jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
@@ -3037,8 +3076,6 @@ hdfsFileInfo* hdfsListDirectory(hdfsFS f
     }
 
     //Save path information in pathList
-    jsize i;
-    jobject tmpStat;
     for (i=0; i < jPathListSize; ++i) {
         tmpStat = (*env)->GetObjectArrayElement(env, jPathList, i);
         if (!tmpStat) {
@@ -3073,7 +3110,7 @@ done:
 
 
 
-hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path)
+hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char *path)
 {
     // JAVA EQUIVALENT:
     //  File f(path);
@@ -3082,6 +3119,11 @@ hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs,
     //  fs.getLength(f)
     //  f.getPath()
 
+    jobject jFS = (jobject)fs;
+    jobject jPath;
+    jthrowable jthr;
+    hdfsFileInfo *fileInfo;
+
     //Get the JNIEnv* corresponding to current thread
     JNIEnv* env = getJNIEnv();
     if (env == NULL) {
@@ -3089,17 +3131,13 @@ hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs,
       return NULL;
     }
 
-    jobject jFS = (jobject)fs;
-
     //Create an object of org.apache.hadoop.fs.Path
-    jobject jPath;
-    jthrowable jthr = constructNewObjectOfPath(env, path, &jPath);
+    jthr = constructNewObjectOfPath(env, path, &jPath);
     if (jthr) {
         errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "hdfsGetPathInfo(%s): constructNewObjectOfPath", path);
         return NULL;
     }
-    hdfsFileInfo *fileInfo;
     jthr = getFileInfo(env, jFS, jPath, &fileInfo);
     destroyLocalReference(env, jPath);
     if (jthr) {

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c Tue Aug 12 17:02:07 2014
@@ -19,20 +19,18 @@
 #include "config.h"
 #include "exception.h"
 #include "jni_helper.h"
+#include "platform.h"
+#include "common/htable.h"
+#include "os/mutexes.h"
+#include "os/thread_local_storage.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)
-
+static struct htable *gClassRefHTable = NULL;
 
 /** The Native return types that methods could return */
-#define VOID          'V'
+#define JVOID         'V'
 #define JOBJECT       'L'
 #define JARRAYOBJECT  '['
 #define JBOOLEAN      'Z'
@@ -51,40 +49,10 @@ static volatile int hashTableInited = 0;
  */
 #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
+ * Length of buffer for retrieving created JVMs.  (We only ever create one.)
  */
-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);
-}
+#define VM_BUF_LENGTH 1
 
 void destroyLocalReference(JNIEnv *env, jobject jObject)
 {
@@ -138,67 +106,6 @@ jthrowable newCStr(JNIEnv *env, jstring 
     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));
-                UNLOCK_HASH_TABLE();
-                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, ...)
@@ -235,7 +142,7 @@ jthrowable invokeMethod(JNIEnv *env, jva
         }
         retval->l = jobj;
     }
-    else if (returnType == VOID) {
+    else if (returnType == JVOID) {
         if (methType == STATIC) {
             (*env)->CallStaticVoidMethodV(env, cls, mid, args);
         }
@@ -325,11 +232,11 @@ jthrowable methodIdFromClass(const char 
 {
     jclass cls;
     jthrowable jthr;
+    jmethodID mid = 0;
 
     jthr = globalClassReference(className, env, &cls);
     if (jthr)
         return jthr;
-    jmethodID mid = 0;
     jthr = validateMethodType(env, methType);
     if (jthr)
         return jthr;
@@ -350,25 +257,50 @@ jthrowable methodIdFromClass(const char 
 
 jthrowable globalClassReference(const char *className, JNIEnv *env, jclass *out)
 {
-    jclass clsLocalRef;
-    jclass cls = searchEntryFromTable(className);
-    if (cls) {
-        *out = cls;
-        return NULL;
+    jthrowable jthr = NULL;
+    jclass local_clazz = NULL;
+    jclass clazz = NULL;
+    int ret;
+
+    mutexLock(&hdfsHashMutex);
+    if (!gClassRefHTable) {
+        gClassRefHTable = htable_alloc(MAX_HASH_TABLE_ELEM, ht_hash_string,
+            ht_compare_string);
+        if (!gClassRefHTable) {
+            jthr = newRuntimeError(env, "htable_alloc failed\n");
+            goto done;
+        }
     }
-    clsLocalRef = (*env)->FindClass(env,className);
-    if (clsLocalRef == NULL) {
-        return getPendingExceptionAndClear(env);
+    clazz = htable_get(gClassRefHTable, className);
+    if (clazz) {
+        *out = clazz;
+        goto done;
     }
-    cls = (*env)->NewGlobalRef(env, clsLocalRef);
-    if (cls == NULL) {
-        (*env)->DeleteLocalRef(env, clsLocalRef);
-        return getPendingExceptionAndClear(env);
+    local_clazz = (*env)->FindClass(env,className);
+    if (!local_clazz) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
     }
-    (*env)->DeleteLocalRef(env, clsLocalRef);
-    insertEntryIntoTable(className, cls);
-    *out = cls;
-    return NULL;
+    clazz = (*env)->NewGlobalRef(env, local_clazz);
+    if (!clazz) {
+        jthr = getPendingExceptionAndClear(env);
+        goto done;
+    }
+    ret = htable_put(gClassRefHTable, (void*)className, clazz);
+    if (ret) {
+        jthr = newRuntimeError(env, "htable_put failed with error "
+                               "code %d\n", ret);
+        goto done;
+    }
+    *out = clazz;
+    jthr = NULL;
+done:
+    mutexUnlock(&hdfsHashMutex);
+    (*env)->DeleteLocalRef(env, local_clazz);
+    if (jthr && clazz) {
+        (*env)->DeleteGlobalRef(env, clazz);
+    }
+    return jthr;
 }
 
 jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name)
@@ -436,14 +368,24 @@ done:
  */
 static JNIEnv* getGlobalJNIEnv(void)
 {
-    const jsize vmBufLength = 1;
-    JavaVM* vmBuf[vmBufLength]; 
+    JavaVM* vmBuf[VM_BUF_LENGTH]; 
     JNIEnv *env;
     jint rv = 0; 
     jint noVMs = 0;
     jthrowable jthr;
+    char *hadoopClassPath;
+    const char *hadoopClassPathVMArg = "-Djava.class.path=";
+    size_t optHadoopClassPathLen;
+    char *optHadoopClassPath;
+    int noArgs = 1;
+    char *hadoopJvmArgs;
+    char jvmArgDelims[] = " ";
+    char *str, *token, *savePtr;
+    JavaVMInitArgs vm_args;
+    JavaVM *vm;
+    JavaVMOption *options;
 
-    rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), vmBufLength, &noVMs);
+    rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), VM_BUF_LENGTH, &noVMs);
     if (rv != 0) {
         fprintf(stderr, "JNI_GetCreatedJavaVMs failed with error: %d\n", rv);
         return NULL;
@@ -451,23 +393,19 @@ static JNIEnv* getGlobalJNIEnv(void)
 
     if (noVMs == 0) {
         //Get the environment variables for initializing the JVM
-        char *hadoopClassPath = getenv("CLASSPATH");
+        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) + 
+        optHadoopClassPathLen = strlen(hadoopClassPath) + 
           strlen(hadoopClassPathVMArg) + 1;
-        char *optHadoopClassPath = malloc(sizeof(char)*optHadoopClassPathLen);
+        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;
+        hadoopJvmArgs = getenv("LIBHDFS_OPTS");
         if (hadoopJvmArgs != NULL)  {
           hadoopJvmArgs = strdup(hadoopJvmArgs);
           for (noArgs = 1, str = hadoopJvmArgs; ; noArgs++, str = NULL) {
@@ -480,7 +418,12 @@ static JNIEnv* getGlobalJNIEnv(void)
         }
 
         // Now that we know the # args, populate the options array
-        JavaVMOption options[noArgs];
+        options = calloc(noArgs, sizeof(JavaVMOption));
+        if (!options) {
+          fputs("Call to calloc failed\n", stderr);
+          free(optHadoopClassPath);
+          return NULL;
+        }
         options[0].optionString = optHadoopClassPath;
         hadoopJvmArgs = getenv("LIBHDFS_OPTS");
 	if (hadoopJvmArgs != NULL)  {
@@ -495,8 +438,6 @@ static JNIEnv* getGlobalJNIEnv(void)
         }
 
         //Create the VM
-        JavaVMInitArgs vm_args;
-        JavaVM *vm;
         vm_args.version = JNI_VERSION_1_2;
         vm_args.options = options;
         vm_args.nOptions = noArgs; 
@@ -508,6 +449,7 @@ static JNIEnv* getGlobalJNIEnv(void)
           free(hadoopJvmArgs);
         }
         free(optHadoopClassPath);
+        free(options);
 
         if (rv != 0) {
             fprintf(stderr, "Call to JNI_CreateJavaVM failed "
@@ -523,7 +465,7 @@ static JNIEnv* getGlobalJNIEnv(void)
     }
     else {
         //Attach this thread to the VM
-        JavaVM* vm = vmBuf[0];
+        vm = vmBuf[0];
         rv = (*vm)->AttachCurrentThread(vm, (void*)&env, 0);
         if (rv != 0) {
             fprintf(stderr, "Call to AttachCurrentThread "
@@ -557,54 +499,27 @@ static JNIEnv* getGlobalJNIEnv(void)
 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;
+    THREAD_LOCAL_STORAGE_GET_QUICK();
+    mutexLock(&jvmMutex);
+    if (threadLocalStorageGet(&env)) {
+      mutexUnlock(&jvmMutex);
+      return NULL;
+    }
+    if (env) {
+      mutexUnlock(&jvmMutex);
+      return env;
     }
 
     env = getGlobalJNIEnv();
-    pthread_mutex_unlock(&jvmMutex);
+    mutexUnlock(&jvmMutex);
     if (!env) {
-        fprintf(stderr, "getJNIEnv: getGlobalJNIEnv failed\n");
-        return NULL;
+      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;
+    if (threadLocalStorageSet(env)) {
+      return NULL;
     }
-#ifdef HAVE_BETTER_TLS
-    quickTls = tls;
-#endif
+    THREAD_LOCAL_STORAGE_SET_QUICK(env);
     return env;
 }
 

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.h Tue Aug 12 17:02:07 2014
@@ -24,8 +24,6 @@
 
 #include <stdlib.h>
 #include <stdarg.h>
-#include <search.h>
-#include <pthread.h>
 #include <errno.h>
 
 #define PATH_SEPARATOR ':'

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c?rev=1617532&r1=1617531&r2=1617532&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c Tue Aug 12 17:02:07 2014
@@ -21,6 +21,7 @@
 #include "hdfs_test.h"
 #include "jni_helper.h"
 #include "native_mini_dfs.h"
+#include "platform.h"
 
 #include <errno.h>
 #include <jni.h>
@@ -347,10 +348,11 @@ error_dlr_nn:
 int nmdConfigureHdfsBuilder(struct NativeMiniDfsCluster *cl,
                             struct hdfsBuilder *bld)
 {
-    int port, ret;
+    int ret;
+    tPort port;
 
     hdfsBuilderSetNameNode(bld, "localhost");
-    port = nmdGetNameNodePort(cl);
+    port = (tPort)nmdGetNameNodePort(cl);
     if (port < 0) {
       fprintf(stderr, "nmdGetNameNodePort failed with error %d\n", -port);
       return EIO;